diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 02b544e2d5..0678c9e4d5 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 3a1b82d044..ffffadc138 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 d48d149728..f3f9f13f3b 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -123,6 +123,16 @@ object CometConf extends ShimCometConf { .getOrElse("COMET_PARQUET_SCAN_IMPL", SCAN_AUTO) .toLowerCase(Locale.ROOT)) + 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/iceberg/1.8.1.diff b/dev/diffs/iceberg/1.8.1.diff index 6bda33042a..e7b58902ca 100644 --- a/dev/diffs/iceberg/1.8.1.diff +++ b/dev/diffs/iceberg/1.8.1.diff @@ -1,5 +1,5 @@ diff --git a/build.gradle b/build.gradle -index 7327b38905d..7967109f039 100644 +index 7327b3890..7967109f0 100644 --- a/build.gradle +++ b/build.gradle @@ -780,6 +780,13 @@ project(':iceberg-parquet') { @@ -17,7 +17,7 @@ index 7327b38905d..7967109f039 100644 exclude group: 'org.apache.avro', module: 'avro' // already shaded by Parquet diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml -index 04ffa8f4edc..3a57af3156a 100644 +index 04ffa8f4e..3a57af315 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -34,6 +34,7 @@ azuresdk-bom = "1.2.31" @@ -39,7 +39,7 @@ index 04ffa8f4edc..3a57af3156a 100644 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 00000000000..ddf6c7de5ae +index 000000000..ddf6c7de5 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java @@ -0,0 +1,255 @@ @@ -300,7 +300,7 @@ index 00000000000..ddf6c7de5ae +} 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 00000000000..a3cba401827 +index 000000000..a3cba4018 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java @@ -0,0 +1,260 @@ @@ -565,7 +565,7 @@ index 00000000000..a3cba401827 + } +} 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 2c37a52449e..3442cfc4375 100644 +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 { @@ -644,7 +644,7 @@ index 2c37a52449e..3442cfc4375 100644 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 1fb2372ba56..142e5fbadf1 100644 +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 { @@ -663,7 +663,7 @@ index 1fb2372ba56..142e5fbadf1 100644 return model; } diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle -index e2d2c7a7ac0..f64232dc57f 100644 +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}") { @@ -704,7 +704,7 @@ index e2d2c7a7ac0..f64232dc57f 100644 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 578845e3da2..0118b30683d 100644 +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 { @@ -716,7 +716,7 @@ index 578845e3da2..0118b30683d 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -725,7 +725,7 @@ index 578845e3da2..0118b30683d 100644 .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 ade19de36fe..150a2cddbc8 100644 +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 { @@ -737,7 +737,7 @@ index ade19de36fe..150a2cddbc8 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -746,7 +746,7 @@ index ade19de36fe..150a2cddbc8 100644 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 64edb1002e9..5bb449f1ac7 100644 +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 { @@ -758,7 +758,7 @@ index 64edb1002e9..5bb449f1ac7 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -767,7 +767,7 @@ index 64edb1002e9..5bb449f1ac7 100644 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 a5d0456b0b2..4af408f4861 100644 +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 { @@ -779,7 +779,7 @@ index a5d0456b0b2..4af408f4861 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -788,7 +788,7 @@ index a5d0456b0b2..4af408f4861 100644 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 c6794e43c63..f7359197407 100644 +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 { @@ -800,7 +800,7 @@ index c6794e43c63..f7359197407 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -809,7 +809,7 @@ index c6794e43c63..f7359197407 100644 .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 ac74fb5a109..e011b8b2510 100644 +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 { @@ -821,7 +821,7 @@ index ac74fb5a109..e011b8b2510 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -830,7 +830,7 @@ index ac74fb5a109..e011b8b2510 100644 .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 68c537e34a4..f66be2f3896 100644 +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 { @@ -846,7 +846,7 @@ index 68c537e34a4..f66be2f3896 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -855,7 +855,7 @@ index 68c537e34a4..f66be2f3896 100644 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 4794863ab1b..8bb508f19f8 100644 +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; @@ -958,7 +958,7 @@ index 4794863ab1b..8bb508f19f8 100644 @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 1440e5d1d3f..85cca62e90f 100644 +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; @@ -1043,7 +1043,7 @@ index 1440e5d1d3f..85cca62e90f 100644 } } 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 047c96314b1..88d691a607a 100644 +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; @@ -1068,7 +1068,7 @@ index 047c96314b1..88d691a607a 100644 @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 6235bfe4865..cba108e4326 100644 +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 { @@ -1085,7 +1085,7 @@ index 6235bfe4865..cba108e4326 100644 } 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 bcc0e514c28..98e80068c51 100644 +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; @@ -1106,7 +1106,7 @@ index bcc0e514c28..98e80068c51 100644 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 d36f1a72747..56f8c9bff93 100644 +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> 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 1fb2372ba56..142e5fbadf1 100644 +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 { @@ -654,7 +654,7 @@ index 1fb2372ba56..142e5fbadf1 100644 return model; } diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle -index 572c32f9292..d155f634a4d 100644 +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}") { @@ -695,7 +695,7 @@ index 572c32f9292..d155f634a4d 100644 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 578845e3da2..0118b30683d 100644 +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 { @@ -707,7 +707,7 @@ index 578845e3da2..0118b30683d 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -716,7 +716,7 @@ index 578845e3da2..0118b30683d 100644 .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 ecf9e6f8a59..0f8cced69aa 100644 +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 { @@ -728,7 +728,7 @@ index ecf9e6f8a59..0f8cced69aa 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -737,7 +737,7 @@ index ecf9e6f8a59..0f8cced69aa 100644 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 64edb1002e9..5bb449f1ac7 100644 +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 { @@ -749,7 +749,7 @@ index 64edb1002e9..5bb449f1ac7 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -758,7 +758,7 @@ index 64edb1002e9..5bb449f1ac7 100644 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 a5d0456b0b2..4af408f4861 100644 +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 { @@ -770,7 +770,7 @@ index a5d0456b0b2..4af408f4861 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -779,7 +779,7 @@ index a5d0456b0b2..4af408f4861 100644 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 c6794e43c63..f7359197407 100644 +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 { @@ -791,7 +791,7 @@ index c6794e43c63..f7359197407 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -800,7 +800,7 @@ index c6794e43c63..f7359197407 100644 .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 ac74fb5a109..e011b8b2510 100644 +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 { @@ -812,7 +812,7 @@ index ac74fb5a109..e011b8b2510 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -821,7 +821,7 @@ index ac74fb5a109..e011b8b2510 100644 .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 68c537e34a4..f66be2f3896 100644 +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 { @@ -837,7 +837,7 @@ index 68c537e34a4..f66be2f3896 100644 + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") -+ .config("spark.sql.iceberg.parquet.reader-type", "COMET") ++ .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") @@ -846,7 +846,7 @@ index 68c537e34a4..f66be2f3896 100644 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 16159dcbdff..eba1a2a0fb1 100644 +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 @@ @@ -946,7 +946,7 @@ index 16159dcbdff..eba1a2a0fb1 100644 @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 04ac69476ad..916face2bf2 100644 +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; @@ -1031,7 +1031,7 @@ index 04ac69476ad..916face2bf2 100644 } } 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 047c96314b1..88d691a607a 100644 +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; @@ -1056,7 +1056,7 @@ index 047c96314b1..88d691a607a 100644 @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 6235bfe4865..cba108e4326 100644 +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 { @@ -1073,7 +1073,7 @@ index 6235bfe4865..cba108e4326 100644 } 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 bcc0e514c28..98e80068c51 100644 +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; @@ -1094,7 +1094,7 @@ index bcc0e514c28..98e80068c51 100644 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 d36f1a72747..56f8c9bff93 100644 +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, + /// 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 aa3ad27bd3..72cb081b4d 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.8.1 + 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/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 52bbc07dc6..0ccd6e3522 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -161,6 +161,31 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val nativeOp = QueryPlanSerde.operator2Proto(scan).get CometNativeScanExec(nativeOp, scan.wrapped, scan.session) + // Fully native Iceberg scan for V2 - convert CometBatchScanExec to CometIcebergNativeScanExec + // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule + case scan: CometBatchScanExec + if scan.wrapped.scan.getClass.getName == + "org.apache.iceberg.spark.source.SparkBatchQueryScan" => + // Extract metadata location for CometIcebergNativeScanExec + try { + val metadataLocation = CometIcebergNativeScanExec.extractMetadataLocation(scan.wrapped) + + // Serialize CometBatchScanExec to extract FileScanTasks and get proto + QueryPlanSerde.operator2Proto(scan) match { + case Some(nativeOp) => + // Create native Iceberg scan exec with the serialized proto + CometIcebergNativeScanExec(nativeOp, scan.wrapped, session, metadataLocation) + case None => + // Serialization failed, fall back to CometBatchScanExec + scan + } + } catch { + case e: Exception => + // If we can't extract metadata, fall back to keeping CometBatchScanExec + withInfo(scan, s"Failed to extract Iceberg metadata location: ${e.getMessage}") + scan + } + // Comet JVM + native scan for V1 and V2 case op if isCometScan(op) => val nativeOp = QueryPlanSerde.operator2Proto(op) 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..a3ab451e49 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -45,7 +45,7 @@ import org.apache.comet.CometConf._ import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, isCometScanEnabled, withInfo, withInfos} import org.apache.comet.DataTypeSupport.isComplexType import org.apache.comet.objectstore.NativeConfig -import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} +import org.apache.comet.parquet.{CometParquetScan, Native} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} import org.apache.comet.shims.CometTypeShim @@ -265,25 +265,381 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com withInfos(scanExec, fallbackReasons.toSet) } - // Iceberg scan - case s: SupportsComet => + // 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]() - if (!s.isCometEnabled) { - fallbackReasons += "Comet extension is not enabled for " + - s"${scanExec.scan.getClass.getSimpleName}: not enabled on data source side" + // 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) } + // Iceberg transform functions not yet supported by iceberg-rust + // These functions may be pushed down in filters but return incorrect results + val unsupportedTransforms = Set("truncate") + + val typeChecker = CometScanTypeChecker(SCAN_NATIVE_DATAFUSION) val schemaSupported = - CometBatchScanExec.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons) + typeChecker.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons) if (!schemaSupported) { fallbackReasons += "Comet extension is not enabled for " + s"${scanExec.scan.getClass.getSimpleName}: Schema not supported" } - if (s.isCometEnabled && schemaSupported) { - // When reading from Iceberg, we automatically enable type promotion + // Check if table uses a FileIO implementation compatible with iceberg-rust + // InMemoryFileIO stores files in Java memory rather than on filesystem/object storage, + // which is incompatible with iceberg-rust's FileIO + val fileIOCompatible = if (schemaSupported) { + try { + // table() is a protected method in SparkScan, + // so we need getDeclaredMethod + setAccessible + val tableMethod = scanExec.scan.getClass.getSuperclass.getSuperclass + .getDeclaredMethod("table") + tableMethod.setAccessible(true) + val table = tableMethod.invoke(scanExec.scan) + + // Check if table uses InMemoryFileIO which stores files in Java HashMap + // rather than on filesystem (incompatible with iceberg-rust FileIO) + val ioMethod = table.getClass.getMethod("io") + val fileIO = ioMethod.invoke(table) + // scalastyle:off classforname + val fileIOClassName = fileIO.getClass.getName + // scalastyle:on classforname + + if (fileIOClassName == "org.apache.iceberg.inmemory.InMemoryFileIO") { + fallbackReasons += "Comet does not support InMemoryFileIO table locations" + false + } else { + // FileIO is compatible with iceberg-rust + true + } + } catch { + case e: Exception => + fallbackReasons += s"Could not check FileIO compatibility: ${e.getMessage}" + false + } + } else { + false + } + + // Check Iceberg table format version + val formatVersionSupported = if (schemaSupported && fileIOCompatible) { + try { + // table() is a protected method in SparkScan, + // so we need getDeclaredMethod + setAccessible + val tableMethod = scanExec.scan.getClass.getSuperclass.getSuperclass + .getDeclaredMethod("table") + tableMethod.setAccessible(true) + val table = tableMethod.invoke(scanExec.scan) + + // Try to get formatVersion directly from table + val formatVersion = + try { + val formatVersionMethod = table.getClass.getMethod("formatVersion") + formatVersionMethod.invoke(table).asInstanceOf[Int] + } catch { + case _: NoSuchMethodException => + // 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") + formatVersionMethod.invoke(metadata).asInstanceOf[Int] + } + + 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 + } + } catch { + case e: Exception => + fallbackReasons += "Could not verify Iceberg table " + + s"format version: ${e.getMessage}" + false + } + } else { + false + } + + // Check if all files are Parquet format and use supported filesystem schemes + val (allParquetFiles, allSupportedFilesystems) = + if (schemaSupported && formatVersionSupported) { + try { + // Use reflection to access the protected tasks() method + val tasksMethod = scanExec.scan.getClass.getSuperclass + .getDeclaredMethod("tasks") + tasksMethod.setAccessible(true) + val tasks = tasksMethod.invoke(scanExec.scan).asInstanceOf[java.util.List[_]] + + // scalastyle:off classforname + val contentScanTaskClass = Class.forName("org.apache.iceberg.ContentScanTask") + val contentFileClass = Class.forName("org.apache.iceberg.ContentFile") + // 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 + var allSupportedFs = true + + tasks.asScala.foreach { task => + val dataFile = fileMethod.invoke(task) + val fileFormat = formatMethod.invoke(dataFile) + + // Check file format + if (fileFormat.toString != "PARQUET") { + allParquet = false + } + + // Check filesystem scheme for data file + val filePath = pathMethod.invoke(dataFile).toString + val uri = new URI(filePath) + val scheme = uri.getScheme + + if (scheme != null && !supportedSchemes.contains(scheme)) { + allSupportedFs = false + fallbackReasons += "Iceberg scan contains files with unsupported filesystem" + + s"scheme: $scheme. " + + s"Comet only supports: ${supportedSchemes.mkString(", ")}" + } + + // 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 => + val deletePath = + try { + val locationMethod = contentFileClass.getMethod("location") + locationMethod.invoke(deleteFile).asInstanceOf[String] + } catch { + case _: NoSuchMethodException => + val pathMethod = contentFileClass.getMethod("path") + pathMethod.invoke(deleteFile).asInstanceOf[CharSequence].toString + } + + val deleteUri = new URI(deletePath) + val deleteScheme = deleteUri.getScheme + + if (deleteScheme != null && !supportedSchemes.contains(deleteScheme)) { + allSupportedFs = false + } + } + } catch { + case _: Exception => + // Ignore errors accessing delete files - they may not be supported + } + } + + if (!allParquet) { + fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " + + "Comet only supports Parquet files in Iceberg tables" + } + + (allParquet, allSupportedFs) + } catch { + case e: Exception => + fallbackReasons += "Could not verify file formats or filesystem schemes: " + + s"${e.getMessage}" + (false, false) + } + } else { + (false, false) + } + + // Partition values are deserialized via iceberg-rust's Literal::try_from_json() + // which has incomplete type support (binary/fixed unimplemented, decimals limited) + val partitionTypesSupported = + if (schemaSupported && formatVersionSupported && allParquetFiles && + allSupportedFilesystems) { + try { + val tableMethod = scanExec.scan.getClass.getSuperclass.getSuperclass + .getDeclaredMethod("table") + tableMethod.setAccessible(true) + val table = tableMethod.invoke(scanExec.scan) + + val specMethod = table.getClass.getMethod("spec") + val partitionSpec = specMethod.invoke(table) + val fieldsMethod = partitionSpec.getClass.getMethod("fields") + val fields = fieldsMethod.invoke(partitionSpec).asInstanceOf[java.util.List[_]] + + // scalastyle:off classforname + val partitionFieldClass = Class.forName("org.apache.iceberg.PartitionField") + // scalastyle:on classforname + val sourceIdMethod = partitionFieldClass.getMethod("sourceId") + + val schemaMethod = table.getClass.getMethod("schema") + val schema = schemaMethod.invoke(table) + val findFieldMethod = schema.getClass.getMethod("findField", classOf[Int]) + + var allSupported = true + fields.asScala.foreach { field => + val sourceId = sourceIdMethod.invoke(field).asInstanceOf[Int] + val column = findFieldMethod.invoke(schema, sourceId.asInstanceOf[Object]) + + if (column != null) { + 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) { + allSupported = false + fallbackReasons += "Partition column with high-precision decimal " + + s"(precision=$precision) is not yet supported by iceberg-rust. " + + "Maximum supported precision for partition columns is 28 " + + "(rust_decimal limitation)" + } + } else if (typeStr == "binary" || typeStr.startsWith("fixed[")) { + // Literal::try_from_json returns todo!() for these types + allSupported = false + fallbackReasons += "Partition column with binary or fixed type is not yet " + + "supported by iceberg-rust (Literal::try_from_json todo!())" + } + } + } + + allSupported + } catch { + case _: Exception => + // Avoid blocking valid queries if reflection fails + true + } + } else { + false + } + + // 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 = + if (schemaSupported && formatVersionSupported && allParquetFiles && + allSupportedFilesystems && partitionTypesSupported) { + try { + val filterExpressionsMethod = + scanExec.scan.getClass.getSuperclass.getSuperclass + .getDeclaredMethod("filterExpressions") + filterExpressionsMethod.setAccessible(true) + val filters = + filterExpressionsMethod.invoke(scanExec.scan).asInstanceOf[java.util.List[_]] + + // 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 + } + } + } catch { + case e: Exception => + // Avoid blocking valid queries if reflection fails + logWarning(s"Could not check for complex type predicates: ${e.getMessage}") + true + } + } else { + false + } + + // Check for unsupported Iceberg transform functions in filter expressions + val transformFunctionsSupported = + if (schemaSupported && formatVersionSupported && allParquetFiles && + allSupportedFilesystems && partitionTypesSupported && + complexTypePredicatesSupported) { + try { + val filterExpressionsMethod = + scanExec.scan.getClass.getSuperclass.getSuperclass + .getDeclaredMethod("filterExpressions") + filterExpressionsMethod.setAccessible(true) + val filters = + filterExpressionsMethod.invoke(scanExec.scan).asInstanceOf[java.util.List[_]] + + val hasUnsupportedTransform = filters.asScala.exists { expr => + val exprStr = expr.toString + unsupportedTransforms.exists { transform => + // Match patterns like: truncate[4](ref(name="data")) + exprStr.contains(s"$transform[") + } + } + + if (hasUnsupportedTransform) { + val foundTransforms = unsupportedTransforms.filter { transform => + filters.asScala.exists(expr => expr.toString.contains(s"$transform[")) + } + fallbackReasons += "Iceberg transform function(s) in filter not yet supported " + + s"by iceberg-rust: ${foundTransforms.mkString(", ")}" + false + } else { + true + } + } catch { + case e: Exception => + // Avoid blocking valid queries if reflection fails + logWarning(s"Could not check for transform functions: ${e.getMessage}") + true + } + } else { + false + } + + if (schemaSupported && formatVersionSupported && allParquetFiles && + allSupportedFilesystems && partitionTypesSupported && + complexTypePredicatesSupported && transformFunctionsSupported) { + // Iceberg tables require type promotion for schema evolution (add/drop columns) SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") CometBatchScanExec( scanExec.clone().asInstanceOf[BatchScanExec], diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 63e18c145a..e8c5665db1 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1065,6 +1065,13 @@ object QueryPlanSerde extends Logging with CometExprShim { None } + // Iceberg scan with native execution enabled + // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometExecRule + case scan: CometBatchScanExec + if scan.wrapped.scan.getClass.getName == + "org.apache.iceberg.spark.source.SparkBatchQueryScan" => + IcebergScanSerde.serializeIcebergScan(scan, builder) + case FilterExec(condition, child) if CometConf.COMET_EXEC_FILTER_ENABLED.get(conf) => val cond = exprToProto(condition, child.output) @@ -1700,6 +1707,7 @@ object QueryPlanSerde extends Logging with CometExprShim { }) nativeScanBuilder.addFilePartitions(partitionBuilder.build()) } + } sealed trait SupportLevel diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala new file mode 100644 index 0000000000..1267871bd0 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -0,0 +1,882 @@ +/* + * 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 + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.comet.CometBatchScanExec +import org.apache.spark.sql.comet.CometIcebergNativeScanExec +import org.apache.spark.sql.types._ + +import org.apache.comet.objectstore.NativeConfig +import org.apache.comet.serde.OperatorOuterClass.{Operator, SparkStructField} +import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType} + +/** + * Serialization logic for Iceberg scan operators. + */ +object IcebergScanSerde extends Logging { + + /** + * 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("UnboundPredicate")) { + 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) + if (attr.isEmpty) { + return None + } + + val opName = operation.toString + + opName match { + case "IS_NULL" => + Some(IsNull(attr.get)) + + case "IS_NOT_NULL" | "NOT_NULL" => + Some(IsNotNull(attr.get)) + + case "EQ" => + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attr.get.dataType) + Some(EqualTo(attr.get, value)) + + case "NOT_EQ" => + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attr.get.dataType) + Some(Not(EqualTo(attr.get, value))) + + case "LT" => + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attr.get.dataType) + Some(LessThan(attr.get, value)) + + case "LT_EQ" => + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attr.get.dataType) + Some(LessThanOrEqual(attr.get, value)) + + case "GT" => + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attr.get.dataType) + Some(GreaterThan(attr.get, value)) + + case "GT_EQ" => + val literalMethod = exprClass.getMethod("literal") + val literal = literalMethod.invoke(icebergExpr) + val value = convertIcebergLiteral(literal, attr.get.dataType) + Some(GreaterThanOrEqual(attr.get, value)) + + case "IN" => + val literalsMethod = exprClass.getMethod("literals") + val literals = literalsMethod.invoke(icebergExpr).asInstanceOf[java.util.List[_]] + val values = + literals.asScala.map(lit => convertIcebergLiteral(lit, attr.get.dataType)) + Some(In(attr.get, values.toSeq)) + + case "NOT_IN" => + val literalsMethod = exprClass.getMethod("literals") + val literals = literalsMethod.invoke(icebergExpr).asInstanceOf[java.util.List[_]] + val values = + literals.asScala.map(lit => convertIcebergLiteral(lit, attr.get.dataType)) + Some(Not(In(attr.get, values.toSeq))) + + case _ => + None + } + } else if (exprClass.getName.endsWith("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("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("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("org.apache.iceberg.expressions.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. + * + * This handles extraction of metadata location, catalog properties, file scan tasks, schemas, + * partition data, delete files, and residual expressions from Iceberg scans. + */ + def serializeIcebergScan( + scan: CometBatchScanExec, + builder: Operator.Builder): Option[OperatorOuterClass.Operator] = { + val icebergScanBuilder = OperatorOuterClass.IcebergScan.newBuilder() + + // Extract metadata location for native execution + val metadataLocation = + try { + CometIcebergNativeScanExec.extractMetadataLocation(scan.wrapped) + } catch { + case e: Exception => + logWarning(s"Failed to extract metadata location from Iceberg scan: ${e.getMessage}") + return None + } + + icebergScanBuilder.setMetadataLocation(metadataLocation) + + val catalogProperties = + 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) + + hadoopToIcebergS3Properties(hadoopS3Options) + } catch { + case e: Exception => + logWarning(s"Failed to extract catalog properties from Iceberg scan: ${e.getMessage}") + e.printStackTrace() + Map.empty[String, String] + } + catalogProperties.foreach { case (key, value) => + icebergScanBuilder.putCatalogProperties(key, value) + } + + // Extract name mapping from table metadata (once per scan, shared by all tasks) + val nameMappingJson = + try { + CometIcebergNativeScanExec.extractNameMapping(scan.wrapped) + } catch { + case e: Exception => + logWarning(s"Failed to extract name mapping from Iceberg table: ${e.getMessage}") + e.printStackTrace() + None + } + + // 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()) + } + + // For schema evolution support: extract the scan's expected schema to use for all tasks. + // When reading old snapshots (VERSION AS OF) after schema changes (add/drop columns), + // individual FileScanTasks may have inconsistent schemas - some with the snapshot schema, + // others with the current table schema. By using the scan's expectedSchema() uniformly, + // we ensure iceberg-rust reads all files with the correct snapshot schema. + val globalNameToFieldId = scala.collection.mutable.Map[String, Int]() + var scanSchemaForTasks: Option[Any] = None + + try { + // expectedSchema() is a protected method in SparkScan that returns the Iceberg Schema + // for this scan (which is the snapshot schema for VERSION AS OF queries). + var scanClass: Class[_] = scan.wrapped.scan.getClass + var schemaMethod: java.lang.reflect.Method = null + + // Search through class hierarchy to find expectedSchema() + while (scanClass != null && schemaMethod == null) { + try { + schemaMethod = scanClass.getDeclaredMethod("expectedSchema") + schemaMethod.setAccessible(true) + } catch { + case _: NoSuchMethodException => scanClass = scanClass.getSuperclass + } + } + + if (schemaMethod == null) { + throw new NoSuchMethodException( + "Could not find expectedSchema() method in class hierarchy") + } + + val scanSchema = schemaMethod.invoke(scan.wrapped.scan) + scanSchemaForTasks = Some(scanSchema) + + // Build a field ID mapping from the scan schema as a fallback. + // This is needed when scan.output includes columns that aren't in some task schemas. + val columnsMethod = scanSchema.getClass.getMethod("columns") + val columns = columnsMethod.invoke(scanSchema).asInstanceOf[java.util.List[_]] + + columns.forEach { 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] + + globalNameToFieldId(name) = fieldId + } catch { + case e: Exception => + logWarning(s"Failed to extract field ID from scan schema column: ${e.getMessage}") + } + } + } catch { + case e: Exception => + logWarning(s"Failed to extract scan schema for field ID mapping: ${e.getMessage}") + } + + // 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("org.apache.iceberg.ContentScanTask") + val fileScanTaskClass = Class.forName("org.apache.iceberg.FileScanTask") + val contentFileClass = Class.forName("org.apache.iceberg.ContentFile") + // scalastyle:on classforname + + val fileMethod = contentScanTaskClass.getMethod("file") + val dataFile = fileMethod.invoke(task) + + val filePath = + try { + val locationMethod = contentFileClass.getMethod("location") + locationMethod.invoke(dataFile).asInstanceOf[String] + } catch { + case _: NoSuchMethodException => + val pathMethod = contentFileClass.getMethod("path") + pathMethod.invoke(dataFile).asInstanceOf[CharSequence].toString + } + taskBuilder.setDataFilePath(filePath) + + // 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("org.apache.iceberg.StructLike") + // 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("org.apache.iceberg.PartitionScanTask") + // 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 == "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 deletesMethod = fileScanTaskClass.getMethod("deletes") + val deletes = deletesMethod + .invoke(task) + .asInstanceOf[java.util.List[_]] + val hasDeletes = !deletes.isEmpty + + val schema: AnyRef = + if (hasDeletes) { + taskSchema + } else { + scanSchemaForTasks.map(_.asInstanceOf[AnyRef]).getOrElse(taskSchema) + } + + // scalastyle:off classforname + val schemaParserClass = Class.forName("org.apache.iceberg.SchemaParser") + val schemaClass = Class.forName("org.apache.iceberg.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 columnsMethod = schema.getClass.getMethod("columns") + val columns = + columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] + + val nameToFieldId = scala.collection.mutable.Map[String, Int]() + columns.forEach { 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] + + nameToFieldId(name) = fieldId + } catch { + case e: Exception => + logWarning(s"Failed to extract field ID from column: ${e.getMessage}") + } + } + + // Extract project_field_ids for scan.output columns. + // For schema evolution: try task schema first, then fall back to + // global scan schema. + scan.output.foreach { attr => + val fieldId = nameToFieldId + .get(attr.name) + .orElse(globalNameToFieldId.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 => + logWarning(s"Failed to extract schema from FileScanTask: ${e.getMessage}") + } + + try { + val formatMethod = contentFileClass.getMethod("format") + val format = formatMethod.invoke(dataFile) + taskBuilder.setDataFileFormat(format.toString) + } catch { + case e: Exception => + logWarning( + "Failed to extract file format from FileScanTask," + + s"defaulting to PARQUET: ${e.getMessage}") + taskBuilder.setDataFileFormat("PARQUET") + } + + try { + val deletesMethod = fileScanTaskClass.getMethod("deletes") + val deletes = deletesMethod + .invoke(task) + .asInstanceOf[java.util.List[_]] + + deletes.asScala.foreach { deleteFile => + try { + // scalastyle:off classforname + val deleteFileClass = Class.forName("org.apache.iceberg.DeleteFile") + // scalastyle:on classforname + + val deletePath = + try { + val locationMethod = contentFileClass.getMethod("location") + locationMethod.invoke(deleteFile).asInstanceOf[String] + } catch { + case _: NoSuchMethodException => + val pathMethod = contentFileClass.getMethod("path") + pathMethod + .invoke(deleteFile) + .asInstanceOf[CharSequence] + .toString + } + + 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 "POSITION_DELETES" => "POSITION_DELETES" + case "EQUALITY_DELETES" => "EQUALITY_DELETES" + case other => other + } + } catch { + case _: Exception => + "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 => + logWarning( + s"Failed to extract deletes from FileScanTask: ${e.getMessage}") + } + + 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}") + } + + // Extract partition spec for proper constant identification + 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("org.apache.iceberg.PartitionSpecParser") + val toJsonMethod = partitionSpecParserClass.getMethod( + "toJson", + Class.forName("org.apache.iceberg.PartitionSpec")) + // 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[_]] + + // 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":[...]} + val jsonBuilder = new StringBuilder() + jsonBuilder.append("{\"type\":\"struct\",\"fields\":[") + + var firstField = true + val iter = fields.iterator() + while (iter.hasNext) { + val field = iter.next() + 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 + + val typeMethod = field.getClass.getMethod("type") + val fieldType = typeMethod.invoke(field) + val fieldTypeStr = fieldType.toString + + if (!firstField) jsonBuilder.append(",") + firstField = false + + jsonBuilder.append("{") + jsonBuilder.append("\"id\":").append(fieldId).append(",") + jsonBuilder.append("\"name\":\"").append(fieldName).append("\",") + jsonBuilder.append("\"required\":").append(required).append(",") + jsonBuilder + .append("\"type\":\"") + .append(fieldTypeStr) + .append("\"") + jsonBuilder.append("}") + } + + jsonBuilder.append("]}") + val partitionTypeJson = jsonBuilder.toString + + 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. + val jsonBuilder = new StringBuilder() + jsonBuilder.append("{") + + var first = true + val iter = fields.iterator() + var idx = 0 + while (iter.hasNext) { + val field = iter.next() + 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]) + + if (!first) jsonBuilder.append(",") + first = false + + jsonBuilder.append("\"").append(fieldId.toString).append("\":") + if (value == null) { + jsonBuilder.append("null") + } else { + value match { + case s: String => + jsonBuilder.append("\"").append(s).append("\"") + // 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 => + jsonBuilder.append("\"").append(f.toString).append("\"") + case d: java.lang.Double if d.isNaN || d.isInfinite => + jsonBuilder.append("\"").append(d.toString).append("\"") + case n: Number => + jsonBuilder.append(n.toString) + case b: java.lang.Boolean => + jsonBuilder.append(b.toString) + case _ => + jsonBuilder.append("\"").append(value.toString).append("\"") + } + } + + idx += 1 + } + + jsonBuilder.append("}") + val partitionJson = jsonBuilder.toString() + taskBuilder.setPartitionDataJson(partitionJson) + } + } + } catch { + case e: Exception => + logWarning( + "Failed to extract partition data from FileScanTask: " + + s"${e.getMessage}") + e.printStackTrace() + } + + // Set name mapping if available (shared by all tasks in this scan) + nameMappingJson.foreach { nameMappingStr => + taskBuilder.setNameMappingJson(nameMappingStr) + } + + partitionBuilder.addFileScanTasks(taskBuilder.build()) + } catch { + case e: Exception => + logWarning(s"Failed to serialize FileScanTask: ${e.getMessage}") + } + } + } catch { + case e: Exception => + logWarning( + s"Failed to extract FileScanTasks from InputPartition: ${e.getMessage}") + } + } + + val builtPartition = partitionBuilder.build() + icebergScanBuilder.addFilePartitions(builtPartition) + } + case _ => + } + } catch { + case e: Exception => + logWarning(s"Failed to extract FileScanTasks from Iceberg scan RDD: ${e.getMessage}") + } + + builder.clearChildren() + Some(builder.setIcebergScan(icebergScanBuilder).build()) + } +} 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..ddc2753ec4 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -0,0 +1,338 @@ +/* + * 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.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) + 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) + } + + 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 + case _ => + false + } + } + + override def hashCode(): Int = + Objects.hashCode( + metadataLocation, + output.asJava, + serializedPlanOpt, + numPartitions: java.lang.Integer) +} + +object CometIcebergNativeScanExec { + + /** + * Extracts metadata location from Iceberg table. + * + * @param scanExec + * The Spark BatchScanExec containing an Iceberg scan + * @return + * Path to the table metadata file + */ + def extractMetadataLocation(scanExec: BatchScanExec): String = { + val scan = scanExec.scan + + // Get table via reflection (table() is protected in SparkScan, need to search up hierarchy) + var clazz: Class[_] = scan.getClass + var tableMethod: java.lang.reflect.Method = null + while (clazz != null && tableMethod == null) { + try { + tableMethod = clazz.getDeclaredMethod("table") + tableMethod.setAccessible(true) + } catch { + case _: NoSuchMethodException => clazz = clazz.getSuperclass + } + } + if (tableMethod == null) { + throw new NoSuchMethodException("Could not find table() method in class hierarchy") + } + + val table = tableMethod.invoke(scan) + + val operationsMethod = table.getClass.getMethod("operations") + val operations = operationsMethod.invoke(table) + + val currentMethod = operations.getClass.getMethod("current") + val metadata = currentMethod.invoke(operations) + + val metadataFileLocationMethod = metadata.getClass.getMethod("metadataFileLocation") + metadataFileLocationMethod.invoke(metadata).asInstanceOf[String] + } + + /** + * Extracts name mapping from Iceberg table metadata properties. + * + * Name mapping is stored in table properties as "schema.name-mapping.default" and provides a + * fallback mapping from field names to field IDs for Parquet files that lack field IDs or have + * field ID conflicts (e.g., Hive tables migrated via add_files). + * + * Per Iceberg spec rule #2: "Use schema.name-mapping.default metadata to map field id to + * columns without field id as described below and use the column if it is present." + * + * @param scanExec + * The Spark BatchScanExec containing an Iceberg scan + * @return + * Optional JSON string of the name mapping, or None if not present in table properties + */ + def extractNameMapping(scanExec: BatchScanExec): Option[String] = { + try { + val scan = scanExec.scan + + // Get table via reflection (same as extractMetadataLocation) + var clazz: Class[_] = scan.getClass + var tableMethod: java.lang.reflect.Method = null + while (clazz != null && tableMethod == null) { + try { + tableMethod = clazz.getDeclaredMethod("table") + tableMethod.setAccessible(true) + } catch { + case _: NoSuchMethodException => clazz = clazz.getSuperclass + } + } + if (tableMethod == null) { + return None + } + + val table = tableMethod.invoke(scan) + + // Get table metadata: table.operations().current() + val operationsMethod = table.getClass.getMethod("operations") + val operations = operationsMethod.invoke(table) + + val currentMethod = operations.getClass.getMethod("current") + val metadata = currentMethod.invoke(operations) + + // Get properties map from metadata + val propertiesMethod = metadata.getClass.getMethod("properties") + val properties = propertiesMethod + .invoke(metadata) + .asInstanceOf[java.util.Map[String, String]] + + // Extract name mapping property + val nameMappingKey = "schema.name-mapping.default" + if (properties.containsKey(nameMappingKey)) { + Some(properties.get(nameMappingKey)) + } else { + None + } + } catch { + case _: Exception => + None + } + } + + /** + * 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 from extractMetadataLocation + * @return + * A new CometIcebergNativeScanExec + */ + def apply( + nativeOp: Operator, + scanExec: BatchScanExec, + session: SparkSession, + metadataLocation: String): 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) + + 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 de6892638a..2e9827e377 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 @@ -363,6 +363,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 @@ -377,10 +378,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 => + _: CometIcebergNativeScanExec | _: ShuffleQueryStageExec | _: AQEShuffleReadExec | + _: CometShuffleExchangeExec | _: CometUnionExec | _: CometTakeOrderedAndProjectExec | + _: CometCoalesceExec | _: ReusedExchangeExec | _: CometBroadcastExchangeExec | + _: BroadcastQueryStageExec | _: CometSparkToColumnarExec => 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..3640a0361f --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala @@ -0,0 +1,136 @@ +/* + * 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 + } + } + + /** + * 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] + })) + + 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..4b10263121 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala @@ -0,0 +1,233 @@ +/* + * 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 shuffledPrimitiveCols = Random.shuffle(df.columns.slice(0, 14).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/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 1854edf590..4fe15a7417 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -385,7 +385,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 =>