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