diff --git a/.github/actions/setup-iceberg-rust-builder/action.yaml b/.github/actions/setup-iceberg-rust-builder/action.yaml new file mode 100644 index 0000000000..23cb9172df --- /dev/null +++ b/.github/actions/setup-iceberg-rust-builder/action.yaml @@ -0,0 +1,39 @@ +# 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. + +name: Setup Iceberg Builder +description: 'Setup Apache Iceberg to run Spark SQL tests' +inputs: + iceberg-version: + description: 'The Apache Iceberg version (e.g., 1.8.1) to build' + required: true +runs: + using: "composite" + steps: + - name: Clone Iceberg repo + uses: actions/checkout@v4 + with: + repository: apache/iceberg + path: apache-iceberg + ref: apache-iceberg-${{inputs.iceberg-version}} + fetch-depth: 1 + + - name: Setup Iceberg for Comet + shell: bash + run: | + cd apache-iceberg + git apply ../dev/diffs/iceberg-rust/${{inputs.iceberg-version}}.diff diff --git a/.github/workflows/iceberg_spark_test.yml b/.github/workflows/iceberg_spark_test.yml index 7eb0a6c2bb..19fa4db5d3 100644 --- a/.github/workflows/iceberg_spark_test.yml +++ b/.github/workflows/iceberg_spark_test.yml @@ -156,3 +156,120 @@ jobs: ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ./gradlew -DsparkVersions=${{ matrix.spark-version.short }} -DscalaVersion=${{ matrix.scala-version }} -DflinkVersions= -DkafkaVersions= \ :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark-version.short }}_${{ matrix.scala-version }}:integrationTest \ -Pquick=true -x javadoc + + iceberg-spark-rust: + if: contains(github.event.pull_request.title, '[iceberg]') + strategy: + matrix: + os: [ubuntu-24.04] + java-version: [11, 17] + iceberg-version: [{short: '1.8', full: '1.8.1'}, {short: '1.9', full: '1.9.1'}, {short: '1.10', full: '1.10.0'}] + spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.7'}] + scala-version: ['2.13'] + fail-fast: false + name: iceberg-spark-rust/${{ matrix.os }}/iceberg-${{ matrix.iceberg-version.full }}/spark-${{ matrix.spark-version.full }}/scala-${{ matrix.scala-version }}/java-${{ matrix.java-version }} + runs-on: ${{ matrix.os }} + container: + image: amd64/rust + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v5 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java-version }} + - name: Build Comet + shell: bash + run: | + PROFILES="-Pspark-${{matrix.spark-version.short}} -Pscala-${{matrix.scala-version}}" make release + - name: Setup Iceberg + uses: ./.github/actions/setup-iceberg-rust-builder + with: + iceberg-version: ${{ matrix.iceberg-version.full }} + - name: Run Iceberg Spark tests (Rust) + run: | + cd apache-iceberg + rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups + ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ./gradlew -DsparkVersions=${{ matrix.spark-version.short }} -DscalaVersion=${{ matrix.scala-version }} -DflinkVersions= -DkafkaVersions= \ + :iceberg-spark:iceberg-spark-${{ matrix.spark-version.short }}_${{ matrix.scala-version }}:test \ + -Pquick=true -x javadoc + + iceberg-spark-extensions-rust: + if: contains(github.event.pull_request.title, '[iceberg]') + strategy: + matrix: + os: [ubuntu-24.04] + java-version: [11, 17] + iceberg-version: [{short: '1.8', full: '1.8.1'}, {short: '1.9', full: '1.9.1'}, {short: '1.10', full: '1.10.0'}] + spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.7'}] + scala-version: ['2.13'] + fail-fast: false + name: iceberg-spark-extensions-rust/${{ matrix.os }}/iceberg-${{ matrix.iceberg-version.full }}/spark-${{ matrix.spark-version.full }}/scala-${{ matrix.scala-version }}/java-${{ matrix.java-version }} + runs-on: ${{ matrix.os }} + container: + image: amd64/rust + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v5 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java-version }} + - name: Build Comet + shell: bash + run: | + PROFILES="-Pspark-${{matrix.spark-version.short}} -Pscala-${{matrix.scala-version}}" make release + - name: Setup Iceberg + uses: ./.github/actions/setup-iceberg-rust-builder + with: + iceberg-version: ${{ matrix.iceberg-version.full }} + - name: Run Iceberg Spark extensions tests (Rust) + run: | + cd apache-iceberg + rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups + ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ./gradlew -DsparkVersions=${{ matrix.spark-version.short }} -DscalaVersion=${{ matrix.scala-version }} -DflinkVersions= -DkafkaVersions= \ + :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark-version.short }}_${{ matrix.scala-version }}:test \ + -Pquick=true -x javadoc + + iceberg-spark-runtime-rust: + if: contains(github.event.pull_request.title, '[iceberg]') + strategy: + matrix: + os: [ubuntu-24.04] + java-version: [11, 17] + iceberg-version: [{short: '1.8', full: '1.8.1'}, {short: '1.9', full: '1.9.1'}, {short: '1.10', full: '1.10.0'}] + spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.7'}] + scala-version: ['2.13'] + fail-fast: false + name: iceberg-spark-runtime-rust/${{ matrix.os }}/iceberg-${{ matrix.iceberg-version.full }}/spark-${{ matrix.spark-version.full }}/scala-${{ matrix.scala-version }}/java-${{ matrix.java-version }} + runs-on: ${{ matrix.os }} + container: + image: amd64/rust + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v5 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java-version }} + - name: Build Comet + shell: bash + run: | + PROFILES="-Pspark-${{matrix.spark-version.short}} -Pscala-${{matrix.scala-version}}" make release + - name: Setup Iceberg + uses: ./.github/actions/setup-iceberg-rust-builder + with: + iceberg-version: ${{ matrix.iceberg-version.full }} + - name: Run Iceberg Spark runtime tests (Rust) + run: | + cd apache-iceberg + rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups + ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ./gradlew -DsparkVersions=${{ matrix.spark-version.short }} -DscalaVersion=${{ matrix.scala-version }} -DflinkVersions= -DkafkaVersions= \ + :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark-version.short }}_${{ matrix.scala-version }}:integrationTest \ + -Pquick=true -x javadoc diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 74ac9c6956..8570c7e7ae 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -103,6 +103,7 @@ jobs: value: | org.apache.comet.CometFuzzTestSuite org.apache.comet.CometFuzzAggregateSuite + org.apache.comet.CometFuzzIcebergSuite org.apache.comet.CometFuzzMathSuite org.apache.comet.DataGeneratorSuite - name: "shuffle" @@ -124,6 +125,7 @@ jobs: org.apache.spark.sql.comet.ParquetDatetimeRebaseV2Suite org.apache.spark.sql.comet.ParquetEncryptionITCase org.apache.comet.exec.CometNativeReaderSuite + org.apache.comet.CometIcebergNativeSuite - name: "exec" value: | org.apache.comet.exec.CometAggregateSuite diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 0478a35eb1..5eb6be4ee1 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -68,6 +68,7 @@ jobs: value: | org.apache.comet.CometFuzzTestSuite org.apache.comet.CometFuzzAggregateSuite + org.apache.comet.CometFuzzIcebergSuite org.apache.comet.CometFuzzMathSuite org.apache.comet.DataGeneratorSuite - name: "shuffle" @@ -89,6 +90,7 @@ jobs: org.apache.spark.sql.comet.ParquetDatetimeRebaseV2Suite org.apache.spark.sql.comet.ParquetEncryptionITCase org.apache.comet.exec.CometNativeReaderSuite + org.apache.comet.CometIcebergNativeSuite - name: "exec" value: | org.apache.comet.exec.CometAggregateSuite diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index acecced284..0484b64f16 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -122,6 +122,16 @@ object CometConf extends ShimCometConf { Set(SCAN_NATIVE_COMET, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) .createWithEnvVarOrDefault("COMET_PARQUET_SCAN_IMPL", SCAN_AUTO) + val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.scan.icebergNative.enabled") + .category(CATEGORY_SCAN) + .doc( + "Whether to enable native Iceberg table scan using iceberg-rust. When enabled, " + + "Iceberg tables are read directly through native execution, bypassing Spark's " + + "DataSource V2 API for better performance.") + .booleanConf + .createWithDefault(false) + val COMET_RESPECT_PARQUET_FILTER_PUSHDOWN: ConfigEntry[Boolean] = conf("spark.comet.parquet.respectFilterPushdown") .category(CATEGORY_PARQUET) diff --git a/common/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala b/common/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala index 885b4686e7..a178b94b42 100644 --- a/common/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala +++ b/common/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala @@ -56,7 +56,7 @@ object NativeConfig { * consistent and standardized cloud storage support across all providers. */ def extractObjectStoreOptions(hadoopConf: Configuration, uri: URI): Map[String, String] = { - val scheme = uri.getScheme.toLowerCase(Locale.ROOT) + val scheme = Option(uri.getScheme).map(_.toLowerCase(Locale.ROOT)).getOrElse("file") import scala.jdk.CollectionConverters._ val options = scala.collection.mutable.Map[String, String]() diff --git a/dev/ci/check-suites.py b/dev/ci/check-suites.py index 8d9acb2d59..279c6a89c9 100644 --- a/dev/ci/check-suites.py +++ b/dev/ci/check-suites.py @@ -34,6 +34,7 @@ def file_to_class_name(path: Path) -> str | None: ignore_list = [ "org.apache.comet.parquet.ParquetReadSuite", # abstract "org.apache.comet.parquet.ParquetReadFromS3Suite", # manual test suite + "org.apache.comet.IcebergReadFromS3Suite", # manual test suite "org.apache.spark.sql.comet.CometPlanStabilitySuite", # abstract "org.apache.spark.sql.comet.ParquetDatetimeRebaseSuite", # abstract "org.apache.comet.exec.CometColumnarShuffleSuite" # abstract diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 4deac778b9..ae1feb692a 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index d3544881af1..9c174496a4b 100644 +index d3544881af1..fbe1c4b9a87 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -513,7 +513,7 @@ index a6b295578d6..91acca4306f 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 2796b1cf154..4816349d690 100644 +index 2796b1cf154..52438178a0e 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} @@ -536,7 +536,7 @@ index 2796b1cf154..4816349d690 100644 val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _) => f -+ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _), _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _), _, _) => f } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) @@ -544,7 +544,7 @@ index 2796b1cf154..4816349d690 100644 val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _) => f -+ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _), _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _), _, _) => f } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) diff --git a/dev/diffs/3.5.7.diff b/dev/diffs/3.5.7.diff index 6c54cd264e..433188a449 100644 --- a/dev/diffs/3.5.7.diff +++ b/dev/diffs/3.5.7.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index 68e2c422a24..d971894ffe6 100644 +index a0e25ce4d8d..7db86212507 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -38,7 +38,7 @@ index 68e2c422a24..d971894ffe6 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index f08b33575fc..424e0da32fd 100644 +index e3d324c8edb..22342150522 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -216,7 +216,7 @@ index 0efe0877e9b..423d3b3d76d 100644 -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index 9815cb816c9..95b5f9992b0 100644 +index e5494726695..00937f025c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -239,7 +239,7 @@ index 9815cb816c9..95b5f9992b0 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/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -index 5a8681aed97..da9d25e2eb4 100644 +index 6f3090d8908..c08a60fb0c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Expand @@ -336,7 +336,7 @@ index 7ee18df3756..d09f70e5d99 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -index 47a311c71d5..342e71cfdd4 100644 +index a1d5d579338..c201d39cc78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression @@ -482,7 +482,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..01e5c601763 100644 +index 93275487f29..33b2e7ad3b1 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} @@ -522,7 +522,7 @@ index 93275487f29..01e5c601763 100644 val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f -+ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _, _) => f } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) @@ -530,7 +530,7 @@ index 93275487f29..01e5c601763 100644 val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f -+ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _, _) => f } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) @@ -624,7 +624,7 @@ index 7af826583bd..3c3def1eb67 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 4d256154c85..66a5473852d 100644 +index 44c8cb92fc3..f098beeca26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -822,7 +822,7 @@ index 4d256154c85..66a5473852d 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1583,6 +1612,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1611,6 +1640,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true @@ -832,7 +832,7 @@ index 4d256154c85..66a5473852d 100644 }.size == 1) } } -@@ -1627,14 +1659,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1655,14 +1687,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") { def check(plan: SparkPlan): Unit = { @@ -855,7 +855,7 @@ index 4d256154c85..66a5473852d 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1770,7 +1808,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1798,7 +1836,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -879,7 +879,7 @@ index c26757c9cff..d55775f09d7 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 793a0da6a86..181bfc16e4b 100644 +index 3cf2bfd17ab..49728c35c42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1521,7 +1521,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -2050,10 +2050,10 @@ 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 4f8a9e39716..fb55ac7a955 100644 +index 8ed9ef1630e..eed2a6f5ad5 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 -@@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1345,7 +1345,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index 612a8975be..2cc0034b49 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index 443d46a4302..63ec4784625 100644 +index 22922143fc3..3c1f5d381ee 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -38,7 +38,7 @@ index 443d46a4302..63ec4784625 100644 org.apache.datasketches diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index 56aad01023d..ad9d6a973e4 100644 +index dcf6223a98b..0458a5bb640 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -90,6 +90,10 @@ @@ -332,7 +332,7 @@ index 1f8c5822e7d..b7de4e28813 100644 WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index e0ad3feda3a..8fc1ba8e10d 100644 +index 0f42502f1d9..f616024a9c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -380,7 +380,7 @@ index e0ad3feda3a..8fc1ba8e10d 100644 withTempView("t0", "t1", "t2") { 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 6ce0a657d5b..02e352d13b0 100644 +index 9db406ff12f..abbc91f5c11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.errors.DataTypeErrors.toSQLId @@ -654,7 +654,7 @@ index 9c90e0105a4..fadf2f0f698 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 9c529d14221..069b7c5adeb 100644 +index 9c529d14221..2f1bc3880fd 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GreaterTha @@ -677,7 +677,7 @@ index 9c529d14221..069b7c5adeb 100644 val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f -+ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _, _) => f } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) @@ -685,7 +685,7 @@ index 9c529d14221..069b7c5adeb 100644 val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f -+ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _), _, _) => f } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) @@ -802,7 +802,7 @@ index 53e47f428c3..a55d8f0c161 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 41f2e5c9a40..8f0dd909492 100644 +index aaac0ebc9aa..fbef0774d46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -1000,7 +1000,7 @@ index 41f2e5c9a40..8f0dd909492 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1586,6 +1615,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1614,6 +1643,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true @@ -1010,7 +1010,7 @@ index 41f2e5c9a40..8f0dd909492 100644 }.size == 1) } } -@@ -1630,14 +1662,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1658,14 +1690,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") { def check(plan: SparkPlan): Unit = { @@ -1033,7 +1033,7 @@ index 41f2e5c9a40..8f0dd909492 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1773,7 +1811,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1801,7 +1839,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -1057,7 +1057,7 @@ index ad424b3a7cc..4ece0117a34 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index b3fce19979e..67edf5eb91c 100644 +index f294ff81021..8a3b818ee94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1524,7 +1524,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1461,7 +1461,7 @@ index 2a0ab21ddb0..e8a5a891105 100644 } finally { spark.listenerManager.unregister(listener) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala -index c24f52bd930..ce4d4c4a694 100644 +index c73e8e16fbb..88cd0d47da3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -20,10 +20,11 @@ import java.sql.Timestamp @@ -1538,7 +1538,7 @@ index f62e092138a..c0404bfe85e 100644 before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala -index 46f2502582c..1f4c8f2201b 100644 +index 04d33ecd3d5..450df347297 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, when} @@ -1679,7 +1679,7 @@ index 1400ee25f43..5b016c3f9c5 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala -index a6490243708..c61af08d4cf 100644 +index 47d5ff67b84..8dc8f65d4b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -20,7 +20,7 @@ import scala.collection.mutable @@ -1691,7 +1691,7 @@ index a6490243708..c61af08d4cf 100644 import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, UnsafeRow} -@@ -385,7 +385,7 @@ class QueryExecutionSuite extends SharedSparkSession { +@@ -400,7 +400,7 @@ class QueryExecutionSuite extends SharedSparkSession { } } @@ -2682,10 +2682,10 @@ index 6080a5e8e4b..9aa8f49a62b 100644 case _ => assert(false, "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 22839d3f0d2..7e66d100e90 100644 +index 4474ec1fd42..97910c4fc3a 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 -@@ -1334,7 +1334,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1344,7 +1344,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } diff --git a/dev/diffs/iceberg-rust/1.10.0.diff b/dev/diffs/iceberg-rust/1.10.0.diff new file mode 100644 index 0000000000..07a8df1042 --- /dev/null +++ b/dev/diffs/iceberg-rust/1.10.0.diff @@ -0,0 +1,1784 @@ +diff --git a/build.gradle b/build.gradle +index 6bc052885..db2aca3a5 100644 +--- a/build.gradle ++++ b/build.gradle +@@ -878,6 +878,13 @@ project(':iceberg-parquet') { + implementation project(':iceberg-core') + implementation project(':iceberg-common') + ++ implementation("org.apache.datafusion:comet-spark-spark${sparkVersionsString}_${scalaVersion}:${libs.versions.comet.get()}") { ++ exclude group: 'org.apache.arrow' ++ exclude group: 'org.apache.parquet' ++ exclude group: 'org.apache.spark' ++ exclude group: 'org.apache.iceberg' ++ } ++ + implementation(libs.parquet.avro) { + exclude group: 'org.apache.avro', module: 'avro' + // already shaded by Parquet +diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml +index eeabe54f5..867018058 100644 +--- a/gradle/libs.versions.toml ++++ b/gradle/libs.versions.toml +@@ -37,7 +37,7 @@ awssdk-s3accessgrants = "2.3.0" + bson-ver = "4.11.5" + caffeine = "2.9.3" + calcite = "1.40.0" +-comet = "0.8.1" ++comet = "0.12.0-SNAPSHOT" + datasketches = "6.2.0" + delta-standalone = "3.3.2" + delta-spark = "3.3.2" +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java +new file mode 100644 +index 000000000..ddf6c7de5 +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.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.iceberg.parquet; ++ ++import java.util.Map; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.iceberg.relocated.com.google.common.collect.Maps; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.schema.LogicalTypeAnnotation; ++import org.apache.parquet.schema.PrimitiveType; ++import org.apache.parquet.schema.Type; ++import org.apache.parquet.schema.Types; ++ ++public class CometTypeUtils { ++ ++ private CometTypeUtils() {} ++ ++ public static ParquetColumnSpec descriptorToParquetColumnSpec(ColumnDescriptor descriptor) { ++ ++ String[] path = descriptor.getPath(); ++ PrimitiveType primitiveType = descriptor.getPrimitiveType(); ++ String physicalType = primitiveType.getPrimitiveTypeName().name(); ++ ++ int typeLength = ++ primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY ++ ? primitiveType.getTypeLength() ++ : 0; ++ ++ boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; ++ ++ // ToDo: extract this into a Util method ++ String logicalTypeName = null; ++ Map logicalTypeParams = Maps.newHashMap(); ++ LogicalTypeAnnotation logicalType = primitiveType.getLogicalTypeAnnotation(); ++ ++ if (logicalType != null) { ++ logicalTypeName = logicalType.getClass().getSimpleName(); ++ ++ // Handle specific logical types ++ if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal = ++ (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("precision", String.valueOf(decimal.getPrecision())); ++ logicalTypeParams.put("scale", String.valueOf(decimal.getScale())); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp = ++ (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(timestamp.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", timestamp.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimeLogicalTypeAnnotation time = ++ (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(time.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", time.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.IntLogicalTypeAnnotation intType = ++ (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned())); ++ logicalTypeParams.put("bitWidth", String.valueOf(intType.getBitWidth())); ++ } ++ } ++ ++ return new ParquetColumnSpec( ++ 1, // ToDo: pass in the correct id ++ path, ++ physicalType, ++ typeLength, ++ isRepeated, ++ descriptor.getMaxDefinitionLevel(), ++ descriptor.getMaxRepetitionLevel(), ++ logicalTypeName, ++ logicalTypeParams); ++ } ++ ++ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { ++ PrimitiveType.PrimitiveTypeName primType = ++ PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); ++ ++ Type.Repetition repetition; ++ if (columnSpec.getMaxRepetitionLevel() > 0) { ++ repetition = Type.Repetition.REPEATED; ++ } else if (columnSpec.getMaxDefinitionLevel() > 0) { ++ repetition = Type.Repetition.OPTIONAL; ++ } else { ++ repetition = Type.Repetition.REQUIRED; ++ } ++ ++ String name = columnSpec.getPath()[columnSpec.getPath().length - 1]; ++ // Reconstruct the logical type from parameters ++ LogicalTypeAnnotation logicalType = null; ++ if (columnSpec.getLogicalTypeName() != null) { ++ logicalType = ++ reconstructLogicalType( ++ columnSpec.getLogicalTypeName(), columnSpec.getLogicalTypeParams()); ++ } ++ ++ PrimitiveType primitiveType; ++ if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { ++ primitiveType = ++ org.apache.parquet.schema.Types.primitive(primType, repetition) ++ .length(columnSpec.getTypeLength()) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } else { ++ primitiveType = ++ Types.primitive(primType, repetition) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } ++ ++ return new ColumnDescriptor( ++ columnSpec.getPath(), ++ primitiveType, ++ columnSpec.getMaxRepetitionLevel(), ++ columnSpec.getMaxDefinitionLevel()); ++ } ++ ++ private static LogicalTypeAnnotation reconstructLogicalType( ++ String logicalTypeName, java.util.Map params) { ++ ++ switch (logicalTypeName) { ++ // MAP ++ case "MapLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.mapType(); ++ ++ // LIST ++ case "ListLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.listType(); ++ ++ // STRING ++ case "StringLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.stringType(); ++ ++ // MAP_KEY_VALUE ++ case "MapKeyValueLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(); ++ ++ // ENUM ++ case "EnumLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.enumType(); ++ ++ // DECIMAL ++ case "DecimalLogicalTypeAnnotation": ++ if (!params.containsKey("scale") || !params.containsKey("precision")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for DecimalLogicalTypeAnnotation: " + params); ++ } ++ int scale = Integer.parseInt(params.get("scale")); ++ int precision = Integer.parseInt(params.get("precision")); ++ return LogicalTypeAnnotation.decimalType(scale, precision); ++ ++ // DATE ++ case "DateLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.dateType(); ++ ++ // TIME ++ case "TimeLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimeLogicalTypeAnnotation: " + params); ++ } ++ ++ boolean isUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String timeUnitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit timeUnit; ++ switch (timeUnitStr) { ++ case "MILLIS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown time unit: " + timeUnitStr); ++ } ++ return LogicalTypeAnnotation.timeType(isUTC, timeUnit); ++ ++ // TIMESTAMP ++ case "TimestampLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimestampLogicalTypeAnnotation: " + params); ++ } ++ boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String unitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit unit; ++ switch (unitStr) { ++ case "MILLIS": ++ unit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ unit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ unit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown timestamp unit: " + unitStr); ++ } ++ return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); ++ ++ // INTEGER ++ case "IntLogicalTypeAnnotation": ++ if (!params.containsKey("isSigned") || !params.containsKey("bitWidth")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for IntLogicalTypeAnnotation: " + params); ++ } ++ boolean isSigned = Boolean.parseBoolean(params.get("isSigned")); ++ int bitWidth = Integer.parseInt(params.get("bitWidth")); ++ return LogicalTypeAnnotation.intType(bitWidth, isSigned); ++ ++ // JSON ++ case "JsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.jsonType(); ++ ++ // BSON ++ case "BsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.bsonType(); ++ ++ // UUID ++ case "UUIDLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.uuidType(); ++ ++ // INTERVAL ++ case "IntervalLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(); ++ ++ default: ++ throw new IllegalArgumentException("Unknown logical type: " + logicalTypeName); ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +new file mode 100644 +index 000000000..a3cba4018 +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +@@ -0,0 +1,260 @@ ++/* ++ * 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.iceberg.parquet; ++ ++import java.io.IOException; ++import java.io.UncheckedIOException; ++import java.nio.ByteBuffer; ++import java.util.List; ++import java.util.Map; ++import java.util.NoSuchElementException; ++import java.util.function.Function; ++import org.apache.comet.parquet.FileReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.ReadOptions; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.parquet.WrappedInputFile; ++import org.apache.hadoop.conf.Configuration; ++import org.apache.iceberg.Schema; ++import org.apache.iceberg.exceptions.RuntimeIOException; ++import org.apache.iceberg.expressions.Expression; ++import org.apache.iceberg.expressions.Expressions; ++import org.apache.iceberg.io.CloseableGroup; ++import org.apache.iceberg.io.CloseableIterable; ++import org.apache.iceberg.io.CloseableIterator; ++import org.apache.iceberg.io.InputFile; ++import org.apache.iceberg.mapping.NameMapping; ++import org.apache.iceberg.relocated.com.google.common.collect.Lists; ++import org.apache.iceberg.util.ByteBuffers; ++import org.apache.parquet.ParquetReadOptions; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; ++import org.apache.parquet.hadoop.metadata.ColumnPath; ++import org.apache.parquet.schema.MessageType; ++ ++public class CometVectorizedParquetReader extends CloseableGroup ++ implements CloseableIterable { ++ private final InputFile input; ++ private final ParquetReadOptions options; ++ private final Schema expectedSchema; ++ private final Function> batchReaderFunc; ++ private final Expression filter; ++ private final boolean reuseContainers; ++ private final boolean caseSensitive; ++ private final int batchSize; ++ private final NameMapping nameMapping; ++ private final Map properties; ++ private Long start = null; ++ private Long length = null; ++ private ByteBuffer fileEncryptionKey = null; ++ private ByteBuffer fileAADPrefix = null; ++ ++ public CometVectorizedParquetReader( ++ InputFile input, ++ Schema expectedSchema, ++ ParquetReadOptions options, ++ Function> readerFunc, ++ NameMapping nameMapping, ++ Expression filter, ++ boolean reuseContainers, ++ boolean caseSensitive, ++ int maxRecordsPerBatch, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.input = input; ++ this.expectedSchema = expectedSchema; ++ this.options = options; ++ this.batchReaderFunc = readerFunc; ++ // replace alwaysTrue with null to avoid extra work evaluating a trivial filter ++ this.filter = filter == Expressions.alwaysTrue() ? null : filter; ++ this.reuseContainers = reuseContainers; ++ this.caseSensitive = caseSensitive; ++ this.batchSize = maxRecordsPerBatch; ++ this.nameMapping = nameMapping; ++ this.properties = properties; ++ this.start = start; ++ this.length = length; ++ this.fileEncryptionKey = fileEncryptionKey; ++ this.fileAADPrefix = fileAADPrefix; ++ } ++ ++ private ReadConf conf = null; ++ ++ private ReadConf init() { ++ if (conf == null) { ++ ReadConf readConf = ++ new ReadConf( ++ input, ++ options, ++ expectedSchema, ++ filter, ++ null, ++ batchReaderFunc, ++ nameMapping, ++ reuseContainers, ++ caseSensitive, ++ batchSize); ++ this.conf = readConf.copy(); ++ return readConf; ++ } ++ return conf; ++ } ++ ++ @Override ++ public CloseableIterator iterator() { ++ FileIterator iter = ++ new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); ++ addCloseable(iter); ++ return iter; ++ } ++ ++ private static class FileIterator implements CloseableIterator { ++ // private final ParquetFileReader reader; ++ private final boolean[] shouldSkip; ++ private final VectorizedReader model; ++ private final long totalValues; ++ private final int batchSize; ++ private final List> columnChunkMetadata; ++ private final boolean reuseContainers; ++ private int nextRowGroup = 0; ++ private long nextRowGroupStart = 0; ++ private long valuesRead = 0; ++ private T last = null; ++ private final FileReader cometReader; ++ private ReadConf conf; ++ ++ FileIterator( ++ ReadConf conf, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.shouldSkip = conf.shouldSkip(); ++ this.totalValues = conf.totalValues(); ++ this.reuseContainers = conf.reuseContainers(); ++ this.model = conf.vectorizedModel(); ++ this.batchSize = conf.batchSize(); ++ this.model.setBatchSize(this.batchSize); ++ this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups(); ++ this.cometReader = ++ newCometReader( ++ conf.file(), ++ conf.projection(), ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ this.conf = conf; ++ } ++ ++ private FileReader newCometReader( ++ InputFile file, ++ MessageType projection, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ try { ++ ReadOptions cometOptions = ReadOptions.builder(new Configuration()).build(); ++ ++ FileReader fileReader = ++ new FileReader( ++ new WrappedInputFile(file), ++ cometOptions, ++ properties, ++ start, ++ length, ++ ByteBuffers.toByteArray(fileEncryptionKey), ++ ByteBuffers.toByteArray(fileAADPrefix)); ++ ++ List columnDescriptors = projection.getColumns(); ++ ++ List specs = Lists.newArrayList(); ++ ++ for (ColumnDescriptor descriptor : columnDescriptors) { ++ ParquetColumnSpec spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ specs.add(spec); ++ } ++ ++ fileReader.setRequestedSchemaFromSpecs(specs); ++ return fileReader; ++ } catch (IOException e) { ++ throw new UncheckedIOException("Failed to open Parquet file: " + file.location(), e); ++ } ++ } ++ ++ @Override ++ public boolean hasNext() { ++ return valuesRead < totalValues; ++ } ++ ++ @Override ++ public T next() { ++ if (!hasNext()) { ++ throw new NoSuchElementException(); ++ } ++ if (valuesRead >= nextRowGroupStart) { ++ advance(); ++ } ++ ++ // batchSize is an integer, so casting to integer is safe ++ int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, batchSize); ++ if (reuseContainers) { ++ this.last = model.read(last, numValuesToRead); ++ } else { ++ this.last = model.read(null, numValuesToRead); ++ } ++ valuesRead += numValuesToRead; ++ ++ return last; ++ } ++ ++ private void advance() { ++ while (shouldSkip[nextRowGroup]) { ++ nextRowGroup += 1; ++ cometReader.skipNextRowGroup(); ++ } ++ RowGroupReader pages; ++ try { ++ pages = cometReader.readNextRowGroup(); ++ } catch (IOException e) { ++ throw new RuntimeIOException(e); ++ } ++ ++ model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup)); ++ nextRowGroupStart += pages.getRowCount(); ++ nextRowGroup += 1; ++ } ++ ++ @Override ++ public void close() throws IOException { ++ model.close(); ++ cometReader.close(); ++ if (conf != null && conf.reader() != null) { ++ conf.reader().close(); ++ } ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +index 6f68fbe15..b740543f3 100644 +--- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +@@ -1161,6 +1161,7 @@ public class Parquet { + private NameMapping nameMapping = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; ++ private boolean isComet; + + private ReadBuilder(InputFile file) { + this.file = file; +@@ -1205,6 +1206,11 @@ public class Parquet { + return this; + } + ++ public ReadBuilder enableComet(boolean enableComet) { ++ this.isComet = enableComet; ++ return this; ++ } ++ + /** + * @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead + */ +@@ -1300,7 +1306,7 @@ public class Parquet { + } + + @Override +- @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) ++ @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity", "MethodLength"}) + public CloseableIterable build() { + FileDecryptionProperties fileDecryptionProperties = null; + if (fileEncryptionKey != null) { +@@ -1352,16 +1358,35 @@ public class Parquet { + } + + if (batchedReaderFunc != null) { +- return new VectorizedParquetReader<>( +- file, +- schema, +- options, +- batchedReaderFunc, +- mapping, +- filter, +- reuseContainers, +- caseSensitive, +- maxRecordsPerBatch); ++ if (isComet) { ++ LOG.info("Comet enabled"); ++ return new CometVectorizedParquetReader<>( ++ file, ++ schema, ++ options, ++ batchedReaderFunc, ++ mapping, ++ filter, ++ reuseContainers, ++ caseSensitive, ++ maxRecordsPerBatch, ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ } else { ++ return new VectorizedParquetReader<>( ++ file, ++ schema, ++ options, ++ batchedReaderFunc, ++ mapping, ++ filter, ++ reuseContainers, ++ caseSensitive, ++ maxRecordsPerBatch); ++ } + } else { + Function> readBuilder = + readerFuncWithSchema != null +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +index 1fb2372ba..142e5fbad 100644 +--- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +@@ -157,6 +157,14 @@ class ReadConf { + return newReader; + } + ++ InputFile file() { ++ return file; ++ } ++ ++ MessageType projection() { ++ return projection; ++ } ++ + ParquetValueReader model() { + return model; + } +diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle +index 69700d843..49ea338a4 100644 +--- a/spark/v3.5/build.gradle ++++ b/spark/v3.5/build.gradle +@@ -264,6 +264,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') ++ integrationImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + // runtime dependencies for running Hive Catalog based integration test + integrationRuntimeOnly project(':iceberg-hive-metastore') +diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +index 4c1a50959..44b401644 100644 +--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java ++++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +@@ -59,6 +59,16 @@ public abstract class ExtensionsTestBase extends CatalogTestBase { + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + .config( + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean())) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +index ecf9e6f8a..3475260ca 100644 +--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java ++++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +@@ -56,6 +56,16 @@ public class TestCallStatementParser { + .master("local[2]") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.extra.prop", "value") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + TestCallStatementParser.parser = spark.sessionState().sqlParser(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +index 64edb1002..0fc10120f 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +@@ -179,6 +179,16 @@ public class DeleteOrphanFilesBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", catalogWarehouse()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local"); + spark = builder.getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +index 77b79384a..01a4c82dc 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +@@ -392,6 +392,16 @@ public class IcebergSortCompactionBenchmark { + "spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", getCatalogWarehouse()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]"); + spark = builder.getOrCreate(); + Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java +index c6794e43c..457d2823e 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java +@@ -239,6 +239,16 @@ public class DVReaderBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]") + .getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java +index ac74fb5a1..eab09293d 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java +@@ -223,6 +223,16 @@ public class DVWriterBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]") + .getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java +index 68c537e34..1e9e90d53 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java +@@ -94,7 +94,19 @@ public abstract class IcebergSourceBenchmark { + } + + protected void setupSpark(boolean enableDictionaryEncoding) { +- SparkSession.Builder builder = SparkSession.builder().config("spark.ui.enabled", false); ++ SparkSession.Builder builder = ++ SparkSession.builder() ++ .config("spark.ui.enabled", false) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true"); + if (!enableDictionaryEncoding) { + builder + .config("parquet.dictionary.page.size", "1") +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +index 81b7d83a7..eba1a2a0f 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +@@ -19,18 +19,22 @@ + package org.apache.iceberg.spark.data.vectorized; + + import java.io.IOException; ++import org.apache.comet.CometConf; + import org.apache.comet.CometSchemaImporter; + import org.apache.comet.parquet.AbstractColumnReader; + import org.apache.comet.parquet.ColumnReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.RowGroupReader; + import org.apache.comet.parquet.TypeUtil; + import org.apache.comet.parquet.Utils; + import org.apache.comet.shaded.arrow.memory.RootAllocator; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.parquet.VectorizedReader; + import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + import org.apache.iceberg.spark.SparkSchemaUtil; + import org.apache.iceberg.types.Types; + import org.apache.parquet.column.ColumnDescriptor; +-import org.apache.parquet.column.page.PageReader; ++import org.apache.spark.sql.internal.SQLConf; + import org.apache.spark.sql.types.DataType; + import org.apache.spark.sql.types.Metadata; + import org.apache.spark.sql.types.StructField; +@@ -42,23 +46,28 @@ class CometColumnReader implements VectorizedReader { + + private final ColumnDescriptor descriptor; + private final DataType sparkType; ++ private final int fieldId; + + // The delegated ColumnReader from Comet side + private AbstractColumnReader delegate; + private boolean initialized = false; + private int batchSize = DEFAULT_BATCH_SIZE; + private CometSchemaImporter importer; ++ private ParquetColumnSpec spec; + +- CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { ++ CometColumnReader(DataType sparkType, ColumnDescriptor descriptor, int fieldId) { + this.sparkType = sparkType; + this.descriptor = descriptor; ++ this.fieldId = fieldId; + } + + CometColumnReader(Types.NestedField field) { + DataType dataType = SparkSchemaUtil.convert(field.type()); + StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); + this.sparkType = dataType; +- this.descriptor = TypeUtil.convertToParquet(structField); ++ this.descriptor = ++ CometTypeUtils.buildColumnDescriptor(TypeUtil.convertToParquetSpec(structField)); ++ this.fieldId = field.fieldId(); + } + + public AbstractColumnReader delegate() { +@@ -92,7 +101,26 @@ class CometColumnReader implements VectorizedReader { + } + + this.importer = new CometSchemaImporter(new RootAllocator()); +- this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); ++ ++ spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ ++ boolean useLegacyTime = ++ Boolean.parseBoolean( ++ SQLConf.get() ++ .getConfString( ++ CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); ++ boolean useLazyMaterialization = ++ Boolean.parseBoolean( ++ SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "false")); ++ this.delegate = ++ Utils.getColumnReader( ++ sparkType, ++ spec, ++ importer, ++ batchSize, ++ true, // Comet sets this to true for native execution ++ useLazyMaterialization, ++ useLegacyTime); + this.initialized = true; + } + +@@ -111,9 +139,9 @@ class CometColumnReader implements VectorizedReader { + *

NOTE: this should be called before reading a new Parquet column chunk, and after {@link + * CometColumnReader#reset} is called. + */ +- public void setPageReader(PageReader pageReader) throws IOException { ++ public void setPageReader(RowGroupReader pageStore) throws IOException { + Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); +- ((ColumnReader) delegate).setPageReader(pageReader); ++ ((ColumnReader) delegate).setRowGroupReader(pageStore, spec); + } + + @Override +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +index 04ac69476..916face2b 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +@@ -22,8 +22,12 @@ import java.io.IOException; + import java.io.UncheckedIOException; + import java.util.List; + import java.util.Map; ++import org.apache.comet.CometRuntimeException; + import org.apache.comet.parquet.AbstractColumnReader; +-import org.apache.comet.parquet.BatchReader; ++import org.apache.comet.parquet.IcebergCometBatchReader; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.vector.CometSelectionVector; ++import org.apache.comet.vector.CometVector; + import org.apache.iceberg.Schema; + import org.apache.iceberg.data.DeleteFilter; + import org.apache.iceberg.parquet.VectorizedReader; +@@ -55,7 +59,7 @@ class CometColumnarBatchReader implements VectorizedReader { + // calling BatchReader.nextBatch, the isDeleted value is not yet available, so + // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is + // available. +- private final BatchReader delegate; ++ private final IcebergCometBatchReader delegate; + private DeleteFilter deletes = null; + private long rowStartPosInBatch = 0; + +@@ -65,9 +69,7 @@ class CometColumnarBatchReader implements VectorizedReader { + this.hasIsDeletedColumn = + readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); + +- AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; +- this.delegate = new BatchReader(abstractColumnReaders); +- delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); ++ this.delegate = new IcebergCometBatchReader(readers.size(), SparkSchemaUtil.convert(schema)); + } + + @Override +@@ -79,19 +81,22 @@ class CometColumnarBatchReader implements VectorizedReader { + && !(readers[i] instanceof CometPositionColumnReader) + && !(readers[i] instanceof CometDeleteColumnReader)) { + readers[i].reset(); +- readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); ++ readers[i].setPageReader((RowGroupReader) pageStore); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); + } + } + ++ AbstractColumnReader[] delegateReaders = new AbstractColumnReader[readers.length]; + for (int i = 0; i < readers.length; i++) { +- delegate.getColumnReaders()[i] = this.readers[i].delegate(); ++ delegateReaders[i] = readers[i].delegate(); + } + ++ delegate.init(delegateReaders); ++ + this.rowStartPosInBatch = +- pageStore ++ ((RowGroupReader) pageStore) + .getRowIndexOffset() + .orElseThrow( + () -> +@@ -148,9 +153,17 @@ class CometColumnarBatchReader implements VectorizedReader { + Pair pair = buildRowIdMapping(vectors); + if (pair != null) { + int[] rowIdMapping = pair.first(); +- numLiveRows = pair.second(); +- for (int i = 0; i < vectors.length; i++) { +- vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); ++ if (pair.second() != null) { ++ numLiveRows = pair.second(); ++ for (int i = 0; i < vectors.length; i++) { ++ if (vectors[i] instanceof CometVector) { ++ vectors[i] = ++ new CometSelectionVector((CometVector) vectors[i], rowIdMapping, numLiveRows); ++ } else { ++ throw new CometRuntimeException( ++ "Unsupported column vector type: " + vectors[i].getClass()); ++ } ++ } + } + } + } +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +index 047c96314..88d691a60 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +@@ -21,6 +21,7 @@ package org.apache.iceberg.spark.data.vectorized; + import java.math.BigDecimal; + import java.nio.ByteBuffer; + import org.apache.comet.parquet.ConstantColumnReader; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.types.Types; + import org.apache.spark.sql.types.DataType; + import org.apache.spark.sql.types.DataTypes; +@@ -34,7 +35,11 @@ class CometConstantColumnReader extends CometColumnReader { + super(field); + // use delegate to set constant value on the native side to be consumed by native execution. + setDelegate( +- new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); ++ new ConstantColumnReader( ++ sparkType(), ++ CometTypeUtils.descriptorToParquetColumnSpec(descriptor()), ++ convertToSparkValue(value), ++ false)); + } + + @Override +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +index 6235bfe48..cba108e43 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +@@ -51,10 +51,10 @@ class CometDeleteColumnReader extends CometColumnReader { + DeleteColumnReader() { + super( + DataTypes.BooleanType, +- TypeUtil.convertToParquet( ++ TypeUtil.convertToParquetSpec( + new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), + false /* useDecimal128 = false */, +- false /* isConstant = false */); ++ false /* isConstant */); + this.isDeleted = new boolean[0]; + } + +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +index bcc0e514c..98e80068c 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +@@ -20,6 +20,7 @@ package org.apache.iceberg.spark.data.vectorized; + + import org.apache.comet.parquet.MetadataColumnReader; + import org.apache.comet.parquet.Native; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.types.Types; + import org.apache.parquet.column.ColumnDescriptor; + import org.apache.spark.sql.types.DataTypes; +@@ -44,7 +45,7 @@ class CometPositionColumnReader extends CometColumnReader { + PositionColumnReader(ColumnDescriptor descriptor) { + super( + DataTypes.LongType, +- descriptor, ++ CometTypeUtils.descriptorToParquetColumnSpec(descriptor), + false /* useDecimal128 = false */, + false /* isConstant = false */); + } +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +index d36f1a727..56f8c9bff 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +@@ -142,6 +142,7 @@ class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor extends BaseReader taskGroup = (ScanTaskGroup) task; ++ return taskGroup.tasks().stream().allMatch(this::supportsCometBatchReads); ++ ++ } else if (task.isFileScanTask() && !task.isDataTask()) { ++ FileScanTask fileScanTask = task.asFileScanTask(); ++ // Comet can't handle delete files for now ++ return fileScanTask.file().format() == FileFormat.PARQUET; ++ ++ } else { ++ return false; ++ } ++ } ++ + // conditions for using ORC batch reads: + // - ORC vectorization is enabled + // - all tasks are of type FileScanTask and read only ORC files with no delete files +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +index 106b296de..967b0d41d 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +@@ -24,6 +24,7 @@ import java.util.Map; + import java.util.Optional; + import java.util.function.Supplier; + import java.util.stream.Collectors; ++import org.apache.comet.parquet.SupportsComet; + import org.apache.iceberg.BlobMetadata; + import org.apache.iceberg.ScanTask; + import org.apache.iceberg.ScanTaskGroup; +@@ -95,7 +96,7 @@ import org.apache.spark.sql.types.StructType; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + +-abstract class SparkScan implements Scan, SupportsReportStatistics { ++abstract class SparkScan implements Scan, SupportsReportStatistics, SupportsComet { + private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; + +@@ -351,4 +352,10 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { + return splitSize; + } + } ++ ++ @Override ++ public boolean isCometEnabled() { ++ SparkBatch batch = (SparkBatch) this.toBatch(); ++ return batch.useCometBatchReads(); ++ } + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +index 404ba7284..00e97e96f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +@@ -90,6 +90,16 @@ public abstract class SparkDistributedDataScanTestBase + .master("local[2]") + .config("spark.serializer", serializer) + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +index 659507e4c..9076ec24d 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +@@ -73,6 +73,16 @@ public class TestSparkDistributedDataScanDeletes + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java +index a218f965e..eca0125ac 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java +@@ -62,6 +62,16 @@ public class TestSparkDistributedDataScanFilterFiles + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +index 2665d7ba8..2381d0aa1 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +@@ -63,6 +63,16 @@ public class TestSparkDistributedDataScanReporting + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +index daf4e29ac..cffe2944e 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +@@ -79,6 +79,17 @@ public abstract class TestBase extends SparkTestHelperBase { + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .config("spark.comet.exec.broadcastExchange.enabled", "false") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java +index 973a17c9a..4490c219d 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java +@@ -65,6 +65,16 @@ public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVect + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +index 1c5905744..543edf3b2 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +@@ -61,6 +61,16 @@ public abstract class ScanTestBase extends AvroDataTestBase { + ScanTestBase.spark = + SparkSession.builder() + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[2]") + .getOrCreate(); + ScanTestBase.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +index 19ec6d13d..e2d0a84d0 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +@@ -144,7 +144,20 @@ public class TestCompressionSettings extends CatalogTestBase { + + @BeforeAll + public static void startSpark() { +- TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestCompressionSettings.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @BeforeEach +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +index a7702b169..e1fdafae8 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +@@ -74,7 +74,20 @@ public class TestDataSourceOptions extends TestBaseWithCatalog { + + @BeforeAll + public static void startSpark() { +- TestDataSourceOptions.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestDataSourceOptions.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +index fd7d52178..c8aab2996 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +@@ -114,6 +114,16 @@ public class TestFilteredScan { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +index 153564f7d..264179459 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +@@ -98,6 +98,16 @@ public class TestForwardCompatibility { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +index f4f57157e..7dd2ed811 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +@@ -51,6 +51,16 @@ public class TestIcebergSpark { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +index e1402396f..8b60b8bc4 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +@@ -118,6 +118,16 @@ public class TestPartitionPruning { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + TestPartitionPruning.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +index 0b6ab2052..dec4026f3 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +@@ -112,6 +112,16 @@ public class TestPartitionValues { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +index 11865db7f..bdb9ae32a 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +@@ -91,6 +91,16 @@ public class TestSnapshotSelection { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +index 3051e27d7..22899c233 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +@@ -125,6 +125,16 @@ public class TestSparkDataFile { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + TestSparkDataFile.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +index 4ccbf86f1..2089bac57 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +@@ -100,6 +100,16 @@ public class TestSparkDataWrite { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +@@ -144,7 +154,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { +@@ -213,7 +223,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + +@@ -258,7 +268,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + +@@ -310,7 +320,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + +@@ -352,7 +362,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + +@@ -391,7 +401,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); +@@ -459,7 +469,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + +@@ -706,7 +716,7 @@ public class TestSparkDataWrite { + // Since write and commit succeeded, the rows should be readable + Dataset result = spark.read().format("iceberg").load(targetLocation); + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual) + .hasSize(records.size() + records2.size()) + .containsExactlyInAnyOrder( +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +index 596d05d30..e681d8170 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +@@ -88,6 +88,16 @@ public class TestSparkReadProjection extends TestReadProjection { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + ImmutableMap config = + ImmutableMap.of( +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +index 42699f466..c5e077d7a 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +@@ -138,6 +138,16 @@ public class TestSparkReaderDeletes extends DeleteReadTests { + .config("spark.ui.liveUpdate.period", 0) + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +@@ -205,9 +215,10 @@ public class TestSparkReaderDeletes extends DeleteReadTests { + catalog.dropTable(TableIdentifier.of("default", name)); + } + +- protected boolean countDeletes() { +- return true; +- } ++protected boolean countDeletes() { ++ // TODO: Enable once iceberg-rust exposes delete count metrics to Comet ++ return false; ++} + + @Override + protected long deleteCount() { +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +index baf7fa8f8..150cc8969 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +@@ -182,6 +182,16 @@ public class TestSparkReaderWithBloomFilter { + SparkSession.builder() + .master("local[2]") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +index 54048bbf2..c87216824 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +@@ -69,6 +69,16 @@ public class TestStructuredStreaming { + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .config("spark.sql.shuffle.partitions", 4) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +index 8b1e3fbfc..b4385c395 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +@@ -75,7 +75,20 @@ public class TestTimestampWithoutZone extends TestBase { + + @BeforeAll + public static void startSpark() { +- TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestTimestampWithoutZone.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +index c3fac70dd..b6c66f3f8 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +@@ -84,6 +84,16 @@ public class TestWriteMetricsConfig { + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + TestWriteMetricsConfig.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +index 5ce56b4fe..df1f914da 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +@@ -63,6 +63,16 @@ public class TestAggregatePushDown extends CatalogTestBase { + SparkSession.builder() + .master("local[2]") + .config("spark.sql.iceberg.aggregate_pushdown", "true") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +index 9d2ce2b38..5e2336884 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +@@ -598,9 +598,7 @@ public class TestFilterPushDown extends TestBaseWithCatalog { + String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); + + if (sparkFilter != null) { +- assertThat(planAsString) +- .as("Post scan filter should match") +- .contains("Filter (" + sparkFilter + ")"); ++ assertThat(planAsString).as("Post scan filter should match").contains("CometFilter"); + } else { + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); + } diff --git a/dev/diffs/iceberg-rust/1.8.1.diff b/dev/diffs/iceberg-rust/1.8.1.diff new file mode 100644 index 0000000000..e7b58902ca --- /dev/null +++ b/dev/diffs/iceberg-rust/1.8.1.diff @@ -0,0 +1,1902 @@ +diff --git a/build.gradle b/build.gradle +index 7327b3890..7967109f0 100644 +--- a/build.gradle ++++ b/build.gradle +@@ -780,6 +780,13 @@ project(':iceberg-parquet') { + implementation project(':iceberg-core') + implementation project(':iceberg-common') + ++ implementation("org.apache.datafusion:comet-spark-spark${sparkVersionsString}_${scalaVersion}:${libs.versions.comet.get()}") { ++ exclude group: 'org.apache.arrow' ++ exclude group: 'org.apache.parquet' ++ exclude group: 'org.apache.spark' ++ exclude group: 'org.apache.iceberg' ++ } ++ + implementation(libs.parquet.avro) { + exclude group: 'org.apache.avro', module: 'avro' + // already shaded by Parquet +diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml +index 04ffa8f4e..3a57af315 100644 +--- a/gradle/libs.versions.toml ++++ b/gradle/libs.versions.toml +@@ -34,6 +34,7 @@ azuresdk-bom = "1.2.31" + awssdk-s3accessgrants = "2.3.0" + caffeine = "2.9.3" + calcite = "1.10.0" ++comet = "0.12.0-SNAPSHOT" + datasketches = "6.2.0" + delta-standalone = "3.3.0" + delta-spark = "3.3.0" +@@ -81,7 +82,7 @@ slf4j = "2.0.16" + snowflake-jdbc = "3.22.0" + spark-hive33 = "3.3.4" + spark-hive34 = "3.4.4" +-spark-hive35 = "3.5.4" ++spark-hive35 = "3.5.6" + sqlite-jdbc = "3.48.0.0" + testcontainers = "1.20.4" + tez010 = "0.10.4" +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java +new file mode 100644 +index 000000000..ddf6c7de5 +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.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.iceberg.parquet; ++ ++import java.util.Map; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.iceberg.relocated.com.google.common.collect.Maps; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.schema.LogicalTypeAnnotation; ++import org.apache.parquet.schema.PrimitiveType; ++import org.apache.parquet.schema.Type; ++import org.apache.parquet.schema.Types; ++ ++public class CometTypeUtils { ++ ++ private CometTypeUtils() {} ++ ++ public static ParquetColumnSpec descriptorToParquetColumnSpec(ColumnDescriptor descriptor) { ++ ++ String[] path = descriptor.getPath(); ++ PrimitiveType primitiveType = descriptor.getPrimitiveType(); ++ String physicalType = primitiveType.getPrimitiveTypeName().name(); ++ ++ int typeLength = ++ primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY ++ ? primitiveType.getTypeLength() ++ : 0; ++ ++ boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; ++ ++ // ToDo: extract this into a Util method ++ String logicalTypeName = null; ++ Map logicalTypeParams = Maps.newHashMap(); ++ LogicalTypeAnnotation logicalType = primitiveType.getLogicalTypeAnnotation(); ++ ++ if (logicalType != null) { ++ logicalTypeName = logicalType.getClass().getSimpleName(); ++ ++ // Handle specific logical types ++ if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal = ++ (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("precision", String.valueOf(decimal.getPrecision())); ++ logicalTypeParams.put("scale", String.valueOf(decimal.getScale())); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp = ++ (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(timestamp.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", timestamp.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimeLogicalTypeAnnotation time = ++ (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(time.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", time.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.IntLogicalTypeAnnotation intType = ++ (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned())); ++ logicalTypeParams.put("bitWidth", String.valueOf(intType.getBitWidth())); ++ } ++ } ++ ++ return new ParquetColumnSpec( ++ 1, // ToDo: pass in the correct id ++ path, ++ physicalType, ++ typeLength, ++ isRepeated, ++ descriptor.getMaxDefinitionLevel(), ++ descriptor.getMaxRepetitionLevel(), ++ logicalTypeName, ++ logicalTypeParams); ++ } ++ ++ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { ++ PrimitiveType.PrimitiveTypeName primType = ++ PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); ++ ++ Type.Repetition repetition; ++ if (columnSpec.getMaxRepetitionLevel() > 0) { ++ repetition = Type.Repetition.REPEATED; ++ } else if (columnSpec.getMaxDefinitionLevel() > 0) { ++ repetition = Type.Repetition.OPTIONAL; ++ } else { ++ repetition = Type.Repetition.REQUIRED; ++ } ++ ++ String name = columnSpec.getPath()[columnSpec.getPath().length - 1]; ++ // Reconstruct the logical type from parameters ++ LogicalTypeAnnotation logicalType = null; ++ if (columnSpec.getLogicalTypeName() != null) { ++ logicalType = ++ reconstructLogicalType( ++ columnSpec.getLogicalTypeName(), columnSpec.getLogicalTypeParams()); ++ } ++ ++ PrimitiveType primitiveType; ++ if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { ++ primitiveType = ++ org.apache.parquet.schema.Types.primitive(primType, repetition) ++ .length(columnSpec.getTypeLength()) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } else { ++ primitiveType = ++ Types.primitive(primType, repetition) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } ++ ++ return new ColumnDescriptor( ++ columnSpec.getPath(), ++ primitiveType, ++ columnSpec.getMaxRepetitionLevel(), ++ columnSpec.getMaxDefinitionLevel()); ++ } ++ ++ private static LogicalTypeAnnotation reconstructLogicalType( ++ String logicalTypeName, java.util.Map params) { ++ ++ switch (logicalTypeName) { ++ // MAP ++ case "MapLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.mapType(); ++ ++ // LIST ++ case "ListLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.listType(); ++ ++ // STRING ++ case "StringLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.stringType(); ++ ++ // MAP_KEY_VALUE ++ case "MapKeyValueLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(); ++ ++ // ENUM ++ case "EnumLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.enumType(); ++ ++ // DECIMAL ++ case "DecimalLogicalTypeAnnotation": ++ if (!params.containsKey("scale") || !params.containsKey("precision")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for DecimalLogicalTypeAnnotation: " + params); ++ } ++ int scale = Integer.parseInt(params.get("scale")); ++ int precision = Integer.parseInt(params.get("precision")); ++ return LogicalTypeAnnotation.decimalType(scale, precision); ++ ++ // DATE ++ case "DateLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.dateType(); ++ ++ // TIME ++ case "TimeLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimeLogicalTypeAnnotation: " + params); ++ } ++ ++ boolean isUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String timeUnitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit timeUnit; ++ switch (timeUnitStr) { ++ case "MILLIS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown time unit: " + timeUnitStr); ++ } ++ return LogicalTypeAnnotation.timeType(isUTC, timeUnit); ++ ++ // TIMESTAMP ++ case "TimestampLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimestampLogicalTypeAnnotation: " + params); ++ } ++ boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String unitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit unit; ++ switch (unitStr) { ++ case "MILLIS": ++ unit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ unit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ unit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown timestamp unit: " + unitStr); ++ } ++ return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); ++ ++ // INTEGER ++ case "IntLogicalTypeAnnotation": ++ if (!params.containsKey("isSigned") || !params.containsKey("bitWidth")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for IntLogicalTypeAnnotation: " + params); ++ } ++ boolean isSigned = Boolean.parseBoolean(params.get("isSigned")); ++ int bitWidth = Integer.parseInt(params.get("bitWidth")); ++ return LogicalTypeAnnotation.intType(bitWidth, isSigned); ++ ++ // JSON ++ case "JsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.jsonType(); ++ ++ // BSON ++ case "BsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.bsonType(); ++ ++ // UUID ++ case "UUIDLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.uuidType(); ++ ++ // INTERVAL ++ case "IntervalLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(); ++ ++ default: ++ throw new IllegalArgumentException("Unknown logical type: " + logicalTypeName); ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +new file mode 100644 +index 000000000..a3cba4018 +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +@@ -0,0 +1,260 @@ ++/* ++ * 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.iceberg.parquet; ++ ++import java.io.IOException; ++import java.io.UncheckedIOException; ++import java.nio.ByteBuffer; ++import java.util.List; ++import java.util.Map; ++import java.util.NoSuchElementException; ++import java.util.function.Function; ++import org.apache.comet.parquet.FileReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.ReadOptions; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.parquet.WrappedInputFile; ++import org.apache.hadoop.conf.Configuration; ++import org.apache.iceberg.Schema; ++import org.apache.iceberg.exceptions.RuntimeIOException; ++import org.apache.iceberg.expressions.Expression; ++import org.apache.iceberg.expressions.Expressions; ++import org.apache.iceberg.io.CloseableGroup; ++import org.apache.iceberg.io.CloseableIterable; ++import org.apache.iceberg.io.CloseableIterator; ++import org.apache.iceberg.io.InputFile; ++import org.apache.iceberg.mapping.NameMapping; ++import org.apache.iceberg.relocated.com.google.common.collect.Lists; ++import org.apache.iceberg.util.ByteBuffers; ++import org.apache.parquet.ParquetReadOptions; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; ++import org.apache.parquet.hadoop.metadata.ColumnPath; ++import org.apache.parquet.schema.MessageType; ++ ++public class CometVectorizedParquetReader extends CloseableGroup ++ implements CloseableIterable { ++ private final InputFile input; ++ private final ParquetReadOptions options; ++ private final Schema expectedSchema; ++ private final Function> batchReaderFunc; ++ private final Expression filter; ++ private final boolean reuseContainers; ++ private final boolean caseSensitive; ++ private final int batchSize; ++ private final NameMapping nameMapping; ++ private final Map properties; ++ private Long start = null; ++ private Long length = null; ++ private ByteBuffer fileEncryptionKey = null; ++ private ByteBuffer fileAADPrefix = null; ++ ++ public CometVectorizedParquetReader( ++ InputFile input, ++ Schema expectedSchema, ++ ParquetReadOptions options, ++ Function> readerFunc, ++ NameMapping nameMapping, ++ Expression filter, ++ boolean reuseContainers, ++ boolean caseSensitive, ++ int maxRecordsPerBatch, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.input = input; ++ this.expectedSchema = expectedSchema; ++ this.options = options; ++ this.batchReaderFunc = readerFunc; ++ // replace alwaysTrue with null to avoid extra work evaluating a trivial filter ++ this.filter = filter == Expressions.alwaysTrue() ? null : filter; ++ this.reuseContainers = reuseContainers; ++ this.caseSensitive = caseSensitive; ++ this.batchSize = maxRecordsPerBatch; ++ this.nameMapping = nameMapping; ++ this.properties = properties; ++ this.start = start; ++ this.length = length; ++ this.fileEncryptionKey = fileEncryptionKey; ++ this.fileAADPrefix = fileAADPrefix; ++ } ++ ++ private ReadConf conf = null; ++ ++ private ReadConf init() { ++ if (conf == null) { ++ ReadConf readConf = ++ new ReadConf( ++ input, ++ options, ++ expectedSchema, ++ filter, ++ null, ++ batchReaderFunc, ++ nameMapping, ++ reuseContainers, ++ caseSensitive, ++ batchSize); ++ this.conf = readConf.copy(); ++ return readConf; ++ } ++ return conf; ++ } ++ ++ @Override ++ public CloseableIterator iterator() { ++ FileIterator iter = ++ new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); ++ addCloseable(iter); ++ return iter; ++ } ++ ++ private static class FileIterator implements CloseableIterator { ++ // private final ParquetFileReader reader; ++ private final boolean[] shouldSkip; ++ private final VectorizedReader model; ++ private final long totalValues; ++ private final int batchSize; ++ private final List> columnChunkMetadata; ++ private final boolean reuseContainers; ++ private int nextRowGroup = 0; ++ private long nextRowGroupStart = 0; ++ private long valuesRead = 0; ++ private T last = null; ++ private final FileReader cometReader; ++ private ReadConf conf; ++ ++ FileIterator( ++ ReadConf conf, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.shouldSkip = conf.shouldSkip(); ++ this.totalValues = conf.totalValues(); ++ this.reuseContainers = conf.reuseContainers(); ++ this.model = conf.vectorizedModel(); ++ this.batchSize = conf.batchSize(); ++ this.model.setBatchSize(this.batchSize); ++ this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups(); ++ this.cometReader = ++ newCometReader( ++ conf.file(), ++ conf.projection(), ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ this.conf = conf; ++ } ++ ++ private FileReader newCometReader( ++ InputFile file, ++ MessageType projection, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ try { ++ ReadOptions cometOptions = ReadOptions.builder(new Configuration()).build(); ++ ++ FileReader fileReader = ++ new FileReader( ++ new WrappedInputFile(file), ++ cometOptions, ++ properties, ++ start, ++ length, ++ ByteBuffers.toByteArray(fileEncryptionKey), ++ ByteBuffers.toByteArray(fileAADPrefix)); ++ ++ List columnDescriptors = projection.getColumns(); ++ ++ List specs = Lists.newArrayList(); ++ ++ for (ColumnDescriptor descriptor : columnDescriptors) { ++ ParquetColumnSpec spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ specs.add(spec); ++ } ++ ++ fileReader.setRequestedSchemaFromSpecs(specs); ++ return fileReader; ++ } catch (IOException e) { ++ throw new UncheckedIOException("Failed to open Parquet file: " + file.location(), e); ++ } ++ } ++ ++ @Override ++ public boolean hasNext() { ++ return valuesRead < totalValues; ++ } ++ ++ @Override ++ public T next() { ++ if (!hasNext()) { ++ throw new NoSuchElementException(); ++ } ++ if (valuesRead >= nextRowGroupStart) { ++ advance(); ++ } ++ ++ // batchSize is an integer, so casting to integer is safe ++ int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, batchSize); ++ if (reuseContainers) { ++ this.last = model.read(last, numValuesToRead); ++ } else { ++ this.last = model.read(null, numValuesToRead); ++ } ++ valuesRead += numValuesToRead; ++ ++ return last; ++ } ++ ++ private void advance() { ++ while (shouldSkip[nextRowGroup]) { ++ nextRowGroup += 1; ++ cometReader.skipNextRowGroup(); ++ } ++ RowGroupReader pages; ++ try { ++ pages = cometReader.readNextRowGroup(); ++ } catch (IOException e) { ++ throw new RuntimeIOException(e); ++ } ++ ++ model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup)); ++ nextRowGroupStart += pages.getRowCount(); ++ nextRowGroup += 1; ++ } ++ ++ @Override ++ public void close() throws IOException { ++ model.close(); ++ cometReader.close(); ++ if (conf != null && conf.reader() != null) { ++ conf.reader().close(); ++ } ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +index 2c37a5244..3442cfc43 100644 +--- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +@@ -1075,6 +1075,7 @@ public class Parquet { + private NameMapping nameMapping = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; ++ private boolean isComet; + + private ReadBuilder(InputFile file) { + this.file = file; +@@ -1172,6 +1173,11 @@ public class Parquet { + return this; + } + ++ public ReadBuilder enableComet(boolean enableComet) { ++ this.isComet = enableComet; ++ return this; ++ } ++ + public ReadBuilder withFileEncryptionKey(ByteBuffer encryptionKey) { + this.fileEncryptionKey = encryptionKey; + return this; +@@ -1182,7 +1188,7 @@ public class Parquet { + return this; + } + +- @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) ++ @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity", "MethodLength"}) + public CloseableIterable build() { + FileDecryptionProperties fileDecryptionProperties = null; + if (fileEncryptionKey != null) { +@@ -1234,16 +1240,35 @@ public class Parquet { + } + + if (batchedReaderFunc != null) { +- return new VectorizedParquetReader<>( +- file, +- schema, +- options, +- batchedReaderFunc, +- mapping, +- filter, +- reuseContainers, +- caseSensitive, +- maxRecordsPerBatch); ++ if (isComet) { ++ LOG.info("Comet enabled"); ++ return new CometVectorizedParquetReader<>( ++ file, ++ schema, ++ options, ++ batchedReaderFunc, ++ mapping, ++ filter, ++ reuseContainers, ++ caseSensitive, ++ maxRecordsPerBatch, ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ } else { ++ return new VectorizedParquetReader<>( ++ file, ++ schema, ++ options, ++ batchedReaderFunc, ++ mapping, ++ filter, ++ reuseContainers, ++ caseSensitive, ++ maxRecordsPerBatch); ++ } + } else { + return new org.apache.iceberg.parquet.ParquetReader<>( + file, schema, options, readerFunc, mapping, filter, reuseContainers, caseSensitive); +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +index 1fb2372ba..142e5fbad 100644 +--- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +@@ -157,6 +157,14 @@ class ReadConf { + return newReader; + } + ++ InputFile file() { ++ return file; ++ } ++ ++ MessageType projection() { ++ return projection; ++ } ++ + ParquetValueReader model() { + return model; + } +diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle +index e2d2c7a7a..f64232dc5 100644 +--- a/spark/v3.5/build.gradle ++++ b/spark/v3.5/build.gradle +@@ -75,7 +75,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { + exclude group: 'org.roaringbitmap' + } + +- compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.5.0" ++ compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + implementation libs.parquet.column + implementation libs.parquet.hadoop +@@ -183,7 +183,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer + testImplementation libs.avro.avro + testImplementation libs.parquet.hadoop + testImplementation libs.awaitility +- testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.5.0" ++ testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + // Required because we remove antlr plugin dependencies from the compile configuration, see note above + runtimeOnly libs.antlr.runtime +@@ -263,6 +263,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') ++ integrationImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + // runtime dependencies for running Hive Catalog based integration test + integrationRuntimeOnly project(':iceberg-hive-metastore') +@@ -300,8 +301,8 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'avro.shaded', 'org.apache.iceberg.shaded.org.apache.avro.shaded' + relocate 'com.thoughtworks.paranamer', 'org.apache.iceberg.shaded.com.thoughtworks.paranamer' +- relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' +- relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' ++// relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' ++// relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' +diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +index 578845e3d..4f44a73db 100644 +--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java ++++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +@@ -57,6 +57,16 @@ public abstract class ExtensionsTestBase extends CatalogTestBase { + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + .config( + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean())) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +index ade19de36..9111397e9 100644 +--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java ++++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +@@ -56,6 +56,16 @@ public class TestCallStatementParser { + .master("local[2]") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.extra.prop", "value") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + TestCallStatementParser.parser = spark.sessionState().sqlParser(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +index 64edb1002..0fc10120f 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +@@ -179,6 +179,16 @@ public class DeleteOrphanFilesBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", catalogWarehouse()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local"); + spark = builder.getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +index a5d0456b0..f0759f837 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +@@ -392,6 +392,16 @@ public class IcebergSortCompactionBenchmark { + "spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", getCatalogWarehouse()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]"); + spark = builder.getOrCreate(); + Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java +index c6794e43c..457d2823e 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java +@@ -239,6 +239,16 @@ public class DVReaderBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]") + .getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java +index ac74fb5a1..eab09293d 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java +@@ -223,6 +223,16 @@ public class DVWriterBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]") + .getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java +index 68c537e34..1e9e90d53 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java +@@ -94,7 +94,19 @@ public abstract class IcebergSourceBenchmark { + } + + protected void setupSpark(boolean enableDictionaryEncoding) { +- SparkSession.Builder builder = SparkSession.builder().config("spark.ui.enabled", false); ++ SparkSession.Builder builder = ++ SparkSession.builder() ++ .config("spark.ui.enabled", false) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true"); + if (!enableDictionaryEncoding) { + builder + .config("parquet.dictionary.page.size", "1") +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +index 4794863ab..8bb508f19 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +@@ -20,21 +20,25 @@ package org.apache.iceberg.spark.data.vectorized; + + import java.io.IOException; + import java.util.Map; ++import org.apache.comet.CometConf; ++import org.apache.comet.CometSchemaImporter; + import org.apache.comet.parquet.AbstractColumnReader; + import org.apache.comet.parquet.ColumnReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.RowGroupReader; + import org.apache.comet.parquet.TypeUtil; + import org.apache.comet.parquet.Utils; +-import org.apache.comet.shaded.arrow.c.CometSchemaImporter; + import org.apache.comet.shaded.arrow.memory.RootAllocator; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.parquet.VectorizedReader; + import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + import org.apache.iceberg.spark.SparkSchemaUtil; + import org.apache.iceberg.types.Types; + import org.apache.parquet.column.ColumnDescriptor; + import org.apache.parquet.column.page.PageReadStore; +-import org.apache.parquet.column.page.PageReader; + import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; + import org.apache.parquet.hadoop.metadata.ColumnPath; ++import org.apache.spark.sql.internal.SQLConf; + import org.apache.spark.sql.types.DataType; + import org.apache.spark.sql.types.Metadata; + import org.apache.spark.sql.types.StructField; +@@ -46,23 +50,28 @@ class CometColumnReader implements VectorizedReader { + + private final ColumnDescriptor descriptor; + private final DataType sparkType; ++ private final int fieldId; + + // The delegated ColumnReader from Comet side + private AbstractColumnReader delegate; + private boolean initialized = false; + private int batchSize = DEFAULT_BATCH_SIZE; + private CometSchemaImporter importer; ++ private ParquetColumnSpec spec; + +- CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { ++ CometColumnReader(DataType sparkType, ColumnDescriptor descriptor, int fieldId) { + this.sparkType = sparkType; + this.descriptor = descriptor; ++ this.fieldId = fieldId; + } + + CometColumnReader(Types.NestedField field) { + DataType dataType = SparkSchemaUtil.convert(field.type()); + StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); + this.sparkType = dataType; +- this.descriptor = TypeUtil.convertToParquet(structField); ++ this.descriptor = ++ CometTypeUtils.buildColumnDescriptor(TypeUtil.convertToParquetSpec(structField)); ++ this.fieldId = field.fieldId(); + } + + public AbstractColumnReader delegate() { +@@ -96,7 +105,26 @@ class CometColumnReader implements VectorizedReader { + } + + this.importer = new CometSchemaImporter(new RootAllocator()); +- this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); ++ ++ spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ ++ boolean useLegacyTime = ++ Boolean.parseBoolean( ++ SQLConf.get() ++ .getConfString( ++ CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); ++ boolean useLazyMaterialization = ++ Boolean.parseBoolean( ++ SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "false")); ++ this.delegate = ++ Utils.getColumnReader( ++ sparkType, ++ spec, ++ importer, ++ batchSize, ++ true, // Comet sets this to true for native execution ++ useLazyMaterialization, ++ useLegacyTime); + this.initialized = true; + } + +@@ -115,9 +143,9 @@ class CometColumnReader implements VectorizedReader { + *

NOTE: this should be called before reading a new Parquet column chunk, and after {@link + * CometColumnReader#reset} is called. + */ +- public void setPageReader(PageReader pageReader) throws IOException { ++ public void setPageReader(RowGroupReader pageStore) throws IOException { + Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); +- ((ColumnReader) delegate).setPageReader(pageReader); ++ ((ColumnReader) delegate).setRowGroupReader(pageStore, spec); + } + + @Override +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +index 1440e5d1d..85cca62e9 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +@@ -22,8 +22,12 @@ import java.io.IOException; + import java.io.UncheckedIOException; + import java.util.List; + import java.util.Map; ++import org.apache.comet.CometRuntimeException; + import org.apache.comet.parquet.AbstractColumnReader; +-import org.apache.comet.parquet.BatchReader; ++import org.apache.comet.parquet.IcebergCometBatchReader; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.vector.CometSelectionVector; ++import org.apache.comet.vector.CometVector; + import org.apache.iceberg.Schema; + import org.apache.iceberg.data.DeleteFilter; + import org.apache.iceberg.parquet.VectorizedReader; +@@ -55,7 +59,7 @@ class CometColumnarBatchReader implements VectorizedReader { + // calling BatchReader.nextBatch, the isDeleted value is not yet available, so + // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is + // available. +- private final BatchReader delegate; ++ private final IcebergCometBatchReader delegate; + private DeleteFilter deletes = null; + private long rowStartPosInBatch = 0; + +@@ -65,9 +69,7 @@ class CometColumnarBatchReader implements VectorizedReader { + this.hasIsDeletedColumn = + readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); + +- AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; +- this.delegate = new BatchReader(abstractColumnReaders); +- delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); ++ this.delegate = new IcebergCometBatchReader(readers.size(), SparkSchemaUtil.convert(schema)); + } + + @Override +@@ -85,19 +87,22 @@ class CometColumnarBatchReader implements VectorizedReader { + && !(readers[i] instanceof CometPositionColumnReader) + && !(readers[i] instanceof CometDeleteColumnReader)) { + readers[i].reset(); +- readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); ++ readers[i].setPageReader((RowGroupReader) pageStore); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); + } + } + ++ AbstractColumnReader[] delegateReaders = new AbstractColumnReader[readers.length]; + for (int i = 0; i < readers.length; i++) { +- delegate.getColumnReaders()[i] = this.readers[i].delegate(); ++ delegateReaders[i] = readers[i].delegate(); + } + ++ delegate.init(delegateReaders); ++ + this.rowStartPosInBatch = +- pageStore ++ ((RowGroupReader) pageStore) + .getRowIndexOffset() + .orElseThrow( + () -> +@@ -154,9 +159,17 @@ class CometColumnarBatchReader implements VectorizedReader { + Pair pair = buildRowIdMapping(vectors); + if (pair != null) { + int[] rowIdMapping = pair.first(); +- numLiveRows = pair.second(); +- for (int i = 0; i < vectors.length; i++) { +- vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); ++ if (pair.second() != null) { ++ numLiveRows = pair.second(); ++ for (int i = 0; i < vectors.length; i++) { ++ if (vectors[i] instanceof CometVector) { ++ vectors[i] = ++ new CometSelectionVector((CometVector) vectors[i], rowIdMapping, numLiveRows); ++ } else { ++ throw new CometRuntimeException( ++ "Unsupported column vector type: " + vectors[i].getClass()); ++ } ++ } + } + } + } +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +index 047c96314..88d691a60 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +@@ -21,6 +21,7 @@ package org.apache.iceberg.spark.data.vectorized; + import java.math.BigDecimal; + import java.nio.ByteBuffer; + import org.apache.comet.parquet.ConstantColumnReader; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.types.Types; + import org.apache.spark.sql.types.DataType; + import org.apache.spark.sql.types.DataTypes; +@@ -34,7 +35,11 @@ class CometConstantColumnReader extends CometColumnReader { + super(field); + // use delegate to set constant value on the native side to be consumed by native execution. + setDelegate( +- new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); ++ new ConstantColumnReader( ++ sparkType(), ++ CometTypeUtils.descriptorToParquetColumnSpec(descriptor()), ++ convertToSparkValue(value), ++ false)); + } + + @Override +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +index 6235bfe48..cba108e43 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +@@ -51,10 +51,10 @@ class CometDeleteColumnReader extends CometColumnReader { + DeleteColumnReader() { + super( + DataTypes.BooleanType, +- TypeUtil.convertToParquet( ++ TypeUtil.convertToParquetSpec( + new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), + false /* useDecimal128 = false */, +- false /* isConstant = false */); ++ false /* isConstant */); + this.isDeleted = new boolean[0]; + } + +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +index bcc0e514c..98e80068c 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +@@ -20,6 +20,7 @@ package org.apache.iceberg.spark.data.vectorized; + + import org.apache.comet.parquet.MetadataColumnReader; + import org.apache.comet.parquet.Native; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.types.Types; + import org.apache.parquet.column.ColumnDescriptor; + import org.apache.spark.sql.types.DataTypes; +@@ -44,7 +45,7 @@ class CometPositionColumnReader extends CometColumnReader { + PositionColumnReader(ColumnDescriptor descriptor) { + super( + DataTypes.LongType, +- descriptor, ++ CometTypeUtils.descriptorToParquetColumnSpec(descriptor), + false /* useDecimal128 = false */, + false /* isConstant = false */); + } +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +index d36f1a727..56f8c9bff 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +@@ -142,6 +142,7 @@ class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor extends BaseReader taskGroup = (ScanTaskGroup) task; ++ return taskGroup.tasks().stream().allMatch(this::supportsCometBatchReads); ++ ++ } else if (task.isFileScanTask() && !task.isDataTask()) { ++ FileScanTask fileScanTask = task.asFileScanTask(); ++ // Comet can't handle delete files for now ++ return fileScanTask.file().format() == FileFormat.PARQUET; ++ ++ } else { ++ return false; ++ } ++ } ++ + // conditions for using ORC batch reads: + // - ORC vectorization is enabled + // - all tasks are of type FileScanTask and read only ORC files with no delete files +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +index 019f3919d..656e0600a 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +@@ -23,6 +23,7 @@ import java.util.List; + import java.util.Map; + import java.util.function.Supplier; + import java.util.stream.Collectors; ++import org.apache.comet.parquet.SupportsComet; + import org.apache.iceberg.BlobMetadata; + import org.apache.iceberg.ScanTask; + import org.apache.iceberg.ScanTaskGroup; +@@ -94,7 +95,7 @@ import org.apache.spark.sql.types.StructType; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + +-abstract class SparkScan implements Scan, SupportsReportStatistics { ++abstract class SparkScan implements Scan, SupportsReportStatistics, SupportsComet { + private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; + +@@ -348,4 +349,10 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { + return splitSize; + } + } ++ ++ @Override ++ public boolean isCometEnabled() { ++ SparkBatch batch = (SparkBatch) this.toBatch(); ++ return batch.useCometBatchReads(); ++ } + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +index 404ba7284..00e97e96f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +@@ -90,6 +90,16 @@ public abstract class SparkDistributedDataScanTestBase + .master("local[2]") + .config("spark.serializer", serializer) + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +index 659507e4c..9076ec24d 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +@@ -73,6 +73,16 @@ public class TestSparkDistributedDataScanDeletes + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java +index a218f965e..eca0125ac 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java +@@ -62,6 +62,16 @@ public class TestSparkDistributedDataScanFilterFiles + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +index 2665d7ba8..2381d0aa1 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +@@ -63,6 +63,16 @@ public class TestSparkDistributedDataScanReporting + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +index de68351f6..2b5325e26 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +@@ -77,6 +77,17 @@ public abstract class TestBase extends SparkTestHelperBase { + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .config("spark.comet.exec.broadcastExchange.enabled", "false") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java +index bc4e722bc..3bbff053f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java +@@ -59,7 +59,20 @@ public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVect + + @BeforeAll + public static void startSpark() { +- spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +index 3a269740b..42ecd7ac7 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +@@ -54,7 +54,20 @@ public abstract class ScanTestBase extends AvroDataTest { + + @BeforeAll + public static void startSpark() { +- ScanTestBase.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ ScanTestBase.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + ScanTestBase.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +index f411920a5..afb7830aa 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +@@ -144,7 +144,20 @@ public class TestCompressionSettings extends CatalogTestBase { + + @BeforeAll + public static void startSpark() { +- TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestCompressionSettings.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @BeforeEach +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +index c4ba96e63..98398968b 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +@@ -75,7 +75,20 @@ public class TestDataSourceOptions extends TestBaseWithCatalog { + + @BeforeAll + public static void startSpark() { +- TestDataSourceOptions.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestDataSourceOptions.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +index 348173596..9017b4a37 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +@@ -110,7 +110,20 @@ public class TestFilteredScan { + + @BeforeAll + public static void startSpark() { +- TestFilteredScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestFilteredScan.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +index 84c99a575..1d60b01df 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +@@ -93,7 +93,20 @@ public class TestForwardCompatibility { + + @BeforeAll + public static void startSpark() { +- TestForwardCompatibility.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestForwardCompatibility.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +index 7eff93d20..7aab7f968 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +@@ -46,7 +46,20 @@ public class TestIcebergSpark { + + @BeforeAll + public static void startSpark() { +- TestIcebergSpark.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestIcebergSpark.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +index 9464f687b..ca4e3e035 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +@@ -112,7 +112,20 @@ public class TestPartitionPruning { + + @BeforeAll + public static void startSpark() { +- TestPartitionPruning.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestPartitionPruning.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + TestPartitionPruning.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + String optionKey = String.format("fs.%s.impl", CountOpenLocalFileSystem.scheme); +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +index 5c218f21c..1d2222821 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +@@ -107,7 +107,20 @@ public class TestPartitionValues { + + @BeforeAll + public static void startSpark() { +- TestPartitionValues.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestPartitionValues.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +index a7334a580..ecd25670f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +@@ -87,7 +87,20 @@ public class TestSnapshotSelection { + + @BeforeAll + public static void startSpark() { +- TestSnapshotSelection.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSnapshotSelection.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +index 182b1ef8f..205892e4a 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +@@ -120,7 +120,20 @@ public class TestSparkDataFile { + + @BeforeAll + public static void startSpark() { +- TestSparkDataFile.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSparkDataFile.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + TestSparkDataFile.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +index fb2b312be..3fca3330d 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +@@ -96,7 +96,20 @@ public class TestSparkDataWrite { + + @BeforeAll + public static void startSpark() { +- TestSparkDataWrite.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSparkDataWrite.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterEach +@@ -140,7 +153,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + for (ManifestFile manifest : +@@ -210,7 +223,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -256,7 +269,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -309,7 +322,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -352,7 +365,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -392,7 +405,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + +@@ -458,7 +471,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -622,7 +635,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + +@@ -708,7 +721,7 @@ public class TestSparkDataWrite { + // Since write and commit succeeded, the rows should be readable + Dataset result = spark.read().format("iceberg").load(targetLocation); + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSize(records.size() + records2.size()); + assertThat(actual) + .describedAs("Result rows should match") +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +index becf6a064..21701450a 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +@@ -83,7 +83,20 @@ public class TestSparkReadProjection extends TestReadProjection { + + @BeforeAll + public static void startSpark() { +- TestSparkReadProjection.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSparkReadProjection.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + ImmutableMap config = + ImmutableMap.of( + "type", "hive", +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +index 4f1cef5d3..f0f427732 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +@@ -136,6 +136,16 @@ public class TestSparkReaderDeletes extends DeleteReadTests { + .config("spark.ui.liveUpdate.period", 0) + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +@@ -204,7 +214,8 @@ public class TestSparkReaderDeletes extends DeleteReadTests { + } + + protected boolean countDeletes() { +- return true; ++ // TODO: Enable once iceberg-rust exposes delete count metrics to Comet ++ return false; + } + + @Override +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +index baf7fa8f8..150cc8969 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +@@ -182,6 +182,16 @@ public class TestSparkReaderWithBloomFilter { + SparkSession.builder() + .master("local[2]") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +index 17db46b85..d7e91b4ee 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +@@ -65,6 +65,16 @@ public class TestStructuredStreaming { + SparkSession.builder() + .master("local[2]") + .config("spark.sql.shuffle.partitions", 4) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +index 306444b9f..b661aa8ce 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +@@ -75,7 +75,20 @@ public class TestTimestampWithoutZone extends TestBase { + + @BeforeAll + public static void startSpark() { +- TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestTimestampWithoutZone.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +index 841268a6b..2fcc92bcb 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +@@ -80,7 +80,20 @@ public class TestWriteMetricsConfig { + + @BeforeAll + public static void startSpark() { +- TestWriteMetricsConfig.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestWriteMetricsConfig.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + TestWriteMetricsConfig.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +index 6e0925270..e0d743180 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +@@ -60,6 +60,16 @@ public class TestAggregatePushDown extends CatalogTestBase { + SparkSession.builder() + .master("local[2]") + .config("spark.sql.iceberg.aggregate_pushdown", "true") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +index 9d2ce2b38..5e2336884 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +@@ -598,9 +598,7 @@ public class TestFilterPushDown extends TestBaseWithCatalog { + String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); + + if (sparkFilter != null) { +- assertThat(planAsString) +- .as("Post scan filter should match") +- .contains("Filter (" + sparkFilter + ")"); ++ assertThat(planAsString).as("Post scan filter should match").contains("CometFilter"); + } else { + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java +index 6719c45ca..251545440 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java +@@ -616,7 +616,7 @@ public class TestStoragePartitionedJoins extends TestBaseWithCatalog { + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.%s = t2.%s " +- + "ORDER BY t1.id, t1.%s", ++ + "ORDER BY t1.id, t1.%s, t1.salary", + sourceColumnName, + tableName, + tableName(OTHER_TABLE_NAME), diff --git a/dev/diffs/iceberg-rust/1.9.1.diff b/dev/diffs/iceberg-rust/1.9.1.diff new file mode 100644 index 0000000000..128241b82f --- /dev/null +++ b/dev/diffs/iceberg-rust/1.9.1.diff @@ -0,0 +1,1891 @@ +diff --git a/build.gradle b/build.gradle +index 998f2ee9e..017e61be9 100644 +--- a/build.gradle ++++ b/build.gradle +@@ -814,6 +814,13 @@ project(':iceberg-parquet') { + implementation project(':iceberg-core') + implementation project(':iceberg-common') + ++ implementation("org.apache.datafusion:comet-spark-spark${sparkVersionsString}_${scalaVersion}:${libs.versions.comet.get()}") { ++ exclude group: 'org.apache.arrow' ++ exclude group: 'org.apache.parquet' ++ exclude group: 'org.apache.spark' ++ exclude group: 'org.apache.iceberg' ++ } ++ + implementation(libs.parquet.avro) { + exclude group: 'org.apache.avro', module: 'avro' + // already shaded by Parquet +diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml +index c50991c5f..3acb395a6 100644 +--- a/gradle/libs.versions.toml ++++ b/gradle/libs.versions.toml +@@ -36,6 +36,7 @@ awssdk-s3accessgrants = "2.3.0" + bson-ver = "4.11.5" + caffeine = "2.9.3" + calcite = "1.39.0" ++comet = "0.12.0-SNAPSHOT" + datasketches = "6.2.0" + delta-standalone = "3.3.1" + delta-spark = "3.3.1" +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java +new file mode 100644 +index 000000000..ddf6c7de5 +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.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.iceberg.parquet; ++ ++import java.util.Map; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.iceberg.relocated.com.google.common.collect.Maps; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.schema.LogicalTypeAnnotation; ++import org.apache.parquet.schema.PrimitiveType; ++import org.apache.parquet.schema.Type; ++import org.apache.parquet.schema.Types; ++ ++public class CometTypeUtils { ++ ++ private CometTypeUtils() {} ++ ++ public static ParquetColumnSpec descriptorToParquetColumnSpec(ColumnDescriptor descriptor) { ++ ++ String[] path = descriptor.getPath(); ++ PrimitiveType primitiveType = descriptor.getPrimitiveType(); ++ String physicalType = primitiveType.getPrimitiveTypeName().name(); ++ ++ int typeLength = ++ primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY ++ ? primitiveType.getTypeLength() ++ : 0; ++ ++ boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; ++ ++ // ToDo: extract this into a Util method ++ String logicalTypeName = null; ++ Map logicalTypeParams = Maps.newHashMap(); ++ LogicalTypeAnnotation logicalType = primitiveType.getLogicalTypeAnnotation(); ++ ++ if (logicalType != null) { ++ logicalTypeName = logicalType.getClass().getSimpleName(); ++ ++ // Handle specific logical types ++ if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal = ++ (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("precision", String.valueOf(decimal.getPrecision())); ++ logicalTypeParams.put("scale", String.valueOf(decimal.getScale())); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp = ++ (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(timestamp.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", timestamp.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimeLogicalTypeAnnotation time = ++ (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(time.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", time.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.IntLogicalTypeAnnotation intType = ++ (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned())); ++ logicalTypeParams.put("bitWidth", String.valueOf(intType.getBitWidth())); ++ } ++ } ++ ++ return new ParquetColumnSpec( ++ 1, // ToDo: pass in the correct id ++ path, ++ physicalType, ++ typeLength, ++ isRepeated, ++ descriptor.getMaxDefinitionLevel(), ++ descriptor.getMaxRepetitionLevel(), ++ logicalTypeName, ++ logicalTypeParams); ++ } ++ ++ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { ++ PrimitiveType.PrimitiveTypeName primType = ++ PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); ++ ++ Type.Repetition repetition; ++ if (columnSpec.getMaxRepetitionLevel() > 0) { ++ repetition = Type.Repetition.REPEATED; ++ } else if (columnSpec.getMaxDefinitionLevel() > 0) { ++ repetition = Type.Repetition.OPTIONAL; ++ } else { ++ repetition = Type.Repetition.REQUIRED; ++ } ++ ++ String name = columnSpec.getPath()[columnSpec.getPath().length - 1]; ++ // Reconstruct the logical type from parameters ++ LogicalTypeAnnotation logicalType = null; ++ if (columnSpec.getLogicalTypeName() != null) { ++ logicalType = ++ reconstructLogicalType( ++ columnSpec.getLogicalTypeName(), columnSpec.getLogicalTypeParams()); ++ } ++ ++ PrimitiveType primitiveType; ++ if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { ++ primitiveType = ++ org.apache.parquet.schema.Types.primitive(primType, repetition) ++ .length(columnSpec.getTypeLength()) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } else { ++ primitiveType = ++ Types.primitive(primType, repetition) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } ++ ++ return new ColumnDescriptor( ++ columnSpec.getPath(), ++ primitiveType, ++ columnSpec.getMaxRepetitionLevel(), ++ columnSpec.getMaxDefinitionLevel()); ++ } ++ ++ private static LogicalTypeAnnotation reconstructLogicalType( ++ String logicalTypeName, java.util.Map params) { ++ ++ switch (logicalTypeName) { ++ // MAP ++ case "MapLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.mapType(); ++ ++ // LIST ++ case "ListLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.listType(); ++ ++ // STRING ++ case "StringLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.stringType(); ++ ++ // MAP_KEY_VALUE ++ case "MapKeyValueLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(); ++ ++ // ENUM ++ case "EnumLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.enumType(); ++ ++ // DECIMAL ++ case "DecimalLogicalTypeAnnotation": ++ if (!params.containsKey("scale") || !params.containsKey("precision")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for DecimalLogicalTypeAnnotation: " + params); ++ } ++ int scale = Integer.parseInt(params.get("scale")); ++ int precision = Integer.parseInt(params.get("precision")); ++ return LogicalTypeAnnotation.decimalType(scale, precision); ++ ++ // DATE ++ case "DateLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.dateType(); ++ ++ // TIME ++ case "TimeLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimeLogicalTypeAnnotation: " + params); ++ } ++ ++ boolean isUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String timeUnitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit timeUnit; ++ switch (timeUnitStr) { ++ case "MILLIS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown time unit: " + timeUnitStr); ++ } ++ return LogicalTypeAnnotation.timeType(isUTC, timeUnit); ++ ++ // TIMESTAMP ++ case "TimestampLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimestampLogicalTypeAnnotation: " + params); ++ } ++ boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String unitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit unit; ++ switch (unitStr) { ++ case "MILLIS": ++ unit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ unit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ unit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown timestamp unit: " + unitStr); ++ } ++ return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); ++ ++ // INTEGER ++ case "IntLogicalTypeAnnotation": ++ if (!params.containsKey("isSigned") || !params.containsKey("bitWidth")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for IntLogicalTypeAnnotation: " + params); ++ } ++ boolean isSigned = Boolean.parseBoolean(params.get("isSigned")); ++ int bitWidth = Integer.parseInt(params.get("bitWidth")); ++ return LogicalTypeAnnotation.intType(bitWidth, isSigned); ++ ++ // JSON ++ case "JsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.jsonType(); ++ ++ // BSON ++ case "BsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.bsonType(); ++ ++ // UUID ++ case "UUIDLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.uuidType(); ++ ++ // INTERVAL ++ case "IntervalLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(); ++ ++ default: ++ throw new IllegalArgumentException("Unknown logical type: " + logicalTypeName); ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +new file mode 100644 +index 000000000..a3cba4018 +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +@@ -0,0 +1,260 @@ ++/* ++ * 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.iceberg.parquet; ++ ++import java.io.IOException; ++import java.io.UncheckedIOException; ++import java.nio.ByteBuffer; ++import java.util.List; ++import java.util.Map; ++import java.util.NoSuchElementException; ++import java.util.function.Function; ++import org.apache.comet.parquet.FileReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.ReadOptions; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.parquet.WrappedInputFile; ++import org.apache.hadoop.conf.Configuration; ++import org.apache.iceberg.Schema; ++import org.apache.iceberg.exceptions.RuntimeIOException; ++import org.apache.iceberg.expressions.Expression; ++import org.apache.iceberg.expressions.Expressions; ++import org.apache.iceberg.io.CloseableGroup; ++import org.apache.iceberg.io.CloseableIterable; ++import org.apache.iceberg.io.CloseableIterator; ++import org.apache.iceberg.io.InputFile; ++import org.apache.iceberg.mapping.NameMapping; ++import org.apache.iceberg.relocated.com.google.common.collect.Lists; ++import org.apache.iceberg.util.ByteBuffers; ++import org.apache.parquet.ParquetReadOptions; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; ++import org.apache.parquet.hadoop.metadata.ColumnPath; ++import org.apache.parquet.schema.MessageType; ++ ++public class CometVectorizedParquetReader extends CloseableGroup ++ implements CloseableIterable { ++ private final InputFile input; ++ private final ParquetReadOptions options; ++ private final Schema expectedSchema; ++ private final Function> batchReaderFunc; ++ private final Expression filter; ++ private final boolean reuseContainers; ++ private final boolean caseSensitive; ++ private final int batchSize; ++ private final NameMapping nameMapping; ++ private final Map properties; ++ private Long start = null; ++ private Long length = null; ++ private ByteBuffer fileEncryptionKey = null; ++ private ByteBuffer fileAADPrefix = null; ++ ++ public CometVectorizedParquetReader( ++ InputFile input, ++ Schema expectedSchema, ++ ParquetReadOptions options, ++ Function> readerFunc, ++ NameMapping nameMapping, ++ Expression filter, ++ boolean reuseContainers, ++ boolean caseSensitive, ++ int maxRecordsPerBatch, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.input = input; ++ this.expectedSchema = expectedSchema; ++ this.options = options; ++ this.batchReaderFunc = readerFunc; ++ // replace alwaysTrue with null to avoid extra work evaluating a trivial filter ++ this.filter = filter == Expressions.alwaysTrue() ? null : filter; ++ this.reuseContainers = reuseContainers; ++ this.caseSensitive = caseSensitive; ++ this.batchSize = maxRecordsPerBatch; ++ this.nameMapping = nameMapping; ++ this.properties = properties; ++ this.start = start; ++ this.length = length; ++ this.fileEncryptionKey = fileEncryptionKey; ++ this.fileAADPrefix = fileAADPrefix; ++ } ++ ++ private ReadConf conf = null; ++ ++ private ReadConf init() { ++ if (conf == null) { ++ ReadConf readConf = ++ new ReadConf( ++ input, ++ options, ++ expectedSchema, ++ filter, ++ null, ++ batchReaderFunc, ++ nameMapping, ++ reuseContainers, ++ caseSensitive, ++ batchSize); ++ this.conf = readConf.copy(); ++ return readConf; ++ } ++ return conf; ++ } ++ ++ @Override ++ public CloseableIterator iterator() { ++ FileIterator iter = ++ new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); ++ addCloseable(iter); ++ return iter; ++ } ++ ++ private static class FileIterator implements CloseableIterator { ++ // private final ParquetFileReader reader; ++ private final boolean[] shouldSkip; ++ private final VectorizedReader model; ++ private final long totalValues; ++ private final int batchSize; ++ private final List> columnChunkMetadata; ++ private final boolean reuseContainers; ++ private int nextRowGroup = 0; ++ private long nextRowGroupStart = 0; ++ private long valuesRead = 0; ++ private T last = null; ++ private final FileReader cometReader; ++ private ReadConf conf; ++ ++ FileIterator( ++ ReadConf conf, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.shouldSkip = conf.shouldSkip(); ++ this.totalValues = conf.totalValues(); ++ this.reuseContainers = conf.reuseContainers(); ++ this.model = conf.vectorizedModel(); ++ this.batchSize = conf.batchSize(); ++ this.model.setBatchSize(this.batchSize); ++ this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups(); ++ this.cometReader = ++ newCometReader( ++ conf.file(), ++ conf.projection(), ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ this.conf = conf; ++ } ++ ++ private FileReader newCometReader( ++ InputFile file, ++ MessageType projection, ++ Map properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ try { ++ ReadOptions cometOptions = ReadOptions.builder(new Configuration()).build(); ++ ++ FileReader fileReader = ++ new FileReader( ++ new WrappedInputFile(file), ++ cometOptions, ++ properties, ++ start, ++ length, ++ ByteBuffers.toByteArray(fileEncryptionKey), ++ ByteBuffers.toByteArray(fileAADPrefix)); ++ ++ List columnDescriptors = projection.getColumns(); ++ ++ List specs = Lists.newArrayList(); ++ ++ for (ColumnDescriptor descriptor : columnDescriptors) { ++ ParquetColumnSpec spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ specs.add(spec); ++ } ++ ++ fileReader.setRequestedSchemaFromSpecs(specs); ++ return fileReader; ++ } catch (IOException e) { ++ throw new UncheckedIOException("Failed to open Parquet file: " + file.location(), e); ++ } ++ } ++ ++ @Override ++ public boolean hasNext() { ++ return valuesRead < totalValues; ++ } ++ ++ @Override ++ public T next() { ++ if (!hasNext()) { ++ throw new NoSuchElementException(); ++ } ++ if (valuesRead >= nextRowGroupStart) { ++ advance(); ++ } ++ ++ // batchSize is an integer, so casting to integer is safe ++ int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, batchSize); ++ if (reuseContainers) { ++ this.last = model.read(last, numValuesToRead); ++ } else { ++ this.last = model.read(null, numValuesToRead); ++ } ++ valuesRead += numValuesToRead; ++ ++ return last; ++ } ++ ++ private void advance() { ++ while (shouldSkip[nextRowGroup]) { ++ nextRowGroup += 1; ++ cometReader.skipNextRowGroup(); ++ } ++ RowGroupReader pages; ++ try { ++ pages = cometReader.readNextRowGroup(); ++ } catch (IOException e) { ++ throw new RuntimeIOException(e); ++ } ++ ++ model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup)); ++ nextRowGroupStart += pages.getRowCount(); ++ nextRowGroup += 1; ++ } ++ ++ @Override ++ public void close() throws IOException { ++ model.close(); ++ cometReader.close(); ++ if (conf != null && conf.reader() != null) { ++ conf.reader().close(); ++ } ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +index 31f9e2a80..520f142c2 100644 +--- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +@@ -1124,6 +1124,7 @@ public class Parquet { + private NameMapping nameMapping = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; ++ private boolean isComet; + + private ReadBuilder(InputFile file) { + this.file = file; +@@ -1168,6 +1169,11 @@ public class Parquet { + return this; + } + ++ public ReadBuilder enableComet(boolean enableComet) { ++ this.isComet = enableComet; ++ return this; ++ } ++ + /** + * @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead + */ +@@ -1263,7 +1269,7 @@ public class Parquet { + } + + @Override +- @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) ++ @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity", "MethodLength"}) + public CloseableIterable build() { + FileDecryptionProperties fileDecryptionProperties = null; + if (fileEncryptionKey != null) { +@@ -1315,16 +1321,35 @@ public class Parquet { + } + + if (batchedReaderFunc != null) { +- return new VectorizedParquetReader<>( +- file, +- schema, +- options, +- batchedReaderFunc, +- mapping, +- filter, +- reuseContainers, +- caseSensitive, +- maxRecordsPerBatch); ++ if (isComet) { ++ LOG.info("Comet enabled"); ++ return new CometVectorizedParquetReader<>( ++ file, ++ schema, ++ options, ++ batchedReaderFunc, ++ mapping, ++ filter, ++ reuseContainers, ++ caseSensitive, ++ maxRecordsPerBatch, ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ } else { ++ return new VectorizedParquetReader<>( ++ file, ++ schema, ++ options, ++ batchedReaderFunc, ++ mapping, ++ filter, ++ reuseContainers, ++ caseSensitive, ++ maxRecordsPerBatch); ++ } + } else { + Function> readBuilder = + readerFuncWithSchema != null +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +index 1fb2372ba..142e5fbad 100644 +--- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +@@ -157,6 +157,14 @@ class ReadConf { + return newReader; + } + ++ InputFile file() { ++ return file; ++ } ++ ++ MessageType projection() { ++ return projection; ++ } ++ + ParquetValueReader model() { + return model; + } +diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle +index 572c32f92..d155f634a 100644 +--- a/spark/v3.5/build.gradle ++++ b/spark/v3.5/build.gradle +@@ -75,7 +75,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { + exclude group: 'org.roaringbitmap' + } + +- compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.5.0" ++ compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + implementation libs.parquet.column + implementation libs.parquet.hadoop +@@ -184,7 +184,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer + testImplementation libs.avro.avro + testImplementation libs.parquet.hadoop + testImplementation libs.awaitility +- testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.5.0" ++ testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + // Required because we remove antlr plugin dependencies from the compile configuration, see note above + runtimeOnly libs.antlr.runtime +@@ -265,6 +265,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') ++ integrationImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + + // runtime dependencies for running Hive Catalog based integration test + integrationRuntimeOnly project(':iceberg-hive-metastore') +@@ -302,8 +303,8 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'avro.shaded', 'org.apache.iceberg.shaded.org.apache.avro.shaded' + relocate 'com.thoughtworks.paranamer', 'org.apache.iceberg.shaded.com.thoughtworks.paranamer' +- relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' +- relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' ++// relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' ++// relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' +diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +index 578845e3d..4f44a73db 100644 +--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java ++++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +@@ -57,6 +57,16 @@ public abstract class ExtensionsTestBase extends CatalogTestBase { + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + .config( + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean())) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +index ecf9e6f8a..3475260ca 100644 +--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java ++++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +@@ -56,6 +56,16 @@ public class TestCallStatementParser { + .master("local[2]") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.extra.prop", "value") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + TestCallStatementParser.parser = spark.sessionState().sqlParser(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +index 64edb1002..0fc10120f 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +@@ -179,6 +179,16 @@ public class DeleteOrphanFilesBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", catalogWarehouse()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local"); + spark = builder.getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +index a5d0456b0..f0759f837 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +@@ -392,6 +392,16 @@ public class IcebergSortCompactionBenchmark { + "spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", getCatalogWarehouse()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]"); + spark = builder.getOrCreate(); + Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java +index c6794e43c..457d2823e 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java +@@ -239,6 +239,16 @@ public class DVReaderBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]") + .getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java +index ac74fb5a1..eab09293d 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java +@@ -223,6 +223,16 @@ public class DVWriterBenchmark { + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .master("local[*]") + .getOrCreate(); + } +diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java +index 68c537e34..1e9e90d53 100644 +--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java ++++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java +@@ -94,7 +94,19 @@ public abstract class IcebergSourceBenchmark { + } + + protected void setupSpark(boolean enableDictionaryEncoding) { +- SparkSession.Builder builder = SparkSession.builder().config("spark.ui.enabled", false); ++ SparkSession.Builder builder = ++ SparkSession.builder() ++ .config("spark.ui.enabled", false) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true"); + if (!enableDictionaryEncoding) { + builder + .config("parquet.dictionary.page.size", "1") +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +index 16159dcbd..eba1a2a0f 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +@@ -19,18 +19,22 @@ + package org.apache.iceberg.spark.data.vectorized; + + import java.io.IOException; ++import org.apache.comet.CometConf; ++import org.apache.comet.CometSchemaImporter; + import org.apache.comet.parquet.AbstractColumnReader; + import org.apache.comet.parquet.ColumnReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.RowGroupReader; + import org.apache.comet.parquet.TypeUtil; + import org.apache.comet.parquet.Utils; +-import org.apache.comet.shaded.arrow.c.CometSchemaImporter; + import org.apache.comet.shaded.arrow.memory.RootAllocator; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.parquet.VectorizedReader; + import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + import org.apache.iceberg.spark.SparkSchemaUtil; + import org.apache.iceberg.types.Types; + import org.apache.parquet.column.ColumnDescriptor; +-import org.apache.parquet.column.page.PageReader; ++import org.apache.spark.sql.internal.SQLConf; + import org.apache.spark.sql.types.DataType; + import org.apache.spark.sql.types.Metadata; + import org.apache.spark.sql.types.StructField; +@@ -42,23 +46,28 @@ class CometColumnReader implements VectorizedReader { + + private final ColumnDescriptor descriptor; + private final DataType sparkType; ++ private final int fieldId; + + // The delegated ColumnReader from Comet side + private AbstractColumnReader delegate; + private boolean initialized = false; + private int batchSize = DEFAULT_BATCH_SIZE; + private CometSchemaImporter importer; ++ private ParquetColumnSpec spec; + +- CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { ++ CometColumnReader(DataType sparkType, ColumnDescriptor descriptor, int fieldId) { + this.sparkType = sparkType; + this.descriptor = descriptor; ++ this.fieldId = fieldId; + } + + CometColumnReader(Types.NestedField field) { + DataType dataType = SparkSchemaUtil.convert(field.type()); + StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); + this.sparkType = dataType; +- this.descriptor = TypeUtil.convertToParquet(structField); ++ this.descriptor = ++ CometTypeUtils.buildColumnDescriptor(TypeUtil.convertToParquetSpec(structField)); ++ this.fieldId = field.fieldId(); + } + + public AbstractColumnReader delegate() { +@@ -92,7 +101,26 @@ class CometColumnReader implements VectorizedReader { + } + + this.importer = new CometSchemaImporter(new RootAllocator()); +- this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); ++ ++ spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ ++ boolean useLegacyTime = ++ Boolean.parseBoolean( ++ SQLConf.get() ++ .getConfString( ++ CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); ++ boolean useLazyMaterialization = ++ Boolean.parseBoolean( ++ SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "false")); ++ this.delegate = ++ Utils.getColumnReader( ++ sparkType, ++ spec, ++ importer, ++ batchSize, ++ true, // Comet sets this to true for native execution ++ useLazyMaterialization, ++ useLegacyTime); + this.initialized = true; + } + +@@ -111,9 +139,9 @@ class CometColumnReader implements VectorizedReader { + *

NOTE: this should be called before reading a new Parquet column chunk, and after {@link + * CometColumnReader#reset} is called. + */ +- public void setPageReader(PageReader pageReader) throws IOException { ++ public void setPageReader(RowGroupReader pageStore) throws IOException { + Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); +- ((ColumnReader) delegate).setPageReader(pageReader); ++ ((ColumnReader) delegate).setRowGroupReader(pageStore, spec); + } + + @Override +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +index 04ac69476..916face2b 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +@@ -22,8 +22,12 @@ import java.io.IOException; + import java.io.UncheckedIOException; + import java.util.List; + import java.util.Map; ++import org.apache.comet.CometRuntimeException; + import org.apache.comet.parquet.AbstractColumnReader; +-import org.apache.comet.parquet.BatchReader; ++import org.apache.comet.parquet.IcebergCometBatchReader; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.vector.CometSelectionVector; ++import org.apache.comet.vector.CometVector; + import org.apache.iceberg.Schema; + import org.apache.iceberg.data.DeleteFilter; + import org.apache.iceberg.parquet.VectorizedReader; +@@ -55,7 +59,7 @@ class CometColumnarBatchReader implements VectorizedReader { + // calling BatchReader.nextBatch, the isDeleted value is not yet available, so + // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is + // available. +- private final BatchReader delegate; ++ private final IcebergCometBatchReader delegate; + private DeleteFilter deletes = null; + private long rowStartPosInBatch = 0; + +@@ -65,9 +69,7 @@ class CometColumnarBatchReader implements VectorizedReader { + this.hasIsDeletedColumn = + readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); + +- AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; +- this.delegate = new BatchReader(abstractColumnReaders); +- delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); ++ this.delegate = new IcebergCometBatchReader(readers.size(), SparkSchemaUtil.convert(schema)); + } + + @Override +@@ -79,19 +81,22 @@ class CometColumnarBatchReader implements VectorizedReader { + && !(readers[i] instanceof CometPositionColumnReader) + && !(readers[i] instanceof CometDeleteColumnReader)) { + readers[i].reset(); +- readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); ++ readers[i].setPageReader((RowGroupReader) pageStore); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); + } + } + ++ AbstractColumnReader[] delegateReaders = new AbstractColumnReader[readers.length]; + for (int i = 0; i < readers.length; i++) { +- delegate.getColumnReaders()[i] = this.readers[i].delegate(); ++ delegateReaders[i] = readers[i].delegate(); + } + ++ delegate.init(delegateReaders); ++ + this.rowStartPosInBatch = +- pageStore ++ ((RowGroupReader) pageStore) + .getRowIndexOffset() + .orElseThrow( + () -> +@@ -148,9 +153,17 @@ class CometColumnarBatchReader implements VectorizedReader { + Pair pair = buildRowIdMapping(vectors); + if (pair != null) { + int[] rowIdMapping = pair.first(); +- numLiveRows = pair.second(); +- for (int i = 0; i < vectors.length; i++) { +- vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); ++ if (pair.second() != null) { ++ numLiveRows = pair.second(); ++ for (int i = 0; i < vectors.length; i++) { ++ if (vectors[i] instanceof CometVector) { ++ vectors[i] = ++ new CometSelectionVector((CometVector) vectors[i], rowIdMapping, numLiveRows); ++ } else { ++ throw new CometRuntimeException( ++ "Unsupported column vector type: " + vectors[i].getClass()); ++ } ++ } + } + } + } +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +index 047c96314..88d691a60 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +@@ -21,6 +21,7 @@ package org.apache.iceberg.spark.data.vectorized; + import java.math.BigDecimal; + import java.nio.ByteBuffer; + import org.apache.comet.parquet.ConstantColumnReader; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.types.Types; + import org.apache.spark.sql.types.DataType; + import org.apache.spark.sql.types.DataTypes; +@@ -34,7 +35,11 @@ class CometConstantColumnReader extends CometColumnReader { + super(field); + // use delegate to set constant value on the native side to be consumed by native execution. + setDelegate( +- new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); ++ new ConstantColumnReader( ++ sparkType(), ++ CometTypeUtils.descriptorToParquetColumnSpec(descriptor()), ++ convertToSparkValue(value), ++ false)); + } + + @Override +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +index 6235bfe48..cba108e43 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +@@ -51,10 +51,10 @@ class CometDeleteColumnReader extends CometColumnReader { + DeleteColumnReader() { + super( + DataTypes.BooleanType, +- TypeUtil.convertToParquet( ++ TypeUtil.convertToParquetSpec( + new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), + false /* useDecimal128 = false */, +- false /* isConstant = false */); ++ false /* isConstant */); + this.isDeleted = new boolean[0]; + } + +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +index bcc0e514c..98e80068c 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +@@ -20,6 +20,7 @@ package org.apache.iceberg.spark.data.vectorized; + + import org.apache.comet.parquet.MetadataColumnReader; + import org.apache.comet.parquet.Native; ++import org.apache.iceberg.parquet.CometTypeUtils; + import org.apache.iceberg.types.Types; + import org.apache.parquet.column.ColumnDescriptor; + import org.apache.spark.sql.types.DataTypes; +@@ -44,7 +45,7 @@ class CometPositionColumnReader extends CometColumnReader { + PositionColumnReader(ColumnDescriptor descriptor) { + super( + DataTypes.LongType, +- descriptor, ++ CometTypeUtils.descriptorToParquetColumnSpec(descriptor), + false /* useDecimal128 = false */, + false /* isConstant = false */); + } +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +index d36f1a727..56f8c9bff 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +@@ -142,6 +142,7 @@ class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor extends BaseReader taskGroup = (ScanTaskGroup) task; ++ return taskGroup.tasks().stream().allMatch(this::supportsCometBatchReads); ++ ++ } else if (task.isFileScanTask() && !task.isDataTask()) { ++ FileScanTask fileScanTask = task.asFileScanTask(); ++ // Comet can't handle delete files for now ++ return fileScanTask.file().format() == FileFormat.PARQUET; ++ ++ } else { ++ return false; ++ } ++ } ++ + // conditions for using ORC batch reads: + // - ORC vectorization is enabled + // - all tasks are of type FileScanTask and read only ORC files with no delete files +diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +index 106b296de..967b0d41d 100644 +--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java ++++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +@@ -24,6 +24,7 @@ import java.util.Map; + import java.util.Optional; + import java.util.function.Supplier; + import java.util.stream.Collectors; ++import org.apache.comet.parquet.SupportsComet; + import org.apache.iceberg.BlobMetadata; + import org.apache.iceberg.ScanTask; + import org.apache.iceberg.ScanTaskGroup; +@@ -95,7 +96,7 @@ import org.apache.spark.sql.types.StructType; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + +-abstract class SparkScan implements Scan, SupportsReportStatistics { ++abstract class SparkScan implements Scan, SupportsReportStatistics, SupportsComet { + private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; + +@@ -351,4 +352,10 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { + return splitSize; + } + } ++ ++ @Override ++ public boolean isCometEnabled() { ++ SparkBatch batch = (SparkBatch) this.toBatch(); ++ return batch.useCometBatchReads(); ++ } + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +index 404ba7284..00e97e96f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +@@ -90,6 +90,16 @@ public abstract class SparkDistributedDataScanTestBase + .master("local[2]") + .config("spark.serializer", serializer) + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +index 659507e4c..9076ec24d 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +@@ -73,6 +73,16 @@ public class TestSparkDistributedDataScanDeletes + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java +index a218f965e..eca0125ac 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java +@@ -62,6 +62,16 @@ public class TestSparkDistributedDataScanFilterFiles + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +index 2665d7ba8..2381d0aa1 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +@@ -63,6 +63,16 @@ public class TestSparkDistributedDataScanReporting + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +index 3e9f3334e..4401e8d0c 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +@@ -77,6 +77,17 @@ public abstract class TestBase extends SparkTestHelperBase { + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .config("spark.comet.exec.broadcastExchange.enabled", "false") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java +index bc4e722bc..3bbff053f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java +@@ -59,7 +59,20 @@ public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVect + + @BeforeAll + public static void startSpark() { +- spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +index 0886df957..b9d49d55c 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +@@ -57,7 +57,20 @@ public abstract class ScanTestBase extends AvroDataTest { + + @BeforeAll + public static void startSpark() { +- ScanTestBase.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ ScanTestBase.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + ScanTestBase.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +index 6b7d86136..c857dafce 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +@@ -144,7 +144,20 @@ public class TestCompressionSettings extends CatalogTestBase { + + @BeforeAll + public static void startSpark() { +- TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestCompressionSettings.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @BeforeEach +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +index c4ba96e63..98398968b 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +@@ -75,7 +75,20 @@ public class TestDataSourceOptions extends TestBaseWithCatalog { + + @BeforeAll + public static void startSpark() { +- TestDataSourceOptions.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestDataSourceOptions.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +index 348173596..9017b4a37 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +@@ -110,7 +110,20 @@ public class TestFilteredScan { + + @BeforeAll + public static void startSpark() { +- TestFilteredScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestFilteredScan.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +index 84c99a575..1d60b01df 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +@@ -93,7 +93,20 @@ public class TestForwardCompatibility { + + @BeforeAll + public static void startSpark() { +- TestForwardCompatibility.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestForwardCompatibility.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +index 7eff93d20..7aab7f968 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +@@ -46,7 +46,20 @@ public class TestIcebergSpark { + + @BeforeAll + public static void startSpark() { +- TestIcebergSpark.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestIcebergSpark.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +index 9464f687b..ca4e3e035 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +@@ -112,7 +112,20 @@ public class TestPartitionPruning { + + @BeforeAll + public static void startSpark() { +- TestPartitionPruning.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestPartitionPruning.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + TestPartitionPruning.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + String optionKey = String.format("fs.%s.impl", CountOpenLocalFileSystem.scheme); +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +index 5c218f21c..1d2222821 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +@@ -107,7 +107,20 @@ public class TestPartitionValues { + + @BeforeAll + public static void startSpark() { +- TestPartitionValues.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestPartitionValues.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +index a7334a580..ecd25670f 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +@@ -87,7 +87,20 @@ public class TestSnapshotSelection { + + @BeforeAll + public static void startSpark() { +- TestSnapshotSelection.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSnapshotSelection.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +index 182b1ef8f..205892e4a 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +@@ -120,7 +120,20 @@ public class TestSparkDataFile { + + @BeforeAll + public static void startSpark() { +- TestSparkDataFile.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSparkDataFile.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + TestSparkDataFile.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +index fb2b312be..3fca3330d 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +@@ -96,7 +96,20 @@ public class TestSparkDataWrite { + + @BeforeAll + public static void startSpark() { +- TestSparkDataWrite.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSparkDataWrite.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterEach +@@ -140,7 +153,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + for (ManifestFile manifest : +@@ -210,7 +223,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -256,7 +269,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -309,7 +322,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -352,7 +365,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -392,7 +405,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + +@@ -458,7 +471,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + } +@@ -622,7 +635,7 @@ public class TestSparkDataWrite { + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); + +@@ -708,7 +721,7 @@ public class TestSparkDataWrite { + // Since write and commit succeeded, the rows should be readable + Dataset result = spark.read().format("iceberg").load(targetLocation); + List actual = +- result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); ++ result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).as("Number of rows should match").hasSize(records.size() + records2.size()); + assertThat(actual) + .describedAs("Result rows should match") +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +index becf6a064..21701450a 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +@@ -83,7 +83,20 @@ public class TestSparkReadProjection extends TestReadProjection { + + @BeforeAll + public static void startSpark() { +- TestSparkReadProjection.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestSparkReadProjection.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + ImmutableMap config = + ImmutableMap.of( + "type", "hive", +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +index 4f1cef5d3..f0f427732 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +@@ -136,6 +136,16 @@ public class TestSparkReaderDeletes extends DeleteReadTests { + .config("spark.ui.liveUpdate.period", 0) + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +@@ -204,7 +214,8 @@ public class TestSparkReaderDeletes extends DeleteReadTests { + } + + protected boolean countDeletes() { +- return true; ++ // TODO: Enable once iceberg-rust exposes delete count metrics to Comet ++ return false; + } + + @Override +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +index baf7fa8f8..150cc8969 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +@@ -182,6 +182,16 @@ public class TestSparkReaderWithBloomFilter { + SparkSession.builder() + .master("local[2]") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +index c84a65cbe..6a6dd8a49 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +@@ -67,6 +67,16 @@ public class TestStructuredStreaming { + SparkSession.builder() + .master("local[2]") + .config("spark.sql.shuffle.partitions", 4) ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .getOrCreate(); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +index 306444b9f..b661aa8ce 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +@@ -75,7 +75,20 @@ public class TestTimestampWithoutZone extends TestBase { + + @BeforeAll + public static void startSpark() { +- TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestTimestampWithoutZone.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + } + + @AfterAll +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +index 841268a6b..2fcc92bcb 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +@@ -80,7 +80,20 @@ public class TestWriteMetricsConfig { + + @BeforeAll + public static void startSpark() { +- TestWriteMetricsConfig.spark = SparkSession.builder().master("local[2]").getOrCreate(); ++ TestWriteMetricsConfig.spark = ++ SparkSession.builder() ++ .master("local[2]") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") ++ .getOrCreate(); + TestWriteMetricsConfig.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +index 6e0925270..e0d743180 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +@@ -60,6 +60,16 @@ public class TestAggregatePushDown extends CatalogTestBase { + SparkSession.builder() + .master("local[2]") + .config("spark.sql.iceberg.aggregate_pushdown", "true") ++ .config("spark.plugins", "org.apache.spark.CometPlugin") ++ .config( ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .config("spark.comet.explainFallback.enabled", "true") ++ .config("spark.comet.scan.icebergNative.enabled", "true") ++ .config("spark.memory.offHeap.enabled", "true") ++ .config("spark.memory.offHeap.size", "10g") ++ .config("spark.comet.use.lazyMaterialization", "false") ++ .config("spark.comet.schemaEvolution.enabled", "true") + .enableHiveSupport() + .getOrCreate(); + +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +index 9d2ce2b38..5e2336884 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +@@ -598,9 +598,7 @@ public class TestFilterPushDown extends TestBaseWithCatalog { + String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); + + if (sparkFilter != null) { +- assertThat(planAsString) +- .as("Post scan filter should match") +- .contains("Filter (" + sparkFilter + ")"); ++ assertThat(planAsString).as("Post scan filter should match").contains("CometFilter"); + } else { + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); + } +diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java +index 6719c45ca..251545440 100644 +--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java ++++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java +@@ -616,7 +616,7 @@ public class TestStoragePartitionedJoins extends TestBaseWithCatalog { + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.%s = t2.%s " +- + "ORDER BY t1.id, t1.%s", ++ + "ORDER BY t1.id, t1.%s, t1.salary", + sourceColumnName, + tableName, + tableName(OTHER_TABLE_NAME), diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 90eb886009..1e77032f7d 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -30,6 +30,7 @@ Comet provides the following configuration settings. |--------|-------------|---------------| | `spark.comet.scan.allowIncompatible` | Some Comet scan implementations are not currently fully compatible with Spark for all datatypes. Set this config to true to allow them anyway. For more information, refer to the [Comet Compatibility Guide](https://datafusion.apache.org/comet/user-guide/compatibility.html). | false | | `spark.comet.scan.enabled` | Whether to enable native scans. When this is turned on, Spark will use Comet to read supported data sources (currently only Parquet is supported natively). Note that to enable native vectorized execution, both this config and `spark.comet.exec.enabled` need to be enabled. | true | +| `spark.comet.scan.icebergNative.enabled` | Whether to enable native Iceberg table scan using iceberg-rust. When enabled, Iceberg tables are read directly through native execution, bypassing Spark's DataSource V2 API for better performance. | false | | `spark.comet.scan.preFetch.enabled` | Whether to enable pre-fetching feature of CometScan. | false | | `spark.comet.scan.preFetch.threadNum` | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | | `spark.hadoop.fs.comet.libhdfs.schemes` | Defines filesystem schemes (e.g., hdfs, webhdfs) that the native side accesses via libhdfs, separated by commas. Valid only when built with hdfs feature enabled. | | diff --git a/native/Cargo.lock b/native/Cargo.lock index fa973f0e4f..f75e180498 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -17,6 +17,17 @@ version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" +[[package]] +name = "ahash" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" +dependencies = [ + "getrandom 0.2.16", + "once_cell", + "version_check", +] + [[package]] name = "ahash" version = "0.8.12" @@ -33,9 +44,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.1.3" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +checksum = "ddd31a130427c27518df266943a5308ed92d4b226cc639f5a8f1002816174301" dependencies = [ "memchr", ] @@ -97,12 +108,43 @@ version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" +[[package]] +name = "apache-avro" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a033b4ced7c585199fb78ef50fca7fe2f444369ec48080c5fd072efa1a03cc7" +dependencies = [ + "bigdecimal", + "bon", + "digest", + "log", + "miniz_oxide", + "num-bigint", + "quad-rand", + "rand 0.9.2", + "regex-lite", + "serde", + "serde_bytes", + "serde_json", + "strum 0.27.2", + "strum_macros 0.27.2", + "thiserror 2.0.17", + "uuid", + "zstd", +] + [[package]] name = "arc-swap" version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" +[[package]] +name = "array-init" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc" + [[package]] name = "arrayref" version = "0.3.9" @@ -156,7 +198,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-buffer", "arrow-data", "arrow-schema", @@ -254,7 +296,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.12.0", "lexical-core", "memchr", "num", @@ -306,7 +348,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-data", @@ -331,6 +373,12 @@ dependencies = [ "regex-syntax", ] +[[package]] +name = "as-any" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0f477b951e452a0b6b4a10b53ccd569042d1d01729b519e02074a9c0958a063" + [[package]] name = "assertables" version = "9.8.2" @@ -360,6 +408,23 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "async-compression" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" +dependencies = [ + "bzip2 0.5.2", + "flate2", + "futures-core", + "memchr", + "pin-project-lite", + "tokio", + "xz2", + "zstd", + "zstd-safe", +] + [[package]] name = "async-executor" version = "1.13.3" @@ -458,7 +523,7 @@ checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -526,9 +591,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.14.1" +version = "1.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879b6c89592deb404ba4dc0ae6b58ffd1795c78991cbb5b8bc441c48a070440d" +checksum = "5932a7d9d28b0d2ea34c6b3779d35e3dd6f6345317c34e73438c4f1f29144151" dependencies = [ "aws-lc-sys", "zeroize", @@ -536,9 +601,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.32.3" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "107a4e9d9cab9963e04e84bb8dee0e25f2a987f9a8bad5ed054abd439caa8f8c" +checksum = "1826f2e4cfc2cd19ee53c42fbf68e2f81ec21108e0b7ecf6a71cf062137360fc" dependencies = [ "bindgen 0.72.1", "cc", @@ -549,9 +614,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.13" +version = "1.5.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f2402da1a5e16868ba98725e5d73f26b8116eaa892e56f2cd0bf5eec7985f70" +checksum = "8fe0fd441565b0b318c76e7206c8d1d0b0166b3e986cf30e890b61feb6192045" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -573,9 +638,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.88.0" +version = "1.89.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d05b276777560aa9a196dbba2e3aada4d8006d3d7eeb3ba7fe0c317227d933c4" +checksum = "a9c1b1af02288f729e95b72bd17988c009aa72e26dcb59b3200f86d7aea726c9" dependencies = [ "aws-credential-types", "aws-runtime", @@ -595,9 +660,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.90.0" +version = "1.91.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9be14d6d9cd761fac3fd234a0f47f7ed6c0df62d83c0eeb7012750e4732879b" +checksum = "4e8122301558dc7c6c68e878af918880b82ff41897a60c8c4e18e4dc4d93e9f1" dependencies = [ "aws-credential-types", "aws-runtime", @@ -617,9 +682,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.90.0" +version = "1.93.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98a862d704c817d865c8740b62d8bbeb5adcb30965e93b471df8a5bcefa20a80" +checksum = "b745a6dca6bde63e23fcd3bfbb33479ab61bf40033cdf2e4a80e7a8735b7413d" dependencies = [ "aws-credential-types", "aws-runtime", @@ -884,8 +949,15 @@ dependencies = [ "num-bigint", "num-integer", "num-traits", + "serde", ] +[[package]] +name = "bimap" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "230c5f1ca6a325a32553f8640d31ac9b49f2411e901e427570154868b46da4f7" + [[package]] name = "bindgen" version = "0.64.0" @@ -925,7 +997,7 @@ dependencies = [ "regex", "rustc-hash 2.1.1", "shlex", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -940,6 +1012,18 @@ version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" +[[package]] +name = "bitvec" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" +dependencies = [ + "funty", + "radium", + "tap", + "wyz", +] + [[package]] name = "blake2" version = "0.10.6" @@ -984,6 +1068,54 @@ dependencies = [ "piper", ] +[[package]] +name = "bon" +version = "3.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebeb9aaf9329dff6ceb65c689ca3db33dbf15f324909c60e4e5eef5701ce31b1" +dependencies = [ + "bon-macros", + "rustversion", +] + +[[package]] +name = "bon-macros" +version = "3.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77e9d642a7e3a318e37c2c9427b5a6a48aa1ad55dcd986f3034ab2239045a645" +dependencies = [ + "darling 0.21.3", + "ident_case", + "prettyplease", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.110", +] + +[[package]] +name = "borsh" +version = "1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad8646f98db542e39fc66e68a20b2144f6a732636df7c2354e74645faaa433ce" +dependencies = [ + "borsh-derive", + "cfg_aliases", +] + +[[package]] +name = "borsh-derive" +version = "1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdd1d3c0c2f5833f22386f252fe8ed005c7f59fdcddeef025c01b4c3b9fd9ac3" +dependencies = [ + "once_cell", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.110", +] + [[package]] name = "brotli" version = "8.0.2" @@ -1011,6 +1143,28 @@ version = "3.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "46c5e41b57b8bba42a04676d81cb89e9ee8e859a1a66f80a5a72e1cb76b34d43" +[[package]] +name = "bytecheck" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23cdc57ce23ac53c931e88a43d06d070a6fd142f2617be5855eb75efc9beb1c2" +dependencies = [ + "bytecheck_derive", + "ptr_meta", + "simdutf8", +] + +[[package]] +name = "bytecheck_derive" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3db406d29fbcd95542e92559bed4d8ad92636d1ca8b3b72ede10b4bcc010e659" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "bytemuck" version = "1.24.0" @@ -1025,9 +1179,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.10.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" +checksum = "b35204fbdc0b3f4446b89fc1ac2cf84a8a68971995d0bf2e925ec7cd960f9cb3" [[package]] name = "bytes-utils" @@ -1039,6 +1193,34 @@ dependencies = [ "either", ] +[[package]] +name = "bzip2" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49ecfb22d906f800d4fe833b6282cf4dc1c298f5057ca0b5445e5c209735ca47" +dependencies = [ + "bzip2-sys", +] + +[[package]] +name = "bzip2" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a53fac24f34a81bc9954b5d6cfce0c21e18ec6959f44f56e8e90e4bb7c346c" +dependencies = [ + "libbz2-rs-sys", +] + +[[package]] +name = "bzip2-sys" +version = "0.1.13+1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225bff33b2141874fe80d71e07d6eec4f85c5c216453dd96388240f96e1acc14" +dependencies = [ + "cc", + "pkg-config", +] + [[package]] name = "cast" version = "0.3.0" @@ -1047,9 +1229,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.45" +version = "1.2.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35900b6c8d709fb1d854671ae27aeaa9eec2f8b01b364e1619a40da3e6fe2afe" +checksum = "b97463e1064cb1b1c1384ad0a0b9c8abd0988e2a91f52606c80ef14aadb63e36" dependencies = [ "find-msvc-tools", "jobserver", @@ -1091,8 +1273,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "145052bdd345b87320e369255277e3fb5152762ad123a901ef5c262dd38fe8d2" dependencies = [ "iana-time-zone", + "js-sys", "num-traits", "serde", + "wasm-bindgen", "windows-link", ] @@ -1146,18 +1330,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.50" +version = "4.5.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2cfd7bf8a6017ddaa4e32ffe7403d547790db06bd171c1c53926faab501623" +checksum = "c9e340e012a1bf4935f5282ed1436d1489548e8f72308207ea5df0e23d2d03f8" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.50" +version = "4.5.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4c05b9e80c5ccd3a7ef080ad7b6ba7d6fc00a985b8b157197075677c82c7a0" +checksum = "d76b5d13eaa18c901fd2f7fca939fefe3a0727a953561fefdf3b2922b8569d00" dependencies = [ "anstyle", "clap_lex", @@ -1194,8 +1378,8 @@ version = "7.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e0d05af1e006a2407bedef5af410552494ce5be9090444dbbcb57258c1af3d56" dependencies = [ - "strum", - "strum_macros", + "strum 0.26.3", + "strum_macros 0.26.4", "unicode-width", ] @@ -1208,6 +1392,12 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "const-oid" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" + [[package]] name = "const-random" version = "0.1.18" @@ -1268,6 +1458,15 @@ dependencies = [ "libc", ] +[[package]] +name = "crc32c" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a47af21622d091a8f0fb295b88bc886ac74efcc613efc19f5d0b21de5c89e47" +dependencies = [ + "rustc_version", +] + [[package]] name = "crc32fast" version = "1.5.0" @@ -1312,6 +1511,15 @@ dependencies = [ "itertools 0.13.0", ] +[[package]] +name = "crossbeam-channel" +version = "0.5.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82b8f8f868b36967f9606790d1903570de9ceaf870a7bf9fbbd3016d636a2cb2" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-deque" version = "0.8.6" @@ -1345,9 +1553,9 @@ checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" [[package]] name = "crypto-common" -version = "0.1.6" +version = "0.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +checksum = "78c8292055d1c1df0cce5d180393dc8cce0abec0a7102adb6c7b1eef6016d60a" dependencies = [ "generic-array", "typenum", @@ -1374,6 +1582,76 @@ dependencies = [ "memchr", ] +[[package]] +name = "darling" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc7f46116c46ff9ab3eb1597a45688b6715c6e628b5c133e288e709a29bcb4ee" +dependencies = [ + "darling_core 0.20.11", + "darling_macro 0.20.11", +] + +[[package]] +name = "darling" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cdf337090841a411e2a7f3deb9187445851f91b309c0c0a29e05f74a00a48c0" +dependencies = [ + "darling_core 0.21.3", + "darling_macro 0.21.3", +] + +[[package]] +name = "darling_core" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d00b9596d185e565c2207a0b01f8bd1a135483d02d9b7b0a54b11da8d53412e" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.110", +] + +[[package]] +name = "darling_core" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1247195ecd7e3c85f83c8d2a366e4210d588e802133e1e355180a9870b517ea4" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.110", +] + +[[package]] +name = "darling_macro" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" +dependencies = [ + "darling_core 0.20.11", + "quote", + "syn 2.0.110", +] + +[[package]] +name = "darling_macro" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" +dependencies = [ + "darling_core 0.21.3", + "quote", + "syn 2.0.110", +] + [[package]] name = "dashmap" version = "6.1.0" @@ -1431,7 +1709,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "regex", "sqlparser", "tempfile", @@ -1505,12 +1783,14 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", + "datafusion-datasource", "datafusion-functions-nested", "datafusion-spark", "futures", "hdfs-sys", "hdrs", "hex", + "iceberg", "itertools 0.14.0", "jni", "lazy_static", @@ -1529,9 +1809,10 @@ dependencies = [ "pprof", "procfs", "prost", - "rand", + "rand 0.9.2", "regex", "reqwest", + "serde_json", "simd-adler32", "snap", "tempfile", @@ -1594,7 +1875,7 @@ dependencies = [ "futures", "hex", "num", - "rand", + "rand 0.9.2", "regex", "thiserror 2.0.17", "tokio", @@ -1607,7 +1888,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5a83760d9a13122d025fbdb1d5d5aaf93dd9ada5e90ea229add92aa30898b2d1" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "arrow-ipc", "base64", @@ -1615,7 +1896,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "hex", - "indexmap", + "indexmap 2.12.0", "libc", "log", "object_store", @@ -1644,8 +1925,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" dependencies = [ "arrow", + "async-compression", "async-trait", "bytes", + "bzip2 0.6.1", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -1656,16 +1939,20 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", + "flate2", "futures", "glob", "itertools 0.14.0", "log", "object_store", "parquet", - "rand", + "rand 0.9.2", "tempfile", "tokio", + "tokio-util", "url", + "xz2", + "zstd", ] [[package]] @@ -1748,7 +2035,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "tokio", ] @@ -1774,7 +2061,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "tempfile", "url", ] @@ -1794,7 +2081,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap", + "indexmap 2.12.0", "paste", "serde_json", "sqlparser", @@ -1808,7 +2095,7 @@ checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" dependencies = [ "arrow", "datafusion-common", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "paste", ] @@ -1835,7 +2122,7 @@ dependencies = [ "itertools 0.14.0", "log", "md-5", - "rand", + "rand 0.9.2", "regex", "sha2", "unicode-segmentation", @@ -1848,7 +2135,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07331fc13603a9da97b74fd8a273f4238222943dffdbbed1c4c6f862a30105bf" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-doc", @@ -1869,7 +2156,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b5951e572a8610b89968a09b5420515a121fbc305c0258651f318dc07c97ab17" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -1950,7 +2237,7 @@ checksum = "ec6f637bce95efac05cdfb9b6c19579ed4aa5f6b94d951cfa5bb054b7bb4f730" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -1965,7 +2252,7 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-physical-expr", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "log", "regex", @@ -1978,7 +2265,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c8668103361a272cbbe3a61f72eca60c9b7c706e87cc3565bcf21e2b277b84f6" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr", @@ -1987,7 +2274,7 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2016,7 +2303,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6652fe7b5bf87e85ed175f571745305565da2c0b599d98e697bcbedc7baa47c3" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2049,7 +2336,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2f7f778a1a838dec124efb96eae6144237d546945587557c9e6936b3414558c" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "arrow-ord", "arrow-schema", @@ -2066,7 +2353,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2147,7 +2434,7 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap", + "indexmap 2.12.0", "log", "regex", "sqlparser", @@ -2164,11 +2451,43 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" +checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" dependencies = [ "powerfmt", + "serde_core", +] + +[[package]] +name = "derive_builder" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "507dfb09ea8b7fa618fcf76e953f4f5e192547945816d5358edffe39f6f94947" +dependencies = [ + "derive_builder_macro", +] + +[[package]] +name = "derive_builder_core" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d5bcf7b024d6835cfb3d473887cd966994907effbe9227e8c8219824d06c4e8" +dependencies = [ + "darling 0.20.11", + "proc-macro2", + "quote", + "syn 2.0.110", +] + +[[package]] +name = "derive_builder_macro" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" +dependencies = [ + "derive_builder_core", + "syn 2.0.110", ] [[package]] @@ -2188,7 +2507,7 @@ checksum = "bda628edc44c4bb645fbe0f758797143e4e07926f7ebf4e9bdfbd3d2ce621df3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", "unicode-xid", ] @@ -2205,6 +2524,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", + "const-oid", "crypto-common", "subtle", ] @@ -2217,7 +2537,22 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", +] + +[[package]] +name = "dissimilar" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8975ffdaa0ef3661bfe02dbdcc06c9f829dfafe6a3c474de366a8d5e44276921" + +[[package]] +name = "dlv-list" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "442039f5147480ba31067cb00ada1adae6892028e40e45fc5de7b7df6dcc1b5f" +dependencies = [ + "const-random", ] [[package]] @@ -2226,6 +2561,12 @@ version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" +[[package]] +name = "dyn-clone" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0881ea181b1df73ff77ffaaf9c7544ecc11e82fba9b5f27b262a3c73a332555" + [[package]] name = "either" version = "1.15.0" @@ -2249,7 +2590,7 @@ checksum = "44f23cf4b44bfce11a86ace86f8a73ffdec849c9fd00a386a53d278bd9e81fb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -2295,6 +2636,16 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "expect-test" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63af43ff4431e848fb47472a920f14fa71c24de13255a5692e93d4e90302acb0" +dependencies = [ + "dissimilar", + "once_cell", +] + [[package]] name = "fastrand" version = "2.3.0" @@ -2303,9 +2654,9 @@ checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "find-msvc-tools" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52051878f80a721bb68ebfbc930e07b65ba72f2da88968ea5c06fd6ca3d3a127" +checksum = "3a3076410a55c90011c298b04d0cfa770b00fa04e1e3c97d3f6c9de105a03844" [[package]] name = "findshlibs" @@ -2401,6 +2752,12 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" +[[package]] +name = "funty" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" + [[package]] name = "futures" version = "0.3.31" @@ -2470,7 +2827,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -2505,9 +2862,9 @@ dependencies = [ [[package]] name = "generic-array" -version = "0.14.9" +version = "0.14.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bb6743198531e02858aeaea5398fcc883e71851fcbcb5a2f773e2fb6cb1edf2" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" dependencies = [ "typenum", "version_check", @@ -2576,7 +2933,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap", + "indexmap 2.12.0", "slab", "tokio", "tokio-util", @@ -2595,13 +2952,22 @@ dependencies = [ "zerocopy", ] +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash 0.7.8", +] + [[package]] name = "hashbrown" version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" dependencies = [ - "ahash", + "ahash 0.8.12", "allocator-api2", ] @@ -2750,9 +3116,9 @@ checksum = "135b12329e5e3ce057a9f972339ea52bc954fe1e9358ef27f95e89716fbc5424" [[package]] name = "hyper" -version = "1.7.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3aa54a13a0dfe7fbe3a59e0c76093041720fdc77b110cc0fc260fafb4dc51e" +checksum = "2ab2d4f250c3d7b1c9fcdff1cece94ea4e2dfbec68614f7b87cb205f24ca9d11" dependencies = [ "atomic-waker", "bytes", @@ -2790,9 +3156,9 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.17" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c6995591a8f1380fcb4ba966a252a4b29188d51d2b89e3a252f5305be65aea8" +checksum = "52e9a2a24dc5c6821e71a7030e1e14b7b632acac55c40e9d2e082c621261bb56" dependencies = [ "base64", "bytes", @@ -2837,38 +3203,93 @@ dependencies = [ ] [[package]] -name = "icu_collections" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" -dependencies = [ - "displaydoc", - "potential_utf", - "yoke", - "zerofrom", - "zerovec", -] - -[[package]] -name = "icu_locale_core" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +name = "iceberg" +version = "0.7.0" +source = "git+https://github.com/apache/iceberg-rust?rev=a667539#a6675394c0b15f90e70b907b0742d5be6ac39a3e" dependencies = [ - "displaydoc", - "litemap", - "tinystr", - "writeable", + "anyhow", + "apache-avro", + "array-init", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-ord", + "arrow-schema", + "arrow-select", + "arrow-string", + "as-any", + "async-trait", + "backon", + "base64", + "bimap", + "bytes", + "chrono", + "derive_builder", + "expect-test", + "flate2", + "fnv", + "futures", + "itertools 0.13.0", + "moka", + "murmur3", + "num-bigint", + "once_cell", + "opendal", + "ordered-float 4.6.0", + "parquet", + "rand 0.8.5", + "reqsign", + "reqwest", + "roaring", + "rust_decimal", + "serde", + "serde_bytes", + "serde_derive", + "serde_json", + "serde_repr", + "serde_with", + "strum 0.27.2", + "thrift", + "tokio", + "typed-builder", + "url", + "uuid", + "zstd", +] + +[[package]] +name = "icu_collections" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c6b649701667bbe825c3b7e6388cb521c23d88644678e83c0c4d0a621a34b43" +dependencies = [ + "displaydoc", + "potential_utf", + "yoke", + "zerofrom", "zerovec", ] [[package]] -name = "icu_normalizer" -version = "2.0.0" +name = "icu_locale_core" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "edba7861004dd3714265b4db54a3c390e880ab658fec5f7db895fae2046b5bb6" dependencies = [ "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_normalizer" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f6c8828b67bf8908d82127b2054ea1b4427ff0230ee9141c54251934ab1b599" +dependencies = [ "icu_collections", "icu_normalizer_data", "icu_properties", @@ -2879,42 +3300,38 @@ dependencies = [ [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "7aedcccd01fc5fe81e6b489c15b247b8b0690feb23304303a9e560f37efc560a" [[package]] name = "icu_properties" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "e93fcd3157766c0c8da2f8cff6ce651a31f0810eaa1c51ec363ef790bbb5fb99" dependencies = [ - "displaydoc", "icu_collections", "icu_locale_core", "icu_properties_data", "icu_provider", - "potential_utf", "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "02845b3647bb045f1100ecd6480ff52f34c35f82d9880e029d329c21d1054899" [[package]] name = "icu_provider" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "85962cf0ce02e1e0a629cc34e7ca3e373ce20dda4c4d7294bbd0bf1fdb59e614" dependencies = [ "displaydoc", "icu_locale_core", - "stable_deref_trait", - "tinystr", "writeable", "yoke", "zerofrom", @@ -2922,6 +3339,12 @@ dependencies = [ "zerovec", ] +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + [[package]] name = "idna" version = "1.1.0" @@ -2943,6 +3366,17 @@ dependencies = [ "icu_properties", ] +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", + "serde", +] + [[package]] name = "indexmap" version = "2.12.0" @@ -2951,6 +3385,8 @@ checksum = "6717a8d2a5a929a1a2eb43a12812498ed141a0bcfb7e8f7844fbdbe4303bba9f" dependencies = [ "equivalent", "hashbrown 0.16.0", + "serde", + "serde_core", ] [[package]] @@ -2959,8 +3395,8 @@ version = "0.11.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ - "ahash", - "indexmap", + "ahash 0.8.12", + "indexmap 2.12.0", "is-terminal", "itoa", "log", @@ -2985,9 +3421,9 @@ checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" [[package]] name = "iri-string" -version = "0.7.8" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbc5ebe9c3a1a7a5127f920a418f7585e9e758e911d0466ed004f393b0e380b2" +checksum = "4f867b9d1d896b67beb18518eda36fdb77a32ea590de864f1325b294a6d14397" dependencies = [ "memchr", "serde", @@ -3073,9 +3509,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" +checksum = "b011eec8cc36da2aab2d5cff675ec18454fad408585853910a202391cf9f8e65" dependencies = [ "once_cell", "wasm-bindgen", @@ -3159,6 +3595,12 @@ dependencies = [ "lexical-util", ] +[[package]] +name = "libbz2-rs-sys" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" + [[package]] name = "libc" version = "0.2.177" @@ -3224,9 +3666,9 @@ checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "6373607a59f0be73a39b6fe456b8192fcc3585f602af20751600e974dd455e77" [[package]] name = "lock_api" @@ -3270,7 +3712,7 @@ dependencies = [ "log-mdc", "mock_instant", "parking_lot", - "rand", + "rand 0.9.2", "serde", "serde-value", "serde_json", @@ -3297,6 +3739,17 @@ dependencies = [ "twox-hash", ] +[[package]] +name = "lzma-sys" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + [[package]] name = "md-5" version = "0.10.6" @@ -3364,12 +3817,39 @@ version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dce6dd36094cac388f119d2e9dc82dc730ef91c32a6222170d630e5414b956e6" +[[package]] +name = "moka" +version = "0.12.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8261cd88c312e0004c1d51baad2980c66528dfdb2bee62003e643a4d8f86b077" +dependencies = [ + "async-lock", + "crossbeam-channel", + "crossbeam-epoch", + "crossbeam-utils", + "equivalent", + "event-listener 5.4.1", + "futures-util", + "parking_lot", + "portable-atomic", + "rustc_version", + "smallvec", + "tagptr", + "uuid", +] + [[package]] name = "multimap" version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" +[[package]] +name = "murmur3" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9252111cf132ba0929b6f8e030cac2a24b507f3a4d6db6fb2896f27b354c714b" + [[package]] name = "nix" version = "0.26.4" @@ -3413,6 +3893,7 @@ checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", + "serde", ] [[package]] @@ -3481,6 +3962,16 @@ dependencies = [ "libm", ] +[[package]] +name = "num_cpus" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91df4bbde75afed763b708b7eee1e8e7651e02d97f6d5dd763e89367e957b23b" +dependencies = [ + "hermit-abi", + "libc", +] + [[package]] name = "object" version = "0.37.3" @@ -3511,8 +4002,8 @@ dependencies = [ "md-5", "parking_lot", "percent-encoding", - "quick-xml 0.38.3", - "rand", + "quick-xml 0.38.4", + "rand 0.9.2", "reqwest", "ring", "rustls-pemfile", @@ -3566,6 +4057,7 @@ dependencies = [ "base64", "bytes", "chrono", + "crc32c", "futures", "getrandom 0.2.16", "hdrs", @@ -3574,7 +4066,8 @@ dependencies = [ "log", "md-5", "percent-encoding", - "quick-xml 0.38.3", + "quick-xml 0.38.4", + "reqsign", "reqwest", "serde", "serde_json", @@ -3597,6 +4090,25 @@ dependencies = [ "num-traits", ] +[[package]] +name = "ordered-float" +version = "4.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7bb71e1b3fa6ca1c61f383464aaf2bb0e2f8e772a1f01d486832464de363b951" +dependencies = [ + "num-traits", +] + +[[package]] +name = "ordered-multimap" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49203cdcae0030493bad186b28da2fa25645fa276a51b6fec8010d281e02ef79" +dependencies = [ + "dlv-list", + "hashbrown 0.14.5", +] + [[package]] name = "outref" version = "0.5.2" @@ -3638,7 +4150,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-cast", @@ -3681,7 +4193,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.12.0", "simdutf8", "uuid", ] @@ -3739,7 +4251,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ "fixedbitset", - "indexmap", + "indexmap 2.12.0", ] [[package]] @@ -3750,7 +4262,7 @@ checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap", + "indexmap 2.12.0", "serde", ] @@ -3789,7 +4301,7 @@ checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -3863,11 +4375,17 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "portable-atomic" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" + [[package]] name = "potential_utf" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" +checksum = "b73949432f5e2a09657003c25bca5e19a0e9c84f8058ca374f49e0ebe605af77" dependencies = [ "zerovec", ] @@ -3916,7 +4434,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" dependencies = [ "proc-macro2", - "syn 2.0.108", + "syn 2.0.110", +] + +[[package]] +name = "proc-macro-crate" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" +dependencies = [ + "toml_edit", ] [[package]] @@ -3978,7 +4505,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.108", + "syn 2.0.110", "tempfile", ] @@ -3992,7 +4519,7 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4004,6 +4531,32 @@ dependencies = [ "prost", ] +[[package]] +name = "ptr_meta" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0738ccf7ea06b608c10564b31debd4f5bc5e197fc8bfe088f68ae5ce81e7a4f1" +dependencies = [ + "ptr_meta_derive", +] + +[[package]] +name = "ptr_meta_derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b845dbfca988fa33db069c0e230574d15a3088f147a87b64c7589eb662c9ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "quad-rand" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" + [[package]] name = "quick-xml" version = "0.26.0" @@ -4015,9 +4568,19 @@ dependencies = [ [[package]] name = "quick-xml" -version = "0.38.3" +version = "0.37.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quick-xml" +version = "0.38.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42a232e7487fc2ef313d96dde7948e7a3c05101870d8985e4fd8d26aedd27b89" +checksum = "b66c2058c55a409d601666cffe35f04333cf1013010882cec174a7467cd4e21c" dependencies = [ "memchr", "serde", @@ -4052,7 +4615,7 @@ dependencies = [ "bytes", "getrandom 0.3.4", "lru-slab", - "rand", + "rand 0.9.2", "ring", "rustc-hash 2.1.1", "rustls", @@ -4080,9 +4643,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.41" +version = "1.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" +checksum = "a338cc41d27e6cc6dce6cefc13a0729dfbb81c262b1f519331575dd80ef3067f" dependencies = [ "proc-macro2", ] @@ -4093,14 +4656,41 @@ version = "5.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" +[[package]] +name = "radium" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha 0.3.1", + "rand_core 0.6.4", +] + [[package]] name = "rand" version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" dependencies = [ - "rand_chacha", - "rand_core", + "rand_chacha 0.9.0", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core 0.6.4", ] [[package]] @@ -4110,7 +4700,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", - "rand_core", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom 0.2.16", ] [[package]] @@ -4151,6 +4750,26 @@ dependencies = [ "bitflags 2.10.0", ] +[[package]] +name = "ref-cast" +version = "1.0.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" +dependencies = [ + "ref-cast-impl", +] + +[[package]] +name = "ref-cast-impl" +version = "1.0.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.110", +] + [[package]] name = "regex" version = "1.12.2" @@ -4186,6 +4805,44 @@ version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" +[[package]] +name = "rend" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "71fe3824f5629716b1589be05dacd749f6aa084c87e00e016714a8cdfccc997c" +dependencies = [ + "bytecheck", +] + +[[package]] +name = "reqsign" +version = "0.16.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43451dbf3590a7590684c25fb8d12ecdcc90ed3ac123433e500447c7d77ed701" +dependencies = [ + "anyhow", + "async-trait", + "base64", + "chrono", + "form_urlencoded", + "getrandom 0.2.16", + "hex", + "hmac", + "home", + "http 1.3.1", + "log", + "percent-encoding", + "quick-xml 0.37.5", + "rand 0.8.5", + "reqwest", + "rust-ini", + "serde", + "serde_json", + "sha1", + "sha2", + "tokio", +] + [[package]] name = "reqwest" version = "0.12.24" @@ -4252,6 +4909,71 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "rkyv" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" +dependencies = [ + "bitvec", + "bytecheck", + "bytes", + "hashbrown 0.12.3", + "ptr_meta", + "rend", + "rkyv_derive", + "seahash", + "tinyvec", + "uuid", +] + +[[package]] +name = "rkyv_derive" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "roaring" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f08d6a905edb32d74a5d5737a0c9d7e950c312f3c46cb0ca0a2ca09ea11878a0" +dependencies = [ + "bytemuck", + "byteorder", +] + +[[package]] +name = "rust-ini" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "796e8d2b6696392a43bea58116b667fb4c29727dc5abd27d6acf338bb4f688c7" +dependencies = [ + "cfg-if", + "ordered-multimap", +] + +[[package]] +name = "rust_decimal" +version = "1.39.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" +dependencies = [ + "arrayvec", + "borsh", + "bytes", + "num-traits", + "rand 0.8.5", + "rkyv", + "serde", + "serde_json", +] + [[package]] name = "rustc-demangle" version = "0.1.26" @@ -4307,9 +5029,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.34" +version = "0.23.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a9586e9ee2b4f8fab52a0048ca7334d7024eef48e2cb9407e3497bb7cab7fa7" +checksum = "533f54bc6a7d4f647e46ad909549eda97bf5afc1585190ef692b4286b198bd8f" dependencies = [ "aws-lc-rs", "once_cell", @@ -4343,9 +5065,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "94182ad936a0c91c324cd46c6511b9510ed16af436d7b5bab34beab0afd55f7a" dependencies = [ "web-time", "zeroize", @@ -4353,9 +5075,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.7" +version = "0.103.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" +checksum = "2ffdfa2f5286e2247234e03f680868ac2815974dc39e00ea15adc445d0aafe52" dependencies = [ "aws-lc-rs", "ring", @@ -4393,12 +5115,42 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "schemars" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cd191f9397d57d581cddd31014772520aa448f65ef991055d7f61582c65165f" +dependencies = [ + "dyn-clone", + "ref-cast", + "serde", + "serde_json", +] + +[[package]] +name = "schemars" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9558e172d4e8533736ba97870c4b2cd63f84b382a3d6eb063da41b91cce17289" +dependencies = [ + "dyn-clone", + "ref-cast", + "serde", + "serde_json", +] + [[package]] name = "scopeguard" version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" +[[package]] +name = "seahash" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" + [[package]] name = "security-framework" version = "3.5.1" @@ -4450,10 +5202,20 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float", + "ordered-float 2.10.1", "serde", ] +[[package]] +name = "serde_bytes" +version = "0.11.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5d440709e79d88e51ac01c4b72fc6cb7314017bb7da9eeff678aa94c10e3ea8" +dependencies = [ + "serde", + "serde_core", +] + [[package]] name = "serde_core" version = "1.0.228" @@ -4471,7 +5233,7 @@ checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4487,6 +5249,17 @@ dependencies = [ "serde_core", ] +[[package]] +name = "serde_repr" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.110", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -4499,13 +5272,44 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_with" +version = "3.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "10574371d41b0d9b2cff89418eda27da52bcaff2cc8741db26382a77c29131f1" +dependencies = [ + "base64", + "chrono", + "hex", + "indexmap 1.9.3", + "indexmap 2.12.0", + "schemars 0.9.0", + "schemars 1.1.0", + "serde_core", + "serde_json", + "serde_with_macros", + "time", +] + +[[package]] +name = "serde_with_macros" +version = "3.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08a72d8216842fdd57820dc78d840bef99248e35fb2554ff923319e60f2d686b" +dependencies = [ + "darling 0.21.3", + "proc-macro2", + "quote", + "syn 2.0.110", +] + [[package]] name = "serde_yaml" version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap", + "indexmap 2.12.0", "itoa", "ryu", "serde", @@ -4622,7 +5426,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4637,12 +5441,27 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + [[package]] name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" +[[package]] +name = "strum" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" +dependencies = [ + "strum_macros 0.27.2", +] + [[package]] name = "strum_macros" version = "0.26.4" @@ -4653,7 +5472,19 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.108", + "syn 2.0.110", +] + +[[package]] +name = "strum_macros" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7695ce3845ea4b33927c055a39dc438a45b059f7c1b3d91d38d10355fb8cbca7" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.110", ] [[package]] @@ -4664,9 +5495,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.16.3" +version = "12.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d03f433c9befeea460a01d750e698aa86caf86dcfbd77d552885cd6c89d52f50" +checksum = "b3d8046c5674ab857104bc4559d505f4809b8060d57806e45d49737c97afeb60" dependencies = [ "debugid", "memmap2", @@ -4676,9 +5507,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.16.3" +version = "12.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13d359ef6192db1760a34321ec4f089245ede4342c27e59be99642f12a859de8" +checksum = "1accb6e5c4b0f682de907623912e616b44be1c9e725775155546669dbff720ec" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -4698,9 +5529,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.108" +version = "2.0.110" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da58917d35242480a05c2897064da0a80589a2a0476c9a3f2fdc83b53502e917" +checksum = "a99801b5bd34ede4cf3fc688c5919368fea4e4814a4664359503e6015b280aea" dependencies = [ "proc-macro2", "quote", @@ -4724,9 +5555,21 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] +[[package]] +name = "tagptr" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b2093cf4c8eb1e67749a6762251bc9cd836b6fc171623bd0a9d324d37af2417" + +[[package]] +name = "tap" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" + [[package]] name = "tempfile" version = "3.23.0" @@ -4766,7 +5609,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4777,7 +5620,7 @@ checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4790,6 +5633,15 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + [[package]] name = "thrift" version = "0.17.0" @@ -4798,7 +5650,9 @@ checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", "integer-encoding", - "ordered-float", + "log", + "ordered-float 2.10.1", + "threadpool", ] [[package]] @@ -4839,6 +5693,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" dependencies = [ "deranged", + "itoa", "num-conv", "powerfmt", "serde", @@ -4873,9 +5728,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "42d3e9c45c09de15d06dd8acf5f4e0e399e85927b7f00711024eb7ae10fa4869" dependencies = [ "displaydoc", "zerovec", @@ -4931,7 +5786,7 @@ checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4946,9 +5801,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.16" +version = "0.7.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14307c986784f72ef81c89db7d9e28d6ac26d16213b109ea501696195e6e3ce5" +checksum = "2efa149fe76073d6e8fd97ef4f4eca7b67f599660115591483572e406e165594" dependencies = [ "bytes", "futures-core", @@ -4957,6 +5812,36 @@ dependencies = [ "tokio", ] +[[package]] +name = "toml_datetime" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" +dependencies = [ + "serde_core", +] + +[[package]] +name = "toml_edit" +version = "0.23.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" +dependencies = [ + "indexmap 2.12.0", + "toml_datetime", + "toml_parser", + "winnow", +] + +[[package]] +name = "toml_parser" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" +dependencies = [ + "winnow", +] + [[package]] name = "tower" version = "0.5.2" @@ -5021,7 +5906,7 @@ checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5045,7 +5930,27 @@ version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ea3136b675547379c4bd395ca6b938e5ad3c3d20fad76e7fe85f9e0d011419c" dependencies = [ - "rand", + "rand 0.9.2", +] + +[[package]] +name = "typed-builder" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd9d30e3a08026c78f246b173243cf07b3696d274debd26680773b6773c2afc7" +dependencies = [ + "typed-builder-macro", +] + +[[package]] +name = "typed-builder-macro" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.110", ] [[package]] @@ -5065,9 +5970,9 @@ checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "unicode-ident" -version = "1.0.20" +version = "1.0.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "462eeb75aeb73aea900253ce739c8e18a67423fadf006037cd3ff27e82748a06" +checksum = "9312f7c4f6ff9069b165498234ce8be658059c6728633667c526e27dc2cf1df5" [[package]] name = "unicode-segmentation" @@ -5198,9 +6103,9 @@ dependencies = [ [[package]] name = "wasm-bindgen" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" +checksum = "da95793dfc411fbbd93f5be7715b0578ec61fe87cb1a42b12eb625caa5c5ea60" dependencies = [ "cfg-if", "once_cell", @@ -5209,25 +6114,11 @@ dependencies = [ "wasm-bindgen-shared", ] -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.104" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" -dependencies = [ - "bumpalo", - "log", - "proc-macro2", - "quote", - "syn 2.0.108", - "wasm-bindgen-shared", -] - [[package]] name = "wasm-bindgen-futures" -version = "0.4.54" +version = "0.4.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" +checksum = "551f88106c6d5e7ccc7cd9a16f312dd3b5d36ea8b4954304657d5dfba115d4a0" dependencies = [ "cfg-if", "js-sys", @@ -5238,9 +6129,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" +checksum = "04264334509e04a7bf8690f2384ef5265f05143a4bff3889ab7a3269adab59c2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5248,22 +6139,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" +checksum = "420bc339d9f322e562942d52e115d57e950d12d88983a14c79b86859ee6c7ebc" dependencies = [ + "bumpalo", "proc-macro2", "quote", - "syn 2.0.108", - "wasm-bindgen-backend", + "syn 2.0.110", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" +checksum = "76f218a38c84bcb33c25ec7059b07847d465ce0e0a76b995e134a45adcb6af76" dependencies = [ "unicode-ident", ] @@ -5283,9 +6174,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" +checksum = "3a1f95c0d03a47f4ae1f7a64643a6bb97465d9b740f0fa8f90ea33915c99a9a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -5303,9 +6194,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b130c0d2d49f8b6889abc456e795e82525204f27c42cf767cf0d7734e089b8" +checksum = "b2878ef029c47c6e8cf779119f20fcf52bde7ad42a731b2a304bc221df17571e" dependencies = [ "rustls-pki-types", ] @@ -5374,7 +6265,7 @@ checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5385,7 +6276,7 @@ checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5643,6 +6534,15 @@ version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" +[[package]] +name = "winnow" +version = "0.7.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "21a0236b59786fed61e2a80582dd500fe61f18b5dca67a4a067d0bc9039339cf" +dependencies = [ + "memchr", +] + [[package]] name = "wit-bindgen" version = "0.46.0" @@ -5651,9 +6551,18 @@ checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "9edde0db4769d2dc68579893f2306b26c6ecfbe0ef499b013d731b7b9247e0b9" + +[[package]] +name = "wyz" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" +dependencies = [ + "tap", +] [[package]] name = "xmlparser" @@ -5667,13 +6576,21 @@ version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" +[[package]] +name = "xz2" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" +dependencies = [ + "lzma-sys", +] + [[package]] name = "yoke" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "72d6e5c6afb84d73944e5cedb052c4680d5657337201555f9f2a16b7406d4954" dependencies = [ - "serde", "stable_deref_trait", "yoke-derive", "zerofrom", @@ -5681,13 +6598,13 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", "synstructure", ] @@ -5708,7 +6625,7 @@ checksum = "88d2b8d9c68ad2b9e4340d7832716a4d21a22a1154777ad56ea55c51a9cf3831" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5728,7 +6645,7 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", "synstructure", ] @@ -5740,9 +6657,9 @@ checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +checksum = "2a59c17a5562d507e4b54960e8569ebee33bee890c70aa3fe7b97e85a9fd7851" dependencies = [ "displaydoc", "yoke", @@ -5751,9 +6668,9 @@ dependencies = [ [[package]] name = "zerovec" -version = "0.11.4" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "6c28719294829477f525be0186d13efa9a3c602f7ec202ca9e353d310fb9a002" dependencies = [ "yoke", "zerofrom", @@ -5762,13 +6679,13 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index 3c609b9bd2..ab1eb19876 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,11 +34,12 @@ edition = "2021" rust-version = "1.86" [workspace.dependencies] -arrow = { version = "56.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "56.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "56.2.0", default-features = false, features = ["experimental"] } datafusion = { version = "50.3.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion-datasource = { version = "50.3.0" } datafusion-spark = { version = "50.3.0" } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } @@ -53,6 +54,8 @@ object_store = { version = "0.12.3", features = ["gcp", "azure", "aws", "http"] url = "2.2" aws-config = "1.8.10" aws-credential-types = "1.2.9" +iceberg = { git = "https://github.com/apache/iceberg-rust", rev = "a667539" } +iceberg-datafusion = { git = "https://github.com/apache/iceberg-rust", rev = "a667539" } [profile.release] debug = true diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index c9f95a1d81..139d15b0d6 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -60,6 +60,7 @@ tempfile = "3.8.0" itertools = "0.14.0" paste = "1.0.14" datafusion = { workspace = true, features = ["parquet_encryption"] } +datafusion-datasource = { workspace = true } datafusion-spark = { workspace = true } once_cell = "1.18.0" regex = { workspace = true } @@ -77,6 +78,8 @@ reqwest = { version = "0.12", default-features = false, features = ["rustls-tls- object_store_opendal = {version = "0.54.0", optional = true} hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} opendal = { version ="0.54.1", optional = true, features = ["services-hdfs"] } +iceberg = { workspace = true } +serde_json = "1.0" uuid = "1.18.1" [target.'cfg(target_os = "linux")'.dependencies] diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs new file mode 100644 index 0000000000..ff6648850b --- /dev/null +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -0,0 +1,477 @@ +// 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. + +//! Native Iceberg table scan operator using iceberg-rust + +use std::any::Any; +use std::collections::{HashMap, VecDeque}; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use arrow::array::RecordBatch; +use arrow::datatypes::SchemaRef; +use datafusion::common::{DataFusionError, Result as DFResult}; +use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::metrics::{ + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, +}; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, +}; +use futures::future::BoxFuture; +use futures::{ready, FutureExt, Stream, StreamExt, TryStreamExt}; +use iceberg::io::FileIO; + +use crate::execution::operators::ExecutionError; +use crate::parquet::parquet_support::SparkParquetOptions; +use crate::parquet::schema_adapter::SparkSchemaAdapterFactory; +use datafusion::datasource::schema_adapter::SchemaAdapterFactory; +use datafusion_comet_spark_expr::EvalMode; +use datafusion_datasource::file_stream::FileStreamMetrics; + +/// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. +/// +/// Executes pre-planned FileScanTasks for efficient parallel scanning. +#[derive(Debug)] +pub struct IcebergScanExec { + /// Iceberg table metadata location for FileIO initialization + metadata_location: String, + /// Output schema after projection + output_schema: SchemaRef, + /// Cached execution plan properties + plan_properties: PlanProperties, + /// Catalog-specific configuration for FileIO + catalog_properties: HashMap, + /// Pre-planned file scan tasks, grouped by partition + file_task_groups: Vec>, + /// Metrics + metrics: ExecutionPlanMetricsSet, +} + +impl IcebergScanExec { + pub fn new( + metadata_location: String, + schema: SchemaRef, + catalog_properties: HashMap, + file_task_groups: 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 metrics = ExecutionPlanMetricsSet::new(); + + Ok(Self { + metadata_location, + output_schema, + plan_properties, + catalog_properties, + file_task_groups, + metrics, + }) + } + + fn compute_properties(schema: SchemaRef, num_partitions: usize) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(num_partitions), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +impl ExecutionPlan for IcebergScanExec { + fn name(&self) -> &str { + "IcebergScanExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.output_schema) + } + + fn properties(&self) -> &PlanProperties { + &self.plan_properties + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _children: Vec>, + ) -> DFResult> { + Ok(self) + } + + fn execute( + &self, + 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() + ))) + } + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } +} + +impl IcebergScanExec { + /// Handles MOR (Merge-On-Read) tables by automatically applying positional and equality + /// deletes via iceberg-rust's ArrowReader. + fn execute_with_tasks( + &self, + tasks: Vec, + partition: usize, + 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); + + // Create parallel file stream that overlaps opening next file with reading current file + let file_stream = IcebergFileStream::new( + tasks, + file_io, + batch_size, + Arc::clone(&output_schema), + metrics, + )?; + + // Note: BatchSplitStream adds overhead. Since we're already setting batch_size in + // iceberg-rust's ArrowReaderBuilder, it should produce correctly sized batches. + // Only use BatchSplitStream as a safety net if needed. + // For now, return the file_stream directly to reduce stream nesting overhead. + + Ok(Box::pin(file_stream)) + } + + fn load_file_io( + catalog_properties: &HashMap, + metadata_location: &str, + ) -> Result { + let mut file_io_builder = FileIO::from_path(metadata_location) + .map_err(|e| DataFusionError::Execution(format!("Failed to create FileIO: {}", e)))?; + + for (key, value) in catalog_properties { + file_io_builder = file_io_builder.with_prop(key, value); + } + + file_io_builder + .build() + .map_err(|e| DataFusionError::Execution(format!("Failed to build FileIO: {}", e))) + } +} + +/// Metrics for IcebergScanExec +struct IcebergScanMetrics { + /// Baseline metrics (output rows, elapsed compute time) + baseline: BaselineMetrics, + /// File stream metrics (time opening, time scanning, etc.) + file_stream: FileStreamMetrics, + /// Count of file splits (FileScanTasks) processed + num_splits: Count, +} + +impl IcebergScanMetrics { + fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { + Self { + baseline: BaselineMetrics::new(metrics, partition), + file_stream: FileStreamMetrics::new(metrics, partition), + num_splits: MetricBuilder::new(metrics).counter("num_splits", partition), + } + } +} + +/// State machine for IcebergFileStream +enum FileStreamState { + /// Idle state - need to start opening next file + Idle, + /// Opening a file + Opening { + future: BoxFuture<'static, DFResult>, + }, + /// Reading from current file while potentially opening next file + Reading { + current: SendableRecordBatchStream, + next: Option>>, + }, + /// Error state + Error, +} + +/// Stream that reads Iceberg files with parallel opening optimization. +/// Opens the next file while reading the current file to overlap IO with compute. +/// +/// Inspired by DataFusion's [`FileStream`] pattern for overlapping file opening with reading. +/// +/// [`FileStream`]: https://github.com/apache/datafusion/blob/main/datafusion/datasource/src/file_stream.rs +struct IcebergFileStream { + schema: SchemaRef, + file_io: FileIO, + batch_size: usize, + tasks: VecDeque, + state: FileStreamState, + metrics: IcebergScanMetrics, +} + +impl IcebergFileStream { + fn new( + tasks: Vec, + file_io: FileIO, + batch_size: usize, + schema: SchemaRef, + metrics: IcebergScanMetrics, + ) -> DFResult { + Ok(Self { + schema, + file_io, + batch_size, + tasks: tasks.into_iter().collect(), + state: FileStreamState::Idle, + metrics, + }) + } + + fn start_next_file( + &mut self, + ) -> Option>> { + let task = self.tasks.pop_front()?; + + self.metrics.num_splits.add(1); + + let file_io = self.file_io.clone(); + let batch_size = self.batch_size; + let schema = Arc::clone(&self.schema); + + Some(Box::pin(async move { + let task_stream = futures::stream::iter(vec![Ok(task)]).boxed(); + + let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io) + .with_batch_size(batch_size) + .with_row_selection_enabled(true) + .build(); + + let stream = reader.read(task_stream).map_err(|e| { + DataFusionError::Execution(format!("Failed to read Iceberg task: {}", e)) + })?; + + let target_schema = Arc::clone(&schema); + + // Schema adaptation handles differences in Arrow field names and metadata + // between the file schema and expected output schema + let mapped_stream = stream + .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))) + .and_then(move |batch| { + let spark_options = SparkParquetOptions::new(EvalMode::Legacy, "UTC", false); + let adapter_factory = SparkSchemaAdapterFactory::new(spark_options, None); + let file_schema = batch.schema(); + let adapter = adapter_factory + .create(Arc::clone(&target_schema), Arc::clone(&file_schema)); + + let result = match adapter.map_schema(file_schema.as_ref()) { + Ok((schema_mapper, _projection)) => { + schema_mapper.map_batch(batch).map_err(|e| { + DataFusionError::Execution(format!("Batch mapping failed: {}", e)) + }) + } + Err(e) => Err(DataFusionError::Execution(format!( + "Schema mapping failed: {}", + e + ))), + }; + futures::future::ready(result) + }); + + Ok(Box::pin(IcebergStreamWrapper { + inner: mapped_stream, + schema, + }) as SendableRecordBatchStream) + })) + } + + fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll>> { + loop { + match &mut self.state { + FileStreamState::Idle => { + self.metrics.file_stream.time_opening.start(); + match self.start_next_file() { + Some(future) => { + self.state = FileStreamState::Opening { future }; + } + None => return Poll::Ready(None), + } + } + FileStreamState::Opening { future } => match ready!(future.poll_unpin(cx)) { + Ok(stream) => { + self.metrics.file_stream.time_opening.stop(); + self.metrics.file_stream.time_scanning_until_data.start(); + self.metrics.file_stream.time_scanning_total.start(); + let next = self.start_next_file(); + self.state = FileStreamState::Reading { + current: stream, + next, + }; + } + Err(e) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + }, + FileStreamState::Reading { current, next } => { + // Poll next file opening future to drive it forward (background IO) + if let Some(next_future) = next { + if let Poll::Ready(result) = next_future.poll_unpin(cx) { + match result { + Ok(stream) => { + *next = Some(Box::pin(futures::future::ready(Ok(stream)))); + } + Err(e) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } + } + + match ready!(self + .metrics + .baseline + .record_poll(current.poll_next_unpin(cx))) + { + Some(result) => { + // Stop time_scanning_until_data on first batch (idempotent) + self.metrics.file_stream.time_scanning_until_data.stop(); + self.metrics.file_stream.time_scanning_total.stop(); + // Restart time_scanning_total for next batch + self.metrics.file_stream.time_scanning_total.start(); + return Poll::Ready(Some(result)); + } + None => { + self.metrics.file_stream.time_scanning_until_data.stop(); + self.metrics.file_stream.time_scanning_total.stop(); + match next.take() { + Some(mut next_future) => match next_future.poll_unpin(cx) { + Poll::Ready(Ok(stream)) => { + self.metrics.file_stream.time_scanning_until_data.start(); + self.metrics.file_stream.time_scanning_total.start(); + let next_next = self.start_next_file(); + self.state = FileStreamState::Reading { + current: stream, + next: next_next, + }; + } + Poll::Ready(Err(e)) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + Poll::Pending => { + self.state = FileStreamState::Opening { + future: next_future, + }; + } + }, + None => { + return Poll::Ready(None); + } + } + } + } + } + FileStreamState::Error => { + return Poll::Ready(None); + } + } + } + } +} + +impl Stream for IcebergFileStream { + type Item = DFResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.metrics.file_stream.time_processing.start(); + let result = self.poll_inner(cx); + self.metrics.file_stream.time_processing.stop(); + result + } +} + +impl RecordBatchStream for IcebergFileStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +/// Wrapper around iceberg-rust's stream that avoids strict schema checks. +/// Returns the expected output schema to prevent rejection of batches with metadata differences. +struct IcebergStreamWrapper { + inner: S, + schema: SchemaRef, +} + +impl Stream for IcebergStreamWrapper +where + S: Stream> + Unpin, +{ + type Item = DFResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.inner.poll_next_unpin(cx) + } +} + +impl RecordBatchStream for IcebergStreamWrapper +where + S: Stream> + Unpin, +{ + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +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 + ) + } +} diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index c8cfebd45e..b3998e2f60 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -22,11 +22,13 @@ use std::fmt::Debug; use jni::objects::GlobalRef; pub use copy::*; +pub use iceberg_scan::*; pub use scan::*; mod copy; mod expand; pub use expand::ExpandExec; +mod iceberg_scan; mod scan; /// Error returned during executing operators. diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index a33df705b3..0fe04a5a41 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -17,6 +17,7 @@ //! Converts Spark physical plan to DataFusion physical plan +use crate::execution::operators::IcebergScanExec; use crate::{ errors::ExpressionError, execution::{ @@ -65,6 +66,7 @@ use datafusion_comet_spark_expr::{ create_negate_expr, BinaryOutputStyle, BloomFilterAgg, BloomFilterMightContain, EvalMode, SparkHour, SparkMinute, SparkSecond, }; +use iceberg::expr::Bind; use crate::execution::operators::ExecutionError::GeneralError; use crate::execution::shuffle::{CometPartitioning, CompressionCodec}; @@ -1368,6 +1370,44 @@ impl PhysicalPlanner { Arc::new(SparkPlan::new(spark_plan.plan_id, Arc::new(scan), vec![])), )) } + OpStruct::IcebergScan(scan) => { + let required_schema: SchemaRef = + convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + + let catalog_properties: HashMap = scan + .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.file_partitions[self.partition as usize].file_scan_tasks, + )?; + let file_task_groups = vec![tasks]; + + let iceberg_scan = IcebergScanExec::new( + metadata_location, + required_schema, + catalog_properties, + file_task_groups, + )?; + + Ok(( + vec![], + Arc::new(SparkPlan::new( + spark_plan.plan_id, + Arc::new(iceberg_scan), + vec![], + )), + )) + } OpStruct::ShuffleWriter(writer) => { assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; @@ -2656,6 +2696,167 @@ fn convert_spark_types_to_arrow_schema( arrow_schema } +/// Converts protobuf FileScanTasks from Scala into iceberg-rust FileScanTask objects. +/// +/// Each task contains a residual predicate that is used for row-group level filtering +/// during Parquet scanning. +fn parse_file_scan_tasks( + proto_tasks: &[spark_operator::IcebergFileScanTask], +) -> Result, ExecutionError> { + let results: Result, _> = proto_tasks + .iter() + .map(|proto_task| { + let schema: iceberg::spec::Schema = serde_json::from_str(&proto_task.schema_json) + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to parse schema JSON: {}", e)) + })?; + + let schema_ref = Arc::new(schema); + + // CometScanRule validates format before serialization + debug_assert_eq!( + proto_task.data_file_format.as_str(), + "PARQUET", + "Only PARQUET format is supported. This indicates a bug in CometScanRule validation." + ); + let data_file_format = iceberg::spec::DataFileFormat::Parquet; + + let deletes: Vec = proto_task + .delete_files + .iter() + .map(|del| { + let file_type = match del.content_type.as_str() { + "POSITION_DELETES" => iceberg::spec::DataContentType::PositionDeletes, + "EQUALITY_DELETES" => iceberg::spec::DataContentType::EqualityDeletes, + other => { + return Err(GeneralError(format!( + "Invalid delete content type '{}'. This indicates a bug in Scala serialization.", + other + ))) + } + }; + + Ok(iceberg::scan::FileScanTaskDeleteFile { + file_path: del.file_path.clone(), + file_type, + partition_spec_id: del.partition_spec_id, + equality_ids: if del.equality_ids.is_empty() { + None + } else { + Some(del.equality_ids.clone()) + }, + }) + }) + .collect::, ExecutionError>>()?; + + // Residuals are serialized with binding=false (name-based references). + // Convert to Iceberg predicate and bind to this file's schema for row-group filtering. + let bound_predicate = proto_task + .residual + .as_ref() + .and_then(|residual_expr| { + convert_spark_expr_to_predicate(residual_expr) + }) + .map( + |pred| -> Result { + let bound = pred.bind(Arc::clone(&schema_ref), true).map_err(|e| { + ExecutionError::GeneralError(format!( + "Failed to bind predicate to schema: {}", + e + )) + })?; + + Ok(bound) + }, + ) + .transpose()?; + + let partition = if let (Some(partition_json), Some(partition_type_json)) = ( + proto_task.partition_data_json.as_ref(), + proto_task.partition_type_json.as_ref(), + ) { + let partition_type: iceberg::spec::StructType = + serde_json::from_str(partition_type_json).map_err(|e| { + ExecutionError::GeneralError(format!( + "Failed to parse partition type JSON: {}", + e + )) + })?; + + let partition_data_value: serde_json::Value = serde_json::from_str(partition_json) + .map_err(|e| { + ExecutionError::GeneralError(format!( + "Failed to parse partition data JSON: {}", + e + )) + })?; + + match iceberg::spec::Literal::try_from_json( + partition_data_value, + &iceberg::spec::Type::Struct(partition_type), + ) { + Ok(Some(iceberg::spec::Literal::Struct(s))) => Some(s), + Ok(None) => None, + Ok(other) => { + return Err(GeneralError(format!( + "Expected struct literal for partition data, got: {:?}", + other + ))) + } + Err(e) => { + return Err(GeneralError(format!( + "Failed to deserialize partition data from JSON: {}", + e + ))) + } + } + } else { + None + }; + + let partition_spec = if let Some(partition_spec_json) = + proto_task.partition_spec_json.as_ref() + { + // Try to parse partition spec, but gracefully handle unknown transforms + // for forward compatibility (e.g., TestForwardCompatibility tests) + match serde_json::from_str::(partition_spec_json) { + Ok(spec) => Some(Arc::new(spec)), + Err(_) => None, + } + } else { + None + }; + + let name_mapping = if let Some(name_mapping_json) = proto_task.name_mapping_json.as_ref() + { + match serde_json::from_str::(name_mapping_json) { + Ok(mapping) => Some(Arc::new(mapping)), + Err(_) => None, // Name mapping is optional + } + } else { + None + }; + + Ok(iceberg::scan::FileScanTask { + data_file_path: proto_task.data_file_path.clone(), + start: proto_task.start, + length: proto_task.length, + record_count: proto_task.record_count, + data_file_format, + schema: schema_ref, + project_field_ids: proto_task.project_field_ids.clone(), + predicate: bound_predicate, + deletes, + partition, + partition_spec, + name_mapping, + }) + }) + .collect(); + + results +} + /// Create CASE WHEN expression and add casting as needed fn create_case_expr( when_then_pairs: Vec<(Arc, Arc)>, @@ -2895,6 +3096,250 @@ fn literal_to_array_ref( } } +// ============================================================================ +// Spark Expression to Iceberg Predicate Conversion +// ============================================================================ +// +// Predicates are converted through Spark expressions rather than directly from +// Iceberg Java to Iceberg Rust. This leverages Comet's existing expression +// serialization infrastructure, which handles hundreds of expression types. +// +// Conversion path: +// Iceberg Expression (Java) -> Spark Catalyst Expression -> Protobuf -> Iceberg Predicate (Rust) +// +// Note: NOT IN predicates are skipped because iceberg-rust's RowGroupMetricsEvaluator::not_in() +// always returns MIGHT_MATCH (never prunes row groups). These are handled by CometFilter post-scan. + +/// Converts a protobuf Spark expression to an Iceberg predicate for row-group filtering. +fn convert_spark_expr_to_predicate( + expr: &spark_expression::Expr, +) -> Option { + use spark_expression::expr::ExprStruct; + + match &expr.expr_struct { + Some(ExprStruct::Eq(binary)) => convert_binary_to_predicate( + &binary.left, + &binary.right, + iceberg::expr::PredicateOperator::Eq, + ), + Some(ExprStruct::Neq(binary)) => convert_binary_to_predicate( + &binary.left, + &binary.right, + iceberg::expr::PredicateOperator::NotEq, + ), + Some(ExprStruct::Lt(binary)) => convert_binary_to_predicate( + &binary.left, + &binary.right, + iceberg::expr::PredicateOperator::LessThan, + ), + Some(ExprStruct::LtEq(binary)) => convert_binary_to_predicate( + &binary.left, + &binary.right, + iceberg::expr::PredicateOperator::LessThanOrEq, + ), + Some(ExprStruct::Gt(binary)) => convert_binary_to_predicate( + &binary.left, + &binary.right, + iceberg::expr::PredicateOperator::GreaterThan, + ), + Some(ExprStruct::GtEq(binary)) => convert_binary_to_predicate( + &binary.left, + &binary.right, + iceberg::expr::PredicateOperator::GreaterThanOrEq, + ), + Some(ExprStruct::IsNull(unary)) => { + if let Some(ref child) = unary.child { + extract_column_reference(child).map(|column| { + iceberg::expr::Predicate::Unary(iceberg::expr::UnaryExpression::new( + iceberg::expr::PredicateOperator::IsNull, + iceberg::expr::Reference::new(column), + )) + }) + } else { + None + } + } + Some(ExprStruct::IsNotNull(unary)) => { + if let Some(ref child) = unary.child { + extract_column_reference(child).map(|column| { + iceberg::expr::Predicate::Unary(iceberg::expr::UnaryExpression::new( + iceberg::expr::PredicateOperator::NotNull, + iceberg::expr::Reference::new(column), + )) + }) + } else { + None + } + } + Some(ExprStruct::And(binary)) => { + let left = binary + .left + .as_ref() + .and_then(|e| convert_spark_expr_to_predicate(e)); + let right = binary + .right + .as_ref() + .and_then(|e| convert_spark_expr_to_predicate(e)); + match (left, right) { + (Some(l), Some(r)) => Some(l.and(r)), + (Some(l), None) => Some(l), + (None, Some(r)) => Some(r), + _ => None, + } + } + Some(ExprStruct::Or(binary)) => { + let left = binary + .left + .as_ref() + .and_then(|e| convert_spark_expr_to_predicate(e)); + let right = binary + .right + .as_ref() + .and_then(|e| convert_spark_expr_to_predicate(e)); + match (left, right) { + (Some(l), Some(r)) => Some(l.or(r)), + _ => None, // OR requires both sides to be valid + } + } + Some(ExprStruct::Not(unary)) => unary + .child + .as_ref() + .and_then(|child| convert_spark_expr_to_predicate(child)) + .map(|p| !p), + Some(ExprStruct::In(in_expr)) => { + // NOT IN predicates don't work correctly with iceberg-rust's row-group filtering. + // The iceberg-rust RowGroupMetricsEvaluator::not_in() always returns MIGHT_MATCH + // (never prunes row groups), even in cases where pruning is possible (e.g., when + // min == max == value and value is in the NOT IN set). + // + // Workaround: Skip NOT IN in predicate pushdown and let CometFilter handle it + // post-scan. This sacrifices row-group pruning for NOT IN but ensures correctness. + if in_expr.negated { + return None; + } + + if let Some(ref value) = in_expr.in_value { + if let Some(column) = extract_column_reference(value) { + let datums: Vec = in_expr + .lists + .iter() + .filter_map(extract_literal_as_datum) + .collect(); + + if datums.len() == in_expr.lists.len() { + Some(iceberg::expr::Reference::new(column).is_in(datums)) + } else { + None + } + } else { + None + } + } else { + None + } + } + _ => None, // Unsupported expression + } +} + +fn convert_binary_to_predicate( + left: &Option>, + right: &Option>, + op: iceberg::expr::PredicateOperator, +) -> Option { + let left_ref = left.as_ref()?; + let right_ref = right.as_ref()?; + + if let (Some(column), Some(datum)) = ( + extract_column_reference(left_ref), + extract_literal_as_datum(right_ref), + ) { + return Some(iceberg::expr::Predicate::Binary( + iceberg::expr::BinaryExpression::new(op, iceberg::expr::Reference::new(column), datum), + )); + } + + if let (Some(datum), Some(column)) = ( + extract_literal_as_datum(left_ref), + extract_column_reference(right_ref), + ) { + let reversed_op = match op { + iceberg::expr::PredicateOperator::LessThan => { + iceberg::expr::PredicateOperator::GreaterThan + } + iceberg::expr::PredicateOperator::LessThanOrEq => { + iceberg::expr::PredicateOperator::GreaterThanOrEq + } + iceberg::expr::PredicateOperator::GreaterThan => { + iceberg::expr::PredicateOperator::LessThan + } + iceberg::expr::PredicateOperator::GreaterThanOrEq => { + iceberg::expr::PredicateOperator::LessThanOrEq + } + _ => op, // Eq and NotEq are symmetric + }; + return Some(iceberg::expr::Predicate::Binary( + iceberg::expr::BinaryExpression::new( + reversed_op, + iceberg::expr::Reference::new(column), + datum, + ), + )); + } + + None +} + +fn extract_column_reference(expr: &spark_expression::Expr) -> Option { + use spark_expression::expr::ExprStruct; + + match &expr.expr_struct { + Some(ExprStruct::Unbound(unbound_ref)) => Some(unbound_ref.name.clone()), + _ => None, + } +} + +fn extract_literal_as_datum(expr: &spark_expression::Expr) -> Option { + use spark_expression::expr::ExprStruct; + + match &expr.expr_struct { + Some(ExprStruct::Literal(literal)) => { + if literal.is_null { + return None; + } + + match &literal.value { + Some(spark_expression::literal::Value::IntVal(v)) => { + Some(iceberg::spec::Datum::int(*v)) + } + Some(spark_expression::literal::Value::LongVal(v)) => { + Some(iceberg::spec::Datum::long(*v)) + } + Some(spark_expression::literal::Value::FloatVal(v)) => { + Some(iceberg::spec::Datum::double(*v as f64)) + } + Some(spark_expression::literal::Value::DoubleVal(v)) => { + Some(iceberg::spec::Datum::double(*v)) + } + Some(spark_expression::literal::Value::StringVal(v)) => { + Some(iceberg::spec::Datum::string(v.clone())) + } + Some(spark_expression::literal::Value::BoolVal(v)) => { + Some(iceberg::spec::Datum::bool(*v)) + } + Some(spark_expression::literal::Value::ByteVal(v)) => { + Some(iceberg::spec::Datum::int(*v)) + } + Some(spark_expression::literal::Value::ShortVal(v)) => { + Some(iceberg::spec::Datum::int(*v)) + } + _ => None, + } + } + _ => None, + } +} + #[cfg(test)] mod tests { use futures::{poll, StreamExt}; diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 3306ad574d..94661a20e6 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -48,6 +48,7 @@ message Operator { HashJoin hash_join = 109; Window window = 110; NativeScan native_scan = 111; + IcebergScan iceberg_scan = 112; } } @@ -107,6 +108,92 @@ message NativeScan { bool encryption_enabled = 14; } +message IcebergScan { + // Schema to read + repeated SparkStructField required_schema = 1; + + // Catalog-specific configuration for FileIO (credentials, S3/GCS config, etc.) + map catalog_properties = 2; + + // Pre-planned file scan tasks grouped by Spark partition + repeated IcebergFilePartition file_partitions = 3; + + // Table metadata file path for FileIO initialization + string metadata_location = 4; +} + +// Groups FileScanTasks for a single Spark partition +message IcebergFilePartition { + repeated IcebergFileScanTask file_scan_tasks = 1; +} + +// Iceberg FileScanTask containing data file, delete files, and residual filter +message IcebergFileScanTask { + // Data file path (e.g., s3://bucket/warehouse/db/table/data/00000-0-abc.parquet) + string data_file_path = 1; + + // Byte range to read (for split files) + uint64 start = 2; + uint64 length = 3; + + // Record count if reading entire file + optional uint64 record_count = 4; + + // File format (PARQUET, AVRO, or ORC) + string data_file_format = 5; + + // File schema as JSON (may differ due to schema evolution) + string schema_json = 6; + + // Field IDs to project + repeated int32 project_field_ids = 7; + + // Delete files for MOR tables + repeated IcebergDeleteFile delete_files = 8; + + // Residual filter after partition pruning (applied at row-group level) + // Example: if scan filter is "date >= '2024-01-01' AND status = 'active'" + // and file partition is date='2024-06-15', residual is "status = 'active'" + optional spark.spark_expression.Expr residual = 9; + + // Partition data from manifest entry (for proper constant identification) + // Serialized as JSON to represent the Struct of partition values + optional string partition_data_json = 10; + + // Partition type schema as JSON (Iceberg StructType for partition fields) + // Used to deserialize partition_data_json into proper Iceberg types + optional string partition_type_json = 12; + + // Partition spec as JSON (entire PartitionSpec object) + // Used to determine which partition fields are identity-transformed (constants) + // The spec includes spec-id embedded in the JSON. + optional string partition_spec_json = 13; + + // Name mapping from table metadata (property: schema.name-mapping.default) + // Used to resolve field IDs from column names when Parquet files lack field IDs + // or have field ID conflicts (e.g., Hive table migrations via add_files). + // Per Iceberg spec rule #2: "Use schema.name-mapping.default metadata to map + // field id to columns without field id". + optional string name_mapping_json = 14; +} + +// Iceberg delete file for MOR tables (positional or equality deletes) +// Positional: (file_path, row_position) pairs to skip +// Equality: Column values to filter out (specified by equality_ids) +message IcebergDeleteFile { + // Delete file path + string file_path = 1; + + // POSITION_DELETES or EQUALITY_DELETES + string content_type = 2; + + // Partition spec ID + int32 partition_spec_id = 3; + + // Equality field IDs (empty for positional deletes) + repeated int32 equality_ids = 4; +} + message Projection { repeated spark.spark_expression.Expr project_list = 1; } diff --git a/spark/pom.xml b/spark/pom.xml index 2ff5fcb33e..43bcd645b2 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -173,8 +173,52 @@ under the License. software.amazon.awssdk s3 + + + + + + spark-3.4 + + + org.apache.iceberg + iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} + 1.5.2 + test + + + + + + spark-3.5 + + true + + + + org.apache.iceberg + iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} + 1.8.1 + test + + + + + + spark-4.0 + + + org.apache.iceberg + iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} + 1.10.0 + test + + + + + diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala new file mode 100644 index 0000000000..e545f8e39c --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -0,0 +1,795 @@ +/* + * 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.iceberg + +import org.apache.spark.internal.Logging + +/** + * Shared reflection utilities for Iceberg operations. + * + * This object provides common reflection methods used across Comet for interacting with Iceberg + * classes. These are needed because many Iceberg methods are protected or package-private. + */ +object IcebergReflection extends Logging { + + /** + * Iceberg class names used throughout Comet. + */ + object ClassNames { + val CONTENT_SCAN_TASK = "org.apache.iceberg.ContentScanTask" + val FILE_SCAN_TASK = "org.apache.iceberg.FileScanTask" + val CONTENT_FILE = "org.apache.iceberg.ContentFile" + val STRUCT_LIKE = "org.apache.iceberg.StructLike" + val PARTITION_SCAN_TASK = "org.apache.iceberg.PartitionScanTask" + val DELETE_FILE = "org.apache.iceberg.DeleteFile" + val LITERAL = "org.apache.iceberg.expressions.Literal" + val SCHEMA_PARSER = "org.apache.iceberg.SchemaParser" + val SCHEMA = "org.apache.iceberg.Schema" + val PARTITION_SPEC_PARSER = "org.apache.iceberg.PartitionSpecParser" + val PARTITION_SPEC = "org.apache.iceberg.PartitionSpec" + val PARTITION_FIELD = "org.apache.iceberg.PartitionField" + val UNBOUND_PREDICATE = "org.apache.iceberg.expressions.UnboundPredicate" + } + + /** + * Iceberg content types. + */ + object ContentTypes { + val POSITION_DELETES = "POSITION_DELETES" + val EQUALITY_DELETES = "EQUALITY_DELETES" + } + + /** + * Iceberg file formats. + */ + object FileFormats { + val PARQUET = "PARQUET" + } + + /** + * Iceberg transform types. + */ + object Transforms { + val IDENTITY = "identity" + } + + /** + * Iceberg type names. + */ + object TypeNames { + val UNKNOWN = "unknown" + } + + /** + * Searches through class hierarchy to find a method (including protected methods). + */ + def findMethodInHierarchy( + clazz: Class[_], + methodName: String): Option[java.lang.reflect.Method] = { + var current: Class[_] = clazz + while (current != null) { + try { + val method = current.getDeclaredMethod(methodName) + method.setAccessible(true) + return Some(method) + } catch { + case _: NoSuchMethodException => current = current.getSuperclass + } + } + None + } + + /** + * Extracts file location from Iceberg ContentFile, handling both location() and path(). + * + * Different Iceberg versions expose file paths differently: + * - Newer versions: location() returns String + * - Older versions: path() returns CharSequence + */ + def extractFileLocation(contentFileClass: Class[_], file: Any): Option[String] = { + try { + val locationMethod = contentFileClass.getMethod("location") + Some(locationMethod.invoke(file).asInstanceOf[String]) + } catch { + case _: NoSuchMethodException => + try { + val pathMethod = contentFileClass.getMethod("path") + Some(pathMethod.invoke(file).asInstanceOf[CharSequence].toString) + } catch { + case _: Exception => None + } + case _: Exception => None + } + } + + /** + * Extracts file location from ContentFile instance using dynamic class lookup. + */ + def extractFileLocation(file: Any): Option[String] = { + try { + // scalastyle:off classforname + val contentFileClass = Class.forName(ClassNames.CONTENT_FILE) + // scalastyle:on classforname + extractFileLocation(contentFileClass, file) + } catch { + case _: Exception => None + } + } + + /** + * Gets the Iceberg Table from a SparkScan. + * + * The table() method is protected in SparkScan, requiring reflection to access. + */ + def getTable(scan: Any): Option[Any] = { + findMethodInHierarchy(scan.getClass, "table").flatMap { tableMethod => + try { + Some(tableMethod.invoke(scan)) + } catch { + case e: Exception => + logError( + s"Iceberg reflection failure: Failed to get table from SparkScan: ${e.getMessage}") + None + } + } + } + + /** + * Gets the tasks from a SparkScan. + * + * The tasks() method is protected in SparkScan, requiring reflection to access. + */ + def getTasks(scan: Any): Option[java.util.List[_]] = { + try { + val tasksMethod = scan.getClass.getSuperclass + .getDeclaredMethod("tasks") + tasksMethod.setAccessible(true) + Some(tasksMethod.invoke(scan).asInstanceOf[java.util.List[_]]) + } catch { + case e: Exception => + logError( + s"Iceberg reflection failure: Failed to get tasks from SparkScan: ${e.getMessage}") + None + } + } + + /** + * Gets the filter expressions from a SparkScan. + * + * The filterExpressions() method is protected in SparkScan. + */ + def getFilterExpressions(scan: Any): Option[java.util.List[_]] = { + try { + val filterExpressionsMethod = scan.getClass.getSuperclass.getSuperclass + .getDeclaredMethod("filterExpressions") + filterExpressionsMethod.setAccessible(true) + Some(filterExpressionsMethod.invoke(scan).asInstanceOf[java.util.List[_]]) + } catch { + case e: Exception => + logError( + "Iceberg reflection failure: Failed to get filter expressions from SparkScan: " + + s"${e.getMessage}") + None + } + } + + /** + * Gets the Iceberg table format version. + * + * Tries to get formatVersion() directly from table, falling back to + * operations().current().formatVersion() for older Iceberg versions. + */ + def getFormatVersion(table: Any): Option[Int] = { + try { + val formatVersionMethod = table.getClass.getMethod("formatVersion") + Some(formatVersionMethod.invoke(table).asInstanceOf[Int]) + } catch { + case _: NoSuchMethodException => + try { + // If not directly available, access via operations/metadata + val opsMethod = table.getClass.getMethod("operations") + val ops = opsMethod.invoke(table) + val currentMethod = ops.getClass.getMethod("current") + val metadata = currentMethod.invoke(ops) + val formatVersionMethod = metadata.getClass.getMethod("formatVersion") + Some(formatVersionMethod.invoke(metadata).asInstanceOf[Int]) + } catch { + case e: Exception => + logError(s"Iceberg reflection failure: Failed to get format version: ${e.getMessage}") + None + } + case e: Exception => + logError(s"Iceberg reflection failure: Failed to get format version: ${e.getMessage}") + None + } + } + + /** + * Gets the FileIO from an Iceberg table. + */ + def getFileIO(table: Any): Option[Any] = { + try { + val ioMethod = table.getClass.getMethod("io") + Some(ioMethod.invoke(table)) + } catch { + case e: Exception => + logError(s"Iceberg reflection failure: Failed to get FileIO from table: ${e.getMessage}") + None + } + } + + /** + * Gets the schema from an Iceberg table. + */ + def getSchema(table: Any): Option[Any] = { + try { + val schemaMethod = table.getClass.getMethod("schema") + Some(schemaMethod.invoke(table)) + } catch { + case e: Exception => + logError(s"Iceberg reflection failure: Failed to get schema from table: ${e.getMessage}") + None + } + } + + /** + * Gets the partition spec from an Iceberg table. + */ + def getPartitionSpec(table: Any): Option[Any] = { + try { + val specMethod = table.getClass.getMethod("spec") + Some(specMethod.invoke(table)) + } catch { + case e: Exception => + logError( + s"Iceberg reflection failure: Failed to get partition spec from table: ${e.getMessage}") + None + } + } + + /** + * Gets the table metadata from an Iceberg table. + * + * @param table + * The Iceberg table instance + * @return + * The TableMetadata object from table.operations().current() + */ + def getTableMetadata(table: Any): Option[Any] = { + try { + val operationsMethod = table.getClass.getMethod("operations") + val operations = operationsMethod.invoke(table) + + val currentMethod = operations.getClass.getMethod("current") + Some(currentMethod.invoke(operations)) + } catch { + case e: Exception => + logError(s"Iceberg reflection failure: Failed to get table metadata: ${e.getMessage}") + None + } + } + + /** + * Gets the metadata file location from an Iceberg table. + * + * @param table + * The Iceberg table instance + * @return + * Path to the table metadata file + */ + def getMetadataLocation(table: Any): Option[String] = { + getTableMetadata(table).flatMap { metadata => + try { + val metadataFileLocationMethod = metadata.getClass.getMethod("metadataFileLocation") + Some(metadataFileLocationMethod.invoke(metadata).asInstanceOf[String]) + } catch { + case e: Exception => + logError( + s"Iceberg reflection failure: Failed to get metadata location: ${e.getMessage}") + None + } + } + } + + /** + * Gets the properties map from an Iceberg table's metadata. + * + * @param table + * The Iceberg table instance + * @return + * Map of table properties + */ + def getTableProperties(table: Any): Option[java.util.Map[String, String]] = { + getTableMetadata(table).flatMap { metadata => + try { + val propertiesMethod = metadata.getClass.getMethod("properties") + Some(propertiesMethod.invoke(metadata).asInstanceOf[java.util.Map[String, String]]) + } catch { + case e: Exception => + logError(s"Iceberg reflection failure: Failed to get table properties: ${e.getMessage}") + None + } + } + } + + /** + * Gets delete files from scan tasks. + * + * @param tasks + * List of Iceberg FileScanTask objects + * @return + * List of all delete files across all tasks + * @throws Exception + * if reflection fails (callers must handle appropriately based on context) + */ + def getDeleteFiles(tasks: java.util.List[_]): java.util.List[_] = { + import scala.jdk.CollectionConverters._ + val allDeletes = new java.util.ArrayList[Any]() + + // scalastyle:off classforname + val fileScanTaskClass = Class.forName(ClassNames.FILE_SCAN_TASK) + // scalastyle:on classforname + + tasks.asScala.foreach { task => + val deletes = getDeleteFilesFromTask(task, fileScanTaskClass) + allDeletes.addAll(deletes) + } + + allDeletes + } + + /** + * Gets delete files from a single FileScanTask. + * + * @param task + * An Iceberg FileScanTask object + * @param fileScanTaskClass + * The FileScanTask class (can be obtained via classforname or passed in if already loaded) + * @return + * List of delete files for this task + * @throws Exception + * if reflection fails (callers must handle appropriately based on context) + */ + def getDeleteFilesFromTask(task: Any, fileScanTaskClass: Class[_]): java.util.List[_] = { + val deletesMethod = fileScanTaskClass.getMethod("deletes") + val deletes = deletesMethod.invoke(task).asInstanceOf[java.util.List[_]] + if (deletes == null) new java.util.ArrayList[Any]() else deletes + } + + /** + * Gets equality field IDs from a delete file. + * + * @param deleteFile + * An Iceberg DeleteFile object + * @return + * List of field IDs used in equality deletes, or empty list for position deletes + */ + def getEqualityFieldIds(deleteFile: Any): java.util.List[_] = { + try { + // scalastyle:off classforname + val deleteFileClass = Class.forName(ClassNames.DELETE_FILE) + // scalastyle:on classforname + val equalityFieldIdsMethod = deleteFileClass.getMethod("equalityFieldIds") + val ids = equalityFieldIdsMethod.invoke(deleteFile).asInstanceOf[java.util.List[_]] + if (ids == null) new java.util.ArrayList[Any]() else ids + } catch { + case _: Exception => + // Position delete files return null/empty for equalityFieldIds + new java.util.ArrayList[Any]() + } + } + + /** + * Gets field name and type from schema by field ID. + * + * @param schema + * Iceberg Schema object + * @param fieldId + * Field ID to look up + * @return + * Tuple of (field name, field type string) + */ + def getFieldInfo(schema: Any, fieldId: Int): Option[(String, String)] = { + try { + val findFieldMethod = schema.getClass.getMethod("findField", classOf[Int]) + val field = findFieldMethod.invoke(schema, fieldId.asInstanceOf[AnyRef]) + if (field != null) { + val nameMethod = field.getClass.getMethod("name") + val typeMethod = field.getClass.getMethod("type") + val fieldName = nameMethod.invoke(field).toString + val fieldType = typeMethod.invoke(field).toString + Some((fieldName, fieldType)) + } else { + None + } + } catch { + case e: Exception => + logError( + "Iceberg reflection failure: Failed to get field info for ID " + + s"$fieldId: ${e.getMessage}") + None + } + } + + /** + * Gets the expected schema from a SparkScan. + * + * The expectedSchema() method is protected in SparkScan and returns the Iceberg Schema for this + * scan (which is the snapshot schema for VERSION AS OF queries). + * + * @param scan + * The SparkScan object + * @return + * The expected Iceberg Schema, or None if reflection fails + */ + def getExpectedSchema(scan: Any): Option[Any] = { + findMethodInHierarchy(scan.getClass, "expectedSchema").flatMap { schemaMethod => + try { + Some(schemaMethod.invoke(scan)) + } catch { + case e: Exception => + logError(s"Failed to get expectedSchema from SparkScan: ${e.getMessage}") + None + } + } + } + + /** + * Builds a field ID mapping from an Iceberg schema. + * + * Extracts the mapping of column names to Iceberg field IDs from the schema's columns. This is + * used for schema evolution support where we need to map between column names and their + * corresponding field IDs. + * + * @param schema + * Iceberg Schema object + * @return + * Map from column name to field ID + */ + def buildFieldIdMapping(schema: Any): Map[String, Int] = { + import scala.jdk.CollectionConverters._ + try { + val columnsMethod = schema.getClass.getMethod("columns") + val columns = columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] + + columns.asScala.flatMap { column => + try { + val nameMethod = column.getClass.getMethod("name") + val name = nameMethod.invoke(column).asInstanceOf[String] + + val fieldIdMethod = column.getClass.getMethod("fieldId") + val fieldId = fieldIdMethod.invoke(column).asInstanceOf[Int] + + Some(name -> fieldId) + } catch { + case e: Exception => + logWarning(s"Failed to extract field ID from column: ${e.getMessage}") + None + } + }.toMap + } catch { + case e: Exception => + logWarning(s"Failed to build field ID mapping from schema: ${e.getMessage}") + Map.empty[String, Int] + } + } + + /** + * Validates file formats and filesystem schemes for Iceberg tasks. + * + * Checks that all data files and delete files are Parquet format and use filesystem schemes + * supported by iceberg-rust (file, s3, s3a, gs, gcs, oss, abfss, abfs, wasbs, wasb). + * + * @param tasks + * List of Iceberg FileScanTask objects + * @return + * (allParquet, unsupportedSchemes) where: - allParquet: true if all files are Parquet format + * \- unsupportedSchemes: Set of unsupported filesystem schemes found (empty if all supported) + */ + def validateFileFormatsAndSchemes(tasks: java.util.List[_]): (Boolean, Set[String]) = { + import scala.jdk.CollectionConverters._ + + // scalastyle:off classforname + val contentScanTaskClass = Class.forName(ClassNames.CONTENT_SCAN_TASK) + val contentFileClass = Class.forName(ClassNames.CONTENT_FILE) + // scalastyle:on classforname + + val fileMethod = contentScanTaskClass.getMethod("file") + val formatMethod = contentFileClass.getMethod("format") + val pathMethod = contentFileClass.getMethod("path") + + // Filesystem schemes supported by iceberg-rust + // See: iceberg-rust/crates/iceberg/src/io/storage.rs parse_scheme() + val supportedSchemes = + Set("file", "s3", "s3a", "gs", "gcs", "oss", "abfss", "abfs", "wasbs", "wasb") + + var allParquet = true + val unsupportedSchemes = scala.collection.mutable.Set[String]() + + tasks.asScala.foreach { task => + val dataFile = fileMethod.invoke(task) + val fileFormat = formatMethod.invoke(dataFile).toString + + // Check file format + if (fileFormat != FileFormats.PARQUET) { + allParquet = false + } else { + // Only check filesystem schemes for Parquet files we'll actually process + try { + val filePath = pathMethod.invoke(dataFile).toString + val uri = new java.net.URI(filePath) + val scheme = uri.getScheme + + if (scheme != null && !supportedSchemes.contains(scheme)) { + unsupportedSchemes += scheme + } + } catch { + case _: java.net.URISyntaxException => + // Ignore URI parsing errors - file paths may contain special characters + // If the path is invalid, we'll fail later during actual file access + } + + // Check delete files if they exist + try { + val deletesMethod = task.getClass.getMethod("deletes") + val deleteFiles = deletesMethod.invoke(task).asInstanceOf[java.util.List[_]] + + deleteFiles.asScala.foreach { deleteFile => + extractFileLocation(contentFileClass, deleteFile).foreach { deletePath => + try { + val deleteUri = new java.net.URI(deletePath) + val deleteScheme = deleteUri.getScheme + + if (deleteScheme != null && !supportedSchemes.contains(deleteScheme)) { + unsupportedSchemes += deleteScheme + } + } catch { + case _: java.net.URISyntaxException => + // Ignore URI parsing errors for delete files too + } + } + } + } catch { + case _: Exception => + // Ignore errors accessing delete files - they may not be supported + } + } + } + + (allParquet, unsupportedSchemes.toSet) + } + + /** + * Validates partition column types for compatibility with iceberg-rust. + * + * iceberg-rust's Literal::try_from_json() has incomplete type support: - Binary/fixed types: + * unimplemented - Decimals: limited to precision <= 28 (rust_decimal crate limitation) + * + * @param partitionSpec + * The Iceberg PartitionSpec + * @param schema + * The Iceberg Schema to look up field types + * @return + * List of unsupported partition types (empty if all supported). Each entry is (fieldName, + * typeStr, reason) + */ + def validatePartitionTypes(partitionSpec: Any, schema: Any): List[(String, String, String)] = { + import scala.jdk.CollectionConverters._ + + val fieldsMethod = partitionSpec.getClass.getMethod("fields") + val fields = fieldsMethod.invoke(partitionSpec).asInstanceOf[java.util.List[_]] + + // scalastyle:off classforname + val partitionFieldClass = Class.forName(ClassNames.PARTITION_FIELD) + // scalastyle:on classforname + val sourceIdMethod = partitionFieldClass.getMethod("sourceId") + val findFieldMethod = schema.getClass.getMethod("findField", classOf[Int]) + + val unsupportedTypes = scala.collection.mutable.ListBuffer[(String, String, String)]() + + fields.asScala.foreach { field => + val sourceId = sourceIdMethod.invoke(field).asInstanceOf[Int] + val column = findFieldMethod.invoke(schema, sourceId.asInstanceOf[Object]) + + if (column != null) { + val nameMethod = column.getClass.getMethod("name") + val fieldName = nameMethod.invoke(column).asInstanceOf[String] + + val typeMethod = column.getClass.getMethod("type") + val icebergType = typeMethod.invoke(column) + val typeStr = icebergType.toString + + // iceberg-rust/crates/iceberg/src/spec/values.rs Literal::try_from_json() + if (typeStr.startsWith("decimal(")) { + val precisionStr = typeStr.substring(8, typeStr.indexOf(',')) + val precision = precisionStr.toInt + // rust_decimal crate maximum precision + if (precision > 28) { + unsupportedTypes += (( + fieldName, + typeStr, + s"High-precision decimal (precision=$precision) exceeds maximum of 28 " + + "(rust_decimal limitation)")) + } + } else if (typeStr == "binary" || typeStr.startsWith("fixed[")) { + unsupportedTypes += (( + fieldName, + typeStr, + "Binary/fixed types not yet supported (Literal::try_from_json todo!())")) + } + } + } + + unsupportedTypes.toList + } + + /** + * Checks if tasks have non-identity transforms in their residual expressions. + * + * Residual expressions are filters that must be evaluated after reading data from Parquet. + * iceberg-rust can only handle simple column references in residuals, not transformed columns. + * Transform functions like truncate, bucket, year, month, day, hour require evaluation by + * Spark. + * + * @param tasks + * List of Iceberg FileScanTask objects + * @return + * Some(transformType) if an unsupported transform is found (e.g., "truncate[4]"), None if all + * transforms are identity or no transforms are present + * @throws Exception + * if reflection fails - caller must handle appropriately (fallback in planning, fatal in + * serialization) + */ + def findNonIdentityTransformInResiduals(tasks: java.util.List[_]): Option[String] = { + import scala.jdk.CollectionConverters._ + + // scalastyle:off classforname + val fileScanTaskClass = Class.forName(ClassNames.FILE_SCAN_TASK) + val contentScanTaskClass = Class.forName(ClassNames.CONTENT_SCAN_TASK) + val unboundPredicateClass = Class.forName(ClassNames.UNBOUND_PREDICATE) + // scalastyle:on classforname + + tasks.asScala.foreach { task => + if (fileScanTaskClass.isInstance(task)) { + try { + val residualMethod = contentScanTaskClass.getMethod("residual") + val residual = residualMethod.invoke(task) + + // Check if residual is an UnboundPredicate with a transform + if (unboundPredicateClass.isInstance(residual)) { + val termMethod = unboundPredicateClass.getMethod("term") + val term = termMethod.invoke(residual) + + // Check if term has a transform + try { + val transformMethod = term.getClass.getMethod("transform") + transformMethod.setAccessible(true) + val transform = transformMethod.invoke(term) + val transformStr = transform.toString + + // Only identity transform is supported in residuals + if (transformStr != Transforms.IDENTITY) { + return Some(transformStr) + } + } catch { + case _: NoSuchMethodException => + // No transform method means it's a simple reference - OK + } + } + } catch { + case _: Exception => + // Skip tasks where we can't get residual - they may not have one + } + } + } + None + } +} + +/** + * Pre-extracted Iceberg metadata for native scan execution. + * + * This class holds all metadata extracted from Iceberg during the planning/validation phase in + * CometScanRule. By extracting all metadata once during validation (where reflection failures + * trigger fallback to Spark), we avoid redundant reflection during serialization (where failures + * would be fatal runtime errors). + * + * @param table + * The Iceberg Table object + * @param metadataLocation + * Path to the table metadata file + * @param nameMapping + * Optional name mapping from table properties (for schema evolution) + * @param tasks + * List of FileScanTask objects from Iceberg planning + * @param scanSchema + * The expectedSchema from the SparkScan (for schema evolution / VERSION AS OF) + * @param globalFieldIdMapping + * Mapping from column names to Iceberg field IDs (built from scanSchema) + * @param catalogProperties + * Catalog properties for FileIO (S3 credentials, regions, etc.) + */ +case class CometIcebergNativeScanMetadata( + table: Any, + metadataLocation: String, + nameMapping: Option[String], + tasks: java.util.List[_], + scanSchema: Any, + globalFieldIdMapping: Map[String, Int], + catalogProperties: Map[String, String], + fileFormat: String) + +object CometIcebergNativeScanMetadata extends Logging { + + /** + * Extracts all Iceberg metadata needed for native scan execution. + * + * This method performs all reflection operations once during planning/validation. If any + * reflection operation fails, returns None to trigger fallback to Spark. + * + * @param scan + * The Spark BatchScanExec.scan (SparkBatchQueryScan) + * @param metadataLocation + * Path to the table metadata file (already extracted) + * @param catalogProperties + * Catalog properties for FileIO (already extracted) + * @return + * Some(metadata) if all reflection succeeds, None to trigger fallback + */ + def extract( + scan: Any, + metadataLocation: String, + catalogProperties: Map[String, String]): Option[CometIcebergNativeScanMetadata] = { + import org.apache.comet.iceberg.IcebergReflection._ + + for { + table <- getTable(scan) + tasks <- getTasks(scan) + scanSchema <- getExpectedSchema(scan) + } yield { + // nameMapping is optional - if it fails we just use None + val nameMapping = getTableProperties(table).flatMap { properties => + val nameMappingKey = "schema.name-mapping.default" + if (properties.containsKey(nameMappingKey)) { + Some(properties.get(nameMappingKey)) + } else { + None + } + } + + val globalFieldIdMapping = buildFieldIdMapping(scanSchema) + + // File format is always PARQUET, + // validated in CometScanRule.validateFileFormatsAndSchemes() + // Hardcoded here for extensibility (future ORC/Avro support would add logic here) + val fileFormat = FileFormats.PARQUET + + CometIcebergNativeScanMetadata( + table = table, + metadataLocation = metadataLocation, + nameMapping = nameMapping, + tasks = tasks, + scanSchema = scanSchema, + globalFieldIdMapping = globalFieldIdMapping, + catalogProperties = catalogProperties, + fileFormat = fileFormat) + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 546510f832..4baedc9196 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -196,6 +196,18 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val nativeOp = operator2Proto(scan).get CometNativeScan.createExec(nativeOp, scan) + // Fully native Iceberg scan for V2 (iceberg-rust path) + // Only handle scans with native metadata; SupportsComet scans fall through to isCometScan + // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule + case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => + operator2Proto(scan) match { + case Some(nativeOp) => + CometIcebergNativeScan.createExec(nativeOp, scan) + case None => + // Serialization failed, fall back to CometBatchScanExec + scan + } + // Comet JVM + native scan for V1 and V2 case op if isCometScan(op) => val nativeOp = operator2Proto(op) @@ -807,6 +819,10 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => CometNativeScan.convert(scan, builder, childOp: _*) + // Fully native Iceberg scan for V2 (iceberg-rust path) + case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => + CometIcebergNativeScan.convert(scan, builder, childOp: _*) + case op if isCometSink(op) => val supportedTypes = op.output.forall(a => supportedDataType(a.dataType, allowComplex = true)) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 7a9e8da445..ad86471766 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -44,6 +44,7 @@ import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} import org.apache.comet.CometConf._ import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, isCometScanEnabled, withInfo, withInfos} import org.apache.comet.DataTypeSupport.isComplexType +import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} @@ -265,8 +266,8 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com withInfos(scanExec, fallbackReasons.toSet) } - // Iceberg scan - case s: SupportsComet => + // Iceberg scan - patched version implementing SupportsComet interface + case s: SupportsComet if !COMET_ICEBERG_NATIVE_ENABLED.get() => val fallbackReasons = new ListBuffer[String]() if (!s.isCometEnabled) { @@ -292,6 +293,281 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com withInfos(scanExec, fallbackReasons.toSet) } + // Iceberg scan - detected by class name (works with unpatched Iceberg) + case _ + if scanExec.scan.getClass.getName == + "org.apache.iceberg.spark.source.SparkBatchQueryScan" => + val fallbackReasons = new ListBuffer[String]() + + // Native Iceberg scan requires both configs to be enabled + if (!COMET_ICEBERG_NATIVE_ENABLED.get()) { + fallbackReasons += "Native Iceberg scan disabled because " + + s"${COMET_ICEBERG_NATIVE_ENABLED.key} is not enabled" + return withInfos(scanExec, fallbackReasons.toSet) + } + + if (!COMET_EXEC_ENABLED.get()) { + fallbackReasons += "Native Iceberg scan disabled because " + + s"${COMET_EXEC_ENABLED.key} is not enabled" + return withInfos(scanExec, fallbackReasons.toSet) + } + + val typeChecker = CometScanTypeChecker(SCAN_NATIVE_DATAFUSION) + val schemaSupported = + typeChecker.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons) + + if (!schemaSupported) { + fallbackReasons += "Comet extension is not enabled for " + + s"${scanExec.scan.getClass.getSimpleName}: Schema not supported" + } + + // Extract all Iceberg metadata once using reflection. + // If any required reflection fails, this returns None, and we fall back to Spark. + // First get metadataLocation and catalogProperties which are needed by the factory. + val metadataLocationOpt = IcebergReflection + .getTable(scanExec.scan) + .flatMap(IcebergReflection.getMetadataLocation) + + val metadataOpt = metadataLocationOpt.flatMap { metadataLocation => + try { + val session = org.apache.spark.sql.SparkSession.active + val hadoopConf = session.sessionState.newHadoopConf() + val metadataUri = new java.net.URI(metadataLocation) + val hadoopS3Options = NativeConfig.extractObjectStoreOptions(hadoopConf, metadataUri) + val catalogProperties = + org.apache.comet.serde.operator.CometIcebergNativeScan + .hadoopToIcebergS3Properties(hadoopS3Options) + + CometIcebergNativeScanMetadata + .extract(scanExec.scan, metadataLocation, catalogProperties) + } catch { + case e: Exception => + logError( + s"Failed to extract catalog properties from Iceberg scan: ${e.getMessage}", + e) + None + } + } + + // If metadata extraction failed, fall back to Spark + val metadata = metadataOpt match { + case Some(m) => m + case None => + fallbackReasons += "Failed to extract Iceberg metadata via reflection" + return withInfos(scanExec, fallbackReasons.toSet) + } + + // Now perform all validation using the pre-extracted metadata + // Check if table uses a FileIO implementation compatible with iceberg-rust + val fileIOCompatible = IcebergReflection.getFileIO(metadata.table) match { + case Some(fileIO) => + val fileIOClassName = fileIO.getClass.getName + if (fileIOClassName == "org.apache.iceberg.inmemory.InMemoryFileIO") { + fallbackReasons += "Comet does not support InMemoryFileIO table locations" + false + } else { + true + } + case None => + fallbackReasons += "Could not check FileIO compatibility" + false + } + + // Check Iceberg table format version + val formatVersionSupported = IcebergReflection.getFormatVersion(metadata.table) match { + case Some(formatVersion) => + if (formatVersion > 2) { + fallbackReasons += "Iceberg table format version " + + s"$formatVersion is not supported. " + + "Comet only supports Iceberg table format V1 and V2" + false + } else { + true + } + case None => + fallbackReasons += "Could not verify Iceberg table format version" + false + } + + // Check if all files are Parquet format and use supported filesystem schemes + val (allParquetFiles, unsupportedSchemes) = + IcebergReflection.validateFileFormatsAndSchemes(metadata.tasks) + + val allSupportedFilesystems = if (unsupportedSchemes.isEmpty) { + true + } else { + fallbackReasons += "Iceberg scan contains files with unsupported filesystem " + + s"schemes: ${unsupportedSchemes.mkString(", ")}. " + + "Comet only supports: file, s3, s3a, gs, gcs, oss, abfss, abfs, wasbs, wasb" + false + } + + if (!allParquetFiles) { + fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " + + "Comet only supports Parquet files in Iceberg tables" + } + + // Partition values are deserialized via iceberg-rust's Literal::try_from_json() + // which has incomplete type support (binary/fixed unimplemented, decimals limited) + val partitionTypesSupported = (for { + partitionSpec <- IcebergReflection.getPartitionSpec(metadata.table) + } yield { + val unsupportedTypes = + IcebergReflection.validatePartitionTypes(partitionSpec, metadata.scanSchema) + + if (unsupportedTypes.nonEmpty) { + unsupportedTypes.foreach { case (fieldName, typeStr, reason) => + fallbackReasons += + s"Partition column '$fieldName' with type $typeStr is not yet supported by " + + s"iceberg-rust: $reason" + } + false + } else { + true + } + }).getOrElse { + // Fall back to Spark if reflection fails - cannot verify safety + val msg = + "Iceberg reflection failure: Could not verify partition types compatibility" + logError(msg) + fallbackReasons += msg + false + } + + // Get filter expressions for complex predicates check + val filterExpressionsOpt = IcebergReflection.getFilterExpressions(scanExec.scan) + + // IS NULL/NOT NULL on complex types fail because iceberg-rust's accessor creation + // only handles primitive fields. Nested field filters work because Iceberg Java + // pre-binds them to field IDs. Element/key access filters don't push down to FileScanTasks. + val complexTypePredicatesSupported = filterExpressionsOpt + .map { filters => + // Empty filters can't trigger accessor issues + if (filters.isEmpty) { + true + } else { + val readSchema = scanExec.scan.readSchema() + + // Identify complex type columns that would trigger accessor creation failures + val complexColumns = readSchema + .filter(field => isComplexType(field.dataType)) + .map(_.name) + .toSet + + // Detect IS NULL/NOT NULL on complex columns (pattern: is_null(ref(name="col"))) + // Nested field filters use different patterns and don't trigger this issue + val hasComplexNullCheck = filters.asScala.exists { expr => + val exprStr = expr.toString + val isNullCheck = exprStr.contains("is_null") || exprStr.contains("not_null") + if (isNullCheck) { + complexColumns.exists { colName => + exprStr.contains(s"""ref(name="$colName")""") + } + } else { + false + } + } + + if (hasComplexNullCheck) { + fallbackReasons += "IS NULL / IS NOT NULL predicates on complex type columns " + + "(struct/array/map) are not yet supported by iceberg-rust " + + "(nested field filters like address.city = 'NYC' are supported)" + false + } else { + true + } + } + } + .getOrElse { + // Fall back to Spark if reflection fails - cannot verify safety + val msg = + "Iceberg reflection failure: Could not check for complex type predicates" + logError(msg) + fallbackReasons += msg + false + } + + // Check for unsupported transform functions in residual expressions + // iceberg-rust can only handle identity transforms in residuals; all other transforms + // (truncate, bucket, year, month, day, hour) must fall back to Spark + val transformFunctionsSupported = + try { + IcebergReflection.findNonIdentityTransformInResiduals(metadata.tasks) match { + case Some(transformType) => + // Found unsupported transform + fallbackReasons += + s"Iceberg transform function '$transformType' in residual expression " + + "is not yet supported by iceberg-rust. " + + "Only identity transforms are supported." + false + case None => + // No unsupported transforms found - safe to use native execution + true + } + } catch { + case e: Exception => + // Reflection failure - cannot verify safety, must fall back + fallbackReasons += "Iceberg reflection failure: Could not check for " + + s"transform functions in residuals: ${e.getMessage}" + false + } + + // Check for unsupported struct types in delete files + val deleteFileTypesSupported = { + var hasUnsupportedDeletes = false + + try { + val deleteFiles = IcebergReflection.getDeleteFiles(metadata.tasks) + + if (!deleteFiles.isEmpty) { + deleteFiles.asScala.foreach { deleteFile => + val equalityFieldIds = IcebergReflection.getEqualityFieldIds(deleteFile) + + if (!equalityFieldIds.isEmpty) { + // Look up field types + equalityFieldIds.asScala.foreach { fieldId => + val fieldInfo = IcebergReflection.getFieldInfo( + metadata.scanSchema, + fieldId.asInstanceOf[Int]) + fieldInfo match { + case Some((fieldName, fieldType)) => + if (fieldType.contains("struct")) { + hasUnsupportedDeletes = true + fallbackReasons += + s"Equality delete on unsupported column type '$fieldName' " + + s"($fieldType) is not yet supported by iceberg-rust. " + + "Struct types in equality deletes " + + "require datum conversion support that is not yet implemented." + } + case None => + } + } + } + } + } + } catch { + case e: Exception => + // Reflection failure means we cannot verify safety - must fall back + hasUnsupportedDeletes = true + fallbackReasons += "Iceberg reflection failure: Could not verify delete file " + + s"types for safety: ${e.getMessage}" + } + + !hasUnsupportedDeletes + } + + if (schemaSupported && fileIOCompatible && formatVersionSupported && allParquetFiles && + allSupportedFilesystems && partitionTypesSupported && + complexTypePredicatesSupported && transformFunctionsSupported && + deleteFileTypesSupported) { + CometBatchScanExec( + scanExec.clone().asInstanceOf[BatchScanExec], + runtimeFilters = scanExec.runtimeFilters, + nativeIcebergScanMetadata = Some(metadata)) + } else { + withInfos(scanExec, fallbackReasons.toSet) + } + case other => withInfo( scanExec, diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala new file mode 100644 index 0000000000..24676bfe3c --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala @@ -0,0 +1,861 @@ +/* + * 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.serde.operator + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeExec} +import org.apache.spark.sql.types._ + +import org.apache.comet.ConfigEntry +import org.apache.comet.iceberg.IcebergReflection +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.OperatorOuterClass.{Operator, SparkStructField} +import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType} + +object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] with Logging { + + override def enabledConfig: Option[ConfigEntry[Boolean]] = None + + /** + * Constants specific to Iceberg expression conversion (not in shared IcebergReflection). + */ + private object Constants { + // Iceberg expression operation names + object Operations { + val IS_NULL = "IS_NULL" + val IS_NOT_NULL = "IS_NOT_NULL" + val NOT_NULL = "NOT_NULL" + val EQ = "EQ" + val NOT_EQ = "NOT_EQ" + val LT = "LT" + val LT_EQ = "LT_EQ" + val GT = "GT" + val GT_EQ = "GT_EQ" + val IN = "IN" + val NOT_IN = "NOT_IN" + } + + // Iceberg expression class name suffixes + object ExpressionTypes { + val UNBOUND_PREDICATE = "UnboundPredicate" + val AND = "And" + val OR = "Or" + val NOT = "Not" + } + } + + /** + * Helper to extract a literal from an Iceberg expression and build a binary predicate. + */ + private def buildBinaryPredicate( + exprClass: Class[_], + icebergExpr: Any, + attribute: Attribute, + builder: (Expression, Expression) => Expression): Option[Expression] = { + try { + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attribute.dataType) + Some(builder(attribute, value)) + } catch { + case _: Exception => None + } + } + + /** + * Serializes delete files from an Iceberg FileScanTask. + * + * Extracts delete files (position deletes and equality deletes) from a FileScanTask and adds + * them to the task builder. Delete files are used for Iceberg's merge-on-read approach where + * updates and deletes are stored separately from data files. + */ + private def serializeDeleteFiles( + task: Any, + contentFileClass: Class[_], + fileScanTaskClass: Class[_], + taskBuilder: OperatorOuterClass.IcebergFileScanTask.Builder): Unit = { + try { + // scalastyle:off classforname + val deleteFileClass = Class.forName(IcebergReflection.ClassNames.DELETE_FILE) + // scalastyle:on classforname + + val deletes = IcebergReflection.getDeleteFilesFromTask(task, fileScanTaskClass) + + deletes.asScala.foreach { deleteFile => + try { + IcebergReflection + .extractFileLocation(contentFileClass, deleteFile) + .foreach { deletePath => + val deleteBuilder = + OperatorOuterClass.IcebergDeleteFile.newBuilder() + deleteBuilder.setFilePath(deletePath) + + val contentType = + try { + val contentMethod = deleteFileClass.getMethod("content") + val content = contentMethod.invoke(deleteFile) + content.toString match { + case IcebergReflection.ContentTypes.POSITION_DELETES => + IcebergReflection.ContentTypes.POSITION_DELETES + case IcebergReflection.ContentTypes.EQUALITY_DELETES => + IcebergReflection.ContentTypes.EQUALITY_DELETES + case other => other + } + } catch { + case _: Exception => + IcebergReflection.ContentTypes.POSITION_DELETES + } + deleteBuilder.setContentType(contentType) + + val specId = + try { + val specIdMethod = deleteFileClass.getMethod("specId") + specIdMethod.invoke(deleteFile).asInstanceOf[Int] + } catch { + case _: Exception => + 0 + } + deleteBuilder.setPartitionSpecId(specId) + + try { + val equalityIdsMethod = + deleteFileClass.getMethod("equalityFieldIds") + val equalityIds = equalityIdsMethod + .invoke(deleteFile) + .asInstanceOf[java.util.List[Integer]] + equalityIds.forEach(id => deleteBuilder.addEqualityIds(id)) + } catch { + case _: Exception => + } + + taskBuilder.addDeleteFiles(deleteBuilder.build()) + } + } catch { + case e: Exception => + logWarning(s"Failed to serialize delete file: ${e.getMessage}") + } + } + } catch { + case e: Exception => + val msg = + "Iceberg reflection failure: Failed to extract deletes from FileScanTask: " + + s"${e.getMessage}" + logError(msg) + throw new RuntimeException(msg, e) + } + } + + /** + * Serializes partition spec and data from an Iceberg FileScanTask. + * + * Extracts partition specification (field definitions and transforms) and partition data + * (actual values) from the task. This information is used by the native execution engine to + * build a constants_map for identity-transformed partition columns and to handle + * partition-level filtering. + */ + private def serializePartitionData( + task: Any, + contentScanTaskClass: Class[_], + fileScanTaskClass: Class[_], + taskBuilder: OperatorOuterClass.IcebergFileScanTask.Builder): Unit = { + try { + val specMethod = fileScanTaskClass.getMethod("spec") + val spec = specMethod.invoke(task) + + if (spec != null) { + // Serialize the entire PartitionSpec to JSON (includes spec-id) + try { + // scalastyle:off classforname + val partitionSpecParserClass = + Class.forName(IcebergReflection.ClassNames.PARTITION_SPEC_PARSER) + val toJsonMethod = partitionSpecParserClass.getMethod( + "toJson", + Class.forName(IcebergReflection.ClassNames.PARTITION_SPEC)) + // scalastyle:on classforname + val partitionSpecJson = toJsonMethod + .invoke(null, spec) + .asInstanceOf[String] + taskBuilder.setPartitionSpecJson(partitionSpecJson) + } catch { + case e: Exception => + logWarning(s"Failed to serialize partition spec to JSON: ${e.getMessage}") + } + + // Get partition data from the task (via file().partition()) + val partitionMethod = contentScanTaskClass.getMethod("partition") + val partitionData = partitionMethod.invoke(task) + + if (partitionData != null) { + // Get the partition type/schema from the spec + val partitionTypeMethod = spec.getClass.getMethod("partitionType") + val partitionType = partitionTypeMethod.invoke(spec) + + // Check if partition type has any fields before serializing + val fieldsMethod = partitionType.getClass.getMethod("fields") + val fields = fieldsMethod + .invoke(partitionType) + .asInstanceOf[java.util.List[_]] + + // Helper to get field type string (shared by both type and data serialization) + def getFieldType(field: Any): String = { + val typeMethod = field.getClass.getMethod("type") + typeMethod.invoke(field).toString + } + + // Only serialize partition type if there are actual partition fields + if (!fields.isEmpty) { + try { + // Manually build StructType JSON to match iceberg-rust expectations. + // Using Iceberg's SchemaParser.toJson() would include schema-level + // metadata (e.g., "schema-id") that iceberg-rust's StructType + // deserializer rejects. We need pure StructType format: + // {"type":"struct","fields":[...]} + import org.json4s.JsonDSL._ + import org.json4s.jackson.JsonMethods._ + + // Filter out fields with unknown types (dropped partition fields). + // Unknown type fields represent partition columns that have been dropped + // from the schema. Per the Iceberg spec, unknown type fields are not + // stored in data files and iceberg-rust doesn't support deserializing + // them. Since these columns are dropped, we don't need to expose their + // partition values when reading. + val fieldsJson = fields.asScala.flatMap { field => + val fieldTypeStr = getFieldType(field) + + // Skip fields with unknown type (dropped partition columns) + if (fieldTypeStr == IcebergReflection.TypeNames.UNKNOWN) { + None + } else { + val fieldIdMethod = field.getClass.getMethod("fieldId") + val fieldId = fieldIdMethod.invoke(field).asInstanceOf[Int] + + val nameMethod = field.getClass.getMethod("name") + val fieldName = nameMethod.invoke(field).asInstanceOf[String] + + val isOptionalMethod = field.getClass.getMethod("isOptional") + val isOptional = + isOptionalMethod.invoke(field).asInstanceOf[Boolean] + val required = !isOptional + + Some( + ("id" -> fieldId) ~ + ("name" -> fieldName) ~ + ("required" -> required) ~ + ("type" -> fieldTypeStr)) + } + }.toList + + // Only serialize if we have non-unknown fields + if (fieldsJson.nonEmpty) { + val partitionTypeJson = compact( + render( + ("type" -> "struct") ~ + ("fields" -> fieldsJson))) + + taskBuilder.setPartitionTypeJson(partitionTypeJson) + } + } catch { + case e: Exception => + logWarning(s"Failed to serialize partition type to JSON: ${e.getMessage}") + } + } + + // Serialize partition data to JSON for iceberg-rust's constants_map. + // The native execution engine uses partition_data_json + + // partition_type_json to build a constants_map, which is the primary + // mechanism for providing partition values to identity-transformed + // partition columns. Non-identity transforms (bucket, truncate, days, + // etc.) read values from data files. + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + + // Filter out fields with unknown type (same as partition type filtering) + val partitionDataMap: Map[String, JValue] = + fields.asScala.zipWithIndex.flatMap { case (field, idx) => + val fieldTypeStr = getFieldType(field) + + // Skip fields with unknown type (dropped partition columns) + if (fieldTypeStr == IcebergReflection.TypeNames.UNKNOWN) { + None + } else { + val fieldIdMethod = field.getClass.getMethod("fieldId") + val fieldId = fieldIdMethod.invoke(field).asInstanceOf[Int] + + val getMethod = + partitionData.getClass.getMethod("get", classOf[Int], classOf[Class[_]]) + val value = getMethod.invoke(partitionData, Integer.valueOf(idx), classOf[Object]) + + val jsonValue: JValue = if (value == null) { + JNull + } else { + value match { + case s: String => JString(s) + // NaN/Infinity are not valid JSON number literals per the + // JSON spec. Serialize as strings (e.g., "NaN", "Infinity") + // which are valid JSON and can be parsed by Rust's + // f32/f64::from_str(). + case f: java.lang.Float if f.isNaN || f.isInfinite => + JString(f.toString) + case d: java.lang.Double if d.isNaN || d.isInfinite => + JString(d.toString) + case n: Number => JDecimal(BigDecimal(n.toString)) + case b: java.lang.Boolean => + JBool(b.booleanValue()) + case other => JString(other.toString) + } + } + + Some(fieldId.toString -> jsonValue) + } + }.toMap + + // Only serialize partition data if we have non-unknown fields + if (partitionDataMap.nonEmpty) { + val partitionJson = compact(render(JObject(partitionDataMap.toList))) + taskBuilder.setPartitionDataJson(partitionJson) + } + } + } + } catch { + case e: Exception => + val msg = + "Iceberg reflection failure: Failed to extract partition data from FileScanTask: " + + s"${e.getMessage}" + logError(msg, e) + throw new RuntimeException(msg, e) + } + } + + /** + * Transforms Hadoop S3A configuration keys to Iceberg FileIO property keys. + * + * Iceberg-rust's FileIO expects Iceberg-format keys (e.g., s3.access-key-id), not Hadoop keys + * (e.g., fs.s3a.access.key). This function converts Hadoop keys extracted from Spark's + * configuration to the format expected by iceberg-rust. + */ + def hadoopToIcebergS3Properties(hadoopProps: Map[String, String]): Map[String, String] = { + hadoopProps.flatMap { case (key, value) => + key match { + // Global S3A configuration keys + case "fs.s3a.access.key" => Some("s3.access-key-id" -> value) + case "fs.s3a.secret.key" => Some("s3.secret-access-key" -> value) + case "fs.s3a.endpoint" => Some("s3.endpoint" -> value) + case "fs.s3a.path.style.access" => Some("s3.path-style-access" -> value) + case "fs.s3a.endpoint.region" => Some("s3.region" -> value) + + // Per-bucket configuration keys (e.g., fs.s3a.bucket.mybucket.access.key) + // Extract bucket name and property, then transform to s3.* format + case k if k.startsWith("fs.s3a.bucket.") => + val parts = k.stripPrefix("fs.s3a.bucket.").split("\\.", 2) + if (parts.length == 2) { + val bucket = parts(0) + val property = parts(1) + property match { + case "access.key" => Some(s"s3.bucket.$bucket.access-key-id" -> value) + case "secret.key" => Some(s"s3.bucket.$bucket.secret-access-key" -> value) + case "endpoint" => Some(s"s3.bucket.$bucket.endpoint" -> value) + case "path.style.access" => Some(s"s3.bucket.$bucket.path-style-access" -> value) + case "endpoint.region" => Some(s"s3.bucket.$bucket.region" -> value) + case _ => None + } + } else { + None + } + + // Pass through any keys that are already in Iceberg format + case k if k.startsWith("s3.") => Some(key -> value) + + // Ignore all other keys + case _ => None + } + } + } + + /** + * Converts Iceberg Expression objects to Spark Catalyst expressions. + * + * This is used to extract per-file residual expressions from Iceberg FileScanTasks. Residuals + * are created by Iceberg's ResidualEvaluator through partial evaluation of scan filters against + * each file's partition data. These residuals enable row-group level filtering in the Parquet + * reader. + * + * The conversion uses reflection because Iceberg expressions are not directly accessible from + * Spark's classpath during query planning. + */ + def convertIcebergExpression(icebergExpr: Any, output: Seq[Attribute]): Option[Expression] = { + try { + val exprClass = icebergExpr.getClass + val attributeMap = output.map(attr => attr.name -> attr).toMap + + // Check for UnboundPredicate + if (exprClass.getName.endsWith(Constants.ExpressionTypes.UNBOUND_PREDICATE)) { + val opMethod = exprClass.getMethod("op") + val termMethod = exprClass.getMethod("term") + val operation = opMethod.invoke(icebergExpr) + val term = termMethod.invoke(icebergExpr) + + // Get column name from term + val refMethod = term.getClass.getMethod("ref") + val ref = refMethod.invoke(term) + val nameMethod = ref.getClass.getMethod("name") + val columnName = nameMethod.invoke(ref).asInstanceOf[String] + + val attr = attributeMap.get(columnName) + + val opName = operation.toString + + attr.flatMap { attribute => + opName match { + case Constants.Operations.IS_NULL => + Some(IsNull(attribute)) + + case Constants.Operations.IS_NOT_NULL | Constants.Operations.NOT_NULL => + Some(IsNotNull(attribute)) + + case Constants.Operations.EQ => + buildBinaryPredicate(exprClass, icebergExpr, attribute, EqualTo) + + case Constants.Operations.NOT_EQ => + buildBinaryPredicate( + exprClass, + icebergExpr, + attribute, + (a, v) => Not(EqualTo(a, v))) + + case Constants.Operations.LT => + buildBinaryPredicate(exprClass, icebergExpr, attribute, LessThan) + + case Constants.Operations.LT_EQ => + buildBinaryPredicate(exprClass, icebergExpr, attribute, LessThanOrEqual) + + case Constants.Operations.GT => + buildBinaryPredicate(exprClass, icebergExpr, attribute, GreaterThan) + + case Constants.Operations.GT_EQ => + buildBinaryPredicate(exprClass, icebergExpr, attribute, GreaterThanOrEqual) + + case Constants.Operations.IN => + val literalsMethod = exprClass.getMethod("literals") + val literals = literalsMethod.invoke(icebergExpr).asInstanceOf[java.util.List[_]] + val values = + literals.asScala.map(lit => convertIcebergLiteral(lit, attribute.dataType)) + Some(In(attribute, values.toSeq)) + + case Constants.Operations.NOT_IN => + val literalsMethod = exprClass.getMethod("literals") + val literals = literalsMethod.invoke(icebergExpr).asInstanceOf[java.util.List[_]] + val values = + literals.asScala.map(lit => convertIcebergLiteral(lit, attribute.dataType)) + Some(Not(In(attribute, values.toSeq))) + + case _ => + None + } + } + } else if (exprClass.getName.endsWith(Constants.ExpressionTypes.AND)) { + val leftMethod = exprClass.getMethod("left") + val rightMethod = exprClass.getMethod("right") + val left = leftMethod.invoke(icebergExpr) + val right = rightMethod.invoke(icebergExpr) + + (convertIcebergExpression(left, output), convertIcebergExpression(right, output)) match { + case (Some(l), Some(r)) => Some(And(l, r)) + case _ => None + } + } else if (exprClass.getName.endsWith(Constants.ExpressionTypes.OR)) { + val leftMethod = exprClass.getMethod("left") + val rightMethod = exprClass.getMethod("right") + val left = leftMethod.invoke(icebergExpr) + val right = rightMethod.invoke(icebergExpr) + + (convertIcebergExpression(left, output), convertIcebergExpression(right, output)) match { + case (Some(l), Some(r)) => Some(Or(l, r)) + case _ => None + } + } else if (exprClass.getName.endsWith(Constants.ExpressionTypes.NOT)) { + val childMethod = exprClass.getMethod("child") + val child = childMethod.invoke(icebergExpr) + + convertIcebergExpression(child, output).map(Not) + } else { + None + } + } catch { + case _: Exception => + None + } + } + + /** + * Converts an Iceberg Literal to a Spark Literal + */ + private def convertIcebergLiteral(icebergLiteral: Any, sparkType: DataType): Literal = { + // Load Literal interface to get value() method (use interface to avoid package-private issues) + // scalastyle:off classforname + val literalClass = Class.forName(IcebergReflection.ClassNames.LITERAL) + // scalastyle:on classforname + val valueMethod = literalClass.getMethod("value") + val value = valueMethod.invoke(icebergLiteral) + + // Convert Java types to Spark internal types + val sparkValue = (value, sparkType) match { + case (s: String, _: StringType) => + org.apache.spark.unsafe.types.UTF8String.fromString(s) + case (v, _) => v + } + + Literal(sparkValue, sparkType) + } + + /** + * Serializes a CometBatchScanExec wrapping an Iceberg SparkBatchQueryScan to protobuf. + * + * Uses pre-extracted metadata from CometScanRule to avoid redundant reflection operations. All + * reflection and validation was done during planning, so serialization failures here would + * indicate a programming error rather than an expected fallback condition. + */ + override def convert( + scan: CometBatchScanExec, + builder: Operator.Builder, + childOp: Operator*): Option[OperatorOuterClass.Operator] = { + val icebergScanBuilder = OperatorOuterClass.IcebergScan.newBuilder() + + // Get pre-extracted metadata from planning phase + // If metadata is None, this is a programming error - metadata should have been extracted + // in CometScanRule before creating CometBatchScanExec + val metadata = scan.nativeIcebergScanMetadata.getOrElse { + logError( + "Programming error: CometBatchScanExec.nativeIcebergScanMetadata is None. " + + "Metadata should have been extracted in CometScanRule.") + return None + } + + // Use pre-extracted metadata (no reflection needed) + icebergScanBuilder.setMetadataLocation(metadata.metadataLocation) + + metadata.catalogProperties.foreach { case (key, value) => + icebergScanBuilder.putCatalogProperties(key, value) + } + + // Set required_schema from output + scan.output.foreach { attr => + val field = SparkStructField + .newBuilder() + .setName(attr.name) + .setNullable(attr.nullable) + serializeDataType(attr.dataType).foreach(field.setDataType) + icebergScanBuilder.addRequiredSchema(field.build()) + } + + // Extract FileScanTasks from the InputPartitions in the RDD + try { + scan.wrapped.inputRDD match { + case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => + val partitions = rdd.partitions + partitions.foreach { partition => + val partitionBuilder = OperatorOuterClass.IcebergFilePartition.newBuilder() + + val inputPartitions = partition + .asInstanceOf[org.apache.spark.sql.execution.datasources.v2.DataSourceRDDPartition] + .inputPartitions + + inputPartitions.foreach { inputPartition => + val inputPartClass = inputPartition.getClass + + try { + val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") + taskGroupMethod.setAccessible(true) + val taskGroup = taskGroupMethod.invoke(inputPartition) + + val taskGroupClass = taskGroup.getClass + val tasksMethod = taskGroupClass.getMethod("tasks") + val tasksCollection = + tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] + + tasksCollection.asScala.foreach { task => + try { + val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() + + // scalastyle:off classforname + val contentScanTaskClass = + Class.forName(IcebergReflection.ClassNames.CONTENT_SCAN_TASK) + val fileScanTaskClass = + Class.forName(IcebergReflection.ClassNames.FILE_SCAN_TASK) + val contentFileClass = + Class.forName(IcebergReflection.ClassNames.CONTENT_FILE) + // scalastyle:on classforname + + val fileMethod = contentScanTaskClass.getMethod("file") + val dataFile = fileMethod.invoke(task) + + val filePathOpt = + IcebergReflection.extractFileLocation(contentFileClass, dataFile) + + filePathOpt match { + case Some(filePath) => + taskBuilder.setDataFilePath(filePath) + case None => + val msg = + "Iceberg reflection failure: Cannot extract file path from data file" + logError(msg) + throw new RuntimeException(msg) + } + + // Extract partition values for Hive-style partitioning + var partitionJsonOpt: Option[String] = None + try { + val partitionMethod = contentFileClass.getMethod("partition") + val partitionStruct = partitionMethod.invoke(dataFile) + + if (partitionStruct != null) { + // scalastyle:off classforname + val structLikeClass = + Class.forName(IcebergReflection.ClassNames.STRUCT_LIKE) + // scalastyle:on classforname + val sizeMethod = structLikeClass.getMethod("size") + val getMethod = + structLikeClass.getMethod("get", classOf[Int], classOf[Class[_]]) + + val partitionSize = + sizeMethod.invoke(partitionStruct).asInstanceOf[Int] + + if (partitionSize > 0) { + // Get the partition spec directly from the task + // scalastyle:off classforname + val partitionScanTaskClass = + Class.forName(IcebergReflection.ClassNames.PARTITION_SCAN_TASK) + // scalastyle:on classforname + val specMethod = partitionScanTaskClass.getMethod("spec") + val partitionSpec = specMethod.invoke(task) + + // Build JSON representation of partition values using json4s + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + + val partitionMap = scala.collection.mutable.Map[String, JValue]() + + if (partitionSpec != null) { + // Get the list of partition fields from the spec + val fieldsMethod = partitionSpec.getClass.getMethod("fields") + val fields = fieldsMethod + .invoke(partitionSpec) + .asInstanceOf[java.util.List[_]] + + for (i <- 0 until partitionSize) { + val value = + getMethod.invoke(partitionStruct, Int.box(i), classOf[Object]) + + // Get the partition field and check its transform type + val partitionField = fields.get(i) + + // Only inject partition values for IDENTITY transforms + val transformMethod = + partitionField.getClass.getMethod("transform") + val transform = transformMethod.invoke(partitionField) + val isIdentity = + transform.toString == IcebergReflection.Transforms.IDENTITY + + if (isIdentity) { + // Get the source field ID + val sourceIdMethod = + partitionField.getClass.getMethod("sourceId") + val sourceFieldId = + sourceIdMethod.invoke(partitionField).asInstanceOf[Int] + + // Convert value to appropriate JValue type + val jsonValue: JValue = if (value == null) { + JNull + } else { + value match { + case s: String => JString(s) + case i: java.lang.Integer => JInt(BigInt(i.intValue())) + case l: java.lang.Long => JInt(BigInt(l.longValue())) + case d: java.lang.Double => JDouble(d.doubleValue()) + case f: java.lang.Float => JDouble(f.doubleValue()) + case b: java.lang.Boolean => JBool(b.booleanValue()) + case n: Number => JDecimal(BigDecimal(n.toString)) + case other => JString(other.toString) + } + } + + partitionMap(sourceFieldId.toString) = jsonValue + } + } + } + + val partitionJson = compact(render(JObject(partitionMap.toList))) + partitionJsonOpt = Some(partitionJson) + } + } + } catch { + case e: Exception => + logWarning( + s"Failed to extract partition values from DataFile: ${e.getMessage}") + } + + val startMethod = contentScanTaskClass.getMethod("start") + val start = startMethod.invoke(task).asInstanceOf[Long] + taskBuilder.setStart(start) + + val lengthMethod = contentScanTaskClass.getMethod("length") + val length = lengthMethod.invoke(task).asInstanceOf[Long] + taskBuilder.setLength(length) + + try { + // Equality deletes require the full table schema to resolve field IDs, + // even for columns not in the projection. Schema evolution requires + // using the snapshot's schema to correctly read old data files. + // These requirements conflict, so we choose based on delete presence. + + val taskSchemaMethod = fileScanTaskClass.getMethod("schema") + val taskSchema = taskSchemaMethod.invoke(task) + + val deletes = + IcebergReflection.getDeleteFilesFromTask(task, fileScanTaskClass) + val hasDeletes = !deletes.isEmpty + + // Use pre-extracted scanSchema for schema evolution support + val schema: AnyRef = + if (hasDeletes) { + taskSchema + } else { + metadata.scanSchema.asInstanceOf[AnyRef] + } + + // scalastyle:off classforname + val schemaParserClass = + Class.forName(IcebergReflection.ClassNames.SCHEMA_PARSER) + val schemaClass = Class.forName(IcebergReflection.ClassNames.SCHEMA) + // scalastyle:on classforname + val toJsonMethod = schemaParserClass.getMethod("toJson", schemaClass) + toJsonMethod.setAccessible(true) + val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] + + taskBuilder.setSchemaJson(schemaJson) + + // Build field ID mapping from the schema we're using + val nameToFieldId = IcebergReflection.buildFieldIdMapping(schema) + + // Extract project_field_ids for scan.output columns. + // For schema evolution: try task schema first, then fall back to + // global scan schema (pre-extracted in metadata). + scan.output.foreach { attr => + val fieldId = nameToFieldId + .get(attr.name) + .orElse(metadata.globalFieldIdMapping.get(attr.name)) + + fieldId match { + case Some(id) => + taskBuilder.addProjectFieldIds(id) + case None => + logWarning( + s"Column '${attr.name}' not found in task or scan schema," + + "skipping projection") + } + } + } catch { + case e: Exception => + val msg = + "Iceberg reflection failure: " + + "Failed to extract schema from FileScanTask: " + + s"${e.getMessage}" + logError(msg) + throw new RuntimeException(msg, e) + } + + taskBuilder.setDataFileFormat(metadata.fileFormat) + + // Serialize delete files (position deletes and equality deletes) + serializeDeleteFiles(task, contentFileClass, fileScanTaskClass, taskBuilder) + + try { + val residualMethod = contentScanTaskClass.getMethod("residual") + val residualExpr = residualMethod.invoke(task) + + val catalystExpr = convertIcebergExpression(residualExpr, scan.output) + + catalystExpr + .flatMap { expr => + exprToProto(expr, scan.output, binding = false) + } + .foreach { protoExpr => + taskBuilder.setResidual(protoExpr) + } + } catch { + case e: Exception => + logWarning( + "Failed to extract residual expression from FileScanTask: " + + s"${e.getMessage}") + } + + // Serialize partition spec and data (field definitions, transforms, values) + serializePartitionData( + task, + contentScanTaskClass, + fileScanTaskClass, + taskBuilder) + + // Set name mapping if available (shared by all tasks, pre-extracted) + metadata.nameMapping.foreach(taskBuilder.setNameMappingJson) + + partitionBuilder.addFileScanTasks(taskBuilder.build()) + } + } + } + } + + val builtPartition = partitionBuilder.build() + icebergScanBuilder.addFilePartitions(builtPartition) + } + case _ => + } + } catch { + case e: Exception => + val msg = + "Iceberg reflection failure: Failed to extract FileScanTasks from Iceberg scan RDD: " + + s"${e.getMessage}" + logError(msg, e) + return None + } + + builder.clearChildren() + Some(builder.setIcebergScan(icebergScanBuilder).build()) + } + + override def createExec(nativeOp: Operator, op: CometBatchScanExec): CometNativeExec = { + import org.apache.spark.sql.comet.CometIcebergNativeScanExec + + // Extract metadata - it must be present at this point + val metadata = op.nativeIcebergScanMetadata.getOrElse { + throw new IllegalStateException( + "Programming error: CometBatchScanExec.nativeIcebergScanMetadata is None. " + + "Metadata should have been extracted in CometScanRule.") + } + + // Extract metadataLocation from the native operator + val metadataLocation = nativeOp.getIcebergScan.getMetadataLocation + + // Create the CometIcebergNativeScanExec using the companion object's apply method + CometIcebergNativeScanExec(nativeOp, op.wrapped, op.session, metadataLocation, metadata) + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala index a4af5f1f34..e4ccb4c63e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala @@ -36,8 +36,12 @@ import org.apache.spark.sql.vectorized._ import com.google.common.base.Objects import org.apache.comet.{DataTypeSupport, MetricsSupport} +import org.apache.comet.iceberg.CometIcebergNativeScanMetadata -case class CometBatchScanExec(wrapped: BatchScanExec, runtimeFilters: Seq[Expression]) +case class CometBatchScanExec( + wrapped: BatchScanExec, + runtimeFilters: Seq[Expression], + nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None) extends DataSourceV2ScanExecBase with CometPlan { def ordering: Option[Seq[SortOrder]] = wrapped.ordering @@ -95,14 +99,18 @@ case class CometBatchScanExec(wrapped: BatchScanExec, runtimeFilters: Seq[Expres override def equals(other: Any): Boolean = other match { case other: CometBatchScanExec => // `wrapped` in `this` and `other` could reference to the same `BatchScanExec` object, - // therefore we need to also check `runtimeFilters` equality here. - this.wrappedScan == other.wrappedScan && this.runtimeFilters == other.runtimeFilters + // check `runtimeFilters` and `nativeIcebergScanMetadata` equality too. + this.wrappedScan == other.wrappedScan && this.runtimeFilters == other.runtimeFilters && + this.nativeIcebergScanMetadata == other.nativeIcebergScanMetadata case _ => false } override def hashCode(): Int = { - Objects.hashCode(wrappedScan, runtimeFilters) + Objects.hashCode( + wrappedScan, + runtimeFilters, + Integer.valueOf(nativeIcebergScanMetadata.map(_.hashCode()).getOrElse(0))) } override def doCanonicalize(): CometBatchScanExec = { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala new file mode 100644 index 0000000000..4a2ecc0d32 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.util.AccumulatorV2 + +import com.google.common.base.Objects + +import org.apache.comet.iceberg.CometIcebergNativeScanMetadata +import org.apache.comet.serde.OperatorOuterClass.Operator + +/** + * Native Iceberg scan operator that delegates file reading to iceberg-rust. + * + * Replaces Spark's Iceberg BatchScanExec to bypass the DataSource V2 API and enable native + * execution. Iceberg's catalog and planning run in Spark to produce FileScanTasks, which are + * serialized to protobuf for the native side to execute using iceberg-rust's FileIO and + * ArrowReader. This provides better performance than reading through Spark's abstraction layers. + */ +case class CometIcebergNativeScanExec( + override val nativeOp: Operator, + override val output: Seq[Attribute], + @transient override val originalPlan: BatchScanExec, + override val serializedPlanOpt: SerializedPlan, + metadataLocation: String, + numPartitions: Int, + nativeIcebergScanMetadata: CometIcebergNativeScanMetadata) + extends CometLeafExec { + + override val supportsColumnar: Boolean = true + + override val nodeName: String = "CometIcebergNativeScan" + + override lazy val outputPartitioning: Partitioning = + UnknownPartitioning(numPartitions) + + override lazy val outputOrdering: Seq[SortOrder] = Nil + + // Capture metric VALUES and TYPES (not objects!) in a serializable case class + // This survives serialization while SQLMetric objects get reset to 0 + private case class MetricValue(name: String, value: Long, metricType: String) + + /** + * Maps Iceberg V2 custom metric types to standard Spark metric types for better UI formatting. + * + * Iceberg uses V2 custom metrics which don't get formatted in Spark UI (they just show raw + * numbers). By mapping to standard Spark types, we get proper formatting: + * - "size" metrics: formatted as KB/MB/GB (e.g., "10.3 GB" instead of "11040868925") + * - "timing" metrics: formatted as ms/s (e.g., "200 ms" instead of "200") + * - "sum" metrics: plain numbers with commas (e.g., "1,000") + * + * This provides better UX than vanilla Iceberg Java which shows raw numbers. + */ + private def mapMetricType(name: String, originalType: String): String = { + import java.util.Locale + + // Only remap V2 custom metrics; leave standard Spark metrics unchanged + if (!originalType.startsWith("v2Custom_")) { + return originalType + } + + // Map based on metric name patterns from Iceberg + val nameLower = name.toLowerCase(Locale.ROOT) + if (nameLower.contains("size")) { + "size" // Will format as KB/MB/GB + } else if (nameLower.contains("duration")) { + "timing" // Will format as ms/s (Iceberg durations are in milliseconds) + } else { + "sum" // Plain number formatting + } + } + + private val capturedMetricValues: Seq[MetricValue] = { + originalPlan.metrics + .filterNot { case (name, _) => + // Filter out metrics that are now runtime metrics incremented on the native side + name == "numOutputRows" || name == "numDeletes" || name == "numSplits" + } + .map { case (name, metric) => + val mappedType = mapMetricType(name, metric.metricType) + MetricValue(name, metric.value, mappedType) + } + .toSeq + } + + /** + * Immutable SQLMetric for planning metrics that don't change during execution. + * + * Regular SQLMetric extends AccumulatorV2, which means when execution completes, accumulator + * updates from executors (which are 0 since they don't update planning metrics) get merged back + * to the driver, overwriting the driver's values with 0. + * + * This class overrides the accumulator methods to make the metric truly immutable once set. + */ + private class ImmutableSQLMetric(metricType: String) extends SQLMetric(metricType, 0) { + + override def merge(other: AccumulatorV2[Long, Long]): Unit = {} + + override def reset(): Unit = {} + } + + override lazy val metrics: Map[String, SQLMetric] = { + val baseMetrics = Map( + "output_rows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "time_elapsed_opening" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "Wall clock time elapsed for file opening"), + "time_elapsed_scanning_until_data" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "Wall clock time elapsed for file scanning + " + + "first record batch of decompression + decoding"), + "time_elapsed_scanning_total" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "Total elapsed wall clock time for scanning + record batch decompression / decoding"), + "time_elapsed_processing" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "Wall clock time elapsed for data decompression + decoding")) + + // Create IMMUTABLE metrics with captured values AND types + // these won't be affected by accumulator merges + val icebergMetrics = capturedMetricValues.map { mv => + // Create the immutable metric with initValue = 0 (Spark 4 requires initValue <= 0) + val metric = new ImmutableSQLMetric(mv.metricType) + // Set the actual value after creation + metric.set(mv.value) + // Register it with SparkContext to assign metadata (name, etc.) + sparkContext.register(metric, mv.name) + mv.name -> metric + }.toMap + + // Add num_splits as a runtime metric (incremented on the native side during execution) + val numSplitsMetric = SQLMetrics.createMetric(sparkContext, "number of file splits processed") + + baseMetrics ++ icebergMetrics + ("num_splits" -> numSplitsMetric) + } + + override protected def doCanonicalize(): CometIcebergNativeScanExec = { + CometIcebergNativeScanExec( + nativeOp, + output.map(QueryPlan.normalizeExpressions(_, output)), + originalPlan.doCanonicalize(), + SerializedPlan(None), + metadataLocation, + numPartitions, + nativeIcebergScanMetadata) + } + + override def stringArgs: Iterator[Any] = + Iterator(output, s"$metadataLocation, ${originalPlan.scan.description()}", numPartitions) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometIcebergNativeScanExec => + this.metadataLocation == other.metadataLocation && + this.output == other.output && + this.serializedPlanOpt == other.serializedPlanOpt && + this.numPartitions == other.numPartitions && + this.nativeIcebergScanMetadata == other.nativeIcebergScanMetadata + case _ => + false + } + } + + override def hashCode(): Int = + Objects.hashCode( + metadataLocation, + output.asJava, + serializedPlanOpt, + numPartitions: java.lang.Integer, + nativeIcebergScanMetadata) +} + +object CometIcebergNativeScanExec { + + /** + * Creates a CometIcebergNativeScanExec from a Spark BatchScanExec. + * + * Determines the number of partitions from Iceberg's output partitioning: + * - KeyGroupedPartitioning: Use Iceberg's partition count + * - Other cases: Use the number of InputPartitions from Iceberg's planning + * + * @param nativeOp + * The serialized native operator + * @param scanExec + * The original Spark BatchScanExec + * @param session + * The SparkSession + * @param metadataLocation + * Path to table metadata file + * @param nativeIcebergScanMetadata + * Pre-extracted Iceberg metadata from planning phase + * @return + * A new CometIcebergNativeScanExec + */ + def apply( + nativeOp: Operator, + scanExec: BatchScanExec, + session: SparkSession, + metadataLocation: String, + nativeIcebergScanMetadata: CometIcebergNativeScanMetadata): CometIcebergNativeScanExec = { + + // Determine number of partitions from Iceberg's output partitioning + val numParts = scanExec.outputPartitioning match { + case p: KeyGroupedPartitioning => + p.numPartitions + case _ => + scanExec.inputRDD.getNumPartitions + } + + val exec = CometIcebergNativeScanExec( + nativeOp, + scanExec.output, + scanExec, + SerializedPlan(None), + metadataLocation, + numParts, + nativeIcebergScanMetadata) + + scanExec.logicalLink.foreach(exec.setLogicalLink) + exec + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 8b31cc028d..c955f79d91 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -371,6 +371,7 @@ abstract class CometNativeExec extends CometExec { * The input sources include the following operators: * - CometScanExec - Comet scan node * - CometBatchScanExec - Comet scan node + * - CometIcebergNativeScanExec - Native Iceberg scan node * - ShuffleQueryStageExec - AQE shuffle stage node on top of Comet shuffle * - AQEShuffleReadExec - AQE shuffle read node on top of Comet shuffle * - CometShuffleExchangeExec - Comet shuffle exchange node @@ -385,10 +386,10 @@ abstract class CometNativeExec extends CometExec { def foreachUntilCometInput(plan: SparkPlan)(func: SparkPlan => Unit): Unit = { plan match { case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec | - _: ShuffleQueryStageExec | _: AQEShuffleReadExec | _: CometShuffleExchangeExec | - _: CometUnionExec | _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec | - _: ReusedExchangeExec | _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec | - _: CometSparkToColumnarExec | _: CometLocalTableScanExec => + _: CometIcebergNativeScanExec | _: ShuffleQueryStageExec | _: AQEShuffleReadExec | + _: CometShuffleExchangeExec | _: CometUnionExec | _: CometTakeOrderedAndProjectExec | + _: CometCoalesceExec | _: ReusedExchangeExec | _: CometBroadcastExchangeExec | + _: BroadcastQueryStageExec | _: CometSparkToColumnarExec | _: CometLocalTableScanExec => func(plan) case _: CometPlan => // Other Comet operators, continue to traverse the tree. diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala new file mode 100644 index 0000000000..92f36f534b --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import java.io.File +import java.nio.file.Files +import java.text.SimpleDateFormat + +import scala.util.Random + +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.comet.CometIcebergNativeScanExec +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DecimalType + +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, SchemaGenOptions} + +class CometFuzzIcebergBase extends CometTestBase with AdaptiveSparkPlanHelper { + + var warehouseDir: File = null + val icebergTableName: String = "hadoop_catalog.db.fuzz_test" + + // Skip these tests if Iceberg is not available in classpath + private def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + private def isIcebergVersionLessThan(targetVersion: String): Boolean = { + try { + val icebergVersion = org.apache.iceberg.IcebergBuild.version() + // Parse version strings like "1.5.2" or "1.6.0-SNAPSHOT" + val current = parseVersion(icebergVersion) + val target = parseVersion(targetVersion) + // Compare tuples: first by major, then minor, then patch + if (current._1 != target._1) current._1 < target._1 + else if (current._2 != target._2) current._2 < target._2 + else current._3 < target._3 + } catch { + case _: Exception => + // If we can't determine the version, assume it's old to be safe + true + } + } + + private def parseVersion(version: String): (Int, Int, Int) = { + val parts = version.split("[.-]").take(3).map(_.filter(_.isDigit)) + val major = if (parts.length > 0 && parts(0).nonEmpty) parts(0).toInt else 0 + val minor = if (parts.length > 1 && parts(1).nonEmpty) parts(1).toInt else 0 + val patch = if (parts.length > 2 && parts(2).nonEmpty) parts(2).toInt else 0 + (major, minor, patch) + } + + /** + * We use Asia/Kathmandu because it has a non-zero number of minutes as the offset, so is an + * interesting edge case. Also, this timezone tends to be different from the default system + * timezone. + * + * Represents UTC+5:45 + */ + val defaultTimezone = "Asia/Kathmandu" + + override def beforeAll(): Unit = { + super.beforeAll() + assume(icebergAvailable, "Iceberg not available in classpath") + warehouseDir = Files.createTempDirectory("comet-iceberg-fuzz-test").toFile + val random = new Random(42) + withSQLConf( + "spark.sql.catalog.hadoop_catalog" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.hadoop_catalog.type" -> "hadoop", + "spark.sql.catalog.hadoop_catalog.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "false", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> defaultTimezone) { + + val schema = FuzzDataGenerator.generateSchema( + SchemaGenOptions( + generateArray = true, + generateStruct = true, + primitiveTypes = SchemaGenOptions.defaultPrimitiveTypes.filterNot { dataType => + // Disable decimals - iceberg-rust doesn't support FIXED_LEN_BYTE_ARRAY in page index yet + dataType.isInstanceOf[DecimalType] || + // Disable ByteType and ShortType for Iceberg < 1.6.0 + // Fixed in https://github.com/apache/iceberg/pull/10349 + (isIcebergVersionLessThan( + "1.6.0") && (dataType == org.apache.spark.sql.types.DataTypes.ByteType || + dataType == org.apache.spark.sql.types.DataTypes.ShortType)) + })) + + val options = + DataGenOptions( + generateNegativeZero = false, + baseDate = + new SimpleDateFormat("YYYY-MM-DD hh:mm:ss").parse("2024-05-25 12:34:56").getTime) + + val df = FuzzDataGenerator.generateDataFrame(random, spark, schema, 1000, options) + df.writeTo(icebergTableName).using("iceberg").create() + } + } + + protected override def afterAll(): Unit = { + try { + spark.sql(s"DROP TABLE IF EXISTS $icebergTableName") + } catch { + case _: Exception => + } + + if (warehouseDir != null) { + def deleteRecursively(file: File): Unit = { + if (file.isDirectory) { + file.listFiles().foreach(deleteRecursively) + } + file.delete() + } + + deleteRecursively(warehouseDir) + } + super.afterAll() + } + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*) { + withSQLConf( + "spark.sql.catalog.hadoop_catalog" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.hadoop_catalog.type" -> "hadoop", + "spark.sql.catalog.hadoop_catalog.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + testFun + } + } + } + + def collectIcebergNativeScans(plan: SparkPlan): Seq[CometIcebergNativeScanExec] = { + collect(plan) { case scan: CometIcebergNativeScanExec => + scan + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala new file mode 100644 index 0000000000..0a188f6cce --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import scala.util.Random + +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType +import org.apache.spark.sql.types._ + +import org.apache.comet.DataTypeSupport.isComplexType +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, ParquetGenerator, SchemaGenOptions} + +class CometFuzzIcebergSuite extends CometFuzzIcebergBase { + + test("select *") { + val sql = s"SELECT * FROM $icebergTableName" + val (_, cometPlan) = checkSparkAnswer(sql) + assert(1 == collectIcebergNativeScans(cometPlan).length) + } + + test("select * with limit") { + val sql = s"SELECT * FROM $icebergTableName LIMIT 500" + val (_, cometPlan) = checkSparkAnswer(sql) + assert(1 == collectIcebergNativeScans(cometPlan).length) + } + + test("order by single column") { + val df = spark.table(icebergTableName) + for (col <- df.columns) { + val sql = s"SELECT $col FROM $icebergTableName ORDER BY $col" + // cannot run fully natively due to range partitioning and sort + val (_, cometPlan) = checkSparkAnswer(sql) + assert(1 == collectIcebergNativeScans(cometPlan).length) + } + } + + test("order by multiple columns") { + val df = spark.table(icebergTableName) + val allCols = df.columns.mkString(",") + val sql = s"SELECT $allCols FROM $icebergTableName ORDER BY $allCols" + // cannot run fully natively due to range partitioning and sort + val (_, cometPlan) = checkSparkAnswer(sql) + assert(1 == collectIcebergNativeScans(cometPlan).length) + } + + test("order by random columns") { + val df = spark.table(icebergTableName) + + for (_ <- 1 to 10) { + // We only do order by permutations of primitive types to exercise native shuffle's + // RangePartitioning which only supports those types. + val primitiveColumns = + df.schema.fields.filterNot(f => isComplexType(f.dataType)).map(_.name) + val shuffledPrimitiveCols = Random.shuffle(primitiveColumns.toList) + val randomSize = Random.nextInt(shuffledPrimitiveCols.length) + 1 + val randomColsSubset = shuffledPrimitiveCols.take(randomSize).toArray.mkString(",") + val sql = s"SELECT $randomColsSubset FROM $icebergTableName ORDER BY $randomColsSubset" + checkSparkAnswerAndOperator(sql) + } + } + + test("distribute by single column (complex types)") { + val df = spark.table(icebergTableName) + val columns = df.schema.fields.filter(f => isComplexType(f.dataType)).map(_.name) + for (col <- columns) { + // DISTRIBUTE BY is equivalent to df.repartition($col) and uses + val sql = s"SELECT $col FROM $icebergTableName DISTRIBUTE BY $col" + val resultDf = spark.sql(sql) + resultDf.collect() + // check for Comet shuffle + val plan = + resultDf.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val cometShuffleExchanges = collectCometShuffleExchanges(plan) + // Iceberg native scan supports complex types + assert(cometShuffleExchanges.length == 1) + } + } + + test("shuffle supports all types") { + val df = spark.table(icebergTableName) + val df2 = df.repartition(8, df.col("c0")).sort("c1") + df2.collect() + val cometShuffles = collectCometShuffleExchanges(df2.queryExecution.executedPlan) + // Iceberg native scan supports complex types + assert(cometShuffles.length == 2) + } + + test("join") { + val df = spark.table(icebergTableName) + df.createOrReplaceTempView("t1") + df.createOrReplaceTempView("t2") + // Filter out complex types - iceberg-rust can't create predicates for struct/array/map equality + val primitiveColumns = df.schema.fields.filterNot(f => isComplexType(f.dataType)).map(_.name) + for (col <- primitiveColumns) { + // cannot run fully native due to HashAggregate + val sql = s"SELECT count(*) FROM t1 JOIN t2 ON t1.$col = t2.$col" + val (_, cometPlan) = checkSparkAnswer(sql) + assert(2 == collectIcebergNativeScans(cometPlan).length) + } + } + + test("decode") { + val df = spark.table(icebergTableName) + // We want to make sure that the schema generator wasn't modified to accidentally omit + // BinaryType, since then this test would not run any queries and silently pass. + var testedBinary = false + for (field <- df.schema.fields if field.dataType == BinaryType) { + testedBinary = true + // Intentionally use odd capitalization of 'utf-8' to test normalization. + val sql = s"SELECT decode(${field.name}, 'utF-8') FROM $icebergTableName" + checkSparkAnswerAndOperator(sql) + } + assert(testedBinary) + } + + test("regexp_replace") { + withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + val df = spark.table(icebergTableName) + // We want to make sure that the schema generator wasn't modified to accidentally omit + // StringType, since then this test would not run any queries and silently pass. + var testedString = false + for (field <- df.schema.fields if field.dataType == StringType) { + testedString = true + val sql = s"SELECT regexp_replace(${field.name}, 'a', 'b') FROM $icebergTableName" + checkSparkAnswerAndOperator(sql) + } + assert(testedString) + } + } + + test("Iceberg temporal types written as INT96") { + testIcebergTemporalTypes(ParquetOutputTimestampType.INT96) + } + + test("Iceberg temporal types written as TIMESTAMP_MICROS") { + testIcebergTemporalTypes(ParquetOutputTimestampType.TIMESTAMP_MICROS) + } + + test("Iceberg temporal types written as TIMESTAMP_MILLIS") { + testIcebergTemporalTypes(ParquetOutputTimestampType.TIMESTAMP_MILLIS) + } + + private def testIcebergTemporalTypes( + outputTimestampType: ParquetOutputTimestampType.Value, + generateArray: Boolean = true, + generateStruct: Boolean = true): Unit = { + + val schema = FuzzDataGenerator.generateSchema( + SchemaGenOptions( + generateArray = generateArray, + generateStruct = generateStruct, + primitiveTypes = SchemaGenOptions.defaultPrimitiveTypes.filterNot { dataType => + // Disable decimals - iceberg-rust doesn't support FIXED_LEN_BYTE_ARRAY in page index yet + dataType.isInstanceOf[DecimalType] + })) + + val options = + DataGenOptions(generateNegativeZero = false) + + withTempPath { filename => + val random = new Random(42) + withSQLConf( + CometConf.COMET_ENABLED.key -> "false", + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> outputTimestampType.toString, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> defaultTimezone) { + ParquetGenerator.makeParquetFile(random, spark, filename.toString, schema, 100, options) + } + + Seq(defaultTimezone, "UTC", "America/Denver").foreach { tz => + Seq(true, false).foreach { inferTimestampNtzEnabled => + Seq(true, false).foreach { int96TimestampConversion => + Seq(true, false).foreach { int96AsTimestamp => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz, + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key -> int96AsTimestamp.toString, + SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION.key -> int96TimestampConversion.toString, + SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key -> inferTimestampNtzEnabled.toString) { + + val df = spark.table(icebergTableName) + + Seq(defaultTimezone, "UTC", "America/Denver").foreach { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + def hasTemporalType(t: DataType): Boolean = t match { + case DataTypes.DateType | DataTypes.TimestampType | + DataTypes.TimestampNTZType => + true + case t: StructType => t.exists(f => hasTemporalType(f.dataType)) + case t: ArrayType => hasTemporalType(t.elementType) + case _ => false + } + + val columns = + df.schema.fields.filter(f => hasTemporalType(f.dataType)).map(_.name) + + for (col <- columns) { + checkSparkAnswer(s"SELECT $col FROM $icebergTableName ORDER BY $col") + } + } + } + } + } + } + } + } + } + } + + def collectCometShuffleExchanges(plan: org.apache.spark.sql.execution.SparkPlan) + : Seq[org.apache.spark.sql.execution.SparkPlan] = { + collect(plan) { + case exchange: org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec => + exchange + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index 05f441f24b..59680bd6bc 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -153,7 +153,9 @@ class CometFuzzTestSuite extends CometFuzzTestBase { for (_ <- 1 to 10) { // We only do order by permutations of primitive types to exercise native shuffle's // RangePartitioning which only supports those types. - val shuffledPrimitiveCols = Random.shuffle(df.columns.slice(0, 14).toList) + val primitiveColumns = + df.schema.fields.filterNot(f => isComplexType(f.dataType)).map(_.name) + val shuffledPrimitiveCols = Random.shuffle(primitiveColumns.toList) val randomSize = Random.nextInt(shuffledPrimitiveCols.length) + 1 val randomColsSubset = shuffledPrimitiveCols.take(randomSize).toArray.mkString(",") val sql = s"SELECT $randomColsSubset FROM t1 ORDER BY $randomColsSubset" diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala new file mode 100644 index 0000000000..d1b9197ffd --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -0,0 +1,2119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import java.io.File +import java.nio.file.Files + +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.comet.CometIcebergNativeScanExec +import org.apache.spark.sql.execution.SparkPlan + +/** + * Test suite for native Iceberg scan using FileScanTasks and iceberg-rust. + * + * Note: Requires Iceberg dependencies to be added to pom.xml + */ +class CometIcebergNativeSuite extends CometTestBase { + + // Skip these tests if Iceberg is not available in classpath + private def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + /** Collects all CometIcebergNativeScanExec nodes from a plan */ + private def collectIcebergNativeScans(plan: SparkPlan): Seq[CometIcebergNativeScanExec] = { + collect(plan) { case scan: CometIcebergNativeScanExec => + scan + } + } + + /** + * Helper to verify query correctness and that exactly one CometIcebergNativeScanExec is used. + * This ensures both correct results and that the native Iceberg scan operator is being used. + */ + private def checkIcebergNativeScan(query: String): Unit = { + val (_, cometPlan) = checkSparkAnswer(query) + val icebergScans = collectIcebergNativeScans(cometPlan) + assert( + icebergScans.length == 1, + s"Expected exactly 1 CometIcebergNativeScanExec but found ${icebergScans.length}. Plan:\n$cometPlan") + } + + test("create and query simple Iceberg table with Hadoop catalog") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.hadoop_catalog" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.hadoop_catalog.type" -> "hadoop", + "spark.sql.catalog.hadoop_catalog.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE hadoop_catalog.db.test_table ( + id INT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO hadoop_catalog.db.test_table + VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) + """) + + checkIcebergNativeScan("SELECT * FROM hadoop_catalog.db.test_table ORDER BY id") + + spark.sql("DROP TABLE hadoop_catalog.db.test_table") + } + } + } + + test("filter pushdown - equality predicates") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.filter_cat.type" -> "hadoop", + "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE filter_cat.db.filter_test ( + id INT, + name STRING, + value DOUBLE, + active BOOLEAN + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO filter_cat.db.filter_test VALUES + (1, 'Alice', 10.5, true), + (2, 'Bob', 20.3, false), + (3, 'Charlie', 30.7, true), + (4, 'Diana', 15.2, false), + (5, 'Eve', 25.8, true) + """) + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE id = 3") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE name = 'Bob'") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE active = true") + + spark.sql("DROP TABLE filter_cat.db.filter_test") + } + } + } + + test("filter pushdown - comparison operators") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.filter_cat.type" -> "hadoop", + "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE filter_cat.db.comparison_test ( + id INT, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO filter_cat.db.comparison_test VALUES + (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) + """) + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value > 20.0") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value >= 20.3") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value < 20.0") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value <= 20.3") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE id != 3") + + spark.sql("DROP TABLE filter_cat.db.comparison_test") + } + } + } + + test("filter pushdown - AND/OR combinations") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.filter_cat.type" -> "hadoop", + "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE filter_cat.db.logical_test ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO filter_cat.db.logical_test VALUES + (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), + (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) + """) + + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.logical_test WHERE category = 'A' AND value > 20.0") + + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.logical_test WHERE category = 'B' OR value > 30.0") + + checkIcebergNativeScan("""SELECT * FROM filter_cat.db.logical_test + WHERE (category = 'A' AND value > 20.0) OR category = 'C'""") + + spark.sql("DROP TABLE filter_cat.db.logical_test") + } + } + } + + test("filter pushdown - NULL checks") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.filter_cat.type" -> "hadoop", + "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE filter_cat.db.null_test ( + id INT, + optional_value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO filter_cat.db.null_test VALUES + (1, 10.5), (2, NULL), (3, 30.7), (4, NULL), (5, 25.8) + """) + + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NULL") + + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NOT NULL") + + spark.sql("DROP TABLE filter_cat.db.null_test") + } + } + } + + test("filter pushdown - IN list") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.filter_cat.type" -> "hadoop", + "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE filter_cat.db.in_test ( + id INT, + name STRING + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO filter_cat.db.in_test VALUES + (1, 'Alice'), (2, 'Bob'), (3, 'Charlie'), + (4, 'Diana'), (5, 'Eve'), (6, 'Frank') + """) + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IN (2, 4, 6)") + + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.in_test WHERE name IN ('Alice', 'Charlie', 'Eve')") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IS NOT NULL") + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id NOT IN (1, 3, 5)") + + spark.sql("DROP TABLE filter_cat.db.in_test") + } + } + } + + test("verify filters are pushed to native scan") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.filter_cat.type" -> "hadoop", + "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE filter_cat.db.filter_debug ( + id INT, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO filter_cat.db.filter_debug VALUES + (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) + """) + + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") + + spark.sql("DROP TABLE filter_cat.db.filter_debug") + } + } + } + + test("small table - verify no duplicate rows (1 file)") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.small_table ( + id INT, + name STRING + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.small_table + VALUES (1, 'Alice'), (2, 'Bob'), (3, 'Charlie') + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.small_table ORDER BY id") + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.small_table") + + spark.sql("DROP TABLE test_cat.db.small_table") + } + } + } + + test("medium table - verify correct partition count (multiple files)") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + "spark.sql.files.maxRecordsPerFile" -> "10") { + + spark.sql(""" + CREATE TABLE test_cat.db.medium_table ( + id INT, + value DOUBLE + ) USING iceberg + """) + + // Insert 100 rows - should create multiple files with maxRecordsPerFile=10 + spark.sql(""" + INSERT INTO test_cat.db.medium_table + SELECT id, CAST(id * 1.5 AS DOUBLE) as value + FROM range(100) + """) + + // Verify results match Spark native (catches duplicates across partitions) + checkIcebergNativeScan("SELECT * FROM test_cat.db.medium_table ORDER BY id") + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.medium_table") + checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.medium_table") + + spark.sql("DROP TABLE test_cat.db.medium_table") + } + } + } + + test("large table - verify no duplicates with many files") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + "spark.sql.files.maxRecordsPerFile" -> "100") { + + spark.sql(""" + CREATE TABLE test_cat.db.large_table ( + id BIGINT, + category STRING, + value DOUBLE + ) USING iceberg + """) + + // Insert 10,000 rows - with maxRecordsPerFile=100, creates ~100 files + spark.sql(""" + INSERT INTO test_cat.db.large_table + SELECT + id, + CASE WHEN id % 3 = 0 THEN 'A' WHEN id % 3 = 1 THEN 'B' ELSE 'C' END as category, + CAST(id * 2.5 AS DOUBLE) as value + FROM range(10000) + """) + + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.large_table") + checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.large_table") + checkIcebergNativeScan( + "SELECT category, COUNT(*) FROM test_cat.db.large_table GROUP BY category ORDER BY category") + + spark.sql("DROP TABLE test_cat.db.large_table") + } + } + } + + test("partitioned table - verify key-grouped partitioning") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.partitioned_table ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (category) + """) + + spark.sql(""" + INSERT INTO test_cat.db.partitioned_table VALUES + (1, 'A', 10.5), (2, 'B', 20.3), (3, 'C', 30.7), + (4, 'A', 15.2), (5, 'B', 25.8), (6, 'C', 35.0), + (7, 'A', 12.1), (8, 'B', 22.5), (9, 'C', 32.9) + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.partitioned_table ORDER BY id") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.partitioned_table WHERE category = 'A' ORDER BY id") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.partitioned_table WHERE category = 'B' ORDER BY id") + checkIcebergNativeScan( + "SELECT category, COUNT(*) FROM test_cat.db.partitioned_table GROUP BY category ORDER BY category") + + spark.sql("DROP TABLE test_cat.db.partitioned_table") + } + } + } + + test("empty table - verify graceful handling") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.empty_table ( + id INT, + name STRING + ) USING iceberg + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table") + checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table WHERE id > 0") + + spark.sql("DROP TABLE test_cat.db.empty_table") + } + } + } + + // MOR (Merge-On-Read) delete file tests. + // Delete files are extracted from FileScanTasks and handled by iceberg-rust's ArrowReader, + // which automatically applies both positional and equality deletes during scan execution. + test("MOR table with POSITIONAL deletes - verify deletes are applied") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.positional_delete_test ( + id INT, + name STRING, + value DOUBLE + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' + ) + """) + + spark.sql(""" + INSERT INTO test_cat.db.positional_delete_test + VALUES + (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7), + (4, 'Diana', 15.2), (5, 'Eve', 25.8), (6, 'Frank', 35.0), + (7, 'Grace', 12.1), (8, 'Hank', 22.5) + """) + + spark.sql("DELETE FROM test_cat.db.positional_delete_test WHERE id IN (2, 4, 6)") + + checkIcebergNativeScan("SELECT * FROM test_cat.db.positional_delete_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.positional_delete_test") + } + } + } + + test("MOR table with EQUALITY deletes - verify deletes are applied") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Create table with equality delete columns specified + // This forces Spark to use equality deletes instead of positional deletes + spark.sql(""" + CREATE TABLE test_cat.db.equality_delete_test ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read', + 'write.delete.equality-delete-columns' = 'id' + ) + """) + + spark.sql(""" + INSERT INTO test_cat.db.equality_delete_test + VALUES + (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), + (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) + """) + + spark.sql("DELETE FROM test_cat.db.equality_delete_test WHERE id IN (2, 4)") + + checkIcebergNativeScan("SELECT * FROM test_cat.db.equality_delete_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.equality_delete_test") + } + } + } + + test("MOR table with multiple delete operations - mixed delete types") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.multi_delete_test ( + id INT, + data STRING + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' + ) + """) + + spark.sql(""" + INSERT INTO test_cat.db.multi_delete_test + SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + FROM range(100) + """) + + spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id < 10") + spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id > 90") + spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id % 10 = 5") + + checkIcebergNativeScan("SELECT * FROM test_cat.db.multi_delete_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.multi_delete_test") + } + } + } + + test("verify no duplicate rows across multiple partitions") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // Create multiple files to ensure multiple partitions + "spark.sql.files.maxRecordsPerFile" -> "50") { + + spark.sql(""" + CREATE TABLE test_cat.db.multipart_test ( + id INT, + data STRING + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.multipart_test + SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + FROM range(500) + """) + + // Critical: COUNT(*) vs COUNT(DISTINCT id) catches duplicates across partitions + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.multipart_test") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.multipart_test WHERE id < 10 ORDER BY id") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.multipart_test WHERE id >= 490 ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.multipart_test") + } + } + } + + test("filter pushdown with multi-partition table") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + "spark.sql.files.maxRecordsPerFile" -> "20") { + + spark.sql(""" + CREATE TABLE test_cat.db.filter_multipart ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.filter_multipart + SELECT + id, + CASE WHEN id % 2 = 0 THEN 'even' ELSE 'odd' END as category, + CAST(id * 1.5 AS DOUBLE) as value + FROM range(200) + """) + + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.filter_multipart WHERE id > 150 ORDER BY id") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.filter_multipart WHERE category = 'even' AND id < 50 ORDER BY id") + checkIcebergNativeScan( + "SELECT COUNT(DISTINCT id) FROM test_cat.db.filter_multipart WHERE id BETWEEN 50 AND 100") + checkIcebergNativeScan( + "SELECT SUM(value) FROM test_cat.db.filter_multipart WHERE category = 'odd'") + + spark.sql("DROP TABLE test_cat.db.filter_multipart") + } + } + } + + test("date partitioned table with date range queries") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.date_partitioned ( + id INT, + event_date DATE, + value STRING + ) USING iceberg + PARTITIONED BY (days(event_date)) + """) + + spark.sql(""" + INSERT INTO test_cat.db.date_partitioned VALUES + (1, DATE '2024-01-01', 'a'), (2, DATE '2024-01-02', 'b'), + (3, DATE '2024-01-03', 'c'), (4, DATE '2024-01-15', 'd'), + (5, DATE '2024-01-16', 'e'), (6, DATE '2024-02-01', 'f') + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.date_partitioned ORDER BY id") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.date_partitioned WHERE event_date = DATE '2024-01-01'") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.date_partitioned WHERE event_date BETWEEN DATE '2024-01-01' AND DATE '2024-01-03' ORDER BY id") + checkIcebergNativeScan( + "SELECT event_date, COUNT(*) FROM test_cat.db.date_partitioned GROUP BY event_date ORDER BY event_date") + + spark.sql("DROP TABLE test_cat.db.date_partitioned") + } + } + } + + test("bucket partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.bucket_partitioned ( + id INT, + value DOUBLE + ) USING iceberg + PARTITIONED BY (bucket(4, id)) + """) + + spark.sql(""" + INSERT INTO test_cat.db.bucket_partitioned + SELECT id, CAST(id * 1.5 AS DOUBLE) as value + FROM range(100) + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.bucket_partitioned ORDER BY id") + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.bucket_partitioned") + checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.bucket_partitioned") + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.bucket_partitioned WHERE id < 20 ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.bucket_partitioned") + } + } + } + + test("partition pruning - bucket transform verifies files are skipped") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.bucket_pruning ( + id INT, + data STRING + ) USING iceberg + PARTITIONED BY (bucket(8, id)) + """) + + (0 until 8).foreach { bucket => + spark.sql(s""" + INSERT INTO test_cat.db.bucket_pruning + SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + FROM range(${bucket * 100}, ${(bucket + 1) * 100}) + """) + } + + val specificIds = Seq(5, 15, 25) + val df = spark.sql(s""" + SELECT * FROM test_cat.db.bucket_pruning + WHERE id IN (${specificIds.mkString(",")}) + """) + + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + val result = df.collect() + assert(result.length == specificIds.length) + + // With bucket partitioning, pruning occurs at the file level, not manifest level + // Bucket transforms use hash-based bucketing, so manifests may contain files from + // multiple buckets. Iceberg can skip individual files based on bucket metadata, + // but cannot skip entire manifests. + assert( + metrics("resultDataFiles").value < 8, + "Bucket pruning should skip some files, but read " + + s"${metrics("resultDataFiles").value} out of 8") + assert( + metrics("skippedDataFiles").value > 0, + "Expected skipped data files due to bucket pruning, got" + + s"${metrics("skippedDataFiles").value}") + + spark.sql("DROP TABLE test_cat.db.bucket_pruning") + } + } + } + + test("partition pruning - truncate transform verifies files are skipped") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.truncate_pruning ( + id INT, + message STRING + ) USING iceberg + PARTITIONED BY (truncate(5, message)) + """) + + val prefixes = Seq("alpha", "bravo", "charlie", "delta", "echo") + prefixes.zipWithIndex.foreach { case (prefix, idx) => + spark.sql(s""" + INSERT INTO test_cat.db.truncate_pruning + SELECT + id, + CONCAT('$prefix', '_suffix_', CAST(id AS STRING)) as message + FROM range(${idx * 10}, ${(idx + 1) * 10}) + """) + } + + val df = spark.sql(""" + SELECT * FROM test_cat.db.truncate_pruning + WHERE message LIKE 'alpha%' + """) + + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + val result = df.collect() + assert(result.length == 10) + assert(result.forall(_.getString(1).startsWith("alpha"))) + + // Partition pruning occurs at the manifest level, not file level + // Each INSERT creates one manifest, so we verify skippedDataManifests + assert( + metrics("resultDataFiles").value == 1, + s"Truncate pruning should only read 1 file, read ${metrics("resultDataFiles").value}") + assert( + metrics("skippedDataManifests").value == 4, + s"Expected 4 skipped manifests, got ${metrics("skippedDataManifests").value}") + + spark.sql("DROP TABLE test_cat.db.truncate_pruning") + } + } + } + + test("partition pruning - hour transform verifies files are skipped") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.hour_pruning ( + id INT, + event_time TIMESTAMP, + data STRING + ) USING iceberg + PARTITIONED BY (hour(event_time)) + """) + + (0 until 6).foreach { hour => + spark.sql(s""" + INSERT INTO test_cat.db.hour_pruning + SELECT + id, + CAST('2024-01-01 $hour:00:00' AS TIMESTAMP) as event_time, + CONCAT('event_', CAST(id AS STRING)) as data + FROM range(${hour * 10}, ${(hour + 1) * 10}) + """) + } + + val df = spark.sql(""" + SELECT * FROM test_cat.db.hour_pruning + WHERE event_time >= CAST('2024-01-01 04:00:00' AS TIMESTAMP) + """) + + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + val result = df.collect() + assert(result.length == 20) + + // Partition pruning occurs at the manifest level, not file level + // Each INSERT creates one manifest, so we verify skippedDataManifests + assert( + metrics("resultDataFiles").value == 2, + s"Hour pruning should read 2 files (hours 4-5), read ${metrics("resultDataFiles").value}") + assert( + metrics("skippedDataManifests").value == 4, + s"Expected 4 skipped manifests (hours 0-3), got ${metrics("skippedDataManifests").value}") + + spark.sql("DROP TABLE test_cat.db.hour_pruning") + } + } + } + + test("schema evolution - add column") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.schema_evolution ( + id INT, + name STRING + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.schema_evolution VALUES (1, 'Alice'), (2, 'Bob') + """) + + spark.sql("ALTER TABLE test_cat.db.schema_evolution ADD COLUMN age INT") + + spark.sql(""" + INSERT INTO test_cat.db.schema_evolution VALUES (3, 'Charlie', 30), (4, 'Diana', 25) + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.schema_evolution ORDER BY id") + checkIcebergNativeScan("SELECT id, name FROM test_cat.db.schema_evolution ORDER BY id") + checkIcebergNativeScan( + "SELECT id, age FROM test_cat.db.schema_evolution WHERE age IS NOT NULL ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.schema_evolution") + } + } + } + + test("schema evolution - drop column") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.drop_column_test ( + id INT, + name STRING, + age INT + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.drop_column_test VALUES (1, 'Alice', 30), (2, 'Bob', 25) + """) + + // Drop the age column + spark.sql("ALTER TABLE test_cat.db.drop_column_test DROP COLUMN age") + + // Insert new data without the age column + spark.sql(""" + INSERT INTO test_cat.db.drop_column_test VALUES (3, 'Charlie'), (4, 'Diana') + """) + + // Read all data - must handle old files (with age) and new files (without age) + checkIcebergNativeScan("SELECT * FROM test_cat.db.drop_column_test ORDER BY id") + checkIcebergNativeScan("SELECT id, name FROM test_cat.db.drop_column_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.drop_column_test") + } + } + } + + test("migration - basic read after migration (fallback for no field ID)") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + val sourceName = "parquet_source" + val destName = "test_cat.db.iceberg_dest" + val dataPath = s"${warehouseDir.getAbsolutePath}/source_data" + + // Step 1: Create regular Parquet table (without field IDs) + spark + .range(10) + .selectExpr( + "CAST(id AS INT) as id", + "CONCAT('name_', CAST(id AS STRING)) as name", + "CAST(id * 2 AS DOUBLE) as value") + .write + .mode("overwrite") + .option("path", dataPath) + .saveAsTable(sourceName) + + // Step 2: Snapshot the Parquet table into Iceberg using SparkActions API + try { + val actionsClass = Class.forName("org.apache.iceberg.spark.actions.SparkActions") + val getMethod = actionsClass.getMethod("get") + val actions = getMethod.invoke(null) + val snapshotMethod = actions.getClass.getMethod("snapshotTable", classOf[String]) + val snapshotAction = snapshotMethod.invoke(actions, sourceName) + val asMethod = snapshotAction.getClass.getMethod("as", classOf[String]) + val snapshotWithDest = asMethod.invoke(snapshotAction, destName) + val executeMethod = snapshotWithDest.getClass.getMethod("execute") + executeMethod.invoke(snapshotWithDest) + + // Step 3: Read the Iceberg table - Parquet files have no field IDs, so position-based mapping is used + checkIcebergNativeScan(s"SELECT * FROM $destName ORDER BY id") + checkIcebergNativeScan(s"SELECT id, name FROM $destName ORDER BY id") + checkIcebergNativeScan(s"SELECT value FROM $destName WHERE id < 5 ORDER BY id") + + spark.sql(s"DROP TABLE $destName") + spark.sql(s"DROP TABLE $sourceName") + } catch { + case _: ClassNotFoundException => + cancel("Iceberg Actions API not available - requires iceberg-spark-runtime") + } + } + } + } + + test("migration - hive-style partitioned table has partition values") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + val sourceName = "parquet_partitioned_source" + val destName = "test_cat.db.iceberg_partitioned" + val dataPath = s"${warehouseDir.getAbsolutePath}/partitioned_data" + + // Hive-style partitioning stores partition values in directory paths, not in data files + spark + .range(10) + .selectExpr( + "CAST(id AS INT) as partition_col", + "CONCAT('data_', CAST(id AS STRING)) as data") + .write + .mode("overwrite") + .partitionBy("partition_col") + .option("path", dataPath) + .saveAsTable(sourceName) + + try { + val actionsClass = Class.forName("org.apache.iceberg.spark.actions.SparkActions") + val getMethod = actionsClass.getMethod("get") + val actions = getMethod.invoke(null) + val snapshotMethod = actions.getClass.getMethod("snapshotTable", classOf[String]) + val snapshotAction = snapshotMethod.invoke(actions, sourceName) + val asMethod = snapshotAction.getClass.getMethod("as", classOf[String]) + val snapshotWithDest = asMethod.invoke(snapshotAction, destName) + val executeMethod = snapshotWithDest.getClass.getMethod("execute") + executeMethod.invoke(snapshotWithDest) + + // Partition columns must have actual values from manifests, not NULL + checkIcebergNativeScan(s"SELECT * FROM $destName ORDER BY partition_col") + checkIcebergNativeScan( + s"SELECT partition_col, data FROM $destName WHERE partition_col < 5 ORDER BY partition_col") + + spark.sql(s"DROP TABLE $destName") + spark.sql(s"DROP TABLE $sourceName") + } catch { + case _: ClassNotFoundException => + cancel("Iceberg Actions API not available - requires iceberg-spark-runtime") + } + } + } + } + + test("projection - column subset, reordering, and duplication") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Create table with multiple columns + spark.sql(""" + CREATE TABLE test_cat.db.proj_test ( + id INT, + name STRING, + value DOUBLE, + flag BOOLEAN + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.proj_test + VALUES (1, 'Alice', 10.5, true), + (2, 'Bob', 20.3, false), + (3, 'Charlie', 30.7, true) + """) + + // Test 1: Column subset (only 2 of 4 columns) + checkIcebergNativeScan("SELECT name, value FROM test_cat.db.proj_test ORDER BY id") + + // Test 2: Reordered columns (reverse order) + checkIcebergNativeScan("SELECT value, name, id FROM test_cat.db.proj_test ORDER BY id") + + // Test 3: Duplicate columns + checkIcebergNativeScan( + "SELECT id, name, id AS id2 FROM test_cat.db.proj_test ORDER BY id") + + // Test 4: Single column + checkIcebergNativeScan("SELECT name FROM test_cat.db.proj_test ORDER BY name") + + // Test 5: Different ordering with subset + checkIcebergNativeScan("SELECT flag, id FROM test_cat.db.proj_test ORDER BY id") + + // Test 6: Multiple duplicates + checkIcebergNativeScan( + "SELECT name, value, name AS name2, value AS value2 FROM test_cat.db.proj_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.proj_test") + } + } + } + + test("complex type - array") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.array_test ( + id INT, + name STRING, + values ARRAY + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.array_test + VALUES (1, 'Alice', array(1, 2, 3)), (2, 'Bob', array(4, 5, 6)) + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.array_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.array_test") + } + } + } + + test("complex type - map") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.map_test ( + id INT, + name STRING, + properties MAP + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.map_test + VALUES (1, 'Alice', map('age', 30, 'score', 95)), (2, 'Bob', map('age', 25, 'score', 87)) + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.map_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.map_test") + } + } + } + + test("complex type - struct") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.struct_test ( + id INT, + name STRING, + address STRUCT + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.struct_test + VALUES (1, 'Alice', struct('NYC', 10001)), (2, 'Bob', struct('LA', 90001)) + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.struct_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.struct_test") + } + } + } + + test("UUID type - native Iceberg UUID column (reproduces type mismatch)") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + import org.apache.iceberg.catalog.TableIdentifier + import org.apache.iceberg.spark.SparkCatalog + import org.apache.iceberg.types.Types + import org.apache.iceberg.{PartitionSpec, Schema} + + // Use Iceberg API to create table with native UUID type + // (not possible via Spark SQL CREATE TABLE) + // Get Spark's catalog instance to ensure the table is visible to Spark + val sparkCatalog = spark.sessionState.catalogManager + .catalog("test_cat") + .asInstanceOf[SparkCatalog] + + spark.sql("CREATE NAMESPACE IF NOT EXISTS test_cat.db") + + // UUID is stored as FixedSizeBinary(16) but must be presented as Utf8 to Spark + val schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "uuid", Types.UUIDType.get())) + val tableIdent = TableIdentifier.of("db", "uuid_test") + sparkCatalog.icebergCatalog.createTable(tableIdent, schema, PartitionSpec.unpartitioned()) + + spark.sql(""" + INSERT INTO test_cat.db.uuid_test VALUES + (1, 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11'), + (2, 'b1ffcd88-8d1a-3de7-aa5c-5aa8ac269a00'), + (3, 'c2aade77-7e0b-2cf6-99e4-4998bc158b22') + """) + + checkIcebergNativeScan("SELECT * FROM test_cat.db.uuid_test ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.uuid_test") + } + } + } + + test("verify all Iceberg planning metrics are populated") { + assume(icebergAvailable, "Iceberg not available in classpath") + + val icebergPlanningMetricNames = Seq( + "totalPlanningDuration", + "totalDataManifest", + "scannedDataManifests", + "skippedDataManifests", + "resultDataFiles", + "skippedDataFiles", + "totalDataFileSize", + "totalDeleteManifests", + "scannedDeleteManifests", + "skippedDeleteManifests", + "totalDeleteFileSize", + "resultDeleteFiles", + "equalityDeleteFiles", + "indexedDeleteFiles", + "positionalDeleteFiles", + "skippedDeleteFiles") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.metrics_test ( + id INT, + value DOUBLE + ) USING iceberg + """) + + // Create multiple files to ensure non-zero manifest/file counts + spark + .range(10000) + .selectExpr("CAST(id AS INT)", "CAST(id * 1.5 AS DOUBLE) as value") + .coalesce(1) + .write + .format("iceberg") + .mode("append") + .saveAsTable("test_cat.db.metrics_test") + + spark + .range(10001, 20000) + .selectExpr("CAST(id AS INT)", "CAST(id * 1.5 AS DOUBLE) as value") + .coalesce(1) + .write + .format("iceberg") + .mode("append") + .saveAsTable("test_cat.db.metrics_test") + + val df = spark.sql("SELECT * FROM test_cat.db.metrics_test WHERE id < 10000") + + // Must extract metrics before collect() because planning happens at plan creation + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + icebergPlanningMetricNames.foreach { metricName => + assert(metrics.contains(metricName), s"metric $metricName was not found") + } + + // Planning metrics are populated during plan creation, so they're already available + assert(metrics("totalDataManifest").value > 0, "totalDataManifest should be > 0") + assert(metrics("resultDataFiles").value > 0, "resultDataFiles should be > 0") + assert(metrics("totalDataFileSize").value > 0, "totalDataFileSize should be > 0") + + df.collect() + + assert(metrics("output_rows").value == 10000) + assert(metrics("num_splits").value > 0) + assert(metrics("time_elapsed_opening").value > 0) + assert(metrics("time_elapsed_scanning_until_data").value > 0) + assert(metrics("time_elapsed_scanning_total").value > 0) + assert(metrics("time_elapsed_processing").value > 0) + // ImmutableSQLMetric prevents these from being reset to 0 after execution + assert( + metrics("totalDataManifest").value > 0, + "totalDataManifest should still be > 0 after execution") + assert( + metrics("resultDataFiles").value > 0, + "resultDataFiles should still be > 0 after execution") + + spark.sql("DROP TABLE test_cat.db.metrics_test") + } + } + } + + test("verify manifest pruning metrics") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Partition by category to enable manifest-level pruning + spark.sql(""" + CREATE TABLE test_cat.db.pruning_test ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (category) + """) + + // Each category gets its own manifest entry + spark.sql(""" + INSERT INTO test_cat.db.pruning_test + SELECT id, 'A' as category, CAST(id * 1.5 AS DOUBLE) as value + FROM range(1000) + """) + + spark.sql(""" + INSERT INTO test_cat.db.pruning_test + SELECT id, 'B' as category, CAST(id * 2.0 AS DOUBLE) as value + FROM range(1000, 2000) + """) + + spark.sql(""" + INSERT INTO test_cat.db.pruning_test + SELECT id, 'C' as category, CAST(id * 2.5 AS DOUBLE) as value + FROM range(2000, 3000) + """) + + // Filter should prune B and C partitions at manifest level + val df = spark.sql("SELECT * FROM test_cat.db.pruning_test WHERE category = 'A'") + + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + // Iceberg prunes entire manifests when all files in a manifest don't match the filter + assert( + metrics("resultDataFiles").value == 1, + s"Expected 1 result data file, got ${metrics("resultDataFiles").value}") + assert( + metrics("scannedDataManifests").value == 1, + s"Expected 1 scanned manifest, got ${metrics("scannedDataManifests").value}") + assert( + metrics("skippedDataManifests").value == 2, + s"Expected 2 skipped manifests, got ${metrics("skippedDataManifests").value}") + + // Verify the query actually returns correct results + val result = df.collect() + assert(metrics("output_rows").value == 1000) + assert(result.length == 1000, s"Expected 1000 rows, got ${result.length}") + + spark.sql("DROP TABLE test_cat.db.pruning_test") + } + } + } + + test("verify delete file metrics - MOR table") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Equality delete columns force MOR behavior instead of COW + spark.sql(""" + CREATE TABLE test_cat.db.delete_metrics ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read', + 'write.delete.equality-delete-columns' = 'id' + ) + """) + + spark.sql(""" + INSERT INTO test_cat.db.delete_metrics + VALUES + (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), + (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) + """) + + spark.sql("DELETE FROM test_cat.db.delete_metrics WHERE id IN (2, 4, 6)") + + val df = spark.sql("SELECT * FROM test_cat.db.delete_metrics") + + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + // Iceberg may convert equality deletes to positional deletes internally + assert( + metrics("resultDeleteFiles").value > 0, + s"Expected result delete files > 0, got ${metrics("resultDeleteFiles").value}") + assert( + metrics("totalDeleteFileSize").value > 0, + s"Expected total delete file size > 0, got ${metrics("totalDeleteFileSize").value}") + + val hasDeletes = metrics("positionalDeleteFiles").value > 0 || + metrics("equalityDeleteFiles").value > 0 + assert(hasDeletes, "Expected either positional or equality delete files > 0") + + val result = df.collect() + assert(metrics("output_rows").value == 3) + assert(result.length == 3, s"Expected 3 rows after deletes, got ${result.length}") + + spark.sql("DROP TABLE test_cat.db.delete_metrics") + } + } + } + + test("verify output_rows metric reflects row-level filtering in scan") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // Create relatively small files to get multiple row groups per file + "spark.sql.files.maxRecordsPerFile" -> "1000") { + + spark.sql(""" + CREATE TABLE test_cat.db.filter_metric_test ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + """) + + // Insert 10,000 rows with mixed category values + // This ensures row groups will have mixed data that can't be completely eliminated + spark.sql(""" + INSERT INTO test_cat.db.filter_metric_test + SELECT + id, + CASE WHEN id % 2 = 0 THEN 'even' ELSE 'odd' END as category, + CAST(id * 1.5 AS DOUBLE) as value + FROM range(10000) + """) + + // Apply a highly selective filter on id that will filter ~99% of rows + // This filter requires row-level evaluation because: + // - Row groups contain ranges of IDs (0-999, 1000-1999, etc.) + // - The first row group (0-999) cannot be fully eliminated by stats alone + // - Row-level filtering must apply "id < 100" to filter out rows 100-999 + val df = spark.sql(""" + SELECT * FROM test_cat.db.filter_metric_test + WHERE id < 100 + """) + + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + val metrics = scanNodes.head.metrics + + // Execute the query to populate metrics + val result = df.collect() + + // The filter "id < 100" should match exactly 100 rows (0-99) + assert(result.length == 100, s"Expected 100 rows after filter, got ${result.length}") + + // CRITICAL: Verify output_rows metric matches the filtered count + // If row-level filtering is working, this should be 100 + // If only row group filtering is working, this would be ~1000 (entire first row group) + assert( + metrics("output_rows").value == 100, + s"Expected output_rows=100 (filtered count), got ${metrics("output_rows").value}. " + + "This indicates row-level filtering may not be working correctly.") + + // Verify the filter actually selected the right rows + val ids = result.map(_.getInt(0)).sorted + assert(ids.head == 0, s"Expected first id=0, got ${ids.head}") + assert(ids.last == 99, s"Expected last id=99, got ${ids.last}") + assert(ids.forall(_ < 100), "All IDs should be < 100") + + spark.sql("DROP TABLE test_cat.db.filter_metric_test") + } + } + } + + test("schema evolution - read old snapshot after column drop (VERSION AS OF)") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // Force LOCAL mode to use iceberg-rust + "spark.sql.iceberg.read.data-planning-mode" -> "local") { + + // This test verifies that Comet correctly handles reading old snapshots after schema changes, + // which is a form of backward schema evolution. This corresponds to these Iceberg Java tests: + // - TestIcebergSourceHadoopTables::testSnapshotReadAfterDropColumn + // - TestIcebergSourceHadoopTables::testSnapshotReadAfterAddAndDropColumn + // - TestIcebergSourceHiveTables::testSnapshotReadAfterDropColumn + // - TestIcebergSourceHiveTables::testSnapshotReadAfterAddAndDropColumn + // - TestSnapshotSelection::testSnapshotSelectionByTagWithSchemaChange + + // Step 1: Create table with columns (id, data, category) + spark.sql(""" + CREATE TABLE test_cat.db.schema_evolution_test ( + id INT, + data STRING, + category STRING + ) USING iceberg + """) + + // Step 2: Write data with all three columns + spark.sql(""" + INSERT INTO test_cat.db.schema_evolution_test + VALUES (1, 'x', 'A'), (2, 'y', 'A'), (3, 'z', 'B') + """) + + // Get snapshot ID before schema change + val snapshotIdBefore = spark + .sql("SELECT snapshot_id FROM test_cat.db.schema_evolution_test.snapshots ORDER BY committed_at DESC LIMIT 1") + .collect()(0) + .getLong(0) + + // Verify data is correct before schema change + checkIcebergNativeScan("SELECT * FROM test_cat.db.schema_evolution_test ORDER BY id") + + // Step 3: Drop the "data" column + spark.sql("ALTER TABLE test_cat.db.schema_evolution_test DROP COLUMN data") + + // Step 4: Read the old snapshot (before column was dropped) using VERSION AS OF + // This requires using the snapshot's schema, not the current table schema + checkIcebergNativeScan( + s"SELECT * FROM test_cat.db.schema_evolution_test VERSION AS OF $snapshotIdBefore ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.schema_evolution_test") + } + } + } + + test("schema evolution - branch read after adding DATE column") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + "spark.sql.iceberg.read.data-planning-mode" -> "local") { + + // Reproduces: TestSelect::readAndWriteWithBranchAfterSchemaChange + // Error: "Iceberg scan error: Unexpected => unexpected target column type Date32" + // + // Issue: When reading old data from a branch after the table schema evolved to add + // a DATE column, the schema adapter fails to handle Date32 type conversion. + + // Step 1: Create table with (id, data, float_col) + spark.sql(""" + CREATE TABLE test_cat.db.date_branch_test ( + id BIGINT, + data STRING, + float_col FLOAT + ) USING iceberg + """) + + // Step 2: Insert data + spark.sql(""" + INSERT INTO test_cat.db.date_branch_test + VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', CAST('NaN' AS FLOAT)) + """) + + // Step 3: Create a branch at this point using Iceberg API + val catalog = spark.sessionState.catalogManager.catalog("test_cat") + val ident = + org.apache.spark.sql.connector.catalog.Identifier.of(Array("db"), "date_branch_test") + val sparkTable = catalog + .asInstanceOf[org.apache.iceberg.spark.SparkCatalog] + .loadTable(ident) + .asInstanceOf[org.apache.iceberg.spark.source.SparkTable] + val table = sparkTable.table() + val snapshotId = table.currentSnapshot().snapshotId() + table.manageSnapshots().createBranch("test_branch", snapshotId).commit() + + // Step 4: Evolve schema - drop float_col, add date_col + spark.sql("ALTER TABLE test_cat.db.date_branch_test DROP COLUMN float_col") + spark.sql("ALTER TABLE test_cat.db.date_branch_test ADD COLUMN date_col DATE") + + // Step 5: Insert more data with the new schema + spark.sql(""" + INSERT INTO test_cat.db.date_branch_test + VALUES (4, 'd', DATE '2024-04-04'), (5, 'e', DATE '2024-05-05') + """) + + // Step 6: Read from the branch using VERSION AS OF + // This reads old data (id, data, float_col) but applies the current schema (id, data, date_col) + // The old data files don't have date_col, so it should be NULL + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.date_branch_test VERSION AS OF 'test_branch' ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.date_branch_test") + } + } + } + + // Complex type filter tests + test("complex type filter - struct column IS NULL") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.struct_filter_test ( + id INT, + name STRING, + address STRUCT + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.struct_filter_test + VALUES + (1, 'Alice', struct('NYC', 10001)), + (2, 'Bob', struct('LA', 90001)), + (3, 'Charlie', NULL) + """) + + // Test filtering on struct IS NULL - this should fall back to Spark + // (iceberg-rust doesn't support IS NULL on complex type columns yet) + checkSparkAnswer( + "SELECT * FROM test_cat.db.struct_filter_test WHERE address IS NULL ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.struct_filter_test") + } + } + } + + test("complex type filter - struct field filter") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.struct_field_filter_test ( + id INT, + name STRING, + address STRUCT + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.struct_field_filter_test + VALUES + (1, 'Alice', struct('NYC', 10001)), + (2, 'Bob', struct('LA', 90001)), + (3, 'Charlie', struct('NYC', 10002)) + """) + + // Test filtering on struct field - this should use native scan now! + // iceberg-rust supports nested field filters like address.city = 'NYC' + checkIcebergNativeScan( + "SELECT * FROM test_cat.db.struct_field_filter_test WHERE address.city = 'NYC' ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.struct_field_filter_test") + } + } + } + + test("complex type filter - entire struct value") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.struct_value_filter_test ( + id INT, + name STRING, + address STRUCT + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.struct_value_filter_test + VALUES + (1, 'Alice', named_struct('city', 'NYC', 'zip', 10001)), + (2, 'Bob', named_struct('city', 'LA', 'zip', 90001)), + (3, 'Charlie', named_struct('city', 'NYC', 'zip', 10001)) + """) + + // Test filtering on entire struct value - this falls back to Spark + // (Iceberg Java doesn't push down this type of filter) + checkSparkAnswer( + "SELECT * FROM test_cat.db.struct_value_filter_test WHERE address = named_struct('city', 'NYC', 'zip', 10001) ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.struct_value_filter_test") + } + } + } + + test("complex type filter - array column IS NULL") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.array_filter_test ( + id INT, + name STRING, + values ARRAY + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.array_filter_test + VALUES + (1, 'Alice', array(1, 2, 3)), + (2, 'Bob', array(4, 5, 6)), + (3, 'Charlie', NULL) + """) + + // Test filtering on array IS NULL - this should fall back to Spark + // (iceberg-rust doesn't support IS NULL on complex type columns yet) + checkSparkAnswer( + "SELECT * FROM test_cat.db.array_filter_test WHERE values IS NULL ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.array_filter_test") + } + } + } + + test("complex type filter - array element filter") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.array_element_filter_test ( + id INT, + name STRING, + values ARRAY + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.array_element_filter_test + VALUES + (1, 'Alice', array(1, 2, 3)), + (2, 'Bob', array(4, 5, 6)), + (3, 'Charlie', array(1, 7, 8)) + """) + + // Test filtering with array_contains - this should fall back to Spark + // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) + checkSparkAnswer( + "SELECT * FROM test_cat.db.array_element_filter_test WHERE array_contains(values, 1) ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.array_element_filter_test") + } + } + } + + test("complex type filter - entire array value") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.array_value_filter_test ( + id INT, + name STRING, + values ARRAY + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.array_value_filter_test + VALUES + (1, 'Alice', array(1, 2, 3)), + (2, 'Bob', array(4, 5, 6)), + (3, 'Charlie', array(1, 2, 3)) + """) + + // Test filtering on entire array value - this should fall back to Spark + // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) + checkSparkAnswer( + "SELECT * FROM test_cat.db.array_value_filter_test WHERE values = array(1, 2, 3) ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.array_value_filter_test") + } + } + } + + test("complex type filter - map column IS NULL") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.map_filter_test ( + id INT, + name STRING, + properties MAP + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.map_filter_test + VALUES + (1, 'Alice', map('age', 30, 'score', 95)), + (2, 'Bob', map('age', 25, 'score', 87)), + (3, 'Charlie', NULL) + """) + + // Test filtering on map IS NULL - this should fall back to Spark + // (iceberg-rust doesn't support IS NULL on complex type columns yet) + checkSparkAnswer( + "SELECT * FROM test_cat.db.map_filter_test WHERE properties IS NULL ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.map_filter_test") + } + } + } + + test("complex type filter - map key access filter") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + spark.sql(""" + CREATE TABLE test_cat.db.map_key_filter_test ( + id INT, + name STRING, + properties MAP + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO test_cat.db.map_key_filter_test + VALUES + (1, 'Alice', map('age', 30, 'score', 95)), + (2, 'Bob', map('age', 25, 'score', 87)), + (3, 'Charlie', map('age', 30, 'score', 80)) + """) + + // Test filtering with map key access - this should fall back to Spark + // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) + checkSparkAnswer( + "SELECT * FROM test_cat.db.map_key_filter_test WHERE properties['age'] = 30 ORDER BY id") + + spark.sql("DROP TABLE test_cat.db.map_key_filter_test") + } + } + } + + // Helper to create temp directory + def withTempIcebergDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("comet-iceberg-test").toFile + try { + f(dir) + } finally { + def deleteRecursively(file: File): Unit = { + if (file.isDirectory) { + file.listFiles().foreach(deleteRecursively) + } + file.delete() + } + deleteRecursively(dir) + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala new file mode 100644 index 0000000000..a8e1a58eac --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import java.net.URI + +import scala.util.Try + +import org.testcontainers.containers.MinIOContainer +import org.testcontainers.utility.DockerImageName + +import org.apache.spark.SparkConf +import org.apache.spark.sql.CometTestBase + +import software.amazon.awssdk.auth.credentials.{AwsBasicCredentials, StaticCredentialsProvider} +import software.amazon.awssdk.services.s3.S3Client +import software.amazon.awssdk.services.s3.model.{CreateBucketRequest, HeadBucketRequest} + +trait CometS3TestBase extends CometTestBase { + + protected var minioContainer: MinIOContainer = _ + protected val userName = "minio-test-user" + protected val password = "minio-test-password" + + protected def testBucketName: String + + override def beforeAll(): Unit = { + minioContainer = new MinIOContainer(DockerImageName.parse("minio/minio:latest")) + .withUserName(userName) + .withPassword(password) + minioContainer.start() + createBucketIfNotExists(testBucketName) + + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + if (minioContainer != null) { + minioContainer.stop() + } + } + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + conf.set("spark.hadoop.fs.s3a.access.key", userName) + conf.set("spark.hadoop.fs.s3a.secret.key", password) + conf.set("spark.hadoop.fs.s3a.endpoint", minioContainer.getS3URL) + conf.set("spark.hadoop.fs.s3a.path.style.access", "true") + } + + protected def createBucketIfNotExists(bucketName: String): Unit = { + val credentials = AwsBasicCredentials.create(userName, password) + val s3Client = S3Client + .builder() + .endpointOverride(URI.create(minioContainer.getS3URL)) + .credentialsProvider(StaticCredentialsProvider.create(credentials)) + .forcePathStyle(true) + .build() + try { + val bucketExists = Try { + s3Client.headBucket(HeadBucketRequest.builder().bucket(bucketName).build()) + true + }.getOrElse(false) + + if (!bucketExists) { + val request = CreateBucketRequest.builder().bucket(bucketName).build() + s3Client.createBucket(request) + } + } finally { + s3Client.close() + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala b/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala new file mode 100644 index 0000000000..c8d360ae57 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import org.apache.spark.SparkConf +import org.apache.spark.sql.comet.CometIcebergNativeScanExec +import org.apache.spark.sql.execution.SparkPlan + +class IcebergReadFromS3Suite extends CometS3TestBase { + + override protected val testBucketName = "test-iceberg-bucket" + + private def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + + conf.set("spark.sql.catalog.s3_catalog", "org.apache.iceberg.spark.SparkCatalog") + conf.set("spark.sql.catalog.s3_catalog.type", "hadoop") + conf.set("spark.sql.catalog.s3_catalog.warehouse", s"s3a://$testBucketName/warehouse") + + conf.set(CometConf.COMET_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") + conf.set(CometConf.COMET_ICEBERG_NATIVE_ENABLED.key, "true") + + conf + } + + /** Collects all CometIcebergNativeScanExec nodes from a plan */ + private def collectIcebergNativeScans(plan: SparkPlan): Seq[CometIcebergNativeScanExec] = { + collect(plan) { case scan: CometIcebergNativeScanExec => + scan + } + } + + /** + * Helper to verify query correctness and that exactly one CometIcebergNativeScanExec is used. + */ + private def checkIcebergNativeScan(query: String): Unit = { + val (_, cometPlan) = checkSparkAnswer(query) + val icebergScans = collectIcebergNativeScans(cometPlan) + assert( + icebergScans.length == 1, + s"Expected exactly 1 CometIcebergNativeScanExec but found ${icebergScans.length}. Plan:\n$cometPlan") + } + + test("create and query simple Iceberg table from MinIO") { + assume(icebergAvailable, "Iceberg not available in classpath") + + spark.sql(""" + CREATE TABLE s3_catalog.db.simple_table ( + id INT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO s3_catalog.db.simple_table + VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) + """) + + checkIcebergNativeScan("SELECT * FROM s3_catalog.db.simple_table ORDER BY id") + + spark.sql("DROP TABLE s3_catalog.db.simple_table") + } + + test("read partitioned Iceberg table from MinIO") { + assume(icebergAvailable, "Iceberg not available in classpath") + + spark.sql(""" + CREATE TABLE s3_catalog.db.partitioned_table ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (category) + """) + + spark.sql(""" + INSERT INTO s3_catalog.db.partitioned_table VALUES + (1, 'A', 10.5), (2, 'B', 20.3), (3, 'C', 30.7), + (4, 'A', 15.2), (5, 'B', 25.8), (6, 'C', 35.0) + """) + + checkIcebergNativeScan("SELECT * FROM s3_catalog.db.partitioned_table ORDER BY id") + checkIcebergNativeScan( + "SELECT * FROM s3_catalog.db.partitioned_table WHERE category = 'A' ORDER BY id") + + spark.sql("DROP TABLE s3_catalog.db.partitioned_table") + } + + test("filter pushdown to S3-backed Iceberg table") { + assume(icebergAvailable, "Iceberg not available in classpath") + + spark.sql(""" + CREATE TABLE s3_catalog.db.filter_test ( + id INT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO s3_catalog.db.filter_test VALUES + (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7), + (4, 'Diana', 15.2), (5, 'Eve', 25.8) + """) + + checkIcebergNativeScan("SELECT * FROM s3_catalog.db.filter_test WHERE id = 3") + checkIcebergNativeScan("SELECT * FROM s3_catalog.db.filter_test WHERE value > 20.0") + checkIcebergNativeScan("SELECT * FROM s3_catalog.db.filter_test WHERE name = 'Alice'") + + spark.sql("DROP TABLE s3_catalog.db.filter_test") + } + + test("multiple files in S3 - verify no duplicates") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withSQLConf("spark.sql.files.maxRecordsPerFile" -> "50") { + spark.sql(""" + CREATE TABLE s3_catalog.db.multifile_test ( + id INT, + data STRING + ) USING iceberg + """) + + spark.sql(""" + INSERT INTO s3_catalog.db.multifile_test + SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + FROM range(200) + """) + + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM s3_catalog.db.multifile_test") + checkIcebergNativeScan( + "SELECT * FROM s3_catalog.db.multifile_test WHERE id < 10 ORDER BY id") + + spark.sql("DROP TABLE s3_catalog.db.multifile_test") + } + } + + test("MOR table with deletes in S3") { + assume(icebergAvailable, "Iceberg not available in classpath") + + spark.sql(""" + CREATE TABLE s3_catalog.db.mor_delete_test ( + id INT, + name STRING, + value DOUBLE + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' + ) + """) + + spark.sql(""" + INSERT INTO s3_catalog.db.mor_delete_test VALUES + (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7), + (4, 'Diana', 15.2), (5, 'Eve', 25.8) + """) + + spark.sql("DELETE FROM s3_catalog.db.mor_delete_test WHERE id IN (2, 4)") + + checkIcebergNativeScan("SELECT * FROM s3_catalog.db.mor_delete_test ORDER BY id") + + spark.sql("DROP TABLE s3_catalog.db.mor_delete_test") + } +} diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromS3Suite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromS3Suite.scala index fb5f90580e..0fd512c61f 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromS3Suite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromS3Suite.scala @@ -19,84 +19,16 @@ package org.apache.comet.parquet -import java.net.URI - -import scala.util.Try - -import org.testcontainers.containers.MinIOContainer -import org.testcontainers.utility.DockerImageName - -import org.apache.spark.SparkConf -import org.apache.spark.sql.CometTestBase -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.comet.CometNativeScanExec -import org.apache.spark.sql.comet.CometScanExec +import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.{col, expr, max, sum} -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials -import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider -import software.amazon.awssdk.services.s3.S3Client -import software.amazon.awssdk.services.s3.model.CreateBucketRequest -import software.amazon.awssdk.services.s3.model.HeadBucketRequest - -class ParquetReadFromS3Suite extends CometTestBase with AdaptiveSparkPlanHelper { +import org.apache.comet.CometS3TestBase - private var minioContainer: MinIOContainer = _ - private val userName = "minio-test-user" - private val password = "minio-test-password" - private val testBucketName = "test-bucket" +class ParquetReadFromS3Suite extends CometS3TestBase with AdaptiveSparkPlanHelper { - override def beforeAll(): Unit = { - // Start MinIO container - minioContainer = new MinIOContainer(DockerImageName.parse("minio/minio:latest")) - .withUserName(userName) - .withPassword(password) - minioContainer.start() - createBucketIfNotExists(testBucketName) - - // Initialize Spark session - super.beforeAll() - } - - override def afterAll(): Unit = { - super.afterAll() - if (minioContainer != null) { - minioContainer.stop() - } - } - - override protected def sparkConf: SparkConf = { - val conf = super.sparkConf - conf.set("spark.hadoop.fs.s3a.access.key", userName) - conf.set("spark.hadoop.fs.s3a.secret.key", password) - conf.set("spark.hadoop.fs.s3a.endpoint", minioContainer.getS3URL) - conf.set("spark.hadoop.fs.s3a.path.style.access", "true") - } - - private def createBucketIfNotExists(bucketName: String): Unit = { - val credentials = AwsBasicCredentials.create(userName, password) - val s3Client = S3Client - .builder() - .endpointOverride(URI.create(minioContainer.getS3URL)) - .credentialsProvider(StaticCredentialsProvider.create(credentials)) - .forcePathStyle(true) - .build() - try { - val bucketExists = Try { - s3Client.headBucket(HeadBucketRequest.builder().bucket(bucketName).build()) - true - }.getOrElse(false) - - if (!bucketExists) { - val request = CreateBucketRequest.builder().bucket(bucketName).build() - s3Client.createBucket(request) - } - } finally { - s3Client.close() - } - } + override protected val testBucketName = "test-bucket" private def writeTestParquetFile(filePath: String): Unit = { val df = spark.range(0, 1000) diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 3f7ce5b4fb..54854cb831 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -394,7 +394,8 @@ abstract class CometTestBase excludedClasses: Class[_]*): Option[SparkPlan] = { val wrapped = wrapCometSparkToColumnar(plan) wrapped.foreach { - case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec => + case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec | + _: CometIcebergNativeScanExec => case _: CometSinkPlaceHolder | _: CometScanWrapper => case _: CometColumnarToRowExec => case _: CometSparkToColumnarExec => diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala index 1cbe27be91..5ee787ad97 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala @@ -138,6 +138,42 @@ trait CometBenchmarkBase extends SqlBasedBenchmark { saveAsEncryptedParquetV1Table(testDf, dir.getCanonicalPath + "/parquetV1") } + protected def prepareIcebergTable( + dir: File, + df: DataFrame, + tableName: String = "icebergTable", + partition: Option[String] = None): Unit = { + val warehouseDir = new File(dir, "iceberg-warehouse") + + // Configure Hadoop catalog (same pattern as CometIcebergNativeSuite) + spark.conf.set("spark.sql.catalog.benchmark_cat", "org.apache.iceberg.spark.SparkCatalog") + spark.conf.set("spark.sql.catalog.benchmark_cat.type", "hadoop") + spark.conf.set("spark.sql.catalog.benchmark_cat.warehouse", warehouseDir.getAbsolutePath) + + val fullTableName = s"benchmark_cat.db.$tableName" + + // Drop table if exists + spark.sql(s"DROP TABLE IF EXISTS $fullTableName") + + // Create a temp view from the DataFrame + df.createOrReplaceTempView("temp_df_for_iceberg") + + // Create Iceberg table from temp view + val partitionClause = partition.map(p => s"PARTITIONED BY ($p)").getOrElse("") + spark.sql(s""" + CREATE TABLE $fullTableName + USING iceberg + TBLPROPERTIES ('format-version'='2', 'write.parquet.compression-codec' = 'snappy') + $partitionClause + AS SELECT * FROM temp_df_for_iceberg + """) + + // Create temp view for benchmarking + spark.table(fullTableName).createOrReplaceTempView(tableName) + + spark.catalog.dropTempView("temp_df_for_iceberg") + } + protected def saveAsEncryptedParquetV1Table(df: DataFrameWriter[Row], dir: String): Unit = { val encoder = Base64.getEncoder val footerKey = diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala index a5db4f290d..9b2dd186dd 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala @@ -98,6 +98,47 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { } } + def icebergScanBenchmark(values: Int, dataType: DataType): Unit = { + // Benchmarks running through spark sql. + val sqlBenchmark = + new Benchmark(s"SQL Single ${dataType.sql} Iceberg Column Scan", values, output = output) + + withTempPath { dir => + withTempTable("icebergTable") { + prepareIcebergTable( + dir, + spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM $tbl"), + "icebergTable") + + val query = dataType match { + case BooleanType => "sum(cast(id as bigint))" + case _ => "sum(id)" + } + + sqlBenchmark.addCase("SQL Iceberg - Spark") { _ => + withSQLConf( + "spark.memory.offHeap.enabled" -> "true", + "spark.memory.offHeap.size" -> "10g") { + spark.sql(s"select $query from icebergTable").noop() + } + } + + sqlBenchmark.addCase("SQL Iceberg - Comet Iceberg-Rust") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + "spark.memory.offHeap.enabled" -> "true", + "spark.memory.offHeap.size" -> "10g", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + spark.sql(s"select $query from icebergTable").noop() + } + } + + sqlBenchmark.run() + } + } + } + def encryptedScanBenchmark(values: Int, dataType: DataType): Unit = { // Benchmarks running through spark sql. val sqlBenchmark = @@ -652,6 +693,13 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { } } + runBenchmarkWithTable("SQL Single Numeric Iceberg Column Scan", 1024 * 1024 * 128) { v => + Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + .foreach { dataType => + icebergScanBenchmark(v, dataType) + } + } + runBenchmarkWithTable("SQL Single Numeric Encrypted Column Scan", 1024 * 1024 * 128) { v => Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) .foreach { dataType =>