From cded0ad53ae140b34670aeddab3bc01865cb9a82 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 5 Oct 2025 21:53:21 -0400 Subject: [PATCH 01/89] CometNativeIcebergScan with iceberg-rust using FileScanTasks. --- .../scala/org/apache/comet/CometConf.scala | 10 + docs/source/user-guide/latest/configs.md | 1 + native/Cargo.lock | 1170 ++++++++++++++++- native/Cargo.toml | 6 +- native/core/Cargo.toml | 3 + .../src/execution/operators/iceberg_scan.rs | 238 ++++ native/core/src/execution/operators/mod.rs | 2 + native/core/src/execution/planner.rs | 475 +++++++ native/proto/src/proto/operator.proto | 95 ++ spark/pom.xml | 7 + .../apache/comet/rules/CometExecRule.scala | 5 + .../apache/comet/rules/CometScanRule.scala | 75 +- .../apache/comet/serde/QueryPlanSerde.scala | 533 +++++++- .../comet/CometIcebergNativeScanExec.scala | 401 ++++++ .../apache/spark/sql/comet/operators.scala | 8 +- .../comet/CometIcebergNativeSuite.scala | 1058 +++++++++++++++ .../org/apache/spark/sql/CometTestBase.scala | 3 +- 17 files changed, 4002 insertions(+), 88 deletions(-) create mode 100644 native/core/src/execution/operators/iceberg_scan.rs create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala create mode 100644 spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index c2a5d05829..c3efb0d6e2 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -108,6 +108,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") + .doc( + "Whether to enable native Iceberg scan using iceberg-rust. When enabled, Comet will " + + "replace Spark's Iceberg BatchScanExec with CometIcebergNativeScanExec. Iceberg " + + "planning is performed by Spark, and the resulting FileScanTasks are serialized " + + "and passed to the native execution layer for reading data files.") + .booleanConf + .createWithDefault(false) + val COMET_RESPECT_PARQUET_FILTER_PUSHDOWN: ConfigEntry[Boolean] = conf("spark.comet.parquet.respectFilterPushdown") .doc( diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index bebca3c443..f2714cf19c 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -84,6 +84,7 @@ Comet provides the following configuration settings. | spark.comet.regexp.allowIncompatible | Comet is not currently fully compatible with Spark for all regular expressions. Set this config to true to allow them anyway. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html). | false | | spark.comet.scan.allowIncompatible | Some Comet scan implementations are not currently fully compatible with Spark for all datatypes. Set this config to true to allow them anyway. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html). | false | | spark.comet.scan.enabled | Whether to enable native scans. When this is turned on, Spark will use Comet to read supported data sources (currently only Parquet is supported natively). Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | true | +| spark.comet.scan.icebergNative.enabled | Whether to enable native Iceberg scan using iceberg-rust. When enabled, Comet will replace Spark's Iceberg BatchScanExec with CometIcebergNativeScanExec. Iceberg planning is performed by Spark, and the resulting FileScanTasks are serialized and passed to the native execution layer for reading data files. | false | | spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. | false | | spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | | spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. Note that this config is only used when `spark.comet.exec.shuffle.mode` is `jvm`. | 10.0 | diff --git a/native/Cargo.lock b/native/Cargo.lock index c707a72537..83042cf363 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -17,6 +17,17 @@ version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" +[[package]] +name = "ahash" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" +dependencies = [ + "getrandom 0.2.16", + "once_cell", + "version_check", +] + [[package]] name = "ahash" version = "0.8.12" @@ -97,12 +108,43 @@ version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" +[[package]] +name = "apache-avro" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a033b4ced7c585199fb78ef50fca7fe2f444369ec48080c5fd072efa1a03cc7" +dependencies = [ + "bigdecimal", + "bon", + "digest", + "log", + "miniz_oxide", + "num-bigint", + "quad-rand", + "rand 0.9.2", + "regex-lite", + "serde", + "serde_bytes", + "serde_json", + "strum 0.27.2", + "strum_macros 0.27.2", + "thiserror 2.0.17", + "uuid", + "zstd", +] + [[package]] name = "arc-swap" version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" +[[package]] +name = "array-init" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc" + [[package]] name = "arrayref" version = "0.3.9" @@ -156,7 +198,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-buffer", "arrow-data", "arrow-schema", @@ -239,6 +281,7 @@ dependencies = [ "arrow-select", "flatbuffers", "lz4_flex", + "zstd", ] [[package]] @@ -254,7 +297,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.11.4", "lexical-core", "memchr", "num", @@ -306,7 +349,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-data", @@ -331,6 +374,12 @@ dependencies = [ "regex-syntax", ] +[[package]] +name = "as-any" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0f477b951e452a0b6b4a10b53ccd569042d1d01729b519e02074a9c0958a063" + [[package]] name = "assertables" version = "9.8.2" @@ -360,6 +409,23 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "async-compression" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" +dependencies = [ + "bzip2 0.5.2", + "flate2", + "futures-core", + "memchr", + "pin-project-lite", + "tokio", + "xz2", + "zstd", + "zstd-safe", +] + [[package]] name = "async-executor" version = "1.13.3" @@ -883,8 +949,15 @@ dependencies = [ "num-bigint", "num-integer", "num-traits", + "serde", ] +[[package]] +name = "bimap" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "230c5f1ca6a325a32553f8640d31ac9b49f2411e901e427570154868b46da4f7" + [[package]] name = "bindgen" version = "0.64.0" @@ -939,6 +1012,18 @@ version = "2.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2261d10cca569e4643e526d8dc2e62e433cc8aba21ab764233731f8d369bf394" +[[package]] +name = "bitvec" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" +dependencies = [ + "funty", + "radium", + "tap", + "wyz", +] + [[package]] name = "blake2" version = "0.10.6" @@ -983,6 +1068,54 @@ dependencies = [ "piper", ] +[[package]] +name = "bon" +version = "3.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2529c31017402be841eb45892278a6c21a000c0a17643af326c73a73f83f0fb" +dependencies = [ + "bon-macros", + "rustversion", +] + +[[package]] +name = "bon-macros" +version = "3.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d82020dadcb845a345591863adb65d74fa8dc5c18a0b6d408470e13b7adc7005" +dependencies = [ + "darling 0.21.3", + "ident_case", + "prettyplease", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.106", +] + +[[package]] +name = "borsh" +version = "1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad8646f98db542e39fc66e68a20b2144f6a732636df7c2354e74645faaa433ce" +dependencies = [ + "borsh-derive", + "cfg_aliases", +] + +[[package]] +name = "borsh-derive" +version = "1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdd1d3c0c2f5833f22386f252fe8ed005c7f59fdcddeef025c01b4c3b9fd9ac3" +dependencies = [ + "once_cell", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "brotli" version = "8.0.2" @@ -1010,6 +1143,28 @@ version = "3.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "46c5e41b57b8bba42a04676d81cb89e9ee8e859a1a66f80a5a72e1cb76b34d43" +[[package]] +name = "bytecheck" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23cdc57ce23ac53c931e88a43d06d070a6fd142f2617be5855eb75efc9beb1c2" +dependencies = [ + "bytecheck_derive", + "ptr_meta", + "simdutf8", +] + +[[package]] +name = "bytecheck_derive" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3db406d29fbcd95542e92559bed4d8ad92636d1ca8b3b72ede10b4bcc010e659" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "bytemuck" version = "1.23.2" @@ -1038,6 +1193,34 @@ dependencies = [ "either", ] +[[package]] +name = "bzip2" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49ecfb22d906f800d4fe833b6282cf4dc1c298f5057ca0b5445e5c209735ca47" +dependencies = [ + "bzip2-sys", +] + +[[package]] +name = "bzip2" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bea8dcd42434048e4f7a304411d9273a411f647446c1234a65ce0554923f4cff" +dependencies = [ + "libbz2-rs-sys", +] + +[[package]] +name = "bzip2-sys" +version = "0.1.13+1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225bff33b2141874fe80d71e07d6eec4f85c5c216453dd96388240f96e1acc14" +dependencies = [ + "cc", + "pkg-config", +] + [[package]] name = "cast" version = "0.3.0" @@ -1090,8 +1273,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "145052bdd345b87320e369255277e3fb5152762ad123a901ef5c262dd38fe8d2" dependencies = [ "iana-time-zone", + "js-sys", "num-traits", "serde", + "wasm-bindgen", "windows-link 0.2.0", ] @@ -1193,8 +1378,8 @@ version = "7.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e0d05af1e006a2407bedef5af410552494ce5be9090444dbbcb57258c1af3d56" dependencies = [ - "strum", - "strum_macros", + "strum 0.26.3", + "strum_macros 0.26.4", "unicode-width", ] @@ -1207,6 +1392,12 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "const-oid" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" + [[package]] name = "const-random" version = "0.1.18" @@ -1267,6 +1458,15 @@ dependencies = [ "libc", ] +[[package]] +name = "crc32c" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a47af21622d091a8f0fb295b88bc886ac74efcc613efc19f5d0b21de5c89e47" +dependencies = [ + "rustc_version", +] + [[package]] name = "crc32fast" version = "1.5.0" @@ -1311,6 +1511,15 @@ dependencies = [ "itertools 0.13.0", ] +[[package]] +name = "crossbeam-channel" +version = "0.5.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82b8f8f868b36967f9606790d1903570de9ceaf870a7bf9fbbd3016d636a2cb2" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-deque" version = "0.8.6" @@ -1373,6 +1582,76 @@ dependencies = [ "memchr", ] +[[package]] +name = "darling" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc7f46116c46ff9ab3eb1597a45688b6715c6e628b5c133e288e709a29bcb4ee" +dependencies = [ + "darling_core 0.20.11", + "darling_macro 0.20.11", +] + +[[package]] +name = "darling" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cdf337090841a411e2a7f3deb9187445851f91b309c0c0a29e05f74a00a48c0" +dependencies = [ + "darling_core 0.21.3", + "darling_macro 0.21.3", +] + +[[package]] +name = "darling_core" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d00b9596d185e565c2207a0b01f8bd1a135483d02d9b7b0a54b11da8d53412e" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.106", +] + +[[package]] +name = "darling_core" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1247195ecd7e3c85f83c8d2a366e4210d588e802133e1e355180a9870b517ea4" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.106", +] + +[[package]] +name = "darling_macro" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" +dependencies = [ + "darling_core 0.20.11", + "quote", + "syn 2.0.106", +] + +[[package]] +name = "darling_macro" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" +dependencies = [ + "darling_core 0.21.3", + "quote", + "syn 2.0.106", +] + [[package]] name = "dashmap" version = "6.1.0" @@ -1398,6 +1677,7 @@ dependencies = [ "arrow-schema", "async-trait", "bytes", + "bzip2 0.6.0", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1423,19 +1703,22 @@ dependencies = [ "datafusion-physical-plan", "datafusion-session", "datafusion-sql", + "flate2", "futures", "itertools 0.14.0", "log", "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "regex", "sqlparser", "tempfile", "tokio", "url", "uuid", + "xz2", + "zstd", ] [[package]] @@ -1509,6 +1792,8 @@ dependencies = [ "hdfs-sys", "hdrs", "hex", + "iceberg", + "iceberg-datafusion", "itertools 0.14.0", "jni", "lazy_static", @@ -1527,9 +1812,10 @@ dependencies = [ "pprof", "procfs", "prost", - "rand", + "rand 0.9.2", "regex", "reqwest", + "serde_json", "simd-adler32", "snap", "tempfile", @@ -1591,7 +1877,7 @@ dependencies = [ "futures", "hex", "num", - "rand", + "rand 0.9.2", "regex", "thiserror 2.0.17", "tokio", @@ -1604,19 +1890,20 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "054873d5563f115f83ef4270b560ac2ce4de713905e825a40cac49d6ff348254" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "arrow-ipc", "base64", "chrono", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.11.4", "libc", "log", "object_store", "parquet", "paste", + "recursive", "sqlparser", "tokio", "web-time", @@ -1640,8 +1927,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d855160469020982880fd9bd0962e033d2f4728f56f85a83d8c90785638b6519" dependencies = [ "arrow", + "async-compression", "async-trait", "bytes", + "bzip2 0.6.0", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -1652,16 +1941,20 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", + "flate2", "futures", "glob", "itertools 0.14.0", "log", "object_store", "parquet", - "rand", + "rand 0.9.2", "tempfile", "tokio", + "tokio-util", "url", + "xz2", + "zstd", ] [[package]] @@ -1743,7 +2036,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "tokio", ] @@ -1768,7 +2061,7 @@ dependencies = [ "log", "object_store", "parking_lot", - "rand", + "rand 0.9.2", "tempfile", "url", ] @@ -1788,8 +2081,9 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap", + "indexmap 2.11.4", "paste", + "recursive", "serde_json", "sqlparser", ] @@ -1802,7 +2096,7 @@ checksum = "4a45bee7d2606bfb41ceb1d904ba7cecf69bd5a6f8f3e6c57c3f5a83d84bdd97" dependencies = [ "arrow", "datafusion-common", - "indexmap", + "indexmap 2.11.4", "itertools 0.14.0", "paste", ] @@ -1829,7 +2123,7 @@ dependencies = [ "itertools 0.14.0", "log", "md-5", - "rand", + "rand 0.9.2", "regex", "sha2", "unicode-segmentation", @@ -1842,7 +2136,7 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b05d47426645aef1e73b1a034c75ab2401bc504175feb191accbe211ec24a342" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-doc", @@ -1863,7 +2157,7 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "05c99f648b2b1743de0c1c19eef07e8cc5a085237f172b2e20bf6934e0a804e4" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -1959,9 +2253,10 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-physical-expr", - "indexmap", + "indexmap 2.11.4", "itertools 0.14.0", "log", + "recursive", "regex", "regex-syntax", ] @@ -1972,7 +2267,7 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "10bb87a605d8ce9672d5347c0293c12211b0c03923fc12fbdc665fe76e6f9e01" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr", @@ -1981,7 +2276,7 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.11.4", "itertools 0.14.0", "log", "parking_lot", @@ -2010,7 +2305,7 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "845eb44ef1e04d2a15c6d955cb146b40a41814a7be4377f0a541857d3e257d6f" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2035,6 +2330,7 @@ dependencies = [ "datafusion-pruning", "itertools 0.14.0", "log", + "recursive", ] [[package]] @@ -2043,7 +2339,7 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e6688d17b78104e169d7069749832c20ff50f112be853d2c058afe46c889064" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "arrow-ord", "arrow-schema", @@ -2060,7 +2356,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.11.4", "itertools 0.14.0", "log", "parking_lot", @@ -2141,8 +2437,9 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap", + "indexmap 2.11.4", "log", + "recursive", "regex", "sqlparser", ] @@ -2163,6 +2460,38 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d630bccd429a5bb5a64b5e94f693bfc48c9f8566418fda4c494cc94f911f87cc" dependencies = [ "powerfmt", + "serde", +] + +[[package]] +name = "derive_builder" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "507dfb09ea8b7fa618fcf76e953f4f5e192547945816d5358edffe39f6f94947" +dependencies = [ + "derive_builder_macro", +] + +[[package]] +name = "derive_builder_core" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d5bcf7b024d6835cfb3d473887cd966994907effbe9227e8c8219824d06c4e8" +dependencies = [ + "darling 0.20.11", + "proc-macro2", + "quote", + "syn 2.0.106", +] + +[[package]] +name = "derive_builder_macro" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" +dependencies = [ + "derive_builder_core", + "syn 2.0.106", ] [[package]] @@ -2199,6 +2528,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", + "const-oid", "crypto-common", "subtle", ] @@ -2214,12 +2544,33 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "dissimilar" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8975ffdaa0ef3661bfe02dbdcc06c9f829dfafe6a3c474de366a8d5e44276921" + +[[package]] +name = "dlv-list" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "442039f5147480ba31067cb00ada1adae6892028e40e45fc5de7b7df6dcc1b5f" +dependencies = [ + "const-random", +] + [[package]] name = "dunce" version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" +[[package]] +name = "dyn-clone" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0881ea181b1df73ff77ffaaf9c7544ecc11e82fba9b5f27b262a3c73a332555" + [[package]] name = "either" version = "1.15.0" @@ -2289,6 +2640,16 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "expect-test" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63af43ff4431e848fb47472a920f14fa71c24de13255a5692e93d4e90302acb0" +dependencies = [ + "dissimilar", + "once_cell", +] + [[package]] name = "fastrand" version = "2.3.0" @@ -2395,6 +2756,12 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" +[[package]] +name = "funty" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" + [[package]] name = "futures" version = "0.3.31" @@ -2570,7 +2937,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap", + "indexmap 2.11.4", "slab", "tokio", "tokio-util", @@ -2588,13 +2955,22 @@ dependencies = [ "num-traits", ] +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash 0.7.8", +] + [[package]] name = "hashbrown" version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" dependencies = [ - "ahash", + "ahash 0.8.12", "allocator-api2", ] @@ -2832,27 +3208,97 @@ dependencies = [ ] [[package]] -name = "icu_collections" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" -dependencies = [ - "displaydoc", - "potential_utf", - "yoke", - "zerofrom", - "zerovec", -] - -[[package]] -name = "icu_locale_core" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +name = "iceberg" +version = "0.7.0" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#c85788fca732a846bf38093f1c5f3abded7c6900" dependencies = [ - "displaydoc", - "litemap", - "tinystr", + "anyhow", + "apache-avro", + "array-init", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-ord", + "arrow-schema", + "arrow-select", + "arrow-string", + "as-any", + "async-trait", + "backon", + "base64", + "bimap", + "bytes", + "chrono", + "derive_builder", + "expect-test", + "fnv", + "futures", + "itertools 0.13.0", + "moka", + "murmur3", + "num-bigint", + "once_cell", + "opendal", + "ordered-float 4.6.0", + "parquet", + "rand 0.8.5", + "reqsign", + "reqwest", + "roaring", + "rust_decimal", + "serde", + "serde_bytes", + "serde_derive", + "serde_json", + "serde_repr", + "serde_with", + "strum 0.27.2", + "thrift", + "tokio", + "typed-builder", + "url", + "uuid", + "zstd", +] + +[[package]] +name = "iceberg-datafusion" +version = "0.7.0" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#c85788fca732a846bf38093f1c5f3abded7c6900" +dependencies = [ + "anyhow", + "async-trait", + "datafusion", + "futures", + "iceberg", + "parquet", + "tokio", + "uuid", +] + +[[package]] +name = "icu_collections" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +dependencies = [ + "displaydoc", + "potential_utf", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locale_core" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", "writeable", "zerovec", ] @@ -2917,6 +3363,12 @@ dependencies = [ "zerovec", ] +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + [[package]] name = "idna" version = "1.1.0" @@ -2938,6 +3390,17 @@ dependencies = [ "icu_properties", ] +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", + "serde", +] + [[package]] name = "indexmap" version = "2.11.4" @@ -2946,6 +3409,8 @@ checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "equivalent", "hashbrown 0.16.0", + "serde", + "serde_core", ] [[package]] @@ -2954,8 +3419,8 @@ version = "0.11.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ - "ahash", - "indexmap", + "ahash 0.8.12", + "indexmap 2.11.4", "is-terminal", "itoa", "log", @@ -3165,6 +3630,12 @@ dependencies = [ "lexical-util", ] +[[package]] +name = "libbz2-rs-sys" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" + [[package]] name = "libc" version = "0.2.176" @@ -3277,7 +3748,7 @@ dependencies = [ "log-mdc", "mock_instant", "parking_lot", - "rand", + "rand 0.9.2", "serde", "serde-value", "serde_json", @@ -3304,6 +3775,17 @@ dependencies = [ "twox-hash", ] +[[package]] +name = "lzma-sys" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + [[package]] name = "md-5" version = "0.10.6" @@ -3370,12 +3852,39 @@ version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dce6dd36094cac388f119d2e9dc82dc730ef91c32a6222170d630e5414b956e6" +[[package]] +name = "moka" +version = "0.12.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8261cd88c312e0004c1d51baad2980c66528dfdb2bee62003e643a4d8f86b077" +dependencies = [ + "async-lock", + "crossbeam-channel", + "crossbeam-epoch", + "crossbeam-utils", + "equivalent", + "event-listener 5.4.1", + "futures-util", + "parking_lot", + "portable-atomic", + "rustc_version", + "smallvec", + "tagptr", + "uuid", +] + [[package]] name = "multimap" version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" +[[package]] +name = "murmur3" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9252111cf132ba0929b6f8e030cac2a24b507f3a4d6db6fb2896f27b354c714b" + [[package]] name = "nix" version = "0.26.4" @@ -3419,6 +3928,7 @@ checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", + "serde", ] [[package]] @@ -3487,6 +3997,16 @@ dependencies = [ "libm", ] +[[package]] +name = "num_cpus" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91df4bbde75afed763b708b7eee1e8e7651e02d97f6d5dd763e89367e957b23b" +dependencies = [ + "hermit-abi", + "libc", +] + [[package]] name = "object" version = "0.36.7" @@ -3518,7 +4038,7 @@ dependencies = [ "parking_lot", "percent-encoding", "quick-xml 0.38.3", - "rand", + "rand 0.9.2", "reqwest", "ring", "rustls-pemfile", @@ -3572,6 +4092,7 @@ dependencies = [ "base64", "bytes", "chrono", + "crc32c", "futures", "getrandom 0.2.16", "hdrs", @@ -3581,6 +4102,7 @@ dependencies = [ "md-5", "percent-encoding", "quick-xml 0.37.5", + "reqsign", "reqwest", "serde", "serde_json", @@ -3603,6 +4125,25 @@ dependencies = [ "num-traits", ] +[[package]] +name = "ordered-float" +version = "4.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7bb71e1b3fa6ca1c61f383464aaf2bb0e2f8e772a1f01d486832464de363b951" +dependencies = [ + "num-traits", +] + +[[package]] +name = "ordered-multimap" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49203cdcae0030493bad186b28da2fa25645fa276a51b6fec8010d281e02ef79" +dependencies = [ + "dlv-list", + "hashbrown 0.14.5", +] + [[package]] name = "outref" version = "0.5.2" @@ -3640,11 +4181,11 @@ dependencies = [ [[package]] name = "parquet" -version = "56.0.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7288a07ed5d25939a90f9cb1ca5afa6855faa08ec7700613511ae64bdb0620c" +checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-cast", @@ -3659,11 +4200,14 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "lz4_flex", "num", "num-bigint", "object_store", + "parquet-variant", + "parquet-variant-compute", + "parquet-variant-json", "paste", "ring", "seq-macro", @@ -3675,6 +4219,48 @@ dependencies = [ "zstd", ] +[[package]] +name = "parquet-variant" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a56bf96fdaf5f9392b447cf1e60bfe4b72149ab3309aa3855c02813cc89ad93f" +dependencies = [ + "arrow-schema", + "chrono", + "half", + "indexmap 2.11.4", + "simdutf8", + "uuid", +] + +[[package]] +name = "parquet-variant-compute" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc7ce683368c9f2672379c12e35b5cd664e81964e234d891b4073a8355015ce7" +dependencies = [ + "arrow", + "arrow-schema", + "chrono", + "half", + "parquet-variant", + "parquet-variant-json", +] + +[[package]] +name = "parquet-variant-json" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af8bde078e883e197efe49d315bfa0ecf8f68879d32a2abf34eb09f40ea88f21" +dependencies = [ + "arrow-schema", + "base64", + "chrono", + "parquet-variant", + "serde_json", + "uuid", +] + [[package]] name = "paste" version = "1.0.15" @@ -3700,7 +4286,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ "fixedbitset", - "indexmap", + "indexmap 2.11.4", ] [[package]] @@ -3711,7 +4297,7 @@ checksum = "54acf3a685220b533e437e264e4d932cfbdc4cc7ec0cd232ed73c08d03b8a7ca" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap", + "indexmap 2.11.4", "serde", ] @@ -3824,6 +4410,12 @@ dependencies = [ "windows-sys 0.61.0", ] +[[package]] +name = "portable-atomic" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" + [[package]] name = "potential_utf" version = "0.1.3" @@ -3880,6 +4472,15 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "proc-macro-crate" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" +dependencies = [ + "toml_edit", +] + [[package]] name = "proc-macro2" version = "1.0.101" @@ -3965,6 +4566,41 @@ dependencies = [ "prost", ] +[[package]] +name = "psm" +version = "0.1.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" +dependencies = [ + "cc", +] + +[[package]] +name = "ptr_meta" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0738ccf7ea06b608c10564b31debd4f5bc5e197fc8bfe088f68ae5ce81e7a4f1" +dependencies = [ + "ptr_meta_derive", +] + +[[package]] +name = "ptr_meta_derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b845dbfca988fa33db069c0e230574d15a3088f147a87b64c7589eb662c9ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "quad-rand" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" + [[package]] name = "quick-xml" version = "0.26.0" @@ -4023,7 +4659,7 @@ dependencies = [ "bytes", "getrandom 0.3.3", "lru-slab", - "rand", + "rand 0.9.2", "ring", "rustc-hash 2.1.1", "rustls", @@ -4064,14 +4700,41 @@ version = "5.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" +[[package]] +name = "radium" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha 0.3.1", + "rand_core 0.6.4", +] + [[package]] name = "rand" version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" dependencies = [ - "rand_chacha", - "rand_core", + "rand_chacha 0.9.0", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core 0.6.4", ] [[package]] @@ -4081,7 +4744,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", - "rand_core", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom 0.2.16", ] [[package]] @@ -4113,6 +4785,26 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "recursive" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0786a43debb760f491b1bc0269fe5e84155353c67482b9e60d0cfb596054b43e" +dependencies = [ + "recursive-proc-macro-impl", + "stacker", +] + +[[package]] +name = "recursive-proc-macro-impl" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" +dependencies = [ + "quote", + "syn 2.0.106", +] + [[package]] name = "redox_syscall" version = "0.5.17" @@ -4122,6 +4814,26 @@ dependencies = [ "bitflags 2.9.4", ] +[[package]] +name = "ref-cast" +version = "1.0.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" +dependencies = [ + "ref-cast-impl", +] + +[[package]] +name = "ref-cast-impl" +version = "1.0.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "regex" version = "1.11.3" @@ -4157,6 +4869,44 @@ version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "caf4aa5b0f434c91fe5c7f1ecb6a5ece2130b02ad2a590589dda5146df959001" +[[package]] +name = "rend" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "71fe3824f5629716b1589be05dacd749f6aa084c87e00e016714a8cdfccc997c" +dependencies = [ + "bytecheck", +] + +[[package]] +name = "reqsign" +version = "0.16.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43451dbf3590a7590684c25fb8d12ecdcc90ed3ac123433e500447c7d77ed701" +dependencies = [ + "anyhow", + "async-trait", + "base64", + "chrono", + "form_urlencoded", + "getrandom 0.2.16", + "hex", + "hmac", + "home", + "http 1.3.1", + "log", + "percent-encoding", + "quick-xml 0.37.5", + "rand 0.8.5", + "reqwest", + "rust-ini", + "serde", + "serde_json", + "sha1", + "sha2", + "tokio", +] + [[package]] name = "reqwest" version = "0.12.23" @@ -4223,6 +4973,71 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "rkyv" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" +dependencies = [ + "bitvec", + "bytecheck", + "bytes", + "hashbrown 0.12.3", + "ptr_meta", + "rend", + "rkyv_derive", + "seahash", + "tinyvec", + "uuid", +] + +[[package]] +name = "rkyv_derive" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "roaring" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f08d6a905edb32d74a5d5737a0c9d7e950c312f3c46cb0ca0a2ca09ea11878a0" +dependencies = [ + "bytemuck", + "byteorder", +] + +[[package]] +name = "rust-ini" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "796e8d2b6696392a43bea58116b667fb4c29727dc5abd27d6acf338bb4f688c7" +dependencies = [ + "cfg-if", + "ordered-multimap", +] + +[[package]] +name = "rust_decimal" +version = "1.38.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8975fc98059f365204d635119cf9c5a60ae67b841ed49b5422a9a7e56cdfac0" +dependencies = [ + "arrayvec", + "borsh", + "bytes", + "num-traits", + "rand 0.8.5", + "rkyv", + "serde", + "serde_json", +] + [[package]] name = "rustc-demangle" version = "0.1.26" @@ -4364,12 +5179,42 @@ dependencies = [ "windows-sys 0.61.0", ] +[[package]] +name = "schemars" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cd191f9397d57d581cddd31014772520aa448f65ef991055d7f61582c65165f" +dependencies = [ + "dyn-clone", + "ref-cast", + "serde", + "serde_json", +] + +[[package]] +name = "schemars" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82d20c4491bc164fa2f6c5d44565947a52ad80b9505d8e36f8d54c27c739fcd0" +dependencies = [ + "dyn-clone", + "ref-cast", + "serde", + "serde_json", +] + [[package]] name = "scopeguard" version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" +[[package]] +name = "seahash" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" + [[package]] name = "security-framework" version = "3.5.0" @@ -4421,8 +5266,18 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float", + "ordered-float 2.10.1", + "serde", +] + +[[package]] +name = "serde_bytes" +version = "0.11.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5d440709e79d88e51ac01c4b72fc6cb7314017bb7da9eeff678aa94c10e3ea8" +dependencies = [ "serde", + "serde_core", ] [[package]] @@ -4458,6 +5313,17 @@ dependencies = [ "serde_core", ] +[[package]] +name = "serde_repr" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -4470,13 +5336,45 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_with" +version = "3.14.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c522100790450cf78eeac1507263d0a350d4d5b30df0c8e1fe051a10c22b376e" +dependencies = [ + "base64", + "chrono", + "hex", + "indexmap 1.9.3", + "indexmap 2.11.4", + "schemars 0.9.0", + "schemars 1.0.4", + "serde", + "serde_derive", + "serde_json", + "serde_with_macros", + "time", +] + +[[package]] +name = "serde_with_macros" +version = "3.14.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "327ada00f7d64abaac1e55a6911e90cf665aa051b9a561c7006c157f4633135e" +dependencies = [ + "darling 0.21.3", + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "serde_yaml" version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap", + "indexmap 2.11.4", "itoa", "ryu", "serde", @@ -4582,6 +5480,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec4b661c54b1e4b603b37873a18c59920e4c51ea8ea2cf527d925424dbd4437c" dependencies = [ "log", + "recursive", "sqlparser_derive", ] @@ -4602,18 +5501,46 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +[[package]] +name = "stacker" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "windows-sys 0.59.0", +] + [[package]] name = "str_stack" version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + [[package]] name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" +[[package]] +name = "strum" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" +dependencies = [ + "strum_macros 0.27.2", +] + [[package]] name = "strum_macros" version = "0.26.4" @@ -4627,6 +5554,18 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "strum_macros" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7695ce3845ea4b33927c055a39dc438a45b059f7c1b3d91d38d10355fb8cbca7" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "subtle" version = "2.6.1" @@ -4698,6 +5637,18 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "tagptr" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b2093cf4c8eb1e67749a6762251bc9cd836b6fc171623bd0a9d324d37af2417" + +[[package]] +name = "tap" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" + [[package]] name = "tempfile" version = "3.23.0" @@ -4761,6 +5712,15 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + [[package]] name = "thrift" version = "0.17.0" @@ -4769,7 +5729,9 @@ checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", "integer-encoding", - "ordered-float", + "log", + "ordered-float 2.10.1", + "threadpool", ] [[package]] @@ -4810,6 +5772,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" dependencies = [ "deranged", + "itoa", "num-conv", "powerfmt", "serde", @@ -4931,6 +5894,36 @@ dependencies = [ "tokio", ] +[[package]] +name = "toml_datetime" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32f1085dec27c2b6632b04c80b3bb1b4300d6495d1e129693bdda7d91e72eec1" +dependencies = [ + "serde_core", +] + +[[package]] +name = "toml_edit" +version = "0.23.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3effe7c0e86fdff4f69cdd2ccc1b96f933e24811c5441d44904e8683e27184b" +dependencies = [ + "indexmap 2.11.4", + "toml_datetime", + "toml_parser", + "winnow", +] + +[[package]] +name = "toml_parser" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cf893c33be71572e0e9aa6dd15e6677937abd686b066eac3f8cd3531688a627" +dependencies = [ + "winnow", +] + [[package]] name = "tower" version = "0.5.2" @@ -5019,7 +6012,27 @@ version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ea3136b675547379c4bd395ca6b938e5ad3c3d20fad76e7fe85f9e0d011419c" dependencies = [ - "rand", + "rand 0.9.2", +] + +[[package]] +name = "typed-builder" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd9d30e3a08026c78f246b173243cf07b3696d274debd26680773b6773c2afc7" +dependencies = [ + "typed-builder-macro", +] + +[[package]] +name = "typed-builder-macro" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.106", ] [[package]] @@ -5632,6 +6645,15 @@ version = "0.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" +[[package]] +name = "winnow" +version = "0.7.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "21a0236b59786fed61e2a80582dd500fe61f18b5dca67a4a067d0bc9039339cf" +dependencies = [ + "memchr", +] + [[package]] name = "wit-bindgen" version = "0.46.0" @@ -5644,6 +6666,15 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +[[package]] +name = "wyz" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" +dependencies = [ + "tap", +] + [[package]] name = "xmlparser" version = "0.13.6" @@ -5656,6 +6687,15 @@ version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" +[[package]] +name = "xz2" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" +dependencies = [ + "lzma-sys", +] + [[package]] name = "yoke" version = "0.8.0" diff --git a/native/Cargo.toml b/native/Cargo.toml index 65b9a7eaf2..d8cd5c1198 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,10 +34,10 @@ edition = "2021" rust-version = "1.86" [workspace.dependencies] -arrow = { version = "56.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "56.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } -parquet = { version = "=56.0.0", default-features = false, features = ["experimental"] } +parquet = { version = "=56.2.0", default-features = false, features = ["experimental"] } datafusion = { version = "50.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-spark = { version = "50.0.0" } datafusion-comet-spark-expr = { path = "spark-expr" } @@ -53,6 +53,8 @@ object_store = { version = "0.12.3", features = ["gcp", "azure", "aws", "http"] url = "2.2" aws-config = "1.6.3" aws-credential-types = "1.2.6" +iceberg = { git = "https://github.com/mbutrovich/iceberg-rust", branch = "df50" } +iceberg-datafusion = { git = "https://github.com/mbutrovich/iceberg-rust", branch = "df50" } [profile.release] debug = true diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index a5d11aed65..3726bf725e 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -77,6 +77,9 @@ reqwest = { version = "0.12", default-features = false, features = ["rustls-tls- object_store_opendal = {version = "0.54.0", optional = true} hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} opendal = { version ="0.54.0", optional = true, features = ["services-hdfs"] } +iceberg = { workspace = true } +iceberg-datafusion = { workspace = true } +serde_json = "1.0" [target.'cfg(target_os = "linux")'.dependencies] procfs = "0.18.0" diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs new file mode 100644 index 0000000000..8a89e4bd36 --- /dev/null +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -0,0 +1,238 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Native Iceberg table scan operator using iceberg-rust + +use std::any::Any; +use std::collections::HashMap; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use datafusion::common::{DataFusionError, Result as DFResult}; +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, +}; +use futures::{StreamExt, TryStreamExt}; +use iceberg::io::FileIO; + +use crate::execution::operators::ExecutionError; + +/// Native Iceberg scan operator that uses iceberg-rust to read Iceberg tables. +/// +/// This operator completely bypasses Spark's DataSource V2 API and uses iceberg-rust +/// for all table metadata loading, file discovery, and data reading. +#[derive(Debug)] +pub struct IcebergScanExec { + /// Path to Iceberg table metadata file or directory (used for FileIO creation) + metadata_location: String, + /// Output schema after projection + output_schema: SchemaRef, + /// Cached execution plan properties + plan_properties: PlanProperties, + /// Catalog-specific configuration properties (used to build FileIO for reading files) + catalog_properties: HashMap, + /// Pre-planned file scan tasks from Scala, grouped by partition. + /// Each inner Vec contains tasks for one Spark partition. + file_task_groups: Option>>, +} + +impl IcebergScanExec { + /// Creates a new IcebergScanExec + pub fn new( + metadata_location: String, + schema: SchemaRef, + catalog_properties: HashMap, + file_task_groups: Option>>, + num_partitions: usize, + ) -> Result { + let output_schema = schema; + + // Compute plan properties with actual partition count + let plan_properties = Self::compute_properties(Arc::clone(&output_schema), num_partitions); + + Ok(Self { + metadata_location, + output_schema, + plan_properties, + catalog_properties, + file_task_groups, + }) + } + + /// Computes execution plan properties with actual partition count + fn compute_properties(schema: SchemaRef, num_partitions: usize) -> PlanProperties { + // Use the actual partition count from Iceberg's planning + // This matches the number of Spark partitions and ensures proper parallelism + 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 { + // Use pre-planned tasks from Scala + // All planning happens on the Scala side using Iceberg's Java API + if let Some(ref task_groups) = self.file_task_groups { + if partition < task_groups.len() { + let tasks = &task_groups[partition]; + + return self.execute_with_tasks(tasks.clone()); + } else { + return Err(DataFusionError::Execution(format!( + "IcebergScanExec: Partition index {} out of range (only {} task groups available)", + partition, + task_groups.len() + ))); + } + } + + // If no tasks were provided, this is an error + Err(DataFusionError::Execution(format!( + "IcebergScanExec: No FileScanTasks provided for partition {}. \ + All scan planning must happen on the Scala side.", + partition + ))) + } +} + +impl IcebergScanExec { + /// Execute with pre-planned tasks (fast path) + /// This avoids re-planning the scan and directly reads the assigned files. + /// + /// **MOR (Merge-On-Read) Table Support:** + /// + /// If the FileScanTasks include delete files (for MOR tables), iceberg-rust's ArrowReader + /// automatically applies the deletes during reading: + /// - Positional deletes: Skips rows at specified positions + /// - Equality deletes: Filters out rows matching delete predicates + /// + /// This ensures that deleted rows are not included in the query results. + fn execute_with_tasks( + &self, + tasks: Vec, + ) -> DFResult { + let output_schema = Arc::clone(&self.output_schema); + let catalog_properties = self.catalog_properties.clone(); + let metadata_location = self.metadata_location.clone(); + + let fut = async move { + // Build FileIO from catalog properties + let file_io = Self::load_file_io(&catalog_properties, &metadata_location)?; + + // Convert Vec to a stream + let task_stream = futures::stream::iter(tasks.into_iter().map(Ok)).boxed(); + + // Use iceberg-rust's ArrowReader to read the tasks + let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io).build(); + + // Read the tasks - this returns Result + // No await needed - read() is synchronous + let stream = reader.read(task_stream).map_err(|e| { + DataFusionError::Execution(format!("Failed to read Iceberg tasks: {}", e)) + })?; + + // Stream already has correct error type mapping + let mapped_stream = stream + .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))); + + Ok::<_, DataFusionError>(Box::pin(mapped_stream) + as Pin< + Box> + Send>, + >) + }; + + let stream = futures::stream::once(fut).try_flatten(); + + Ok(Box::pin(RecordBatchStreamAdapter::new( + output_schema, + stream, + ))) + } + + /// Build FileIO from catalog properties + fn load_file_io( + catalog_properties: &HashMap, + metadata_location: &str, + ) -> Result { + // Create a FileIO builder + let mut file_io_builder = FileIO::from_path(metadata_location) + .map_err(|e| DataFusionError::Execution(format!("Failed to create FileIO: {}", e)))?; + + // Add catalog properties as configuration + 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))) + } +} + +impl DisplayAs for IcebergScanExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "IcebergScanExec: metadata_location={}, num_tasks={:?}", + self.metadata_location, + self.file_task_groups + .as_ref() + .map(|groups| groups.iter().map(|g| g.len()).sum::()) + ) + } +} 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 517c037e93..0f9f7466f1 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -18,6 +18,7 @@ //! Converts Spark physical plan to DataFusion physical plan use crate::execution::operators::CopyMode; +use crate::execution::operators::IcebergScanExec; use crate::{ errors::ExpressionError, execution::{ @@ -66,6 +67,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}; @@ -1396,6 +1398,77 @@ impl PhysicalPlanner { Arc::new(SparkPlan::new(spark_plan.plan_id, Arc::new(scan), vec![])), )) } + OpStruct::IcebergScan(scan) => { + // Convert schema + let required_schema: SchemaRef = + convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + + // No projection needed - Spark already projects columns before the scan. + // The required_schema contains exactly the columns we need to read. + // Passing None tells iceberg-rust to use the full required_schema, + // which is the same as passing Some([0, 1, 2, ...]). + + // Extract catalog configuration + let catalog_properties: HashMap = scan + .catalog_properties + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + + // Extract metadata_location from catalog_properties (required) + let metadata_location = catalog_properties + .get("metadata_location") + .ok_or_else(|| { + ExecutionError::GeneralError( + "metadata_location not found in catalog_properties".to_string(), + ) + })? + .clone(); + + // Parse pre-planned FileScanTasks if provided (grouped by partition) + // + // NOTE: We no longer convert scan-level data_filters to predicates here. + // Instead, each FileScanTask contains its own residual expression, which is + // the result of Iceberg's ResidualEvaluator partially evaluating the scan + // filter against that file's partition data. This per-file residual is what + // gets used for row-group level filtering in the Parquet reader. + let file_task_groups = if !scan.file_partitions.is_empty() { + let mut task_groups: Vec> = + Vec::with_capacity(scan.file_partitions.len()); + for partition in &scan.file_partitions { + let tasks = parse_file_scan_tasks(&partition.file_scan_tasks)?; + task_groups.push(tasks); + } + Some(task_groups) + } else { + None + }; + + // Get num_partitions (default to 1 if not specified) + let num_partitions = if scan.num_partitions > 0 { + scan.num_partitions as usize + } else { + 1 + }; + + // Create IcebergScanExec + let iceberg_scan = IcebergScanExec::new( + metadata_location, + required_schema, + catalog_properties, + file_task_groups, + num_partitions, + )?; + + 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)?; @@ -2697,6 +2770,128 @@ fn convert_spark_types_to_arrow_schema( arrow_schema } +/// Parse protobuf FileScanTasks into iceberg-rust FileScanTask objects. +/// +/// This converts the protobuf representation of Iceberg file scan tasks (passed from Scala) +/// into native iceberg-rust FileScanTask objects that can be executed directly. +/// +/// Each task contains a residual expression which is the result of Iceberg's ResidualEvaluator +/// partially evaluating the scan filter against that file's partition data. This residual 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| { + // Parse schema from JSON using iceberg's built-in parser + 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); + + // Parse file format + let data_file_format = match proto_task.data_file_format.as_str() { + "PARQUET" => iceberg::spec::DataFileFormat::Parquet, + "AVRO" => iceberg::spec::DataFileFormat::Avro, + "ORC" => iceberg::spec::DataFileFormat::Orc, + other => { + return Err(ExecutionError::GeneralError(format!( + "Unsupported file format: {}", + other + ))); + } + }; + + // Parse delete files for MOR (Merge-On-Read) table support + // Delete files allow Iceberg to track deletions separately from data files: + // - Positional deletes: Specify exact row positions to skip in data files + // - Equality deletes: Specify column values that should be filtered out + // These deletes are automatically applied by iceberg-rust's ArrowReader during scanning. + // + // NOTE: Spark's DataSource V2 API does not expose delete files through InputPartitions. + // Spark applies MOR deletes during query planning before creating FileScanTasks, so + // task.deletes() typically returns empty even for MOR tables. This is expected. + + 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(ExecutionError::GeneralError(format!( + "Unsupported delete content type: {}", + 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>>()?; + + // Extract and convert residual expression from this task + // + // The residual is a Spark expression that represents the remaining filter + // conditions after partition pruning. Iceberg's ResidualEvaluator creates + // these by partially evaluating the scan filter against each file's partition data. + // + // Process: + // 1. Residuals are serialized from Scala with binding=false, so we receive + // UnboundReference (name-based) expressions rather than BoundReference (index-based) + // 2. Convert Spark expression to Iceberg predicate (name-based) + // 3. Bind the predicate to this file's schema for row-group filtering + let bound_predicate = proto_task + .residual + .as_ref() + .and_then(|residual_expr| { + // Convert Spark expression to Iceberg predicate + 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()?; + + // Build FileScanTask matching iceberg-rust's structure + 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, + }) + }) + .collect(); + + results +} + /// Create CASE WHEN expression and add casting as needed fn create_case_expr( when_then_pairs: Vec<(Arc, Arc)>, @@ -2936,6 +3131,286 @@ fn literal_to_array_ref( } } +// ============================================================================ +// Spark Expression to Iceberg Predicate Conversion +// ============================================================================ +// +// Predicate Pushdown Design: +// =========================== +// For row-group level filtering in Parquet files, predicates follow this conversion path: +// +// 1. Iceberg Expression (Java) - extracted from Spark's Iceberg scan planning +// | [~100 lines: Scala convertIcebergExpression() in QueryPlanSerde.scala] +// v +// 2. Spark Catalyst Expression - standard Spark filter representation +// | [~3000 lines: existing exprToProto() infrastructure in QueryPlanSerde.scala] +// v +// 3. Protobuf Spark Expr - serialized for Rust communication (data_filters field) +// | [~200 lines: convert_spark_filters_to_iceberg_predicate() below] +// v +// 4. Iceberg Predicate (Rust) - bound to each file's schema for row-group pruning +// +// This design leverages Comet's existing expression serialization infrastructure +// (which handles hundreds of expression types) rather than implementing a separate +// Iceberg Java -> Iceberg Rust serialization path. Any new expression types added +// to Comet automatically flow through to Iceberg predicate pushdown. +// +// Supported predicates: =, !=, <, <=, >, >=, IS NULL, IS NOT NULL, IN, AND, OR, NOT +// +// Note: NOT IN predicates are intentionally skipped here because iceberg-rust's +// RowGroupMetricsEvaluator::not_in() always returns MIGHT_MATCH (never prunes). +// These are handled by post-scan CometFilter instead. +// +// Example: For query "SELECT * FROM table WHERE id > 10 AND status = 'active'" +// 1. Iceberg planning extracts: [GreaterThan(id, 10), EqualTo(status, "active")] +// 2. Converted to Catalyst: [GreaterThan(AttributeRef("id"), Literal(10)), ...] +// 3. Serialized to protobuf: [Expr{gt: BinaryExpr{left: ..., right: ...}}, ...] +// 4. Converted here to: Predicate::And(Predicate::Binary(...), Predicate::Binary(...)) +// 5. Bound to schema and passed to iceberg-rust's ArrowReader for row-group filtering +// +// Performance improvement from predicate pushdown: +// - Fewer row groups read (Parquet statistics used for pruning) +// - Reduced I/O (skip row groups that can't match) +// - Faster query execution + +/// Converts a protobuf Spark expression to an Iceberg predicate for row-group filtering. +/// This allows predicate pushdown into Parquet readers. +/// +/// Residuals are serialized with binding=false, so they contain UnboundReference (name-based) +/// rather than BoundReference (index-based), which makes conversion straightforward. +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()?; + + // Try left as column, right as literal + 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), + )); + } + + // Try right as column, left as literal (reverse operator) + 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)) => { + // Check for null literals + if literal.is_null { + return None; + } + + // Match on the oneof value field + 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 57e012b369..541634d121 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; } } @@ -106,6 +107,100 @@ message NativeScan { map object_store_options = 13; } +message IcebergScan { + // Schema to read + repeated SparkStructField required_schema = 1; + + // Catalog-specific configuration properties used to build FileIO for reading files + // MUST contain "metadata_location" key for the table metadata file/directory + // Examples: S3 credentials, warehouse location, metadata_location, etc. + map catalog_properties = 2; + + // Pre-planned file scan tasks grouped by Spark partition + // Each IcebergFilePartition contains tasks for one Spark partition. + // This structure mirrors SparkFilePartition used by NativeScan. + // NOTE: Each task contains its own residual filter expression for row-group pruning. + repeated IcebergFilePartition file_partitions = 3; + + // Total number of Spark partitions + // Used to set up proper partitioning in DataFusion execution plan + int32 num_partitions = 4; +} + +// Groups FileScanTasks for a single Spark partition +// This mirrors SparkFilePartition structure used by NativeScan +message IcebergFilePartition { + repeated IcebergFileScanTask file_scan_tasks = 1; +} + +// Represents a single Iceberg FileScanTask from iceberg-rust +// This maps directly to iceberg::scan::FileScanTask +message IcebergFileScanTask { + // Path to the data file (e.g., s3://bucket/warehouse/db/table/data/00000-0-abc.parquet) + string data_file_path = 1; + + // Byte range within the file to read (for split files) + uint64 start = 2; + uint64 length = 3; + + // Optional: Record count if reading entire file + optional uint64 record_count = 4; + + // File format: "PARQUET", "AVRO", "ORC" + string data_file_format = 5; + + // Iceberg schema for this file (JSON serialized) + // Each file may have a different schema due to schema evolution + string schema_json = 6; + + // Field IDs to project from the file (subset of schema) + repeated int32 project_field_ids = 7; + + // Delete files that apply to this data file (for Merge-On-Read tables) + repeated IcebergDeleteFile delete_files = 8; + + // Residual filter expression for this specific file + // + // This is NOT the same as the scan-level data_filters in IcebergScan! + // + // The residual is created by Iceberg's ResidualEvaluator which partially evaluates + // the scan filter against this file's partition values. For example: + // - Original filter: date >= '2024-01-01' AND status = 'active' + // - File partition: date = '2024-06-15' + // - Residual for this file: status = 'active' (date filter proven true by partition) + // + // Different files can have different residuals based on their partition values. + // This allows optimal row-group level filtering in the Parquet reader. + // + // If empty, no filtering is needed for this file (all rows match). + optional spark.spark_expression.Expr residual = 9; +} + +// Represents an Iceberg delete file (equality or positional deletes) +// Used for MOR (Merge-On-Read) tables where deletes are tracked separately from data files. +// +// Iceberg supports two delete strategies: +// 1. Positional deletes: Delete files contain (file_path, row_position) pairs +// indicating specific rows to skip when reading data files. +// 2. Equality deletes: Delete files contain values for specific columns (identified by +// equality_ids). Any row in the data file matching those values should be filtered out. +// +// The Rust side (iceberg-rust ArrowReader) applies these deletes automatically during scanning. +message IcebergDeleteFile { + // Path to the delete file + string file_path = 1; + + // Content type: "POSITION_DELETES" or "EQUALITY_DELETES" + string content_type = 2; + + // Partition spec ID + int32 partition_spec_id = 3; + + // Equality field IDs (for equality deletes only, empty for positional deletes) + // These IDs identify which columns to use for matching when applying equality 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 77e2d09c6c..dd1323a2b9 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -173,6 +173,13 @@ under the License. software.amazon.awssdk s3 + + + org.apache.iceberg + iceberg-spark-runtime-3.5_2.12 + 1.8.1 + 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 f572417bd6..56ab1b977f 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -160,6 +160,11 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val nativeOp = QueryPlanSerde.operator2Proto(scan).get CometNativeScanExec(nativeOp, scan.wrapped, scan.session) + // CometIcebergNativeScanExec is already a CometNativeExec and will be serialized + // during convertBlock(). Don't wrap it in CometScanWrapper. + case scan: CometIcebergNativeScanExec => + 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 cbca7304d2..cbb932b47e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Generic import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MetadataColumnHelper} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues -import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometIcebergNativeScanExec, CometScanExec, SerializedPlan} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.v2.BatchScanExec @@ -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.shims.CometTypeShim /** @@ -268,15 +268,12 @@ 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" - } - val schemaSupported = CometBatchScanExec.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons) @@ -285,12 +282,60 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com s"${scanExec.scan.getClass.getSimpleName}: Schema not supported" } - if (s.isCometEnabled && schemaSupported) { - // When reading from Iceberg, we automatically enable type promotion - SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") - CometBatchScanExec( - scanExec.clone().asInstanceOf[BatchScanExec], - runtimeFilters = scanExec.runtimeFilters) + if (schemaSupported) { + // Check if native Iceberg execution is enabled + if (CometConf.COMET_ICEBERG_NATIVE_ENABLED.get() && + CometConf.COMET_EXEC_ENABLED.get()) { + + // Try to extract catalog info for native execution + CometIcebergNativeScanExec.extractCatalogInfo(scanExec, session) match { + case Some(catalogInfo) => + // Create native Iceberg scan exec with IcebergScan operator (without tasks) + // Tasks will be extracted per-partition during execution in doExecuteColumnar() + // First create a temporary exec to serialize + val tempExec = CometIcebergNativeScanExec( + org.apache.comet.serde.OperatorOuterClass.Operator.newBuilder().build(), + scanExec.output, + scanExec, + SerializedPlan(None), + catalogInfo.catalogType, + catalogInfo.properties, + catalogInfo.namespace, + catalogInfo.tableName, + 1) + + // Now serialize it to get the IcebergScan operator (without tasks) + val nativeOp = + org.apache.comet.serde.QueryPlanSerde.operator2Proto(tempExec).getOrElse { + // If serialization fails, fall back to Spark + return scanExec + } + + val nativeScan = + CometIcebergNativeScanExec(nativeOp, scanExec, session, catalogInfo) + + // When reading from Iceberg, automatically enable type promotion + SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") + + nativeScan + + case None => + // Catalog not supported, fall back to normal Comet batch scan + fallbackReasons += + "Native Iceberg execution enabled but catalog type not supported " + + s"(${scanExec.table.name()})" + SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") + CometBatchScanExec( + scanExec.clone().asInstanceOf[BatchScanExec], + runtimeFilters = scanExec.runtimeFilters) + } + } else { + // Use regular Comet batch scan (Spark's Iceberg reader with Comet vectors) + SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") + CometBatchScanExec( + scanExec.clone().asInstanceOf[BatchScanExec], + runtimeFilters = scanExec.runtimeFilters) + } } else { withInfos(scanExec, fallbackReasons.toSet) } 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 4d1daacd61..e8382da581 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -49,7 +49,7 @@ import org.apache.comet.CometSparkSessionExtensions.{isCometScan, withInfo} import org.apache.comet.expressions._ import org.apache.comet.objectstore.NativeConfig import org.apache.comet.serde.ExprOuterClass.{AggExpr, Expr, ScalarFunc} -import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, BuildSide, JoinType, Operator} +import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, BuildSide, JoinType, Operator, SparkStructField} import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto} import org.apache.comet.serde.Types.{DataType => ProtoDataType} import org.apache.comet.serde.Types.DataType._ @@ -1180,6 +1180,371 @@ object QueryPlanSerde extends Logging with CometExprShim { None } + // Fully native Iceberg scan for V2 using iceberg-rust + // + // IMPORTANT: This serialization happens on the Spark worker, not the driver! + // Each worker receives a specific InputPartition via Spark's RDD distribution. + // We extract the FileScanTasks from the InputPartition and serialize them to protobuf. + // + // Flow: + // 1. Driver: Iceberg's planInputPartitions() creates InputPartition[] + // (each contains ScanTaskGroup) + // 2. Driver: Spark creates BatchScanExec.inputRDD with these InputPartitions + // 3. Spark distributes RDD partitions to workers + // 4. Worker: This code runs and extracts FileScanTasks from the worker's + // specific InputPartition + // 5. Worker: FileScanTasks are serialized to protobuf and sent to Rust + // 6. Rust: Reads only the files specified in those tasks + case scan: CometIcebergNativeScanExec => + val icebergScanBuilder = OperatorOuterClass.IcebergScan.newBuilder() + + // Serialize catalog properties (contains metadata_location, credentials, S3 config, etc.) + // The native side will extract metadata_location from this map + scan.catalogProperties.foreach { case (key, value) => + icebergScanBuilder.putCatalogProperties(key, value) + } + + // Set number of partitions + icebergScanBuilder.setNumPartitions(scan.numPartitions) + + // Set required_schema from output + scan.output.foreach { attr => + val field = SparkStructField + .newBuilder() + .setName(attr.name) + serializeDataType(attr.dataType).foreach(field.setDataType) + icebergScanBuilder.addRequiredSchema(field.build()) + } + + // No need to serialize projection_vector - scan.output already contains only + // the projected columns from Spark's optimization. The native side will use + // None for projection, which tells iceberg-rust to use the full schema. + + // No need to serialize scan-level data_filters - each FileScanTask already contains + // its own residual expression which is the optimized per-file filter from Iceberg's + // ResidualEvaluator. The residuals are used for row-group level filtering. + + // Extract FileScanTasks from the InputPartitions in the RDD + // Group tasks by Spark partition (similar to how NativeScan groups PartitionedFiles) + var actualNumPartitions = 0 + try { + scan.originalPlan.inputRDD match { + case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => + val partitions = rdd.partitions + partitions.foreach { partition => + // Create a partition builder for this Spark partition + val partitionBuilder = OperatorOuterClass.IcebergFilePartition.newBuilder() + + val inputPartitions = partition + .asInstanceOf[ + org.apache.spark.sql.execution.datasources.v2.DataSourceRDDPartition] + .inputPartitions + + inputPartitions.foreach { inputPartition => + // Extract FileScanTasks from this InputPartition using reflection + // InputPartition is SparkInputPartition containing ScanTaskGroup + val inputPartClass = inputPartition.getClass + + // Get the task group and extract tasks + try { + // Call taskGroup() to get ScanTaskGroup + val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") + taskGroupMethod.setAccessible(true) + val taskGroup = taskGroupMethod.invoke(inputPartition) + + // Call tasks() on ScanTaskGroup to get Collection + val taskGroupClass = taskGroup.getClass + val tasksMethod = taskGroupClass.getMethod("tasks") + val tasksCollection = + tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] + + // Convert to Scala and serialize each task + import scala.jdk.CollectionConverters._ + tasksCollection.asScala.foreach { task => + // Serialize this FileScanTask to protobuf + val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() + + // Extract task properties using reflection + val taskClass = task.getClass + + // Get file() -> DataFile + val fileMethod = taskClass.getDeclaredMethod("file") + fileMethod.setAccessible(true) + val dataFile = fileMethod.invoke(task) + + // Get path from DataFile - use location() (or path()) + val dataFileClass = dataFile.getClass + val filePath = + try { + // Try location() first (newer API) + val locationMethod = dataFileClass.getMethod("location") + locationMethod.invoke(dataFile).asInstanceOf[String] + } catch { + case _: NoSuchMethodException => + // Fall back to path() (older API, returns CharSequence) + val pathMethod = dataFileClass.getMethod("path") + pathMethod.invoke(dataFile).asInstanceOf[CharSequence].toString + } + taskBuilder.setDataFilePath(filePath) + + // Get start offset + val startMethod = taskClass.getDeclaredMethod("start") + startMethod.setAccessible(true) + val start = startMethod.invoke(task).asInstanceOf[Long] + taskBuilder.setStart(start) + + // Get length + val lengthMethod = taskClass.getDeclaredMethod("length") + lengthMethod.setAccessible(true) + val length = lengthMethod.invoke(task).asInstanceOf[Long] + taskBuilder.setLength(length) + + // Get schema and serialize to JSON + try { + val schemaMethod = taskClass.getMethod("schema") + schemaMethod.setAccessible(true) + val schema = schemaMethod.invoke(task) + + // Use Iceberg's SchemaParser.toJson(schema) + // scalastyle:off classforname + // Cannot use Utils.classForName as it's not accessible outside Spark + 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) + + // Extract field IDs from the REQUIRED output schema, + // not the full task schema. + // This ensures we only project the columns actually needed by the query + val columnsMethod = schema.getClass.getMethod("columns") + columnsMethod.setAccessible(true) + val columns = + columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] + + // Build a map of column name -> field ID from the task schema + val nameToFieldId = scala.collection.mutable.Map[String, Int]() + columns.forEach { column => + try { + val nameMethod = column.getClass.getMethod("name") + nameMethod.setAccessible(true) + val name = nameMethod.invoke(column).asInstanceOf[String] + + val fieldIdMethod = column.getClass.getMethod("fieldId") + fieldIdMethod.setAccessible(true) + val fieldId = fieldIdMethod.invoke(column).asInstanceOf[Int] + + nameToFieldId(name) = fieldId + } catch { + case _: Exception => // Skip if can't get field ID + } + } + + // Now add field IDs ONLY for columns in scan.output (the required schema) + scan.output.foreach { attr => + nameToFieldId.get(attr.name) match { + case Some(fieldId) => + taskBuilder.addProjectFieldIds(fieldId) + case None => + } + } + } catch { + case e: Exception => + // Could not extract schema from task + logWarning( + s"Failed to extract schema from FileScanTask: ${e.getMessage}") + } + + // Get file format + try { + val formatMethod = dataFileClass.getMethod("format") + formatMethod.setAccessible(true) + val format = formatMethod.invoke(dataFile) + taskBuilder.setDataFileFormat(format.toString) + } catch { + case e: Exception => + // Could not extract file format, defaulting to Parquet + logWarning( + "Failed to extract file format from FileScanTask," + + s"defaulting to PARQUET: ${e.getMessage}") + taskBuilder.setDataFileFormat("PARQUET") // Default to Parquet + } + + // Extract delete files from FileScanTask for MOR (Merge-On-Read) tables. + // When present, these are serialized to protobuf and passed to + // iceberg-rust, which automatically applies deletes during reading. + try { + val deletesMethod = taskClass.getDeclaredMethod("deletes") + deletesMethod.setAccessible(true) + val deletes = deletesMethod + .invoke(task) + .asInstanceOf[java.util.List[_]] + + // Serialize delete files if present + deletes.asScala.foreach { deleteFile => + try { + val deleteFileClass = deleteFile.getClass + + // Get file path - try location() first, then path() + val deletePath = + try { + val locationMethod = deleteFileClass.getMethod("location") + locationMethod.setAccessible(true) + locationMethod.invoke(deleteFile).asInstanceOf[String] + } catch { + case _: NoSuchMethodException => + // Fall back to path() + val pathMethod = deleteFileClass.getDeclaredMethod("path") + pathMethod.setAccessible(true) + pathMethod + .invoke(deleteFile) + .asInstanceOf[CharSequence] + .toString + } + + val deleteBuilder = + OperatorOuterClass.IcebergDeleteFile.newBuilder() + deleteBuilder.setFilePath(deletePath) + + // Get content type (POSITION_DELETES or EQUALITY_DELETES) + val contentType = + try { + val contentMethod = deleteFileClass.getMethod("content") + contentMethod.setAccessible(true) + val content = contentMethod.invoke(deleteFile) + content.toString match { + case "POSITION_DELETES" => "POSITION_DELETES" + case "EQUALITY_DELETES" => "EQUALITY_DELETES" + case other => other + } + } catch { + case _: Exception => + // Default to POSITION_DELETES if can't determine + "POSITION_DELETES" + } + deleteBuilder.setContentType(contentType) + + // Get partition spec ID + val specId = + try { + val specIdMethod = deleteFileClass.getMethod("specId") + specIdMethod.setAccessible(true) + specIdMethod.invoke(deleteFile).asInstanceOf[Int] + } catch { + case _: Exception => + // Default to 0 if can't get spec ID + 0 + } + deleteBuilder.setPartitionSpecId(specId) + + // Get equality field IDs (for equality deletes) + try { + val equalityIdsMethod = + deleteFileClass.getMethod("equalityFieldIds") + equalityIdsMethod.setAccessible(true) + val equalityIds = equalityIdsMethod + .invoke(deleteFile) + .asInstanceOf[java.util.List[Integer]] + equalityIds.forEach(id => deleteBuilder.addEqualityIds(id)) + } catch { + case _: Exception => + // No equality IDs (likely positional deletes) + } + + taskBuilder.addDeleteFiles(deleteBuilder.build()) + } catch { + case e: Exception => + // Failed to serialize delete file - log and continue + logWarning(s"Failed to serialize delete file: ${e.getMessage}") + } + } + } catch { + case _: NoSuchMethodException => + // FileScanTask doesn't have deletes() method + case _: Exception => + // Failed to extract deletes + } + + // Extract residual expression for row-group level filtering + // + // The residual is created by Iceberg's ResidualEvaluator which partially + // evaluates the scan filter against this file's partition values. + // Different files may have different residuals based on their partitions. + // + // For example: + // - Original filter: date >= '2024-01-01' AND status = 'active' + // - File partition: date = '2024-06-15' + // - Residual: status = 'active' (date condition proven true by partition) + // + // This residual is what should be applied during Parquet row-group + // scanning. + try { + val residualMethod = taskClass.getMethod("residual") + residualMethod.setAccessible(true) + val residualExpr = residualMethod.invoke(task) + + // Convert Iceberg Expression to Catalyst Expression + // The residual is an org.apache.iceberg.expressions.Expression + val catalystExpr = convertIcebergExpression(residualExpr, scan.output) + + // Serialize to protobuf WITHOUT binding to indices + // Iceberg residuals are already unbound (name-based), so we keep them + // unbound in the protobuf to avoid unnecessary index->name resolution + // in Rust + catalystExpr + .flatMap { expr => + exprToProto(expr, scan.output, binding = false) + } + .foreach { protoExpr => + taskBuilder.setResidual(protoExpr) + } + } catch { + case _: NoSuchMethodException => + // residual() method not available, skip + case e: Exception => + // Failed to extract/convert residual, continue without it + logWarning( + "Failed to extract residual expression from FileScanTask: " + + s"${e.getMessage}") + } + + // Add task to THIS partition's builder + partitionBuilder.addFileScanTasks(taskBuilder.build()) + } + } catch { + case e: Exception => + // Could not extract tasks from this InputPartition + logWarning( + s"Failed to extract FileScanTasks from InputPartition: ${e.getMessage}") + } + } + + // Add this partition to the scan builder + val builtPartition = partitionBuilder.build() + icebergScanBuilder.addFilePartitions(builtPartition) + actualNumPartitions += 1 + } + case _ => + // Not a DataSourceRDD, cannot extract tasks + } + } catch { + case e: Exception => + logWarning(s"Failed to extract FileScanTasks from Iceberg scan RDD: ${e.getMessage}") + } + + // Set number of partitions for proper data distribution + // Use the actual count of partitions we serialized, not scan.numPartitions + val numPartitions = + if (actualNumPartitions > 0) actualNumPartitions else scan.numPartitions + icebergScanBuilder.setNumPartitions(numPartitions) + + // Iceberg scans don't have children + builder.clearChildren() + + Some(builder.setIcebergScan(icebergScanBuilder).build()) + case FilterExec(condition, child) if CometConf.COMET_EXEC_FILTER_ENABLED.get(conf) => val cond = exprToProto(condition, child.output) @@ -1824,6 +2189,172 @@ object QueryPlanSerde extends Logging with CometExprShim { }) nativeScanBuilder.addFilePartitions(partitionBuilder.build()) } + + /** + * 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. + */ + private 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 e: Exception => + None + } + } + + /** + * Converts an Iceberg Literal to a Spark Literal + */ + private def convertIcebergLiteral(icebergLiteral: Any, sparkType: DataType): Literal = { + // Find value() method in class hierarchy (may be in parent class) + def findValueMethod(clazz: Class[_]): Option[java.lang.reflect.Method] = { + try { + val method = clazz.getDeclaredMethod("value") + method.setAccessible(true) + Some(method) + } catch { + case _: NoSuchMethodException => + if (clazz.getSuperclass != null) { + findValueMethod(clazz.getSuperclass) + } else { + None + } + } + } + + val valueMethod = findValueMethod(icebergLiteral.getClass).getOrElse( + throw new RuntimeException(s"Could not find value() method on ${icebergLiteral.getClass}")) + 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) + } } sealed trait SupportLevel 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..4554af4fc6 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -0,0 +1,401 @@ +/* + * 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 com.google.common.base.Objects + +import org.apache.comet.serde.OperatorOuterClass.Operator + +/** + * Comet fully native Iceberg scan node for DataSource V2 that delegates to iceberg-rust. + * + * This replaces Spark's Iceberg BatchScanExec with a native implementation that: + * 1. Extracts catalog configuration from Spark session 2. Serializes catalog info (type, + * properties, namespace, table name) to protobuf 3. Extracts FileScanTasks from Iceberg's + * InputPartitions during planning (on driver) 4. Uses iceberg-rust's catalog implementations + * to load the table and read data natively + * + * **How FileScanTask Serialization Works:** + * + * This implementation follows the same pattern as CometNativeScanExec for PartitionedFiles: + * + * 1. **At Planning Time (on Driver):** + * - CometScanRule creates CometIcebergNativeScanExec with originalPlan (BatchScanExec) + * - originalPlan.inputRDD is a DataSourceRDD containing DataSourceRDDPartition objects + * - Each partition contains InputPartition objects (from Iceberg's planInputPartitions()) + * - Each InputPartition wraps a ScanTaskGroup containing FileScanTask objects + * + * 2. **During Serialization (in QueryPlanSerde.operator2Proto):** + * - When serializing CometIcebergNativeScanExec, we iterate through ALL RDD partitions + * - For each partition, extract InputPartitions and their FileScanTasks using reflection + * - Serialize each FileScanTask (file path, start, length, delete files) into protobuf + * - This happens ONCE on the driver, not per-worker + * + * 3. **At Execution Time (on Workers):** + * - The serialized plan (with all FileScanTasks) is sent to workers + * - Standard CometNativeExec.doExecuteColumnar() flow executes the native plan + * - Rust receives IcebergScan operator with FileScanTasks for ALL partitions + * - Each worker reads only the tasks for its partition index + * + * **Key Insight:** Unlike the initial approach which tried to extract tasks per-partition at + * execution time, this approach extracts ALL tasks at planning time (just like PartitionedFiles). + * This works because: + * - The RDD and its partitions exist on the driver + * - We don't need TaskContext to access InputPartitions + * - Iceberg has already done the planning and assigned tasks to partitions + * - We just need to serialize this information into the protobuf plan + * + * **Why This Works With Filters:** When a filter is on top of CometIcebergNativeScanExec: + * - Filter's convertBlock() serializes both filter and scan together + * - The scan's nativeOp (created by operator2Proto) already contains all FileScanTasks + * - The combined filter+scan native plan is executed as one unit + * - No special RDD or per-partition logic needed + */ +case class CometIcebergNativeScanExec( + override val nativeOp: Operator, + override val output: Seq[Attribute], + @transient override val originalPlan: BatchScanExec, + override val serializedPlanOpt: SerializedPlan, + catalogType: String, + catalogProperties: Map[String, String], + namespace: Seq[String], + tableName: String, + numPartitions: Int) // Number of Spark partitions for proper parallelism + extends CometLeafExec { + + override val supportsColumnar: Boolean = true + + // No need to override doExecuteColumnar - parent CometLeafExec handles it + // FileScanTasks are serialized at planning time in QueryPlanSerde.operator2Proto() + // just like PartitionedFiles are for CometNativeScanExec + + override val nodeName: String = + s"CometIcebergNativeScan ${namespace.mkString(".")}.$tableName ($catalogType)" + + // Use the actual number of partitions from Iceberg's planning. + // FileScanTasks are extracted and serialized at planning time (in QueryPlanSerde.operator2Proto), + // grouped by partition. Each partition receives only its assigned tasks via the protobuf message. + // The Rust side uses the partition index to select the correct task group. + override lazy val outputPartitioning: Partitioning = + UnknownPartitioning(numPartitions) + + override lazy val outputOrdering: Seq[SortOrder] = Nil + + override protected def doCanonicalize(): CometIcebergNativeScanExec = { + CometIcebergNativeScanExec( + nativeOp, + output.map(QueryPlan.normalizeExpressions(_, output)), + originalPlan.doCanonicalize(), + SerializedPlan(None), + catalogType, + catalogProperties, + namespace, + tableName, + numPartitions) + } + + override def stringArgs: Iterator[Any] = + Iterator(output, catalogType, namespace, tableName, numPartitions) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometIcebergNativeScanExec => + this.catalogType == other.catalogType && + this.catalogProperties == other.catalogProperties && + this.namespace == other.namespace && + this.tableName == other.tableName && + this.output == other.output && + this.serializedPlanOpt == other.serializedPlanOpt && + this.numPartitions == other.numPartitions + case _ => + false + } + } + + override def hashCode(): Int = + Objects.hashCode( + catalogType, + namespace.asJava, + tableName, + output.asJava, + serializedPlanOpt, + numPartitions: java.lang.Integer) + + override lazy val metrics: Map[String, SQLMetric] = { + Map( + "output_rows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "time_elapsed_opening" -> + SQLMetrics.createNanoTimingMetric( + sparkContext, + "Wall clock time elapsed for catalog loading and table opening"), + "time_elapsed_scanning_until_data" -> + SQLMetrics.createNanoTimingMetric( + sparkContext, + "Wall clock time elapsed for scanning + first record batch"), + "time_elapsed_scanning_total" -> + SQLMetrics.createNanoTimingMetric( + sparkContext, + "Total wall clock time for scanning + decompression/decoding"), + "time_elapsed_processing" -> + SQLMetrics.createNanoTimingMetric( + sparkContext, + "Wall clock time elapsed for data decompression + decoding"), + "bytes_scanned" -> + SQLMetrics.createSizeMetric(sparkContext, "Number of bytes scanned"), + "files_scanned" -> + SQLMetrics.createMetric(sparkContext, "Number of data files scanned"), + "manifest_files_scanned" -> + SQLMetrics.createMetric(sparkContext, "Number of manifest files scanned")) + } +} + +object CometIcebergNativeScanExec { + + /** + * Information extracted from an Iceberg table scan for native execution. + */ + case class IcebergCatalogInfo( + catalogType: String, + properties: Map[String, String], + namespace: Seq[String], + tableName: String) + + /** + * Extracts Iceberg catalog configuration from a Spark BatchScanExec. + * + * This method: + * 1. Gets the catalog name from the table identifier 2. Extracts catalog configuration from + * Spark session config 3. Maps Spark's catalog implementation class to iceberg-rust + * catalog types 4. Returns catalog info ready for serialization + * + * @param scanExec + * The Spark BatchScanExec containing an Iceberg scan + * @param session + * The SparkSession to extract catalog config from + * @return + * Some(IcebergCatalogInfo) if catalog is supported, None otherwise + */ + def extractCatalogInfo( + scanExec: BatchScanExec, + session: SparkSession): Option[IcebergCatalogInfo] = { + try { + // Get the full table name from Spark's table identifier + // Format: "catalog_name.namespace.table_name" or "namespace.table_name" + val fullTableName = scanExec.table.name() + val parts = fullTableName.split('.') + + if (parts.length < 2) { + return None // Need at least namespace.table + } + + // Determine catalog name and table path + // If 3+ parts: parts(0) is catalog, parts(1..-2) is namespace, parts(-1) is table + // If 2 parts: default catalog, parts(0) is namespace, parts(1) is table + val (catalogName, namespaceParts, tableNamePart) = if (parts.length >= 3) { + (parts.head, parts.slice(1, parts.length - 1).toSeq, parts.last) + } else { + // Try to get default catalog from config + val defaultCatalog = session.conf + .getOption("spark.sql.catalog.spark_catalog") + .map(_ => "spark_catalog") + .getOrElse(return None) + (defaultCatalog, Seq(parts.head), parts.last) + } + + // Get catalog properties from Spark session config + val catalogPrefix = s"spark.sql.catalog.$catalogName" + + // Check both catalog-impl and type properties + val catalogImpl = session.conf.getOption(s"$catalogPrefix.catalog-impl") + val catalogType = session.conf.getOption(s"$catalogPrefix.type") + + // Handle Hadoop catalog specially - it uses direct metadata file access + if (catalogType.contains("hadoop") || + catalogImpl.exists(impl => impl.contains("HadoopCatalog"))) { + + // Hadoop catalog is filesystem-based, need to extract metadata location + // Try to get it from the table object via reflection + try { + val scan = scanExec.scan + val scanClass = scan.getClass + + // Try to get the table via reflection + // Iceberg's SparkBatchQueryScan extends SparkScan which has protected table() method + // Need to search up the class hierarchy + def findTableMethod(clazz: Class[_]): Option[java.lang.reflect.Method] = { + if (clazz == null || clazz == classOf[Object]) { + None + } else { + try { + val method = clazz.getDeclaredMethod("table") + method.setAccessible(true) + Some(method) + } catch { + case _: NoSuchMethodException => + // Try superclass + findTableMethod(clazz.getSuperclass) + } + } + } + + val tableMethod = findTableMethod(scanClass).getOrElse { + throw new NoSuchMethodException("Could not find table() method in class hierarchy") + } + + val table = tableMethod.invoke(scan) + + // Get the metadata location from table.operations().current().metadataFileLocation() + val tableClass = table.getClass + val operationsMethod = tableClass.getMethod("operations") + val operations = operationsMethod.invoke(table) + + val operationsClass = operations.getClass + val currentMethod = operationsClass.getMethod("current") + val metadata = currentMethod.invoke(operations) + + val metadataClass = metadata.getClass + val metadataFileLocationMethod = metadataClass.getMethod("metadataFileLocation") + val metadataLocation = metadataFileLocationMethod.invoke(metadata).asInstanceOf[String] + + // Return catalog info with actual metadata file location + return Some( + IcebergCatalogInfo( + catalogType = "hadoop", + properties = Map("metadata_location" -> metadataLocation), + namespace = namespaceParts, + tableName = tableNamePart)) + } catch { + case e: Exception => + // If reflection fails, fall back to returning None + return None + } + } + + // Get the catalog implementation class + val implClass = catalogImpl.getOrElse(return None) + + // Map Spark's catalog implementation to iceberg-rust catalog type + val icebergCatalogType = implClass match { + case impl if impl.contains("RESTCatalog") || impl.contains("rest") => "rest" + case impl if impl.contains("GlueCatalog") || impl.contains("glue") => "glue" + case impl if impl.contains("HiveCatalog") || impl.contains("hive") => "hms" + case impl if impl.contains("JdbcCatalog") || impl.contains("jdbc") => "sql" + case _ => return None // Unsupported catalog type + } + + // Extract all catalog properties with the prefix + val catalogProps = session.conf.getAll + .filter { case (k, _) => k.startsWith(catalogPrefix + ".") } + .map { case (k, v) => + // Remove prefix: "spark.sql.catalog.mycatalog.uri" -> "uri" + val key = k.stripPrefix(catalogPrefix + ".") + // Skip the catalog-impl property itself + if (key == "catalog-impl" || key == "type") None else Some((key, v)) + } + .flatten + .toMap + + Some( + IcebergCatalogInfo( + catalogType = icebergCatalogType, + properties = catalogProps, + namespace = namespaceParts, + tableName = tableNamePart)) + } catch { + case _: Exception => + None + } + } + + /** + * Creates a CometIcebergNativeScanExec from a Spark BatchScanExec. + * + * This method is called on the driver to create the Comet operator. The key step is determining + * the number of partitions to use, which affects parallelism. + * + * **Partition Count Strategy:** + * - For KeyGroupedPartitioning: Use Iceberg's partition count (data is grouped by partition + * keys) + * - For other cases: Use the number of partitions in inputRDD, which Spark computes based on + * the number of InputPartition objects returned by Iceberg's planInputPartitions() + * + * **How FileScanTasks Flow to Workers:** + * 1. Iceberg's planInputPartitions() creates InputPartition[] (each contains ScanTaskGroup) + * 2. Spark creates BatchScanExec.inputRDD with these InputPartitions 3. Each RDD partition + * wraps one InputPartition 4. Spark ships RDD partitions to workers 5. On worker: + * QueryPlanSerde extracts FileScanTasks from the InputPartition 6. FileScanTasks are + * serialized to protobuf and sent to Rust 7. Rust reads only the files specified in those + * tasks + * + * @param nativeOp + * The serialized native operator + * @param scanExec + * The original Spark BatchScanExec + * @param session + * The SparkSession + * @param catalogInfo + * The extracted catalog information + * @return + * A new CometIcebergNativeScanExec + */ + def apply( + nativeOp: Operator, + scanExec: BatchScanExec, + session: SparkSession, + catalogInfo: IcebergCatalogInfo): CometIcebergNativeScanExec = { + + // Determine number of partitions from Iceberg's output partitioning + // KeyGroupedPartitioning means Iceberg grouped data by partition keys + // Otherwise, use the number of InputPartitions that Iceberg created + val numParts = scanExec.outputPartitioning match { + case p: KeyGroupedPartitioning => + // Use Iceberg's key-grouped partition count + p.numPartitions + case _ => + // For unpartitioned tables or other partitioning schemes, + // use the InputPartition count from inputRDD + // This is already computed by Spark based on Iceberg's file planning + scanExec.inputRDD.getNumPartitions + } + + val exec = CometIcebergNativeScanExec( + nativeOp, + scanExec.output, + scanExec, + SerializedPlan(None), // Will be serialized per-partition during execution + catalogInfo.catalogType, + catalogInfo.properties, + catalogInfo.namespace, + catalogInfo.tableName, + 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 a7cfacc475..511b952156 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 @@ -334,10 +334,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/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala new file mode 100644 index 0000000000..35215288eb --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -0,0 +1,1058 @@ +/* + * 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 with catalog support. + * + * Tests the end-to-end flow: + * 1. Create Iceberg table with Hadoop catalog 2. Enable native Iceberg execution 3. Query table + * via Comet 4. Verify data correctness + * + * 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_SHUFFLE_MODE.key -> "native", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + // Create Iceberg table + spark.sql(""" + CREATE TABLE hadoop_catalog.db.test_table ( + id INT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + // Insert test data + spark.sql(""" + INSERT INTO hadoop_catalog.db.test_table + VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) + """) + + // Query with Comet native Iceberg scan and verify results + operator + checkIcebergNativeScan("SELECT * FROM hadoop_catalog.db.test_table ORDER BY id") + + // Cleanup + spark.sql("DROP TABLE hadoop_catalog.db.test_table") + } + } + } + + test("verify catalog info extraction") { + assume(icebergAvailable, "Iceberg not available in classpath") + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.my_catalog" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.my_catalog.type" -> "hadoop", + "spark.sql.catalog.my_catalog.warehouse" -> warehouseDir.getAbsolutePath, + "spark.sql.catalog.my_catalog.custom.prop" -> "test_value", + "spark.comet.enabled" -> "true", + "spark.comet.exec.enabled" -> "true", + "spark.comet.scan.icebergNative.enabled" -> "true") { + + // Create simple table + spark.sql(""" + CREATE TABLE my_catalog.namespace.table1 ( + id INT, + data STRING + ) USING iceberg + """) + + spark.sql("INSERT INTO my_catalog.namespace.table1 VALUES (1, 'test')") + + // Execute query and verify results + operator + checkIcebergNativeScan("SELECT * FROM my_catalog.namespace.table1") + + // Cleanup + spark.sql("DROP TABLE my_catalog.namespace.table1") + } + } + } + + test("verify native scan is actually used in plan") { + 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, + "spark.comet.enabled" -> "true", + "spark.comet.exec.enabled" -> "true", + "spark.comet.scan.icebergNative.enabled" -> "true") { + + // Create table + spark.sql(""" + CREATE TABLE test_cat.db.verify_table ( + x INT, + y DOUBLE + ) USING iceberg + """) + + spark.sql("INSERT INTO test_cat.db.verify_table VALUES (42, 3.14)") + + // Query and verify results + operator + checkIcebergNativeScan("SELECT x, y FROM test_cat.db.verify_table WHERE x = 42") + + // Cleanup + spark.sql("DROP TABLE test_cat.db.verify_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") { + + // Create table with multiple rows + spark.sql(""" + CREATE TABLE filter_cat.db.filter_test ( + id INT, + name STRING, + value DOUBLE, + active BOOLEAN + ) USING iceberg + """) + + // Insert test data + 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) + """) + + // Test: WHERE id = 3 + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE id = 3") + + // Test: WHERE name = 'Bob' + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE name = 'Bob'") + + // Test: WHERE active = true + 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) + """) + + // Test: WHERE value > 20.0 + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value > 20.0") + + // Test: WHERE value >= 20.3 + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value >= 20.3") + + // Test: WHERE value < 20.0 + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value < 20.0") + + // Test: WHERE value <= 20.3 + checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value <= 20.3") + + // Test: WHERE id != 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) + """) + + // Test: WHERE category = 'A' AND value > 20.0 + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.logical_test WHERE category = 'A' AND value > 20.0") + + // Test: WHERE category = 'B' OR value > 30.0 + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.logical_test WHERE category = 'B' OR value > 30.0") + + // Test: WHERE (category = 'A' AND value > 20.0) OR category = 'C' + 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 + """) + + // Insert data with some NULLs + spark.sql(""" + INSERT INTO filter_cat.db.null_test VALUES + (1, 10.5), (2, NULL), (3, 30.7), (4, NULL), (5, 25.8) + """) + + // Test: WHERE optional_value IS NULL + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NULL") + + // Test: WHERE optional_value IS NOT 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') + """) + + // Test: WHERE id IN (2, 4, 6) + checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IN (2, 4, 6)") + + // Test: WHERE name IN ('Alice', 'Charlie', 'Eve') + checkIcebergNativeScan( + "SELECT * FROM filter_cat.db.in_test WHERE name IN ('Alice', 'Charlie', 'Eve')") + + // Test: WHERE id IS NOT NULL (should return all 6 rows) + checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IS NOT NULL") + + // Test: WHERE id NOT IN (1, 3, 5) + 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) + """) + + // Query with a filter and verify results match Spark + checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") + + // Also verify the plan contains native Iceberg scan + val df = spark.sql("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") + val plan = df.queryExecution.executedPlan.toString() + assert(plan.contains("CometIcebergNativeScan"), "Should use native Iceberg scan") + + 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') + """) + + // Verify results match Spark native (catches duplicates and correctness issues) + 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", + // Force smaller file size to create multiple files + "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) + """) + + // Critical tests - if partitioning is broken, COUNT(*) will be N times too large + 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) + """) + + // Verify all queries match Spark native + 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 + """) + + // Empty table should work correctly with all queries + 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' + ) + """) + + // Insert initial data + 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) + """) + + // Delete specific rows (creates positional delete files) + spark.sql("DELETE FROM test_cat.db.positional_delete_test WHERE id IN (2, 4, 6)") + + // Query and verify results match Spark (both apply MOR deletes at read time) + 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' + ) + """) + + // Insert initial data + 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) + """) + + // Delete rows using equality delete (matches on id column) + spark.sql("DELETE FROM test_cat.db.equality_delete_test WHERE id IN (2, 4)") + + // Query and verify results match Spark (both apply MOR deletes at read time) + 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' + ) + """) + + // Insert data in batches + spark.sql(""" + INSERT INTO test_cat.db.multi_delete_test + SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + FROM range(100) + """) + + // Perform multiple deletes + 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") + + // Query and verify results match Spark (both apply MOR deletes at read time) + 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) + """) + + // Test various filters with multi-partition scan + 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') + """) + + // Test date range queries with partition pruning + 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) + """) + + // Test bucket partitioning correctness + 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") + } + } + } + + // TODO: Re-enable when iceberg-rust supports schema evolution in projections + // Currently iceberg-rust errors when projecting columns that don't exist in old files. + // See: https://github.com/apache/iceberg-rust/blob/main/crates/iceberg/src/arrow/reader.rs#L586-L601 + // The strict validation at line 586: `if column_map.len() != leaf_field_ids.len()` + // prevents reading new columns from evolved schemas as NULL values. + ignore("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 + """) + + // Insert data with original schema + spark.sql(""" + INSERT INTO test_cat.db.schema_evolution VALUES (1, 'Alice'), (2, 'Bob') + """) + + // Add a new column + spark.sql("ALTER TABLE test_cat.db.schema_evolution ADD COLUMN age INT") + + // Insert data with new schema + spark.sql(""" + INSERT INTO test_cat.db.schema_evolution VALUES (3, 'Charlie', 30), (4, 'Diana', 25) + """) + + // Query should handle both old and new schemas + 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("very large file count - scalability") { + 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 many small files + "spark.sql.files.maxRecordsPerFile" -> "10") { + + spark.sql(""" + CREATE TABLE test_cat.db.scalability_test ( + id INT, + data STRING + ) USING iceberg + """) + + // Insert 1000 rows with maxRecordsPerFile=10 creates ~100 files + spark.sql(""" + INSERT INTO test_cat.db.scalability_test + SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + FROM range(1000) + """) + + // Test scalability with many files + checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.scalability_test") + checkIcebergNativeScan("SELECT SUM(id) FROM test_cat.db.scalability_test") + + spark.sql("DROP TABLE test_cat.db.scalability_test") + } + } + } + + 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") + } + } + } + + // Helper to create temp directory + def withTempIcebergDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("comet-iceberg-test").toFile + try { + f(dir) + } finally { + // Cleanup + 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/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index d4b7b029a3..a3f6af9454 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -201,7 +201,8 @@ abstract class CometTestBase protected def checkCometOperators(plan: SparkPlan, excludedClasses: Class[_]*): Unit = { val wrapped = wrapCometSparkToColumnar(plan) wrapped.foreach { - case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec => + case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec | + _: CometIcebergNativeScanExec => case _: CometSinkPlaceHolder | _: CometScanWrapper => case _: CometColumnarToRowExec => case _: CometSparkToColumnarExec => From 4f3004bea30c42221671f4b9e80ecf55610d4775 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 5 Oct 2025 22:26:53 -0400 Subject: [PATCH 02/89] Clean up tests a little. --- .../comet/CometIcebergNativeSuite.scala | 161 +----------------- 1 file changed, 3 insertions(+), 158 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 35215288eb..c9026fae6f 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -27,11 +27,7 @@ import org.apache.spark.sql.comet.CometIcebergNativeScanExec import org.apache.spark.sql.execution.SparkPlan /** - * Test suite for native Iceberg scan with catalog support. - * - * Tests the end-to-end flow: - * 1. Create Iceberg table with Hadoop catalog 2. Enable native Iceberg execution 3. Query table - * via Comet 4. Verify data correctness + * Test suite for native Iceberg scan using FileScanTasks and iceberg-rust. * * Note: Requires Iceberg dependencies to be added to pom.xml */ @@ -59,6 +55,7 @@ class CometIcebergNativeSuite extends CometTestBase { * This ensures both correct results and that the native Iceberg scan operator is being used. */ private def checkIcebergNativeScan(query: String): Unit = { + println(sql(query).explain(true)) val (_, cometPlan) = checkSparkAnswer(query) val icebergScans = collectIcebergNativeScans(cometPlan) assert( @@ -76,10 +73,8 @@ class CometIcebergNativeSuite extends CometTestBase { "spark.sql.catalog.hadoop_catalog.warehouse" -> warehouseDir.getAbsolutePath, CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_MODE.key -> "native", - CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - // Create Iceberg table + spark.sql(""" CREATE TABLE hadoop_catalog.db.test_table ( id INT, @@ -88,84 +83,18 @@ class CometIcebergNativeSuite extends CometTestBase { ) USING iceberg """) - // Insert test data spark.sql(""" INSERT INTO hadoop_catalog.db.test_table VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) """) - // Query with Comet native Iceberg scan and verify results + operator checkIcebergNativeScan("SELECT * FROM hadoop_catalog.db.test_table ORDER BY id") - // Cleanup spark.sql("DROP TABLE hadoop_catalog.db.test_table") } } } - test("verify catalog info extraction") { - assume(icebergAvailable, "Iceberg not available in classpath") - - withTempIcebergDir { warehouseDir => - withSQLConf( - "spark.sql.catalog.my_catalog" -> "org.apache.iceberg.spark.SparkCatalog", - "spark.sql.catalog.my_catalog.type" -> "hadoop", - "spark.sql.catalog.my_catalog.warehouse" -> warehouseDir.getAbsolutePath, - "spark.sql.catalog.my_catalog.custom.prop" -> "test_value", - "spark.comet.enabled" -> "true", - "spark.comet.exec.enabled" -> "true", - "spark.comet.scan.icebergNative.enabled" -> "true") { - - // Create simple table - spark.sql(""" - CREATE TABLE my_catalog.namespace.table1 ( - id INT, - data STRING - ) USING iceberg - """) - - spark.sql("INSERT INTO my_catalog.namespace.table1 VALUES (1, 'test')") - - // Execute query and verify results + operator - checkIcebergNativeScan("SELECT * FROM my_catalog.namespace.table1") - - // Cleanup - spark.sql("DROP TABLE my_catalog.namespace.table1") - } - } - } - - test("verify native scan is actually used in plan") { - 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, - "spark.comet.enabled" -> "true", - "spark.comet.exec.enabled" -> "true", - "spark.comet.scan.icebergNative.enabled" -> "true") { - - // Create table - spark.sql(""" - CREATE TABLE test_cat.db.verify_table ( - x INT, - y DOUBLE - ) USING iceberg - """) - - spark.sql("INSERT INTO test_cat.db.verify_table VALUES (42, 3.14)") - - // Query and verify results + operator - checkIcebergNativeScan("SELECT x, y FROM test_cat.db.verify_table WHERE x = 42") - - // Cleanup - spark.sql("DROP TABLE test_cat.db.verify_table") - } - } - } - test("filter pushdown - equality predicates") { assume(icebergAvailable, "Iceberg not available in classpath") @@ -178,7 +107,6 @@ class CometIcebergNativeSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - // Create table with multiple rows spark.sql(""" CREATE TABLE filter_cat.db.filter_test ( id INT, @@ -188,7 +116,6 @@ class CometIcebergNativeSuite extends CometTestBase { ) USING iceberg """) - // Insert test data spark.sql(""" INSERT INTO filter_cat.db.filter_test VALUES (1, 'Alice', 10.5, true), @@ -198,13 +125,10 @@ class CometIcebergNativeSuite extends CometTestBase { (5, 'Eve', 25.8, true) """) - // Test: WHERE id = 3 checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE id = 3") - // Test: WHERE name = 'Bob' checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE name = 'Bob'") - // Test: WHERE active = true checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE active = true") spark.sql("DROP TABLE filter_cat.db.filter_test") @@ -236,19 +160,14 @@ class CometIcebergNativeSuite extends CometTestBase { (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) """) - // Test: WHERE value > 20.0 checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value > 20.0") - // Test: WHERE value >= 20.3 checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value >= 20.3") - // Test: WHERE value < 20.0 checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value < 20.0") - // Test: WHERE value <= 20.3 checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value <= 20.3") - // Test: WHERE id != 3 checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE id != 3") spark.sql("DROP TABLE filter_cat.db.comparison_test") @@ -282,15 +201,12 @@ class CometIcebergNativeSuite extends CometTestBase { (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) """) - // Test: WHERE category = 'A' AND value > 20.0 checkIcebergNativeScan( "SELECT * FROM filter_cat.db.logical_test WHERE category = 'A' AND value > 20.0") - // Test: WHERE category = 'B' OR value > 30.0 checkIcebergNativeScan( "SELECT * FROM filter_cat.db.logical_test WHERE category = 'B' OR value > 30.0") - // Test: WHERE (category = 'A' AND value > 20.0) OR category = 'C' checkIcebergNativeScan("""SELECT * FROM filter_cat.db.logical_test WHERE (category = 'A' AND value > 20.0) OR category = 'C'""") @@ -318,17 +234,14 @@ class CometIcebergNativeSuite extends CometTestBase { ) USING iceberg """) - // Insert data with some NULLs spark.sql(""" INSERT INTO filter_cat.db.null_test VALUES (1, 10.5), (2, NULL), (3, 30.7), (4, NULL), (5, 25.8) """) - // Test: WHERE optional_value IS NULL checkIcebergNativeScan( "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NULL") - // Test: WHERE optional_value IS NOT NULL checkIcebergNativeScan( "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NOT NULL") @@ -362,17 +275,13 @@ class CometIcebergNativeSuite extends CometTestBase { (4, 'Diana'), (5, 'Eve'), (6, 'Frank') """) - // Test: WHERE id IN (2, 4, 6) checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IN (2, 4, 6)") - // Test: WHERE name IN ('Alice', 'Charlie', 'Eve') checkIcebergNativeScan( "SELECT * FROM filter_cat.db.in_test WHERE name IN ('Alice', 'Charlie', 'Eve')") - // Test: WHERE id IS NOT NULL (should return all 6 rows) checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IS NOT NULL") - // Test: WHERE id NOT IN (1, 3, 5) checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id NOT IN (1, 3, 5)") spark.sql("DROP TABLE filter_cat.db.in_test") @@ -404,14 +313,8 @@ class CometIcebergNativeSuite extends CometTestBase { (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) """) - // Query with a filter and verify results match Spark checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") - // Also verify the plan contains native Iceberg scan - val df = spark.sql("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") - val plan = df.queryExecution.executedPlan.toString() - assert(plan.contains("CometIcebergNativeScan"), "Should use native Iceberg scan") - spark.sql("DROP TABLE filter_cat.db.filter_debug") } } @@ -441,7 +344,6 @@ class CometIcebergNativeSuite extends CometTestBase { VALUES (1, 'Alice'), (2, 'Bob'), (3, 'Charlie') """) - // Verify results match Spark native (catches duplicates and correctness issues) checkIcebergNativeScan("SELECT * FROM test_cat.db.small_table ORDER BY id") checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.small_table") @@ -461,7 +363,6 @@ class CometIcebergNativeSuite extends CometTestBase { CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // Force smaller file size to create multiple files "spark.sql.files.maxRecordsPerFile" -> "10") { spark.sql(""" @@ -519,7 +420,6 @@ class CometIcebergNativeSuite extends CometTestBase { FROM range(10000) """) - // Critical tests - if partitioning is broken, COUNT(*) will be N times too large checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.large_table") checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.large_table") checkIcebergNativeScan( @@ -558,7 +458,6 @@ class CometIcebergNativeSuite extends CometTestBase { (7, 'A', 12.1), (8, 'B', 22.5), (9, 'C', 32.9) """) - // Verify all queries match Spark native 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") @@ -591,7 +490,6 @@ class CometIcebergNativeSuite extends CometTestBase { ) USING iceberg """) - // Empty table should work correctly with all queries checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table") checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table WHERE id > 0") @@ -627,7 +525,6 @@ class CometIcebergNativeSuite extends CometTestBase { ) """) - // Insert initial data spark.sql(""" INSERT INTO test_cat.db.positional_delete_test VALUES @@ -636,10 +533,8 @@ class CometIcebergNativeSuite extends CometTestBase { (7, 'Grace', 12.1), (8, 'Hank', 22.5) """) - // Delete specific rows (creates positional delete files) spark.sql("DELETE FROM test_cat.db.positional_delete_test WHERE id IN (2, 4, 6)") - // Query and verify results match Spark (both apply MOR deletes at read time) checkIcebergNativeScan("SELECT * FROM test_cat.db.positional_delete_test ORDER BY id") spark.sql("DROP TABLE test_cat.db.positional_delete_test") @@ -674,7 +569,6 @@ class CometIcebergNativeSuite extends CometTestBase { ) """) - // Insert initial data spark.sql(""" INSERT INTO test_cat.db.equality_delete_test VALUES @@ -682,10 +576,8 @@ class CometIcebergNativeSuite extends CometTestBase { (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) """) - // Delete rows using equality delete (matches on id column) spark.sql("DELETE FROM test_cat.db.equality_delete_test WHERE id IN (2, 4)") - // Query and verify results match Spark (both apply MOR deletes at read time) checkIcebergNativeScan("SELECT * FROM test_cat.db.equality_delete_test ORDER BY id") spark.sql("DROP TABLE test_cat.db.equality_delete_test") @@ -716,19 +608,16 @@ class CometIcebergNativeSuite extends CometTestBase { ) """) - // Insert data in batches spark.sql(""" INSERT INTO test_cat.db.multi_delete_test SELECT id, CONCAT('data_', CAST(id AS STRING)) as data FROM range(100) """) - // Perform multiple deletes 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") - // Query and verify results match Spark (both apply MOR deletes at read time) checkIcebergNativeScan("SELECT * FROM test_cat.db.multi_delete_test ORDER BY id") spark.sql("DROP TABLE test_cat.db.multi_delete_test") @@ -805,7 +694,6 @@ class CometIcebergNativeSuite extends CometTestBase { FROM range(200) """) - // Test various filters with multi-partition scan checkIcebergNativeScan( "SELECT * FROM test_cat.db.filter_multipart WHERE id > 150 ORDER BY id") checkIcebergNativeScan( @@ -848,7 +736,6 @@ class CometIcebergNativeSuite extends CometTestBase { (5, DATE '2024-01-16', 'e'), (6, DATE '2024-02-01', 'f') """) - // Test date range queries with partition pruning 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'") @@ -924,20 +811,16 @@ class CometIcebergNativeSuite extends CometTestBase { ) USING iceberg """) - // Insert data with original schema spark.sql(""" INSERT INTO test_cat.db.schema_evolution VALUES (1, 'Alice'), (2, 'Bob') """) - // Add a new column spark.sql("ALTER TABLE test_cat.db.schema_evolution ADD COLUMN age INT") - // Insert data with new schema spark.sql(""" INSERT INTO test_cat.db.schema_evolution VALUES (3, 'Charlie', 30), (4, 'Diana', 25) """) - // Query should handle both old and new schemas 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( @@ -948,43 +831,6 @@ class CometIcebergNativeSuite extends CometTestBase { } } - test("very large file count - scalability") { - 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 many small files - "spark.sql.files.maxRecordsPerFile" -> "10") { - - spark.sql(""" - CREATE TABLE test_cat.db.scalability_test ( - id INT, - data STRING - ) USING iceberg - """) - - // Insert 1000 rows with maxRecordsPerFile=10 creates ~100 files - spark.sql(""" - INSERT INTO test_cat.db.scalability_test - SELECT id, CONCAT('data_', CAST(id AS STRING)) as data - FROM range(1000) - """) - - // Test scalability with many files - checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.scalability_test") - checkIcebergNativeScan("SELECT SUM(id) FROM test_cat.db.scalability_test") - - spark.sql("DROP TABLE test_cat.db.scalability_test") - } - } - } - test("projection - column subset, reordering, and duplication") { assume(icebergAvailable, "Iceberg not available in classpath") @@ -1045,7 +891,6 @@ class CometIcebergNativeSuite extends CometTestBase { try { f(dir) } finally { - // Cleanup def deleteRecursively(file: File): Unit = { if (file.isDirectory) { file.listFiles().foreach(deleteRecursively) From 4afec43b0db958fcda15fc28e2fa91918fb1451d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 5 Oct 2025 22:40:07 -0400 Subject: [PATCH 03/89] Remove old comment. --- native/core/src/execution/operators/iceberg_scan.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 8a89e4bd36..dc292af573 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -151,9 +151,6 @@ impl ExecutionPlan for IcebergScanExec { } impl IcebergScanExec { - /// Execute with pre-planned tasks (fast path) - /// This avoids re-planning the scan and directly reads the assigned files. - /// /// **MOR (Merge-On-Read) Table Support:** /// /// If the FileScanTasks include delete files (for MOR tables), iceberg-rust's ArrowReader From fc97ce9ab315df83e0abc4e5ad0fd2092c933c84 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 06:58:05 -0400 Subject: [PATCH 04/89] Fix machete and missing suite CI failures. --- .github/workflows/pr_build_linux.yml | 1 + .github/workflows/pr_build_macos.yml | 1 + native/Cargo.lock | 146 --------------------------- native/core/Cargo.toml | 3 +- 4 files changed, 3 insertions(+), 148 deletions(-) diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 9b918ad8b2..319af00ed1 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -123,6 +123,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 fb6a8295bc..cec4d69f14 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -88,6 +88,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/native/Cargo.lock b/native/Cargo.lock index 83042cf363..0ae007e5a2 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -281,7 +281,6 @@ dependencies = [ "arrow-select", "flatbuffers", "lz4_flex", - "zstd", ] [[package]] @@ -409,23 +408,6 @@ dependencies = [ "pin-project-lite", ] -[[package]] -name = "async-compression" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" -dependencies = [ - "bzip2 0.5.2", - "flate2", - "futures-core", - "memchr", - "pin-project-lite", - "tokio", - "xz2", - "zstd", - "zstd-safe", -] - [[package]] name = "async-executor" version = "1.13.3" @@ -1193,34 +1175,6 @@ dependencies = [ "either", ] -[[package]] -name = "bzip2" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49ecfb22d906f800d4fe833b6282cf4dc1c298f5057ca0b5445e5c209735ca47" -dependencies = [ - "bzip2-sys", -] - -[[package]] -name = "bzip2" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bea8dcd42434048e4f7a304411d9273a411f647446c1234a65ce0554923f4cff" -dependencies = [ - "libbz2-rs-sys", -] - -[[package]] -name = "bzip2-sys" -version = "0.1.13+1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225bff33b2141874fe80d71e07d6eec4f85c5c216453dd96388240f96e1acc14" -dependencies = [ - "cc", - "pkg-config", -] - [[package]] name = "cast" version = "0.3.0" @@ -1677,7 +1631,6 @@ dependencies = [ "arrow-schema", "async-trait", "bytes", - "bzip2 0.6.0", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1703,7 +1656,6 @@ dependencies = [ "datafusion-physical-plan", "datafusion-session", "datafusion-sql", - "flate2", "futures", "itertools 0.14.0", "log", @@ -1717,8 +1669,6 @@ dependencies = [ "tokio", "url", "uuid", - "xz2", - "zstd", ] [[package]] @@ -1793,7 +1743,6 @@ dependencies = [ "hdrs", "hex", "iceberg", - "iceberg-datafusion", "itertools 0.14.0", "jni", "lazy_static", @@ -1903,7 +1852,6 @@ dependencies = [ "object_store", "parquet", "paste", - "recursive", "sqlparser", "tokio", "web-time", @@ -1927,10 +1875,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d855160469020982880fd9bd0962e033d2f4728f56f85a83d8c90785638b6519" dependencies = [ "arrow", - "async-compression", "async-trait", "bytes", - "bzip2 0.6.0", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -1941,7 +1887,6 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", - "flate2", "futures", "glob", "itertools 0.14.0", @@ -1951,10 +1896,7 @@ dependencies = [ "rand 0.9.2", "tempfile", "tokio", - "tokio-util", "url", - "xz2", - "zstd", ] [[package]] @@ -2083,7 +2025,6 @@ dependencies = [ "datafusion-physical-expr-common", "indexmap 2.11.4", "paste", - "recursive", "serde_json", "sqlparser", ] @@ -2256,7 +2197,6 @@ dependencies = [ "indexmap 2.11.4", "itertools 0.14.0", "log", - "recursive", "regex", "regex-syntax", ] @@ -2330,7 +2270,6 @@ dependencies = [ "datafusion-pruning", "itertools 0.14.0", "log", - "recursive", ] [[package]] @@ -2439,7 +2378,6 @@ dependencies = [ "datafusion-expr", "indexmap 2.11.4", "log", - "recursive", "regex", "sqlparser", ] @@ -3262,21 +3200,6 @@ dependencies = [ "zstd", ] -[[package]] -name = "iceberg-datafusion" -version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#c85788fca732a846bf38093f1c5f3abded7c6900" -dependencies = [ - "anyhow", - "async-trait", - "datafusion", - "futures", - "iceberg", - "parquet", - "tokio", - "uuid", -] - [[package]] name = "icu_collections" version = "2.0.0" @@ -3630,12 +3553,6 @@ dependencies = [ "lexical-util", ] -[[package]] -name = "libbz2-rs-sys" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" - [[package]] name = "libc" version = "0.2.176" @@ -3775,17 +3692,6 @@ dependencies = [ "twox-hash", ] -[[package]] -name = "lzma-sys" -version = "0.1.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" -dependencies = [ - "cc", - "libc", - "pkg-config", -] - [[package]] name = "md-5" version = "0.10.6" @@ -4566,15 +4472,6 @@ dependencies = [ "prost", ] -[[package]] -name = "psm" -version = "0.1.26" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" -dependencies = [ - "cc", -] - [[package]] name = "ptr_meta" version = "0.1.4" @@ -4785,26 +4682,6 @@ dependencies = [ "crossbeam-utils", ] -[[package]] -name = "recursive" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0786a43debb760f491b1bc0269fe5e84155353c67482b9e60d0cfb596054b43e" -dependencies = [ - "recursive-proc-macro-impl", - "stacker", -] - -[[package]] -name = "recursive-proc-macro-impl" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" -dependencies = [ - "quote", - "syn 2.0.106", -] - [[package]] name = "redox_syscall" version = "0.5.17" @@ -5480,7 +5357,6 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec4b661c54b1e4b603b37873a18c59920e4c51ea8ea2cf527d925424dbd4437c" dependencies = [ "log", - "recursive", "sqlparser_derive", ] @@ -5501,19 +5377,6 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" -[[package]] -name = "stacker" -version = "0.1.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" -dependencies = [ - "cc", - "cfg-if", - "libc", - "psm", - "windows-sys 0.59.0", -] - [[package]] name = "str_stack" version = "0.1.0" @@ -6687,15 +6550,6 @@ version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" -[[package]] -name = "xz2" -version = "0.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" -dependencies = [ - "lzma-sys", -] - [[package]] name = "yoke" version = "0.8.0" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 3726bf725e..2c263ad86e 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -78,7 +78,6 @@ object_store_opendal = {version = "0.54.0", optional = true} hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} opendal = { version ="0.54.0", optional = true, features = ["services-hdfs"] } iceberg = { workspace = true } -iceberg-datafusion = { workspace = true } serde_json = "1.0" [target.'cfg(target_os = "linux")'.dependencies] @@ -105,7 +104,7 @@ jemalloc = ["tikv-jemallocator", "tikv-jemalloc-ctl"] # exclude optional packages from cargo machete verifications [package.metadata.cargo-machete] -ignored = ["datafusion-comet-objectstore-hdfs", "hdfs-sys"] +ignored = ["hdfs-sys"] [lib] name = "comet" From cca4911e884e93906e454bfd9e5c5ab9c6196b1e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 08:28:59 -0400 Subject: [PATCH 05/89] Fix unused variables. --- .../src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 2 +- .../org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 e8382da581..3cb3c511b2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2317,7 +2317,7 @@ object QueryPlanSerde extends Logging with CometExprShim { None } } catch { - case e: Exception => + case _: Exception => None } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 4554af4fc6..a3440e175f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -291,7 +291,7 @@ object CometIcebergNativeScanExec { namespace = namespaceParts, tableName = tableNamePart)) } catch { - case e: Exception => + case _: Exception => // If reflection fails, fall back to returning None return None } From 93f466de866ef004d347edbcebaa7a40af50d77d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 09:26:26 -0400 Subject: [PATCH 06/89] Spark 4.0 needs Iceberg 1.10, let's see if that works in CI. --- spark/pom.xml | 49 +++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 43 insertions(+), 6 deletions(-) diff --git a/spark/pom.xml b/spark/pom.xml index dd1323a2b9..6fd853b6ac 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -174,14 +174,51 @@ under the License. s3 - - org.apache.iceberg - iceberg-spark-runtime-3.5_2.12 - 1.8.1 - test - + + + + + 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 + + + + + From 970b6921c7af99526910f299f6da8dcb1dc6b996 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 09:29:49 -0400 Subject: [PATCH 07/89] Remove errant println. --- .../test/scala/org/apache/comet/CometIcebergNativeSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index c9026fae6f..3ca01dc658 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -55,7 +55,6 @@ class CometIcebergNativeSuite extends CometTestBase { * This ensures both correct results and that the native Iceberg scan operator is being used. */ private def checkIcebergNativeScan(query: String): Unit = { - println(sql(query).explain(true)) val (_, cometPlan) = checkSparkAnswer(query) val icebergScans = collectIcebergNativeScans(cometPlan) assert( From c44973b4be9bdd05669f14524df3002d67404533 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 09:39:25 -0400 Subject: [PATCH 08/89] Remove old path() code path. --- .../org/apache/comet/serde/QueryPlanSerde.scala | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) 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 3cb3c511b2..f86bc3eebe 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1272,19 +1272,10 @@ object QueryPlanSerde extends Logging with CometExprShim { fileMethod.setAccessible(true) val dataFile = fileMethod.invoke(task) - // Get path from DataFile - use location() (or path()) + // Get path from DataFile using location() (added in Iceberg 1.7.0) val dataFileClass = dataFile.getClass - val filePath = - try { - // Try location() first (newer API) - val locationMethod = dataFileClass.getMethod("location") - locationMethod.invoke(dataFile).asInstanceOf[String] - } catch { - case _: NoSuchMethodException => - // Fall back to path() (older API, returns CharSequence) - val pathMethod = dataFileClass.getMethod("path") - pathMethod.invoke(dataFile).asInstanceOf[CharSequence].toString - } + val locationMethod = dataFileClass.getMethod("location") + val filePath = locationMethod.invoke(dataFile).asInstanceOf[String] taskBuilder.setDataFilePath(filePath) // Get start offset From 0f83fd42dc94cf6d0a9c4f7e17a1888f05bfae0c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 09:49:20 -0400 Subject: [PATCH 09/89] Update old comment. --- native/core/src/execution/operators/iceberg_scan.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index dc292af573..f3537281a0 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -39,8 +39,8 @@ use crate::execution::operators::ExecutionError; /// Native Iceberg scan operator that uses iceberg-rust to read Iceberg tables. /// -/// This operator completely bypasses Spark's DataSource V2 API and uses iceberg-rust -/// for all table metadata loading, file discovery, and data reading. +/// This operator completely bypasses Spark's DataSource V2 API and uses iceberg-rust by using +/// pre-planned FileScanTasks to read data. #[derive(Debug)] pub struct IcebergScanExec { /// Path to Iceberg table metadata file or directory (used for FileIO creation) From 6cbbd0905371437a999910614d8645e7226867c9 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 11:49:36 -0400 Subject: [PATCH 10/89] Iceberg 1.5.x compatible reflection. Use 1.5.2 for Spark 3.4 and 3.5. --- spark/pom.xml | 6 +- .../apache/comet/serde/QueryPlanSerde.scala | 433 ++++++++---------- 2 files changed, 203 insertions(+), 236 deletions(-) diff --git a/spark/pom.xml b/spark/pom.xml index 6fd853b6ac..d74a71c377 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -178,14 +178,14 @@ under the License. - + spark-3.4 org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} - 1.8.1 + 1.5.2 test @@ -200,7 +200,7 @@ under the License. org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} - 1.8.1 + 1.5.2 test 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 f86bc3eebe..3960edcdc2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1226,6 +1226,12 @@ object QueryPlanSerde extends Logging with CometExprShim { // Extract FileScanTasks from the InputPartitions in the RDD // Group tasks by Spark partition (similar to how NativeScan groups PartitionedFiles) + // + // Reflection Strategy (for Iceberg 1.5.x - 1.10.x compatibility): + // - SparkInputPartition: package-private Spark class, use getDeclaredMethod + setAccessible + // - Iceberg API methods: use Class.forName() on public interfaces, then getMethod() + // (avoids IllegalAccessException on package-private implementation classes like + // BaseFileScanTask$SplitScanTask in Iceberg 1.5.x) var actualNumPartitions = 0 try { scan.originalPlan.inputRDD match { @@ -1247,7 +1253,8 @@ object QueryPlanSerde extends Logging with CometExprShim { // Get the task group and extract tasks try { - // Call taskGroup() to get ScanTaskGroup + // SparkInputPartition is package-private, so we need getDeclaredMethod + setAccessible + // This is different from Iceberg API classes which have public interfaces val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") taskGroupMethod.setAccessible(true) val taskGroup = taskGroupMethod.invoke(inputPartition) @@ -1261,258 +1268,231 @@ object QueryPlanSerde extends Logging with CometExprShim { // Convert to Scala and serialize each task import scala.jdk.CollectionConverters._ tasksCollection.asScala.foreach { task => - // Serialize this FileScanTask to protobuf - val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() - - // Extract task properties using reflection - val taskClass = task.getClass - - // Get file() -> DataFile - val fileMethod = taskClass.getDeclaredMethod("file") - fileMethod.setAccessible(true) - val dataFile = fileMethod.invoke(task) - - // Get path from DataFile using location() (added in Iceberg 1.7.0) - val dataFileClass = dataFile.getClass - val locationMethod = dataFileClass.getMethod("location") - val filePath = locationMethod.invoke(dataFile).asInstanceOf[String] - taskBuilder.setDataFilePath(filePath) - - // Get start offset - val startMethod = taskClass.getDeclaredMethod("start") - startMethod.setAccessible(true) - val start = startMethod.invoke(task).asInstanceOf[Long] - taskBuilder.setStart(start) - - // Get length - val lengthMethod = taskClass.getDeclaredMethod("length") - lengthMethod.setAccessible(true) - val length = lengthMethod.invoke(task).asInstanceOf[Long] - taskBuilder.setLength(length) - - // Get schema and serialize to JSON try { - val schemaMethod = taskClass.getMethod("schema") - schemaMethod.setAccessible(true) - val schema = schemaMethod.invoke(task) + val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() - // Use Iceberg's SchemaParser.toJson(schema) + // Load interface classes to avoid IllegalAccessException on package-private implementations // scalastyle:off classforname - // Cannot use Utils.classForName as it's not accessible outside Spark - val schemaParserClass = Class.forName("org.apache.iceberg.SchemaParser") - val schemaClass = Class.forName("org.apache.iceberg.Schema") + 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 toJsonMethod = schemaParserClass.getMethod("toJson", schemaClass) - toJsonMethod.setAccessible(true) - val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] - taskBuilder.setSchemaJson(schemaJson) - - // Extract field IDs from the REQUIRED output schema, - // not the full task schema. - // This ensures we only project the columns actually needed by the query - val columnsMethod = schema.getClass.getMethod("columns") - columnsMethod.setAccessible(true) - val columns = - columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] - - // Build a map of column name -> field ID from the task schema - val nameToFieldId = scala.collection.mutable.Map[String, Int]() - columns.forEach { column => - try { - val nameMethod = column.getClass.getMethod("name") - nameMethod.setAccessible(true) - val name = nameMethod.invoke(column).asInstanceOf[String] - val fieldIdMethod = column.getClass.getMethod("fieldId") - fieldIdMethod.setAccessible(true) - val fieldId = fieldIdMethod.invoke(column).asInstanceOf[Int] + val fileMethod = contentScanTaskClass.getMethod("file") + val dataFile = fileMethod.invoke(task) - nameToFieldId(name) = fieldId + val filePath = + try { + // Try location() first (1.7.0+) + val locationMethod = contentFileClass.getMethod("location") + locationMethod.invoke(dataFile).asInstanceOf[String] } catch { - case _: Exception => // Skip if can't get field ID + case _: NoSuchMethodException => + // Fall back to path() (pre-1.7.0, returns CharSequence) + val pathMethod = contentFileClass.getMethod("path") + pathMethod.invoke(dataFile).asInstanceOf[CharSequence].toString } - } + taskBuilder.setDataFilePath(filePath) + + 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 { + val schemaMethod = fileScanTaskClass.getMethod("schema") + val schema = schemaMethod.invoke(task) + + // 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) + + // Extract field IDs from the REQUIRED output schema, not the full task schema. + // This ensures we only project the columns actually needed by the query + 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] - // Now add field IDs ONLY for columns in scan.output (the required schema) - scan.output.foreach { attr => - nameToFieldId.get(attr.name) match { - case Some(fieldId) => - taskBuilder.addProjectFieldIds(fieldId) - case None => + nameToFieldId(name) = fieldId + } catch { + case e: Exception => + logWarning( + s"Failed to extract field ID from column: ${e.getMessage}") + } } - } - } catch { - case e: Exception => - // Could not extract schema from task - logWarning( - s"Failed to extract schema from FileScanTask: ${e.getMessage}") - } - // Get file format - try { - val formatMethod = dataFileClass.getMethod("format") - formatMethod.setAccessible(true) - val format = formatMethod.invoke(dataFile) - taskBuilder.setDataFileFormat(format.toString) - } catch { - case e: Exception => - // Could not extract file format, defaulting to Parquet - logWarning( - "Failed to extract file format from FileScanTask," + - s"defaulting to PARQUET: ${e.getMessage}") - taskBuilder.setDataFileFormat("PARQUET") // Default to Parquet - } + scan.output.foreach { attr => + nameToFieldId.get(attr.name) match { + case Some(fieldId) => + taskBuilder.addProjectFieldIds(fieldId) + case None => + } + } + } catch { + case e: Exception => + logWarning( + s"Failed to extract schema from FileScanTask: ${e.getMessage}") + } - // Extract delete files from FileScanTask for MOR (Merge-On-Read) tables. - // When present, these are serialized to protobuf and passed to - // iceberg-rust, which automatically applies deletes during reading. - try { - val deletesMethod = taskClass.getDeclaredMethod("deletes") - deletesMethod.setAccessible(true) - val deletes = deletesMethod - .invoke(task) - .asInstanceOf[java.util.List[_]] - - // Serialize delete files if present - deletes.asScala.foreach { deleteFile => - try { - val deleteFileClass = deleteFile.getClass + 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") + } - // Get file path - try location() first, then path() - val deletePath = - try { - val locationMethod = deleteFileClass.getMethod("location") - locationMethod.setAccessible(true) - locationMethod.invoke(deleteFile).asInstanceOf[String] - } catch { - case _: NoSuchMethodException => - // Fall back to path() - val pathMethod = deleteFileClass.getDeclaredMethod("path") - pathMethod.setAccessible(true) - pathMethod - .invoke(deleteFile) - .asInstanceOf[CharSequence] - .toString - } + // Extract delete files from FileScanTask for MOR (Merge-On-Read) tables. + // When present, these are serialized to protobuf and passed to + // iceberg-rust, which automatically applies deletes during reading. + try { + val deletesMethod = fileScanTaskClass.getMethod("deletes") + val deletes = deletesMethod + .invoke(task) + .asInstanceOf[java.util.List[_]] - val deleteBuilder = - OperatorOuterClass.IcebergDeleteFile.newBuilder() - deleteBuilder.setFilePath(deletePath) + deletes.asScala.foreach { deleteFile => + try { + // scalastyle:off classforname + val deleteFileClass = Class.forName("org.apache.iceberg.DeleteFile") + // scalastyle:on classforname + + val deletePath = + try { + // Try location() first (1.7.0+) + val locationMethod = contentFileClass.getMethod("location") + locationMethod.invoke(deleteFile).asInstanceOf[String] + } catch { + case _: NoSuchMethodException => + val pathMethod = contentFileClass.getMethod("path") + pathMethod + .invoke(deleteFile) + .asInstanceOf[CharSequence] + .toString + } - // Get content type (POSITION_DELETES or EQUALITY_DELETES) - val contentType = - try { - val contentMethod = deleteFileClass.getMethod("content") - contentMethod.setAccessible(true) - val content = contentMethod.invoke(deleteFile) - content.toString match { - case "POSITION_DELETES" => "POSITION_DELETES" - case "EQUALITY_DELETES" => "EQUALITY_DELETES" - case other => other + 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" } - } catch { - case _: Exception => - // Default to POSITION_DELETES if can't determine - "POSITION_DELETES" - } - deleteBuilder.setContentType(contentType) + deleteBuilder.setContentType(contentType) + + val specId = + try { + val specIdMethod = deleteFileClass.getMethod("specId") + specIdMethod.invoke(deleteFile).asInstanceOf[Int] + } catch { + case _: Exception => + 0 + } + deleteBuilder.setPartitionSpecId(specId) - // Get partition spec ID - val specId = try { - val specIdMethod = deleteFileClass.getMethod("specId") - specIdMethod.setAccessible(true) - specIdMethod.invoke(deleteFile).asInstanceOf[Int] + val equalityIdsMethod = + deleteFileClass.getMethod("equalityFieldIds") + val equalityIds = equalityIdsMethod + .invoke(deleteFile) + .asInstanceOf[java.util.List[Integer]] + equalityIds.forEach(id => deleteBuilder.addEqualityIds(id)) } catch { case _: Exception => - // Default to 0 if can't get spec ID - 0 } - deleteBuilder.setPartitionSpecId(specId) - // Get equality field IDs (for equality deletes) - try { - val equalityIdsMethod = - deleteFileClass.getMethod("equalityFieldIds") - equalityIdsMethod.setAccessible(true) - val equalityIds = equalityIdsMethod - .invoke(deleteFile) - .asInstanceOf[java.util.List[Integer]] - equalityIds.forEach(id => deleteBuilder.addEqualityIds(id)) + taskBuilder.addDeleteFiles(deleteBuilder.build()) } catch { - case _: Exception => - // No equality IDs (likely positional deletes) + case e: Exception => + logWarning(s"Failed to serialize delete file: ${e.getMessage}") } - - taskBuilder.addDeleteFiles(deleteBuilder.build()) - } catch { - case e: Exception => - // Failed to serialize delete file - log and continue - logWarning(s"Failed to serialize delete file: ${e.getMessage}") } + } catch { + case e: Exception => + logWarning( + s"Failed to extract deletes from FileScanTask: ${e.getMessage}") } - } catch { - case _: NoSuchMethodException => - // FileScanTask doesn't have deletes() method - case _: Exception => - // Failed to extract deletes - } - // Extract residual expression for row-group level filtering - // - // The residual is created by Iceberg's ResidualEvaluator which partially - // evaluates the scan filter against this file's partition values. - // Different files may have different residuals based on their partitions. - // - // For example: - // - Original filter: date >= '2024-01-01' AND status = 'active' - // - File partition: date = '2024-06-15' - // - Residual: status = 'active' (date condition proven true by partition) - // - // This residual is what should be applied during Parquet row-group - // scanning. - try { - val residualMethod = taskClass.getMethod("residual") - residualMethod.setAccessible(true) - val residualExpr = residualMethod.invoke(task) - - // Convert Iceberg Expression to Catalyst Expression - // The residual is an org.apache.iceberg.expressions.Expression - val catalystExpr = convertIcebergExpression(residualExpr, scan.output) - - // Serialize to protobuf WITHOUT binding to indices - // Iceberg residuals are already unbound (name-based), so we keep them - // unbound in the protobuf to avoid unnecessary index->name resolution - // in Rust - catalystExpr - .flatMap { expr => - exprToProto(expr, scan.output, binding = false) - } - .foreach { protoExpr => - taskBuilder.setResidual(protoExpr) - } + // Extract residual expression for row-group level filtering. + // + // The residual is created by Iceberg's ResidualEvaluator which partially + // evaluates the scan filter against this file's partition values. + // Different files may have different residuals based on their partitions. + // + // For example: + // - Original filter: date >= '2024-01-01' AND status = 'active' + // - File partition: date = '2024-06-15' + // - Residual: status = 'active' (date condition proven true by partition) + // + // This residual is what should be applied during Parquet row-group scanning. + try { + val residualMethod = contentScanTaskClass.getMethod("residual") + val residualExpr = residualMethod.invoke(task) + + val catalystExpr = convertIcebergExpression(residualExpr, scan.output) + + // Serialize to protobuf WITHOUT binding to indices. + // Iceberg residuals are already unbound (name-based), so we keep them + // unbound in the protobuf to avoid unnecessary index->name resolution in Rust + 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}") + } + + partitionBuilder.addFileScanTasks(taskBuilder.build()) } catch { - case _: NoSuchMethodException => - // residual() method not available, skip case e: Exception => - // Failed to extract/convert residual, continue without it - logWarning( - "Failed to extract residual expression from FileScanTask: " + - s"${e.getMessage}") + logWarning(s"Failed to serialize FileScanTask: ${e.getMessage}") } - - // Add task to THIS partition's builder - partitionBuilder.addFileScanTasks(taskBuilder.build()) } } catch { case e: Exception => - // Could not extract tasks from this InputPartition logWarning( s"Failed to extract FileScanTasks from InputPartition: ${e.getMessage}") } } - // Add this partition to the scan builder val builtPartition = partitionBuilder.build() icebergScanBuilder.addFilePartitions(builtPartition) actualNumPartitions += 1 @@ -2317,24 +2297,11 @@ object QueryPlanSerde extends Logging with CometExprShim { * Converts an Iceberg Literal to a Spark Literal */ private def convertIcebergLiteral(icebergLiteral: Any, sparkType: DataType): Literal = { - // Find value() method in class hierarchy (may be in parent class) - def findValueMethod(clazz: Class[_]): Option[java.lang.reflect.Method] = { - try { - val method = clazz.getDeclaredMethod("value") - method.setAccessible(true) - Some(method) - } catch { - case _: NoSuchMethodException => - if (clazz.getSuperclass != null) { - findValueMethod(clazz.getSuperclass) - } else { - None - } - } - } - - val valueMethod = findValueMethod(icebergLiteral.getClass).getOrElse( - throw new RuntimeException(s"Could not find value() method on ${icebergLiteral.getClass}")) + // 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 From 6966a125fd16842f3db2d3f874021a0b8bde7ef3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 6 Oct 2025 12:24:56 -0400 Subject: [PATCH 11/89] Fix scalastyle issues. --- .../apache/comet/serde/QueryPlanSerde.scala | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) 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 3960edcdc2..23aad077b2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1229,7 +1229,7 @@ object QueryPlanSerde extends Logging with CometExprShim { // // Reflection Strategy (for Iceberg 1.5.x - 1.10.x compatibility): // - SparkInputPartition: package-private Spark class, use getDeclaredMethod + setAccessible - // - Iceberg API methods: use Class.forName() on public interfaces, then getMethod() + // - Iceberg API methods: load public interfaces by name, then use getMethod() // (avoids IllegalAccessException on package-private implementation classes like // BaseFileScanTask$SplitScanTask in Iceberg 1.5.x) var actualNumPartitions = 0 @@ -1253,8 +1253,9 @@ object QueryPlanSerde extends Logging with CometExprShim { // Get the task group and extract tasks try { - // SparkInputPartition is package-private, so we need getDeclaredMethod + setAccessible - // This is different from Iceberg API classes which have public interfaces + // SparkInputPartition is package-private, so we need + // getDeclaredMethod + setAccessible. This is different from + // Iceberg API classes which have public interfaces val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") taskGroupMethod.setAccessible(true) val taskGroup = taskGroupMethod.invoke(inputPartition) @@ -1271,7 +1272,8 @@ object QueryPlanSerde extends Logging with CometExprShim { try { val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() - // Load interface classes to avoid IllegalAccessException on package-private implementations + // Load interface classes to avoid IllegalAccessException on + // package-private implementations // scalastyle:off classforname val contentScanTaskClass = Class.forName("org.apache.iceberg.ContentScanTask") @@ -1316,8 +1318,9 @@ object QueryPlanSerde extends Logging with CometExprShim { val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] taskBuilder.setSchemaJson(schemaJson) - // Extract field IDs from the REQUIRED output schema, not the full task schema. - // This ensures we only project the columns actually needed by the query + // Extract field IDs from the REQUIRED output schema, not the full + // task schema. This ensures we only project the columns actually + // needed by the query val columnsMethod = schema.getClass.getMethod("columns") val columns = columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] @@ -1456,7 +1459,8 @@ object QueryPlanSerde extends Logging with CometExprShim { // - File partition: date = '2024-06-15' // - Residual: status = 'active' (date condition proven true by partition) // - // This residual is what should be applied during Parquet row-group scanning. + // This residual is what should be applied during Parquet row-group + // scanning. try { val residualMethod = contentScanTaskClass.getMethod("residual") val residualExpr = residualMethod.invoke(task) @@ -1464,8 +1468,9 @@ object QueryPlanSerde extends Logging with CometExprShim { val catalystExpr = convertIcebergExpression(residualExpr, scan.output) // Serialize to protobuf WITHOUT binding to indices. - // Iceberg residuals are already unbound (name-based), so we keep them - // unbound in the protobuf to avoid unnecessary index->name resolution in Rust + // Iceberg residuals are already unbound (name-based), so we keep + // them unbound in the protobuf to avoid unnecessary index->name + // resolution in Rust catalystExpr .flatMap { expr => exprToProto(expr, scan.output, binding = false) From a0f4d63fe32497682c93bc052b2b57065c7fa5d3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 7 Oct 2025 13:42:37 -0400 Subject: [PATCH 12/89] Remove unused import. --- spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 9250b4e784..afd8ee4a52 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 From a9cebfd378edd506d38aa3796e68d6b50ba3e704 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 7 Oct 2025 15:13:09 -0400 Subject: [PATCH 13/89] Clean up docs a bit. --- .../comet/CometIcebergNativeScanExec.scala | 73 ++++++------------- 1 file changed, 22 insertions(+), 51 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index a3440e175f..86c2c08a98 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -33,13 +33,13 @@ import com.google.common.base.Objects import org.apache.comet.serde.OperatorOuterClass.Operator /** - * Comet fully native Iceberg scan node for DataSource V2 that delegates to iceberg-rust. + * Comet fully native Iceberg scan node for DataSource V2. * - * This replaces Spark's Iceberg BatchScanExec with a native implementation that: - * 1. Extracts catalog configuration from Spark session 2. Serializes catalog info (type, - * properties, namespace, table name) to protobuf 3. Extracts FileScanTasks from Iceberg's - * InputPartitions during planning (on driver) 4. Uses iceberg-rust's catalog implementations - * to load the table and read data natively + * Replaces Spark's Iceberg BatchScanExec by extracting FileScanTasks from Iceberg's planning and + * serializing them to protobuf for native execution. All catalog access and planning happens in + * Spark's Iceberg integration; the Rust side uses iceberg-rust's FileIO and ArrowReader to read + * data files based on the pre-planned FileScanTasks. Catalog properties are used to configure the + * FileIO (for credentials, regions, etc.) * * **How FileScanTask Serialization Works:** * @@ -63,19 +63,11 @@ import org.apache.comet.serde.OperatorOuterClass.Operator * - Rust receives IcebergScan operator with FileScanTasks for ALL partitions * - Each worker reads only the tasks for its partition index * - * **Key Insight:** Unlike the initial approach which tried to extract tasks per-partition at - * execution time, this approach extracts ALL tasks at planning time (just like PartitionedFiles). - * This works because: - * - The RDD and its partitions exist on the driver - * - We don't need TaskContext to access InputPartitions - * - Iceberg has already done the planning and assigned tasks to partitions - * - We just need to serialize this information into the protobuf plan + * All tasks are extracted at planning time because the RDD and partitions exist on the driver, + * and Iceberg has already assigned tasks to partitions. * - * **Why This Works With Filters:** When a filter is on top of CometIcebergNativeScanExec: - * - Filter's convertBlock() serializes both filter and scan together - * - The scan's nativeOp (created by operator2Proto) already contains all FileScanTasks - * - The combined filter+scan native plan is executed as one unit - * - No special RDD or per-partition logic needed + * **Filters:** When a filter is on top of this scan, both are serialized together and executed as + * one unit. No special RDD or per-partition logic needed. */ case class CometIcebergNativeScanExec( override val nativeOp: Operator, @@ -91,17 +83,13 @@ case class CometIcebergNativeScanExec( override val supportsColumnar: Boolean = true - // No need to override doExecuteColumnar - parent CometLeafExec handles it // FileScanTasks are serialized at planning time in QueryPlanSerde.operator2Proto() - // just like PartitionedFiles are for CometNativeScanExec override val nodeName: String = s"CometIcebergNativeScan ${namespace.mkString(".")}.$tableName ($catalogType)" - // Use the actual number of partitions from Iceberg's planning. - // FileScanTasks are extracted and serialized at planning time (in QueryPlanSerde.operator2Proto), - // grouped by partition. Each partition receives only its assigned tasks via the protobuf message. - // The Rust side uses the partition index to select the correct task group. + // FileScanTasks are serialized at planning time and grouped by partition. + // Rust uses the partition index to select the correct task group. override lazy val outputPartitioning: Partitioning = UnknownPartitioning(numPartitions) @@ -153,7 +141,7 @@ case class CometIcebergNativeScanExec( "time_elapsed_opening" -> SQLMetrics.createNanoTimingMetric( sparkContext, - "Wall clock time elapsed for catalog loading and table opening"), + "Wall clock time elapsed for FileIO initialization"), "time_elapsed_scanning_until_data" -> SQLMetrics.createNanoTimingMetric( sparkContext, @@ -178,7 +166,7 @@ case class CometIcebergNativeScanExec( object CometIcebergNativeScanExec { /** - * Information extracted from an Iceberg table scan for native execution. + * Configuration extracted from Spark catalog for FileIO setup and identification. */ case class IcebergCatalogInfo( catalogType: String, @@ -189,10 +177,8 @@ object CometIcebergNativeScanExec { /** * Extracts Iceberg catalog configuration from a Spark BatchScanExec. * - * This method: - * 1. Gets the catalog name from the table identifier 2. Extracts catalog configuration from - * Spark session config 3. Maps Spark's catalog implementation class to iceberg-rust - * catalog types 4. Returns catalog info ready for serialization + * Gets the catalog name from the table identifier and extracts configuration from Spark + * session. * * @param scanExec * The Spark BatchScanExec containing an Iceberg scan @@ -300,7 +286,7 @@ object CometIcebergNativeScanExec { // Get the catalog implementation class val implClass = catalogImpl.getOrElse(return None) - // Map Spark's catalog implementation to iceberg-rust catalog type + // Identify catalog type for UI/debugging val icebergCatalogType = implClass match { case impl if impl.contains("RESTCatalog") || impl.contains("rest") => "rest" case impl if impl.contains("GlueCatalog") || impl.contains("glue") => "glue" @@ -336,22 +322,9 @@ object CometIcebergNativeScanExec { /** * Creates a CometIcebergNativeScanExec from a Spark BatchScanExec. * - * This method is called on the driver to create the Comet operator. The key step is determining - * the number of partitions to use, which affects parallelism. - * - * **Partition Count Strategy:** - * - For KeyGroupedPartitioning: Use Iceberg's partition count (data is grouped by partition - * keys) - * - For other cases: Use the number of partitions in inputRDD, which Spark computes based on - * the number of InputPartition objects returned by Iceberg's planInputPartitions() - * - * **How FileScanTasks Flow to Workers:** - * 1. Iceberg's planInputPartitions() creates InputPartition[] (each contains ScanTaskGroup) - * 2. Spark creates BatchScanExec.inputRDD with these InputPartitions 3. Each RDD partition - * wraps one InputPartition 4. Spark ships RDD partitions to workers 5. On worker: - * QueryPlanSerde extracts FileScanTasks from the InputPartition 6. FileScanTasks are - * serialized to protobuf and sent to Rust 7. Rust reads only the files specified in those - * tasks + * 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 @@ -378,9 +351,7 @@ object CometIcebergNativeScanExec { // Use Iceberg's key-grouped partition count p.numPartitions case _ => - // For unpartitioned tables or other partitioning schemes, - // use the InputPartition count from inputRDD - // This is already computed by Spark based on Iceberg's file planning + // Use the number of InputPartitions from inputRDD scanExec.inputRDD.getNumPartitions } @@ -388,7 +359,7 @@ object CometIcebergNativeScanExec { nativeOp, scanExec.output, scanExec, - SerializedPlan(None), // Will be serialized per-partition during execution + SerializedPlan(None), catalogInfo.catalogType, catalogInfo.properties, catalogInfo.namespace, From 6b2175ad7ed2617e5d77fbfc4734202af6c9b575 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 7 Oct 2025 18:01:58 -0400 Subject: [PATCH 14/89] Refactor and cleanup. --- native/core/src/execution/planner.rs | 11 +- native/proto/src/proto/operator.proto | 8 +- .../apache/comet/rules/CometExecRule.scala | 29 ++- .../apache/comet/rules/CometScanRule.scala | 60 +---- .../apache/comet/serde/QueryPlanSerde.scala | 115 +++------- .../comet/CometIcebergNativeScanExec.scala | 207 ++++-------------- 6 files changed, 114 insertions(+), 316 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 102f6de1dd..fe29d2bc7b 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1417,15 +1417,8 @@ impl PhysicalPlanner { .map(|(k, v)| (k.clone(), v.clone())) .collect(); - // Extract metadata_location from catalog_properties (required) - let metadata_location = catalog_properties - .get("metadata_location") - .ok_or_else(|| { - ExecutionError::GeneralError( - "metadata_location not found in catalog_properties".to_string(), - ) - })? - .clone(); + // Get metadata location from separate field + let metadata_location = scan.metadata_location.clone(); // Parse pre-planned FileScanTasks if provided (grouped by partition) // diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 650d59e381..0dcbef64b1 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -113,8 +113,8 @@ message IcebergScan { repeated SparkStructField required_schema = 1; // Catalog-specific configuration properties used to build FileIO for reading files - // MUST contain "metadata_location" key for the table metadata file/directory - // Examples: S3 credentials, warehouse location, metadata_location, etc. + // TODO: Extract credentials, S3/GCS config, etc. for authentication + // Currently empty - metadata_location is in separate field below map catalog_properties = 2; // Pre-planned file scan tasks grouped by Spark partition @@ -126,6 +126,10 @@ message IcebergScan { // Total number of Spark partitions // Used to set up proper partitioning in DataFusion execution plan int32 num_partitions = 4; + + // Path to the table metadata file (e.g., s3://bucket/warehouse/db.db/table/metadata/v1.metadata.json) + // Required for FileIO initialization + string metadata_location = 5; } // Groups FileScanTasks for a single Spark partition 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 56ab1b977f..45f6ab64aa 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -160,10 +160,31 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val nativeOp = QueryPlanSerde.operator2Proto(scan).get CometNativeScanExec(nativeOp, scan.wrapped, scan.session) - // CometIcebergNativeScanExec is already a CometNativeExec and will be serialized - // during convertBlock(). Don't wrap it in CometScanWrapper. - case scan: CometIcebergNativeScanExec => - scan + // Fully native Iceberg scan for V2 - convert CometBatchScanExec to CometIcebergNativeScanExec + case scan: CometBatchScanExec + if CometConf.COMET_ICEBERG_NATIVE_ENABLED.get(conf) && + CometConf.COMET_EXEC_ENABLED.get(conf) && + 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) => 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 afd8ee4a52..de81b86e73 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Generic import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MetadataColumnHelper} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues -import org.apache.spark.sql.comet.{CometBatchScanExec, CometIcebergNativeScanExec, CometScanExec, SerializedPlan} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.v2.BatchScanExec @@ -277,59 +277,11 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } if (schemaSupported) { - // Check if native Iceberg execution is enabled - if (CometConf.COMET_ICEBERG_NATIVE_ENABLED.get() && - CometConf.COMET_EXEC_ENABLED.get()) { - - // Try to extract catalog info for native execution - CometIcebergNativeScanExec.extractCatalogInfo(scanExec, session) match { - case Some(catalogInfo) => - // Create native Iceberg scan exec with IcebergScan operator (without tasks) - // Tasks will be extracted per-partition during execution in doExecuteColumnar() - // First create a temporary exec to serialize - val tempExec = CometIcebergNativeScanExec( - org.apache.comet.serde.OperatorOuterClass.Operator.newBuilder().build(), - scanExec.output, - scanExec, - SerializedPlan(None), - catalogInfo.catalogType, - catalogInfo.properties, - catalogInfo.namespace, - catalogInfo.tableName, - 1) - - // Now serialize it to get the IcebergScan operator (without tasks) - val nativeOp = - org.apache.comet.serde.QueryPlanSerde.operator2Proto(tempExec).getOrElse { - // If serialization fails, fall back to Spark - return scanExec - } - - val nativeScan = - CometIcebergNativeScanExec(nativeOp, scanExec, session, catalogInfo) - - // When reading from Iceberg, automatically enable type promotion - SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") - - nativeScan - - case None => - // Catalog not supported, fall back to normal Comet batch scan - fallbackReasons += - "Native Iceberg execution enabled but catalog type not supported " + - s"(${scanExec.table.name()})" - SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") - CometBatchScanExec( - scanExec.clone().asInstanceOf[BatchScanExec], - runtimeFilters = scanExec.runtimeFilters) - } - } else { - // Use regular Comet batch scan (Spark's Iceberg reader with Comet vectors) - SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") - CometBatchScanExec( - scanExec.clone().asInstanceOf[BatchScanExec], - runtimeFilters = scanExec.runtimeFilters) - } + // When reading from Iceberg, automatically enable type promotion + SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") + CometBatchScanExec( + scanExec.clone().asInstanceOf[BatchScanExec], + runtimeFilters = scanExec.runtimeFilters) } else { withInfos(scanExec, fallbackReasons.toSet) } 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 1d97286e81..2d957b41eb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, NormalizeNaNAndZero} import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet._ @@ -1183,32 +1184,43 @@ object QueryPlanSerde extends Logging with CometExprShim { None } - // Fully native Iceberg scan for V2 using iceberg-rust - // - // IMPORTANT: This serialization happens on the Spark worker, not the driver! - // Each worker receives a specific InputPartition via Spark's RDD distribution. - // We extract the FileScanTasks from the InputPartition and serialize them to protobuf. - // - // Flow: - // 1. Driver: Iceberg's planInputPartitions() creates InputPartition[] - // (each contains ScanTaskGroup) - // 2. Driver: Spark creates BatchScanExec.inputRDD with these InputPartitions - // 3. Spark distributes RDD partitions to workers - // 4. Worker: This code runs and extracts FileScanTasks from the worker's - // specific InputPartition - // 5. Worker: FileScanTasks are serialized to protobuf and sent to Rust - // 6. Rust: Reads only the files specified in those tasks - case scan: CometIcebergNativeScanExec => + // Iceberg scan with native execution enabled - + // detected by CometBatchScanExec wrapping SparkBatchQueryScan + case scan: CometBatchScanExec + if CometConf.COMET_ICEBERG_NATIVE_ENABLED.get() && + CometConf.COMET_EXEC_ENABLED.get() && + scan.wrapped.scan.getClass.getName == + "org.apache.iceberg.spark.source.SparkBatchQueryScan" => val icebergScanBuilder = OperatorOuterClass.IcebergScan.newBuilder() - // Serialize catalog properties (contains metadata_location, credentials, S3 config, etc.) - // The native side will extract metadata_location from this map - scan.catalogProperties.foreach { case (key, value) => - icebergScanBuilder.putCatalogProperties(key, value) + // 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 + } + + // Set metadata location + icebergScanBuilder.setMetadataLocation(metadataLocation) + + // Serialize catalog properties (for authentication - currently empty) + // TODO: Extract credentials, S3 config, etc. + + // Determine number of partitions from Iceberg's output partitioning + // TODO: Add a test case for both partitioning schemes + val numParts = scan.wrapped.outputPartitioning match { + case p: KeyGroupedPartitioning => + p.numPartitions + case _ => + scan.wrapped.inputRDD.getNumPartitions } // Set number of partitions - icebergScanBuilder.setNumPartitions(scan.numPartitions) + icebergScanBuilder.setNumPartitions(numParts) // Set required_schema from output scan.output.foreach { attr => @@ -1219,29 +1231,14 @@ object QueryPlanSerde extends Logging with CometExprShim { icebergScanBuilder.addRequiredSchema(field.build()) } - // No need to serialize projection_vector - scan.output already contains only - // the projected columns from Spark's optimization. The native side will use - // None for projection, which tells iceberg-rust to use the full schema. - - // No need to serialize scan-level data_filters - each FileScanTask already contains - // its own residual expression which is the optimized per-file filter from Iceberg's - // ResidualEvaluator. The residuals are used for row-group level filtering. - // Extract FileScanTasks from the InputPartitions in the RDD - // Group tasks by Spark partition (similar to how NativeScan groups PartitionedFiles) - // - // Reflection Strategy (for Iceberg 1.5.x - 1.10.x compatibility): - // - SparkInputPartition: package-private Spark class, use getDeclaredMethod + setAccessible - // - Iceberg API methods: load public interfaces by name, then use getMethod() - // (avoids IllegalAccessException on package-private implementation classes like - // BaseFileScanTask$SplitScanTask in Iceberg 1.5.x) + // (Same logic as the previous CometIcebergNativeScanExec case) var actualNumPartitions = 0 try { - scan.originalPlan.inputRDD match { + scan.wrapped.inputRDD match { case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => val partitions = rdd.partitions partitions.foreach { partition => - // Create a partition builder for this Spark partition val partitionBuilder = OperatorOuterClass.IcebergFilePartition.newBuilder() val inputPartitions = partition @@ -1250,33 +1247,23 @@ object QueryPlanSerde extends Logging with CometExprShim { .inputPartitions inputPartitions.foreach { inputPartition => - // Extract FileScanTasks from this InputPartition using reflection - // InputPartition is SparkInputPartition containing ScanTaskGroup val inputPartClass = inputPartition.getClass - // Get the task group and extract tasks try { - // SparkInputPartition is package-private, so we need - // getDeclaredMethod + setAccessible. This is different from - // Iceberg API classes which have public interfaces val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") taskGroupMethod.setAccessible(true) val taskGroup = taskGroupMethod.invoke(inputPartition) - // Call tasks() on ScanTaskGroup to get Collection val taskGroupClass = taskGroup.getClass val tasksMethod = taskGroupClass.getMethod("tasks") val tasksCollection = tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] - // Convert to Scala and serialize each task import scala.jdk.CollectionConverters._ tasksCollection.asScala.foreach { task => try { val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() - // Load interface classes to avoid IllegalAccessException on - // package-private implementations // scalastyle:off classforname val contentScanTaskClass = Class.forName("org.apache.iceberg.ContentScanTask") @@ -1289,12 +1276,10 @@ object QueryPlanSerde extends Logging with CometExprShim { val filePath = try { - // Try location() first (1.7.0+) val locationMethod = contentFileClass.getMethod("location") locationMethod.invoke(dataFile).asInstanceOf[String] } catch { case _: NoSuchMethodException => - // Fall back to path() (pre-1.7.0, returns CharSequence) val pathMethod = contentFileClass.getMethod("path") pathMethod.invoke(dataFile).asInstanceOf[CharSequence].toString } @@ -1321,9 +1306,6 @@ object QueryPlanSerde extends Logging with CometExprShim { val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] taskBuilder.setSchemaJson(schemaJson) - // Extract field IDs from the REQUIRED output schema, not the full - // task schema. This ensures we only project the columns actually - // needed by the query val columnsMethod = schema.getClass.getMethod("columns") val columns = columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] @@ -1370,9 +1352,6 @@ object QueryPlanSerde extends Logging with CometExprShim { taskBuilder.setDataFileFormat("PARQUET") } - // Extract delete files from FileScanTask for MOR (Merge-On-Read) tables. - // When present, these are serialized to protobuf and passed to - // iceberg-rust, which automatically applies deletes during reading. try { val deletesMethod = fileScanTaskClass.getMethod("deletes") val deletes = deletesMethod @@ -1387,7 +1366,6 @@ object QueryPlanSerde extends Logging with CometExprShim { val deletePath = try { - // Try location() first (1.7.0+) val locationMethod = contentFileClass.getMethod("location") locationMethod.invoke(deleteFile).asInstanceOf[String] } catch { @@ -1451,29 +1429,12 @@ object QueryPlanSerde extends Logging with CometExprShim { s"Failed to extract deletes from FileScanTask: ${e.getMessage}") } - // Extract residual expression for row-group level filtering. - // - // The residual is created by Iceberg's ResidualEvaluator which partially - // evaluates the scan filter against this file's partition values. - // Different files may have different residuals based on their partitions. - // - // For example: - // - Original filter: date >= '2024-01-01' AND status = 'active' - // - File partition: date = '2024-06-15' - // - Residual: status = 'active' (date condition proven true by partition) - // - // This residual is what should be applied during Parquet row-group - // scanning. try { val residualMethod = contentScanTaskClass.getMethod("residual") val residualExpr = residualMethod.invoke(task) val catalystExpr = convertIcebergExpression(residualExpr, scan.output) - // Serialize to protobuf WITHOUT binding to indices. - // Iceberg residuals are already unbound (name-based), so we keep - // them unbound in the protobuf to avoid unnecessary index->name - // resolution in Rust catalystExpr .flatMap { expr => exprToProto(expr, scan.output, binding = false) @@ -1513,15 +1474,11 @@ object QueryPlanSerde extends Logging with CometExprShim { logWarning(s"Failed to extract FileScanTasks from Iceberg scan RDD: ${e.getMessage}") } - // Set number of partitions for proper data distribution - // Use the actual count of partitions we serialized, not scan.numPartitions val numPartitions = - if (actualNumPartitions > 0) actualNumPartitions else scan.numPartitions + if (actualNumPartitions > 0) actualNumPartitions else numParts icebergScanBuilder.setNumPartitions(numPartitions) - // Iceberg scans don't have children builder.clearChildren() - Some(builder.setIcebergScan(icebergScanBuilder).build()) case FilterExec(condition, child) if CometConf.COMET_EXEC_FILTER_ENABLED.get(conf) => diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 86c2c08a98..85a6d53949 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -74,19 +74,16 @@ case class CometIcebergNativeScanExec( override val output: Seq[Attribute], @transient override val originalPlan: BatchScanExec, override val serializedPlanOpt: SerializedPlan, - catalogType: String, - catalogProperties: Map[String, String], - namespace: Seq[String], - tableName: String, - numPartitions: Int) // Number of Spark partitions for proper parallelism + metadataLocation: String, + catalogProperties: Map[String, String], // TODO: Extract for authentication + numPartitions: Int) extends CometLeafExec { override val supportsColumnar: Boolean = true // FileScanTasks are serialized at planning time in QueryPlanSerde.operator2Proto() - override val nodeName: String = - s"CometIcebergNativeScan ${namespace.mkString(".")}.$tableName ($catalogType)" + override val nodeName: String = "CometIcebergNativeScan" // FileScanTasks are serialized at planning time and grouped by partition. // Rust uses the partition index to select the correct task group. @@ -101,23 +98,19 @@ case class CometIcebergNativeScanExec( output.map(QueryPlan.normalizeExpressions(_, output)), originalPlan.doCanonicalize(), SerializedPlan(None), - catalogType, + metadataLocation, catalogProperties, - namespace, - tableName, numPartitions) } override def stringArgs: Iterator[Any] = - Iterator(output, catalogType, namespace, tableName, numPartitions) + Iterator(output, metadataLocation, numPartitions) override def equals(obj: Any): Boolean = { obj match { case other: CometIcebergNativeScanExec => - this.catalogType == other.catalogType && + this.metadataLocation == other.metadataLocation && this.catalogProperties == other.catalogProperties && - this.namespace == other.namespace && - this.tableName == other.tableName && this.output == other.output && this.serializedPlanOpt == other.serializedPlanOpt && this.numPartitions == other.numPartitions @@ -128,9 +121,7 @@ case class CometIcebergNativeScanExec( override def hashCode(): Int = Objects.hashCode( - catalogType, - namespace.asJava, - tableName, + metadataLocation, output.asJava, serializedPlanOpt, numPartitions: java.lang.Integer) @@ -166,157 +157,43 @@ case class CometIcebergNativeScanExec( object CometIcebergNativeScanExec { /** - * Configuration extracted from Spark catalog for FileIO setup and identification. - */ - case class IcebergCatalogInfo( - catalogType: String, - properties: Map[String, String], - namespace: Seq[String], - tableName: String) - - /** - * Extracts Iceberg catalog configuration from a Spark BatchScanExec. + * Extracts metadata location from Iceberg table. * - * Gets the catalog name from the table identifier and extracts configuration from Spark - * session. + * TODO: Also extract catalog properties (credentials, S3 config, etc.) for authentication * * @param scanExec * The Spark BatchScanExec containing an Iceberg scan - * @param session - * The SparkSession to extract catalog config from * @return - * Some(IcebergCatalogInfo) if catalog is supported, None otherwise + * Path to the table metadata file */ - def extractCatalogInfo( - scanExec: BatchScanExec, - session: SparkSession): Option[IcebergCatalogInfo] = { - try { - // Get the full table name from Spark's table identifier - // Format: "catalog_name.namespace.table_name" or "namespace.table_name" - val fullTableName = scanExec.table.name() - val parts = fullTableName.split('.') - - if (parts.length < 2) { - return None // Need at least namespace.table - } - - // Determine catalog name and table path - // If 3+ parts: parts(0) is catalog, parts(1..-2) is namespace, parts(-1) is table - // If 2 parts: default catalog, parts(0) is namespace, parts(1) is table - val (catalogName, namespaceParts, tableNamePart) = if (parts.length >= 3) { - (parts.head, parts.slice(1, parts.length - 1).toSeq, parts.last) - } else { - // Try to get default catalog from config - val defaultCatalog = session.conf - .getOption("spark.sql.catalog.spark_catalog") - .map(_ => "spark_catalog") - .getOrElse(return None) - (defaultCatalog, Seq(parts.head), parts.last) - } - - // Get catalog properties from Spark session config - val catalogPrefix = s"spark.sql.catalog.$catalogName" - - // Check both catalog-impl and type properties - val catalogImpl = session.conf.getOption(s"$catalogPrefix.catalog-impl") - val catalogType = session.conf.getOption(s"$catalogPrefix.type") - - // Handle Hadoop catalog specially - it uses direct metadata file access - if (catalogType.contains("hadoop") || - catalogImpl.exists(impl => impl.contains("HadoopCatalog"))) { - - // Hadoop catalog is filesystem-based, need to extract metadata location - // Try to get it from the table object via reflection - try { - val scan = scanExec.scan - val scanClass = scan.getClass - - // Try to get the table via reflection - // Iceberg's SparkBatchQueryScan extends SparkScan which has protected table() method - // Need to search up the class hierarchy - def findTableMethod(clazz: Class[_]): Option[java.lang.reflect.Method] = { - if (clazz == null || clazz == classOf[Object]) { - None - } else { - try { - val method = clazz.getDeclaredMethod("table") - method.setAccessible(true) - Some(method) - } catch { - case _: NoSuchMethodException => - // Try superclass - findTableMethod(clazz.getSuperclass) - } - } - } - - val tableMethod = findTableMethod(scanClass).getOrElse { - throw new NoSuchMethodException("Could not find table() method in class hierarchy") - } - - val table = tableMethod.invoke(scan) - - // Get the metadata location from table.operations().current().metadataFileLocation() - val tableClass = table.getClass - val operationsMethod = tableClass.getMethod("operations") - val operations = operationsMethod.invoke(table) - - val operationsClass = operations.getClass - val currentMethod = operationsClass.getMethod("current") - val metadata = currentMethod.invoke(operations) - - val metadataClass = metadata.getClass - val metadataFileLocationMethod = metadataClass.getMethod("metadataFileLocation") - val metadataLocation = metadataFileLocationMethod.invoke(metadata).asInstanceOf[String] - - // Return catalog info with actual metadata file location - return Some( - IcebergCatalogInfo( - catalogType = "hadoop", - properties = Map("metadata_location" -> metadataLocation), - namespace = namespaceParts, - tableName = tableNamePart)) - } catch { - case _: Exception => - // If reflection fails, fall back to returning None - return None - } + 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") + } - // Get the catalog implementation class - val implClass = catalogImpl.getOrElse(return None) + val table = tableMethod.invoke(scan) - // Identify catalog type for UI/debugging - val icebergCatalogType = implClass match { - case impl if impl.contains("RESTCatalog") || impl.contains("rest") => "rest" - case impl if impl.contains("GlueCatalog") || impl.contains("glue") => "glue" - case impl if impl.contains("HiveCatalog") || impl.contains("hive") => "hms" - case impl if impl.contains("JdbcCatalog") || impl.contains("jdbc") => "sql" - case _ => return None // Unsupported catalog type - } + val operationsMethod = table.getClass.getMethod("operations") + val operations = operationsMethod.invoke(table) - // Extract all catalog properties with the prefix - val catalogProps = session.conf.getAll - .filter { case (k, _) => k.startsWith(catalogPrefix + ".") } - .map { case (k, v) => - // Remove prefix: "spark.sql.catalog.mycatalog.uri" -> "uri" - val key = k.stripPrefix(catalogPrefix + ".") - // Skip the catalog-impl property itself - if (key == "catalog-impl" || key == "type") None else Some((key, v)) - } - .flatten - .toMap + val currentMethod = operations.getClass.getMethod("current") + val metadata = currentMethod.invoke(operations) - Some( - IcebergCatalogInfo( - catalogType = icebergCatalogType, - properties = catalogProps, - namespace = namespaceParts, - tableName = tableNamePart)) - } catch { - case _: Exception => - None - } + val metadataFileLocationMethod = metadata.getClass.getMethod("metadataFileLocation") + metadataFileLocationMethod.invoke(metadata).asInstanceOf[String] } /** @@ -332,8 +209,8 @@ object CometIcebergNativeScanExec { * The original Spark BatchScanExec * @param session * The SparkSession - * @param catalogInfo - * The extracted catalog information + * @param metadataLocation + * Path to table metadata file from extractMetadataLocation * @return * A new CometIcebergNativeScanExec */ @@ -341,17 +218,13 @@ object CometIcebergNativeScanExec { nativeOp: Operator, scanExec: BatchScanExec, session: SparkSession, - catalogInfo: IcebergCatalogInfo): CometIcebergNativeScanExec = { + metadataLocation: String): CometIcebergNativeScanExec = { // Determine number of partitions from Iceberg's output partitioning - // KeyGroupedPartitioning means Iceberg grouped data by partition keys - // Otherwise, use the number of InputPartitions that Iceberg created val numParts = scanExec.outputPartitioning match { case p: KeyGroupedPartitioning => - // Use Iceberg's key-grouped partition count p.numPartitions case _ => - // Use the number of InputPartitions from inputRDD scanExec.inputRDD.getNumPartitions } @@ -360,10 +233,8 @@ object CometIcebergNativeScanExec { scanExec.output, scanExec, SerializedPlan(None), - catalogInfo.catalogType, - catalogInfo.properties, - catalogInfo.namespace, - catalogInfo.tableName, + metadataLocation, + Map.empty, // TODO: Extract catalog properties for authentication numParts) scanExec.logicalLink.foreach(exec.setLogicalLink) From 3618407c29fc6cce6f136bbdadea7ab9aaa9d8f8 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 7 Oct 2025 18:21:34 -0400 Subject: [PATCH 15/89] Refactor and cleanup. --- .../src/execution/operators/iceberg_scan.rs | 40 +++-------- native/proto/src/proto/operator.proto | 69 ++++++------------- 2 files changed, 29 insertions(+), 80 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index f3537281a0..b03fd2f226 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -39,25 +39,22 @@ use crate::execution::operators::ExecutionError; /// Native Iceberg scan operator that uses iceberg-rust to read Iceberg tables. /// -/// This operator completely bypasses Spark's DataSource V2 API and uses iceberg-rust by using -/// pre-planned FileScanTasks to read data. +/// Bypasses Spark's DataSource V2 API by reading pre-planned FileScanTasks directly. #[derive(Debug)] pub struct IcebergScanExec { - /// Path to Iceberg table metadata file or directory (used for FileIO creation) + /// Iceberg table metadata location for FileIO initialization metadata_location: String, /// Output schema after projection output_schema: SchemaRef, /// Cached execution plan properties plan_properties: PlanProperties, - /// Catalog-specific configuration properties (used to build FileIO for reading files) + /// Catalog-specific configuration for FileIO catalog_properties: HashMap, - /// Pre-planned file scan tasks from Scala, grouped by partition. - /// Each inner Vec contains tasks for one Spark partition. + /// Pre-planned file scan tasks from Scala, grouped by Spark partition file_task_groups: Option>>, } impl IcebergScanExec { - /// Creates a new IcebergScanExec pub fn new( metadata_location: String, schema: SchemaRef, @@ -67,7 +64,6 @@ impl IcebergScanExec { ) -> Result { let output_schema = schema; - // Compute plan properties with actual partition count let plan_properties = Self::compute_properties(Arc::clone(&output_schema), num_partitions); Ok(Self { @@ -79,10 +75,8 @@ impl IcebergScanExec { }) } - /// Computes execution plan properties with actual partition count fn compute_properties(schema: SchemaRef, num_partitions: usize) -> PlanProperties { - // Use the actual partition count from Iceberg's planning - // This matches the number of Spark partitions and ensures proper parallelism + // Matches Spark partition count to ensure proper parallelism PlanProperties::new( EquivalenceProperties::new(schema), Partitioning::UnknownPartitioning(num_partitions), @@ -125,8 +119,7 @@ impl ExecutionPlan for IcebergScanExec { partition: usize, _context: Arc, ) -> DFResult { - // Use pre-planned tasks from Scala - // All planning happens on the Scala side using Iceberg's Java API + // Execute pre-planned tasks from Scala (planning happens via Iceberg's Java API) if let Some(ref task_groups) = self.file_task_groups { if partition < task_groups.len() { let tasks = &task_groups[partition]; @@ -141,7 +134,6 @@ impl ExecutionPlan for IcebergScanExec { } } - // If no tasks were provided, this is an error Err(DataFusionError::Execution(format!( "IcebergScanExec: No FileScanTasks provided for partition {}. \ All scan planning must happen on the Scala side.", @@ -151,14 +143,8 @@ impl ExecutionPlan for IcebergScanExec { } impl IcebergScanExec { - /// **MOR (Merge-On-Read) Table Support:** - /// - /// If the FileScanTasks include delete files (for MOR tables), iceberg-rust's ArrowReader - /// automatically applies the deletes during reading: - /// - Positional deletes: Skips rows at specified positions - /// - Equality deletes: Filters out rows matching delete predicates - /// - /// This ensures that deleted rows are not included in the query results. + /// 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, @@ -168,22 +154,17 @@ impl IcebergScanExec { let metadata_location = self.metadata_location.clone(); let fut = async move { - // Build FileIO from catalog properties let file_io = Self::load_file_io(&catalog_properties, &metadata_location)?; - // Convert Vec to a stream let task_stream = futures::stream::iter(tasks.into_iter().map(Ok)).boxed(); - // Use iceberg-rust's ArrowReader to read the tasks let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io).build(); - // Read the tasks - this returns Result - // No await needed - read() is synchronous + // read() is synchronous and returns Result let stream = reader.read(task_stream).map_err(|e| { DataFusionError::Execution(format!("Failed to read Iceberg tasks: {}", e)) })?; - // Stream already has correct error type mapping let mapped_stream = stream .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))); @@ -201,16 +182,13 @@ impl IcebergScanExec { ))) } - /// Build FileIO from catalog properties fn load_file_io( catalog_properties: &HashMap, metadata_location: &str, ) -> Result { - // Create a FileIO builder let mut file_io_builder = FileIO::from_path(metadata_location) .map_err(|e| DataFusionError::Execution(format!("Failed to create FileIO: {}", e)))?; - // Add catalog properties as configuration for (key, value) in catalog_properties { file_io_builder = file_io_builder.with_prop(key, value); } diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 0dcbef64b1..c42966f825 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -112,97 +112,68 @@ message IcebergScan { // Schema to read repeated SparkStructField required_schema = 1; - // Catalog-specific configuration properties used to build FileIO for reading files - // TODO: Extract credentials, S3/GCS config, etc. for authentication - // Currently empty - metadata_location is in separate field below + // Catalog-specific configuration for FileIO (credentials, S3/GCS config, etc.) map catalog_properties = 2; // Pre-planned file scan tasks grouped by Spark partition - // Each IcebergFilePartition contains tasks for one Spark partition. - // This structure mirrors SparkFilePartition used by NativeScan. - // NOTE: Each task contains its own residual filter expression for row-group pruning. repeated IcebergFilePartition file_partitions = 3; - // Total number of Spark partitions - // Used to set up proper partitioning in DataFusion execution plan + // Number of Spark partitions for DataFusion parallelism int32 num_partitions = 4; - // Path to the table metadata file (e.g., s3://bucket/warehouse/db.db/table/metadata/v1.metadata.json) - // Required for FileIO initialization + // Table metadata file path for FileIO initialization string metadata_location = 5; } // Groups FileScanTasks for a single Spark partition -// This mirrors SparkFilePartition structure used by NativeScan message IcebergFilePartition { repeated IcebergFileScanTask file_scan_tasks = 1; } -// Represents a single Iceberg FileScanTask from iceberg-rust -// This maps directly to iceberg::scan::FileScanTask +// Iceberg FileScanTask containing data file, delete files, and residual filter message IcebergFileScanTask { - // Path to the data file (e.g., s3://bucket/warehouse/db/table/data/00000-0-abc.parquet) + // Data file path (e.g., s3://bucket/warehouse/db/table/data/00000-0-abc.parquet) string data_file_path = 1; - // Byte range within the file to read (for split files) + // Byte range to read (for split files) uint64 start = 2; uint64 length = 3; - // Optional: Record count if reading entire file + // Record count if reading entire file optional uint64 record_count = 4; - // File format: "PARQUET", "AVRO", "ORC" + // File format (PARQUET, AVRO, or ORC) string data_file_format = 5; - // Iceberg schema for this file (JSON serialized) - // Each file may have a different schema due to schema evolution + // File schema as JSON (may differ due to schema evolution) string schema_json = 6; - // Field IDs to project from the file (subset of schema) + // Field IDs to project repeated int32 project_field_ids = 7; - // Delete files that apply to this data file (for Merge-On-Read tables) + // Delete files for MOR tables repeated IcebergDeleteFile delete_files = 8; - // Residual filter expression for this specific file - // - // This is NOT the same as the scan-level data_filters in IcebergScan! - // - // The residual is created by Iceberg's ResidualEvaluator which partially evaluates - // the scan filter against this file's partition values. For example: - // - Original filter: date >= '2024-01-01' AND status = 'active' - // - File partition: date = '2024-06-15' - // - Residual for this file: status = 'active' (date filter proven true by partition) - // - // Different files can have different residuals based on their partition values. - // This allows optimal row-group level filtering in the Parquet reader. - // - // If empty, no filtering is needed for this file (all rows match). + // 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; } -// Represents an Iceberg delete file (equality or positional deletes) -// Used for MOR (Merge-On-Read) tables where deletes are tracked separately from data files. -// -// Iceberg supports two delete strategies: -// 1. Positional deletes: Delete files contain (file_path, row_position) pairs -// indicating specific rows to skip when reading data files. -// 2. Equality deletes: Delete files contain values for specific columns (identified by -// equality_ids). Any row in the data file matching those values should be filtered out. -// -// The Rust side (iceberg-rust ArrowReader) applies these deletes automatically during scanning. +// 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 { - // Path to the delete file + // Delete file path string file_path = 1; - // Content type: "POSITION_DELETES" or "EQUALITY_DELETES" + // POSITION_DELETES or EQUALITY_DELETES string content_type = 2; // Partition spec ID int32 partition_spec_id = 3; - // Equality field IDs (for equality deletes only, empty for positional deletes) - // These IDs identify which columns to use for matching when applying equality deletes + // Equality field IDs (empty for positional deletes) repeated int32 equality_ids = 4; } From 8091a812e81cec52dcf460bd6dec8394b7e87d31 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 8 Oct 2025 06:45:53 -0400 Subject: [PATCH 16/89] Add IcebergFileStream based on DataFusion, add benchmark. Bump the Iceberg version back to 1.8.1 after hitting known segfaults with old versions. --- .../src/execution/operators/iceberg_scan.rs | 239 +++++++++++++++--- spark/pom.xml | 4 +- .../sql/benchmark/CometBenchmarkBase.scala | 36 +++ .../sql/benchmark/CometReadBenchmark.scala | 182 ++++++++----- 4 files changed, 362 insertions(+), 99 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index b03fd2f226..5f7ebf2b06 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -18,21 +18,24 @@ //! Native Iceberg table scan operator using iceberg-rust use std::any::Any; -use std::collections::HashMap; +use std::collections::{HashMap, VecDeque}; use std::fmt; use std::pin::Pin; use std::sync::Arc; +use std::task::{Context, Poll}; +use arrow::array::RecordBatch; use arrow::datatypes::SchemaRef; use datafusion::common::{DataFusionError, Result as DFResult}; -use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }; -use futures::{StreamExt, TryStreamExt}; +use futures::future::BoxFuture; +use futures::{ready, FutureExt, Stream, StreamExt, TryStreamExt}; use iceberg::io::FileIO; use crate::execution::operators::ExecutionError; @@ -117,14 +120,14 @@ impl ExecutionPlan for IcebergScanExec { fn execute( &self, partition: usize, - _context: Arc, + context: Arc, ) -> DFResult { // Execute pre-planned tasks from Scala (planning happens via Iceberg's Java API) if let Some(ref task_groups) = self.file_task_groups { if partition < task_groups.len() { let tasks = &task_groups[partition]; - return self.execute_with_tasks(tasks.clone()); + return self.execute_with_tasks(tasks.clone(), context); } else { return Err(DataFusionError::Execution(format!( "IcebergScanExec: Partition index {} out of range (only {} task groups available)", @@ -148,38 +151,26 @@ impl IcebergScanExec { fn execute_with_tasks( &self, tasks: Vec, + context: Arc, ) -> DFResult { let output_schema = Arc::clone(&self.output_schema); - let catalog_properties = self.catalog_properties.clone(); - let metadata_location = self.metadata_location.clone(); - let fut = async move { - let file_io = Self::load_file_io(&catalog_properties, &metadata_location)?; + // Create FileIO synchronously + let file_io = Self::load_file_io(&self.catalog_properties, &self.metadata_location)?; - let task_stream = futures::stream::iter(tasks.into_iter().map(Ok)).boxed(); + // Get batch size from context + let batch_size = context.session_config().batch_size(); - let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io).build(); + // 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))?; - // read() is synchronous and returns Result - let stream = reader.read(task_stream).map_err(|e| { - DataFusionError::Execution(format!("Failed to read Iceberg tasks: {}", e)) - })?; - - let mapped_stream = stream - .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))); - - Ok::<_, DataFusionError>(Box::pin(mapped_stream) - as Pin< - Box> + Send>, - >) - }; + // 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. - let stream = futures::stream::once(fut).try_flatten(); - - Ok(Box::pin(RecordBatchStreamAdapter::new( - output_schema, - stream, - ))) + Ok(Box::pin(file_stream)) } fn load_file_io( @@ -199,6 +190,194 @@ impl IcebergScanExec { } } +/// 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, +} + +impl IcebergFileStream { + fn new( + tasks: Vec, + file_io: FileIO, + batch_size: usize, + schema: SchemaRef, + ) -> DFResult { + Ok(Self { + schema, + file_io, + batch_size, + tasks: tasks.into_iter().collect(), + state: FileStreamState::Idle, + }) + } + + /// Start opening the next file + fn start_next_file( + &mut self, + ) -> Option>> { + let task = self.tasks.pop_front()?; + let file_io = self.file_io.clone(); + let batch_size = self.batch_size; + let schema = Arc::clone(&self.schema); + + Some(Box::pin(async move { + // Create a single-task stream + let task_stream = futures::stream::iter(vec![Ok(task)]).boxed(); + + // Create reader with optimizations + let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io) + .with_batch_size(batch_size) + .with_row_selection_enabled(true) + .build(); + + // Read the task + let stream = reader.read(task_stream).map_err(|e| { + DataFusionError::Execution(format!("Failed to read Iceberg task: {}", e)) + })?; + + // Map errors and wrap minimally - RecordBatchStreamAdapter is needed to provide schema + let mapped_stream = stream + .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))); + + Ok( + Box::pin(RecordBatchStreamAdapter::new(schema, mapped_stream)) + as SendableRecordBatchStream, + ) + })) + } + + fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll>> { + loop { + match &mut self.state { + FileStreamState::Idle => { + // Start opening the first file + match self.start_next_file() { + Some(future) => { + self.state = FileStreamState::Opening { future }; + } + None => return Poll::Ready(None), + } + } + FileStreamState::Opening { future } => { + // Wait for file to open + match ready!(future.poll_unpin(cx)) { + Ok(stream) => { + // File opened, start reading and open next file in parallel + 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) { + // Next file is ready, store it + 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))); + } + } + } + } + + // Poll current stream for next batch + match ready!(current.poll_next_unpin(cx)) { + Some(result) => { + return Poll::Ready(Some(result)); + } + None => { + // Current file is done, move to next file if available + match next.take() { + Some(mut next_future) => { + // Check if next file is already opened + match next_future.poll_unpin(cx) { + Poll::Ready(Ok(stream)) => { + 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 => { + // Still opening, wait for it + self.state = FileStreamState::Opening { + future: next_future, + }; + } + } + } + None => { + // No more files + 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.poll_inner(cx) + } +} + +impl RecordBatchStream for IcebergFileStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + impl DisplayAs for IcebergScanExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { write!( diff --git a/spark/pom.xml b/spark/pom.xml index d74a71c377..d39f56cf31 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -185,7 +185,7 @@ under the License. org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} - 1.5.2 + 1.8.1 test @@ -200,7 +200,7 @@ under the License. org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} - 1.5.2 + 1.8.1 test 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..d819fad017 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 = @@ -631,78 +672,85 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { } override def runCometBenchmark(mainArgs: Array[String]): Unit = { - runBenchmarkWithTable("Parquet Reader", 1024 * 1024 * 15) { v => - Seq( - BooleanType, - ByteType, - ShortType, - IntegerType, - LongType, - FloatType, - DoubleType, - StringType).foreach { dataType => - readerBenchmark(v, dataType) - } - } - - runBenchmarkWithTable("SQL Single Numeric Column Scan", 1024 * 1024 * 128) { v => - Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) - .foreach { dataType => - numericScanBenchmark(v, dataType) - } - } - - runBenchmarkWithTable("SQL Single Numeric Encrypted Column Scan", 1024 * 1024 * 128) { v => +// runBenchmarkWithTable("Parquet Reader", 1024 * 1024 * 15) { v => +// Seq( +// BooleanType, +// ByteType, +// ShortType, +// IntegerType, +// LongType, +// FloatType, +// DoubleType, +// StringType).foreach { dataType => +// readerBenchmark(v, dataType) +// } +// } +// +// runBenchmarkWithTable("SQL Single Numeric Column Scan", 1024 * 1024 * 128) { v => +// Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) +// .foreach { dataType => +// numericScanBenchmark(v, dataType) +// } +// } + + runBenchmarkWithTable("SQL Single Numeric Iceberg Column Scan", 1024 * 1024 * 128) { v => Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) .foreach { dataType => - encryptedScanBenchmark(v, dataType) + icebergScanBenchmark(v, dataType) } } - runBenchmark("SQL Decimal Column Scan") { - withTempTable(tbl) { - import spark.implicits._ - spark.range(1024 * 1024 * 15).map(_ => Random.nextInt).createOrReplaceTempView(tbl) - - Seq((5, 2), (18, 4), (20, 8)).foreach { case (precision, scale) => - decimalScanBenchmark(1024 * 1024 * 15, precision, scale) - } - } - } - - runBenchmarkWithTable("String Scan with Dictionary", 1024 * 1024 * 15) { v => - stringWithDictionaryScanBenchmark(v) - } - - runBenchmarkWithTable("Numeric Filter Scan", 1024 * 1024 * 10) { v => - for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { - numericFilterScanBenchmark(v, fractionOfZeros) - } - } - - runBenchmarkWithTable("String with Nulls Scan", 1024 * 1024 * 10) { v => - for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { - stringWithNullsScanBenchmark(v, fractionOfNulls) - } - } - - runBenchmarkWithTable("Single Column Scan From Wide Columns", 1024 * 1024 * 1) { v => - for (columnWidth <- List(10, 50, 100)) { - columnsBenchmark(v, columnWidth) - } - } - - runBenchmarkWithTable("Large String Filter Scan", 1024 * 1024) { v => - for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { - largeStringFilterScanBenchmark(v, fractionOfZeros) - } - } - - runBenchmarkWithTable("Sorted Lg Str Filter Scan", 1024 * 1024) { v => - for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { - sortedLgStrFilterScanBenchmark(v, fractionOfZeros) - } - } +// runBenchmarkWithTable("SQL Single Numeric Encrypted Column Scan", 1024 * 1024 * 128) { v => +// Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) +// .foreach { dataType => +// encryptedScanBenchmark(v, dataType) +// } +// } +// +// runBenchmark("SQL Decimal Column Scan") { +// withTempTable(tbl) { +// import spark.implicits._ +// spark.range(1024 * 1024 * 15).map(_ => Random.nextInt).createOrReplaceTempView(tbl) +// +// Seq((5, 2), (18, 4), (20, 8)).foreach { case (precision, scale) => +// decimalScanBenchmark(1024 * 1024 * 15, precision, scale) +// } +// } +// } +// +// runBenchmarkWithTable("String Scan with Dictionary", 1024 * 1024 * 15) { v => +// stringWithDictionaryScanBenchmark(v) +// } +// +// runBenchmarkWithTable("Numeric Filter Scan", 1024 * 1024 * 10) { v => +// for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { +// numericFilterScanBenchmark(v, fractionOfZeros) +// } +// } +// +// runBenchmarkWithTable("String with Nulls Scan", 1024 * 1024 * 10) { v => +// for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { +// stringWithNullsScanBenchmark(v, fractionOfNulls) +// } +// } +// +// runBenchmarkWithTable("Single Column Scan From Wide Columns", 1024 * 1024 * 1) { v => +// for (columnWidth <- List(10, 50, 100)) { +// columnsBenchmark(v, columnWidth) +// } +// } +// +// runBenchmarkWithTable("Large String Filter Scan", 1024 * 1024) { v => +// for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { +// largeStringFilterScanBenchmark(v, fractionOfZeros) +// } +// } +// +// runBenchmarkWithTable("Sorted Lg Str Filter Scan", 1024 * 1024) { v => +// for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { +// sortedLgStrFilterScanBenchmark(v, fractionOfZeros) +// } +// } } } From 880599ed63c5dc0f76d89641955fed74f4733d3c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 8 Oct 2025 08:54:24 -0400 Subject: [PATCH 17/89] Fix CometReadBenchmark. --- .../sql/benchmark/CometReadBenchmark.scala | 142 +++++++++--------- 1 file changed, 71 insertions(+), 71 deletions(-) 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 d819fad017..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 @@ -672,26 +672,26 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { } override def runCometBenchmark(mainArgs: Array[String]): Unit = { -// runBenchmarkWithTable("Parquet Reader", 1024 * 1024 * 15) { v => -// Seq( -// BooleanType, -// ByteType, -// ShortType, -// IntegerType, -// LongType, -// FloatType, -// DoubleType, -// StringType).foreach { dataType => -// readerBenchmark(v, dataType) -// } -// } -// -// runBenchmarkWithTable("SQL Single Numeric Column Scan", 1024 * 1024 * 128) { v => -// Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) -// .foreach { dataType => -// numericScanBenchmark(v, dataType) -// } -// } + runBenchmarkWithTable("Parquet Reader", 1024 * 1024 * 15) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType).foreach { dataType => + readerBenchmark(v, dataType) + } + } + + runBenchmarkWithTable("SQL Single Numeric Column Scan", 1024 * 1024 * 128) { v => + Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + .foreach { dataType => + numericScanBenchmark(v, dataType) + } + } runBenchmarkWithTable("SQL Single Numeric Iceberg Column Scan", 1024 * 1024 * 128) { v => Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) @@ -700,57 +700,57 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { } } -// runBenchmarkWithTable("SQL Single Numeric Encrypted Column Scan", 1024 * 1024 * 128) { v => -// Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) -// .foreach { dataType => -// encryptedScanBenchmark(v, dataType) -// } -// } -// -// runBenchmark("SQL Decimal Column Scan") { -// withTempTable(tbl) { -// import spark.implicits._ -// spark.range(1024 * 1024 * 15).map(_ => Random.nextInt).createOrReplaceTempView(tbl) -// -// Seq((5, 2), (18, 4), (20, 8)).foreach { case (precision, scale) => -// decimalScanBenchmark(1024 * 1024 * 15, precision, scale) -// } -// } -// } -// -// runBenchmarkWithTable("String Scan with Dictionary", 1024 * 1024 * 15) { v => -// stringWithDictionaryScanBenchmark(v) -// } -// -// runBenchmarkWithTable("Numeric Filter Scan", 1024 * 1024 * 10) { v => -// for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { -// numericFilterScanBenchmark(v, fractionOfZeros) -// } -// } -// -// runBenchmarkWithTable("String with Nulls Scan", 1024 * 1024 * 10) { v => -// for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { -// stringWithNullsScanBenchmark(v, fractionOfNulls) -// } -// } -// -// runBenchmarkWithTable("Single Column Scan From Wide Columns", 1024 * 1024 * 1) { v => -// for (columnWidth <- List(10, 50, 100)) { -// columnsBenchmark(v, columnWidth) -// } -// } -// -// runBenchmarkWithTable("Large String Filter Scan", 1024 * 1024) { v => -// for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { -// largeStringFilterScanBenchmark(v, fractionOfZeros) -// } -// } -// -// runBenchmarkWithTable("Sorted Lg Str Filter Scan", 1024 * 1024) { v => -// for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { -// sortedLgStrFilterScanBenchmark(v, fractionOfZeros) -// } -// } + runBenchmarkWithTable("SQL Single Numeric Encrypted Column Scan", 1024 * 1024 * 128) { v => + Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + .foreach { dataType => + encryptedScanBenchmark(v, dataType) + } + } + + runBenchmark("SQL Decimal Column Scan") { + withTempTable(tbl) { + import spark.implicits._ + spark.range(1024 * 1024 * 15).map(_ => Random.nextInt).createOrReplaceTempView(tbl) + + Seq((5, 2), (18, 4), (20, 8)).foreach { case (precision, scale) => + decimalScanBenchmark(1024 * 1024 * 15, precision, scale) + } + } + } + + runBenchmarkWithTable("String Scan with Dictionary", 1024 * 1024 * 15) { v => + stringWithDictionaryScanBenchmark(v) + } + + runBenchmarkWithTable("Numeric Filter Scan", 1024 * 1024 * 10) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { + numericFilterScanBenchmark(v, fractionOfZeros) + } + } + + runBenchmarkWithTable("String with Nulls Scan", 1024 * 1024 * 10) { v => + for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { + stringWithNullsScanBenchmark(v, fractionOfNulls) + } + } + + runBenchmarkWithTable("Single Column Scan From Wide Columns", 1024 * 1024 * 1) { v => + for (columnWidth <- List(10, 50, 100)) { + columnsBenchmark(v, columnWidth) + } + } + + runBenchmarkWithTable("Large String Filter Scan", 1024 * 1024) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { + largeStringFilterScanBenchmark(v, fractionOfZeros) + } + } + + runBenchmarkWithTable("Sorted Lg Str Filter Scan", 1024 * 1024) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { + sortedLgStrFilterScanBenchmark(v, fractionOfZeros) + } + } } } From 878c971182bbbbb59d256dfbe3869f0513de24ea Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 16 Oct 2025 16:13:07 -0400 Subject: [PATCH 18/89] Fixes after bringing in upstream/main. --- .../scala/org/apache/comet/CometConf.scala | 1 + docs/source/user-guide/latest/configs.md | 1 + native/Cargo.lock | 84 ++++++++++--------- 3 files changed, 48 insertions(+), 38 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index e068c854d8..6fa44dfe99 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -125,6 +125,7 @@ object CometConf extends ShimCometConf { val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.icebergNative.enabled") + .category(CATEGORY_SCAN) .doc( "Whether to enable native Iceberg scan using iceberg-rust. When enabled, Comet will " + "replace Spark's Iceberg BatchScanExec with CometIcebergNativeScanExec. Iceberg " + diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index a299a75738..1f4dfa6b27 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -32,6 +32,7 @@ Comet provides the following configuration settings. | `spark.comet.convert.parquet.enabled` | When enabled, data from Spark (non-native) Parquet v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and `spark.comet.exec.enabled` need to be enabled. | false | | `spark.comet.scan.allowIncompatible` | Some Comet scan implementations are not currently fully compatible with Spark for all datatypes. Set this config to true to allow them anyway. For more information, refer to the [Comet Compatibility Guide](https://datafusion.apache.org/comet/user-guide/compatibility.html). | false | | `spark.comet.scan.enabled` | Whether to enable native scans. When this is turned on, Spark will use Comet to read supported data sources (currently only Parquet is supported natively). Note that to enable native vectorized execution, both this config and `spark.comet.exec.enabled` need to be enabled. | true | +| `spark.comet.scan.icebergNative.enabled` | Whether to enable native Iceberg scan using iceberg-rust. When enabled, Comet will replace Spark's Iceberg BatchScanExec with CometIcebergNativeScanExec. Iceberg planning is performed by Spark, and the resulting FileScanTasks are serialized and passed to the native execution layer for reading data files. | false | | `spark.comet.scan.preFetch.enabled` | Whether to enable pre-fetching feature of CometScan. | false | | `spark.comet.scan.preFetch.threadNum` | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | | `spark.comet.sparkToColumnar.enabled` | Whether to enable Spark to Arrow columnar conversion. When this is turned on, Comet will convert operators in `spark.comet.sparkToColumnar.supportedOperatorList` into Arrow columnar format before processing. | false | diff --git a/native/Cargo.lock b/native/Cargo.lock index 5cc34ea711..b130034fd4 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -584,16 +584,15 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.32.2" +version = "0.32.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2b715a6010afb9e457ca2b7c9d2b9c344baa8baed7b38dc476034c171b32575" +checksum = "107a4e9d9cab9963e04e84bb8dee0e25f2a987f9a8bad5ed054abd439caa8f8c" dependencies = [ "bindgen 0.72.1", "cc", "cmake", "dunce", "fs_extra", - "libloading 0.8.8", ] [[package]] @@ -1053,9 +1052,9 @@ dependencies = [ [[package]] name = "bon" -version = "3.7.2" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2529c31017402be841eb45892278a6c21a000c0a17643af326c73a73f83f0fb" +checksum = "ebeb9aaf9329dff6ceb65c689ca3db33dbf15f324909c60e4e5eef5701ce31b1" dependencies = [ "bon-macros", "rustversion", @@ -1063,9 +1062,9 @@ dependencies = [ [[package]] name = "bon-macros" -version = "3.7.2" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d82020dadcb845a345591863adb65d74fa8dc5c18a0b6d408470e13b7adc7005" +checksum = "77e9d642a7e3a318e37c2c9427b5a6a48aa1ad55dcd986f3034ab2239045a645" dependencies = [ "darling 0.21.3", "ident_case", @@ -1211,9 +1210,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fd1289c04a9ea8cb22300a459a72a385d7c73d3259e2ed7dcb2af674838cfa9" +checksum = "9330f8b2ff13f34540b44e946ef35111825727b38d33286ef986142615121801" [[package]] name = "cfg_aliases" @@ -1232,7 +1231,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -1280,7 +1279,7 @@ checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" dependencies = [ "glob", "libc", - "libloading 0.8.8", + "libloading 0.8.9", ] [[package]] @@ -2403,7 +2402,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" dependencies = [ "powerfmt", - "serde", + "serde_core", ] [[package]] @@ -3152,7 +3151,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#c85788fca732a846bf38093f1c5f3abded7c6900" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#d822f5efee8a56ed8389c3602472d240205b4a0a" dependencies = [ "anyhow", "apache-avro", @@ -3564,12 +3563,12 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" +checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" dependencies = [ "cfg-if", - "windows-targets 0.53.5", + "windows-link", ] [[package]] @@ -4500,6 +4499,16 @@ dependencies = [ "memchr", ] +[[package]] +name = "quick-xml" +version = "0.37.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quick-xml" version = "0.38.3" @@ -4884,9 +4893,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.38.0" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8975fc98059f365204d635119cf9c5a60ae67b841ed49b5422a9a7e56cdfac0" +checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" dependencies = [ "arrayvec", "borsh", @@ -4968,9 +4977,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcff2dd52b58a8d98a70243663a0d234c4e2b79235637849d15913394a247d3" +checksum = "9980d917ebb0c0536119ba501e90834767bffc3d60641457fd84a1f3fd337923" dependencies = [ "openssl-probe", "rustls-pki-types", @@ -5198,9 +5207,9 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.14.1" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c522100790450cf78eeac1507263d0a350d4d5b30df0c8e1fe051a10c22b376e" +checksum = "6093cd8c01b25262b84927e0f7151692158fab02d961e04c979d3903eba7ecc5" dependencies = [ "base64", "chrono", @@ -5209,8 +5218,7 @@ dependencies = [ "indexmap 2.11.4", "schemars 0.9.0", "schemars 1.0.4", - "serde", - "serde_derive", + "serde_core", "serde_json", "serde_with_macros", "time", @@ -5218,9 +5226,9 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.14.1" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327ada00f7d64abaac1e55a6911e90cf665aa051b9a561c7006c157f4633135e" +checksum = "a7e6c180db0816026a61afa1cff5344fb7ebded7e4d3062772179f2501481c27" dependencies = [ "darling 0.21.3", "proc-macro2", @@ -5582,9 +5590,9 @@ dependencies = [ [[package]] name = "tikv-jemalloc-ctl" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f21f216790c8df74ce3ab25b534e0718da5a1916719771d3fec23315c99e468b" +checksum = "661f1f6a57b3a36dc9174a2c10f19513b4866816e13425d3e418b11cc37bc24c" dependencies = [ "libc", "paste", @@ -5593,9 +5601,9 @@ dependencies = [ [[package]] name = "tikv-jemalloc-sys" -version = "0.6.0+5.3.0-1-ge13ca993e8ccb9ba9847cc330696e02839f328f7" +version = "0.6.1+5.3.0-1-ge13ca993e8ccb9ba9847cc330696e02839f328f7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd3c60906412afa9c2b5b5a48ca6a5abe5736aec9eb48ad05037a677e52e4e2d" +checksum = "cd8aa5b2ab86a2cefa406d889139c162cbb230092f7d1d7cbc1716405d852a3b" dependencies = [ "cc", "libc", @@ -5603,9 +5611,9 @@ dependencies = [ [[package]] name = "tikv-jemallocator" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cec5ff18518d81584f477e9bfdf957f5bb0979b0bac3af4ca30b5b3ae2d2865" +checksum = "0359b4327f954e0567e69fb191cf1436617748813819c94b8cd4a431422d053a" dependencies = [ "libc", "tikv-jemalloc-sys", @@ -5739,18 +5747,18 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32f1085dec27c2b6632b04c80b3bb1b4300d6495d1e129693bdda7d91e72eec1" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" dependencies = [ "serde_core", ] [[package]] name = "toml_edit" -version = "0.23.6" +version = "0.23.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3effe7c0e86fdff4f69cdd2ccc1b96f933e24811c5441d44904e8683e27184b" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" dependencies = [ "indexmap 2.11.4", "toml_datetime", @@ -5760,9 +5768,9 @@ dependencies = [ [[package]] name = "toml_parser" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cf893c33be71572e0e9aa6dd15e6677937abd686b066eac3f8cd3531688a627" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" dependencies = [ "winnow", ] From e66799e2cb81baf26f242d27d8d9a0003b221e7a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 16 Oct 2025 17:18:39 -0400 Subject: [PATCH 19/89] Basic complex type support. --- .../src/execution/operators/iceberg_scan.rs | 75 +++++++++++++-- .../apache/comet/rules/CometScanRule.scala | 3 +- .../apache/comet/serde/QueryPlanSerde.scala | 1 + .../comet/CometIcebergNativeSuite.scala | 96 +++++++++++++++++++ 4 files changed, 166 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 5f7ebf2b06..f1e50839a1 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -30,7 +30,6 @@ use datafusion::common::{DataFusionError, Result as DFResult}; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }; @@ -39,6 +38,10 @@ use futures::{ready, FutureExt, Stream, StreamExt, TryStreamExt}; use iceberg::io::FileIO; use crate::execution::operators::ExecutionError; +use crate::parquet::parquet_support::SparkParquetOptions; +use crate::parquet::schema_adapter::SparkSchemaAdapterFactory; +use datafusion::datasource::schema_adapter::SchemaAdapterFactory; +use datafusion_comet_spark_expr::EvalMode; /// Native Iceberg scan operator that uses iceberg-rust to read Iceberg tables. /// @@ -65,6 +68,7 @@ impl IcebergScanExec { file_task_groups: Option>>, num_partitions: usize, ) -> Result { + // Don't normalize - just use the schema as provided by Spark let output_schema = schema; let plan_properties = Self::compute_properties(Arc::clone(&output_schema), num_partitions); @@ -261,14 +265,41 @@ impl IcebergFileStream { DataFusionError::Execution(format!("Failed to read Iceberg task: {}", e)) })?; - // Map errors and wrap minimally - RecordBatchStreamAdapter is needed to provide schema - let mapped_stream = stream - .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))); + // Clone schema for transformation + let target_schema = Arc::clone(&schema); - Ok( - Box::pin(RecordBatchStreamAdapter::new(schema, mapped_stream)) - as SendableRecordBatchStream, - ) + // Apply schema adaptation to each batch (same approach as regular Parquet scans) + // This handles differences in field names ("element" vs "item", "key_value" vs "entries") + // and metadata (PARQUET:field_id) just like regular Parquet scans + let mapped_stream = stream + .map_err(|e| DataFusionError::Execution(format!("Iceberg scan error: {}", e))) + .and_then(move |batch| { + // Use SparkSchemaAdapter to transform the 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)); + + // Apply the schema mapping + 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) })) } @@ -378,6 +409,34 @@ impl RecordBatchStream for IcebergFileStream { } } +/// Wrapper around iceberg-rust's stream that reports Comet's schema without validation. +/// This avoids strict schema checks that would reject batches with PARQUET:field_id metadata. +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 { + // Return Comet's schema, not the batch schema with metadata + Arc::clone(&self.schema) + } +} + impl DisplayAs for IcebergScanExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { write!( 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 cdea911342..2ec1d94425 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -271,8 +271,9 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com "org.apache.iceberg.spark.source.SparkBatchQueryScan" => val fallbackReasons = new ListBuffer[String]() + 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 " + 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 4bb336771b..c3cc8f6cae 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1133,6 +1133,7 @@ object QueryPlanSerde extends Logging with CometExprShim { val field = SparkStructField .newBuilder() .setName(attr.name) + .setNullable(attr.nullable) serializeDataType(attr.dataType).foreach(field.setDataType) icebergScanBuilder.addRequiredSchema(field.build()) } diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 3ca01dc658..7aaa129e1c 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -884,6 +884,102 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From 4f2f3b88baa3413ce39cb0b31cb728593ec8a56a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Oct 2025 17:58:12 -0400 Subject: [PATCH 20/89] CometFuzzIceberg stuff. --- .../comet/testing/FuzzDataGenerator.scala | 257 ++++++++++++++++++ .../comet/testing/ParquetGenerator.scala | 210 +------------- .../apache/comet/CometFuzzIcebergBase.scala | 130 +++++++++ .../apache/comet/CometFuzzIcebergSuite.scala | 227 ++++++++++++++++ 4 files changed, 616 insertions(+), 208 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/testing/FuzzDataGenerator.scala create mode 100644 spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala create mode 100644 spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala diff --git a/spark/src/main/scala/org/apache/comet/testing/FuzzDataGenerator.scala b/spark/src/main/scala/org/apache/comet/testing/FuzzDataGenerator.scala new file mode 100644 index 0000000000..c6239a9a13 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/testing/FuzzDataGenerator.scala @@ -0,0 +1,257 @@ +/* + * 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.testing + +import java.math.{BigDecimal, RoundingMode} +import java.nio.charset.Charset +import java.sql.Timestamp +import java.text.SimpleDateFormat +import java.time.{Instant, LocalDateTime, ZoneId} + +import scala.collection.mutable.ListBuffer +import scala.util.Random + +import org.apache.commons.lang3.RandomStringUtils +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types._ + +object FuzzDataGenerator { + + /** + * Date to use as base for generating temporal columns. Random integers will be added to or + * subtracted from this value. + * + * Date was chosen to trigger generating a timestamp that's larger than a 64-bit nanosecond + * timestamp can represent so that we test support for INT96 timestamps. + */ + val defaultBaseDate: Long = + new SimpleDateFormat("YYYY-MM-DD hh:mm:ss").parse("3333-05-25 12:34:56").getTime + + private val primitiveTypes = Seq( + DataTypes.BooleanType, + DataTypes.ByteType, + DataTypes.ShortType, + DataTypes.IntegerType, + DataTypes.LongType, + DataTypes.FloatType, + DataTypes.DoubleType, + DataTypes.createDecimalType(10, 2), + DataTypes.createDecimalType(36, 18), + DataTypes.DateType, + DataTypes.TimestampType, + DataTypes.TimestampNTZType, + DataTypes.StringType, + DataTypes.BinaryType) + + private def filteredPrimitives(excludeTypes: Seq[DataType]) = { + + primitiveTypes.filterNot { dataType => + excludeTypes.exists { + case _: DecimalType => + // For DecimalType, match if the type is also a DecimalType (ignore precision/scale) + dataType.isInstanceOf[DecimalType] + case excludeType => + dataType == excludeType + } + } + } + + def generateDataFrame( + r: Random, + spark: SparkSession, + numRows: Int, + options: DataGenOptions): DataFrame = { + + val filteredPrimitiveTypes = filteredPrimitives(options.excludeTypes) + val dataTypes = ListBuffer[DataType]() + dataTypes.appendAll(filteredPrimitiveTypes) + + val arraysOfPrimitives = filteredPrimitiveTypes.map(DataTypes.createArrayType) + + if (options.generateStruct) { + dataTypes += StructType(filteredPrimitiveTypes.zipWithIndex.map(x => + StructField(s"c${x._2}", x._1, nullable = true))) + + if (options.generateArray) { + dataTypes += StructType(arraysOfPrimitives.zipWithIndex.map(x => + StructField(s"c${x._2}", x._1, nullable = true))) + } + } + + if (options.generateMap) { + dataTypes += MapType(DataTypes.IntegerType, DataTypes.StringType) + } + + if (options.generateArray) { + dataTypes.appendAll(arraysOfPrimitives) + + if (options.generateStruct) { + dataTypes += DataTypes.createArrayType( + StructType(filteredPrimitiveTypes.zipWithIndex.map(x => + StructField(s"c${x._2}", x._1, nullable = true)))) + } + + if (options.generateMap) { + dataTypes += DataTypes.createArrayType( + MapType(DataTypes.IntegerType, DataTypes.StringType)) + } + } + + // generate schema using random data types + val fields = dataTypes.zipWithIndex + .map(i => StructField(s"c${i._2}", i._1, nullable = true)) + val schema = StructType(fields) + + // generate columnar data + val cols: Seq[Seq[Any]] = + schema.fields.map(f => generateColumn(r, f.dataType, numRows, options)).toSeq + + // convert to rows + val rows = Range(0, numRows).map(rowIndex => { + Row.fromSeq(cols.map(_(rowIndex))) + }) + + spark.createDataFrame(spark.sparkContext.parallelize(rows), schema) + } + + private def generateColumn( + r: Random, + dataType: DataType, + numRows: Int, + options: DataGenOptions): Seq[Any] = { + dataType match { + case ArrayType(elementType, _) => + val values = generateColumn(r, elementType, numRows, options) + val list = ListBuffer[Any]() + for (i <- 0 until numRows) { + if (i % 10 == 0 && options.allowNull) { + list += null + } else { + list += Range(0, r.nextInt(5)).map(j => values((i + j) % values.length)).toArray + } + } + list + case StructType(fields) => + val values = fields.map(f => generateColumn(r, f.dataType, numRows, options)) + Range(0, numRows).map(i => Row(values.indices.map(j => values(j)(i)): _*)) + case MapType(keyType, valueType, _) => + val mapOptions = options.copy(allowNull = false) + val k = generateColumn(r, keyType, numRows, mapOptions) + val v = generateColumn(r, valueType, numRows, mapOptions) + k.zip(v).map(x => Map(x._1 -> x._2)) + case DataTypes.BooleanType => + generateColumn(r, DataTypes.LongType, numRows, options) + .map(_.asInstanceOf[Long].toShort) + .map(s => s % 2 == 0) + case DataTypes.ByteType => + generateColumn(r, DataTypes.LongType, numRows, options) + .map(_.asInstanceOf[Long].toByte) + case DataTypes.ShortType => + generateColumn(r, DataTypes.LongType, numRows, options) + .map(_.asInstanceOf[Long].toShort) + case DataTypes.IntegerType => + generateColumn(r, DataTypes.LongType, numRows, options) + .map(_.asInstanceOf[Long].toInt) + case DataTypes.LongType => + Range(0, numRows).map(_ => { + r.nextInt(50) match { + case 0 if options.allowNull => null + case 1 => 0L + case 2 => Byte.MinValue.toLong + case 3 => Byte.MaxValue.toLong + case 4 => Short.MinValue.toLong + case 5 => Short.MaxValue.toLong + case 6 => Int.MinValue.toLong + case 7 => Int.MaxValue.toLong + case 8 => Long.MinValue + case 9 => Long.MaxValue + case _ => r.nextLong() + } + }) + case DataTypes.FloatType => + Range(0, numRows).map(_ => { + r.nextInt(20) match { + case 0 if options.allowNull => null + case 1 => Float.NegativeInfinity + case 2 => Float.PositiveInfinity + case 3 => Float.MinValue + case 4 => Float.MaxValue + case 5 => 0.0f + case 6 if options.generateNegativeZero => -0.0f + case _ => r.nextFloat() + } + }) + case DataTypes.DoubleType => + Range(0, numRows).map(_ => { + r.nextInt(20) match { + case 0 if options.allowNull => null + case 1 => Double.NegativeInfinity + case 2 => Double.PositiveInfinity + case 3 => Double.MinValue + case 4 => Double.MaxValue + case 5 => 0.0 + case 6 if options.generateNegativeZero => -0.0 + case _ => r.nextDouble() + } + }) + case dt: DecimalType => + Range(0, numRows).map(_ => + new BigDecimal(r.nextDouble()).setScale(dt.scale, RoundingMode.HALF_UP)) + case DataTypes.StringType => + Range(0, numRows).map(_ => { + r.nextInt(10) match { + case 0 if options.allowNull => null + case 1 => r.nextInt().toByte.toString + case 2 => r.nextLong().toString + case 3 => r.nextDouble().toString + case 4 => RandomStringUtils.randomAlphabetic(8) + case _ => r.nextString(8) + } + }) + case DataTypes.BinaryType => + generateColumn(r, DataTypes.StringType, numRows, options) + .map { + case x: String => + x.getBytes(Charset.defaultCharset()) + case _ => + null + } + case DataTypes.DateType => + Range(0, numRows).map(_ => new java.sql.Date(options.baseDate + r.nextInt())) + case DataTypes.TimestampType => + Range(0, numRows).map(_ => new Timestamp(options.baseDate + r.nextInt())) + case DataTypes.TimestampNTZType => + Range(0, numRows).map(_ => + LocalDateTime.ofInstant( + Instant.ofEpochMilli(options.baseDate + r.nextInt()), + ZoneId.systemDefault())) + case _ => throw new IllegalStateException(s"Cannot generate data for $dataType yet") + } + } +} + +case class DataGenOptions( + allowNull: Boolean = true, + generateNegativeZero: Boolean = true, + baseDate: Long = FuzzDataGenerator.defaultBaseDate, + generateArray: Boolean = false, + generateStruct: Boolean = false, + generateMap: Boolean = false, + excludeTypes: Seq[DataType] = Seq.empty) diff --git a/spark/src/main/scala/org/apache/comet/testing/ParquetGenerator.scala b/spark/src/main/scala/org/apache/comet/testing/ParquetGenerator.scala index afae5b7876..27e40c9d74 100644 --- a/spark/src/main/scala/org/apache/comet/testing/ParquetGenerator.scala +++ b/spark/src/main/scala/org/apache/comet/testing/ParquetGenerator.scala @@ -19,47 +19,12 @@ package org.apache.comet.testing -import java.math.{BigDecimal, RoundingMode} -import java.nio.charset.Charset -import java.sql.Timestamp -import java.text.SimpleDateFormat -import java.time.{Instant, LocalDateTime, ZoneId} - -import scala.collection.mutable.ListBuffer import scala.util.Random -import org.apache.commons.lang3.RandomStringUtils -import org.apache.spark.sql.{Row, SaveMode, SparkSession} -import org.apache.spark.sql.types._ +import org.apache.spark.sql.{SaveMode, SparkSession} object ParquetGenerator { - /** - * Date to use as base for generating temporal columns. Random integers will be added to or - * subtracted from this value. - * - * Date was chosen to trigger generating a timestamp that's larger than a 64-bit nanosecond - * timestamp can represent so that we test support for INT96 timestamps. - */ - val defaultBaseDate: Long = - new SimpleDateFormat("YYYY-MM-DD hh:mm:ss").parse("3333-05-25 12:34:56").getTime - - private val primitiveTypes = Seq( - DataTypes.BooleanType, - DataTypes.ByteType, - DataTypes.ShortType, - DataTypes.IntegerType, - DataTypes.LongType, - DataTypes.FloatType, - DataTypes.DoubleType, - DataTypes.createDecimalType(10, 2), - DataTypes.createDecimalType(36, 18), - DataTypes.DateType, - DataTypes.TimestampType, - DataTypes.TimestampNTZType, - DataTypes.StringType, - DataTypes.BinaryType) - def makeParquetFile( r: Random, spark: SparkSession, @@ -67,179 +32,8 @@ object ParquetGenerator { numRows: Int, options: DataGenOptions): Unit = { - val dataTypes = ListBuffer[DataType]() - dataTypes.appendAll(primitiveTypes) - - val arraysOfPrimitives = primitiveTypes.map(DataTypes.createArrayType) - - if (options.generateStruct) { - dataTypes += StructType( - primitiveTypes.zipWithIndex.map(x => StructField(s"c${x._2}", x._1, true))) - - if (options.generateArray) { - dataTypes += StructType( - arraysOfPrimitives.zipWithIndex.map(x => StructField(s"c${x._2}", x._1, true))) - } - } - - if (options.generateMap) { - dataTypes += MapType(DataTypes.IntegerType, DataTypes.StringType) - } - - if (options.generateArray) { - dataTypes.appendAll(arraysOfPrimitives) - - if (options.generateStruct) { - dataTypes += DataTypes.createArrayType( - StructType(primitiveTypes.zipWithIndex.map(x => StructField(s"c${x._2}", x._1, true)))) - } - - if (options.generateMap) { - dataTypes += DataTypes.createArrayType( - MapType(DataTypes.IntegerType, DataTypes.StringType)) - } - } + val df = FuzzDataGenerator.generateDataFrame(r, spark, numRows, options) - // generate schema using random data types - val fields = dataTypes.zipWithIndex - .map(i => StructField(s"c${i._2}", i._1, nullable = true)) - .toSeq - val schema = StructType(fields) - - // generate columnar data - val cols: Seq[Seq[Any]] = - fields.map(f => generateColumn(r, f.dataType, numRows, options)).toSeq - - // convert to rows - val rows = Range(0, numRows).map(rowIndex => { - Row.fromSeq(cols.map(_(rowIndex))) - }) - - val df = spark.createDataFrame(spark.sparkContext.parallelize(rows), schema) df.write.mode(SaveMode.Overwrite).parquet(filename) } - - def generateColumn( - r: Random, - dataType: DataType, - numRows: Int, - options: DataGenOptions): Seq[Any] = { - dataType match { - case ArrayType(elementType, _) => - val values = generateColumn(r, elementType, numRows, options) - val list = ListBuffer[Any]() - for (i <- 0 until numRows) { - if (i % 10 == 0 && options.allowNull) { - list += null - } else { - list += Range(0, r.nextInt(5)).map(j => values((i + j) % values.length)).toArray - } - } - list.toSeq - case StructType(fields) => - val values = fields.map(f => generateColumn(r, f.dataType, numRows, options)) - Range(0, numRows).map(i => Row(values.indices.map(j => values(j)(i)): _*)) - case MapType(keyType, valueType, _) => - val mapOptions = options.copy(allowNull = false) - val k = generateColumn(r, keyType, numRows, mapOptions) - val v = generateColumn(r, valueType, numRows, mapOptions) - k.zip(v).map(x => Map(x._1 -> x._2)) - case DataTypes.BooleanType => - generateColumn(r, DataTypes.LongType, numRows, options) - .map(_.asInstanceOf[Long].toShort) - .map(s => s % 2 == 0) - case DataTypes.ByteType => - generateColumn(r, DataTypes.LongType, numRows, options) - .map(_.asInstanceOf[Long].toByte) - case DataTypes.ShortType => - generateColumn(r, DataTypes.LongType, numRows, options) - .map(_.asInstanceOf[Long].toShort) - case DataTypes.IntegerType => - generateColumn(r, DataTypes.LongType, numRows, options) - .map(_.asInstanceOf[Long].toInt) - case DataTypes.LongType => - Range(0, numRows).map(_ => { - r.nextInt(50) match { - case 0 if options.allowNull => null - case 1 => 0L - case 2 => Byte.MinValue.toLong - case 3 => Byte.MaxValue.toLong - case 4 => Short.MinValue.toLong - case 5 => Short.MaxValue.toLong - case 6 => Int.MinValue.toLong - case 7 => Int.MaxValue.toLong - case 8 => Long.MinValue - case 9 => Long.MaxValue - case _ => r.nextLong() - } - }) - case DataTypes.FloatType => - Range(0, numRows).map(_ => { - r.nextInt(20) match { - case 0 if options.allowNull => null - case 1 => Float.NegativeInfinity - case 2 => Float.PositiveInfinity - case 3 => Float.MinValue - case 4 => Float.MaxValue - case 5 => 0.0f - case 6 if options.generateNegativeZero => -0.0f - case _ => r.nextFloat() - } - }) - case DataTypes.DoubleType => - Range(0, numRows).map(_ => { - r.nextInt(20) match { - case 0 if options.allowNull => null - case 1 => Double.NegativeInfinity - case 2 => Double.PositiveInfinity - case 3 => Double.MinValue - case 4 => Double.MaxValue - case 5 => 0.0 - case 6 if options.generateNegativeZero => -0.0 - case _ => r.nextDouble() - } - }) - case dt: DecimalType => - Range(0, numRows).map(_ => - new BigDecimal(r.nextDouble()).setScale(dt.scale, RoundingMode.HALF_UP)) - case DataTypes.StringType => - Range(0, numRows).map(_ => { - r.nextInt(10) match { - case 0 if options.allowNull => null - case 1 => r.nextInt().toByte.toString - case 2 => r.nextLong().toString - case 3 => r.nextDouble().toString - case 4 => RandomStringUtils.randomAlphabetic(8) - case _ => r.nextString(8) - } - }) - case DataTypes.BinaryType => - generateColumn(r, DataTypes.StringType, numRows, options) - .map { - case x: String => - x.getBytes(Charset.defaultCharset()) - case _ => - null - } - case DataTypes.DateType => - Range(0, numRows).map(_ => new java.sql.Date(options.baseDate + r.nextInt())) - case DataTypes.TimestampType => - Range(0, numRows).map(_ => new Timestamp(options.baseDate + r.nextInt())) - case DataTypes.TimestampNTZType => - Range(0, numRows).map(_ => - LocalDateTime.ofInstant( - Instant.ofEpochMilli(options.baseDate + r.nextInt()), - ZoneId.systemDefault())) - case _ => throw new IllegalStateException(s"Cannot generate data for $dataType yet") - } - } - } - -case class DataGenOptions( - allowNull: Boolean = true, - generateNegativeZero: Boolean = true, - baseDate: Long = ParquetGenerator.defaultBaseDate, - generateArray: Boolean = false, - generateStruct: Boolean = false, - generateMap: Boolean = false) 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..3b04fffa61 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala @@ -0,0 +1,130 @@ +/* + * 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} + +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 options = + DataGenOptions( + generateArray = true, + generateStruct = true, + generateNegativeZero = false, + // Disable decimals - iceberg-rust doesn't support FIXED_LEN_BYTE_ARRAY in page index yet + excludeTypes = Seq(DecimalType(10, 2)), + // override base date due to known issues with experimental scans + baseDate = + new SimpleDateFormat("YYYY-MM-DD hh:mm:ss").parse("2024-05-25 12:34:56").getTime) + + val df = FuzzDataGenerator.generateDataFrame(random, spark, 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..dbd5d0308d --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala @@ -0,0 +1,227 @@ +/* + * 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, ParquetGenerator} + +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 options = + DataGenOptions( + generateArray = generateArray, + generateStruct = generateStruct, + 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, 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 + } + } +} From 3371cc15be854533f6b405451339ff61ccff0343 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Oct 2025 11:28:53 -0400 Subject: [PATCH 21/89] format and fix conflicts. --- native/Cargo.lock | 2 +- .../apache/comet/CometFuzzIcebergBase.scala | 20 +++++++++---------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index dd72bd5769..b130034fd4 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -4126,7 +4126,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.11.4", "simdutf8", "uuid", ] diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala index 3b04fffa61..6563a21316 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala @@ -109,17 +109,17 @@ class CometFuzzIcebergBase extends CometTestBase with AdaptiveSparkPlanHelper { 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 - } + 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] = { From 1c40d43b29b59bc80effe8c68d89442e3beb712e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Oct 2025 13:04:53 -0400 Subject: [PATCH 22/89] Basic S3 test and properties support --- dev/ci/check-suites.py | 1 + .../apache/comet/serde/QueryPlanSerde.scala | 73 ++++++- .../comet/CometIcebergNativeScanExec.scala | 2 - .../org/apache/comet/CometS3TestBase.scala | 91 +++++++++ .../apache/comet/IcebergReadFromS3Suite.scala | 193 ++++++++++++++++++ .../parquet/ParquetReadFromS3Suite.scala | 78 +------ 6 files changed, 361 insertions(+), 77 deletions(-) create mode 100644 spark/src/test/scala/org/apache/comet/CometS3TestBase.scala create mode 100644 spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala 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/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index c3cc8f6cae..c5e482cd84 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -269,6 +269,57 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[VariancePop] -> CometVariancePop, classOf[VarianceSamp] -> CometVarianceSamp) + /** + * 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. + * + * @param hadoopProps + * Map of Hadoop configuration properties (fs.s3a.* keys) + * @return + * Map with keys transformed to Iceberg format (s3.* keys) + */ + private 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 // Ignore unrecognized per-bucket properties + } + } 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 + } + } + } + def supportedDataType(dt: DataType, allowComplex: Boolean = false): Boolean = dt match { case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | @@ -1113,8 +1164,26 @@ object QueryPlanSerde extends Logging with CometExprShim { // Set metadata location icebergScanBuilder.setMetadataLocation(metadataLocation) - // Serialize catalog properties (for authentication - currently empty) - // TODO: Extract credentials, S3 config, etc. + 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) + } // Determine number of partitions from Iceberg's output partitioning // TODO: Add a test case for both partitioning schemes diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 85a6d53949..b177a4e155 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -159,8 +159,6 @@ object CometIcebergNativeScanExec { /** * Extracts metadata location from Iceberg table. * - * TODO: Also extract catalog properties (credentials, S3 config, etc.) for authentication - * * @param scanExec * The Spark BatchScanExec containing an Iceberg scan * @return 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) From 40c9a07912d7990daca358b63080a3d928c22ba4 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Oct 2025 15:26:48 -0400 Subject: [PATCH 23/89] Fix NPE. --- .../main/scala/org/apache/comet/objectstore/NativeConfig.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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]() From 236b339d206296255f1ae2374e8d11b52e9c45d9 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Oct 2025 22:18:29 -0400 Subject: [PATCH 24/89] Support migrated tables via https://github.com/apache/iceberg-rust/pull/1777. --- docs/source/user-guide/latest/configs.md | 1 + native/Cargo.lock | 2 +- .../comet/CometIcebergNativeSuite.scala | 103 +++++++++++++++++- 3 files changed, 99 insertions(+), 7 deletions(-) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 58f2cc0c60..0987193078 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -269,6 +269,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.Reverse.enabled` | Enable Comet acceleration for `Reverse` | true | | `spark.comet.expression.Round.enabled` | Enable Comet acceleration for `Round` | true | | `spark.comet.expression.Second.enabled` | Enable Comet acceleration for `Second` | true | +| `spark.comet.expression.Sha1.enabled` | Enable Comet acceleration for `Sha1` | true | | `spark.comet.expression.Sha2.enabled` | Enable Comet acceleration for `Sha2` | true | | `spark.comet.expression.ShiftLeft.enabled` | Enable Comet acceleration for `ShiftLeft` | true | | `spark.comet.expression.ShiftRight.enabled` | Enable Comet acceleration for `ShiftRight` | true | diff --git a/native/Cargo.lock b/native/Cargo.lock index b130034fd4..079e0c8fbe 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3151,7 +3151,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#d822f5efee8a56ed8389c3602472d240205b4a0a" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#662cb58879145d796e5f0461a038da3739b6aeca" dependencies = [ "anyhow", "apache-avro", diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 7aaa129e1c..4df9abf72f 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -786,12 +786,7 @@ class CometIcebergNativeSuite extends CometTestBase { } } - // TODO: Re-enable when iceberg-rust supports schema evolution in projections - // Currently iceberg-rust errors when projecting columns that don't exist in old files. - // See: https://github.com/apache/iceberg-rust/blob/main/crates/iceberg/src/arrow/reader.rs#L586-L601 - // The strict validation at line 586: `if column_map.len() != leaf_field_ids.len()` - // prevents reading new columns from evolved schemas as NULL values. - ignore("schema evolution - add column") { + test("schema evolution - add column") { assume(icebergAvailable, "Iceberg not available in classpath") withTempIcebergDir { warehouseDir => @@ -830,6 +825,102 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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("projection - column subset, reordering, and duplication") { assume(icebergAvailable, "Iceberg not available in classpath") From ce367cc5350dccbf0435f866f5fce1a6a600b6e0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 22 Oct 2025 07:23:16 -0400 Subject: [PATCH 25/89] Update df50 commit based on field ID fix. --- native/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 079e0c8fbe..abcbabb897 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3151,7 +3151,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#662cb58879145d796e5f0461a038da3739b6aeca" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#a28ed1d3cb5f1680d5eaa7df895f0260cc0c87a6" dependencies = [ "anyhow", "apache-avro", @@ -3334,7 +3334,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "equivalent", - "hashbrown 0.16.0", + "hashbrown 0.15.5", "serde", "serde_core", ] From bd6c60922225bc0193881b3f420c4ddbb476b041 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 22 Oct 2025 09:38:32 -0400 Subject: [PATCH 26/89] Bump df50 commit. --- native/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index abcbabb897..a3081d36ad 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3151,7 +3151,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#a28ed1d3cb5f1680d5eaa7df895f0260cc0c87a6" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#19381f662f0bab83ff8424b6f0a8a42908c4b88e" dependencies = [ "anyhow", "apache-avro", @@ -3334,7 +3334,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "serde", "serde_core", ] From 33fa891839475cd14eaaf646bb4a3724311a6ed1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 22 Oct 2025 13:10:52 -0400 Subject: [PATCH 27/89] Support hive-partitioned Parquet files migrated to Iceberg tables with default values. --- native/core/src/execution/planner.rs | 3 +- .../apache/comet/serde/QueryPlanSerde.scala | 151 +++++++++++++++++- .../comet/CometIcebergNativeSuite.scala | 54 +++++++ 3 files changed, 206 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index ca4daa6a0c..c4315a7232 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2778,11 +2778,12 @@ fn parse_file_scan_tasks( let results: Result, _> = proto_tasks .iter() .map(|proto_task| { - // Parse schema from JSON using iceberg's built-in parser + // Parse schema from JSON (already contains partition values injected on Scala side) 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); // Parse file format 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 e9380dbe7d..09589b3a8a 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1262,6 +1262,90 @@ object QueryPlanSerde extends Logging with CometExprShim { } taskBuilder.setDataFilePath(filePath) + // Extract partition values for Hive-style partitioning + // These values are needed to populate partition columns + // that don't exist in data files + 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 + // PartitionScanTask has a spec() method + // 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 + // Format: {"field_id": value, ...} + 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 source field ID from the partition spec + val partitionField = fields.get(i) + 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) @@ -1280,7 +1364,14 @@ object QueryPlanSerde extends Logging with CometExprShim { // scalastyle:on classforname val toJsonMethod = schemaParserClass.getMethod("toJson", schemaClass) toJsonMethod.setAccessible(true) - val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] + var schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] + + // Inject partition values into schema if present + partitionJsonOpt.foreach { partitionJson => + schemaJson = + injectPartitionValuesIntoSchemaJson(schemaJson, partitionJson) + } + taskBuilder.setSchemaJson(schemaJson) val columnsMethod = schema.getClass.getMethod("columns") @@ -2256,6 +2347,64 @@ object QueryPlanSerde extends Logging with CometExprShim { Literal(sparkValue, sparkType) } + + /** + * Injects partition values into Iceberg schema JSON as "initial-default" values. + * + * For Hive-style partitioned tables migrated to Iceberg, partition values are stored in + * directory structure, not in data files. This function adds those values to the schema so + * iceberg-rust's RecordBatchTransformer can populate partition columns. + * + * @param schemaJson + * The Iceberg schema as JSON string + * @param partitionJson + * The partition values as JSON string: {"field_id": value, ...} + * @return + * Modified schema JSON with initial-default values injected + */ + private def injectPartitionValuesIntoSchemaJson( + schemaJson: String, + partitionJson: String): String = { + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + + try { + // Parse both JSONs + implicit val formats: Formats = DefaultFormats + val schemaValue = parse(schemaJson) + val partitionMap = parse(partitionJson).extract[Map[String, JValue]] + + // Transform the schema fields to inject initial-default values + val transformedSchema = schemaValue.transformField { case ("fields", JArray(fields)) => + val updatedFields = fields.map { + case fieldObj: JObject => + // Check if this field has a partition value + fieldObj \ "id" match { + case JInt(fieldId) => + partitionMap.get(fieldId.toString) match { + case Some(partitionValue) => + // Add "initial-default" to this field + fieldObj merge JObject("initial-default" -> partitionValue) + case None => + fieldObj + } + case _ => + fieldObj + } + case other => other + } + ("fields", JArray(updatedFields)) + } + + // Serialize back to JSON + compact(render(transformedSchema)) + } catch { + case e: Exception => + logWarning(s"Failed to inject partition values into schema JSON: ${e.getMessage}") + // Return original schema on error + schemaJson + } + } } sealed trait SupportLevel diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 4df9abf72f..f7b5ac34b7 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -921,6 +921,60 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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") From ca13cc61cdf7496ec6ae7fc4fb6b6a875a0da044 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 22 Oct 2025 16:53:30 -0400 Subject: [PATCH 28/89] Bump df50. --- native/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index a3081d36ad..301fc18b31 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3151,7 +3151,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#19381f662f0bab83ff8424b6f0a8a42908c4b88e" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#e59f1b1ecb46e95531f8a490b5609ed400821a28" dependencies = [ "anyhow", "apache-avro", @@ -3334,7 +3334,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "equivalent", - "hashbrown 0.16.0", + "hashbrown 0.15.5", "serde", "serde_core", ] From e19e2016cc5b544ac4cbeb1b99255eab7ecc4c61 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 22 Oct 2025 17:31:30 -0400 Subject: [PATCH 29/89] Fix after merging main. --- .../apache/comet/CometFuzzIcebergBase.scala | 20 ++++++++++++------- .../apache/comet/CometFuzzIcebergSuite.scala | 16 ++++++++++----- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala index 6563a21316..3640a0361f 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergBase.scala @@ -35,7 +35,7 @@ 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} +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, SchemaGenOptions} class CometFuzzIcebergBase extends CometTestBase with AdaptiveSparkPlanHelper { @@ -72,18 +72,23 @@ class CometFuzzIcebergBase extends CometTestBase with AdaptiveSparkPlanHelper { "spark.sql.catalog.hadoop_catalog.warehouse" -> warehouseDir.getAbsolutePath, CometConf.COMET_ENABLED.key -> "false", SQLConf.SESSION_LOCAL_TIMEZONE.key -> defaultTimezone) { - val options = - DataGenOptions( + + 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, - // Disable decimals - iceberg-rust doesn't support FIXED_LEN_BYTE_ARRAY in page index yet - excludeTypes = Seq(DecimalType(10, 2)), - // override base date due to known issues with experimental scans baseDate = new SimpleDateFormat("YYYY-MM-DD hh:mm:ss").parse("2024-05-25 12:34:56").getTime) - val df = FuzzDataGenerator.generateDataFrame(random, spark, 1000, options) + val df = FuzzDataGenerator.generateDataFrame(random, spark, schema, 1000, options) df.writeTo(icebergTableName).using("iceberg").create() } } @@ -102,6 +107,7 @@ class CometFuzzIcebergBase extends CometTestBase with AdaptiveSparkPlanHelper { } file.delete() } + deleteRecursively(warehouseDir) } super.afterAll() diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala index dbd5d0308d..4b10263121 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala @@ -27,7 +27,7 @@ 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, ParquetGenerator} +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, ParquetGenerator, SchemaGenOptions} class CometFuzzIcebergSuite extends CometFuzzIcebergBase { @@ -162,11 +162,17 @@ class CometFuzzIcebergSuite extends CometFuzzIcebergBase { generateArray: Boolean = true, generateStruct: Boolean = true): Unit = { - val options = - DataGenOptions( + val schema = FuzzDataGenerator.generateSchema( + SchemaGenOptions( generateArray = generateArray, generateStruct = generateStruct, - generateNegativeZero = false) + 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) @@ -174,7 +180,7 @@ class CometFuzzIcebergSuite extends CometFuzzIcebergBase { 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, 100, options) + ParquetGenerator.makeParquetFile(random, spark, filename.toString, schema, 100, options) } Seq(defaultTimezone, "UTC", "America/Denver").foreach { tz => From 52019a968ea02b6ce8f008fc6cc95ee392fdbd4a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 22 Oct 2025 21:25:47 -0400 Subject: [PATCH 30/89] update df50. --- native/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 301fc18b31..6bcc207a02 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3151,7 +3151,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#e59f1b1ecb46e95531f8a490b5609ed400821a28" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#147103b641f03932d461f169ae427469fec8bd0f" dependencies = [ "anyhow", "apache-avro", @@ -3334,7 +3334,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "serde", "serde_core", ] From e62a1eed2cd585ed8b6ee249d244d7460a40f3a5 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 13:43:46 -0400 Subject: [PATCH 31/89] fall back for table format v3, ORC, and Avro scans. --- .../apache/comet/rules/CometScanRule.scala | 85 ++++++++++++++++++- 1 file changed, 84 insertions(+), 1 deletion(-) 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 2ec1d94425..5b3938c768 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -280,7 +280,90 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com s"${scanExec.scan.getClass.getSimpleName}: Schema not supported" } - if (schemaSupported) { + // Check Iceberg table format version + val formatVersionSupported = 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) + + // 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 += s"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 += s"Could not verify Iceberg table " + + s"format version: ${e.getMessage}" + false + } + } else { + false + } + + // Check if all files are Parquet format + val allParquetFiles = 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") + + // Check that all FileScanTasks are for Parquet files + val allParquet = tasks.asScala.forall { task => + val dataFile = fileMethod.invoke(task) + val fileFormat = formatMethod.invoke(dataFile) + fileFormat.toString == "PARQUET" + } + + if (!allParquet) { + fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " + + "Comet only supports Parquet files in Iceberg tables" + } + + allParquet + } catch { + case e: Exception => + fallbackReasons += s"Could not verify file formats: ${e.getMessage}" + false + } + } else { + false + } + + if (schemaSupported && formatVersionSupported && allParquetFiles) { // When reading from Iceberg, automatically enable type promotion SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") CometBatchScanExec( From b97f36a965e348a3cd19f29a0e6955fb943b43da Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 14:40:43 -0400 Subject: [PATCH 32/89] Fix TestFilterPushDown Iceberg Java suite by including filters in explain info. --- .../org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index b177a4e155..c22854690d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -104,7 +104,7 @@ case class CometIcebergNativeScanExec( } override def stringArgs: Iterator[Any] = - Iterator(output, metadataLocation, numPartitions) + Iterator(output, s"$metadataLocation, ${originalPlan.scan.description()}", numPartitions) override def equals(obj: Any): Boolean = { obj match { From 08bfd709b2430142729374e1414b81918949c1db Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 15:10:03 -0400 Subject: [PATCH 33/89] Fix format. --- spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5b3938c768..b16c27c66b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -307,7 +307,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } if (formatVersion > 2) { - fallbackReasons += s"Iceberg table format version " + + fallbackReasons += "Iceberg table format version " + s"$formatVersion is not supported. " + "Comet only supports Iceberg table format V1 and V2" false From a3bf1868863fb6771efdc0eb863c4b2293f59084 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 16:31:35 -0400 Subject: [PATCH 34/89] Fix format. --- spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b16c27c66b..157c5a753d 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -316,7 +316,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } } catch { case e: Exception => - fallbackReasons += s"Could not verify Iceberg table " + + fallbackReasons += "Could not verify Iceberg table " + s"format version: ${e.getMessage}" false } From a51652f548d321d81b66a0eb08c09e585cf7334e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 20:28:51 -0400 Subject: [PATCH 35/89] Fix UUID Iceberg type. --- native/Cargo.lock | 1 + native/core/Cargo.toml | 1 + native/core/src/parquet/parquet_support.rs | 24 ++++++++++- .../comet/CometIcebergNativeSuite.scala | 43 +++++++++++++++++++ 4 files changed, 68 insertions(+), 1 deletion(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 6bcc207a02..2384cd68ed 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1774,6 +1774,7 @@ dependencies = [ "tikv-jemallocator", "tokio", "url", + "uuid", "zstd", ] diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 66a8c04553..639cd9a421 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -79,6 +79,7 @@ hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} opendal = { version ="0.54.1", optional = true, features = ["services-hdfs"] } iceberg = { workspace = true } serde_json = "1.0" +uuid = "1.0" [target.'cfg(target_os = "linux")'.dependencies] procfs = "0.18.0" diff --git a/native/core/src/parquet/parquet_support.rs b/native/core/src/parquet/parquet_support.rs index 00208e3161..0b5c45d24d 100644 --- a/native/core/src/parquet/parquet_support.rs +++ b/native/core/src/parquet/parquet_support.rs @@ -16,7 +16,7 @@ // under the License. use crate::execution::operators::ExecutionError; -use arrow::array::{ListArray, MapArray}; +use arrow::array::{FixedSizeBinaryArray, ListArray, MapArray, StringArray}; use arrow::buffer::NullBuffer; use arrow::compute::can_cast_types; use arrow::datatypes::{FieldRef, Fields}; @@ -200,6 +200,28 @@ fn parquet_convert_array( (Map(_, ordered_from), Map(_, ordered_to)) if ordered_from == ordered_to => parquet_convert_map_to_map(array.as_map(), to_type, parquet_options, *ordered_to) , + // Iceberg stores UUIDs as 16-byte fixed binary but Spark expects string representation. + // Arrow doesn't support casting FixedSizeBinary to Utf8, so we handle it manually. + (FixedSizeBinary(16), Utf8) => { + let binary_array = array + .as_any() + .downcast_ref::() + .expect("Expected a FixedSizeBinaryArray"); + + let string_array: StringArray = binary_array + .iter() + .map(|opt_bytes| { + opt_bytes.map(|bytes| { + let uuid = uuid::Uuid::from_bytes( + bytes.try_into().expect("Expected 16 bytes") + ); + uuid.to_string() + }) + }) + .collect(); + + Ok(Arc::new(string_array)) + } // If Arrow cast supports the cast, delegate the cast to Arrow _ if can_cast_types(from_type, to_type) => { Ok(cast_with_options(&array, to_type, &PARQUET_OPTIONS)?) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index f7b5ac34b7..898b4b334c 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1125,6 +1125,49 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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.{Namespace, TableIdentifier} + import org.apache.iceberg.hadoop.HadoopCatalog + 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) + val catalog = + new HadoopCatalog(spark.sessionState.newHadoopConf(), warehouseDir.getAbsolutePath) + catalog.createNamespace(Namespace.of("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") + catalog.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") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From b06800cfa7d6271f02d67046484c30efb870c874 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 21:49:27 -0400 Subject: [PATCH 36/89] Fix UUID Iceberg test. --- .../apache/comet/CometIcebergNativeSuite.scala | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 898b4b334c..612d15ca82 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1137,22 +1137,26 @@ class CometIcebergNativeSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - import org.apache.iceberg.catalog.{Namespace, TableIdentifier} - import org.apache.iceberg.hadoop.HadoopCatalog + 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) - val catalog = - new HadoopCatalog(spark.sessionState.newHadoopConf(), warehouseDir.getAbsolutePath) - catalog.createNamespace(Namespace.of("db")) + // 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") - catalog.createTable(tableIdent, schema, PartitionSpec.unpartitioned()) + sparkCatalog.icebergCatalog.createTable(tableIdent, schema, PartitionSpec.unpartitioned()) spark.sql(""" INSERT INTO test_cat.db.uuid_test VALUES From 905dc9790c36fc1ea77e1e092ae2129c93d08ccf Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 23 Oct 2025 23:26:52 -0400 Subject: [PATCH 37/89] Bump df50. --- native/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 2384cd68ed..40aec3fc5a 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3152,7 +3152,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#147103b641f03932d461f169ae427469fec8bd0f" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#77e7975efa31efbe3fcc76aaa463e28f88451cdc" dependencies = [ "anyhow", "apache-avro", @@ -3335,7 +3335,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "equivalent", - "hashbrown 0.16.0", + "hashbrown 0.15.5", "serde", "serde_core", ] From f8714bc46e9093a79ddb58f0cd04aeee78d002f0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 09:03:45 -0400 Subject: [PATCH 38/89] Iceberg planning and output_rows metrics. --- .../src/execution/operators/iceberg_scan.rs | 32 ++- .../comet/CometIcebergNativeScanExec.scala | 112 ++++++--- .../comet/CometIcebergNativeSuite.scala | 229 ++++++++++++++++++ 3 files changed, 343 insertions(+), 30 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index f1e50839a1..03cdfbc24c 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -30,6 +30,7 @@ use datafusion::common::{DataFusionError, Result as DFResult}; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }; @@ -58,6 +59,8 @@ pub struct IcebergScanExec { catalog_properties: HashMap, /// Pre-planned file scan tasks from Scala, grouped by Spark partition file_task_groups: Option>>, + /// Metrics + metrics: ExecutionPlanMetricsSet, } impl IcebergScanExec { @@ -73,12 +76,15 @@ impl IcebergScanExec { 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, }) } @@ -131,7 +137,7 @@ impl ExecutionPlan for IcebergScanExec { if partition < task_groups.len() { let tasks = &task_groups[partition]; - return self.execute_with_tasks(tasks.clone(), context); + return self.execute_with_tasks(tasks.clone(), partition, context); } else { return Err(DataFusionError::Execution(format!( "IcebergScanExec: Partition index {} out of range (only {} task groups available)", @@ -147,6 +153,10 @@ impl ExecutionPlan for IcebergScanExec { partition ))) } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } } impl IcebergScanExec { @@ -155,6 +165,7 @@ impl IcebergScanExec { fn execute_with_tasks( &self, tasks: Vec, + partition: usize, context: Arc, ) -> DFResult { let output_schema = Arc::clone(&self.output_schema); @@ -165,9 +176,17 @@ impl IcebergScanExec { // Get batch size from context let batch_size = context.session_config().batch_size(); + // Create baseline metrics for this partition + let baseline_metrics = BaselineMetrics::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))?; + let file_stream = IcebergFileStream::new( + tasks, + file_io, + batch_size, + Arc::clone(&output_schema), + baseline_metrics, + )?; // Note: BatchSplitStream adds overhead. Since we're already setting batch_size in // iceberg-rust's ArrowReaderBuilder, it should produce correctly sized batches. @@ -223,6 +242,7 @@ struct IcebergFileStream { batch_size: usize, tasks: VecDeque, state: FileStreamState, + baseline_metrics: BaselineMetrics, } impl IcebergFileStream { @@ -231,6 +251,7 @@ impl IcebergFileStream { file_io: FileIO, batch_size: usize, schema: SchemaRef, + baseline_metrics: BaselineMetrics, ) -> DFResult { Ok(Self { schema, @@ -238,6 +259,7 @@ impl IcebergFileStream { batch_size, tasks: tasks.into_iter().collect(), state: FileStreamState::Idle, + baseline_metrics, }) } @@ -352,6 +374,10 @@ impl IcebergFileStream { // Poll current stream for next batch match ready!(current.poll_next_unpin(cx)) { Some(result) => { + // Record output metrics if batch is successful + if let Ok(ref batch) = result { + self.baseline_metrics.record_output(batch.num_rows()); + } return Poll::Ready(Some(result)); } None => { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index c22854690d..67b831200a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -27,6 +27,7 @@ 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 @@ -92,6 +93,90 @@ case class CometIcebergNativeScanExec( 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(_._1 == "numOutputRows") + .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, initialValue: Long) + extends SQLMetric(metricType, initialValue) { + + // Override merge to do nothing - planning metrics are not updated during execution + override def merge(other: AccumulatorV2[Long, Long]): Unit = { + // Do nothing - this metric's value is immutable + } + + // Override reset to do nothing - planning metrics should never be reset + override def reset(): Unit = { + // Do nothing - this metric's value is immutable + } + } + + override lazy val metrics: Map[String, SQLMetric] = { + val baseMetrics = Map( + "output_rows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + // 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 original type and initial value + val metric = new ImmutableSQLMetric(mv.metricType, mv.value) + // Register it with SparkContext to assign metadata (name, etc.) + sparkContext.register(metric, mv.name) + mv.name -> metric + }.toMap + + baseMetrics ++ icebergMetrics + } + override protected def doCanonicalize(): CometIcebergNativeScanExec = { CometIcebergNativeScanExec( nativeOp, @@ -125,33 +210,6 @@ case class CometIcebergNativeScanExec( output.asJava, serializedPlanOpt, numPartitions: java.lang.Integer) - - override lazy val metrics: Map[String, SQLMetric] = { - Map( - "output_rows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "time_elapsed_opening" -> - SQLMetrics.createNanoTimingMetric( - sparkContext, - "Wall clock time elapsed for FileIO initialization"), - "time_elapsed_scanning_until_data" -> - SQLMetrics.createNanoTimingMetric( - sparkContext, - "Wall clock time elapsed for scanning + first record batch"), - "time_elapsed_scanning_total" -> - SQLMetrics.createNanoTimingMetric( - sparkContext, - "Total wall clock time for scanning + decompression/decoding"), - "time_elapsed_processing" -> - SQLMetrics.createNanoTimingMetric( - sparkContext, - "Wall clock time elapsed for data decompression + decoding"), - "bytes_scanned" -> - SQLMetrics.createSizeMetric(sparkContext, "Number of bytes scanned"), - "files_scanned" -> - SQLMetrics.createMetric(sparkContext, "Number of data files scanned"), - "manifest_files_scanned" -> - SQLMetrics.createMetric(sparkContext, "Number of manifest files scanned")) - } } object CometIcebergNativeScanExec { diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 612d15ca82..0e8fe11f6e 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1172,6 +1172,235 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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() + + // ImmutableSQLMetric prevents these from being reset to 0 after execution + assert(metrics("output_rows").value == 10000) + 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(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(result.length == 3, s"Expected 3 rows after deletes, got ${result.length}") + + spark.sql("DROP TABLE test_cat.db.delete_metrics") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From 5f8256ec17c34ac6361c781fc68e06c57fa4bfb8 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 09:10:31 -0400 Subject: [PATCH 39/89] more output_rows tests. --- .../test/scala/org/apache/comet/CometIcebergNativeSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 0e8fe11f6e..221a44a912 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1329,6 +1329,7 @@ class CometIcebergNativeSuite extends CometTestBase { // 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") @@ -1394,6 +1395,7 @@ class CometIcebergNativeSuite extends CometTestBase { 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") From 50a60eefb67e2ddeb9dc8b98301ad0e3725fb04a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 11:14:16 -0400 Subject: [PATCH 40/89] Dump DF 50.3 and df50 iceberg-rust commit. --- native/Cargo.lock | 78 +++++++++++++++++++++++------------------------ 1 file changed, 39 insertions(+), 39 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 3496230f64..4152dacf84 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -296,7 +296,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.11.4", + "indexmap 2.12.0", "lexical-core", "memchr", "num", @@ -643,9 +643,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.88.0" +version = "1.89.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a68d675582afea0e94d38b6ca9c5aaae4ca14f1d36faa6edb19b42e687e70d7" +checksum = "695dc67bb861ccb8426c9129b91c30e266a0e3d85650cafdf62fcca14c8fd338" dependencies = [ "aws-credential-types", "aws-runtime", @@ -1072,7 +1072,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1095,7 +1095,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1183,9 +1183,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.42" +version = "1.2.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81bbf3b3619004ad9bd139f62a9ab5cfe467f307455a0d307b0cf58bf070feaa" +checksum = "739eb0f94557554b3ca9a86d2d37bebd49c5e6d0c1d2bda35ba5bdac830befc2" dependencies = [ "find-msvc-tools", "jobserver", @@ -1567,7 +1567,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1581,7 +1581,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1592,7 +1592,7 @@ checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core 0.20.11", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1603,7 +1603,7 @@ checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" dependencies = [ "darling_core 0.21.3", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1849,7 +1849,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "hex", - "indexmap 2.11.4", + "indexmap 2.12.0", "libc", "log", "object_store", @@ -2028,7 +2028,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap 2.11.4", + "indexmap 2.12.0", "paste", "serde_json", "sqlparser", @@ -2042,7 +2042,7 @@ checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" dependencies = [ "arrow", "datafusion-common", - "indexmap 2.11.4", + "indexmap 2.12.0", "itertools 0.14.0", "paste", ] @@ -2199,7 +2199,7 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-physical-expr", - "indexmap 2.11.4", + "indexmap 2.12.0", "itertools 0.14.0", "log", "regex", @@ -2221,7 +2221,7 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap 2.11.4", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2300,7 +2300,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap 2.11.4", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2381,7 +2381,7 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap 2.11.4", + "indexmap 2.12.0", "log", "regex", "sqlparser", @@ -2424,7 +2424,7 @@ dependencies = [ "darling 0.20.11", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -2434,7 +2434,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -2880,7 +2880,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap 2.11.4", + "indexmap 2.12.0", "slab", "tokio", "tokio-util", @@ -3152,7 +3152,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#77e7975efa31efbe3fcc76aaa463e28f88451cdc" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#3c5a298ff8f629ed420483708e544f8da6bc4670" dependencies = [ "anyhow", "apache-avro", @@ -3335,7 +3335,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6717a8d2a5a929a1a2eb43a12812498ed141a0bcfb7e8f7844fbdbe4303bba9f" dependencies = [ "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "serde", "serde_core", ] @@ -3347,7 +3347,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ "ahash 0.8.12", - "indexmap 2.11.4", + "indexmap 2.12.0", "is-terminal", "itoa", "log", @@ -4127,7 +4127,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.11.4", + "indexmap 2.12.0", "simdutf8", "uuid", ] @@ -4185,7 +4185,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ "fixedbitset", - "indexmap 2.11.4", + "indexmap 2.12.0", ] [[package]] @@ -4196,7 +4196,7 @@ checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap 2.11.4", + "indexmap 2.12.0", "serde", ] @@ -4701,7 +4701,7 @@ checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -5191,7 +5191,7 @@ checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -5208,15 +5208,15 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.15.0" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6093cd8c01b25262b84927e0f7151692158fab02d961e04c979d3903eba7ecc5" +checksum = "aa66c845eee442168b2c8134fec70ac50dc20e760769c8ba0ad1319ca1959b04" dependencies = [ "base64", "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.11.4", + "indexmap 2.12.0", "schemars 0.9.0", "schemars 1.0.4", "serde_core", @@ -5227,14 +5227,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.15.0" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7e6c180db0816026a61afa1cff5344fb7ebded7e4d3062772179f2501481c27" +checksum = "b91a903660542fced4e99881aa481bdbaec1634568ee02e0b8bd57c64cb38955" dependencies = [ "darling 0.21.3", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -5243,7 +5243,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.11.4", + "indexmap 2.12.0", "itoa", "ryu", "serde", @@ -5418,7 +5418,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -5761,7 +5761,7 @@ version = "0.23.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" dependencies = [ - "indexmap 2.11.4", + "indexmap 2.12.0", "toml_datetime", "toml_parser", "winnow", @@ -5884,7 +5884,7 @@ checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] From 3611b8a4bb7c3044c23d18f6b9b960422eaa0560 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 19:12:20 -0400 Subject: [PATCH 41/89] Update metrics recording for iceberg_scan.rs. --- native/core/src/execution/operators/iceberg_scan.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 03cdfbc24c..33cb4701e7 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -371,13 +371,12 @@ impl IcebergFileStream { } } - // Poll current stream for next batch - match ready!(current.poll_next_unpin(cx)) { + // Poll current stream for next batch and record metrics + match ready!(self + .baseline_metrics + .record_poll(current.poll_next_unpin(cx))) + { Some(result) => { - // Record output metrics if batch is successful - if let Ok(ref batch) = result { - self.baseline_metrics.record_output(batch.num_rows()); - } return Poll::Ready(Some(result)); } None => { From 6361943fd8308b04408460af3e38ec34f881a089 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 19:49:40 -0400 Subject: [PATCH 42/89] FileStreamMetrics for iceberg_scan.rs --- native/Cargo.lock | 90 +++++++++++++++++-- native/core/Cargo.toml | 1 + .../src/execution/operators/iceberg_scan.rs | 27 +++++- .../comet/CometIcebergNativeScanExec.scala | 15 +++- .../comet/CometIcebergNativeSuite.scala | 6 +- 5 files changed, 130 insertions(+), 9 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 4152dacf84..2a6442884e 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -408,6 +408,23 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "async-compression" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" +dependencies = [ + "bzip2 0.5.2", + "flate2", + "futures-core", + "memchr", + "pin-project-lite", + "tokio", + "xz2", + "zstd", + "zstd-safe", +] + [[package]] name = "async-executor" version = "1.13.3" @@ -1175,6 +1192,34 @@ dependencies = [ "either", ] +[[package]] +name = "bzip2" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49ecfb22d906f800d4fe833b6282cf4dc1c298f5057ca0b5445e5c209735ca47" +dependencies = [ + "bzip2-sys", +] + +[[package]] +name = "bzip2" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a53fac24f34a81bc9954b5d6cfce0c21e18ec6959f44f56e8e90e4bb7c346c" +dependencies = [ + "libbz2-rs-sys", +] + +[[package]] +name = "bzip2-sys" +version = "0.1.13+1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225bff33b2141874fe80d71e07d6eec4f85c5c216453dd96388240f96e1acc14" +dependencies = [ + "cc", + "pkg-config", +] + [[package]] name = "cast" version = "0.3.0" @@ -1737,6 +1782,7 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", + "datafusion-datasource", "datafusion-functions-nested", "datafusion-spark", "futures", @@ -1878,8 +1924,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" dependencies = [ "arrow", + "async-compression", "async-trait", "bytes", + "bzip2 0.6.1", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -1890,6 +1938,7 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", + "flate2", "futures", "glob", "itertools 0.14.0", @@ -1899,7 +1948,10 @@ dependencies = [ "rand 0.9.2", "tempfile", "tokio", + "tokio-util", "url", + "xz2", + "zstd", ] [[package]] @@ -2553,7 +2605,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -3388,7 +3440,7 @@ checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -3546,6 +3598,12 @@ dependencies = [ "lexical-util", ] +[[package]] +name = "libbz2-rs-sys" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" + [[package]] name = "libc" version = "0.2.177" @@ -3684,6 +3742,17 @@ dependencies = [ "twox-hash", ] +[[package]] +name = "lzma-sys" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + [[package]] name = "md-5" version = "0.10.6" @@ -4572,7 +4641,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] @@ -4958,7 +5027,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -5514,7 +5583,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix 1.1.2", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -6183,7 +6252,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -6524,6 +6593,15 @@ version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" +[[package]] +name = "xz2" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" +dependencies = [ + "lzma-sys", +] + [[package]] name = "yoke" version = "0.8.0" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index b0258a46ab..6ef4f2a8b4 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -60,6 +60,7 @@ tempfile = "3.8.0" itertools = "0.14.0" paste = "1.0.14" datafusion = { workspace = true, features = ["parquet_encryption"] } +datafusion-datasource = { version = "50.3.0" } datafusion-spark = { workspace = true } once_cell = "1.18.0" regex = { workspace = true } diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 33cb4701e7..03b650215a 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -43,6 +43,7 @@ use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkSchemaAdapterFactory; use datafusion::datasource::schema_adapter::SchemaAdapterFactory; use datafusion_comet_spark_expr::EvalMode; +use datafusion_datasource::file_stream::FileStreamMetrics; /// Native Iceberg scan operator that uses iceberg-rust to read Iceberg tables. /// @@ -186,6 +187,8 @@ impl IcebergScanExec { batch_size, Arc::clone(&output_schema), baseline_metrics, + &self.metrics, + partition, )?; // Note: BatchSplitStream adds overhead. Since we're already setting batch_size in @@ -243,6 +246,7 @@ struct IcebergFileStream { tasks: VecDeque, state: FileStreamState, baseline_metrics: BaselineMetrics, + file_stream_metrics: FileStreamMetrics, } impl IcebergFileStream { @@ -252,7 +256,11 @@ impl IcebergFileStream { batch_size: usize, schema: SchemaRef, baseline_metrics: BaselineMetrics, + metrics: &ExecutionPlanMetricsSet, + partition: usize, ) -> DFResult { + let file_stream_metrics = FileStreamMetrics::new(metrics, partition); + Ok(Self { schema, file_io, @@ -260,6 +268,7 @@ impl IcebergFileStream { tasks: tasks.into_iter().collect(), state: FileStreamState::Idle, baseline_metrics, + file_stream_metrics, }) } @@ -330,6 +339,7 @@ impl IcebergFileStream { match &mut self.state { FileStreamState::Idle => { // Start opening the first file + self.file_stream_metrics.time_opening.start(); match self.start_next_file() { Some(future) => { self.state = FileStreamState::Opening { future }; @@ -342,6 +352,9 @@ impl IcebergFileStream { match ready!(future.poll_unpin(cx)) { Ok(stream) => { // File opened, start reading and open next file in parallel + self.file_stream_metrics.time_opening.stop(); + self.file_stream_metrics.time_scanning_until_data.start(); + self.file_stream_metrics.time_scanning_total.start(); let next = self.start_next_file(); self.state = FileStreamState::Reading { current: stream, @@ -377,15 +390,24 @@ impl IcebergFileStream { .record_poll(current.poll_next_unpin(cx))) { Some(result) => { + // Stop time_scanning_until_data on first batch (idempotent) + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + // Restart time_scanning_total for next batch + self.file_stream_metrics.time_scanning_total.start(); return Poll::Ready(Some(result)); } None => { // Current file is done, move to next file if available + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); match next.take() { Some(mut next_future) => { // Check if next file is already opened match next_future.poll_unpin(cx) { Poll::Ready(Ok(stream)) => { + self.file_stream_metrics.time_scanning_until_data.start(); + self.file_stream_metrics.time_scanning_total.start(); let next_next = self.start_next_file(); self.state = FileStreamState::Reading { current: stream, @@ -424,7 +446,10 @@ impl Stream for IcebergFileStream { type Item = DFResult; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_inner(cx) + self.file_stream_metrics.time_processing.start(); + let result = self.poll_inner(cx); + self.file_stream_metrics.time_processing.stop(); + result } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 67b831200a..cc20a16576 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -162,7 +162,20 @@ case class CometIcebergNativeScanExec( override lazy val metrics: Map[String, SQLMetric] = { val baseMetrics = Map( - "output_rows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + "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 diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 221a44a912..28fb662850 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1250,8 +1250,12 @@ class CometIcebergNativeSuite extends CometTestBase { df.collect() - // ImmutableSQLMetric prevents these from being reset to 0 after execution assert(metrics("output_rows").value == 10000) + 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") From b3c88b9ba989c71e1ec1acd663df62615b13da7e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 19:53:01 -0400 Subject: [PATCH 43/89] Fix format. --- native/core/src/execution/operators/iceberg_scan.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 03b650215a..eb9afb67c7 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -406,7 +406,9 @@ impl IcebergFileStream { // Check if next file is already opened match next_future.poll_unpin(cx) { Poll::Ready(Ok(stream)) => { - self.file_stream_metrics.time_scanning_until_data.start(); + self.file_stream_metrics + .time_scanning_until_data + .start(); self.file_stream_metrics.time_scanning_total.start(); let next_next = self.start_next_file(); self.state = FileStreamState::Reading { From b35917138b4091dbca898fa4256a139b87f94c51 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 20:45:16 -0400 Subject: [PATCH 44/89] numSplits metric. --- native/proto/src/proto/operator.proto | 3 + .../apache/comet/rules/CometExecRule.scala | 15 ++- .../apache/comet/serde/QueryPlanSerde.scala | 3 + .../comet/CometIcebergNativeScanExec.scala | 121 +++++++++++++++++- .../comet/CometIcebergNativeSuite.scala | 2 + 5 files changed, 137 insertions(+), 7 deletions(-) diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index c42966f825..9dfe49b6f6 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -123,6 +123,9 @@ message IcebergScan { // Table metadata file path for FileIO initialization string metadata_location = 5; + + // Total number of file scan tasks (splits) across all partitions + int64 num_splits = 6; } // Groups FileScanTasks for a single Spark partition 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 a816c00521..8a3e8d9a1e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -174,8 +174,21 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // Serialize CometBatchScanExec to extract FileScanTasks and get proto QueryPlanSerde.operator2Proto(scan) match { case Some(nativeOp) => + // Extract num_splits from the serialized protobuf + val numSplits = if (nativeOp.hasIcebergScan) { + nativeOp.getIcebergScan.getNumSplits + } else { + 0L + } + // Create native Iceberg scan exec with the serialized proto - CometIcebergNativeScanExec(nativeOp, scan.wrapped, session, metadataLocation) + // and extracted numSplits + CometIcebergNativeScanExec( + nativeOp, + scan.wrapped, + session, + metadataLocation, + numSplits) case None => // Serialization failed, fall back to CometBatchScanExec scan 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 09589b3a8a..8df2b02863 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1211,6 +1211,7 @@ object QueryPlanSerde extends Logging with CometExprShim { // Extract FileScanTasks from the InputPartitions in the RDD // (Same logic as the previous CometIcebergNativeScanExec case) var actualNumPartitions = 0 + var numSplitsCount = 0L try { scan.wrapped.inputRDD match { case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => @@ -1518,6 +1519,7 @@ object QueryPlanSerde extends Logging with CometExprShim { } partitionBuilder.addFileScanTasks(taskBuilder.build()) + numSplitsCount += 1 } catch { case e: Exception => logWarning(s"Failed to serialize FileScanTask: ${e.getMessage}") @@ -1544,6 +1546,7 @@ object QueryPlanSerde extends Logging with CometExprShim { val numPartitions = if (actualNumPartitions > 0) actualNumPartitions else numParts + icebergScanBuilder.setNumSplits(numSplitsCount) icebergScanBuilder.setNumPartitions(numPartitions) builder.clearChildren() diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index cc20a16576..a8c3d2de3e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -77,7 +77,8 @@ case class CometIcebergNativeScanExec( override val serializedPlanOpt: SerializedPlan, metadataLocation: String, catalogProperties: Map[String, String], // TODO: Extract for authentication - numPartitions: Int) + numPartitions: Int, + numSplits: Long) extends CometLeafExec { override val supportsColumnar: Boolean = true @@ -187,7 +188,11 @@ case class CometIcebergNativeScanExec( mv.name -> metric }.toMap - baseMetrics ++ icebergMetrics + // Add numSplits as an immutable planning metric + val numSplitsMetric = new ImmutableSQLMetric("sum", numSplits) + sparkContext.register(numSplitsMetric, "File splits read based on read.split.target-size") + + baseMetrics ++ icebergMetrics + ("numSplits" -> numSplitsMetric) } override protected def doCanonicalize(): CometIcebergNativeScanExec = { @@ -198,7 +203,8 @@ case class CometIcebergNativeScanExec( SerializedPlan(None), metadataLocation, catalogProperties, - numPartitions) + numPartitions, + numSplits) } override def stringArgs: Iterator[Any] = @@ -211,7 +217,8 @@ case class CometIcebergNativeScanExec( this.catalogProperties == other.catalogProperties && this.output == other.output && this.serializedPlanOpt == other.serializedPlanOpt && - this.numPartitions == other.numPartitions + this.numPartitions == other.numPartitions && + this.numSplits == other.numSplits case _ => false } @@ -222,11 +229,54 @@ case class CometIcebergNativeScanExec( metadataLocation, output.asJava, serializedPlanOpt, - numPartitions: java.lang.Integer) + numPartitions: java.lang.Integer, + numSplits: java.lang.Long) } object CometIcebergNativeScanExec { + /** + * Counts the total number of FileScanTasks across all partitions. + * + * @param scanExec + * The Spark BatchScanExec containing an Iceberg scan + * @return + * Total number of file scan tasks (splits) + */ + private def countFileScanTasks(scanExec: BatchScanExec): Long = { + try { + scanExec.inputRDD match { + case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => + var totalTasks = 0L + rdd.partitions.foreach { partition => + val inputPartitions = partition + .asInstanceOf[org.apache.spark.sql.execution.datasources.v2.DataSourceRDDPartition] + .inputPartitions + + inputPartitions.foreach { inputPartition => + try { + val taskGroupMethod = inputPartition.getClass.getDeclaredMethod("taskGroup") + taskGroupMethod.setAccessible(true) + val taskGroup = taskGroupMethod.invoke(inputPartition) + + val tasksMethod = taskGroup.getClass.getMethod("tasks") + val tasksCollection = + tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] + + totalTasks += tasksCollection.size() + } catch { + case _: Exception => // Skip partitions we can't count + } + } + } + totalTasks + case _ => 0L + } + } catch { + case _: Exception => 0L + } + } + /** * Extracts metadata location from Iceberg table. * @@ -272,6 +322,8 @@ object CometIcebergNativeScanExec { * - KeyGroupedPartitioning: Use Iceberg's partition count * - Other cases: Use the number of InputPartitions from Iceberg's planning * + * Also counts the total number of FileScanTasks for the numSplits metric. + * * @param nativeOp * The serialized native operator * @param scanExec @@ -297,6 +349,62 @@ object CometIcebergNativeScanExec { scanExec.inputRDD.getNumPartitions } + // Count total file scan tasks (splits) for metrics + val numSplits = countFileScanTasks(scanExec) + + val exec = CometIcebergNativeScanExec( + nativeOp, + scanExec.output, + scanExec, + SerializedPlan(None), + metadataLocation, + Map.empty, // TODO: Extract catalog properties for authentication + numParts, + numSplits) + + scanExec.logicalLink.foreach(exec.setLogicalLink) + exec + } + + /** + * Creates a CometIcebergNativeScanExec with a pre-computed numSplits count. + * + * Why this overload exists: During serialization in QueryPlanSerde.operator2Proto(), we iterate + * through ALL FileScanTasks to serialize them into the protobuf. While doing this, we count + * them (numSplitsCount). Since we've already paid the cost of iterating and counting, we pass + * that count through the protobuf back to CometExecRule. This overload accepts that + * pre-computed count to avoid re-iterating the RDD partitions and re-counting tasks using + * reflection (which the 4-parameter overload does via countFileScanTasks()). This ensures we + * use the same count consistently and avoid double work. + * + * @param nativeOp + * The serialized native operator (contains the serialized FileScanTasks) + * @param scanExec + * The original Spark BatchScanExec + * @param session + * The SparkSession + * @param metadataLocation + * Path to table metadata file from extractMetadataLocation + * @param numSplits + * Count of FileScanTasks computed during serialization in QueryPlanSerde + * @return + * A new CometIcebergNativeScanExec + */ + def apply( + nativeOp: Operator, + scanExec: BatchScanExec, + session: SparkSession, + metadataLocation: String, + numSplits: Long): 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, @@ -304,7 +412,8 @@ object CometIcebergNativeScanExec { SerializedPlan(None), metadataLocation, Map.empty, // TODO: Extract catalog properties for authentication - numParts) + numParts, + numSplits) scanExec.logicalLink.foreach(exec.setLogicalLink) exec diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 28fb662850..200bc1a742 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1250,6 +1250,8 @@ class CometIcebergNativeSuite extends CometTestBase { df.collect() + metrics.foreach { metric => println(metric) } + assert(metrics("output_rows").value == 10000) assert(metrics("time_elapsed_opening").value > 0) assert(metrics("time_elapsed_scanning_until_data").value > 0) From f0b2d54cf07fb81cffb47cfe0ed7c910da46a3f5 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 25 Oct 2025 21:29:03 -0400 Subject: [PATCH 45/89] more filtering tests. --- .../comet/CometIcebergNativeSuite.scala | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 200bc1a742..5e6e86449e 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1409,6 +1409,82 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From a5129d87a2d17311c753cacfed8e6e85b4d9aeb6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 26 Oct 2025 18:59:45 -0400 Subject: [PATCH 46/89] Change num_splits to be a runtime count instead of serialization time. --- .../src/execution/operators/iceberg_scan.rs | 77 +++++++---- native/proto/src/proto/operator.proto | 3 - .../apache/comet/rules/CometExecRule.scala | 15 +-- .../apache/comet/serde/QueryPlanSerde.scala | 3 - .../comet/CometIcebergNativeScanExec.scala | 127 ++---------------- .../comet/CometIcebergNativeSuite.scala | 3 +- 6 files changed, 62 insertions(+), 166 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index eb9afb67c7..c470774c3c 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -30,7 +30,9 @@ use datafusion::common::{DataFusionError, Result as DFResult}; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use datafusion::physical_plan::metrics::{ + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, +}; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }; @@ -177,8 +179,8 @@ impl IcebergScanExec { // Get batch size from context let batch_size = context.session_config().batch_size(); - // Create baseline metrics for this partition - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + // Create metrics for this partition (wraps both baseline and file stream metrics) + 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( @@ -186,9 +188,7 @@ impl IcebergScanExec { file_io, batch_size, Arc::clone(&output_schema), - baseline_metrics, - &self.metrics, - partition, + metrics, )?; // Note: BatchSplitStream adds overhead. Since we're already setting batch_size in @@ -216,6 +216,26 @@ impl IcebergScanExec { } } +/// 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 @@ -245,8 +265,7 @@ struct IcebergFileStream { batch_size: usize, tasks: VecDeque, state: FileStreamState, - baseline_metrics: BaselineMetrics, - file_stream_metrics: FileStreamMetrics, + metrics: IcebergScanMetrics, } impl IcebergFileStream { @@ -255,20 +274,15 @@ impl IcebergFileStream { file_io: FileIO, batch_size: usize, schema: SchemaRef, - baseline_metrics: BaselineMetrics, - metrics: &ExecutionPlanMetricsSet, - partition: usize, + metrics: IcebergScanMetrics, ) -> DFResult { - let file_stream_metrics = FileStreamMetrics::new(metrics, partition); - Ok(Self { schema, file_io, batch_size, tasks: tasks.into_iter().collect(), state: FileStreamState::Idle, - baseline_metrics, - file_stream_metrics, + metrics, }) } @@ -277,6 +291,9 @@ impl IcebergFileStream { &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); @@ -339,7 +356,7 @@ impl IcebergFileStream { match &mut self.state { FileStreamState::Idle => { // Start opening the first file - self.file_stream_metrics.time_opening.start(); + self.metrics.file_stream.time_opening.start(); match self.start_next_file() { Some(future) => { self.state = FileStreamState::Opening { future }; @@ -352,9 +369,9 @@ impl IcebergFileStream { match ready!(future.poll_unpin(cx)) { Ok(stream) => { // File opened, start reading and open next file in parallel - self.file_stream_metrics.time_opening.stop(); - self.file_stream_metrics.time_scanning_until_data.start(); - self.file_stream_metrics.time_scanning_total.start(); + 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, @@ -386,30 +403,32 @@ impl IcebergFileStream { // Poll current stream for next batch and record metrics match ready!(self - .baseline_metrics + .metrics + .baseline .record_poll(current.poll_next_unpin(cx))) { Some(result) => { // Stop time_scanning_until_data on first batch (idempotent) - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); + 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.file_stream_metrics.time_scanning_total.start(); + self.metrics.file_stream.time_scanning_total.start(); return Poll::Ready(Some(result)); } None => { // Current file is done, move to next file if available - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); + self.metrics.file_stream.time_scanning_until_data.stop(); + self.metrics.file_stream.time_scanning_total.stop(); match next.take() { Some(mut next_future) => { // Check if next file is already opened match next_future.poll_unpin(cx) { Poll::Ready(Ok(stream)) => { - self.file_stream_metrics + self.metrics + .file_stream .time_scanning_until_data .start(); - self.file_stream_metrics.time_scanning_total.start(); + self.metrics.file_stream.time_scanning_total.start(); let next_next = self.start_next_file(); self.state = FileStreamState::Reading { current: stream, @@ -448,9 +467,9 @@ impl Stream for IcebergFileStream { type Item = DFResult; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.file_stream_metrics.time_processing.start(); + self.metrics.file_stream.time_processing.start(); let result = self.poll_inner(cx); - self.file_stream_metrics.time_processing.stop(); + self.metrics.file_stream.time_processing.stop(); result } } diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 9dfe49b6f6..c42966f825 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -123,9 +123,6 @@ message IcebergScan { // Table metadata file path for FileIO initialization string metadata_location = 5; - - // Total number of file scan tasks (splits) across all partitions - int64 num_splits = 6; } // Groups FileScanTasks for a single Spark partition 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 8a3e8d9a1e..a816c00521 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -174,21 +174,8 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // Serialize CometBatchScanExec to extract FileScanTasks and get proto QueryPlanSerde.operator2Proto(scan) match { case Some(nativeOp) => - // Extract num_splits from the serialized protobuf - val numSplits = if (nativeOp.hasIcebergScan) { - nativeOp.getIcebergScan.getNumSplits - } else { - 0L - } - // Create native Iceberg scan exec with the serialized proto - // and extracted numSplits - CometIcebergNativeScanExec( - nativeOp, - scan.wrapped, - session, - metadataLocation, - numSplits) + CometIcebergNativeScanExec(nativeOp, scan.wrapped, session, metadataLocation) case None => // Serialization failed, fall back to CometBatchScanExec scan 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 8df2b02863..09589b3a8a 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1211,7 +1211,6 @@ object QueryPlanSerde extends Logging with CometExprShim { // Extract FileScanTasks from the InputPartitions in the RDD // (Same logic as the previous CometIcebergNativeScanExec case) var actualNumPartitions = 0 - var numSplitsCount = 0L try { scan.wrapped.inputRDD match { case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => @@ -1519,7 +1518,6 @@ object QueryPlanSerde extends Logging with CometExprShim { } partitionBuilder.addFileScanTasks(taskBuilder.build()) - numSplitsCount += 1 } catch { case e: Exception => logWarning(s"Failed to serialize FileScanTask: ${e.getMessage}") @@ -1546,7 +1544,6 @@ object QueryPlanSerde extends Logging with CometExprShim { val numPartitions = if (actualNumPartitions > 0) actualNumPartitions else numParts - icebergScanBuilder.setNumSplits(numSplitsCount) icebergScanBuilder.setNumPartitions(numPartitions) builder.clearChildren() diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index a8c3d2de3e..53558d8171 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -77,8 +77,7 @@ case class CometIcebergNativeScanExec( override val serializedPlanOpt: SerializedPlan, metadataLocation: String, catalogProperties: Map[String, String], // TODO: Extract for authentication - numPartitions: Int, - numSplits: Long) + numPartitions: Int) extends CometLeafExec { override val supportsColumnar: Boolean = true @@ -130,7 +129,10 @@ case class CometIcebergNativeScanExec( private val capturedMetricValues: Seq[MetricValue] = { originalPlan.metrics - .filterNot(_._1 == "numOutputRows") + .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) @@ -188,11 +190,10 @@ case class CometIcebergNativeScanExec( mv.name -> metric }.toMap - // Add numSplits as an immutable planning metric - val numSplitsMetric = new ImmutableSQLMetric("sum", numSplits) - sparkContext.register(numSplitsMetric, "File splits read based on read.split.target-size") + // 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 + ("numSplits" -> numSplitsMetric) + baseMetrics ++ icebergMetrics + ("num_splits" -> numSplitsMetric) } override protected def doCanonicalize(): CometIcebergNativeScanExec = { @@ -203,8 +204,7 @@ case class CometIcebergNativeScanExec( SerializedPlan(None), metadataLocation, catalogProperties, - numPartitions, - numSplits) + numPartitions) } override def stringArgs: Iterator[Any] = @@ -217,8 +217,7 @@ case class CometIcebergNativeScanExec( this.catalogProperties == other.catalogProperties && this.output == other.output && this.serializedPlanOpt == other.serializedPlanOpt && - this.numPartitions == other.numPartitions && - this.numSplits == other.numSplits + this.numPartitions == other.numPartitions case _ => false } @@ -229,54 +228,11 @@ case class CometIcebergNativeScanExec( metadataLocation, output.asJava, serializedPlanOpt, - numPartitions: java.lang.Integer, - numSplits: java.lang.Long) + numPartitions: java.lang.Integer) } object CometIcebergNativeScanExec { - /** - * Counts the total number of FileScanTasks across all partitions. - * - * @param scanExec - * The Spark BatchScanExec containing an Iceberg scan - * @return - * Total number of file scan tasks (splits) - */ - private def countFileScanTasks(scanExec: BatchScanExec): Long = { - try { - scanExec.inputRDD match { - case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => - var totalTasks = 0L - rdd.partitions.foreach { partition => - val inputPartitions = partition - .asInstanceOf[org.apache.spark.sql.execution.datasources.v2.DataSourceRDDPartition] - .inputPartitions - - inputPartitions.foreach { inputPartition => - try { - val taskGroupMethod = inputPartition.getClass.getDeclaredMethod("taskGroup") - taskGroupMethod.setAccessible(true) - val taskGroup = taskGroupMethod.invoke(inputPartition) - - val tasksMethod = taskGroup.getClass.getMethod("tasks") - val tasksCollection = - tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] - - totalTasks += tasksCollection.size() - } catch { - case _: Exception => // Skip partitions we can't count - } - } - } - totalTasks - case _ => 0L - } - } catch { - case _: Exception => 0L - } - } - /** * Extracts metadata location from Iceberg table. * @@ -322,8 +278,6 @@ object CometIcebergNativeScanExec { * - KeyGroupedPartitioning: Use Iceberg's partition count * - Other cases: Use the number of InputPartitions from Iceberg's planning * - * Also counts the total number of FileScanTasks for the numSplits metric. - * * @param nativeOp * The serialized native operator * @param scanExec @@ -349,62 +303,6 @@ object CometIcebergNativeScanExec { scanExec.inputRDD.getNumPartitions } - // Count total file scan tasks (splits) for metrics - val numSplits = countFileScanTasks(scanExec) - - val exec = CometIcebergNativeScanExec( - nativeOp, - scanExec.output, - scanExec, - SerializedPlan(None), - metadataLocation, - Map.empty, // TODO: Extract catalog properties for authentication - numParts, - numSplits) - - scanExec.logicalLink.foreach(exec.setLogicalLink) - exec - } - - /** - * Creates a CometIcebergNativeScanExec with a pre-computed numSplits count. - * - * Why this overload exists: During serialization in QueryPlanSerde.operator2Proto(), we iterate - * through ALL FileScanTasks to serialize them into the protobuf. While doing this, we count - * them (numSplitsCount). Since we've already paid the cost of iterating and counting, we pass - * that count through the protobuf back to CometExecRule. This overload accepts that - * pre-computed count to avoid re-iterating the RDD partitions and re-counting tasks using - * reflection (which the 4-parameter overload does via countFileScanTasks()). This ensures we - * use the same count consistently and avoid double work. - * - * @param nativeOp - * The serialized native operator (contains the serialized FileScanTasks) - * @param scanExec - * The original Spark BatchScanExec - * @param session - * The SparkSession - * @param metadataLocation - * Path to table metadata file from extractMetadataLocation - * @param numSplits - * Count of FileScanTasks computed during serialization in QueryPlanSerde - * @return - * A new CometIcebergNativeScanExec - */ - def apply( - nativeOp: Operator, - scanExec: BatchScanExec, - session: SparkSession, - metadataLocation: String, - numSplits: Long): 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, @@ -412,8 +310,7 @@ object CometIcebergNativeScanExec { SerializedPlan(None), metadataLocation, Map.empty, // TODO: Extract catalog properties for authentication - numParts, - numSplits) + numParts) scanExec.logicalLink.foreach(exec.setLogicalLink) exec diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 5e6e86449e..aff1cc5fce 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1250,9 +1250,8 @@ class CometIcebergNativeSuite extends CometTestBase { df.collect() - metrics.foreach { metric => println(metric) } - 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) From 861a57504a53afa6f4a703504f799c1c256aa23b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 26 Oct 2025 19:06:03 -0400 Subject: [PATCH 47/89] Fix Spark 4 with ImmutableSQLMetric. --- .../spark/sql/comet/CometIcebergNativeScanExec.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 53558d8171..cb436da04a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -149,8 +149,7 @@ case class CometIcebergNativeScanExec( * * This class overrides the accumulator methods to make the metric truly immutable once set. */ - private class ImmutableSQLMetric(metricType: String, initialValue: Long) - extends SQLMetric(metricType, initialValue) { + private class ImmutableSQLMetric(metricType: String) extends SQLMetric(metricType, 0) { // Override merge to do nothing - planning metrics are not updated during execution override def merge(other: AccumulatorV2[Long, Long]): Unit = { @@ -183,8 +182,10 @@ case class CometIcebergNativeScanExec( // 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 original type and initial value - val metric = new ImmutableSQLMetric(mv.metricType, mv.value) + // 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 From 27a1a754adeb17547f7dfb52908d18c0f80e77b3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 10:00:54 -0400 Subject: [PATCH 48/89] New 1.9.1.diff --- dev/diffs/iceberg/1.9.1.diff | 170 ++++++++++++++++++----------------- 1 file changed, 90 insertions(+), 80 deletions(-) diff --git a/dev/diffs/iceberg/1.9.1.diff b/dev/diffs/iceberg/1.9.1.diff index 6faf09ec70..128241b82f 100644 --- a/dev/diffs/iceberg/1.9.1.diff +++ b/dev/diffs/iceberg/1.9.1.diff @@ -1,5 +1,5 @@ diff --git a/build.gradle b/build.gradle -index 998f2ee9ea6..017e61be98c 100644 +index 998f2ee9e..017e61be9 100644 --- a/build.gradle +++ b/build.gradle @@ -814,6 +814,13 @@ project(':iceberg-parquet') { @@ -17,7 +17,7 @@ index 998f2ee9ea6..017e61be98c 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 c50991c5fc6..f7ad00f0b78 100644 +index c50991c5f..3acb395a6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -36,6 +36,7 @@ awssdk-s3accessgrants = "2.3.0" @@ -30,7 +30,7 @@ index c50991c5fc6..f7ad00f0b78 100644 delta-spark = "3.3.1" diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java new file mode 100644 -index 00000000000..ddf6c7de5ae +index 000000000..ddf6c7de5 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java @@ -0,0 +1,255 @@ @@ -291,7 +291,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 @@ @@ -556,7 +556,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 31f9e2a80a6..520f142c212 100644 +index 31f9e2a80..520f142c2 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1124,6 +1124,7 @@ public class Parquet { @@ -635,7 +635,7 @@ index 31f9e2a80a6..520f142c212 100644 Function> readBuilder = readerFuncWithSchema != null diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java -index 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 Date: Mon, 27 Oct 2025 10:04:08 -0400 Subject: [PATCH 49/89] New 1.8.1.diff --- dev/diffs/iceberg/1.8.1.diff | 170 ++++++++++++++++++----------------- 1 file changed, 90 insertions(+), 80 deletions(-) diff --git a/dev/diffs/iceberg/1.8.1.diff b/dev/diffs/iceberg/1.8.1.diff index 15c3885be6..d554db1c5e 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..a909cd552c1 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..a909cd552c1 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 Date: Mon, 27 Oct 2025 11:29:59 -0400 Subject: [PATCH 50/89] Fall back on unsupported file schemes, but add new tests to verify partition pruning works okay. This fixes TestPartitionPruning Iceberg Java tests. --- .../apache/comet/rules/CometScanRule.scala | 97 +++++---- .../comet/CometIcebergNativeSuite.scala | 184 +++++++++++++++++- 2 files changed, 244 insertions(+), 37 deletions(-) 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 7b17ed99b3..c5d95a5819 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -324,46 +324,71 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com false } - // Check if all files are Parquet format - val allParquetFiles = 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") - - // Check that all FileScanTasks are for Parquet files - val allParquet = tasks.asScala.forall { task => - val dataFile = fileMethod.invoke(task) - val fileFormat = formatMethod.invoke(dataFile) - fileFormat.toString == "PARQUET" - } + // 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 + val filePath = pathMethod.invoke(dataFile).toString + val scheme = new URI(filePath).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(", ")}" + } + } - if (!allParquet) { - fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " + - "Comet only supports Parquet files in Iceberg tables" - } + if (!allParquet) { + fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " + + "Comet only supports Parquet files in Iceberg tables" + } - allParquet - } catch { - case e: Exception => - fallbackReasons += s"Could not verify file formats: ${e.getMessage}" - false + (allParquet, allSupportedFs) + } catch { + case e: Exception => + fallbackReasons += "Could not verify file formats or filesystem schemes: " + + s"${e.getMessage}" + (false, false) + } + } else { + (false, false) } - } else { - false - } - if (schemaSupported && formatVersionSupported && allParquetFiles) { + if (schemaSupported && formatVersionSupported && allParquetFiles && + allSupportedFilesystems) { // When reading from Iceberg, automatically enable type promotion SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") CometBatchScanExec( diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index aff1cc5fce..bccb990ac8 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -774,7 +774,6 @@ class CometIcebergNativeSuite extends CometTestBase { FROM range(100) """) - // Test bucket partitioning correctness 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") @@ -786,6 +785,189 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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) + + // Partition pruning occurs at the manifest level, not file level + // Each INSERT creates one manifest, so we verify skippedDataManifests + assert( + metrics("resultDataFiles").value < 8, + s"Bucket pruning should skip some files, but read ${metrics("resultDataFiles").value} out of 8") + assert( + metrics("skippedDataManifests").value > 0, + s"Expected skipped manifests due to bucket pruning, got ${metrics("skippedDataManifests").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") From 591ff749eb303a880961e2dd1ae484e46dca3db6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 14:04:23 -0400 Subject: [PATCH 51/89] Fix partitioning test in CometIcebergNativeSuite --- .../org/apache/comet/CometIcebergNativeSuite.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index bccb990ac8..cd35a287a9 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -830,14 +830,18 @@ class CometIcebergNativeSuite extends CometTestBase { val result = df.collect() assert(result.length == specificIds.length) - // Partition pruning occurs at the manifest level, not file level - // Each INSERT creates one manifest, so we verify skippedDataManifests + // 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, - s"Bucket pruning should skip some files, but read ${metrics("resultDataFiles").value} out of 8") + "Bucket pruning should skip some files, but read " + + s"${metrics("resultDataFiles").value} out of 8") assert( - metrics("skippedDataManifests").value > 0, - s"Expected skipped manifests due to bucket pruning, got ${metrics("skippedDataManifests").value}") + 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") } From 2311d6035e7bd31976c5adcb089d4fc14625cf98 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 15:50:04 -0400 Subject: [PATCH 52/89] Fix schema evolution with snapshots. --- .../apache/comet/serde/QueryPlanSerde.scala | 89 +++++++++++++++++-- .../comet/CometIcebergNativeSuite.scala | 78 ++++++++++++++++ 2 files changed, 162 insertions(+), 5 deletions(-) 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 09589b3a8a..af8d79368e 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1208,6 +1208,62 @@ object QueryPlanSerde extends Logging with CometExprShim { 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 // (Same logic as the previous CometIcebergNativeScanExec case) var actualNumPartitions = 0 @@ -1355,8 +1411,17 @@ object QueryPlanSerde extends Logging with CometExprShim { taskBuilder.setLength(length) try { - val schemaMethod = fileScanTaskClass.getMethod("schema") - val schema = schemaMethod.invoke(task) + // Use the scan's expected schema for ALL tasks instead of each task's + // individual schema. This is critical for schema evolution: when + // reading old snapshots after column changes, some tasks may have the + // current table schema instead of the snapshot schema.Using the scan + // schema ensures iceberg-rust reads all files correctly. + val schema: AnyRef = + scanSchemaForTasks.map(_.asInstanceOf[AnyRef]).getOrElse { + // Fallback to task schema if scan schema extraction failed + val schemaMethod = fileScanTaskClass.getMethod("schema") + schemaMethod.invoke(task) + } // scalastyle:off classforname val schemaParserClass = Class.forName("org.apache.iceberg.SchemaParser") @@ -1374,6 +1439,8 @@ object QueryPlanSerde extends Logging with CometExprShim { taskBuilder.setSchemaJson(schemaJson) + // Build field ID mapping from the schema we're using + // (scan or task schema) val columnsMethod = schema.getClass.getMethod("columns") val columns = columnsMethod.invoke(schema).asInstanceOf[java.util.List[_]] @@ -1395,11 +1462,23 @@ object QueryPlanSerde extends Logging with CometExprShim { } } + // Extract project_field_ids for scan.output columns. + // For schema evolution: try task schema first, then fall back to + // global scan schema. + // This handles cases where scan.output has columns not present in + // some task schemas. scan.output.foreach { attr => - nameToFieldId.get(attr.name) match { - case Some(fieldId) => - taskBuilder.addProjectFieldIds(fieldId) + 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 { diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index cd35a287a9..933a340541 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1670,6 +1670,84 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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 + val beforeDrop = + spark.sql("SELECT * FROM test_cat.db.schema_evolution_test ORDER BY id").collect() + assert(beforeDrop.length == 3) + assert(beforeDrop(0).getString(1) == "x", "Row 1 should have data='x' before drop") + assert(beforeDrop(1).getString(1) == "y", "Row 2 should have data='y' before drop") + assert(beforeDrop(2).getString(1) == "z", "Row 3 should have data='z' before drop") + + // 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 + val snapshotRead = spark + .sql( + s"SELECT * FROM test_cat.db.schema_evolution_test VERSION AS OF $snapshotIdBefore ORDER BY id") + .collect() + + // Verify the snapshot read returns the original data with all columns + assert(snapshotRead.length == 3, "Should have 3 rows") + assert( + snapshotRead(0).getString(1) == "x", + s"Row 1 should have data='x', got ${snapshotRead(0).get(1)}") + assert( + snapshotRead(1).getString(1) == "y", + s"Row 2 should have data='y', got ${snapshotRead(1).get(1)}") + assert( + snapshotRead(2).getString(1) == "z", + s"Row 3 should have data='z', got ${snapshotRead(2).get(1)}") + + spark.sql("DROP TABLE test_cat.db.schema_evolution_test") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From 0c9a78dc282395666bda8737ab1451f33bd37c4a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 20:28:37 -0400 Subject: [PATCH 53/89] Fix schemas for delete files. --- .../apache/comet/serde/QueryPlanSerde.scala | 32 +++++++++++++------ 1 file changed, 23 insertions(+), 9 deletions(-) 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 af8d79368e..9c348e58cb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1411,16 +1411,30 @@ object QueryPlanSerde extends Logging with CometExprShim { taskBuilder.setLength(length) try { - // Use the scan's expected schema for ALL tasks instead of each task's - // individual schema. This is critical for schema evolution: when - // reading old snapshots after column changes, some tasks may have the - // current table schema instead of the snapshot schema.Using the scan - // schema ensures iceberg-rust reads all files correctly. + // Schema selection strategy: + // - For tasks with delete files: Use task schema (not scan schema) + // Delete files may reference columns not in the projection, so we need + // the full schema to resolve equality delete field IDs + // - For tasks without deletes: Use scan schema for schema evolution + // This ensures old snapshots read correctly after column drops + + val taskSchemaMethod = fileScanTaskClass.getMethod("schema") + val taskSchema = taskSchemaMethod.invoke(task) + + // Check if task has delete files + val deletesMethod = fileScanTaskClass.getMethod("deletes") + val deletes = deletesMethod + .invoke(task) + .asInstanceOf[java.util.List[_]] + val hasDeletes = !deletes.isEmpty + val schema: AnyRef = - scanSchemaForTasks.map(_.asInstanceOf[AnyRef]).getOrElse { - // Fallback to task schema if scan schema extraction failed - val schemaMethod = fileScanTaskClass.getMethod("schema") - schemaMethod.invoke(task) + if (hasDeletes) { + // Use task schema when deletes are present + taskSchema + } else { + // Use scan schema for schema evolution when no deletes + scanSchemaForTasks.map(_.asInstanceOf[AnyRef]).getOrElse(taskSchema) } // scalastyle:off classforname From 87f436aa8c04629962285f46ac32656aa81f20ed Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 21:15:56 -0400 Subject: [PATCH 54/89] Fall back for now for unsupported partitioning types and filter expressions with complex types. --- .../apache/comet/rules/CometScanRule.scala | 120 +++++++++++++++++- .../apache/comet/serde/QueryPlanSerde.scala | 13 +- 2 files changed, 122 insertions(+), 11 deletions(-) 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 c5d95a5819..f8e064b284 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -387,9 +387,125 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com (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 e: Exception => + // Avoid blocking valid queries if reflection fails + true + } + } else { + false + } + + // iceberg-rust cannot bind predicates on complex types (struct/array/map). + // Detecting which columns filters reference requires parsing Iceberg expressions, + // so we use string matching as a conservative approximation. + 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[_]] + + // No filters means no predicate binding issues + if (filters.isEmpty) { + true + } else { + val readSchema = scanExec.scan.readSchema() + + val complexTypeColumns = readSchema + .filter(field => isComplexType(field.dataType)) + .map(_.name) + .toSet + + // String matching is imperfect but avoids parsing Iceberg Expression AST + val hasComplexTypePredicate = filters.asScala.exists { expr => + val exprStr = expr.toString + complexTypeColumns.exists(colName => exprStr.contains(colName)) + } + + if (hasComplexTypePredicate) { + fallbackReasons += "Filter predicates reference complex type columns " + + "(struct/array/map), which are not yet supported by iceberg-rust" + 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 + } + if (schemaSupported && formatVersionSupported && allParquetFiles && - allSupportedFilesystems) { - // When reading from Iceberg, automatically enable type promotion + allSupportedFilesystems && partitionTypesSupported && + complexTypePredicatesSupported) { + // 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 9c348e58cb..c9051b1ca3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1411,17 +1411,14 @@ object QueryPlanSerde extends Logging with CometExprShim { taskBuilder.setLength(length) try { - // Schema selection strategy: - // - For tasks with delete files: Use task schema (not scan schema) - // Delete files may reference columns not in the projection, so we need - // the full schema to resolve equality delete field IDs - // - For tasks without deletes: Use scan schema for schema evolution - // This ensures old snapshots read correctly after column drops + // 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) - // Check if task has delete files val deletesMethod = fileScanTaskClass.getMethod("deletes") val deletes = deletesMethod .invoke(task) @@ -1430,10 +1427,8 @@ object QueryPlanSerde extends Logging with CometExprShim { val schema: AnyRef = if (hasDeletes) { - // Use task schema when deletes are present taskSchema } else { - // Use scan schema for schema evolution when no deletes scanSchemaForTasks.map(_.asInstanceOf[AnyRef]).getOrElse(taskSchema) } From 5a88d19c84d390d7b86cdfb2e72dfb332c3151b1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 21:41:42 -0400 Subject: [PATCH 55/89] Fix compilation --- spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f8e064b284..7af95cfc00 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -445,7 +445,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com allSupported } catch { - case e: Exception => + case _: Exception => // Avoid blocking valid queries if reflection fails true } From b0e6452fa0ca51244d365e3eb9be5a9143bf67fa Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 21:56:29 -0400 Subject: [PATCH 56/89] date32 schema change test. --- .../comet/CometIcebergNativeSuite.scala | 84 +++++++++++++++++++ 1 file changed, 84 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 933a340541..e68ff2438c 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1748,6 +1748,90 @@ class CometIcebergNativeSuite extends CometTestBase { } } + 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 + // THIS CURRENTLY FAILS with "unexpected target column type Date32" + val branchRead = spark + .sql( + "SELECT * FROM test_cat.db.date_branch_test VERSION AS OF 'test_branch' ORDER BY id") + .collect() + + // Verify: Should get 3 rows with date_col = NULL + assert(branchRead.length == 3, "Should have 3 rows from branch") + assert(branchRead(0).getLong(0) == 1, "Row 1 should have id=1") + assert(branchRead(0).getString(1) == "a", "Row 1 should have data='a'") + assert(branchRead(0).isNullAt(2), "Row 1 should have date_col=NULL") + + assert(branchRead(1).getLong(0) == 2, "Row 2 should have id=2") + assert(branchRead(1).getString(1) == "b", "Row 2 should have data='b'") + assert(branchRead(1).isNullAt(2), "Row 2 should have date_col=NULL") + + assert(branchRead(2).getLong(0) == 3, "Row 3 should have id=3") + assert(branchRead(2).getString(1) == "c", "Row 3 should have data='c'") + assert(branchRead(2).isNullAt(2), "Row 3 should have date_col=NULL") + + spark.sql("DROP TABLE test_cat.db.date_branch_test") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From 5485508f73af9329281c3edc6d841eff08057ab5 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 22:37:26 -0400 Subject: [PATCH 57/89] bump df50 --- native/Cargo.lock | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 2a6442884e..0609b8f34f 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -2605,7 +2605,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -3204,7 +3204,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#3c5a298ff8f629ed420483708e544f8da6bc4670" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#feb8f05ec55f31b911e7cc6319585fe4570f7819" dependencies = [ "anyhow", "apache-avro", @@ -3440,7 +3440,7 @@ checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -4641,7 +4641,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -5027,7 +5027,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -5583,7 +5583,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix 1.1.2", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -6252,7 +6252,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] From eb3b93df053095af70b9d9a254b748fe84cff30d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 29 Oct 2025 11:55:46 -0400 Subject: [PATCH 58/89] adjust fallback logic for complex types, add new tests. --- .../apache/comet/rules/CometScanRule.scala | 32 +- .../comet/CometIcebergNativeSuite.scala | 351 ++++++++++++++++-- 2 files changed, 331 insertions(+), 52 deletions(-) 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 7af95cfc00..315993fb75 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -453,9 +453,9 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com false } - // iceberg-rust cannot bind predicates on complex types (struct/array/map). - // Detecting which columns filters reference requires parsing Iceberg expressions, - // so we use string matching as a conservative approximation. + // 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) { @@ -467,26 +467,36 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com val filters = filterExpressionsMethod.invoke(scanExec.scan).asInstanceOf[java.util.List[_]] - // No filters means no predicate binding issues + // Empty filters can't trigger accessor issues if (filters.isEmpty) { true } else { val readSchema = scanExec.scan.readSchema() - val complexTypeColumns = readSchema + // Identify complex type columns that would trigger accessor creation failures + val complexColumns = readSchema .filter(field => isComplexType(field.dataType)) .map(_.name) .toSet - // String matching is imperfect but avoids parsing Iceberg Expression AST - val hasComplexTypePredicate = filters.asScala.exists { expr => + // 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 - complexTypeColumns.exists(colName => exprStr.contains(colName)) + val isNullCheck = exprStr.contains("is_null") || exprStr.contains("not_null") + if (isNullCheck) { + complexColumns.exists { colName => + exprStr.contains(s"""ref(name="$colName")""") + } + } else { + false + } } - if (hasComplexTypePredicate) { - fallbackReasons += "Filter predicates reference complex type columns " + - "(struct/array/map), which are not yet supported by iceberg-rust" + 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 diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index e68ff2438c..d1b9197ffd 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -1714,34 +1714,15 @@ class CometIcebergNativeSuite extends CometTestBase { .getLong(0) // Verify data is correct before schema change - val beforeDrop = - spark.sql("SELECT * FROM test_cat.db.schema_evolution_test ORDER BY id").collect() - assert(beforeDrop.length == 3) - assert(beforeDrop(0).getString(1) == "x", "Row 1 should have data='x' before drop") - assert(beforeDrop(1).getString(1) == "y", "Row 2 should have data='y' before drop") - assert(beforeDrop(2).getString(1) == "z", "Row 3 should have data='z' before drop") + 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 - val snapshotRead = spark - .sql( - s"SELECT * FROM test_cat.db.schema_evolution_test VERSION AS OF $snapshotIdBefore ORDER BY id") - .collect() - - // Verify the snapshot read returns the original data with all columns - assert(snapshotRead.length == 3, "Should have 3 rows") - assert( - snapshotRead(0).getString(1) == "x", - s"Row 1 should have data='x', got ${snapshotRead(0).get(1)}") - assert( - snapshotRead(1).getString(1) == "y", - s"Row 2 should have data='y', got ${snapshotRead(1).get(1)}") - assert( - snapshotRead(2).getString(1) == "z", - s"Row 3 should have data='z', got ${snapshotRead(2).get(1)}") + 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") } @@ -1807,31 +1788,319 @@ class CometIcebergNativeSuite extends CometTestBase { // 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 - // THIS CURRENTLY FAILS with "unexpected target column type Date32" - val branchRead = spark - .sql( - "SELECT * FROM test_cat.db.date_branch_test VERSION AS OF 'test_branch' ORDER BY id") - .collect() - - // Verify: Should get 3 rows with date_col = NULL - assert(branchRead.length == 3, "Should have 3 rows from branch") - assert(branchRead(0).getLong(0) == 1, "Row 1 should have id=1") - assert(branchRead(0).getString(1) == "a", "Row 1 should have data='a'") - assert(branchRead(0).isNullAt(2), "Row 1 should have date_col=NULL") - - assert(branchRead(1).getLong(0) == 2, "Row 2 should have id=2") - assert(branchRead(1).getString(1) == "b", "Row 2 should have data='b'") - assert(branchRead(1).isNullAt(2), "Row 2 should have date_col=NULL") - - assert(branchRead(2).getLong(0) == 3, "Row 3 should have id=3") - assert(branchRead(2).getString(1) == "c", "Row 3 should have data='c'") - assert(branchRead(2).isNullAt(2), "Row 3 should have date_col=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 From 1740f185d11427cc5a02843e1d308db4d11ef62d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 29 Oct 2025 17:57:37 -0400 Subject: [PATCH 59/89] Bump df50. --- native/Cargo.lock | 140 ++++++++++++++++++++-------------------------- 1 file changed, 60 insertions(+), 80 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 0609b8f34f..d69fcf1d90 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -44,9 +44,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.1.3" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +checksum = "ddd31a130427c27518df266943a5308ed92d4b226cc639f5a8f1002816174301" dependencies = [ "memchr", ] @@ -1329,18 +1329,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.50" +version = "4.5.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2cfd7bf8a6017ddaa4e32ffe7403d547790db06bd171c1c53926faab501623" +checksum = "4c26d721170e0295f191a69bd9a1f93efcdb0aff38684b61ab5750468972e5f5" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.50" +version = "4.5.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4c05b9e80c5ccd3a7ef080ad7b6ba7d6fc00a985b8b157197075677c82c7a0" +checksum = "75835f0c7bf681bfd05abe44e965760fea999a5286c6eb2d59883634fd02011a" dependencies = [ "anstyle", "clap_lex", @@ -2450,9 +2450,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" +checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" dependencies = [ "powerfmt", "serde_core", @@ -3204,7 +3204,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#feb8f05ec55f31b911e7cc6319585fe4570f7819" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#e90466bf8b4355570a68d54d69750e756020beb9" dependencies = [ "anyhow", "apache-avro", @@ -3258,9 +3258,9 @@ dependencies = [ [[package]] name = "icu_collections" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +checksum = "4c6b649701667bbe825c3b7e6388cb521c23d88644678e83c0c4d0a621a34b43" dependencies = [ "displaydoc", "potential_utf", @@ -3271,9 +3271,9 @@ dependencies = [ [[package]] name = "icu_locale_core" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +checksum = "edba7861004dd3714265b4db54a3c390e880ab658fec5f7db895fae2046b5bb6" dependencies = [ "displaydoc", "litemap", @@ -3284,11 +3284,10 @@ dependencies = [ [[package]] name = "icu_normalizer" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "5f6c8828b67bf8908d82127b2054ea1b4427ff0230ee9141c54251934ab1b599" dependencies = [ - "displaydoc", "icu_collections", "icu_normalizer_data", "icu_properties", @@ -3299,42 +3298,38 @@ dependencies = [ [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "7aedcccd01fc5fe81e6b489c15b247b8b0690feb23304303a9e560f37efc560a" [[package]] name = "icu_properties" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "e93fcd3157766c0c8da2f8cff6ce651a31f0810eaa1c51ec363ef790bbb5fb99" dependencies = [ - "displaydoc", "icu_collections", "icu_locale_core", "icu_properties_data", "icu_provider", - "potential_utf", "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "02845b3647bb045f1100ecd6480ff52f34c35f82d9880e029d329c21d1054899" [[package]] name = "icu_provider" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "85962cf0ce02e1e0a629cc34e7ca3e373ce20dda4c4d7294bbd0bf1fdb59e614" dependencies = [ "displaydoc", "icu_locale_core", - "stable_deref_trait", - "tinystr", "writeable", "yoke", "zerofrom", @@ -3512,9 +3507,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" +checksum = "b011eec8cc36da2aab2d5cff675ec18454fad408585853910a202391cf9f8e65" dependencies = [ "once_cell", "wasm-bindgen", @@ -3669,9 +3664,9 @@ checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "6373607a59f0be73a39b6fe456b8192fcc3585f602af20751600e974dd455e77" [[package]] name = "lock_api" @@ -4386,9 +4381,9 @@ checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" [[package]] name = "potential_utf" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" +checksum = "b73949432f5e2a09657003c25bca5e19a0e9c84f8058ca374f49e0ebe605af77" dependencies = [ "zerovec", ] @@ -5068,9 +5063,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "94182ad936a0c91c324cd46c6511b9510ed16af436d7b5bab34beab0afd55f7a" dependencies = [ "web-time", "zeroize", @@ -5078,9 +5073,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.7" +version = "0.103.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" +checksum = "2ffdfa2f5286e2247234e03f680868ac2815974dc39e00ea15adc445d0aafe52" dependencies = [ "aws-lc-rs", "ring", @@ -5731,9 +5726,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "42d3e9c45c09de15d06dd8acf5f4e0e399e85927b7f00711024eb7ae10fa4869" dependencies = [ "displaydoc", "zerovec", @@ -6106,9 +6101,9 @@ dependencies = [ [[package]] name = "wasm-bindgen" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" +checksum = "da95793dfc411fbbd93f5be7715b0578ec61fe87cb1a42b12eb625caa5c5ea60" dependencies = [ "cfg-if", "once_cell", @@ -6117,25 +6112,11 @@ dependencies = [ "wasm-bindgen-shared", ] -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.104" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" -dependencies = [ - "bumpalo", - "log", - "proc-macro2", - "quote", - "syn 2.0.108", - "wasm-bindgen-shared", -] - [[package]] name = "wasm-bindgen-futures" -version = "0.4.54" +version = "0.4.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" +checksum = "551f88106c6d5e7ccc7cd9a16f312dd3b5d36ea8b4954304657d5dfba115d4a0" dependencies = [ "cfg-if", "js-sys", @@ -6146,9 +6127,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" +checksum = "04264334509e04a7bf8690f2384ef5265f05143a4bff3889ab7a3269adab59c2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -6156,22 +6137,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" +checksum = "420bc339d9f322e562942d52e115d57e950d12d88983a14c79b86859ee6c7ebc" dependencies = [ + "bumpalo", "proc-macro2", "quote", "syn 2.0.108", - "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" +checksum = "76f218a38c84bcb33c25ec7059b07847d465ce0e0a76b995e134a45adcb6af76" dependencies = [ "unicode-ident", ] @@ -6191,9 +6172,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" +checksum = "3a1f95c0d03a47f4ae1f7a64643a6bb97465d9b740f0fa8f90ea33915c99a9a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -6568,9 +6549,9 @@ checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "9edde0db4769d2dc68579893f2306b26c6ecfbe0ef499b013d731b7b9247e0b9" [[package]] name = "wyz" @@ -6604,11 +6585,10 @@ dependencies = [ [[package]] name = "yoke" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "72d6e5c6afb84d73944e5cedb052c4680d5657337201555f9f2a16b7406d4954" dependencies = [ - "serde", "stable_deref_trait", "yoke-derive", "zerofrom", @@ -6616,9 +6596,9 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", @@ -6675,9 +6655,9 @@ checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +checksum = "2a59c17a5562d507e4b54960e8569ebee33bee890c70aa3fe7b97e85a9fd7851" dependencies = [ "displaydoc", "yoke", @@ -6686,9 +6666,9 @@ dependencies = [ [[package]] name = "zerovec" -version = "0.11.4" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "6c28719294829477f525be0186d13efa9a3c602f7ec202ca9e353d310fb9a002" dependencies = [ "yoke", "zerofrom", @@ -6697,9 +6677,9 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", From d9a5a1edecc3c6cbae801502763376441d52fe4c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 29 Oct 2025 20:24:30 -0400 Subject: [PATCH 60/89] Bump df50. --- native/Cargo.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index d69fcf1d90..8b241fc76b 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3204,7 +3204,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#e90466bf8b4355570a68d54d69750e756020beb9" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#b7f9f1acf75ee910fb4b85715ce9c4cc9a36cd3c" dependencies = [ "anyhow", "apache-avro", From f76cc99eab3cddc1ca6372d746a108456481b444 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 29 Oct 2025 21:12:20 -0400 Subject: [PATCH 61/89] Bump df50. --- native/Cargo.lock | 13 ++++++------- native/Cargo.toml | 4 ++-- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 8b241fc76b..d4c92bf23c 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -2605,7 +2605,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -3204,7 +3204,6 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#b7f9f1acf75ee910fb4b85715ce9c4cc9a36cd3c" dependencies = [ "anyhow", "apache-avro", @@ -3435,7 +3434,7 @@ checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -4636,7 +4635,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] @@ -5022,7 +5021,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -5578,7 +5577,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix 1.1.2", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -6233,7 +6232,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index 61b2338372..df71fbfd8a 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -53,8 +53,8 @@ object_store = { version = "0.12.3", features = ["gcp", "azure", "aws", "http"] url = "2.2" aws-config = "1.6.3" aws-credential-types = "1.2.7" -iceberg = { git = "https://github.com/mbutrovich/iceberg-rust", branch = "df50" } -iceberg-datafusion = { git = "https://github.com/mbutrovich/iceberg-rust", branch = "df50" } +iceberg = { path = "/Users/matt/git/iceberg-rust/crates/iceberg" } +iceberg-datafusion = { path = "/Users/matt/git/iceberg-rust/crates/datafusion_iceberg" } [profile.release] debug = true From f33fb389548d9d2b0f0807dc16d9f23f8b12f2d3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 29 Oct 2025 22:06:43 -0400 Subject: [PATCH 62/89] Bump df50. --- native/Cargo.lock | 13 +++++++------ native/Cargo.toml | 4 ++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index d4c92bf23c..d2298f8a91 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -2605,7 +2605,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -3204,6 +3204,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#5b396e7977571e7def782dbcf30c2ccaba824900" dependencies = [ "anyhow", "apache-avro", @@ -3434,7 +3435,7 @@ checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -4635,7 +4636,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -5021,7 +5022,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -5577,7 +5578,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix 1.1.2", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -6232,7 +6233,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index df71fbfd8a..61b2338372 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -53,8 +53,8 @@ object_store = { version = "0.12.3", features = ["gcp", "azure", "aws", "http"] url = "2.2" aws-config = "1.6.3" aws-credential-types = "1.2.7" -iceberg = { path = "/Users/matt/git/iceberg-rust/crates/iceberg" } -iceberg-datafusion = { path = "/Users/matt/git/iceberg-rust/crates/datafusion_iceberg" } +iceberg = { git = "https://github.com/mbutrovich/iceberg-rust", branch = "df50" } +iceberg-datafusion = { git = "https://github.com/mbutrovich/iceberg-rust", branch = "df50" } [profile.release] debug = true From 133772deddafdc709606707d8cdef9f3faee010e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 30 Oct 2025 13:01:51 -0400 Subject: [PATCH 63/89] Serialize PartitionSpec stuff. Fixes ~50 spark-extensions tests from Iceberg Java, down to 26 failures. --- native/Cargo.lock | 6 +- native/core/src/execution/planner.rs | 54 ++++++ native/proto/src/proto/operator.proto | 15 ++ .../apache/comet/serde/QueryPlanSerde.scala | 183 ++++++++++++++++-- 4 files changed, 234 insertions(+), 24 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index d2298f8a91..abc87857a2 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3204,7 +3204,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#5b396e7977571e7def782dbcf30c2ccaba824900" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#bc2bb2a9cee3783f1888d497a950de48aae4844f" dependencies = [ "anyhow", "apache-avro", @@ -5968,9 +5968,9 @@ checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "unicode-ident" -version = "1.0.20" +version = "1.0.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "462eeb75aeb73aea900253ce739c8e18a67423fadf006037cd3ff27e82748a06" +checksum = "9312f7c4f6ff9069b165498234ce8be658059c6728633667c526e27dc2cf1df5" [[package]] name = "unicode-segmentation" diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index facf241619..79bf8a5043 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2869,6 +2869,57 @@ fn parse_file_scan_tasks( ) .transpose()?; + // Parse partition data if available (for proper constant identification per Iceberg spec) + 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(), + ) { + // Parse the partition type schema + 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 + )) + })?; + + // Parse the partition data JSON with the proper schema + 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 + )) + })?; + + // Convert to Iceberg Literal using the partition type + 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(_) => None, + Err(_) => None, + } + } else { + None + }; + + // Parse partition spec if available + let partition_spec = if let Some(partition_spec_json) = + proto_task.partition_spec_json.as_ref() + { + match serde_json::from_str::(partition_spec_json) { + Ok(spec) => Some(Arc::new(spec)), + Err(_) => None, + } + } else { + None + }; + + let partition_spec_id = proto_task.partition_spec_id; + // Build FileScanTask matching iceberg-rust's structure Ok(iceberg::scan::FileScanTask { data_file_path: proto_task.data_file_path.clone(), @@ -2880,6 +2931,9 @@ fn parse_file_scan_tasks( project_field_ids: proto_task.project_field_ids.clone(), predicate: bound_predicate, deletes, + partition, + partition_spec_id, + partition_spec, }) }) .collect(); diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index c42966f825..671a3cdd00 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -158,6 +158,21 @@ message IcebergFileScanTask { // 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 spec ID for this file (used to lookup the PartitionSpec) + optional int32 partition_spec_id = 11; + + // 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) + optional string partition_spec_json = 13; } // Iceberg delete file for MOR tables (positional or equality deletes) 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 c9051b1ca3..30349a60e9 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1365,30 +1365,46 @@ object QueryPlanSerde extends Logging with CometExprShim { val value = getMethod.invoke(partitionStruct, Int.box(i), classOf[Object]) - // Get the source field ID from the partition spec + // Get the partition field and check its transform type val partitionField = fields.get(i) - 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) + + // Only inject partition values for IDENTITY transforms + // Per Iceberg spec: "Return the value from partition metadata + // if an Identity Transform exists" + // Non-identity transforms (bucket, truncate, year, etc.) + // store DERIVED values + // in partition metadata, not source column values. Source + // columns must be read from data files. + 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 + partitionMap(sourceFieldId.toString) = jsonValue + } } } @@ -1605,6 +1621,131 @@ object QueryPlanSerde extends Logging with CometExprShim { s"${e.getMessage}") } + // Extract partition data and spec ID for proper constant identification + try { + // Get partition spec from the task first + val specMethod = fileScanTaskClass.getMethod("spec") + val spec = specMethod.invoke(task) + + if (spec != null) { + val specIdMethod = spec.getClass.getMethod("specId") + val specId = specIdMethod.invoke(spec).asInstanceOf[Int] + taskBuilder.setPartitionSpecId(specId) + + // Serialize the entire PartitionSpec to JSON + 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) + + // Serialize partition type to JSON using Iceberg's SchemaParser + // Allows Rust to deserialize partition data with proper types + try { + // scalastyle:off classforname + val schemaParserClass = + Class.forName("org.apache.iceberg.SchemaParser") + val toJsonMethod = schemaParserClass.getMethod( + "toJson", + Class.forName("org.apache.iceberg.types.Type")) + // scalastyle:on classforname + val partitionTypeJson = toJsonMethod + .invoke(null, partitionType) + .asInstanceOf[String] + taskBuilder.setPartitionTypeJson(partitionTypeJson) + } catch { + case e: Exception => + logWarning( + s"Failed to serialize partition type to JSON: ${e.getMessage}") + } + + // Serialize partition data to JSON using Iceberg's StructLike + // Build JSON object with field IDs (not names) as keys + val fieldsMethod = partitionType.getClass.getMethod("fields") + val fields = fieldsMethod + .invoke(partitionType) + .asInstanceOf[java.util.List[_]] + + val jsonBuilder = new StringBuilder() + jsonBuilder.append("{") + + var first = true + val iter = fields.iterator() + var idx = 0 + while (iter.hasNext) { + val field = iter.next() + // Use field ID as key for proper Iceberg JSON deserialization + val fieldIdMethod = field.getClass.getMethod("fieldId") + val fieldId = fieldIdMethod.invoke(field).asInstanceOf[Int] + + // Get value from partition data at this index + 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 + + // Use field ID as the JSON key + jsonBuilder.append("\"").append(fieldId.toString).append("\":") + if (value == null) { + jsonBuilder.append("null") + } else { + // Simple JSON encoding for basic types + value match { + case s: String => + jsonBuilder.append("\"").append(s).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) + } else {} + } else {} + } catch { + case e: Exception => + logWarning( + "Failed to extract partition data from FileScanTask: " + + s"${e.getMessage}") + e.printStackTrace() + } + partitionBuilder.addFileScanTasks(taskBuilder.build()) } catch { case e: Exception => From bf1342f6f0040afa6f1523ee01f9cbcfb5f6b6cc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 30 Oct 2025 17:20:21 -0400 Subject: [PATCH 64/89] Bump df50. --- native/Cargo.lock | 39 ++++++++++++++++++++------------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index abc87857a2..e48b42ab29 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -638,9 +638,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.86.0" +version = "1.87.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a0abbfab841446cce6e87af853a3ba2cc1bc9afcd3f3550dd556c43d434c86d" +checksum = "f4af747ffcb5aa8da8be8f0679ef6940f1afdb8c2e10c36738c9ebeb8d17b95e" dependencies = [ "aws-credential-types", "aws-runtime", @@ -682,9 +682,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.88.0" +version = "1.89.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d30990923f4f675523c51eb1c0dec9b752fb267b36a61e83cbc219c9d86da715" +checksum = "928e87698cd916cf1efd5268148347269e6d2911028742c0061ff6261e639e3c" dependencies = [ "aws-credential-types", "aws-runtime", @@ -738,15 +738,16 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.62.4" +version = "0.62.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3feafd437c763db26aa04e0cc7591185d0961e64c61885bece0fb9d50ceac671" +checksum = "445d5d720c99eed0b4aa674ed00d835d9b1427dd73e04adaf2f94c6b2d6f9fca" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", "bytes", "bytes-utils", "futures-core", + "futures-util", "http 0.2.12", "http 1.3.1", "http-body 0.4.6", @@ -758,9 +759,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.3" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1053b5e587e6fa40ce5a79ea27957b04ba660baa02b28b7436f64850152234f1" +checksum = "623254723e8dfd535f566ee7b2381645f8981da086b5c4aa26c0c41582bb1d2c" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -782,9 +783,9 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.61.6" +version = "0.61.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cff418fc8ec5cadf8173b10125f05c2e7e1d46771406187b2c878557d4503390" +checksum = "2db31f727935fc63c6eeae8b37b438847639ec330a9161ece694efba257e0c54" dependencies = [ "aws-smithy-types", ] @@ -810,9 +811,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.9.3" +version = "1.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40ab99739082da5347660c556689256438defae3bcefd66c52b095905730e404" +checksum = "0bbe9d018d646b96c7be063dd07987849862b0e6d07c778aad7d93d1be6c1ef0" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -834,9 +835,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.1" +version = "1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3683c5b152d2ad753607179ed71988e8cfd52964443b4f74fd8e552d0bbfeb46" +checksum = "ec7204f9fd94749a7c53b26da1b961b4ac36bf070ef1e0b94bb09f79d4f6c193" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -851,9 +852,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.3" +version = "1.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f5b3a7486f6690ba25952cabf1e7d75e34d69eaff5081904a47bc79074d6457" +checksum = "25f535879a207fce0db74b679cfc3e91a3159c8144d717d55f5832aea9eef46e" dependencies = [ "base64-simd", "bytes", @@ -874,9 +875,9 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.11" +version = "0.60.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9c34127e8c624bc2999f3b657e749c1393bedc9cd97b92a804db8ced4d2e163" +checksum = "eab77cdd036b11056d2a30a7af7b775789fb024bf216acc13884c6c97752ae56" dependencies = [ "xmlparser", ] @@ -3204,7 +3205,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#bc2bb2a9cee3783f1888d497a950de48aae4844f" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#e86cc3eaf1f5a3d6d476638eb889066f4f59bed7" dependencies = [ "anyhow", "apache-avro", From caf21c5b8bbf38e6bad0482111cc48951dd499fe Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 30 Oct 2025 22:32:35 -0400 Subject: [PATCH 65/89] Bump df50. --- native/Cargo.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index e48b42ab29..3ec19e59c3 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -3205,7 +3205,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#e86cc3eaf1f5a3d6d476638eb889066f4f59bed7" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#3d8cd1b552b2b90b1e1b9621c2dae8341aaa7253" dependencies = [ "anyhow", "apache-avro", From a2021b5db5bab6240ea9858557a7058005699725 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 31 Oct 2025 10:39:14 -0400 Subject: [PATCH 66/89] Fall back on InMemoryFileIO tables (views). --- .../apache/comet/rules/CometScanRule.scala | 72 ++++++++++++++++++- 1 file changed, 69 insertions(+), 3 deletions(-) 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 315993fb75..4ad66e9a83 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -280,8 +280,44 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com s"${scanExec.scan.getClass.getSimpleName}: Schema not supported" } + // 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) { + val formatVersionSupported = if (schemaSupported && fileIOCompatible) { try { // table() is a protected method in SparkScan, // so we need getDeclaredMethod + setAccessible @@ -360,15 +396,45 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com allParquet = false } - // Check filesystem scheme + // Check filesystem scheme for data file val filePath = pathMethod.invoke(dataFile).toString - val scheme = new URI(filePath).getScheme + 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) { From 03afbbd3294bfc8cb34dd8b11fa5a812464c0484 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 31 Oct 2025 13:02:57 -0400 Subject: [PATCH 67/89] Fall back on truncate function. --- .../apache/comet/rules/CometScanRule.scala | 47 ++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) 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 4ad66e9a83..4bf14c249d 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -271,6 +271,10 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com "org.apache.iceberg.spark.source.SparkBatchQueryScan" => val fallbackReasons = new ListBuffer[String]() + // 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 = typeChecker.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons) @@ -578,9 +582,50 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com 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) { + complexTypePredicatesSupported && transformFunctionsSupported) { // Iceberg tables require type promotion for schema evolution (add/drop columns) SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") CometBatchScanExec( From 9ae3605c7db6776d529d6e4c2922adae50f954e0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 08:57:38 -0500 Subject: [PATCH 68/89] Add fuzz iceberg suite to CI again (it got lost when updating main) --- .github/workflows/pr_build_linux.yml | 1 + .github/workflows/pr_build_macos.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index fa1aebaa5d..153712b15b 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" diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 2d59bf8c9f..6e884baf64 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" From e3b0806cae7f5fa5aaa0dd94c09b73632f8446f6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 09:39:24 -0500 Subject: [PATCH 69/89] Apply #2675's partitioning fix to IcebergScanExec. --- native/core/src/execution/planner.rs | 24 ++++++++++-------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index c42c18ba24..817192e00f 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1403,31 +1403,27 @@ impl PhysicalPlanner { // Get metadata location from separate field let metadata_location = scan.metadata_location.clone(); - // Parse pre-planned FileScanTasks if provided (grouped by partition) + // Parse pre-planned FileScanTasks for this partition only // // NOTE: We no longer convert scan-level data_filters to predicates here. // Instead, each FileScanTask contains its own residual expression, which is // the result of Iceberg's ResidualEvaluator partially evaluating the scan // filter against that file's partition data. This per-file residual is what // gets used for row-group level filtering in the Parquet reader. + // + // Comet's native side corresponds to a single Spark partition, so we extract + // only this partition's FileScanTasks and pass them as partition 0 for execution. let file_task_groups = if !scan.file_partitions.is_empty() { - let mut task_groups: Vec> = - Vec::with_capacity(scan.file_partitions.len()); - for partition in &scan.file_partitions { - let tasks = parse_file_scan_tasks(&partition.file_scan_tasks)?; - task_groups.push(tasks); - } - Some(task_groups) + let tasks = parse_file_scan_tasks( + &scan.file_partitions[self.partition as usize].file_scan_tasks, + )?; + Some(vec![tasks]) // Single partition (partition 0) for execution } else { None }; - // Get num_partitions (default to 1 if not specified) - let num_partitions = if scan.num_partitions > 0 { - scan.num_partitions as usize - } else { - 1 - }; + // Always use 1 partition since we're only passing this partition's tasks + let num_partitions = 1; // Create IcebergScanExec let iceberg_scan = IcebergScanExec::new( From 2497ead9b46ced28d11eec2ab0c700843f70242b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 10:17:42 -0500 Subject: [PATCH 70/89] move IcebergScan serialization logic to a new file. --- .../apache/comet/serde/QueryPlanSerde.scala | 893 +----------------- .../org/apache/comet/serde/icebergScan.scala | 893 ++++++++++++++++++ 2 files changed, 895 insertions(+), 891 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/serde/icebergScan.scala 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 f2d02f3fdb..fd775d25a5 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -271,57 +271,6 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[VariancePop] -> CometVariancePop, classOf[VarianceSamp] -> CometVarianceSamp) - /** - * 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. - * - * @param hadoopProps - * Map of Hadoop configuration properties (fs.s3a.* keys) - * @return - * Map with keys transformed to Iceberg format (s3.* keys) - */ - private 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 // Ignore unrecognized per-bucket properties - } - } 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 - } - } - } - def supportedDataType(dt: DataType, allowComplex: Boolean = false): Boolean = dt match { case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | @@ -1116,641 +1065,13 @@ object QueryPlanSerde extends Logging with CometExprShim { None } - // Iceberg scan with native execution enabled - - // detected by CometBatchScanExec wrapping SparkBatchQueryScan + // Iceberg scan with native execution enabled case scan: CometBatchScanExec if CometConf.COMET_ICEBERG_NATIVE_ENABLED.get() && CometConf.COMET_EXEC_ENABLED.get() && scan.wrapped.scan.getClass.getName == "org.apache.iceberg.spark.source.SparkBatchQueryScan" => - 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 - } - - // Set metadata location - 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) - } - - // Determine number of partitions from Iceberg's output partitioning - // TODO: Add a test case for both partitioning schemes - val numParts = scan.wrapped.outputPartitioning match { - case p: KeyGroupedPartitioning => - p.numPartitions - case _ => - scan.wrapped.inputRDD.getNumPartitions - } - - // Set number of partitions - icebergScanBuilder.setNumPartitions(numParts) - - // 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 - // (Same logic as the previous CometIcebergNativeScanExec case) - var actualNumPartitions = 0 - 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[_]] - - import scala.jdk.CollectionConverters._ - 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 - // These values are needed to populate partition columns - // that don't exist in data files - 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 - // PartitionScanTask has a spec() method - // 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 - // Format: {"field_id": value, ...} - 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 - // Per Iceberg spec: "Return the value from partition metadata - // if an Identity Transform exists" - // Non-identity transforms (bucket, truncate, year, etc.) - // store DERIVED values - // in partition metadata, not source column values. Source - // columns must be read from data files. - 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) - var schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] - - // Inject partition values into schema if present - partitionJsonOpt.foreach { partitionJson => - schemaJson = - injectPartitionValuesIntoSchemaJson(schemaJson, partitionJson) - } - - taskBuilder.setSchemaJson(schemaJson) - - // Build field ID mapping from the schema we're using - // (scan or task schema) - 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. - // This handles cases where scan.output has columns not present in - // some task schemas. - 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 data and spec ID for proper constant identification - try { - // Get partition spec from the task first - val specMethod = fileScanTaskClass.getMethod("spec") - val spec = specMethod.invoke(task) - - if (spec != null) { - val specIdMethod = spec.getClass.getMethod("specId") - val specId = specIdMethod.invoke(spec).asInstanceOf[Int] - taskBuilder.setPartitionSpecId(specId) - - // Serialize the entire PartitionSpec to JSON - 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) - - // Serialize partition type to JSON using Iceberg's SchemaParser - // Allows Rust to deserialize partition data with proper types - try { - // scalastyle:off classforname - val schemaParserClass = - Class.forName("org.apache.iceberg.SchemaParser") - val toJsonMethod = schemaParserClass.getMethod( - "toJson", - Class.forName("org.apache.iceberg.types.Type")) - // scalastyle:on classforname - val partitionTypeJson = toJsonMethod - .invoke(null, partitionType) - .asInstanceOf[String] - taskBuilder.setPartitionTypeJson(partitionTypeJson) - } catch { - case e: Exception => - logWarning( - s"Failed to serialize partition type to JSON: ${e.getMessage}") - } - - // Serialize partition data to JSON using Iceberg's StructLike - // Build JSON object with field IDs (not names) as keys - val fieldsMethod = partitionType.getClass.getMethod("fields") - val fields = fieldsMethod - .invoke(partitionType) - .asInstanceOf[java.util.List[_]] - - val jsonBuilder = new StringBuilder() - jsonBuilder.append("{") - - var first = true - val iter = fields.iterator() - var idx = 0 - while (iter.hasNext) { - val field = iter.next() - // Use field ID as key for proper Iceberg JSON deserialization - val fieldIdMethod = field.getClass.getMethod("fieldId") - val fieldId = fieldIdMethod.invoke(field).asInstanceOf[Int] - - // Get value from partition data at this index - 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 - - // Use field ID as the JSON key - jsonBuilder.append("\"").append(fieldId.toString).append("\":") - if (value == null) { - jsonBuilder.append("null") - } else { - // Simple JSON encoding for basic types - value match { - case s: String => - jsonBuilder.append("\"").append(s).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) - } else {} - } else {} - } catch { - case e: Exception => - logWarning( - "Failed to extract partition data from FileScanTask: " + - s"${e.getMessage}") - e.printStackTrace() - } - - 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) - actualNumPartitions += 1 - } - case _ => - // Not a DataSourceRDD, cannot extract tasks - } - } catch { - case e: Exception => - logWarning(s"Failed to extract FileScanTasks from Iceberg scan RDD: ${e.getMessage}") - } - - val numPartitions = - if (actualNumPartitions > 0) actualNumPartitions else numParts - icebergScanBuilder.setNumPartitions(numPartitions) - - builder.clearChildren() - Some(builder.setIcebergScan(icebergScanBuilder).build()) + IcebergScanSerde.serializeIcebergScan(scan, builder) case FilterExec(condition, child) if CometConf.COMET_EXEC_FILTER_ENABLED.get(conf) => val cond = exprToProto(condition, child.output) @@ -2388,216 +1709,6 @@ object QueryPlanSerde extends Logging with CometExprShim { nativeScanBuilder.addFilePartitions(partitionBuilder.build()) } - /** - * 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. - */ - private 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) - } - - /** - * Injects partition values into Iceberg schema JSON as "initial-default" values. - * - * For Hive-style partitioned tables migrated to Iceberg, partition values are stored in - * directory structure, not in data files. This function adds those values to the schema so - * iceberg-rust's RecordBatchTransformer can populate partition columns. - * - * @param schemaJson - * The Iceberg schema as JSON string - * @param partitionJson - * The partition values as JSON string: {"field_id": value, ...} - * @return - * Modified schema JSON with initial-default values injected - */ - private def injectPartitionValuesIntoSchemaJson( - schemaJson: String, - partitionJson: String): String = { - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - - try { - // Parse both JSONs - implicit val formats: Formats = DefaultFormats - val schemaValue = parse(schemaJson) - val partitionMap = parse(partitionJson).extract[Map[String, JValue]] - - // Transform the schema fields to inject initial-default values - val transformedSchema = schemaValue.transformField { case ("fields", JArray(fields)) => - val updatedFields = fields.map { - case fieldObj: JObject => - // Check if this field has a partition value - fieldObj \ "id" match { - case JInt(fieldId) => - partitionMap.get(fieldId.toString) match { - case Some(partitionValue) => - // Add "initial-default" to this field - fieldObj merge JObject("initial-default" -> partitionValue) - case None => - fieldObj - } - case _ => - fieldObj - } - case other => other - } - ("fields", JArray(updatedFields)) - } - - // Serialize back to JSON - compact(render(transformedSchema)) - } catch { - case e: Exception => - logWarning(s"Failed to inject partition values into schema JSON: ${e.getMessage}") - // Return original schema on error - schemaJson - } - } } 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..33d90cae39 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -0,0 +1,893 @@ +/* + * 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.catalyst.plans.physical.KeyGroupedPartitioning +import org.apache.spark.sql.comet.CometBatchScanExec +import org.apache.spark.sql.comet.CometIcebergNativeScanExec +import org.apache.spark.sql.types._ + +import org.apache.comet.CometConf +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) + } + + /** + * Injects partition values into Iceberg schema JSON as "initial-default" values. + * + * For Hive-style partitioned tables migrated to Iceberg, partition values are stored in + * directory structure, not in data files. This function adds those values to the schema so + * iceberg-rust's RecordBatchTransformer can populate partition columns. + */ + def injectPartitionValuesIntoSchemaJson(schemaJson: String, partitionJson: String): String = { + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + + try { + // Parse both JSONs + implicit val formats: Formats = DefaultFormats + val schemaValue = parse(schemaJson) + val partitionMap = parse(partitionJson).extract[Map[String, JValue]] + + // Transform the schema fields to inject initial-default values + val transformedSchema = schemaValue.transformField { case ("fields", JArray(fields)) => + val updatedFields = fields.map { + case fieldObj: JObject => + // Check if this field has a partition value + fieldObj \ "id" match { + case JInt(fieldId) => + partitionMap.get(fieldId.toString) match { + case Some(partitionValue) => + // Add "initial-default" to this field + fieldObj merge JObject("initial-default" -> partitionValue) + case None => + fieldObj + } + case _ => + fieldObj + } + case other => other + } + ("fields", JArray(updatedFields)) + } + + // Serialize back to JSON + compact(render(transformedSchema)) + } catch { + case e: Exception => + logWarning(s"Failed to inject partition values into schema JSON: ${e.getMessage}") + schemaJson + } + } + + /** + * 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) + } + + // Determine number of partitions from Iceberg's output partitioning + val numParts = scan.wrapped.outputPartitioning match { + case p: KeyGroupedPartitioning => + p.numPartitions + case _ => + scan.wrapped.inputRDD.getNumPartitions + } + + icebergScanBuilder.setNumPartitions(numParts) + + // 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 + var actualNumPartitions = 0 + 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) + var schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] + + // Inject partition values into schema if present + partitionJsonOpt.foreach { partitionJson => + schemaJson = + injectPartitionValuesIntoSchemaJson(schemaJson, partitionJson) + } + + 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 data and spec ID for proper constant identification + try { + // Get partition spec from the task first + val specMethod = fileScanTaskClass.getMethod("spec") + val spec = specMethod.invoke(task) + + if (spec != null) { + val specIdMethod = spec.getClass.getMethod("specId") + val specId = specIdMethod.invoke(spec).asInstanceOf[Int] + taskBuilder.setPartitionSpecId(specId) + + // Serialize the entire PartitionSpec to JSON + 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) + + // Serialize partition type to JSON using Iceberg's SchemaParser + try { + // scalastyle:off classforname + val schemaParserClass = + Class.forName("org.apache.iceberg.SchemaParser") + val toJsonMethod = schemaParserClass.getMethod( + "toJson", + Class.forName("org.apache.iceberg.types.Type")) + // scalastyle:on classforname + val partitionTypeJson = toJsonMethod + .invoke(null, partitionType) + .asInstanceOf[String] + taskBuilder.setPartitionTypeJson(partitionTypeJson) + } catch { + case e: Exception => + logWarning( + s"Failed to serialize partition type to JSON: ${e.getMessage}") + } + + // Serialize partition data to JSON using Iceberg's StructLike + val fieldsMethod = partitionType.getClass.getMethod("fields") + val fields = fieldsMethod + .invoke(partitionType) + .asInstanceOf[java.util.List[_]] + + 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("\"") + 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() + } + + 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) + actualNumPartitions += 1 + } + case _ => + } + } catch { + case e: Exception => + logWarning(s"Failed to extract FileScanTasks from Iceberg scan RDD: ${e.getMessage}") + } + + val numPartitions = + if (actualNumPartitions > 0) actualNumPartitions else numParts + icebergScanBuilder.setNumPartitions(numPartitions) + + builder.clearChildren() + Some(builder.setIcebergScan(icebergScanBuilder).build()) + } +} From cf09648010508e65f5879f01bfe3b62b16fe3283 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 10:37:12 -0500 Subject: [PATCH 71/89] separate checks and serialization logic, reduce redundant checks --- .../org/apache/comet/rules/CometExecRule.scala | 5 ++--- .../org/apache/comet/rules/CometScanRule.scala | 13 +++++++++++++ .../org/apache/comet/serde/QueryPlanSerde.scala | 8 +++----- 3 files changed, 18 insertions(+), 8 deletions(-) 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 4a09d3d2e9..91a768892c 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -162,10 +162,9 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { 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 CometConf.COMET_ICEBERG_NATIVE_ENABLED.get(conf) && - CometConf.COMET_EXEC_ENABLED.get(conf) && - scan.wrapped.scan.getClass.getName == + if scan.wrapped.scan.getClass.getName == "org.apache.iceberg.spark.source.SparkBatchQueryScan" => // Extract metadata location for CometIcebergNativeScanExec try { 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 4bf14c249d..86c5944ac4 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -271,6 +271,19 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com "org.apache.iceberg.spark.source.SparkBatchQueryScan" => val fallbackReasons = new ListBuffer[String]() + // Native Iceberg scan requires both configs to be enabled + if (!COMET_ICEBERG_NATIVE_ENABLED.get()) { + fallbackReasons += s"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 += s"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") 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 fd775d25a5..f39155b992 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, NormalizeNaNAndZero} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet._ @@ -51,7 +50,7 @@ import org.apache.comet.expressions._ import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.CometParquetUtils import org.apache.comet.serde.ExprOuterClass.{AggExpr, Expr, ScalarFunc} -import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, BuildSide, JoinType, Operator, SparkStructField} +import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, BuildSide, JoinType, Operator} import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto} import org.apache.comet.serde.Types.{DataType => ProtoDataType} import org.apache.comet.serde.Types.DataType._ @@ -1066,10 +1065,9 @@ object QueryPlanSerde extends Logging with CometExprShim { } // Iceberg scan with native execution enabled + // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometExecRule case scan: CometBatchScanExec - if CometConf.COMET_ICEBERG_NATIVE_ENABLED.get() && - CometConf.COMET_EXEC_ENABLED.get() && - scan.wrapped.scan.getClass.getName == + if scan.wrapped.scan.getClass.getName == "org.apache.iceberg.spark.source.SparkBatchQueryScan" => IcebergScanSerde.serializeIcebergScan(scan, builder) From 1f86a8e8287b1a2fb3db9d72c112eba2805af2b5 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 10:59:18 -0500 Subject: [PATCH 72/89] remove num_partitions serialization --- .../src/execution/operators/iceberg_scan.rs | 43 ++++++++----------- native/core/src/execution/planner.rs | 21 +++++---- native/proto/src/proto/operator.proto | 5 +-- .../apache/comet/rules/CometScanRule.scala | 4 +- .../org/apache/comet/serde/icebergScan.scala | 18 -------- 5 files changed, 30 insertions(+), 61 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index c470774c3c..b45f1e2c92 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -61,7 +61,7 @@ pub struct IcebergScanExec { /// Catalog-specific configuration for FileIO catalog_properties: HashMap, /// Pre-planned file scan tasks from Scala, grouped by Spark partition - file_task_groups: Option>>, + file_task_groups: Vec>, /// Metrics metrics: ExecutionPlanMetricsSet, } @@ -71,12 +71,14 @@ impl IcebergScanExec { metadata_location: String, schema: SchemaRef, catalog_properties: HashMap, - file_task_groups: Option>>, - num_partitions: usize, + file_task_groups: Vec>, ) -> Result { // Don't normalize - just use the schema as provided by Spark let output_schema = schema; + // Derive num_partitions from file_task_groups length + let num_partitions = file_task_groups.len(); + let plan_properties = Self::compute_properties(Arc::clone(&output_schema), num_partitions); let metrics = ExecutionPlanMetricsSet::new(); @@ -136,25 +138,16 @@ impl ExecutionPlan for IcebergScanExec { context: Arc, ) -> DFResult { // Execute pre-planned tasks from Scala (planning happens via Iceberg's Java API) - if let Some(ref task_groups) = self.file_task_groups { - if partition < task_groups.len() { - let tasks = &task_groups[partition]; - - return self.execute_with_tasks(tasks.clone(), partition, context); - } else { - return Err(DataFusionError::Execution(format!( - "IcebergScanExec: Partition index {} out of range (only {} task groups available)", - partition, - task_groups.len() - ))); - } + 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() + ))) } - - Err(DataFusionError::Execution(format!( - "IcebergScanExec: No FileScanTasks provided for partition {}. \ - All scan planning must happen on the Scala side.", - partition - ))) } fn metrics(&self) -> Option { @@ -510,13 +503,11 @@ where impl DisplayAs for IcebergScanExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + let num_tasks: usize = self.file_task_groups.iter().map(|g| g.len()).sum(); write!( f, - "IcebergScanExec: metadata_location={}, num_tasks={:?}", - self.metadata_location, - self.file_task_groups - .as_ref() - .map(|groups| groups.iter().map(|g| g.len()).sum::()) + "IcebergScanExec: metadata_location={}, num_tasks={}", + self.metadata_location, num_tasks ) } } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 817192e00f..fa62028eeb 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1413,17 +1413,17 @@ impl PhysicalPlanner { // // Comet's native side corresponds to a single Spark partition, so we extract // only this partition's FileScanTasks and pass them as partition 0 for execution. - let file_task_groups = if !scan.file_partitions.is_empty() { - let tasks = parse_file_scan_tasks( - &scan.file_partitions[self.partition as usize].file_scan_tasks, - )?; - Some(vec![tasks]) // Single partition (partition 0) for execution - } else { - None - }; - // Always use 1 partition since we're only passing this partition's tasks - let num_partitions = 1; + // If file_partitions is empty, this is a logic error in Scala serialization + 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]; // Single partition (partition 0) for execution // Create IcebergScanExec let iceberg_scan = IcebergScanExec::new( @@ -1431,7 +1431,6 @@ impl PhysicalPlanner { required_schema, catalog_properties, file_task_groups, - num_partitions, )?; Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index bf3a37fe34..c9960f2392 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -118,11 +118,8 @@ message IcebergScan { // Pre-planned file scan tasks grouped by Spark partition repeated IcebergFilePartition file_partitions = 3; - // Number of Spark partitions for DataFusion parallelism - int32 num_partitions = 4; - // Table metadata file path for FileIO initialization - string metadata_location = 5; + string metadata_location = 4; } // Groups FileScanTasks for a single Spark partition 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 86c5944ac4..a3ab451e49 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -273,13 +273,13 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com // Native Iceberg scan requires both configs to be enabled if (!COMET_ICEBERG_NATIVE_ENABLED.get()) { - fallbackReasons += s"Native Iceberg scan disabled because " + + 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 += s"Native Iceberg scan disabled because " + + fallbackReasons += "Native Iceberg scan disabled because " + s"${COMET_EXEC_ENABLED.key} is not enabled" return withInfos(scanExec, fallbackReasons.toSet) } diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index 33d90cae39..8641670779 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -23,12 +23,10 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning import org.apache.spark.sql.comet.CometBatchScanExec import org.apache.spark.sql.comet.CometIcebergNativeScanExec import org.apache.spark.sql.types._ -import org.apache.comet.CometConf import org.apache.comet.objectstore.NativeConfig import org.apache.comet.serde.OperatorOuterClass.{Operator, SparkStructField} import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType} @@ -325,16 +323,6 @@ object IcebergScanSerde extends Logging { icebergScanBuilder.putCatalogProperties(key, value) } - // Determine number of partitions from Iceberg's output partitioning - val numParts = scan.wrapped.outputPartitioning match { - case p: KeyGroupedPartitioning => - p.numPartitions - case _ => - scan.wrapped.inputRDD.getNumPartitions - } - - icebergScanBuilder.setNumPartitions(numParts) - // Set required_schema from output scan.output.foreach { attr => val field = SparkStructField @@ -402,7 +390,6 @@ object IcebergScanSerde extends Logging { } // Extract FileScanTasks from the InputPartitions in the RDD - var actualNumPartitions = 0 try { scan.wrapped.inputRDD match { case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => @@ -874,7 +861,6 @@ object IcebergScanSerde extends Logging { val builtPartition = partitionBuilder.build() icebergScanBuilder.addFilePartitions(builtPartition) - actualNumPartitions += 1 } case _ => } @@ -883,10 +869,6 @@ object IcebergScanSerde extends Logging { logWarning(s"Failed to extract FileScanTasks from Iceberg scan RDD: ${e.getMessage}") } - val numPartitions = - if (actualNumPartitions > 0) actualNumPartitions else numParts - icebergScanBuilder.setNumPartitions(numPartitions) - builder.clearChildren() Some(builder.setIcebergScan(icebergScanBuilder).build()) } From c5ce7595b1b6a029604578096a77e8c0390f0ef3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 11:17:10 -0500 Subject: [PATCH 73/89] clean up planner.rs deserialization and comments --- native/core/src/execution/planner.rs | 54 +++++++++++++++------------- 1 file changed, 29 insertions(+), 25 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index fa62028eeb..bb268136c5 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1388,11 +1388,6 @@ impl PhysicalPlanner { let required_schema: SchemaRef = convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); - // No projection needed - Spark already projects columns before the scan. - // The required_schema contains exactly the columns we need to read. - // Passing None tells iceberg-rust to use the full required_schema, - // which is the same as passing Some([0, 1, 2, ...]). - // Extract catalog configuration let catalog_properties: HashMap = scan .catalog_properties @@ -2752,18 +2747,13 @@ fn parse_file_scan_tasks( let schema_ref = Arc::new(schema); - // Parse file format - let data_file_format = match proto_task.data_file_format.as_str() { - "PARQUET" => iceberg::spec::DataFileFormat::Parquet, - "AVRO" => iceberg::spec::DataFileFormat::Avro, - "ORC" => iceberg::spec::DataFileFormat::Orc, - other => { - return Err(ExecutionError::GeneralError(format!( - "Unsupported file format: {}", - other - ))); - } - }; + // CometScanRule validates all files are PARQUET 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; // Parse delete files for MOR (Merge-On-Read) table support // Delete files allow Iceberg to track deletions separately from data files: @@ -2783,8 +2773,8 @@ fn parse_file_scan_tasks( "POSITION_DELETES" => iceberg::spec::DataContentType::PositionDeletes, "EQUALITY_DELETES" => iceberg::spec::DataContentType::EqualityDeletes, other => { - return Err(ExecutionError::GeneralError(format!( - "Unsupported delete content type: {}", + return Err(GeneralError(format!( + "Invalid delete content type '{}'. This indicates a bug in Scala serialization.", other ))) } @@ -2865,8 +2855,18 @@ fn parse_file_scan_tasks( ) { Ok(Some(iceberg::spec::Literal::Struct(s))) => Some(s), Ok(None) => None, - Ok(_) => None, - Err(_) => 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 @@ -2876,10 +2876,14 @@ fn parse_file_scan_tasks( let partition_spec = if let Some(partition_spec_json) = proto_task.partition_spec_json.as_ref() { - match serde_json::from_str::(partition_spec_json) { - Ok(spec) => Some(Arc::new(spec)), - Err(_) => None, - } + let spec = serde_json::from_str::(partition_spec_json) + .map_err(|e| { + GeneralError(format!( + "Failed to parse partition spec JSON: {}", + e + )) + })?; + Some(Arc::new(spec)) } else { None }; From b53fa78be89f9bfe4b6974bf1e5089717b26669d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 11:40:17 -0500 Subject: [PATCH 74/89] clean up iceberg_scan.rs comments --- .../src/execution/operators/iceberg_scan.rs | 118 ++++++------------ 1 file changed, 41 insertions(+), 77 deletions(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index b45f1e2c92..ff6648850b 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -47,9 +47,9 @@ use datafusion::datasource::schema_adapter::SchemaAdapterFactory; use datafusion_comet_spark_expr::EvalMode; use datafusion_datasource::file_stream::FileStreamMetrics; -/// Native Iceberg scan operator that uses iceberg-rust to read Iceberg tables. +/// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. /// -/// Bypasses Spark's DataSource V2 API by reading pre-planned FileScanTasks directly. +/// Executes pre-planned FileScanTasks for efficient parallel scanning. #[derive(Debug)] pub struct IcebergScanExec { /// Iceberg table metadata location for FileIO initialization @@ -60,7 +60,7 @@ pub struct IcebergScanExec { plan_properties: PlanProperties, /// Catalog-specific configuration for FileIO catalog_properties: HashMap, - /// Pre-planned file scan tasks from Scala, grouped by Spark partition + /// Pre-planned file scan tasks, grouped by partition file_task_groups: Vec>, /// Metrics metrics: ExecutionPlanMetricsSet, @@ -73,12 +73,8 @@ impl IcebergScanExec { catalog_properties: HashMap, file_task_groups: Vec>, ) -> Result { - // Don't normalize - just use the schema as provided by Spark let output_schema = schema; - - // Derive num_partitions from file_task_groups length let num_partitions = file_task_groups.len(); - let plan_properties = Self::compute_properties(Arc::clone(&output_schema), num_partitions); let metrics = ExecutionPlanMetricsSet::new(); @@ -94,7 +90,6 @@ impl IcebergScanExec { } fn compute_properties(schema: SchemaRef, num_partitions: usize) -> PlanProperties { - // Matches Spark partition count to ensure proper parallelism PlanProperties::new( EquivalenceProperties::new(schema), Partitioning::UnknownPartitioning(num_partitions), @@ -137,7 +132,6 @@ impl ExecutionPlan for IcebergScanExec { partition: usize, context: Arc, ) -> DFResult { - // Execute pre-planned tasks from Scala (planning happens via Iceberg's Java API) if partition < self.file_task_groups.len() { let tasks = &self.file_task_groups[partition]; self.execute_with_tasks(tasks.clone(), partition, context) @@ -165,14 +159,9 @@ impl IcebergScanExec { context: Arc, ) -> DFResult { let output_schema = Arc::clone(&self.output_schema); - - // Create FileIO synchronously let file_io = Self::load_file_io(&self.catalog_properties, &self.metadata_location)?; - - // Get batch size from context let batch_size = context.session_config().batch_size(); - // Create metrics for this partition (wraps both baseline and file stream metrics) let metrics = IcebergScanMetrics::new(&self.metrics, partition); // Create parallel file stream that overlaps opening next file with reading current file @@ -279,7 +268,6 @@ impl IcebergFileStream { }) } - /// Start opening the next file fn start_next_file( &mut self, ) -> Option>> { @@ -292,37 +280,30 @@ impl IcebergFileStream { let schema = Arc::clone(&self.schema); Some(Box::pin(async move { - // Create a single-task stream let task_stream = futures::stream::iter(vec![Ok(task)]).boxed(); - // Create reader with optimizations let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io) .with_batch_size(batch_size) .with_row_selection_enabled(true) .build(); - // Read the task let stream = reader.read(task_stream).map_err(|e| { DataFusionError::Execution(format!("Failed to read Iceberg task: {}", e)) })?; - // Clone schema for transformation let target_schema = Arc::clone(&schema); - // Apply schema adaptation to each batch (same approach as regular Parquet scans) - // This handles differences in field names ("element" vs "item", "key_value" vs "entries") - // and metadata (PARQUET:field_id) just like regular Parquet scans + // 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| { - // Use SparkSchemaAdapter to transform the 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)); - // Apply the schema mapping let result = match adapter.map_schema(file_schema.as_ref()) { Ok((schema_mapper, _projection)) => { schema_mapper.map_batch(batch).map_err(|e| { @@ -348,7 +329,6 @@ impl IcebergFileStream { loop { match &mut self.state { FileStreamState::Idle => { - // Start opening the first file self.metrics.file_stream.time_opening.start(); match self.start_next_file() { Some(future) => { @@ -357,31 +337,26 @@ impl IcebergFileStream { None => return Poll::Ready(None), } } - FileStreamState::Opening { future } => { - // Wait for file to open - match ready!(future.poll_unpin(cx)) { - Ok(stream) => { - // File opened, start reading and open next file in parallel - 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::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) { - // Next file is ready, store it match result { Ok(stream) => { *next = Some(Box::pin(futures::future::ready(Ok(stream)))); @@ -394,7 +369,6 @@ impl IcebergFileStream { } } - // Poll current stream for next batch and record metrics match ready!(self .metrics .baseline @@ -409,39 +383,30 @@ impl IcebergFileStream { return Poll::Ready(Some(result)); } None => { - // Current file is done, move to next file if available self.metrics.file_stream.time_scanning_until_data.stop(); self.metrics.file_stream.time_scanning_total.stop(); match next.take() { - Some(mut next_future) => { - // Check if next file is already opened - 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 => { - // Still opening, wait for it - self.state = FileStreamState::Opening { - future: next_future, - }; - } + 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 => { - // No more files return Poll::Ready(None); } } @@ -473,8 +438,8 @@ impl RecordBatchStream for IcebergFileStream { } } -/// Wrapper around iceberg-rust's stream that reports Comet's schema without validation. -/// This avoids strict schema checks that would reject batches with PARQUET:field_id metadata. +/// 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, @@ -496,7 +461,6 @@ where S: Stream> + Unpin, { fn schema(&self) -> SchemaRef { - // Return Comet's schema, not the batch schema with metadata Arc::clone(&self.schema) } } From 58e3b3a0d077519b2443cfb2bcd6a9818901f531 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 12:25:56 -0500 Subject: [PATCH 75/89] clean up CometIcebergNativeScanExec comments --- .../comet/CometIcebergNativeScanExec.scala | 57 +++---------------- 1 file changed, 7 insertions(+), 50 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index cb436da04a..4907a90ff9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -34,41 +34,12 @@ import com.google.common.base.Objects import org.apache.comet.serde.OperatorOuterClass.Operator /** - * Comet fully native Iceberg scan node for DataSource V2. + * Native Iceberg scan operator that delegates file reading to iceberg-rust. * - * Replaces Spark's Iceberg BatchScanExec by extracting FileScanTasks from Iceberg's planning and - * serializing them to protobuf for native execution. All catalog access and planning happens in - * Spark's Iceberg integration; the Rust side uses iceberg-rust's FileIO and ArrowReader to read - * data files based on the pre-planned FileScanTasks. Catalog properties are used to configure the - * FileIO (for credentials, regions, etc.) - * - * **How FileScanTask Serialization Works:** - * - * This implementation follows the same pattern as CometNativeScanExec for PartitionedFiles: - * - * 1. **At Planning Time (on Driver):** - * - CometScanRule creates CometIcebergNativeScanExec with originalPlan (BatchScanExec) - * - originalPlan.inputRDD is a DataSourceRDD containing DataSourceRDDPartition objects - * - Each partition contains InputPartition objects (from Iceberg's planInputPartitions()) - * - Each InputPartition wraps a ScanTaskGroup containing FileScanTask objects - * - * 2. **During Serialization (in QueryPlanSerde.operator2Proto):** - * - When serializing CometIcebergNativeScanExec, we iterate through ALL RDD partitions - * - For each partition, extract InputPartitions and their FileScanTasks using reflection - * - Serialize each FileScanTask (file path, start, length, delete files) into protobuf - * - This happens ONCE on the driver, not per-worker - * - * 3. **At Execution Time (on Workers):** - * - The serialized plan (with all FileScanTasks) is sent to workers - * - Standard CometNativeExec.doExecuteColumnar() flow executes the native plan - * - Rust receives IcebergScan operator with FileScanTasks for ALL partitions - * - Each worker reads only the tasks for its partition index - * - * All tasks are extracted at planning time because the RDD and partitions exist on the driver, - * and Iceberg has already assigned tasks to partitions. - * - * **Filters:** When a filter is on top of this scan, both are serialized together and executed as - * one unit. No special RDD or per-partition logic needed. + * 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, @@ -76,18 +47,13 @@ case class CometIcebergNativeScanExec( @transient override val originalPlan: BatchScanExec, override val serializedPlanOpt: SerializedPlan, metadataLocation: String, - catalogProperties: Map[String, String], // TODO: Extract for authentication numPartitions: Int) extends CometLeafExec { override val supportsColumnar: Boolean = true - // FileScanTasks are serialized at planning time in QueryPlanSerde.operator2Proto() - override val nodeName: String = "CometIcebergNativeScan" - // FileScanTasks are serialized at planning time and grouped by partition. - // Rust uses the partition index to select the correct task group. override lazy val outputPartitioning: Partitioning = UnknownPartitioning(numPartitions) @@ -151,15 +117,9 @@ case class CometIcebergNativeScanExec( */ private class ImmutableSQLMetric(metricType: String) extends SQLMetric(metricType, 0) { - // Override merge to do nothing - planning metrics are not updated during execution - override def merge(other: AccumulatorV2[Long, Long]): Unit = { - // Do nothing - this metric's value is immutable - } + override def merge(other: AccumulatorV2[Long, Long]): Unit = {} - // Override reset to do nothing - planning metrics should never be reset - override def reset(): Unit = { - // Do nothing - this metric's value is immutable - } + override def reset(): Unit = {} } override lazy val metrics: Map[String, SQLMetric] = { @@ -204,7 +164,6 @@ case class CometIcebergNativeScanExec( originalPlan.doCanonicalize(), SerializedPlan(None), metadataLocation, - catalogProperties, numPartitions) } @@ -215,7 +174,6 @@ case class CometIcebergNativeScanExec( obj match { case other: CometIcebergNativeScanExec => this.metadataLocation == other.metadataLocation && - this.catalogProperties == other.catalogProperties && this.output == other.output && this.serializedPlanOpt == other.serializedPlanOpt && this.numPartitions == other.numPartitions @@ -310,7 +268,6 @@ object CometIcebergNativeScanExec { scanExec, SerializedPlan(None), metadataLocation, - Map.empty, // TODO: Extract catalog properties for authentication numParts) scanExec.logicalLink.foreach(exec.setLogicalLink) From fca2dd74930651b7e5131bc199c89bfcb5585936 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 12:35:24 -0500 Subject: [PATCH 76/89] clean up more scala comments --- common/src/main/scala/org/apache/comet/CometConf.scala | 7 +++---- .../main/scala/org/apache/spark/sql/comet/operators.scala | 1 + 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index c2b3e8607d..f3f9f13f3b 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -127,10 +127,9 @@ object CometConf extends ShimCometConf { conf("spark.comet.scan.icebergNative.enabled") .category(CATEGORY_SCAN) .doc( - "Whether to enable native Iceberg scan using iceberg-rust. When enabled, Comet will " + - "replace Spark's Iceberg BatchScanExec with CometIcebergNativeScanExec. Iceberg " + - "planning is performed by Spark, and the resulting FileScanTasks are serialized " + - "and passed to the native execution layer for reading data files.") + "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) 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 77fa4d1741..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 From 6f77912078da51b2001c8773e974b52a0fbd3697 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 12:45:07 -0500 Subject: [PATCH 77/89] Clean up planner.rs comments. --- native/core/src/execution/planner.rs | 113 ++++----------------------- 1 file changed, 16 insertions(+), 97 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index bb268136c5..a4b3b88130 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1384,32 +1384,17 @@ impl PhysicalPlanner { )) } OpStruct::IcebergScan(scan) => { - // Convert schema let required_schema: SchemaRef = convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); - // Extract catalog configuration let catalog_properties: HashMap = scan .catalog_properties .iter() .map(|(k, v)| (k.clone(), v.clone())) .collect(); - // Get metadata location from separate field let metadata_location = scan.metadata_location.clone(); - // Parse pre-planned FileScanTasks for this partition only - // - // NOTE: We no longer convert scan-level data_filters to predicates here. - // Instead, each FileScanTask contains its own residual expression, which is - // the result of Iceberg's ResidualEvaluator partially evaluating the scan - // filter against that file's partition data. This per-file residual is what - // gets used for row-group level filtering in the Parquet reader. - // - // Comet's native side corresponds to a single Spark partition, so we extract - // only this partition's FileScanTasks and pass them as partition 0 for execution. - - // If file_partitions is empty, this is a logic error in Scala serialization debug_assert!( !scan.file_partitions.is_empty(), "IcebergScan must have at least one file partition. This indicates a bug in Scala serialization." @@ -1418,9 +1403,8 @@ impl PhysicalPlanner { let tasks = parse_file_scan_tasks( &scan.file_partitions[self.partition as usize].file_scan_tasks, )?; - let file_task_groups = vec![tasks]; // Single partition (partition 0) for execution + let file_task_groups = vec![tasks]; - // Create IcebergScanExec let iceberg_scan = IcebergScanExec::new( metadata_location, required_schema, @@ -2725,21 +2709,16 @@ fn convert_spark_types_to_arrow_schema( arrow_schema } -/// Parse protobuf FileScanTasks into iceberg-rust FileScanTask objects. +/// Converts protobuf FileScanTasks from Scala into iceberg-rust FileScanTask objects. /// -/// This converts the protobuf representation of Iceberg file scan tasks (passed from Scala) -/// into native iceberg-rust FileScanTask objects that can be executed directly. -/// -/// Each task contains a residual expression which is the result of Iceberg's ResidualEvaluator -/// partially evaluating the scan filter against that file's partition data. This residual is -/// used for row-group level filtering during Parquet scanning. +/// 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| { - // Parse schema from JSON (already contains partition values injected on Scala side) 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)) @@ -2747,7 +2726,7 @@ fn parse_file_scan_tasks( let schema_ref = Arc::new(schema); - // CometScanRule validates all files are PARQUET before serialization + // CometScanRule validates format before serialization debug_assert_eq!( proto_task.data_file_format.as_str(), "PARQUET", @@ -2755,16 +2734,8 @@ fn parse_file_scan_tasks( ); let data_file_format = iceberg::spec::DataFileFormat::Parquet; - // Parse delete files for MOR (Merge-On-Read) table support - // Delete files allow Iceberg to track deletions separately from data files: - // - Positional deletes: Specify exact row positions to skip in data files - // - Equality deletes: Specify column values that should be filtered out - // These deletes are automatically applied by iceberg-rust's ArrowReader during scanning. - // - // NOTE: Spark's DataSource V2 API does not expose delete files through InputPartitions. - // Spark applies MOR deletes during query planning before creating FileScanTasks, so - // task.deletes() typically returns empty even for MOR tables. This is expected. - + // Spark's DataSource V2 API does not expose delete files through InputPartitions. + // Spark applies MOR deletes during query planning before creating FileScanTasks. let deletes: Vec = proto_task .delete_files .iter() @@ -2793,22 +2764,12 @@ fn parse_file_scan_tasks( }) .collect::, ExecutionError>>()?; - // Extract and convert residual expression from this task - // - // The residual is a Spark expression that represents the remaining filter - // conditions after partition pruning. Iceberg's ResidualEvaluator creates - // these by partially evaluating the scan filter against each file's partition data. - // - // Process: - // 1. Residuals are serialized from Scala with binding=false, so we receive - // UnboundReference (name-based) expressions rather than BoundReference (index-based) - // 2. Convert Spark expression to Iceberg predicate (name-based) - // 3. Bind the predicate to this file's schema for row-group filtering + // 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 expression to Iceberg predicate convert_spark_expr_to_predicate(residual_expr) }) .map( @@ -2825,12 +2786,10 @@ fn parse_file_scan_tasks( ) .transpose()?; - // Parse partition data if available (for proper constant identification per Iceberg spec) 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(), ) { - // Parse the partition type schema let partition_type: iceberg::spec::StructType = serde_json::from_str(partition_type_json).map_err(|e| { ExecutionError::GeneralError(format!( @@ -2839,7 +2798,6 @@ fn parse_file_scan_tasks( )) })?; - // Parse the partition data JSON with the proper schema let partition_data_value: serde_json::Value = serde_json::from_str(partition_json) .map_err(|e| { ExecutionError::GeneralError(format!( @@ -2848,7 +2806,6 @@ fn parse_file_scan_tasks( )) })?; - // Convert to Iceberg Literal using the partition type match iceberg::spec::Literal::try_from_json( partition_data_value, &iceberg::spec::Type::Struct(partition_type), @@ -2872,7 +2829,6 @@ fn parse_file_scan_tasks( None }; - // Parse partition spec if available let partition_spec = if let Some(partition_spec_json) = proto_task.partition_spec_json.as_ref() { @@ -2890,7 +2846,6 @@ fn parse_file_scan_tasks( let partition_spec_id = proto_task.partition_spec_id; - // Build FileScanTask matching iceberg-rust's structure Ok(iceberg::scan::FileScanTask { data_file_path: proto_task.data_file_path.clone(), start: proto_task.start, @@ -3154,49 +3109,17 @@ fn literal_to_array_ref( // Spark Expression to Iceberg Predicate Conversion // ============================================================================ // -// Predicate Pushdown Design: -// =========================== -// For row-group level filtering in Parquet files, predicates follow this conversion path: +// 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. // -// 1. Iceberg Expression (Java) - extracted from Spark's Iceberg scan planning -// | [~100 lines: Scala convertIcebergExpression() in QueryPlanSerde.scala] -// v -// 2. Spark Catalyst Expression - standard Spark filter representation -// | [~3000 lines: existing exprToProto() infrastructure in QueryPlanSerde.scala] -// v -// 3. Protobuf Spark Expr - serialized for Rust communication (data_filters field) -// | [~200 lines: convert_spark_filters_to_iceberg_predicate() below] -// v -// 4. Iceberg Predicate (Rust) - bound to each file's schema for row-group pruning +// Conversion path: +// Iceberg Expression (Java) -> Spark Catalyst Expression -> Protobuf -> Iceberg Predicate (Rust) // -// This design leverages Comet's existing expression serialization infrastructure -// (which handles hundreds of expression types) rather than implementing a separate -// Iceberg Java -> Iceberg Rust serialization path. Any new expression types added -// to Comet automatically flow through to Iceberg predicate pushdown. -// -// Supported predicates: =, !=, <, <=, >, >=, IS NULL, IS NOT NULL, IN, AND, OR, NOT -// -// Note: NOT IN predicates are intentionally skipped here because iceberg-rust's -// RowGroupMetricsEvaluator::not_in() always returns MIGHT_MATCH (never prunes). -// These are handled by post-scan CometFilter instead. -// -// Example: For query "SELECT * FROM table WHERE id > 10 AND status = 'active'" -// 1. Iceberg planning extracts: [GreaterThan(id, 10), EqualTo(status, "active")] -// 2. Converted to Catalyst: [GreaterThan(AttributeRef("id"), Literal(10)), ...] -// 3. Serialized to protobuf: [Expr{gt: BinaryExpr{left: ..., right: ...}}, ...] -// 4. Converted here to: Predicate::And(Predicate::Binary(...), Predicate::Binary(...)) -// 5. Bound to schema and passed to iceberg-rust's ArrowReader for row-group filtering -// -// Performance improvement from predicate pushdown: -// - Fewer row groups read (Parquet statistics used for pruning) -// - Reduced I/O (skip row groups that can't match) -// - Faster query execution +// 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. -/// This allows predicate pushdown into Parquet readers. -/// -/// Residuals are serialized with binding=false, so they contain UnboundReference (name-based) -/// rather than BoundReference (index-based), which makes conversion straightforward. fn convert_spark_expr_to_predicate( expr: &spark_expression::Expr, ) -> Option { @@ -3336,7 +3259,6 @@ fn convert_binary_to_predicate( let left_ref = left.as_ref()?; let right_ref = right.as_ref()?; - // Try left as column, right as literal if let (Some(column), Some(datum)) = ( extract_column_reference(left_ref), extract_literal_as_datum(right_ref), @@ -3346,7 +3268,6 @@ fn convert_binary_to_predicate( )); } - // Try right as column, left as literal (reverse operator) if let (Some(datum), Some(column)) = ( extract_literal_as_datum(left_ref), extract_column_reference(right_ref), @@ -3392,12 +3313,10 @@ fn extract_literal_as_datum(expr: &spark_expression::Expr) -> Option { - // Check for null literals if literal.is_null { return None; } - // Match on the oneof value field match &literal.value { Some(spark_expression::literal::Value::IntVal(v)) => { Some(iceberg::spec::Datum::int(*v)) From b88facf35cab255f012d964d5c08a052d8e6af0d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 12:47:00 -0500 Subject: [PATCH 78/89] clean up more planner.rs comments --- native/core/src/execution/planner.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index a4b3b88130..a88ffba4f8 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2734,8 +2734,6 @@ fn parse_file_scan_tasks( ); let data_file_format = iceberg::spec::DataFileFormat::Parquet; - // Spark's DataSource V2 API does not expose delete files through InputPartitions. - // Spark applies MOR deletes during query planning before creating FileScanTasks. let deletes: Vec = proto_task .delete_files .iter() From 47894e72b82e657f6eb987c1f9c8ae2e90f119bc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 12:54:52 -0500 Subject: [PATCH 79/89] fix conflicts with main --- docs/source/user-guide/latest/configs.md | 2 +- native/Cargo.lock | 1208 +++++++++++++++++++--- 2 files changed, 1063 insertions(+), 147 deletions(-) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 558d1117a0..eb5e7eee45 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -32,7 +32,7 @@ Comet provides the following configuration settings. | `spark.comet.convert.parquet.enabled` | When enabled, data from Spark (non-native) Parquet v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and `spark.comet.exec.enabled` need to be enabled. | false | | `spark.comet.scan.allowIncompatible` | Some Comet scan implementations are not currently fully compatible with Spark for all datatypes. Set this config to true to allow them anyway. For more information, refer to the [Comet Compatibility Guide](https://datafusion.apache.org/comet/user-guide/compatibility.html). | false | | `spark.comet.scan.enabled` | Whether to enable native scans. When this is turned on, Spark will use Comet to read supported data sources (currently only Parquet is supported natively). Note that to enable native vectorized execution, both this config and `spark.comet.exec.enabled` need to be enabled. | true | -| `spark.comet.scan.icebergNative.enabled` | Whether to enable native Iceberg scan using iceberg-rust. When enabled, Comet will replace Spark's Iceberg BatchScanExec with CometIcebergNativeScanExec. Iceberg planning is performed by Spark, and the resulting FileScanTasks are serialized and passed to the native execution layer for reading data files. | false | +| `spark.comet.scan.icebergNative.enabled` | Whether to enable native Iceberg table scan using iceberg-rust. When enabled, Iceberg tables are read directly through native execution, bypassing Spark's DataSource V2 API for better performance. | false | | `spark.comet.scan.preFetch.enabled` | Whether to enable pre-fetching feature of CometScan. | false | | `spark.comet.scan.preFetch.threadNum` | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | | `spark.comet.sparkToColumnar.enabled` | Whether to enable Spark to Arrow columnar conversion. When this is turned on, Comet will convert operators in `spark.comet.sparkToColumnar.supportedOperatorList` into Arrow columnar format before processing. | false | diff --git a/native/Cargo.lock b/native/Cargo.lock index 929af8499f..a93972966a 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -17,6 +17,17 @@ version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" +[[package]] +name = "ahash" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" +dependencies = [ + "getrandom 0.2.16", + "once_cell", + "version_check", +] + [[package]] name = "ahash" version = "0.8.12" @@ -33,9 +44,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.1.3" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +checksum = "ddd31a130427c27518df266943a5308ed92d4b226cc639f5a8f1002816174301" dependencies = [ "memchr", ] @@ -97,12 +108,43 @@ version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" +[[package]] +name = "apache-avro" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a033b4ced7c585199fb78ef50fca7fe2f444369ec48080c5fd072efa1a03cc7" +dependencies = [ + "bigdecimal", + "bon", + "digest", + "log", + "miniz_oxide", + "num-bigint", + "quad-rand", + "rand 0.9.2", + "regex-lite", + "serde", + "serde_bytes", + "serde_json", + "strum 0.27.2", + "strum_macros 0.27.2", + "thiserror 2.0.17", + "uuid", + "zstd", +] + [[package]] name = "arc-swap" version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" +[[package]] +name = "array-init" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc" + [[package]] name = "arrayref" version = "0.3.9" @@ -156,7 +198,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-buffer", "arrow-data", "arrow-schema", @@ -254,7 +296,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.12.0", "lexical-core", "memchr", "num", @@ -306,7 +348,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-data", @@ -331,6 +373,12 @@ dependencies = [ "regex-syntax", ] +[[package]] +name = "as-any" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0f477b951e452a0b6b4a10b53ccd569042d1d01729b519e02074a9c0958a063" + [[package]] name = "assertables" version = "9.8.2" @@ -360,6 +408,23 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "async-compression" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" +dependencies = [ + "bzip2 0.5.2", + "flate2", + "futures-core", + "memchr", + "pin-project-lite", + "tokio", + "xz2", + "zstd", + "zstd-safe", +] + [[package]] name = "async-executor" version = "1.13.3" @@ -884,8 +949,15 @@ dependencies = [ "num-bigint", "num-integer", "num-traits", + "serde", ] +[[package]] +name = "bimap" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "230c5f1ca6a325a32553f8640d31ac9b49f2411e901e427570154868b46da4f7" + [[package]] name = "bindgen" version = "0.64.0" @@ -940,6 +1012,18 @@ version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" +[[package]] +name = "bitvec" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" +dependencies = [ + "funty", + "radium", + "tap", + "wyz", +] + [[package]] name = "blake2" version = "0.10.6" @@ -984,6 +1068,54 @@ dependencies = [ "piper", ] +[[package]] +name = "bon" +version = "3.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebeb9aaf9329dff6ceb65c689ca3db33dbf15f324909c60e4e5eef5701ce31b1" +dependencies = [ + "bon-macros", + "rustversion", +] + +[[package]] +name = "bon-macros" +version = "3.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77e9d642a7e3a318e37c2c9427b5a6a48aa1ad55dcd986f3034ab2239045a645" +dependencies = [ + "darling 0.21.3", + "ident_case", + "prettyplease", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.108", +] + +[[package]] +name = "borsh" +version = "1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad8646f98db542e39fc66e68a20b2144f6a732636df7c2354e74645faaa433ce" +dependencies = [ + "borsh-derive", + "cfg_aliases", +] + +[[package]] +name = "borsh-derive" +version = "1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdd1d3c0c2f5833f22386f252fe8ed005c7f59fdcddeef025c01b4c3b9fd9ac3" +dependencies = [ + "once_cell", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.108", +] + [[package]] name = "brotli" version = "8.0.2" @@ -1011,6 +1143,28 @@ version = "3.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "46c5e41b57b8bba42a04676d81cb89e9ee8e859a1a66f80a5a72e1cb76b34d43" +[[package]] +name = "bytecheck" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23cdc57ce23ac53c931e88a43d06d070a6fd142f2617be5855eb75efc9beb1c2" +dependencies = [ + "bytecheck_derive", + "ptr_meta", + "simdutf8", +] + +[[package]] +name = "bytecheck_derive" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3db406d29fbcd95542e92559bed4d8ad92636d1ca8b3b72ede10b4bcc010e659" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "bytemuck" version = "1.24.0" @@ -1039,6 +1193,34 @@ dependencies = [ "either", ] +[[package]] +name = "bzip2" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49ecfb22d906f800d4fe833b6282cf4dc1c298f5057ca0b5445e5c209735ca47" +dependencies = [ + "bzip2-sys", +] + +[[package]] +name = "bzip2" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a53fac24f34a81bc9954b5d6cfce0c21e18ec6959f44f56e8e90e4bb7c346c" +dependencies = [ + "libbz2-rs-sys", +] + +[[package]] +name = "bzip2-sys" +version = "0.1.13+1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225bff33b2141874fe80d71e07d6eec4f85c5c216453dd96388240f96e1acc14" +dependencies = [ + "cc", + "pkg-config", +] + [[package]] name = "cast" version = "0.3.0" @@ -1091,8 +1273,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "145052bdd345b87320e369255277e3fb5152762ad123a901ef5c262dd38fe8d2" dependencies = [ "iana-time-zone", + "js-sys", "num-traits", "serde", + "wasm-bindgen", "windows-link", ] @@ -1146,18 +1330,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.50" +version = "4.5.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2cfd7bf8a6017ddaa4e32ffe7403d547790db06bd171c1c53926faab501623" +checksum = "4c26d721170e0295f191a69bd9a1f93efcdb0aff38684b61ab5750468972e5f5" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.50" +version = "4.5.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4c05b9e80c5ccd3a7ef080ad7b6ba7d6fc00a985b8b157197075677c82c7a0" +checksum = "75835f0c7bf681bfd05abe44e965760fea999a5286c6eb2d59883634fd02011a" dependencies = [ "anstyle", "clap_lex", @@ -1194,8 +1378,8 @@ version = "7.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e0d05af1e006a2407bedef5af410552494ce5be9090444dbbcb57258c1af3d56" dependencies = [ - "strum", - "strum_macros", + "strum 0.26.3", + "strum_macros 0.26.4", "unicode-width", ] @@ -1208,6 +1392,12 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "const-oid" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" + [[package]] name = "const-random" version = "0.1.18" @@ -1268,6 +1458,15 @@ dependencies = [ "libc", ] +[[package]] +name = "crc32c" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a47af21622d091a8f0fb295b88bc886ac74efcc613efc19f5d0b21de5c89e47" +dependencies = [ + "rustc_version", +] + [[package]] name = "crc32fast" version = "1.5.0" @@ -1312,6 +1511,15 @@ dependencies = [ "itertools 0.13.0", ] +[[package]] +name = "crossbeam-channel" +version = "0.5.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82b8f8f868b36967f9606790d1903570de9ceaf870a7bf9fbbd3016d636a2cb2" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-deque" version = "0.8.6" @@ -1374,6 +1582,76 @@ dependencies = [ "memchr", ] +[[package]] +name = "darling" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc7f46116c46ff9ab3eb1597a45688b6715c6e628b5c133e288e709a29bcb4ee" +dependencies = [ + "darling_core 0.20.11", + "darling_macro 0.20.11", +] + +[[package]] +name = "darling" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cdf337090841a411e2a7f3deb9187445851f91b309c0c0a29e05f74a00a48c0" +dependencies = [ + "darling_core 0.21.3", + "darling_macro 0.21.3", +] + +[[package]] +name = "darling_core" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d00b9596d185e565c2207a0b01f8bd1a135483d02d9b7b0a54b11da8d53412e" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.108", +] + +[[package]] +name = "darling_core" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1247195ecd7e3c85f83c8d2a366e4210d588e802133e1e355180a9870b517ea4" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.108", +] + +[[package]] +name = "darling_macro" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" +dependencies = [ + "darling_core 0.20.11", + "quote", + "syn 2.0.108", +] + +[[package]] +name = "darling_macro" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" +dependencies = [ + "darling_core 0.21.3", + "quote", + "syn 2.0.108", +] + [[package]] name = "dashmap" version = "6.1.0" @@ -1431,7 +1709,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "regex", "sqlparser", "tempfile", @@ -1505,12 +1783,14 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", + "datafusion-datasource", "datafusion-functions-nested", "datafusion-spark", "futures", "hdfs-sys", "hdrs", "hex", + "iceberg", "itertools 0.14.0", "jni", "lazy_static", @@ -1529,9 +1809,10 @@ dependencies = [ "pprof", "procfs", "prost", - "rand", + "rand 0.9.2", "regex", "reqwest", + "serde_json", "simd-adler32", "snap", "tempfile", @@ -1594,7 +1875,7 @@ dependencies = [ "futures", "hex", "num", - "rand", + "rand 0.9.2", "regex", "thiserror 2.0.17", "tokio", @@ -1607,7 +1888,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5a83760d9a13122d025fbdb1d5d5aaf93dd9ada5e90ea229add92aa30898b2d1" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "arrow-ipc", "base64", @@ -1615,7 +1896,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "hex", - "indexmap", + "indexmap 2.12.0", "libc", "log", "object_store", @@ -1644,8 +1925,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" dependencies = [ "arrow", + "async-compression", "async-trait", "bytes", + "bzip2 0.6.1", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -1656,16 +1939,20 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", + "flate2", "futures", "glob", "itertools 0.14.0", "log", "object_store", "parquet", - "rand", + "rand 0.9.2", "tempfile", "tokio", + "tokio-util", "url", + "xz2", + "zstd", ] [[package]] @@ -1748,7 +2035,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "tokio", ] @@ -1774,7 +2061,7 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand", + "rand 0.9.2", "tempfile", "url", ] @@ -1794,7 +2081,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap", + "indexmap 2.12.0", "paste", "serde_json", "sqlparser", @@ -1808,7 +2095,7 @@ checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" dependencies = [ "arrow", "datafusion-common", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "paste", ] @@ -1835,7 +2122,7 @@ dependencies = [ "itertools 0.14.0", "log", "md-5", - "rand", + "rand 0.9.2", "regex", "sha2", "unicode-segmentation", @@ -1848,7 +2135,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07331fc13603a9da97b74fd8a273f4238222943dffdbbed1c4c6f862a30105bf" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-doc", @@ -1869,7 +2156,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b5951e572a8610b89968a09b5420515a121fbc305c0258651f318dc07c97ab17" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -1965,7 +2252,7 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-physical-expr", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "log", "regex", @@ -1978,7 +2265,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c8668103361a272cbbe3a61f72eca60c9b7c706e87cc3565bcf21e2b277b84f6" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr", @@ -1987,7 +2274,7 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2016,7 +2303,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6652fe7b5bf87e85ed175f571745305565da2c0b599d98e697bcbedc7baa47c3" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2049,7 +2336,7 @@ version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2f7f778a1a838dec124efb96eae6144237d546945587557c9e6936b3414558c" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow", "arrow-ord", "arrow-schema", @@ -2066,7 +2353,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2147,7 +2434,7 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap", + "indexmap 2.12.0", "log", "regex", "sqlparser", @@ -2164,11 +2451,43 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" +checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" dependencies = [ "powerfmt", + "serde_core", +] + +[[package]] +name = "derive_builder" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "507dfb09ea8b7fa618fcf76e953f4f5e192547945816d5358edffe39f6f94947" +dependencies = [ + "derive_builder_macro", +] + +[[package]] +name = "derive_builder_core" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d5bcf7b024d6835cfb3d473887cd966994907effbe9227e8c8219824d06c4e8" +dependencies = [ + "darling 0.20.11", + "proc-macro2", + "quote", + "syn 2.0.108", +] + +[[package]] +name = "derive_builder_macro" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" +dependencies = [ + "derive_builder_core", + "syn 2.0.108", ] [[package]] @@ -2205,6 +2524,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", + "const-oid", "crypto-common", "subtle", ] @@ -2220,12 +2540,33 @@ dependencies = [ "syn 2.0.108", ] +[[package]] +name = "dissimilar" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8975ffdaa0ef3661bfe02dbdcc06c9f829dfafe6a3c474de366a8d5e44276921" + +[[package]] +name = "dlv-list" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "442039f5147480ba31067cb00ada1adae6892028e40e45fc5de7b7df6dcc1b5f" +dependencies = [ + "const-random", +] + [[package]] name = "dunce" version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" +[[package]] +name = "dyn-clone" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0881ea181b1df73ff77ffaaf9c7544ecc11e82fba9b5f27b262a3c73a332555" + [[package]] name = "either" version = "1.15.0" @@ -2295,6 +2636,16 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "expect-test" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63af43ff4431e848fb47472a920f14fa71c24de13255a5692e93d4e90302acb0" +dependencies = [ + "dissimilar", + "once_cell", +] + [[package]] name = "fastrand" version = "2.3.0" @@ -2401,6 +2752,12 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" +[[package]] +name = "funty" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" + [[package]] name = "futures" version = "0.3.31" @@ -2576,7 +2933,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap", + "indexmap 2.12.0", "slab", "tokio", "tokio-util", @@ -2595,13 +2952,22 @@ dependencies = [ "zerocopy", ] +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash 0.7.8", +] + [[package]] name = "hashbrown" version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" dependencies = [ - "ahash", + "ahash 0.8.12", "allocator-api2", ] @@ -2837,38 +3203,92 @@ dependencies = [ ] [[package]] -name = "icu_collections" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" -dependencies = [ - "displaydoc", - "potential_utf", - "yoke", - "zerofrom", - "zerovec", -] - -[[package]] -name = "icu_locale_core" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +name = "iceberg" +version = "0.7.0" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#3d8cd1b552b2b90b1e1b9621c2dae8341aaa7253" dependencies = [ - "displaydoc", - "litemap", - "tinystr", - "writeable", - "zerovec", + "anyhow", + "apache-avro", + "array-init", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-ord", + "arrow-schema", + "arrow-select", + "arrow-string", + "as-any", + "async-trait", + "backon", + "base64", + "bimap", + "bytes", + "chrono", + "derive_builder", + "expect-test", + "fnv", + "futures", + "itertools 0.13.0", + "moka", + "murmur3", + "num-bigint", + "once_cell", + "opendal", + "ordered-float 4.6.0", + "parquet", + "rand 0.8.5", + "reqsign", + "reqwest", + "roaring", + "rust_decimal", + "serde", + "serde_bytes", + "serde_derive", + "serde_json", + "serde_repr", + "serde_with", + "strum 0.27.2", + "thrift", + "tokio", + "typed-builder", + "url", + "uuid", + "zstd", ] [[package]] -name = "icu_normalizer" -version = "2.0.0" +name = "icu_collections" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c6b649701667bbe825c3b7e6388cb521c23d88644678e83c0c4d0a621a34b43" +dependencies = [ + "displaydoc", + "potential_utf", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locale_core" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "edba7861004dd3714265b4db54a3c390e880ab658fec5f7db895fae2046b5bb6" dependencies = [ "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_normalizer" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f6c8828b67bf8908d82127b2054ea1b4427ff0230ee9141c54251934ab1b599" +dependencies = [ "icu_collections", "icu_normalizer_data", "icu_properties", @@ -2879,42 +3299,38 @@ dependencies = [ [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "7aedcccd01fc5fe81e6b489c15b247b8b0690feb23304303a9e560f37efc560a" [[package]] name = "icu_properties" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "e93fcd3157766c0c8da2f8cff6ce651a31f0810eaa1c51ec363ef790bbb5fb99" dependencies = [ - "displaydoc", "icu_collections", "icu_locale_core", "icu_properties_data", "icu_provider", - "potential_utf", "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "02845b3647bb045f1100ecd6480ff52f34c35f82d9880e029d329c21d1054899" [[package]] name = "icu_provider" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "85962cf0ce02e1e0a629cc34e7ca3e373ce20dda4c4d7294bbd0bf1fdb59e614" dependencies = [ "displaydoc", "icu_locale_core", - "stable_deref_trait", - "tinystr", "writeable", "yoke", "zerofrom", @@ -2922,6 +3338,12 @@ dependencies = [ "zerovec", ] +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + [[package]] name = "idna" version = "1.1.0" @@ -2943,6 +3365,17 @@ dependencies = [ "icu_properties", ] +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", + "serde", +] + [[package]] name = "indexmap" version = "2.12.0" @@ -2951,6 +3384,8 @@ checksum = "6717a8d2a5a929a1a2eb43a12812498ed141a0bcfb7e8f7844fbdbe4303bba9f" dependencies = [ "equivalent", "hashbrown 0.16.0", + "serde", + "serde_core", ] [[package]] @@ -2959,8 +3394,8 @@ version = "0.11.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ - "ahash", - "indexmap", + "ahash 0.8.12", + "indexmap 2.12.0", "is-terminal", "itoa", "log", @@ -3073,9 +3508,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" +checksum = "b011eec8cc36da2aab2d5cff675ec18454fad408585853910a202391cf9f8e65" dependencies = [ "once_cell", "wasm-bindgen", @@ -3159,6 +3594,12 @@ dependencies = [ "lexical-util", ] +[[package]] +name = "libbz2-rs-sys" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" + [[package]] name = "libc" version = "0.2.177" @@ -3224,9 +3665,9 @@ checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "6373607a59f0be73a39b6fe456b8192fcc3585f602af20751600e974dd455e77" [[package]] name = "lock_api" @@ -3270,7 +3711,7 @@ dependencies = [ "log-mdc", "mock_instant", "parking_lot", - "rand", + "rand 0.9.2", "serde", "serde-value", "serde_json", @@ -3297,6 +3738,17 @@ dependencies = [ "twox-hash", ] +[[package]] +name = "lzma-sys" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + [[package]] name = "md-5" version = "0.10.6" @@ -3364,12 +3816,39 @@ version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dce6dd36094cac388f119d2e9dc82dc730ef91c32a6222170d630e5414b956e6" +[[package]] +name = "moka" +version = "0.12.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8261cd88c312e0004c1d51baad2980c66528dfdb2bee62003e643a4d8f86b077" +dependencies = [ + "async-lock", + "crossbeam-channel", + "crossbeam-epoch", + "crossbeam-utils", + "equivalent", + "event-listener 5.4.1", + "futures-util", + "parking_lot", + "portable-atomic", + "rustc_version", + "smallvec", + "tagptr", + "uuid", +] + [[package]] name = "multimap" version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" +[[package]] +name = "murmur3" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9252111cf132ba0929b6f8e030cac2a24b507f3a4d6db6fb2896f27b354c714b" + [[package]] name = "nix" version = "0.26.4" @@ -3413,6 +3892,7 @@ checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", + "serde", ] [[package]] @@ -3481,6 +3961,16 @@ dependencies = [ "libm", ] +[[package]] +name = "num_cpus" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91df4bbde75afed763b708b7eee1e8e7651e02d97f6d5dd763e89367e957b23b" +dependencies = [ + "hermit-abi", + "libc", +] + [[package]] name = "object" version = "0.37.3" @@ -3512,7 +4002,7 @@ dependencies = [ "parking_lot", "percent-encoding", "quick-xml 0.38.3", - "rand", + "rand 0.9.2", "reqwest", "ring", "rustls-pemfile", @@ -3566,6 +4056,7 @@ dependencies = [ "base64", "bytes", "chrono", + "crc32c", "futures", "getrandom 0.2.16", "hdrs", @@ -3575,6 +4066,7 @@ dependencies = [ "md-5", "percent-encoding", "quick-xml 0.38.3", + "reqsign", "reqwest", "serde", "serde_json", @@ -3597,6 +4089,25 @@ dependencies = [ "num-traits", ] +[[package]] +name = "ordered-float" +version = "4.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7bb71e1b3fa6ca1c61f383464aaf2bb0e2f8e772a1f01d486832464de363b951" +dependencies = [ + "num-traits", +] + +[[package]] +name = "ordered-multimap" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49203cdcae0030493bad186b28da2fa25645fa276a51b6fec8010d281e02ef79" +dependencies = [ + "dlv-list", + "hashbrown 0.14.5", +] + [[package]] name = "outref" version = "0.5.2" @@ -3638,7 +4149,7 @@ version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" dependencies = [ - "ahash", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-cast", @@ -3681,7 +4192,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.12.0", "simdutf8", "uuid", ] @@ -3739,7 +4250,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ "fixedbitset", - "indexmap", + "indexmap 2.12.0", ] [[package]] @@ -3750,7 +4261,7 @@ checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap", + "indexmap 2.12.0", "serde", ] @@ -3863,11 +4374,17 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "portable-atomic" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" + [[package]] name = "potential_utf" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" +checksum = "b73949432f5e2a09657003c25bca5e19a0e9c84f8058ca374f49e0ebe605af77" dependencies = [ "zerovec", ] @@ -3919,6 +4436,15 @@ dependencies = [ "syn 2.0.108", ] +[[package]] +name = "proc-macro-crate" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" +dependencies = [ + "toml_edit", +] + [[package]] name = "proc-macro2" version = "1.0.103" @@ -4004,6 +4530,32 @@ dependencies = [ "prost", ] +[[package]] +name = "ptr_meta" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0738ccf7ea06b608c10564b31debd4f5bc5e197fc8bfe088f68ae5ce81e7a4f1" +dependencies = [ + "ptr_meta_derive", +] + +[[package]] +name = "ptr_meta_derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b845dbfca988fa33db069c0e230574d15a3088f147a87b64c7589eb662c9ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "quad-rand" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" + [[package]] name = "quick-xml" version = "0.26.0" @@ -4013,6 +4565,16 @@ dependencies = [ "memchr", ] +[[package]] +name = "quick-xml" +version = "0.37.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quick-xml" version = "0.38.3" @@ -4052,7 +4614,7 @@ dependencies = [ "bytes", "getrandom 0.3.4", "lru-slab", - "rand", + "rand 0.9.2", "ring", "rustc-hash 2.1.1", "rustls", @@ -4093,14 +4655,41 @@ version = "5.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" +[[package]] +name = "radium" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha 0.3.1", + "rand_core 0.6.4", +] + [[package]] name = "rand" version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" dependencies = [ - "rand_chacha", - "rand_core", + "rand_chacha 0.9.0", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core 0.6.4", ] [[package]] @@ -4110,7 +4699,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", - "rand_core", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom 0.2.16", ] [[package]] @@ -4151,6 +4749,26 @@ dependencies = [ "bitflags 2.10.0", ] +[[package]] +name = "ref-cast" +version = "1.0.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" +dependencies = [ + "ref-cast-impl", +] + +[[package]] +name = "ref-cast-impl" +version = "1.0.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.108", +] + [[package]] name = "regex" version = "1.12.2" @@ -4186,6 +4804,44 @@ version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" +[[package]] +name = "rend" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "71fe3824f5629716b1589be05dacd749f6aa084c87e00e016714a8cdfccc997c" +dependencies = [ + "bytecheck", +] + +[[package]] +name = "reqsign" +version = "0.16.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43451dbf3590a7590684c25fb8d12ecdcc90ed3ac123433e500447c7d77ed701" +dependencies = [ + "anyhow", + "async-trait", + "base64", + "chrono", + "form_urlencoded", + "getrandom 0.2.16", + "hex", + "hmac", + "home", + "http 1.3.1", + "log", + "percent-encoding", + "quick-xml 0.37.5", + "rand 0.8.5", + "reqwest", + "rust-ini", + "serde", + "serde_json", + "sha1", + "sha2", + "tokio", +] + [[package]] name = "reqwest" version = "0.12.24" @@ -4252,6 +4908,71 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "rkyv" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" +dependencies = [ + "bitvec", + "bytecheck", + "bytes", + "hashbrown 0.12.3", + "ptr_meta", + "rend", + "rkyv_derive", + "seahash", + "tinyvec", + "uuid", +] + +[[package]] +name = "rkyv_derive" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "roaring" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f08d6a905edb32d74a5d5737a0c9d7e950c312f3c46cb0ca0a2ca09ea11878a0" +dependencies = [ + "bytemuck", + "byteorder", +] + +[[package]] +name = "rust-ini" +version = "0.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "796e8d2b6696392a43bea58116b667fb4c29727dc5abd27d6acf338bb4f688c7" +dependencies = [ + "cfg-if", + "ordered-multimap", +] + +[[package]] +name = "rust_decimal" +version = "1.39.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" +dependencies = [ + "arrayvec", + "borsh", + "bytes", + "num-traits", + "rand 0.8.5", + "rkyv", + "serde", + "serde_json", +] + [[package]] name = "rustc-demangle" version = "0.1.26" @@ -4343,9 +5064,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "94182ad936a0c91c324cd46c6511b9510ed16af436d7b5bab34beab0afd55f7a" dependencies = [ "web-time", "zeroize", @@ -4353,9 +5074,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.7" +version = "0.103.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" +checksum = "2ffdfa2f5286e2247234e03f680868ac2815974dc39e00ea15adc445d0aafe52" dependencies = [ "aws-lc-rs", "ring", @@ -4393,12 +5114,42 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "schemars" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cd191f9397d57d581cddd31014772520aa448f65ef991055d7f61582c65165f" +dependencies = [ + "dyn-clone", + "ref-cast", + "serde", + "serde_json", +] + +[[package]] +name = "schemars" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1317c3bf3e7df961da95b0a56a172a02abead31276215a0497241a7624b487ce" +dependencies = [ + "dyn-clone", + "ref-cast", + "serde", + "serde_json", +] + [[package]] name = "scopeguard" version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" +[[package]] +name = "seahash" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" + [[package]] name = "security-framework" version = "3.5.1" @@ -4450,8 +5201,18 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float", + "ordered-float 2.10.1", + "serde", +] + +[[package]] +name = "serde_bytes" +version = "0.11.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5d440709e79d88e51ac01c4b72fc6cb7314017bb7da9eeff678aa94c10e3ea8" +dependencies = [ "serde", + "serde_core", ] [[package]] @@ -4487,6 +5248,17 @@ dependencies = [ "serde_core", ] +[[package]] +name = "serde_repr" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.108", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -4499,13 +5271,44 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_with" +version = "3.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa66c845eee442168b2c8134fec70ac50dc20e760769c8ba0ad1319ca1959b04" +dependencies = [ + "base64", + "chrono", + "hex", + "indexmap 1.9.3", + "indexmap 2.12.0", + "schemars 0.9.0", + "schemars 1.0.5", + "serde_core", + "serde_json", + "serde_with_macros", + "time", +] + +[[package]] +name = "serde_with_macros" +version = "3.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b91a903660542fced4e99881aa481bdbaec1634568ee02e0b8bd57c64cb38955" +dependencies = [ + "darling 0.21.3", + "proc-macro2", + "quote", + "syn 2.0.108", +] + [[package]] name = "serde_yaml" version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap", + "indexmap 2.12.0", "itoa", "ryu", "serde", @@ -4637,12 +5440,27 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + [[package]] name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" +[[package]] +name = "strum" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" +dependencies = [ + "strum_macros 0.27.2", +] + [[package]] name = "strum_macros" version = "0.26.4" @@ -4656,6 +5474,18 @@ dependencies = [ "syn 2.0.108", ] +[[package]] +name = "strum_macros" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7695ce3845ea4b33927c055a39dc438a45b059f7c1b3d91d38d10355fb8cbca7" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.108", +] + [[package]] name = "subtle" version = "2.6.1" @@ -4727,6 +5557,18 @@ dependencies = [ "syn 2.0.108", ] +[[package]] +name = "tagptr" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b2093cf4c8eb1e67749a6762251bc9cd836b6fc171623bd0a9d324d37af2417" + +[[package]] +name = "tap" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" + [[package]] name = "tempfile" version = "3.23.0" @@ -4790,6 +5632,15 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + [[package]] name = "thrift" version = "0.17.0" @@ -4798,7 +5649,9 @@ checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", "integer-encoding", - "ordered-float", + "log", + "ordered-float 2.10.1", + "threadpool", ] [[package]] @@ -4839,6 +5692,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" dependencies = [ "deranged", + "itoa", "num-conv", "powerfmt", "serde", @@ -4873,9 +5727,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "42d3e9c45c09de15d06dd8acf5f4e0e399e85927b7f00711024eb7ae10fa4869" dependencies = [ "displaydoc", "zerovec", @@ -4946,9 +5800,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.16" +version = "0.7.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14307c986784f72ef81c89db7d9e28d6ac26d16213b109ea501696195e6e3ce5" +checksum = "2efa149fe76073d6e8fd97ef4f4eca7b67f599660115591483572e406e165594" dependencies = [ "bytes", "futures-core", @@ -4957,6 +5811,36 @@ dependencies = [ "tokio", ] +[[package]] +name = "toml_datetime" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" +dependencies = [ + "serde_core", +] + +[[package]] +name = "toml_edit" +version = "0.23.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" +dependencies = [ + "indexmap 2.12.0", + "toml_datetime", + "toml_parser", + "winnow", +] + +[[package]] +name = "toml_parser" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" +dependencies = [ + "winnow", +] + [[package]] name = "tower" version = "0.5.2" @@ -5045,7 +5929,27 @@ version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ea3136b675547379c4bd395ca6b938e5ad3c3d20fad76e7fe85f9e0d011419c" dependencies = [ - "rand", + "rand 0.9.2", +] + +[[package]] +name = "typed-builder" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd9d30e3a08026c78f246b173243cf07b3696d274debd26680773b6773c2afc7" +dependencies = [ + "typed-builder-macro", +] + +[[package]] +name = "typed-builder-macro" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.108", ] [[package]] @@ -5065,9 +5969,9 @@ checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "unicode-ident" -version = "1.0.20" +version = "1.0.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "462eeb75aeb73aea900253ce739c8e18a67423fadf006037cd3ff27e82748a06" +checksum = "9312f7c4f6ff9069b165498234ce8be658059c6728633667c526e27dc2cf1df5" [[package]] name = "unicode-segmentation" @@ -5198,9 +6102,9 @@ dependencies = [ [[package]] name = "wasm-bindgen" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" +checksum = "da95793dfc411fbbd93f5be7715b0578ec61fe87cb1a42b12eb625caa5c5ea60" dependencies = [ "cfg-if", "once_cell", @@ -5209,25 +6113,11 @@ dependencies = [ "wasm-bindgen-shared", ] -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.104" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" -dependencies = [ - "bumpalo", - "log", - "proc-macro2", - "quote", - "syn 2.0.108", - "wasm-bindgen-shared", -] - [[package]] name = "wasm-bindgen-futures" -version = "0.4.54" +version = "0.4.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" +checksum = "551f88106c6d5e7ccc7cd9a16f312dd3b5d36ea8b4954304657d5dfba115d4a0" dependencies = [ "cfg-if", "js-sys", @@ -5238,9 +6128,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" +checksum = "04264334509e04a7bf8690f2384ef5265f05143a4bff3889ab7a3269adab59c2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5248,22 +6138,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" +checksum = "420bc339d9f322e562942d52e115d57e950d12d88983a14c79b86859ee6c7ebc" dependencies = [ + "bumpalo", "proc-macro2", "quote", "syn 2.0.108", - "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" +checksum = "76f218a38c84bcb33c25ec7059b07847d465ce0e0a76b995e134a45adcb6af76" dependencies = [ "unicode-ident", ] @@ -5283,9 +6173,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" +checksum = "3a1f95c0d03a47f4ae1f7a64643a6bb97465d9b740f0fa8f90ea33915c99a9a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -5303,9 +6193,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b130c0d2d49f8b6889abc456e795e82525204f27c42cf767cf0d7734e089b8" +checksum = "b2878ef029c47c6e8cf779119f20fcf52bde7ad42a731b2a304bc221df17571e" dependencies = [ "rustls-pki-types", ] @@ -5643,6 +6533,15 @@ version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" +[[package]] +name = "winnow" +version = "0.7.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "21a0236b59786fed61e2a80582dd500fe61f18b5dca67a4a067d0bc9039339cf" +dependencies = [ + "memchr", +] + [[package]] name = "wit-bindgen" version = "0.46.0" @@ -5651,9 +6550,18 @@ checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "9edde0db4769d2dc68579893f2306b26c6ecfbe0ef499b013d731b7b9247e0b9" + +[[package]] +name = "wyz" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" +dependencies = [ + "tap", +] [[package]] name = "xmlparser" @@ -5667,13 +6575,21 @@ version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" +[[package]] +name = "xz2" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" +dependencies = [ + "lzma-sys", +] + [[package]] name = "yoke" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "72d6e5c6afb84d73944e5cedb052c4680d5657337201555f9f2a16b7406d4954" dependencies = [ - "serde", "stable_deref_trait", "yoke-derive", "zerofrom", @@ -5681,9 +6597,9 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", @@ -5740,9 +6656,9 @@ checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +checksum = "2a59c17a5562d507e4b54960e8569ebee33bee890c70aa3fe7b97e85a9fd7851" dependencies = [ "displaydoc", "yoke", @@ -5751,9 +6667,9 @@ dependencies = [ [[package]] name = "zerovec" -version = "0.11.4" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "6c28719294829477f525be0186d13efa9a3c602f7ec202ca9e353d310fb9a002" dependencies = [ "yoke", "zerofrom", @@ -5762,9 +6678,9 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", From fdc149ecb78acd5b143c6903f035498d24aa913d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 13:38:47 -0500 Subject: [PATCH 80/89] Fix TestForwardCompatibility --- native/core/src/execution/planner.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index a88ffba4f8..c57ade5a17 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2830,14 +2830,12 @@ fn parse_file_scan_tasks( let partition_spec = if let Some(partition_spec_json) = proto_task.partition_spec_json.as_ref() { - let spec = serde_json::from_str::(partition_spec_json) - .map_err(|e| { - GeneralError(format!( - "Failed to parse partition spec JSON: {}", - e - )) - })?; - Some(Arc::new(spec)) + // 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 }; From d63829db539e65904df8d9a00eb37ddc55ff1aa1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 18:41:20 -0500 Subject: [PATCH 81/89] Fix serialization of partitionData, bump df50 to fix deserialization bug in iceberg-rust --- native/Cargo.lock | 14 ++-- .../org/apache/comet/serde/icebergScan.scala | 67 +++++++++++++------ .../comet/CometIcebergNativeSuite.scala | 63 +++++++++++++++++ 3 files changed, 117 insertions(+), 27 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index a93972966a..09617d5260 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -2606,7 +2606,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -3205,7 +3205,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#3d8cd1b552b2b90b1e1b9621c2dae8341aaa7253" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#dc46c59810c6003d715eaaeeb6d547d1e3e6f2f6" dependencies = [ "anyhow", "apache-avro", @@ -3436,7 +3436,7 @@ checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -4637,7 +4637,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] @@ -5023,7 +5023,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -5579,7 +5579,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix 1.1.2", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -6234,7 +6234,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index 8641670779..f38133f221 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -766,31 +766,58 @@ object IcebergScanSerde extends Logging { val partitionTypeMethod = spec.getClass.getMethod("partitionType") val partitionType = partitionTypeMethod.invoke(spec) - // Serialize partition type to JSON using Iceberg's SchemaParser - try { - // scalastyle:off classforname - val schemaParserClass = - Class.forName("org.apache.iceberg.SchemaParser") - val toJsonMethod = schemaParserClass.getMethod( - "toJson", - Class.forName("org.apache.iceberg.types.Type")) - // scalastyle:on classforname - val partitionTypeJson = toJsonMethod - .invoke(null, partitionType) - .asInstanceOf[String] - taskBuilder.setPartitionTypeJson(partitionTypeJson) - } catch { - case e: Exception => - logWarning( - s"Failed to serialize partition type to JSON: ${e.getMessage}") - } - - // Serialize partition data to JSON using Iceberg's StructLike + // 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) { + // Serialize partition type to JSON using Iceberg's SchemaParser + try { + // scalastyle:off classforname + val jsonUtilClass = Class.forName("org.apache.iceberg.util.JsonUtil") + val factoryMethod = jsonUtilClass.getMethod("factory") + val factory = factoryMethod.invoke(null) + + val writer = new java.io.StringWriter() + val createGeneratorMethod = + factory.getClass.getMethod( + "createGenerator", + classOf[java.io.Writer]) + val generator = createGeneratorMethod.invoke(factory, writer) + + val schemaParserClass = + Class.forName("org.apache.iceberg.SchemaParser") + val typeClass = Class.forName("org.apache.iceberg.types.Type") + // Use shaded Jackson class from Iceberg + val generatorClass = Class.forName( + "org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonGenerator") + val toJsonMethod = schemaParserClass.getDeclaredMethod( + "toJson", + typeClass, + generatorClass) + // scalastyle:on classforname + + toJsonMethod.setAccessible(true) + toJsonMethod.invoke(null, partitionType, generator) + + // Close the generator to ensure all data is written + val closeMethod = generator.getClass.getMethod("close") + closeMethod.invoke(generator) + val partitionTypeJson = writer.toString + taskBuilder.setPartitionTypeJson(partitionTypeJson) + } catch { + case e: Exception => + logWarning( + s"Failed to serialize partition type to JSON: ${e.getMessage}") + } + } else { + // No partition fields to serialize (unpartitioned table or all non-identity transforms) + } + + // Serialize partition data to JSON using Iceberg's StructLike val jsonBuilder = new StringBuilder() jsonBuilder.append("{") diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index d1b9197ffd..25c8b0191f 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -2101,6 +2101,69 @@ class CometIcebergNativeSuite extends CometTestBase { } } + test("partition type JSON serialization for identity-partitioned tables") { + 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 an identity-partitioned table + spark.sql(""" + CREATE TABLE test_cat.db.partition_type_test ( + id INT, + category STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (category) + """) + + spark.sql(""" + INSERT INTO test_cat.db.partition_type_test VALUES + (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7) + """) + + // Execute a query and get the plan + val df = spark.sql("SELECT * FROM test_cat.db.partition_type_test") + val scanNodes = df.queryExecution.executedPlan + .collectLeaves() + .collect { case s: CometIcebergNativeScanExec => s } + + assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // Verify that partition_type_json is populated in the protobuf + val icebergScan = scanNodes.head.nativeOp.getIcebergScan + assert(icebergScan.getFilePartitionsCount > 0, "Expected at least one file partition") + + var foundPartitionTypeJson = false + val partitions = icebergScan.getFilePartitionsList + partitions.forEach { partition => + partition.getFileScanTasksList.forEach { task => + if (task.hasPartitionTypeJson && task.getPartitionTypeJson.nonEmpty) { + foundPartitionTypeJson = true + // Verify it's valid JSON by checking it contains expected Iceberg type structure + assert( + task.getPartitionTypeJson.contains("\"type\"") && + task.getPartitionTypeJson.contains("\"fields\""), + s"partition_type_json should contain Iceberg type structure, got: ${task.getPartitionTypeJson}") + } + } + } + + assert( + foundPartitionTypeJson, + "Expected at least one task to have partition_type_json populated for identity-partitioned table") + + spark.sql("DROP TABLE test_cat.db.partition_type_test") + } + } + } + // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From f2f1807b35316781c2e25d1fc6c6e192e16045f6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 18:41:39 -0500 Subject: [PATCH 82/89] Format --- .../src/main/scala/org/apache/comet/serde/icebergScan.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index f38133f221..709479a1ac 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -772,12 +772,13 @@ object IcebergScanSerde extends Logging { .invoke(partitionType) .asInstanceOf[java.util.List[_]] - // Only serialize partition type if there are actual partition fields + // Only serialize partition type if there are actual partition fields if (!fields.isEmpty) { // Serialize partition type to JSON using Iceberg's SchemaParser try { // scalastyle:off classforname - val jsonUtilClass = Class.forName("org.apache.iceberg.util.JsonUtil") + val jsonUtilClass = + Class.forName("org.apache.iceberg.util.JsonUtil") val factoryMethod = jsonUtilClass.getMethod("factory") val factory = factoryMethod.invoke(null) From 32c35b9f967494ac0b0095d0fa07127cc85ca227 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 19:00:29 -0500 Subject: [PATCH 83/89] Fix format --- .../src/main/scala/org/apache/comet/serde/icebergScan.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index 709479a1ac..85b41ec9ac 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -794,7 +794,8 @@ object IcebergScanSerde extends Logging { val typeClass = Class.forName("org.apache.iceberg.types.Type") // Use shaded Jackson class from Iceberg val generatorClass = Class.forName( - "org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonGenerator") + "org.apache.iceberg.shaded.com." + + "fasterxml.jackson.core.JsonGenerator") val toJsonMethod = schemaParserClass.getDeclaredMethod( "toJson", typeClass, @@ -812,7 +813,8 @@ object IcebergScanSerde extends Logging { } catch { case e: Exception => logWarning( - s"Failed to serialize partition type to JSON: ${e.getMessage}") + "Failed to serialize partition type to JSON:" + + s" ${e.getMessage}") } } else { // No partition fields to serialize (unpartitioned table or all non-identity transforms) From 1a169b33b8a88b81079e457fc9260d8784f0a254 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 19:02:01 -0500 Subject: [PATCH 84/89] Fix format for realsies --- spark/src/main/scala/org/apache/comet/serde/icebergScan.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index 85b41ec9ac..cbf1e394c2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -817,7 +817,8 @@ object IcebergScanSerde extends Logging { s" ${e.getMessage}") } } else { - // No partition fields to serialize (unpartitioned table or all non-identity transforms) + // No partition fields to serialize (unpartitioned table or + // all non-identity transforms) } // Serialize partition data to JSON using Iceberg's StructLike From c58d2ce4f13dec1a75f1be2d612bab508f112503 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 11:31:49 -0500 Subject: [PATCH 85/89] name mapping changes for iceberg-rust #1821. --- native/Cargo.lock | 14 ++--- native/core/src/execution/planner.rs | 13 +++- native/proto/src/proto/operator.proto | 11 +++- .../org/apache/comet/serde/icebergScan.scala | 59 ++++++++--------- .../comet/CometIcebergNativeScanExec.scala | 62 ++++++++++++++++++ .../comet/CometIcebergNativeSuite.scala | 63 ------------------- 6 files changed, 113 insertions(+), 109 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 09617d5260..a6c0d6f538 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -2606,7 +2606,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -3205,7 +3205,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.7.0" -source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#dc46c59810c6003d715eaaeeb6d547d1e3e6f2f6" +source = "git+https://github.com/mbutrovich/iceberg-rust?branch=df50#3d5f7c8d3ffa77d39c85d84d500a05d046adfb34" dependencies = [ "anyhow", "apache-avro", @@ -3436,7 +3436,7 @@ checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -4637,7 +4637,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -5023,7 +5023,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -5579,7 +5579,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix 1.1.2", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -6234,7 +6234,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index c57ade5a17..017343b8f1 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2840,7 +2840,16 @@ fn parse_file_scan_tasks( None }; - let partition_spec_id = proto_task.partition_spec_id; + // Extract name mapping from JSON if present + 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(), @@ -2853,8 +2862,8 @@ fn parse_file_scan_tasks( predicate: bound_predicate, deletes, partition, - partition_spec_id, partition_spec, + name_mapping, }) }) .collect(); diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index c9960f2392..94661a20e6 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -160,16 +160,21 @@ message IcebergFileScanTask { // Serialized as JSON to represent the Struct of partition values optional string partition_data_json = 10; - // Partition spec ID for this file (used to lookup the PartitionSpec) - optional int32 partition_spec_id = 11; - // 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) diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index cbf1e394c2..b6a1cb9192 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -323,6 +323,17 @@ object IcebergScanSerde extends Logging { 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 @@ -727,18 +738,14 @@ object IcebergScanSerde extends Logging { s"${e.getMessage}") } - // Extract partition data and spec ID for proper constant identification + // Extract partition spec for proper constant identification try { - // Get partition spec from the task first + // Get partition spec from the task val specMethod = fileScanTaskClass.getMethod("spec") val spec = specMethod.invoke(task) if (spec != null) { - val specIdMethod = spec.getClass.getMethod("specId") - val specId = specIdMethod.invoke(spec).asInstanceOf[Int] - taskBuilder.setPartitionSpecId(specId) - - // Serialize the entire PartitionSpec to JSON + // Serialize the entire PartitionSpec to JSON (includes spec-id) try { // scalastyle:off classforname val partitionSpecParserClass = @@ -775,40 +782,19 @@ object IcebergScanSerde extends Logging { // Only serialize partition type if there are actual partition fields if (!fields.isEmpty) { // Serialize partition type to JSON using Iceberg's SchemaParser + // Use the simple overload that returns String directly try { // scalastyle:off classforname - val jsonUtilClass = - Class.forName("org.apache.iceberg.util.JsonUtil") - val factoryMethod = jsonUtilClass.getMethod("factory") - val factory = factoryMethod.invoke(null) - - val writer = new java.io.StringWriter() - val createGeneratorMethod = - factory.getClass.getMethod( - "createGenerator", - classOf[java.io.Writer]) - val generator = createGeneratorMethod.invoke(factory, writer) - val schemaParserClass = Class.forName("org.apache.iceberg.SchemaParser") val typeClass = Class.forName("org.apache.iceberg.types.Type") - // Use shaded Jackson class from Iceberg - val generatorClass = Class.forName( - "org.apache.iceberg.shaded.com." + - "fasterxml.jackson.core.JsonGenerator") - val toJsonMethod = schemaParserClass.getDeclaredMethod( - "toJson", - typeClass, - generatorClass) + // Use the simple toJson(Type) method that returns String + val toJsonMethod = schemaParserClass.getMethod("toJson", typeClass) // scalastyle:on classforname - toJsonMethod.setAccessible(true) - toJsonMethod.invoke(null, partitionType, generator) - - // Close the generator to ensure all data is written - val closeMethod = generator.getClass.getMethod("close") - closeMethod.invoke(generator) - val partitionTypeJson = writer.toString + val partitionTypeJson = toJsonMethod + .invoke(null, partitionType) + .asInstanceOf[String] taskBuilder.setPartitionTypeJson(partitionTypeJson) } catch { case e: Exception => @@ -877,6 +863,11 @@ object IcebergScanSerde extends Logging { 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 => diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 4907a90ff9..ddc2753ec4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -230,6 +230,68 @@ object CometIcebergNativeScanExec { 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. * diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 25c8b0191f..d1b9197ffd 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -2101,69 +2101,6 @@ class CometIcebergNativeSuite extends CometTestBase { } } - test("partition type JSON serialization for identity-partitioned tables") { - 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 an identity-partitioned table - spark.sql(""" - CREATE TABLE test_cat.db.partition_type_test ( - id INT, - category STRING, - value DOUBLE - ) USING iceberg - PARTITIONED BY (category) - """) - - spark.sql(""" - INSERT INTO test_cat.db.partition_type_test VALUES - (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7) - """) - - // Execute a query and get the plan - val df = spark.sql("SELECT * FROM test_cat.db.partition_type_test") - val scanNodes = df.queryExecution.executedPlan - .collectLeaves() - .collect { case s: CometIcebergNativeScanExec => s } - - assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // Verify that partition_type_json is populated in the protobuf - val icebergScan = scanNodes.head.nativeOp.getIcebergScan - assert(icebergScan.getFilePartitionsCount > 0, "Expected at least one file partition") - - var foundPartitionTypeJson = false - val partitions = icebergScan.getFilePartitionsList - partitions.forEach { partition => - partition.getFileScanTasksList.forEach { task => - if (task.hasPartitionTypeJson && task.getPartitionTypeJson.nonEmpty) { - foundPartitionTypeJson = true - // Verify it's valid JSON by checking it contains expected Iceberg type structure - assert( - task.getPartitionTypeJson.contains("\"type\"") && - task.getPartitionTypeJson.contains("\"fields\""), - s"partition_type_json should contain Iceberg type structure, got: ${task.getPartitionTypeJson}") - } - } - } - - assert( - foundPartitionTypeJson, - "Expected at least one task to have partition_type_json populated for identity-partitioned table") - - spark.sql("DROP TABLE test_cat.db.partition_type_test") - } - } - } - // Helper to create temp directory def withTempIcebergDir(f: File => Unit): Unit = { val dir = Files.createTempDirectory("comet-iceberg-test").toFile From c962714451585b944c2e6b9a1ec2e644b5ab4cb6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 11:36:12 -0500 Subject: [PATCH 86/89] clean up stray comments, format --- native/core/src/execution/planner.rs | 1 - spark/src/main/scala/org/apache/comet/serde/icebergScan.scala | 1 - 2 files changed, 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 017343b8f1..e9a5b6017e 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2840,7 +2840,6 @@ fn parse_file_scan_tasks( None }; - // Extract name mapping from JSON if present let name_mapping = if let Some(name_mapping_json) = proto_task.name_mapping_json.as_ref() { match serde_json::from_str::(name_mapping_json) { diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index b6a1cb9192..41e5cef982 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -740,7 +740,6 @@ object IcebergScanSerde extends Logging { // Extract partition spec for proper constant identification try { - // Get partition spec from the task val specMethod = fileScanTaskClass.getMethod("spec") val spec = specMethod.invoke(task) From a52c69d93a5d32857664d80c89907b5341844c55 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 5 Nov 2025 20:19:20 -0500 Subject: [PATCH 87/89] Update 1.8.1.diff with spotlessApply. --- dev/diffs/iceberg/1.8.1.diff | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/diffs/iceberg/1.8.1.diff b/dev/diffs/iceberg/1.8.1.diff index d554db1c5e..e7b58902ca 100644 --- a/dev/diffs/iceberg/1.8.1.diff +++ b/dev/diffs/iceberg/1.8.1.diff @@ -1202,7 +1202,7 @@ index 019f3919d..656e0600a 100644 + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java -index 404ba7284..e6d266909 100644 +index 404ba7284..00e97e96f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java @@ -90,6 +90,16 @@ public abstract class SparkDistributedDataScanTestBase @@ -1211,8 +1211,8 @@ index 404ba7284..e6d266909 100644 .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") + .config("spark.plugins", "org.apache.spark.CometPlugin") + .config( -+ "spark.shuffle.manager", -+ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ "spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .config("spark.comet.explainFallback.enabled", "true") + .config("spark.comet.scan.icebergNative.enabled", "true") + .config("spark.memory.offHeap.enabled", "true") From 2cd4d7d833f93bc9b191c295576ba911032f975c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 6 Nov 2025 14:12:19 -0500 Subject: [PATCH 88/89] No longer inject partition default-values, it's redundant now that we have the constants_map set up in RecordBatchTransformer. --- .../source/user-guide/latest/compatibility.md | 4 +- .../org/apache/comet/serde/icebergScan.scala | 135 ++++++++---------- 2 files changed, 63 insertions(+), 76 deletions(-) diff --git a/docs/source/user-guide/latest/compatibility.md b/docs/source/user-guide/latest/compatibility.md index 908693ff5f..7a18e3b4a9 100644 --- a/docs/source/user-guide/latest/compatibility.md +++ b/docs/source/user-guide/latest/compatibility.md @@ -56,8 +56,8 @@ and sorting on floating-point data can be enabled by setting `spark.comet.expres ## Incompatible Expressions Expressions that are not 100% Spark-compatible will fall back to Spark by default and can be enabled by setting -`spark.comet.expression.EXPRNAME.allowIncompatible=true`, where `EXPRNAME` is the Spark expression class name. See -the [Comet Supported Expressions Guide](expressions.md) for more information on this configuration setting. +`spark.comet.expression.EXPRNAME.allowIncompatible=true`, where `EXPRNAME` is the Spark expression class name. See +the [Comet Supported Expressions Guide](expressions.md) for more information on this configuration setting. It is also possible to specify `spark.comet.expression.allowIncompatible=true` to enable all incompatible expressions. diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index 41e5cef982..1daef375da 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -232,54 +232,6 @@ object IcebergScanSerde extends Logging { Literal(sparkValue, sparkType) } - /** - * Injects partition values into Iceberg schema JSON as "initial-default" values. - * - * For Hive-style partitioned tables migrated to Iceberg, partition values are stored in - * directory structure, not in data files. This function adds those values to the schema so - * iceberg-rust's RecordBatchTransformer can populate partition columns. - */ - def injectPartitionValuesIntoSchemaJson(schemaJson: String, partitionJson: String): String = { - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - - try { - // Parse both JSONs - implicit val formats: Formats = DefaultFormats - val schemaValue = parse(schemaJson) - val partitionMap = parse(partitionJson).extract[Map[String, JValue]] - - // Transform the schema fields to inject initial-default values - val transformedSchema = schemaValue.transformField { case ("fields", JArray(fields)) => - val updatedFields = fields.map { - case fieldObj: JObject => - // Check if this field has a partition value - fieldObj \ "id" match { - case JInt(fieldId) => - partitionMap.get(fieldId.toString) match { - case Some(partitionValue) => - // Add "initial-default" to this field - fieldObj merge JObject("initial-default" -> partitionValue) - case None => - fieldObj - } - case _ => - fieldObj - } - case other => other - } - ("fields", JArray(updatedFields)) - } - - // Serialize back to JSON - compact(render(transformedSchema)) - } catch { - case e: Exception => - logWarning(s"Failed to inject partition values into schema JSON: ${e.getMessage}") - schemaJson - } - } - /** * Serializes a CometBatchScanExec wrapping an Iceberg SparkBatchQueryScan to protobuf. * @@ -576,13 +528,7 @@ object IcebergScanSerde extends Logging { // scalastyle:on classforname val toJsonMethod = schemaParserClass.getMethod("toJson", schemaClass) toJsonMethod.setAccessible(true) - var schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] - - // Inject partition values into schema if present - partitionJsonOpt.foreach { partitionJson => - schemaJson = - injectPartitionValuesIntoSchemaJson(schemaJson, partitionJson) - } + val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] taskBuilder.setSchemaJson(schemaJson) @@ -780,33 +726,64 @@ object IcebergScanSerde extends Logging { // Only serialize partition type if there are actual partition fields if (!fields.isEmpty) { - // Serialize partition type to JSON using Iceberg's SchemaParser - // Use the simple overload that returns String directly try { - // scalastyle:off classforname - val schemaParserClass = - Class.forName("org.apache.iceberg.SchemaParser") - val typeClass = Class.forName("org.apache.iceberg.types.Type") - // Use the simple toJson(Type) method that returns String - val toJsonMethod = schemaParserClass.getMethod("toJson", typeClass) - // scalastyle:on classforname - - val partitionTypeJson = toJsonMethod - .invoke(null, partitionType) - .asInstanceOf[String] + // 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( - "Failed to serialize partition type to JSON:" + - s" ${e.getMessage}") + s"Failed to serialize partition type to JSON: ${e.getMessage}") } - } else { - // No partition fields to serialize (unpartitioned table or - // all non-identity transforms) } - // Serialize partition data to JSON using Iceberg's StructLike + // 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("{") @@ -837,6 +814,16 @@ object IcebergScanSerde extends Logging { value match { case s: String => jsonBuilder.append("\"").append(s).append("\"") + case f: java.lang.Float if f.isNaN || f.isInfinite => + // 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(). + jsonBuilder.append("\"").append(f.toString).append("\"") + case d: java.lang.Double if d.isNaN || d.isInfinite => + // 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(). + jsonBuilder.append("\"").append(d.toString).append("\"") case n: Number => jsonBuilder.append(n.toString) case b: java.lang.Boolean => From d88c9118933156caaff09005f8a7eea554cb052c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 6 Nov 2025 14:16:41 -0500 Subject: [PATCH 89/89] Fix format. --- .../org/apache/comet/serde/icebergScan.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala index 1daef375da..1267871bd0 100644 --- a/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/icebergScan.scala @@ -780,10 +780,11 @@ object IcebergScanSerde extends Logging { } // 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. + // 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("{") @@ -814,15 +815,13 @@ object IcebergScanSerde extends Logging { 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 => - // 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(). jsonBuilder.append("\"").append(f.toString).append("\"") case d: java.lang.Double if d.isNaN || d.isInfinite => - // 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(). jsonBuilder.append("\"").append(d.toString).append("\"") case n: Number => jsonBuilder.append(n.toString)