From fb5c73781b127811fece563f6ced7d11a04485f5 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 09:58:02 +0200 Subject: [PATCH 01/68] WIP, initial draft of incremental scan --- Cargo.lock | 438 +++++++++++---- Cargo.toml | 19 +- .../iceberg/src/arrow/delete_file_loader.rs | 1 + crates/iceberg/src/arrow/delete_filter.rs | 21 +- crates/iceberg/src/arrow/incremental.rs | 232 ++++++++ crates/iceberg/src/arrow/mod.rs | 3 + crates/iceberg/src/arrow/reader.rs | 209 ++++++- crates/iceberg/src/delete_file_index.rs | 29 + crates/iceberg/src/delete_vector.rs | 4 +- crates/iceberg/src/lib.rs | 3 + crates/iceberg/src/scan/context.rs | 28 +- .../iceberg/src/scan/incremental/context.rs | 120 ++++ crates/iceberg/src/scan/incremental/mod.rs | 522 ++++++++++++++++++ crates/iceberg/src/scan/incremental/task.rs | 87 +++ crates/iceberg/src/scan/mod.rs | 3 + crates/iceberg/src/table.rs | 10 + crates/iceberg/src/util/mod.rs | 19 + crates/iceberg/src/util/snapshot.rs | 72 +++ .../src/writer/file_writer/parquet_writer.rs | 2 +- crates/playground/Cargo.toml | 15 + crates/playground/src/main.rs | 61 ++ 21 files changed, 1747 insertions(+), 151 deletions(-) create mode 100644 crates/iceberg/src/arrow/incremental.rs create mode 100644 crates/iceberg/src/scan/incremental/context.rs create mode 100644 crates/iceberg/src/scan/incremental/mod.rs create mode 100644 crates/iceberg/src/scan/incremental/task.rs create mode 100644 crates/iceberg/src/util/mod.rs create mode 100644 crates/iceberg/src/util/snapshot.rs create mode 100644 crates/playground/Cargo.toml create mode 100644 crates/playground/src/main.rs diff --git a/Cargo.lock b/Cargo.lock index 16ffef0654..a688990e8e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -233,19 +233,32 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" dependencies = [ - "arrow-arith", - "arrow-array", - "arrow-buffer", - "arrow-cast", + "arrow-arith 55.2.0", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-cast 55.2.0", "arrow-csv", - "arrow-data", - "arrow-ipc", + "arrow-data 55.2.0", + "arrow-ipc 55.2.0", "arrow-json", - "arrow-ord", + "arrow-ord 55.2.0", "arrow-row", - "arrow-schema", - "arrow-select", - "arrow-string", + "arrow-schema 55.2.0", + "arrow-select 55.2.0", + "arrow-string 55.2.0", +] + +[[package]] +name = "arrow-arith" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", + "chrono", + "num", ] [[package]] @@ -254,11 +267,26 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30feb679425110209ae35c3fbf82404a39a4c0436bb3ec36164d8bffed2a4ce4" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", + "chrono", + "num", +] + +[[package]] +name = "arrow-array" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "ahash 0.8.12", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", "chrono", + "half", + "hashbrown 0.15.5", "num", ] @@ -269,9 +297,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" dependencies = [ "ahash 0.8.12", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", "chrono", "chrono-tz 0.10.4", "half", @@ -279,6 +307,16 @@ dependencies = [ "num", ] +[[package]] +name = "arrow-buffer" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "bytes", + "half", + "num", +] + [[package]] name = "arrow-buffer" version = "55.2.0" @@ -290,17 +328,36 @@ dependencies = [ "num", ] +[[package]] +name = "arrow-cast" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", + "arrow-select 54.2.1", + "atoi", + "base64 0.22.1", + "chrono", + "half", + "lexical-core", + "num", + "ryu", +] + [[package]] name = "arrow-cast" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e4f12eccc3e1c05a766cafb31f6a60a46c2f8efec9b74c6e0648766d30686af8" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", + "arrow-select 55.2.0", "atoi", "base64 0.22.1", "chrono", @@ -317,38 +374,61 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" dependencies = [ - "arrow-array", - "arrow-cast", - "arrow-schema", + "arrow-array 55.2.0", + "arrow-cast 55.2.0", + "arrow-schema 55.2.0", "chrono", "csv", "csv-core", "regex", ] +[[package]] +name = "arrow-data" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "arrow-buffer 54.2.1", + "arrow-schema 54.2.1", + "half", + "num", +] + [[package]] name = "arrow-data" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8de1ce212d803199684b658fc4ba55fb2d7e87b213de5af415308d2fee3619c2" dependencies = [ - "arrow-buffer", - "arrow-schema", + "arrow-buffer 55.2.0", + "arrow-schema 55.2.0", "half", "num", ] +[[package]] +name = "arrow-ipc" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", + "flatbuffers 24.12.23", +] + [[package]] name = "arrow-ipc" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9ea5967e8b2af39aff5d9de2197df16e305f47f404781d3230b2dc672da5d92" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "flatbuffers", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", + "flatbuffers 25.2.10", "lz4_flex", ] @@ -358,11 +438,11 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-cast 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", "chrono", "half", "indexmap 2.11.0", @@ -374,17 +454,29 @@ dependencies = [ "simdutf8", ] +[[package]] +name = "arrow-ord" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", + "arrow-select 54.2.1", +] + [[package]] name = "arrow-ord" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6506e3a059e3be23023f587f79c82ef0bcf6d293587e3272d20f2d30b969b5a7" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", + "arrow-select 55.2.0", ] [[package]] @@ -393,13 +485,18 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", "half", ] +[[package]] +name = "arrow-schema" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" + [[package]] name = "arrow-schema" version = "55.2.0" @@ -410,6 +507,19 @@ dependencies = [ "serde_json", ] +[[package]] +name = "arrow-select" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "ahash 0.8.12", + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", + "num", +] + [[package]] name = "arrow-select" version = "55.2.0" @@ -417,24 +527,40 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" dependencies = [ "ahash 0.8.12", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", "num", ] +[[package]] +name = "arrow-string" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-data 54.2.1", + "arrow-schema 54.2.1", + "arrow-select 54.2.1", + "memchr", + "num", + "regex", + "regex-syntax", +] + [[package]] name = "arrow-string" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0377d532850babb4d927a06294314b316e23311503ed580ec6ce6a0158f49d40" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-data 55.2.0", + "arrow-schema 55.2.0", + "arrow-select 55.2.0", "memchr", "num", "regex", @@ -1129,7 +1255,7 @@ version = "0.69.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" dependencies = [ - "bitflags", + "bitflags 2.9.4", "cexpr", "clang-sys", "itertools 0.12.1", @@ -1146,6 +1272,12 @@ dependencies = [ "which", ] +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + [[package]] name = "bitflags" version = "2.9.4" @@ -1268,6 +1400,17 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "brotli" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor 4.0.3", +] + [[package]] name = "brotli" version = "8.0.2" @@ -1276,7 +1419,17 @@ checksum = "4bd8b9603c7aa97359dbd97ecf258968c95f3adddd6db2f7e7a5bef101c84560" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", - "brotli-decompressor", + "brotli-decompressor 5.0.0", +] + +[[package]] +name = "brotli-decompressor" +version = "4.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a334ef7c9e23abf0ce748e8cd309037da93e606ad52eb372e4ce327a0dcfbdfd" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", ] [[package]] @@ -1897,8 +2050,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a11e19a7ccc5bb979c95c1dceef663eab39c9061b3bbf8d1937faf0f03bf41f" dependencies = [ "arrow", - "arrow-ipc", - "arrow-schema", + "arrow-ipc 55.2.0", + "arrow-schema 55.2.0", "async-trait", "bytes", "bzip2 0.5.2", @@ -1933,7 +2086,7 @@ dependencies = [ "log", "object_store", "parking_lot", - "parquet", + "parquet 55.2.0", "rand 0.9.2", "regex", "sqlparser", @@ -2012,7 +2165,7 @@ dependencies = [ "mimalloc", "object_store", "parking_lot", - "parquet", + "parquet 55.2.0", "regex", "rustyline", "tokio", @@ -2028,7 +2181,7 @@ dependencies = [ "ahash 0.8.12", "apache-avro 0.17.0", "arrow", - "arrow-ipc", + "arrow-ipc 55.2.0", "base64 0.22.1", "half", "hashbrown 0.14.5", @@ -2036,7 +2189,7 @@ dependencies = [ "libc", "log", "object_store", - "parquet", + "parquet 55.2.0", "paste", "recursive", "sqlparser", @@ -2081,7 +2234,7 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", - "parquet", + "parquet 55.2.0", "rand 0.9.2", "tempfile", "tokio", @@ -2192,7 +2345,7 @@ dependencies = [ "log", "object_store", "parking_lot", - "parquet", + "parquet 55.2.0", "rand 0.9.2", "tokio", ] @@ -2263,7 +2416,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cdf9a9cf655265861a20453b1e58357147eab59bdc90ce7f2f68f1f35104d3bb" dependencies = [ "arrow", - "arrow-buffer", + "arrow-buffer 55.2.0", "base64 0.22.1", "blake2", "blake3", @@ -2326,7 +2479,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2ab331806e34f5545e5f03396e4d5068077395b1665795d8f88c14ec4f1e0b7a" dependencies = [ "arrow", - "arrow-ord", + "arrow-ord 55.2.0", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -2477,8 +2630,8 @@ checksum = "b46cbdf21a01206be76d467f325273b22c559c744a012ead5018dfe79597de08" dependencies = [ "ahash 0.8.12", "arrow", - "arrow-ord", - "arrow-schema", + "arrow-ord 55.2.0", + "arrow-schema 55.2.0", "async-trait", "chrono", "datafusion-common", @@ -2725,6 +2878,12 @@ dependencies = [ "const-random", ] +[[package]] +name = "dotenv" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77c90badedccf4105eca100756a0b1289e191f6fcbdadd3cee1d2f614f97da8f" + [[package]] name = "dotenvy" version = "0.15.7" @@ -2942,13 +3101,23 @@ version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" +[[package]] +name = "flatbuffers" +version = "24.12.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f1baf0dbf96932ec9a3038d57900329c015b0bfb7b63d904f3bc27e2b02a096" +dependencies = [ + "bitflags 1.3.2", + "rustc_version", +] + [[package]] name = "flatbuffers" version = "25.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" dependencies = [ - "bitflags", + "bitflags 2.9.4", "rustc_version", ] @@ -3217,7 +3386,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0bf760ebf69878d9fd8f110c89703d90ce35095324d1f1edcb595c63945ee757" dependencies = [ - "bitflags", + "bitflags 2.9.4", "ignore", "walkdir", ] @@ -3598,14 +3767,14 @@ dependencies = [ "anyhow", "apache-avro 0.20.0", "array-init", - "arrow-arith", - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-ord", - "arrow-schema", - "arrow-select", - "arrow-string", + "arrow-arith 54.2.1", + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-cast 54.2.1", + "arrow-ord 54.2.1", + "arrow-schema 54.2.1", + "arrow-select 54.2.1", + "arrow-string 54.2.1", "as-any", "async-trait", "backon", @@ -3627,7 +3796,7 @@ dependencies = [ "once_cell", "opendal", "ordered-float 4.6.0", - "parquet", + "parquet 54.2.1", "pretty_assertions", "rand 0.8.5", "regex", @@ -3776,7 +3945,7 @@ dependencies = [ "expect-test", "futures", "iceberg", - "parquet", + "parquet 54.2.1", "tempfile", "tokio", "uuid", @@ -3796,8 +3965,8 @@ dependencies = [ name = "iceberg-integration-tests" version = "0.7.0" dependencies = [ - "arrow-array", - "arrow-schema", + "arrow-array 54.2.1", + "arrow-schema 54.2.1", "ctor", "datafusion", "futures", @@ -3806,7 +3975,7 @@ dependencies = [ "iceberg-datafusion", "iceberg_test_utils", "ordered-float 2.10.1", - "parquet", + "parquet 54.2.1", "tokio", "uuid", ] @@ -4053,7 +4222,7 @@ version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "046fa2d4d00aea763528b4950358d0ead425372445dc8ff86312b3c69ff7727b" dependencies = [ - "bitflags", + "bitflags 2.9.4", "cfg-if", "libc", ] @@ -4313,7 +4482,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "391290121bad3d37fbddad76d8f5d1c1c314cfc646d143d7e07a3086ddff0ce3" dependencies = [ - "bitflags", + "bitflags 2.9.4", "libc", "redox_syscall", ] @@ -4426,7 +4595,7 @@ version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" dependencies = [ - "twox-hash", + "twox-hash 2.1.2", ] [[package]] @@ -4670,7 +4839,7 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ - "bitflags", + "bitflags 2.9.4", "cfg-if", "cfg_aliases", "libc", @@ -4683,7 +4852,7 @@ version = "0.30.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "74523f3a35e05aba87a1d978330aef40f67b0304ac79c1c00b294c9830543db6" dependencies = [ - "bitflags", + "bitflags 2.9.4", "cfg-if", "cfg_aliases", "libc", @@ -5013,6 +5182,40 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "parquet" +version = "54.2.1" +source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" +dependencies = [ + "ahash 0.8.12", + "arrow-array 54.2.1", + "arrow-buffer 54.2.1", + "arrow-cast 54.2.1", + "arrow-data 54.2.1", + "arrow-ipc 54.2.1", + "arrow-schema 54.2.1", + "arrow-select 54.2.1", + "base64 0.22.1", + "brotli 7.0.0", + "bytes", + "chrono", + "flate2", + "futures", + "half", + "hashbrown 0.15.5", + "lz4_flex", + "num", + "num-bigint", + "paste", + "seq-macro", + "simdutf8", + "snap", + "thrift", + "tokio", + "twox-hash 1.6.3", + "zstd", +] + [[package]] name = "parquet" version = "55.2.0" @@ -5020,15 +5223,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" dependencies = [ "ahash 0.8.12", - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-ipc", - "arrow-schema", - "arrow-select", + "arrow-array 55.2.0", + "arrow-buffer 55.2.0", + "arrow-cast 55.2.0", + "arrow-data 55.2.0", + "arrow-ipc 55.2.0", + "arrow-schema 55.2.0", + "arrow-select 55.2.0", "base64 0.22.1", - "brotli", + "brotli 8.0.2", "bytes", "chrono", "flate2", @@ -5045,7 +5248,7 @@ dependencies = [ "snap", "thrift", "tokio", - "twox-hash", + "twox-hash 2.1.2", "zstd", ] @@ -5367,6 +5570,17 @@ version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" +[[package]] +name = "playground" +version = "0.7.0" +dependencies = [ + "dotenv", + "futures-util", + "iceberg", + "iceberg-catalog-rest", + "tokio", +] + [[package]] name = "polling" version = "3.10.0" @@ -5828,7 +6042,7 @@ version = "0.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" dependencies = [ - "bitflags", + "bitflags 2.9.4", ] [[package]] @@ -6169,7 +6383,7 @@ version = "0.38.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" dependencies = [ - "bitflags", + "bitflags 2.9.4", "errno", "libc", "linux-raw-sys 0.4.15", @@ -6182,7 +6396,7 @@ version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" dependencies = [ - "bitflags", + "bitflags 2.9.4", "errno", "libc", "linux-raw-sys 0.9.4", @@ -6302,7 +6516,7 @@ version = "16.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "62fd9ca5ebc709e8535e8ef7c658eb51457987e48c98ead2be482172accc408d" dependencies = [ - "bitflags", + "bitflags 2.9.4", "cfg-if", "clipboard-win", "fd-lock", @@ -6444,7 +6658,7 @@ version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags", + "bitflags 2.9.4", "core-foundation 0.9.4", "core-foundation-sys", "libc", @@ -6457,7 +6671,7 @@ version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "60b369d18893388b345804dc0007963c99b7d665ae71d275812d828c6f089640" dependencies = [ - "bitflags", + "bitflags 2.9.4", "core-foundation 0.10.1", "core-foundation-sys", "libc", @@ -6981,7 +7195,7 @@ checksum = "aa003f0038df784eb8fecbbac13affe3da23b45194bd57dba231c8f48199c526" dependencies = [ "atoi", "base64 0.22.1", - "bitflags", + "bitflags 2.9.4", "byteorder", "bytes", "crc", @@ -7022,7 +7236,7 @@ checksum = "db58fcd5a53cf07c184b154801ff91347e4c30d17a3562a635ff028ad5deda46" dependencies = [ "atoi", "base64 0.22.1", - "bitflags", + "bitflags 2.9.4", "byteorder", "crc", "dotenvy", @@ -7565,7 +7779,7 @@ version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" dependencies = [ - "bitflags", + "bitflags 2.9.4", "bytes", "futures-util", "http 1.3.1", @@ -7657,6 +7871,16 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "static_assertions", +] + [[package]] name = "twox-hash" version = "2.1.2" diff --git a/Cargo.toml b/Cargo.toml index 46c99cc3d8..6920bcc226 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -25,6 +25,7 @@ members = [ "crates/integrations/*", "crates/sqllogictest", "crates/test_utils", + "crates/playground", ] resolver = "2" @@ -42,14 +43,14 @@ rust-version = "1.87" anyhow = "1.0.72" apache-avro = { version = "0.20", features = ["zstandard"] } array-init = "2" -arrow-arith = { version = "55.1" } -arrow-array = { version = "55.1" } -arrow-buffer = { version = "55.1" } -arrow-cast = { version = "55.1" } -arrow-ord = { version = "55.1" } -arrow-schema = { version = "55.1" } -arrow-select = { version = "55.1" } -arrow-string = { version = "55.1" } +arrow-arith = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-array = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-buffer = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-cast = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-ord = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-schema = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-select = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-string = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } as-any = "0.3.2" async-trait = "0.1.88" aws-config = "1.8.1" @@ -94,7 +95,7 @@ num-bigint = "0.4.6" once_cell = "1.20" opendal = "0.54.0" ordered-float = "4" -parquet = "55.1" +parquet = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } pilota = "0.11.10" port_scanner = "0.1.5" pretty_assertions = "1.4" diff --git a/crates/iceberg/src/arrow/delete_file_loader.rs b/crates/iceberg/src/arrow/delete_file_loader.rs index 592ef2eb4a..01c11e8363 100644 --- a/crates/iceberg/src/arrow/delete_file_loader.rs +++ b/crates/iceberg/src/arrow/delete_file_loader.rs @@ -63,6 +63,7 @@ impl BasicDeleteFileLoader { data_file_path, self.file_io.clone(), false, + false, // don't include row ordinals for delete files ) .await? .build()? diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index b853baa993..b499887ac8 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -35,11 +35,17 @@ enum EqDelState { } #[derive(Debug, Default)] -struct DeleteFileFilterState { +pub(crate) struct DeleteFileFilterState { delete_vectors: HashMap>>, equality_deletes: HashMap, } +impl DeleteFileFilterState { + pub fn delete_vectors(&self) -> &HashMap>> { + &self.delete_vectors + } +} + #[derive(Clone, Debug, Default)] pub(crate) struct DeleteFilter { state: Arc>, @@ -65,6 +71,19 @@ impl DeleteFilter { .and_then(|st| st.delete_vectors.get(delete_file_path).cloned()) } + pub(crate) fn with_read(&self, f: F) -> Result + where + F: FnOnce(&DeleteFileFilterState) -> Result, + { + let state = self.state.read().map_err(|e| { + Error::new( + ErrorKind::Unexpected, + format!("Failed to acquire read lock: {}", e), + ) + })?; + f(&state) + } + pub(crate) fn try_start_eq_del_load(&self, file_path: &str) -> Option> { let mut state = self.state.write().unwrap(); diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs new file mode 100644 index 0000000000..79df49e373 --- /dev/null +++ b/crates/iceberg/src/arrow/incremental.rs @@ -0,0 +1,232 @@ +use std::pin::Pin; +use std::sync::Arc; + +use apache_avro::types::Record; +use arrow_array::{RecordBatch, UInt64Array}; +use arrow_schema::{DataType, Field, Schema as ArrowSchema}; +use futures::{Stream, StreamExt, TryStreamExt}; +use roaring::RoaringTreemap; + +use crate::arrow::record_batch_transformer::RecordBatchTransformer; +use crate::arrow::{ArrowBatchEmitter, ArrowReader, RESERVED_COL_NAME_POS, RESERVED_FIELD_ID_POS}; +use crate::io::FileIO; +use crate::scan::ArrowRecordBatchStream; +use crate::scan::incremental::{ + AppendedFileScanTask, IncrementalFileScanTask, IncrementalFileScanTaskStream, +}; +use crate::spec::Schema; +use crate::{Error, ErrorKind, Result}; + +/// The type of incremental batch: appended data or deleted records. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum IncrementalBatchType { + /// Appended records. + Append, + /// Deleted records. + Delete, +} + +/// The stream of incremental Arrow `RecordBatch`es with batch type. +pub type IncrementalArrowBatchRecordStream = + Pin> + Send + 'static>>; + +impl ArrowBatchEmitter + for IncrementalFileScanTaskStream +{ + /// Take a stream of `IncrementalFileScanTasks` and reads all the files. Returns a + /// stream of Arrow `RecordBatch`es containing the data from the files. + fn read(self, reader: ArrowReader) -> Result { + let file_io = reader.file_io.clone(); + let batch_size = reader.batch_size; + let concurrency_limit_data_files = reader.concurrency_limit_data_files; + let include_row_ordinals = reader.include_row_ordinals; + let include_file_path = reader.include_file_path; + + let stream = self + .map_ok(move |task| { + let file_io = file_io.clone(); + + process_incremental_file_scan_task( + task, + batch_size, + file_io, + include_row_ordinals, + include_file_path, + ) + }) + .map_err(|err| { + Error::new(ErrorKind::Unexpected, "file scan task generate failed").with_source(err) + }) + .try_buffer_unordered(concurrency_limit_data_files) + .try_flatten_unordered(concurrency_limit_data_files); + + Ok(Box::pin(stream) as IncrementalArrowBatchRecordStream) + } +} + +async fn process_incremental_file_scan_task( + task: IncrementalFileScanTask, + batch_size: Option, + file_io: FileIO, + include_row_ordinals: bool, + include_file_path: bool, +) -> Result { + match task { + IncrementalFileScanTask::Append(append_task) => { + process_incremental_append_task( + append_task, + batch_size, + file_io, + include_row_ordinals, + include_file_path, + ) + .await + .map(|stream| { + // Map the stream to include the batch type + let typed_stream = stream.map(|batch_result| { + batch_result.map(|batch| (IncrementalBatchType::Append, batch)) + }); + Box::pin(typed_stream) as IncrementalArrowBatchRecordStream + }) + } + IncrementalFileScanTask::Delete(file_path, delete_vector) => { + // Clone the `RoaringTreemap` underlying the delete vector to take ownership. + let bit_map = { + let guard = delete_vector.lock().unwrap(); + guard.inner.clone() + }; + process_incremental_delete_task(file_path, bit_map, batch_size).map(|stream| { + // Map the stream to include the batch type + let typed_stream = stream.map(|batch_result| { + batch_result.map(|batch| (IncrementalBatchType::Delete, batch)) + }); + Box::pin(typed_stream) as IncrementalArrowBatchRecordStream + }) + } + } +} + +async fn process_incremental_append_task( + task: AppendedFileScanTask, + batch_size: Option, + file_io: FileIO, + include_row_ordinals: bool, + include_file_path: bool, +) -> Result { + let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( + &task.data_file_path, + file_io.clone(), + true, + include_row_ordinals, + ) + .await?; + + // Add reserved field IDs for metadata columns when enabled + let mut extended_project_field_ids = task.project_field_ids.clone(); + let extended_schema = if include_row_ordinals { + // Per Iceberg spec, _pos column has reserved field ID RESERVED_FIELD_ID_POS + extended_project_field_ids.push(RESERVED_FIELD_ID_POS); + + // Extend the schema to include the _pos field so RecordBatchTransformer can find it + let mut fields = task.schema.as_struct().fields().to_vec(); + use crate::spec::{NestedField, PrimitiveType, Type}; + fields.push(Arc::new(NestedField::required( + RESERVED_FIELD_ID_POS, + RESERVED_COL_NAME_POS, + Type::Primitive(PrimitiveType::Long), + ))); + Arc::new( + Schema::builder() + .with_schema_id(task.schema.schema_id()) + .with_fields(fields) + .build()?, + ) + } else { + task.schema_ref() + }; + + // Create a projection mask for the batch stream to select which columns in the + // Parquet file that we want in the response + let projection_mask = ArrowReader::get_arrow_projection_mask( + &extended_project_field_ids, + &extended_schema, + record_batch_stream_builder.parquet_schema(), + record_batch_stream_builder.schema(), + include_row_ordinals, + )?; + record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); + + // RecordBatchTransformer performs any transformations required on the RecordBatches + // that come back from the file, such as type promotion, default column insertion + // and column re-ordering + let mut record_batch_transformer = + RecordBatchTransformer::build(extended_schema, &extended_project_field_ids); + + if let Some(batch_size) = batch_size { + record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); + } + + // Apply positional deletes as row selections. + let row_selection = if let Some(positional_delete_indexes) = task.positional_deletes { + Some(ArrowReader::build_deletes_row_selection( + record_batch_stream_builder.metadata().row_groups(), + &None, + &positional_delete_indexes.lock().unwrap(), + )?) + } else { + None + }; + + if let Some(row_selection) = row_selection { + record_batch_stream_builder = record_batch_stream_builder.with_row_selection(row_selection); + } + + // Build the batch stream and send all the RecordBatches that it generates + // to the requester. + let file_path = task.data_file_path.clone(); + println!("Reading data file: {}", file_path); + let record_batch_stream = record_batch_stream_builder + .build()? + .map(move |batch| match batch { + Ok(batch) => { + let batch = record_batch_transformer.process_record_batch(batch)?; + if include_file_path { + ArrowReader::add_file_path_column(batch, &file_path) + } else { + Ok(batch) + } + } + Err(err) => Err(err.into()), + }); + + Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream) +} + +fn process_incremental_delete_task( + file_path: String, + delete_vector: RoaringTreemap, + batch_size: Option, +) -> Result { + let stream = futures::stream::iter(delete_vector.into_iter()) + .chunks(batch_size.unwrap_or(1024)) + .map(move |chunk| { + let array = UInt64Array::from_iter(chunk.into_iter()); + RecordBatch::try_new( + Arc::new(ArrowSchema::new(vec![Field::new( + "pos", + DataType::UInt64, + false, + )])), + vec![Arc::new(array)], + ) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RecordBatch for DeleteVector", + ) + }) + .and_then(|batch| ArrowReader::add_file_path_column(batch, &file_path)) + }); + + Ok(Box::pin(stream) as ArrowRecordBatchStream) +} diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index d32cbeb8f9..db240ea336 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -32,6 +32,9 @@ pub(crate) mod record_batch_projector; pub(crate) mod record_batch_transformer; mod value; +mod incremental; +pub use incremental::*; + pub use reader::*; pub use value::*; pub(crate) mod record_batch_partition_splitter; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 05aa6a4c9f..4cea771f95 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -33,6 +33,7 @@ use arrow_string::like::starts_with; use bytes::Bytes; use fnv::FnvHashSet; use futures::future::BoxFuture; +use futures::stream::BoxStream; use futures::{FutureExt, StreamExt, TryFutureExt, TryStreamExt, try_join}; use parquet::arrow::arrow_reader::{ ArrowPredicateFn, ArrowReaderOptions, RowFilter, RowSelection, RowSelector, @@ -57,6 +58,16 @@ use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; +/// Reserved field ID for the row ordinal (_pos) column per Iceberg spec +pub(crate) const RESERVED_FIELD_ID_POS: i32 = 2147483645; +/// Reserved field ID for the file path (_file) column per Iceberg spec +pub(crate) const RESERVED_FIELD_ID_FILE: i32 = 2147483646; + +/// Column name for the row ordinal metadata column per Iceberg spec +pub(crate) const RESERVED_COL_NAME_POS: &str = "_pos"; +/// Column name for the file path metadata column per Iceberg spec +pub(crate) const RESERVED_COL_NAME_FILE: &str = "_file"; + /// Builder to create ArrowReader pub struct ArrowReaderBuilder { batch_size: Option, @@ -64,6 +75,8 @@ pub struct ArrowReaderBuilder { concurrency_limit_data_files: usize, row_group_filtering_enabled: bool, row_selection_enabled: bool, + include_row_ordinals: bool, + include_file_path: bool, // TODO: Perhaps we should have a generic API that allows specifying extra field IDs, or metadata column names... } impl ArrowReaderBuilder { @@ -77,6 +90,8 @@ impl ArrowReaderBuilder { concurrency_limit_data_files: num_cpus, row_group_filtering_enabled: true, row_selection_enabled: false, + include_row_ordinals: false, + include_file_path: false, } } @@ -105,6 +120,20 @@ impl ArrowReaderBuilder { self } + /// Determines whether to include row ordinals in the output. + /// When enabled, adds a `_pos` column containing the row position within each parquet file. + pub fn with_row_ordinals(mut self, include_row_ordinals: bool) -> Self { + self.include_row_ordinals = include_row_ordinals; + self + } + + /// Determines whether to include file path in the output. + /// When enabled, adds a `_file` column containing the file path for each row. + pub fn with_file_path(mut self, include_file_path: bool) -> Self { + self.include_file_path = include_file_path; + self + } + /// Build the ArrowReader. pub fn build(self) -> ArrowReader { ArrowReader { @@ -117,6 +146,8 @@ impl ArrowReaderBuilder { concurrency_limit_data_files: self.concurrency_limit_data_files, row_group_filtering_enabled: self.row_group_filtering_enabled, row_selection_enabled: self.row_selection_enabled, + include_row_ordinals: self.include_row_ordinals, + include_file_path: self.include_file_path, } } } @@ -124,15 +155,23 @@ impl ArrowReaderBuilder { /// Reads data from Parquet files #[derive(Clone)] pub struct ArrowReader { - batch_size: Option, - file_io: FileIO, - delete_file_loader: CachingDeleteFileLoader, + pub(crate) batch_size: Option, + pub(crate) file_io: FileIO, + pub(crate) delete_file_loader: CachingDeleteFileLoader, /// the maximum number of data files that can be fetched at the same time - concurrency_limit_data_files: usize, + pub(crate) concurrency_limit_data_files: usize, - row_group_filtering_enabled: bool, - row_selection_enabled: bool, + pub(crate) row_group_filtering_enabled: bool, + pub(crate) row_selection_enabled: bool, + pub(crate) include_row_ordinals: bool, + pub(crate) include_file_path: bool, +} + +/// Trait indicating that Arrow batches can be read from an implementing type. +pub trait ArrowBatchEmitter { + /// Reads Arrow batches from the implementing type. + fn read(self, reader: R) -> Result>>; } impl ArrowReader { @@ -144,6 +183,8 @@ impl ArrowReader { let concurrency_limit_data_files = self.concurrency_limit_data_files; let row_group_filtering_enabled = self.row_group_filtering_enabled; let row_selection_enabled = self.row_selection_enabled; + let include_row_ordinals = self.include_row_ordinals; + let include_file_path = self.include_file_path; let stream = tasks .map_ok(move |task| { @@ -156,6 +197,8 @@ impl ArrowReader { self.delete_file_loader.clone(), row_group_filtering_enabled, row_selection_enabled, + include_row_ordinals, + include_file_path, ) }) .map_err(|err| { @@ -175,6 +218,8 @@ impl ArrowReader { delete_file_loader: CachingDeleteFileLoader, row_group_filtering_enabled: bool, row_selection_enabled: bool, + include_row_ordinals: bool, + include_file_path: bool, ) -> Result { let should_load_page_index = (row_selection_enabled && task.predicate.is_some()) || !task.deletes.is_empty(); @@ -185,16 +230,36 @@ impl ArrowReader { &task.data_file_path, file_io.clone(), should_load_page_index, + include_row_ordinals, ) .await?; + // Add reserved field IDs for metadata columns when enabled + let mut extended_project_field_ids = task.project_field_ids.clone(); + let extended_schema = if include_row_ordinals { + // Per Iceberg spec, _pos column has reserved field ID RESERVED_FIELD_ID_POS + extended_project_field_ids.push(RESERVED_FIELD_ID_POS); + + // Extend the schema to include the _pos field so RecordBatchTransformer can find it + let mut fields = task.schema.as_struct().fields().to_vec(); + use crate::spec::{NestedField, PrimitiveType, Type}; + fields.push(Arc::new(NestedField::required(RESERVED_FIELD_ID_POS, RESERVED_COL_NAME_POS, Type::Primitive(PrimitiveType::Long)))); + Arc::new(Schema::builder() + .with_schema_id(task.schema.schema_id()) + .with_fields(fields) + .build()?) + } else { + task.schema_ref() + }; + // Create a projection mask for the batch stream to select which columns in the // Parquet file that we want in the response let projection_mask = Self::get_arrow_projection_mask( - &task.project_field_ids, - &task.schema, + &extended_project_field_ids, + &extended_schema, record_batch_stream_builder.parquet_schema(), record_batch_stream_builder.schema(), + include_row_ordinals, )?; record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); @@ -202,7 +267,7 @@ impl ArrowReader { // that come back from the file, such as type promotion, default column insertion // and column re-ordering let mut record_batch_transformer = - RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()); + RecordBatchTransformer::build(extended_schema, &extended_project_field_ids); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -312,11 +377,19 @@ impl ArrowReader { // Build the batch stream and send all the RecordBatches that it generates // to the requester. + let file_path = task.data_file_path.clone(); let record_batch_stream = record_batch_stream_builder .build()? .map(move |batch| match batch { - Ok(batch) => record_batch_transformer.process_record_batch(batch), + Ok(batch) => { + let batch = record_batch_transformer.process_record_batch(batch)?; + if include_file_path { + Self::add_file_path_column(batch, &file_path) + } else { + Ok(batch) + } + } Err(err) => Err(err.into()), }); @@ -327,6 +400,7 @@ impl ArrowReader { data_file_path: &str, file_io: FileIO, should_load_page_index: bool, + include_row_ordinals: bool, ) -> Result>> { // Get the metadata for the Parquet file we need to read and build // a reader for the data within @@ -339,11 +413,18 @@ impl ArrowReader { .with_preload_page_index(should_load_page_index); // Create the record batch stream builder, which wraps the parquet file reader - let record_batch_stream_builder = ParquetRecordBatchStreamBuilder::new_with_options( + let mut record_batch_stream_builder = ParquetRecordBatchStreamBuilder::new_with_options( parquet_file_reader, ArrowReaderOptions::new(), ) .await?; + + // Add row number column if requested (for changelog scans) + // Per Iceberg spec, the position column should be named RESERVED_COL_NAME_POS with field ID RESERVED_FIELD_ID_POS + if include_row_ordinals { + record_batch_stream_builder = record_batch_stream_builder.with_row_number_column(RESERVED_COL_NAME_POS); + } + Ok(record_batch_stream_builder) } @@ -352,7 +433,7 @@ impl ArrowReader { /// Using the Parquet page index, we build a `RowSelection` that rejects rows that are indicated /// as having been deleted by a positional delete, taking into account any row groups that have /// been skipped entirely by the filter predicate - fn build_deletes_row_selection( + pub(crate) fn build_deletes_row_selection( row_group_metadata_list: &[RowGroupMetaData], selected_row_groups: &Option>, positional_deletes: &DeleteVector, @@ -457,7 +538,62 @@ impl ArrowReader { Ok(results.into()) } - fn build_field_id_set_and_map( + /// Adds a `_file` column to the RecordBatch containing the file path. + /// Uses Run-End Encoding (RLE) for maximum memory efficiency when the same + /// file path is repeated across all rows. + pub(crate) fn add_file_path_column(batch: RecordBatch, file_path: &str) -> Result { + use arrow_array::{Int32Array, RunArray, StringArray}; + use arrow_schema::Field; + use std::collections::HashMap; + + let num_rows = batch.num_rows(); + + // Use Run-End Encoded array for optimal memory efficiency + // For a constant value repeated num_rows times, this stores: + // - run_ends: [num_rows] (one i32) + // - values: [file_path] (one string) + let run_ends = Int32Array::from(vec![num_rows as i32]); + let values = StringArray::from(vec![file_path]); + // TODO @vustef L0: These may not be supported in Julia's Arrow.jl, see what alternatives we have... + let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RunArray for _file column", + ) + .with_source(e) + })?; + + let mut columns = batch.columns().to_vec(); + columns.push(Arc::new(file_array) as ArrayRef); + + let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + // DataType is RunEndEncoded with Int32 run ends and Utf8 values + // Note: values field is nullable to match what StringArray::from() creates // TODO @vustef: Not sure why is that the case, fix it. + let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); + let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); + let file_field = Field::new( + RESERVED_COL_NAME_FILE, + DataType::RunEndEncoded(run_ends_field, values_field), + false + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_FILE.to_string(), + )])); + fields.push(Arc::new(file_field)); + + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to add _file column to RecordBatch", + ) + .with_source(e) + }) + } + + pub(crate) fn build_field_id_set_and_map( parquet_schema: &SchemaDescriptor, predicate: &BoundPredicate, ) -> Result<(HashSet, HashMap)> { @@ -475,7 +611,7 @@ impl ArrowReader { /// Insert the leaf field id into the field_ids using for projection. /// For nested type, it will recursively insert the leaf field id. - fn include_leaf_field_id(field: &NestedField, field_ids: &mut Vec) { + pub(crate) fn include_leaf_field_id(field: &NestedField, field_ids: &mut Vec) { match field.field_type.as_ref() { Type::Primitive(_) => { field_ids.push(field.id); @@ -495,12 +631,35 @@ impl ArrowReader { } } - fn get_arrow_projection_mask( + pub(crate) fn get_arrow_projection_mask( field_ids: &[i32], iceberg_schema_of_task: &Schema, parquet_schema: &SchemaDescriptor, arrow_schema: &ArrowSchemaRef, + include_row_ordinals: bool, ) -> Result { + // WORKAROUND: The parquet reader's `with_row_number_column()` method adds the _pos column + // dynamically when building record batches, but does NOT update the schema returned by + // ParquetRecordBatchStreamBuilder::schema(). This means the arrow_schema parameter here + // won't include the _pos field even though it will be present in the actual record batches. + // To work around this, we manually extend the arrow_schema to include the _pos field + // when include_row_ordinals is true. + let arrow_schema = if include_row_ordinals { + let mut fields: Vec<_> = arrow_schema.fields().iter().cloned().collect(); + let pos_field = arrow_schema::Field::new(RESERVED_COL_NAME_POS, DataType::Int64, false) + .with_metadata(std::collections::HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_POS.to_string(), + )])); + fields.push(Arc::new(pos_field)); + Arc::new(ArrowSchema::new_with_metadata( + fields, + arrow_schema.metadata().clone(), + )) + } else { + arrow_schema.clone() + }; + fn type_promotion_is_valid( file_type: Option<&PrimitiveType>, projected_type: Option<&PrimitiveType>, @@ -540,6 +699,7 @@ impl ArrowReader { let mut column_map = HashMap::new(); let fields = arrow_schema.fields(); + // Pre-project only the fields that have been selected, possibly avoiding converting // some Arrow types that are not yet supported. let mut projected_fields: HashMap = HashMap::new(); @@ -602,6 +762,12 @@ impl ArrowReader { let mut indices = vec![]; for field_id in leaf_field_ids { + // Skip _pos field - it's added dynamically by the parquet reader via + // with_row_number_column(), not read from the parquet file itself + if field_id == RESERVED_FIELD_ID_POS { + continue; + } + if let Some(col_idx) = column_map.get(&field_id) { indices.push(*col_idx); } else { @@ -1373,19 +1539,16 @@ impl ArrowFileReader { } impl AsyncFileReader for ArrowFileReader { - fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { Box::pin( self.r - .read(range.start..range.end) + .read(range.start as u64..range.end as u64) .map_err(|err| parquet::errors::ParquetError::External(Box::new(err))), ) } - // TODO: currently we don't respect `ArrowReaderOptions` cause it don't expose any method to access the option field - // we will fix it after `v55.1.0` is released in https://github.com/apache/arrow-rs/issues/7393 fn get_metadata( &mut self, - _options: Option<&'_ ArrowReaderOptions>, ) -> BoxFuture<'_, parquet::errors::Result>> { async move { let reader = ParquetMetaDataReader::new() @@ -1393,7 +1556,7 @@ impl AsyncFileReader for ArrowFileReader { .with_column_indexes(self.preload_column_index) .with_page_indexes(self.preload_page_index) .with_offset_indexes(self.preload_offset_index); - let size = self.meta.size; + let size = self.meta.size as usize; let meta = reader.load_and_finish(self, size).await?; Ok(Arc::new(meta)) @@ -1580,6 +1743,7 @@ message schema { &schema, &parquet_schema, &arrow_schema, + false, ) .unwrap_err(); @@ -1595,6 +1759,7 @@ message schema { &schema, &parquet_schema, &arrow_schema, + false, ) .unwrap_err(); @@ -1606,7 +1771,7 @@ message schema { // Finally avoid selecting fields with unsupported data types let mask = - ArrowReader::get_arrow_projection_mask(&[1], &schema, &parquet_schema, &arrow_schema) + ArrowReader::get_arrow_projection_mask(&[1], &schema, &parquet_schema, &arrow_schema, false) .expect("Some ProjectionMask"); assert_eq!(mask, ProjectionMask::leaves(&parquet_schema, vec![0])); } diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index d8f7a872e1..447ff61188 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -73,6 +73,7 @@ impl DeleteFileIndex { let mut guard = state.write().unwrap(); *guard = DeleteFileIndexState::Populated(populated_delete_file_index); } + notify.notify_waiters(); } }); @@ -106,6 +107,26 @@ impl DeleteFileIndex { _ => unreachable!("Cannot be any other state than loaded"), } } + + pub(crate) async fn positional_deletes(&self) -> Vec { + let notifier = { + let guard = self.state.read().unwrap(); + match *guard { + DeleteFileIndexState::Populating(ref notifier) => notifier.clone(), + DeleteFileIndexState::Populated(ref index) => { + return index.positional_deletes(); + } + } + }; + + notifier.notified().await; + + let guard = self.state.read().unwrap(); + match guard.deref() { + DeleteFileIndexState::Populated(index) => index.positional_deletes(), + _ => unreachable!("Cannot be any other state than loaded"), + } + } } impl PopulatedDeleteFileIndex { @@ -207,4 +228,12 @@ impl PopulatedDeleteFileIndex { results } + + fn positional_deletes(&self) -> Vec { + self.pos_deletes_by_partition + .values() + .flatten() + .map(|ctx| ctx.as_ref().into()) + .collect() + } } diff --git a/crates/iceberg/src/delete_vector.rs b/crates/iceberg/src/delete_vector.rs index f382bf079e..db13e9aba0 100644 --- a/crates/iceberg/src/delete_vector.rs +++ b/crates/iceberg/src/delete_vector.rs @@ -23,9 +23,9 @@ use roaring::treemap::BitmapIter; use crate::{Error, ErrorKind, Result}; -#[derive(Debug, Default)] +#[derive(Debug, Default, Clone, PartialEq)] pub struct DeleteVector { - inner: RoaringTreemap, + pub inner: RoaringTreemap, } impl DeleteVector { diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs index aae8efed74..9c9c7460fb 100644 --- a/crates/iceberg/src/lib.rs +++ b/crates/iceberg/src/lib.rs @@ -97,3 +97,6 @@ pub mod writer; mod delete_vector; pub mod puffin; + +/// Utility functions and modules. +pub mod util; diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 3f7c29dbf4..2290270b57 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -33,19 +33,23 @@ use crate::spec::{ }; use crate::{Error, ErrorKind, Result}; +pub(crate) type ManifestEntryFilterFn = dyn Fn(&ManifestEntryRef) -> bool + Send + Sync; + /// Wraps a [`ManifestFile`] alongside the objects that are needed /// to process it in a thread-safe manner pub(crate) struct ManifestFileContext { - manifest_file: ManifestFile, + pub manifest_file: ManifestFile, - sender: Sender, + pub sender: Sender, - field_ids: Arc>, - bound_predicates: Option>, - object_cache: Arc, - snapshot_schema: SchemaRef, - expression_evaluator_cache: Arc, - delete_file_index: DeleteFileIndex, + pub field_ids: Arc>, + pub bound_predicates: Option>, + pub object_cache: Arc, + pub snapshot_schema: SchemaRef, + pub expression_evaluator_cache: Arc, + pub delete_file_index: DeleteFileIndex, + + pub filter_fn: Option>, } /// Wraps a [`ManifestEntryRef`] alongside the objects that are needed @@ -74,12 +78,15 @@ impl ManifestFileContext { mut sender, expression_evaluator_cache, delete_file_index, + filter_fn, .. } = self; + let filter_fn = filter_fn.unwrap_or_else(|| Arc::new(|_| true)); + let manifest = object_cache.get_manifest(&manifest_file).await?; - for manifest_entry in manifest.entries() { + for manifest_entry in manifest.entries().iter().filter(|e| filter_fn(e)) { let manifest_entry_context = ManifestEntryContext { // TODO: refactor to avoid the expensive ManifestEntry clone manifest_entry: manifest_entry.clone(), @@ -224,6 +231,7 @@ impl PlanContext { partition_bound_predicate, tx, delete_file_idx.clone(), + None, ); filtered_mfcs.push(Ok(mfc)); @@ -238,6 +246,7 @@ impl PlanContext { partition_filter: Option>, sender: Sender, delete_file_index: DeleteFileIndex, + filter_fn: Option>, ) -> ManifestFileContext { let bound_predicates = if let (Some(ref partition_bound_predicate), Some(snapshot_bound_predicate)) = @@ -260,6 +269,7 @@ impl PlanContext { field_ids: self.field_ids.clone(), expression_evaluator_cache: self.expression_evaluator_cache.clone(), delete_file_index, + filter_fn, } } } diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs new file mode 100644 index 0000000000..5f4d795222 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -0,0 +1,120 @@ +use std::collections::HashSet; +use std::sync::Arc; + +use futures::channel::mpsc::Sender; + +use crate::Result; +use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; +use crate::delete_file_index::DeleteFileIndex; +use crate::io::object_cache::ObjectCache; +use crate::scan::context::{ManifestEntryContext, ManifestEntryFilterFn, ManifestFileContext}; +use crate::scan::{ExpressionEvaluatorCache, ManifestEvaluatorCache, PartitionFilterCache}; +use crate::spec::{ + ManifestContentType, ManifestEntryRef, ManifestFile, Operation, SchemaRef, SnapshotRef, + TableMetadataRef, +}; + +#[derive(Debug)] +pub(crate) struct IncrementalPlanContext { + /// The snapshots involved in the incremental scan. + pub snapshots: Vec, + + /// The snapshot to start the incremental scan from. + pub from_snapshot: Option, + + /// The metadata of the table being scanned. + pub table_metadata: TableMetadataRef, + + /// The schema of the snapshot to end the incremental scan at. + pub to_snapshot_schema: SchemaRef, + + /// The object cache to use for the scan. + pub object_cache: Arc, + + /// The field IDs to scan. + pub field_ids: Arc>, + + /// The partition filter cache to use for the scan. + pub partition_filter_cache: Arc, + + /// The manifest evaluator cache to use for the scan. + pub manifest_evaluator_cache: Arc, + + /// The expression evaluator cache to use for the scan. + pub expression_evaluator_cache: Arc, + + /// The caching delete file loader to use for the scan. + pub caching_delete_file_loader: CachingDeleteFileLoader, +} + +impl IncrementalPlanContext { + pub(crate) async fn build_manifest_file_contexts( + &self, + tx_data: Sender, + delete_file_idx: DeleteFileIndex, + delete_file_tx: Sender, + ) -> Result> + 'static>> { + let (manifest_files, filter_fn) = { + let snapshot_ids: HashSet = self + .snapshots + .iter() + .filter(|snapshot| { + matches!( + snapshot.summary().operation, + Operation::Append | Operation::Delete + ) + }) + .map(|snapshot| snapshot.snapshot_id()) + .collect(); + + let mut manifest_files = HashSet::::new(); + for snapshot in self.snapshots.iter() { + let manifest_list = self + .object_cache + .get_manifest_list(&snapshot, &self.table_metadata) + .await?; + for entry in manifest_list.entries() { + if !snapshot_ids.contains(&entry.added_snapshot_id) { + continue; + } + manifest_files.insert(entry.clone()); + } + } + let filter_fn: Option> = + Some(Arc::new(move |entry: &ManifestEntryRef| { + entry + .snapshot_id() + .map(|id| snapshot_ids.contains(&id)) + .unwrap_or(true) // Include entries without `snapshot_id`. + })); + + (manifest_files, filter_fn) + }; + + // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. + let mut mfcs = vec![]; + for manifest_file in &manifest_files { + let tx = if manifest_file.content == ManifestContentType::Deletes { + delete_file_tx.clone() + } else { + tx_data.clone() + }; + + let mfc = ManifestFileContext { + manifest_file: manifest_file.clone(), + bound_predicates: None, + sender: tx, + object_cache: self.object_cache.clone(), + snapshot_schema: self.to_snapshot_schema.clone(), + field_ids: self.field_ids.clone(), + expression_evaluator_cache: self.expression_evaluator_cache.clone(), + delete_file_index: delete_file_idx.clone(), + filter_fn: filter_fn.clone(), + }; + + mfcs.push(Ok(mfc)); + } + + Ok(Box::new(mfcs.into_iter())) + } +} diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs new file mode 100644 index 0000000000..1d4474f40f --- /dev/null +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -0,0 +1,522 @@ +//! Incremental table scan implementation. + +use std::collections::HashSet; +use std::sync::Arc; + +use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; +use crate::arrow::delete_filter::DeleteFilter; +use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, IncrementalArrowBatchRecordStream}; +use crate::delete_file_index::DeleteFileIndex; +use crate::io::FileIO; +use crate::scan::{ArrowRecordBatchStream, DeleteFileContext}; +use crate::scan::cache::{ExpressionEvaluatorCache, ManifestEvaluatorCache, PartitionFilterCache}; +use crate::scan::context::ManifestEntryContext; +use crate::spec::{DataContentType, ManifestStatus, Snapshot, SnapshotRef}; +use crate::table::Table; +use crate::util::snapshot::ancestors_between; +use crate::utils::available_parallelism; +use crate::{Error, ErrorKind, Result}; + +mod context; +use context::*; +mod task; +use futures::channel::mpsc::{Sender, channel}; +use futures::{SinkExt, StreamExt, TryStreamExt}; +use itertools::Itertools; +pub use task::*; + +use crate::runtime::spawn; + +/// Builder for an incremental table scan. +#[derive(Debug)] +pub struct IncrementalTableScanBuilder<'a> { + table: &'a Table, + // Defaults to `None`, which means all columns. + column_names: Option>, + from_snapshot_id: Option, + to_snapshot_id: i64, + batch_size: Option, + concurrency_limit_data_files: usize, + concurrency_limit_manifest_entries: usize, + concurrency_limit_manifest_files: usize, +} + +impl<'a> IncrementalTableScanBuilder<'a> { + pub(crate) fn new(table: &'a Table, from_snapshot_id: i64, to_snapshot_id: i64) -> Self { + let num_cpus = available_parallelism().get(); + Self { + table, + column_names: None, + from_snapshot_id: Some(from_snapshot_id), + to_snapshot_id, + batch_size: None, + concurrency_limit_data_files: num_cpus, + concurrency_limit_manifest_entries: num_cpus, + concurrency_limit_manifest_files: num_cpus, + } + } + + /// Set the batch size for reading data files. + pub fn with_batch_size(mut self, batch_size: Option) -> Self { + self.batch_size = batch_size; + self + } + + /// Select all columns of the table. + pub fn select_all(mut self) -> Self { + self.column_names = None; + self + } + + /// Select no columns of the table. + pub fn select_empty(mut self) -> Self { + self.column_names = Some(vec![]); + self + } + + /// Select some columns of the table. + pub fn select(mut self, column_names: impl IntoIterator) -> Self { + self.column_names = Some( + column_names + .into_iter() + .map(|item| item.to_string()) + .collect(), + ); + self + } + + /// Set the `from_snapshot_id` for the incremental scan. + pub fn from_snapshot_id(mut self, from_snapshot_id: i64) -> Self { + self.from_snapshot_id = Some(from_snapshot_id); + self + } + + /// Set the `to_snapshot_id` for the incremental scan. + pub fn to_snapshot_id(mut self, to_snapshot_id: i64) -> Self { + self.to_snapshot_id = to_snapshot_id; + self + } + + /// Set the concurrency limit for reading data files. + pub fn with_concurrency_limit_data_files(mut self, limit: usize) -> Self { + self.concurrency_limit_data_files = limit; + self + } + + /// Set the concurrency limit for reading manifest entries. + pub fn with_concurrency_limit_manifest_entries(mut self, limit: usize) -> Self { + self.concurrency_limit_manifest_entries = limit; + self + } + + /// Set the concurrency limit for reading manifest files. + pub fn with_concurrency_limit_manifest_files(mut self, limit: usize) -> Self { + self.concurrency_limit_manifest_files = limit; + self + } + + /// Build the incremental table scan. + pub fn build(self) -> Result { + let snapshot_from: Option> = match self.from_snapshot_id { + Some(id) => Some( + self.table + .metadata() + .snapshot_by_id(id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", id), + ) + })? + .clone(), + ), + None => None, + }; + + let snapshot_to: Arc = self + .table + .metadata() + .snapshot_by_id(self.to_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", self.to_snapshot_id), + ) + })? + .clone(); + + // TODO: What properties do we need to verify about the snapshots? What about + // schema changes? + let snapshots = ancestors_between( + &self.table.metadata_ref(), + snapshot_to.snapshot_id(), + snapshot_from.as_ref().map(|s| s.snapshot_id()), + ) + .collect_vec(); + + assert_eq!( + snapshots.first().map(|s| s.snapshot_id()), + Some(snapshot_to.snapshot_id()) + ); + + let schema = snapshot_to.schema(self.table.metadata())?; + + if let Some(column_names) = self.column_names.as_ref() { + for column_name in column_names { + if schema.field_by_name(column_name).is_none() { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Column {} not found in table. Schema: {}", + column_name, schema + ), + )); + } + } + } + + let mut field_ids = vec![]; + let column_names = self.column_names.clone().unwrap_or_else(|| { + schema + .as_struct() + .fields() + .iter() + .map(|f| f.name.clone()) + .collect() + }); + + for column_name in column_names.iter() { + let field_id = schema.field_id_by_name(column_name).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Column {} not found in table. Schema: {}", + column_name, schema + ), + ) + })?; + + schema + .as_struct() + .field_by_id(field_id) + .ok_or_else(|| { + Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Column {} is not a direct child of schema but a nested field, which is not supported now. Schema: {}", + column_name, schema + ), + ) + })?; + + field_ids.push(field_id); + } + + let plan_context = IncrementalPlanContext { + snapshots, + from_snapshot: snapshot_from, + table_metadata: self.table.metadata_ref(), + to_snapshot_schema: schema, + object_cache: self.table.object_cache().clone(), + field_ids: Arc::new(field_ids), + partition_filter_cache: Arc::new(PartitionFilterCache::new()), + manifest_evaluator_cache: Arc::new(ManifestEvaluatorCache::new()), + expression_evaluator_cache: Arc::new(ExpressionEvaluatorCache::new()), + caching_delete_file_loader: CachingDeleteFileLoader::new( + self.table.file_io().clone(), + self.concurrency_limit_data_files, + ), + }; + + Ok(IncrementalTableScan { + plan_context, + file_io: self.table.file_io().clone(), + column_names: Some(column_names), + batch_size: self.batch_size, + concurrency_limit_data_files: self.concurrency_limit_data_files, + concurrency_limit_manifest_entries: self.concurrency_limit_manifest_entries, + concurrency_limit_manifest_files: self.concurrency_limit_manifest_files, + }) + } +} + +/// An incremental table scan. +#[derive(Debug)] +pub struct IncrementalTableScan { + plan_context: IncrementalPlanContext, + file_io: FileIO, + column_names: Option>, + batch_size: Option, + concurrency_limit_data_files: usize, + concurrency_limit_manifest_entries: usize, + concurrency_limit_manifest_files: usize, +} + +impl IncrementalTableScan { + /// Returns the optional `from` snapshot of this incremental table scan. + pub fn snapshot_from(&self) -> Option<&SnapshotRef> { + self.plan_context.from_snapshot.as_ref() + } + + /// Returns the snapshots involved in this incremental table scan. + pub fn snapshots(&self) -> &[SnapshotRef] { + &self.plan_context.snapshots + } + + /// Returns the `to` snapshot of this incremental table scan. + pub fn snapshot_to(&self) -> &SnapshotRef { + self.snapshots() + .first() + .expect("There is always at least one snapshot") + } + + /// Returns the selected column names of this incremental table scan. + /// If `None`, all columns are selected. + pub fn column_names(&self) -> Option<&[String]> { + self.column_names.as_deref() + } + + /// Plans the files to be read in this incremental table scan. + pub async fn plan_files(&self) -> Result { + let concurrency_limit_manifest_files = self.concurrency_limit_manifest_files; + let concurrency_limit_manifest_entries = self.concurrency_limit_manifest_entries; + + // Used to stream `ManifestEntryContexts` between stages of the planning operation. + let (manifest_entry_data_ctx_tx, manifest_entry_data_ctx_rx) = + channel(concurrency_limit_manifest_files); + let (manifest_entry_delete_ctx_tx, manifest_entry_delete_ctx_rx) = + channel(concurrency_limit_manifest_files); + + // Used to stream the results back to the caller. + let (file_scan_task_tx, file_scan_task_rx) = channel(concurrency_limit_manifest_entries); + + let (delete_file_idx, delete_file_tx) = DeleteFileIndex::new(); + + let manifest_file_contexts = self + .plan_context + .build_manifest_file_contexts( + manifest_entry_data_ctx_tx, + delete_file_idx.clone(), + manifest_entry_delete_ctx_tx, + ) + .await?; + + let mut channel_for_manifest_error: Sender> = file_scan_task_tx.clone(); + + // Concurrently load all [`Manifest`]s and stream their [`ManifestEntry`]s + spawn(async move { + let result = futures::stream::iter(manifest_file_contexts) + .try_for_each_concurrent(concurrency_limit_manifest_files, |ctx| async move { + ctx.fetch_manifest_and_stream_manifest_entries().await + }) + .await; + + if let Err(error) = result { + let _ = channel_for_manifest_error.send(Err(error)).await; + } + }); + + let mut channel_for_data_manifest_entry_error = file_scan_task_tx.clone(); + let mut channel_for_delete_manifest_entry_error = file_scan_task_tx.clone(); + + // Process the delete file [`ManifestEntry`] stream in parallel + spawn(async move { + let result = manifest_entry_delete_ctx_rx + .map(|me_ctx| Ok((me_ctx, delete_file_tx.clone()))) + .try_for_each_concurrent( + concurrency_limit_manifest_entries, + |(manifest_entry_context, tx)| async move { + spawn(async move { + Self::process_delete_manifest_entry(tx, manifest_entry_context).await + }) + .await + }, + ) + .await; + + if let Err(error) = result { + let _ = channel_for_delete_manifest_entry_error + .send(Err(error)) + .await; + } + }) + .await; + + // TODO: Streaming this into the delete index seems somewhat redundant, as we + // could directly stream into the CachingDeleteFileLoader and instantly load the + // delete files. + let positional_deletes = delete_file_idx.positional_deletes().await; + let result = self + .plan_context + .caching_delete_file_loader + .load_deletes( + &positional_deletes, + self.plan_context.to_snapshot_schema.clone(), + ) + .await; + + let delete_filter = match result { + Ok(loaded_deletes) => loaded_deletes.unwrap(), + Err(e) => { + return Err(Error::new( + ErrorKind::Unexpected, + format!("Failed to load positional deletes: {}", e), + )); + } + }; + + // Process the data file [`ManifestEntry`] stream in parallel + let filter = delete_filter.clone(); + spawn(async move { + let result = manifest_entry_data_ctx_rx + .map(|me_ctx| Ok((me_ctx, file_scan_task_tx.clone()))) + .try_for_each_concurrent( + concurrency_limit_manifest_entries, + |(manifest_entry_context, tx)| { + let filter = filter.clone(); + async move { + if manifest_entry_context.manifest_entry.status() + == ManifestStatus::Added + { + spawn(async move { + Self::process_data_manifest_entry( + tx, + manifest_entry_context, + &filter, + ) + .await + }) + .await + } else if manifest_entry_context.manifest_entry.status() + == ManifestStatus::Deleted + { + // TODO: Process deleted files + Ok(()) + } else { + Ok(()) + } + } + }, + ) + .await; + + if let Err(error) = result { + let _ = channel_for_data_manifest_entry_error.send(Err(error)).await; + } + }); + + let mut tasks = file_scan_task_rx.try_collect::>().await?; + + let appended_files = tasks + .iter() + .filter_map(|task| match task { + IncrementalFileScanTask::Append(append_task) => { + Some(append_task.data_file_path.clone()) + } + _ => None, + }) + .collect::>(); + + delete_filter.with_read(|state| { + for (path, delete_vector) in state.delete_vectors().iter() { + if !appended_files.contains::(path) { + let delete_task = + IncrementalFileScanTask::Delete(path.clone(), delete_vector.clone()); + tasks.push(delete_task); + } + } + Ok(()) + })?; + + for task in tasks.iter() { + match task { + IncrementalFileScanTask::Append(append_task) => { + println!( + "Planned incremental append file scan task: {:?}, deletes: {:?}", + append_task.data_file_path, append_task.positional_deletes, + ); + } + IncrementalFileScanTask::Delete(delete_path, _) => { + println!( + "Planned incremental delete file scan task: {:?}", + delete_path, + ); + } + } + } + + // We actually would not need a stream here, but we can keep it compatible with + // other scan types. + Ok(futures::stream::iter(tasks).map(|t| Ok(t)).boxed()) + } + + /// Returns an [`IncrementalArrowBatchRecordStream`] for this incremental table scan. + pub async fn to_arrow(&self) -> Result { + let file_scan_task_stream = self.plan_files().await?; + let mut arrow_reader_builder = + ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); + + if let Some(batch_size) = self.batch_size { + arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); + } + + let arrow_reader = arrow_reader_builder.build(); + file_scan_task_stream.read(arrow_reader) + } + + async fn process_delete_manifest_entry( + mut delete_file_ctx_tx: Sender, + manifest_entry_context: ManifestEntryContext, + ) -> Result<()> { + // Skip processing this manifest entry if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() { + return Ok(()); + } + + // Abort the plan if we encounter a manifest entry for a data file + if manifest_entry_context.manifest_entry.content_type() == DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a data file in a delete file manifest", + )); + } + + delete_file_ctx_tx + .send(DeleteFileContext { + manifest_entry: manifest_entry_context.manifest_entry.clone(), + partition_spec_id: manifest_entry_context.partition_spec_id, + }) + .await?; + Ok(()) + } + + async fn process_data_manifest_entry( + mut file_scan_task_tx: Sender>, + manifest_entry_context: ManifestEntryContext, + delete_filter: &DeleteFilter, + ) -> Result<()> { + // Skip processing this manifest entry if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() { + return Ok(()); + } + + // Abort the plan if we encounter a manifest entry for a delete file + if manifest_entry_context.manifest_entry.content_type() != DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a delete file in a data file manifest", + )); + } + + let file_scan_task = IncrementalFileScanTask::append_from_manifest_entry( + &manifest_entry_context, + delete_filter, + ); + + file_scan_task_tx.send(Ok(file_scan_task)).await?; + Ok(()) + } +} diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs new file mode 100644 index 0000000000..2ef6a63bd2 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -0,0 +1,87 @@ +use std::sync::{Arc, Mutex}; + +use futures::stream::BoxStream; + +use crate::Result; +use crate::arrow::delete_filter::DeleteFilter; +use crate::delete_vector::DeleteVector; +use crate::scan::context::ManifestEntryContext; +use crate::spec::{DataFileFormat, Schema, SchemaRef}; + +/// A file scan task for appended data files in an incremental scan. +#[derive(Debug, Clone)] +pub struct AppendedFileScanTask { + /// The start offset of the file to scan. + pub start: u64, + /// The length of the file to scan. + pub length: u64, + /// The number of records in the file. + pub record_count: Option, + /// The path to the data file to scan. + pub data_file_path: String, + /// The format of the data file to scan. + pub data_file_format: DataFileFormat, + /// The schema of the data file to scan. + pub schema: crate::spec::SchemaRef, + /// The field ids to project. + pub project_field_ids: Vec, + /// The optional positional deletes associated with this data file. + pub positional_deletes: Option>>, +} + +impl AppendedFileScanTask { + /// Returns the data file path of this appended file scan task. + pub fn data_file_path(&self) -> &str { + &self.data_file_path + } + + /// Returns the schema of this file scan task as a reference + pub fn schema(&self) -> &Schema { + &self.schema + } + + /// Returns the schema of this file scan task as a SchemaRef + pub fn schema_ref(&self) -> SchemaRef { + self.schema.clone() + } +} + +/// The stream of incremental file scan tasks. +pub type IncrementalFileScanTaskStream = BoxStream<'static, Result>; + +/// An incremental file scan task, which can be either an appended data file. +#[derive(Debug, Clone)] +pub enum IncrementalFileScanTask { + /// An appended data file. + Append(AppendedFileScanTask), + /// Deleted records of a data file. + Delete(String, Arc>), +} + +impl IncrementalFileScanTask { + /// Create an `IncrementalFileScanTask::Append` from a `ManifestEntryContext` and `DeleteFilter`. + pub(crate) fn append_from_manifest_entry( + manifest_entry_context: &ManifestEntryContext, + delete_filter: &DeleteFilter, + ) -> Self { + let data_file_path = manifest_entry_context.manifest_entry.file_path(); + IncrementalFileScanTask::Append(AppendedFileScanTask { + start: 0, + length: manifest_entry_context.manifest_entry.file_size_in_bytes(), + record_count: Some(manifest_entry_context.manifest_entry.record_count()), + data_file_path: data_file_path.to_string(), + data_file_format: manifest_entry_context.manifest_entry.file_format(), + schema: manifest_entry_context.snapshot_schema.clone(), + project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), + positional_deletes: delete_filter.get_delete_vector_for_path(&data_file_path), + }) + } + + /// Returns the data file path of this incremental file scan task. + pub fn data_file_path(&self) -> &str { + match self { + IncrementalFileScanTask::Append(task) => task.data_file_path(), + IncrementalFileScanTask::Delete(path, _) => path, + } + } +} diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 3d14b3cce4..682691dc49 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -21,6 +21,9 @@ mod cache; use cache::*; mod context; use context::*; + +pub mod incremental; + mod task; use std::sync::Arc; diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index d4e696ce84..1c8b43ca04 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -24,6 +24,7 @@ use crate::inspect::MetadataTable; use crate::io::FileIO; use crate::io::object_cache::ObjectCache; use crate::scan::TableScanBuilder; +use crate::scan::incremental::IncrementalTableScanBuilder; use crate::spec::{TableMetadata, TableMetadataRef}; use crate::{Error, ErrorKind, Result, TableIdent}; @@ -224,6 +225,15 @@ impl Table { TableScanBuilder::new(self) } + /// Creates an incremental table scan between two snapshots. + pub fn incremental_scan( + &self, + from_snapshot_id: i64, + to_snapshot_id: i64, + ) -> IncrementalTableScanBuilder<'_> { + IncrementalTableScanBuilder::new(self, from_snapshot_id, to_snapshot_id) + } + /// Creates a metadata table which provides table-like APIs for inspecting metadata. /// See [`MetadataTable`] for more details. pub fn inspect(&self) -> MetadataTable<'_> { diff --git a/crates/iceberg/src/util/mod.rs b/crates/iceberg/src/util/mod.rs new file mode 100644 index 0000000000..b614c981ec --- /dev/null +++ b/crates/iceberg/src/util/mod.rs @@ -0,0 +1,19 @@ +// 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. + +/// Utilities for working with snapshots. +pub mod snapshot; diff --git a/crates/iceberg/src/util/snapshot.rs b/crates/iceberg/src/util/snapshot.rs new file mode 100644 index 0000000000..62aa6769ec --- /dev/null +++ b/crates/iceberg/src/util/snapshot.rs @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::spec::{SnapshotRef, TableMetadataRef}; + +struct Ancestors { + next: Option, + get_snapshot: Box Option + Send>, +} + +impl Iterator for Ancestors { + type Item = SnapshotRef; + + fn next(&mut self) -> Option { + let snapshot = self.next.take()?; + let result = snapshot.clone(); + self.next = snapshot + .parent_snapshot_id() + .and_then(|id| (self.get_snapshot)(id)); + Some(result) + } +} + +/// Iterate starting from `snapshot` (inclusive) to the root snapshot. +pub fn ancestors_of( + table_metadata: &TableMetadataRef, + snapshot: i64, +) -> Box + Send> { + if let Some(snapshot) = table_metadata.snapshot_by_id(snapshot) { + let table_metadata = table_metadata.clone(); + Box::new(Ancestors { + next: Some(snapshot.clone()), + get_snapshot: Box::new(move |id| table_metadata.snapshot_by_id(id).cloned()), + }) + } else { + Box::new(std::iter::empty()) + } +} + +/// Iterate starting from `snapshot` (inclusive) to `oldest_snapshot_id` (exclusive). +pub fn ancestors_between( + table_metadata: &TableMetadataRef, + latest_snapshot_id: i64, + oldest_snapshot_id: Option, +) -> Box + Send> { + let Some(oldest_snapshot_id) = oldest_snapshot_id else { + return Box::new(ancestors_of(table_metadata, latest_snapshot_id)); + }; + + if latest_snapshot_id == oldest_snapshot_id { + return Box::new(std::iter::empty()); + } + + Box::new( + ancestors_of(table_metadata, latest_snapshot_id) + .take_while(move |snapshot| snapshot.snapshot_id() != oldest_snapshot_id), + ) +} diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 620f27df33..aff7d26a30 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -330,7 +330,7 @@ impl ParquetWriter { let reader = input_file.reader().await?; let mut parquet_reader = ArrowFileReader::new(file_metadata, reader); - let parquet_metadata = parquet_reader.get_metadata(None).await.map_err(|err| { + let parquet_metadata = parquet_reader.get_metadata().await.map_err(|err| { Error::new( ErrorKind::DataInvalid, format!("Error reading Parquet metadata: {}", err), diff --git a/crates/playground/Cargo.toml b/crates/playground/Cargo.toml new file mode 100644 index 0000000000..d09970ba6c --- /dev/null +++ b/crates/playground/Cargo.toml @@ -0,0 +1,15 @@ +[package] +name = "playground" +edition.workspace = true +homepage.workspace = true +version.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +dotenv = "0.15.0" +futures-util = "0.3.31" +iceberg.workspace = true +iceberg-catalog-rest = { workspace = true } +tokio.workspace = true diff --git a/crates/playground/src/main.rs b/crates/playground/src/main.rs new file mode 100644 index 0000000000..f7dc65ae34 --- /dev/null +++ b/crates/playground/src/main.rs @@ -0,0 +1,61 @@ +use futures_util::TryStreamExt; +use tokio; +use std::collections::HashMap; +use std::env; +use dotenv::dotenv; +use iceberg::{NamespaceIdent, CatalogBuilder, Catalog}; +use iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, S3_SECRET_ACCESS_KEY}; +use iceberg_catalog_rest::{REST_CATALOG_PROP_URI, REST_CATALOG_PROP_WAREHOUSE, RestCatalogBuilder}; + +#[tokio::main(flavor = "multi_thread")] +async fn main() -> Result<(), Box> { + dotenv().ok(); + + let args: Vec = env::args().collect(); + if args.len() < 5 { + eprintln!("Usage: {} ", args[0]); + return Err("Not enough arguments".into()); + } + let namespace = &args[1]; + let table_name = &args[2]; + let from_snapshot_id: i64 = args[3].parse().unwrap(); + let to_snapshot_id: i64 = args[4].parse().unwrap(); + + let bucket = env::var("S3_BUCKET").expect("S3_BUCKET must be set"); + + // Create catalog + let catalog = RestCatalogBuilder::default() + .load( + "rest", + HashMap::from([ + (REST_CATALOG_PROP_URI.to_string(), "http://localhost:8181".to_string()), + (REST_CATALOG_PROP_WAREHOUSE.to_string(), format!("s3://{}", bucket)), + (S3_ACCESS_KEY_ID.to_string(), env::var("AWS_ACCESS_KEY_ID").unwrap_or("admin".to_string())), + (S3_SECRET_ACCESS_KEY.to_string(), env::var("AWS_SECRET_ACCESS_KEY").unwrap_or("password".to_string())), + (S3_REGION.to_string(), env::var("AWS_REGION").unwrap_or("us-east-1".to_string())), + (S3_ENDPOINT.to_string(), env::var("AWS_ENDPOINT_URL").unwrap_or("http://localhost:9000".to_string())), + ]) + ) + .await?; + + let namespace = NamespaceIdent::new(namespace.clone()); + // Error if namespace doesn't exist + if !catalog.namespace_exists(&namespace).await? { + return Err("Namespace does not exist".into()); + } + + let table = catalog.load_table(&iceberg::TableIdent::new(namespace.clone(), table_name.clone())).await?; + let mut stream = table + .incremental_scan(from_snapshot_id, to_snapshot_id) + .build()? + .to_arrow().await?; + + let mut rows = 0; + while let Some((batch_type, batch)) = stream.try_next().await? { + rows += batch.num_rows(); + println!("Batch: {:?}, Type: {:?}", batch, batch_type); + } + + println!("Total rows: {:?}", rows); + Ok(()) +} From f255405218f78ae645316143f98e8ea50a37aa3b Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 11:41:56 +0200 Subject: [PATCH 02/68] . --- Cargo.lock | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c5741199c5..60d900babb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4188,17 +4188,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "io-uring" -version = "0.7.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046fa2d4d00aea763528b4950358d0ead425372445dc8ff86312b3c69ff7727b" -dependencies = [ - "bitflags 2.9.4", - "cfg-if", - "libc", -] - [[package]] name = "ipnet" version = "2.11.0" From be932c865c2400d24b72f71ace3c819d0045b3a5 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 11:59:43 +0200 Subject: [PATCH 03/68] . --- Cargo.lock | 433 +++++------------- Cargo.toml | 18 +- .../iceberg/src/arrow/delete_file_loader.rs | 1 - crates/iceberg/src/arrow/incremental.rs | 83 +--- crates/iceberg/src/arrow/reader.rs | 121 +---- crates/iceberg/src/scan/incremental/mod.rs | 11 +- .../src/writer/file_writer/parquet_writer.rs | 2 +- 7 files changed, 155 insertions(+), 514 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 60d900babb..fe76f7e4b8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -224,32 +224,19 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" dependencies = [ - "arrow-arith 55.2.0", - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-cast 55.2.0", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", "arrow-csv", - "arrow-data 55.2.0", - "arrow-ipc 55.2.0", + "arrow-data", + "arrow-ipc", "arrow-json", - "arrow-ord 55.2.0", + "arrow-ord", "arrow-row", - "arrow-schema 55.2.0", - "arrow-select 55.2.0", - "arrow-string 55.2.0", -] - -[[package]] -name = "arrow-arith" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", - "chrono", - "num", + "arrow-schema", + "arrow-select", + "arrow-string", ] [[package]] @@ -258,26 +245,11 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30feb679425110209ae35c3fbf82404a39a4c0436bb3ec36164d8bffed2a4ce4" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", - "chrono", - "num", -] - -[[package]] -name = "arrow-array" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "ahash 0.8.12", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", "chrono", - "half", - "hashbrown 0.15.5", "num", ] @@ -288,9 +260,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" dependencies = [ "ahash 0.8.12", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", + "arrow-buffer", + "arrow-data", + "arrow-schema", "chrono", "chrono-tz 0.10.4", "half", @@ -298,16 +270,6 @@ dependencies = [ "num", ] -[[package]] -name = "arrow-buffer" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "bytes", - "half", - "num", -] - [[package]] name = "arrow-buffer" version = "55.2.0" @@ -319,36 +281,17 @@ dependencies = [ "num", ] -[[package]] -name = "arrow-cast" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", - "arrow-select 54.2.1", - "atoi", - "base64 0.22.1", - "chrono", - "half", - "lexical-core", - "num", - "ryu", -] - [[package]] name = "arrow-cast" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e4f12eccc3e1c05a766cafb31f6a60a46c2f8efec9b74c6e0648766d30686af8" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", - "arrow-select 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", "atoi", "base64 0.22.1", "chrono", @@ -365,61 +308,38 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" dependencies = [ - "arrow-array 55.2.0", - "arrow-cast 55.2.0", - "arrow-schema 55.2.0", + "arrow-array", + "arrow-cast", + "arrow-schema", "chrono", "csv", "csv-core", "regex", ] -[[package]] -name = "arrow-data" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "arrow-buffer 54.2.1", - "arrow-schema 54.2.1", - "half", - "num", -] - [[package]] name = "arrow-data" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8de1ce212d803199684b658fc4ba55fb2d7e87b213de5af415308d2fee3619c2" dependencies = [ - "arrow-buffer 55.2.0", - "arrow-schema 55.2.0", + "arrow-buffer", + "arrow-schema", "half", "num", ] -[[package]] -name = "arrow-ipc" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", - "flatbuffers 24.12.23", -] - [[package]] name = "arrow-ipc" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9ea5967e8b2af39aff5d9de2197df16e305f47f404781d3230b2dc672da5d92" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", - "flatbuffers 25.2.10", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "flatbuffers", "lz4_flex", ] @@ -429,11 +349,11 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-cast 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", "chrono", "half", "indexmap 2.11.0", @@ -445,29 +365,17 @@ dependencies = [ "simdutf8", ] -[[package]] -name = "arrow-ord" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", - "arrow-select 54.2.1", -] - [[package]] name = "arrow-ord" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6506e3a059e3be23023f587f79c82ef0bcf6d293587e3272d20f2d30b969b5a7" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", - "arrow-select 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", ] [[package]] @@ -476,18 +384,13 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", "half", ] -[[package]] -name = "arrow-schema" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" - [[package]] name = "arrow-schema" version = "55.2.0" @@ -498,19 +401,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "arrow-select" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "ahash 0.8.12", - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", - "num", -] - [[package]] name = "arrow-select" version = "55.2.0" @@ -518,40 +408,24 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" dependencies = [ "ahash 0.8.12", - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", "num", ] -[[package]] -name = "arrow-string" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-data 54.2.1", - "arrow-schema 54.2.1", - "arrow-select 54.2.1", - "memchr", - "num", - "regex", - "regex-syntax", -] - [[package]] name = "arrow-string" version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0377d532850babb4d927a06294314b316e23311503ed580ec6ce6a0158f49d40" dependencies = [ - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-data 55.2.0", - "arrow-schema 55.2.0", - "arrow-select 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", "memchr", "num", "regex", @@ -1231,7 +1105,7 @@ version = "0.69.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" dependencies = [ - "bitflags 2.9.4", + "bitflags", "cexpr", "clang-sys", "itertools 0.12.1", @@ -1248,12 +1122,6 @@ dependencies = [ "which", ] -[[package]] -name = "bitflags" -version = "1.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" - [[package]] name = "bitflags" version = "2.9.4" @@ -1376,17 +1244,6 @@ dependencies = [ "syn 2.0.106", ] -[[package]] -name = "brotli" -version = "7.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", - "brotli-decompressor 4.0.3", -] - [[package]] name = "brotli" version = "8.0.2" @@ -1395,17 +1252,7 @@ checksum = "4bd8b9603c7aa97359dbd97ecf258968c95f3adddd6db2f7e7a5bef101c84560" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", - "brotli-decompressor 5.0.0", -] - -[[package]] -name = "brotli-decompressor" -version = "4.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a334ef7c9e23abf0ce748e8cd309037da93e606ad52eb372e4ce327a0dcfbdfd" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", + "brotli-decompressor", ] [[package]] @@ -1695,7 +1542,7 @@ version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fde0e0ec90c9dfb3b4b1a0891a7dcd0e2bffde2f7efed5fe7c9bb00e5bfb915e" dependencies = [ - "windows-sys 0.48.0", + "windows-sys 0.59.0", ] [[package]] @@ -2026,8 +1873,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a11e19a7ccc5bb979c95c1dceef663eab39c9061b3bbf8d1937faf0f03bf41f" dependencies = [ "arrow", - "arrow-ipc 55.2.0", - "arrow-schema 55.2.0", + "arrow-ipc", + "arrow-schema", "async-trait", "bytes", "bzip2 0.5.2", @@ -2062,7 +1909,7 @@ dependencies = [ "log", "object_store", "parking_lot", - "parquet 55.2.0", + "parquet", "rand 0.9.2", "regex", "sqlparser", @@ -2141,7 +1988,7 @@ dependencies = [ "mimalloc", "object_store", "parking_lot", - "parquet 55.2.0", + "parquet", "regex", "rustyline", "tokio", @@ -2157,7 +2004,7 @@ dependencies = [ "ahash 0.8.12", "apache-avro 0.17.0", "arrow", - "arrow-ipc 55.2.0", + "arrow-ipc", "base64 0.22.1", "half", "hashbrown 0.14.5", @@ -2165,7 +2012,7 @@ dependencies = [ "libc", "log", "object_store", - "parquet 55.2.0", + "parquet", "paste", "recursive", "sqlparser", @@ -2210,7 +2057,7 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", - "parquet 55.2.0", + "parquet", "rand 0.9.2", "tempfile", "tokio", @@ -2321,7 +2168,7 @@ dependencies = [ "log", "object_store", "parking_lot", - "parquet 55.2.0", + "parquet", "rand 0.9.2", "tokio", ] @@ -2392,7 +2239,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cdf9a9cf655265861a20453b1e58357147eab59bdc90ce7f2f68f1f35104d3bb" dependencies = [ "arrow", - "arrow-buffer 55.2.0", + "arrow-buffer", "base64 0.22.1", "blake2", "blake3", @@ -2455,7 +2302,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2ab331806e34f5545e5f03396e4d5068077395b1665795d8f88c14ec4f1e0b7a" dependencies = [ "arrow", - "arrow-ord 55.2.0", + "arrow-ord", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -2606,8 +2453,8 @@ checksum = "b46cbdf21a01206be76d467f325273b22c559c744a012ead5018dfe79597de08" dependencies = [ "ahash 0.8.12", "arrow", - "arrow-ord 55.2.0", - "arrow-schema 55.2.0", + "arrow-ord", + "arrow-schema", "async-trait", "chrono", "datafusion-common", @@ -3062,7 +2909,7 @@ checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", "rustix 1.0.8", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -3077,23 +2924,13 @@ version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" -[[package]] -name = "flatbuffers" -version = "24.12.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f1baf0dbf96932ec9a3038d57900329c015b0bfb7b63d904f3bc27e2b02a096" -dependencies = [ - "bitflags 1.3.2", - "rustc_version", -] - [[package]] name = "flatbuffers" version = "25.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" dependencies = [ - "bitflags 2.9.4", + "bitflags", "rustc_version", ] @@ -3356,7 +3193,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0bf760ebf69878d9fd8f110c89703d90ce35095324d1f1edcb595c63945ee757" dependencies = [ - "bitflags 2.9.4", + "bitflags", "ignore", "walkdir", ] @@ -3700,7 +3537,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.5.10", + "socket2 0.6.0", "tokio", "tower-service", "tracing", @@ -3737,14 +3574,14 @@ dependencies = [ "anyhow", "apache-avro 0.20.0", "array-init", - "arrow-arith 54.2.1", - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-cast 54.2.1", - "arrow-ord 54.2.1", - "arrow-schema 54.2.1", - "arrow-select 54.2.1", - "arrow-string 54.2.1", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-ord", + "arrow-schema", + "arrow-select", + "arrow-string", "as-any", "async-trait", "backon", @@ -3766,7 +3603,7 @@ dependencies = [ "once_cell", "opendal", "ordered-float 4.6.0", - "parquet 54.2.1", + "parquet", "pretty_assertions", "rand 0.8.5", "regex", @@ -3917,7 +3754,7 @@ dependencies = [ "expect-test", "futures", "iceberg", - "parquet 54.2.1", + "parquet", "tempfile", "tokio", "uuid", @@ -3937,8 +3774,8 @@ dependencies = [ name = "iceberg-integration-tests" version = "0.7.0" dependencies = [ - "arrow-array 54.2.1", - "arrow-schema 54.2.1", + "arrow-array", + "arrow-schema", "ctor", "datafusion", "futures", @@ -3947,7 +3784,7 @@ dependencies = [ "iceberg-datafusion", "iceberg_test_utils", "ordered-float 2.10.1", - "parquet 54.2.1", + "parquet", "tokio", "uuid", ] @@ -4443,7 +4280,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "391290121bad3d37fbddad76d8f5d1c1c314cfc646d143d7e07a3086ddff0ce3" dependencies = [ - "bitflags 2.9.4", + "bitflags", "libc", "redox_syscall", ] @@ -4556,7 +4393,7 @@ version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" dependencies = [ - "twox-hash 2.1.2", + "twox-hash", ] [[package]] @@ -4800,7 +4637,7 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ - "bitflags 2.9.4", + "bitflags", "cfg-if", "cfg_aliases", "libc", @@ -4813,7 +4650,7 @@ version = "0.30.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "74523f3a35e05aba87a1d978330aef40f67b0304ac79c1c00b294c9830543db6" dependencies = [ - "bitflags 2.9.4", + "bitflags", "cfg-if", "cfg_aliases", "libc", @@ -5134,40 +4971,6 @@ dependencies = [ "windows-targets 0.52.6", ] -[[package]] -name = "parquet" -version = "54.2.1" -source = "git+https://github.com/jkylling/arrow-rs?branch=feature%2Fparquet-reader-row-numbers#1a1e6b6a3ff25cc5adec5b9aa2f6615159609663" -dependencies = [ - "ahash 0.8.12", - "arrow-array 54.2.1", - "arrow-buffer 54.2.1", - "arrow-cast 54.2.1", - "arrow-data 54.2.1", - "arrow-ipc 54.2.1", - "arrow-schema 54.2.1", - "arrow-select 54.2.1", - "base64 0.22.1", - "brotli 7.0.0", - "bytes", - "chrono", - "flate2", - "futures", - "half", - "hashbrown 0.15.5", - "lz4_flex", - "num", - "num-bigint", - "paste", - "seq-macro", - "simdutf8", - "snap", - "thrift", - "tokio", - "twox-hash 1.6.3", - "zstd", -] - [[package]] name = "parquet" version = "55.2.0" @@ -5175,15 +4978,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" dependencies = [ "ahash 0.8.12", - "arrow-array 55.2.0", - "arrow-buffer 55.2.0", - "arrow-cast 55.2.0", - "arrow-data 55.2.0", - "arrow-ipc 55.2.0", - "arrow-schema 55.2.0", - "arrow-select 55.2.0", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", "base64 0.22.1", - "brotli 8.0.2", + "brotli", "bytes", "chrono", "flate2", @@ -5200,7 +5003,7 @@ dependencies = [ "snap", "thrift", "tokio", - "twox-hash 2.1.2", + "twox-hash", "zstd", ] @@ -5825,7 +5628,7 @@ dependencies = [ "quinn-udp", "rustc-hash 2.1.1", "rustls 0.23.31", - "socket2 0.5.10", + "socket2 0.6.0", "thiserror 2.0.16", "tokio", "tracing", @@ -5862,7 +5665,7 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.10", + "socket2 0.6.0", "tracing", "windows-sys 0.60.2", ] @@ -5994,7 +5797,7 @@ version = "0.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" dependencies = [ - "bitflags 2.9.4", + "bitflags", ] [[package]] @@ -6329,11 +6132,11 @@ version = "0.38.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" dependencies = [ - "bitflags 2.9.4", + "bitflags", "errno", "libc", "linux-raw-sys 0.4.15", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -6342,7 +6145,7 @@ version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" dependencies = [ - "bitflags 2.9.4", + "bitflags", "errno", "libc", "linux-raw-sys 0.9.4", @@ -6462,7 +6265,7 @@ version = "16.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "62fd9ca5ebc709e8535e8ef7c658eb51457987e48c98ead2be482172accc408d" dependencies = [ - "bitflags 2.9.4", + "bitflags", "cfg-if", "clipboard-win", "fd-lock", @@ -6604,7 +6407,7 @@ version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags 2.9.4", + "bitflags", "core-foundation 0.9.4", "core-foundation-sys", "libc", @@ -6617,7 +6420,7 @@ version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "60b369d18893388b345804dc0007963c99b7d665ae71d275812d828c6f089640" dependencies = [ - "bitflags 2.9.4", + "bitflags", "core-foundation 0.10.1", "core-foundation-sys", "libc", @@ -7141,7 +6944,7 @@ checksum = "aa003f0038df784eb8fecbbac13affe3da23b45194bd57dba231c8f48199c526" dependencies = [ "atoi", "base64 0.22.1", - "bitflags 2.9.4", + "bitflags", "byteorder", "bytes", "crc", @@ -7182,7 +6985,7 @@ checksum = "db58fcd5a53cf07c184b154801ff91347e4c30d17a3562a635ff028ad5deda46" dependencies = [ "atoi", "base64 0.22.1", - "bitflags 2.9.4", + "bitflags", "byteorder", "crc", "dotenvy", @@ -7251,7 +7054,7 @@ dependencies = [ "cfg-if", "libc", "psm", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -7722,7 +7525,7 @@ version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" dependencies = [ - "bitflags 2.9.4", + "bitflags", "bytes", "futures-util", "http 1.3.1", @@ -7814,16 +7617,6 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" -[[package]] -name = "twox-hash" -version = "1.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" -dependencies = [ - "cfg-if", - "static_assertions", -] - [[package]] name = "twox-hash" version = "2.1.2" diff --git a/Cargo.toml b/Cargo.toml index 7d008231b5..e459b05549 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,14 +43,14 @@ rust-version = "1.87" anyhow = "1.0.72" apache-avro = { version = "0.20", features = ["zstandard"] } array-init = "2" -arrow-arith = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-array = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-buffer = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-cast = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-ord = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-schema = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-select = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } -arrow-string = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +arrow-arith = { version = "55.1" } +arrow-array = { version = "55.1" } +arrow-buffer = { version = "55.1" } +arrow-cast = { version = "55.1" } +arrow-ord = { version = "55.1" } +arrow-schema = { version = "55.1" } +arrow-select = { version = "55.1" } +arrow-string = { version = "55.1" } as-any = "0.3.2" async-trait = "0.1.88" aws-config = "1.8.1" @@ -96,7 +96,7 @@ num-bigint = "0.4.6" once_cell = "1.20" opendal = "0.54.0" ordered-float = "4" -parquet = { git = "https://github.com/jkylling/arrow-rs", branch = "feature/parquet-reader-row-numbers" } +parquet = "55.1" pilota = "0.11.10" port_scanner = "0.1.5" pretty_assertions = "1.4" diff --git a/crates/iceberg/src/arrow/delete_file_loader.rs b/crates/iceberg/src/arrow/delete_file_loader.rs index 01c11e8363..592ef2eb4a 100644 --- a/crates/iceberg/src/arrow/delete_file_loader.rs +++ b/crates/iceberg/src/arrow/delete_file_loader.rs @@ -63,7 +63,6 @@ impl BasicDeleteFileLoader { data_file_path, self.file_io.clone(), false, - false, // don't include row ordinals for delete files ) .await? .build()? diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 79df49e373..f06263d95b 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -1,20 +1,18 @@ use std::pin::Pin; use std::sync::Arc; -use apache_avro::types::Record; use arrow_array::{RecordBatch, UInt64Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use futures::{Stream, StreamExt, TryStreamExt}; use roaring::RoaringTreemap; use crate::arrow::record_batch_transformer::RecordBatchTransformer; -use crate::arrow::{ArrowBatchEmitter, ArrowReader, RESERVED_COL_NAME_POS, RESERVED_FIELD_ID_POS}; +use crate::arrow::{ArrowBatchEmitter, ArrowReader}; use crate::io::FileIO; use crate::scan::ArrowRecordBatchStream; use crate::scan::incremental::{ AppendedFileScanTask, IncrementalFileScanTask, IncrementalFileScanTaskStream, }; -use crate::spec::Schema; use crate::{Error, ErrorKind, Result}; /// The type of incremental batch: appended data or deleted records. @@ -39,20 +37,12 @@ impl ArrowBatchEmitter let file_io = reader.file_io.clone(); let batch_size = reader.batch_size; let concurrency_limit_data_files = reader.concurrency_limit_data_files; - let include_row_ordinals = reader.include_row_ordinals; - let include_file_path = reader.include_file_path; let stream = self .map_ok(move |task| { let file_io = file_io.clone(); - process_incremental_file_scan_task( - task, - batch_size, - file_io, - include_row_ordinals, - include_file_path, - ) + process_incremental_file_scan_task(task, batch_size, file_io) }) .map_err(|err| { Error::new(ErrorKind::Unexpected, "file scan task generate failed").with_source(err) @@ -68,26 +58,18 @@ async fn process_incremental_file_scan_task( task: IncrementalFileScanTask, batch_size: Option, file_io: FileIO, - include_row_ordinals: bool, - include_file_path: bool, ) -> Result { match task { IncrementalFileScanTask::Append(append_task) => { - process_incremental_append_task( - append_task, - batch_size, - file_io, - include_row_ordinals, - include_file_path, - ) - .await - .map(|stream| { - // Map the stream to include the batch type - let typed_stream = stream.map(|batch_result| { - batch_result.map(|batch| (IncrementalBatchType::Append, batch)) - }); - Box::pin(typed_stream) as IncrementalArrowBatchRecordStream - }) + process_incremental_append_task(append_task, batch_size, file_io) + .await + .map(|stream| { + // Map the stream to include the batch type + let typed_stream = stream.map(|batch_result| { + batch_result.map(|batch| (IncrementalBatchType::Append, batch)) + }); + Box::pin(typed_stream) as IncrementalArrowBatchRecordStream + }) } IncrementalFileScanTask::Delete(file_path, delete_vector) => { // Clone the `RoaringTreemap` underlying the delete vector to take ownership. @@ -110,49 +92,21 @@ async fn process_incremental_append_task( task: AppendedFileScanTask, batch_size: Option, file_io: FileIO, - include_row_ordinals: bool, - include_file_path: bool, ) -> Result { let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( &task.data_file_path, file_io.clone(), true, - include_row_ordinals, ) .await?; - // Add reserved field IDs for metadata columns when enabled - let mut extended_project_field_ids = task.project_field_ids.clone(); - let extended_schema = if include_row_ordinals { - // Per Iceberg spec, _pos column has reserved field ID RESERVED_FIELD_ID_POS - extended_project_field_ids.push(RESERVED_FIELD_ID_POS); - - // Extend the schema to include the _pos field so RecordBatchTransformer can find it - let mut fields = task.schema.as_struct().fields().to_vec(); - use crate::spec::{NestedField, PrimitiveType, Type}; - fields.push(Arc::new(NestedField::required( - RESERVED_FIELD_ID_POS, - RESERVED_COL_NAME_POS, - Type::Primitive(PrimitiveType::Long), - ))); - Arc::new( - Schema::builder() - .with_schema_id(task.schema.schema_id()) - .with_fields(fields) - .build()?, - ) - } else { - task.schema_ref() - }; - // Create a projection mask for the batch stream to select which columns in the // Parquet file that we want in the response let projection_mask = ArrowReader::get_arrow_projection_mask( - &extended_project_field_ids, - &extended_schema, + &task.project_field_ids, + &task.schema_ref(), record_batch_stream_builder.parquet_schema(), record_batch_stream_builder.schema(), - include_row_ordinals, )?; record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); @@ -160,7 +114,7 @@ async fn process_incremental_append_task( // that come back from the file, such as type promotion, default column insertion // and column re-ordering let mut record_batch_transformer = - RecordBatchTransformer::build(extended_schema, &extended_project_field_ids); + RecordBatchTransformer::build(task.schema_ref(), &task.project_field_ids); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -188,14 +142,7 @@ async fn process_incremental_append_task( let record_batch_stream = record_batch_stream_builder .build()? .map(move |batch| match batch { - Ok(batch) => { - let batch = record_batch_transformer.process_record_batch(batch)?; - if include_file_path { - ArrowReader::add_file_path_column(batch, &file_path) - } else { - Ok(batch) - } - } + Ok(batch) => record_batch_transformer.process_record_batch(batch), Err(err) => Err(err.into()), }); diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index b06f9ece72..1846d2f82d 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -58,13 +58,9 @@ use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; -/// Reserved field ID for the row ordinal (_pos) column per Iceberg spec -pub(crate) const RESERVED_FIELD_ID_POS: i32 = 2147483645; /// Reserved field ID for the file path (_file) column per Iceberg spec pub(crate) const RESERVED_FIELD_ID_FILE: i32 = 2147483646; -/// Column name for the row ordinal metadata column per Iceberg spec -pub(crate) const RESERVED_COL_NAME_POS: &str = "_pos"; /// Column name for the file path metadata column per Iceberg spec pub(crate) const RESERVED_COL_NAME_FILE: &str = "_file"; @@ -75,8 +71,6 @@ pub struct ArrowReaderBuilder { concurrency_limit_data_files: usize, row_group_filtering_enabled: bool, row_selection_enabled: bool, - include_row_ordinals: bool, - include_file_path: bool, // TODO: Perhaps we should have a generic API that allows specifying extra field IDs, or metadata column names... } impl ArrowReaderBuilder { @@ -90,8 +84,6 @@ impl ArrowReaderBuilder { concurrency_limit_data_files: num_cpus, row_group_filtering_enabled: true, row_selection_enabled: false, - include_row_ordinals: false, - include_file_path: false, } } @@ -120,20 +112,6 @@ impl ArrowReaderBuilder { self } - /// Determines whether to include row ordinals in the output. - /// When enabled, adds a `_pos` column containing the row position within each parquet file. - pub fn with_row_ordinals(mut self, include_row_ordinals: bool) -> Self { - self.include_row_ordinals = include_row_ordinals; - self - } - - /// Determines whether to include file path in the output. - /// When enabled, adds a `_file` column containing the file path for each row. - pub fn with_file_path(mut self, include_file_path: bool) -> Self { - self.include_file_path = include_file_path; - self - } - /// Build the ArrowReader. pub fn build(self) -> ArrowReader { ArrowReader { @@ -146,8 +124,6 @@ impl ArrowReaderBuilder { concurrency_limit_data_files: self.concurrency_limit_data_files, row_group_filtering_enabled: self.row_group_filtering_enabled, row_selection_enabled: self.row_selection_enabled, - include_row_ordinals: self.include_row_ordinals, - include_file_path: self.include_file_path, } } } @@ -164,8 +140,6 @@ pub struct ArrowReader { pub(crate) row_group_filtering_enabled: bool, pub(crate) row_selection_enabled: bool, - pub(crate) include_row_ordinals: bool, - pub(crate) include_file_path: bool, } /// Trait indicating that Arrow batches can be read from an implementing type. @@ -183,8 +157,6 @@ impl ArrowReader { let concurrency_limit_data_files = self.concurrency_limit_data_files; let row_group_filtering_enabled = self.row_group_filtering_enabled; let row_selection_enabled = self.row_selection_enabled; - let include_row_ordinals = self.include_row_ordinals; - let include_file_path = self.include_file_path; let stream = tasks .map_ok(move |task| { @@ -197,8 +169,6 @@ impl ArrowReader { self.delete_file_loader.clone(), row_group_filtering_enabled, row_selection_enabled, - include_row_ordinals, - include_file_path, ) }) .map_err(|err| { @@ -218,8 +188,6 @@ impl ArrowReader { delete_file_loader: CachingDeleteFileLoader, row_group_filtering_enabled: bool, row_selection_enabled: bool, - include_row_ordinals: bool, - include_file_path: bool, ) -> Result { let should_load_page_index = (row_selection_enabled && task.predicate.is_some()) || !task.deletes.is_empty(); @@ -230,36 +198,16 @@ impl ArrowReader { &task.data_file_path, file_io.clone(), should_load_page_index, - include_row_ordinals, ) .await?; - // Add reserved field IDs for metadata columns when enabled - let mut extended_project_field_ids = task.project_field_ids.clone(); - let extended_schema = if include_row_ordinals { - // Per Iceberg spec, _pos column has reserved field ID RESERVED_FIELD_ID_POS - extended_project_field_ids.push(RESERVED_FIELD_ID_POS); - - // Extend the schema to include the _pos field so RecordBatchTransformer can find it - let mut fields = task.schema.as_struct().fields().to_vec(); - use crate::spec::{NestedField, PrimitiveType, Type}; - fields.push(Arc::new(NestedField::required(RESERVED_FIELD_ID_POS, RESERVED_COL_NAME_POS, Type::Primitive(PrimitiveType::Long)))); - Arc::new(Schema::builder() - .with_schema_id(task.schema.schema_id()) - .with_fields(fields) - .build()?) - } else { - task.schema_ref() - }; - // Create a projection mask for the batch stream to select which columns in the // Parquet file that we want in the response let projection_mask = Self::get_arrow_projection_mask( - &extended_project_field_ids, - &extended_schema, + &task.project_field_ids, + &task.schema, record_batch_stream_builder.parquet_schema(), record_batch_stream_builder.schema(), - include_row_ordinals, )?; record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); @@ -267,7 +215,7 @@ impl ArrowReader { // that come back from the file, such as type promotion, default column insertion // and column re-ordering let mut record_batch_transformer = - RecordBatchTransformer::build(extended_schema, &extended_project_field_ids); + RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -377,19 +325,11 @@ impl ArrowReader { // Build the batch stream and send all the RecordBatches that it generates // to the requester. - let file_path = task.data_file_path.clone(); let record_batch_stream = record_batch_stream_builder .build()? .map(move |batch| match batch { - Ok(batch) => { - let batch = record_batch_transformer.process_record_batch(batch)?; - if include_file_path { - Self::add_file_path_column(batch, &file_path) - } else { - Ok(batch) - } - } + Ok(batch) => record_batch_transformer.process_record_batch(batch), Err(err) => Err(err.into()), }); @@ -400,7 +340,6 @@ impl ArrowReader { data_file_path: &str, file_io: FileIO, should_load_page_index: bool, - include_row_ordinals: bool, ) -> Result>> { // Get the metadata for the Parquet file we need to read and build // a reader for the data within @@ -413,18 +352,12 @@ impl ArrowReader { .with_preload_page_index(should_load_page_index); // Create the record batch stream builder, which wraps the parquet file reader - let mut record_batch_stream_builder = ParquetRecordBatchStreamBuilder::new_with_options( + let record_batch_stream_builder = ParquetRecordBatchStreamBuilder::new_with_options( parquet_file_reader, ArrowReaderOptions::new(), ) .await?; - // Add row number column if requested (for changelog scans) - // Per Iceberg spec, the position column should be named RESERVED_COL_NAME_POS with field ID RESERVED_FIELD_ID_POS - if include_row_ordinals { - record_batch_stream_builder = record_batch_stream_builder.with_row_number_column(RESERVED_COL_NAME_POS); - } - Ok(record_batch_stream_builder) } @@ -542,9 +475,10 @@ impl ArrowReader { /// Uses Run-End Encoding (RLE) for maximum memory efficiency when the same /// file path is repeated across all rows. pub(crate) fn add_file_path_column(batch: RecordBatch, file_path: &str) -> Result { + use std::collections::HashMap; + use arrow_array::{Int32Array, RunArray, StringArray}; use arrow_schema::Field; - use std::collections::HashMap; let num_rows = batch.num_rows(); @@ -575,7 +509,7 @@ impl ArrowReader { let file_field = Field::new( RESERVED_COL_NAME_FILE, DataType::RunEndEncoded(run_ends_field, values_field), - false + false, ) .with_metadata(HashMap::from([( PARQUET_FIELD_ID_META_KEY.to_string(), @@ -636,30 +570,7 @@ impl ArrowReader { iceberg_schema_of_task: &Schema, parquet_schema: &SchemaDescriptor, arrow_schema: &ArrowSchemaRef, - include_row_ordinals: bool, ) -> Result { - // WORKAROUND: The parquet reader's `with_row_number_column()` method adds the _pos column - // dynamically when building record batches, but does NOT update the schema returned by - // ParquetRecordBatchStreamBuilder::schema(). This means the arrow_schema parameter here - // won't include the _pos field even though it will be present in the actual record batches. - // To work around this, we manually extend the arrow_schema to include the _pos field - // when include_row_ordinals is true. - let arrow_schema = if include_row_ordinals { - let mut fields: Vec<_> = arrow_schema.fields().iter().cloned().collect(); - let pos_field = arrow_schema::Field::new(RESERVED_COL_NAME_POS, DataType::Int64, false) - .with_metadata(std::collections::HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_POS.to_string(), - )])); - fields.push(Arc::new(pos_field)); - Arc::new(ArrowSchema::new_with_metadata( - fields, - arrow_schema.metadata().clone(), - )) - } else { - arrow_schema.clone() - }; - fn type_promotion_is_valid( file_type: Option<&PrimitiveType>, projected_type: Option<&PrimitiveType>, @@ -699,7 +610,6 @@ impl ArrowReader { let mut column_map = HashMap::new(); let fields = arrow_schema.fields(); - // Pre-project only the fields that have been selected, possibly avoiding converting // some Arrow types that are not yet supported. let mut projected_fields: HashMap = HashMap::new(); @@ -749,12 +659,6 @@ impl ArrowReader { // but old Parquet files don't have them yet. let mut indices = vec![]; for field_id in leaf_field_ids { - // Skip _pos field - it's added dynamically by the parquet reader via - // with_row_number_column(), not read from the parquet file itself - if field_id == RESERVED_FIELD_ID_POS { - continue; - } - if let Some(col_idx) = column_map.get(&field_id) { indices.push(*col_idx); } @@ -1529,7 +1433,7 @@ impl ArrowFileReader { } impl AsyncFileReader for ArrowFileReader { - fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { Box::pin( self.r .read(range.start as u64..range.end as u64) @@ -1539,6 +1443,7 @@ impl AsyncFileReader for ArrowFileReader { fn get_metadata( &mut self, + _options: Option<&'_ ArrowReaderOptions>, ) -> BoxFuture<'_, parquet::errors::Result>> { async move { let reader = ParquetMetaDataReader::new() @@ -1546,7 +1451,7 @@ impl AsyncFileReader for ArrowFileReader { .with_column_indexes(self.preload_column_index) .with_page_indexes(self.preload_page_index) .with_offset_indexes(self.preload_offset_index); - let size = self.meta.size as usize; + let size = self.meta.size; let meta = reader.load_and_finish(self, size).await?; Ok(Arc::new(meta)) @@ -1733,7 +1638,6 @@ message schema { &schema, &parquet_schema, &arrow_schema, - false, ) .unwrap_err(); @@ -1749,7 +1653,6 @@ message schema { &schema, &parquet_schema, &arrow_schema, - false, ) .unwrap_err(); @@ -1761,7 +1664,7 @@ message schema { // Finally avoid selecting fields with unsupported data types let mask = - ArrowReader::get_arrow_projection_mask(&[1], &schema, &parquet_schema, &arrow_schema, false) + ArrowReader::get_arrow_projection_mask(&[1], &schema, &parquet_schema, &arrow_schema) .expect("Some ProjectionMask"); assert_eq!(mask, ProjectionMask::leaves(&parquet_schema, vec![0])); } diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 1d4474f40f..5d3b73ebe8 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -8,7 +8,7 @@ use crate::arrow::delete_filter::DeleteFilter; use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, IncrementalArrowBatchRecordStream}; use crate::delete_file_index::DeleteFileIndex; use crate::io::FileIO; -use crate::scan::{ArrowRecordBatchStream, DeleteFileContext}; +use crate::scan::DeleteFileContext; use crate::scan::cache::{ExpressionEvaluatorCache, ManifestEvaluatorCache, PartitionFilterCache}; use crate::scan::context::ManifestEntryContext; use crate::spec::{DataContentType, ManifestStatus, Snapshot, SnapshotRef}; @@ -453,11 +453,10 @@ impl IncrementalTableScan { /// Returns an [`IncrementalArrowBatchRecordStream`] for this incremental table scan. pub async fn to_arrow(&self) -> Result { let file_scan_task_stream = self.plan_files().await?; - let mut arrow_reader_builder = - ArrowReaderBuilder::new(self.file_io.clone()) - .with_data_file_concurrency_limit(self.concurrency_limit_data_files) - .with_row_group_filtering_enabled(true) - .with_row_selection_enabled(true); + let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); if let Some(batch_size) = self.batch_size { arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index aff7d26a30..620f27df33 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -330,7 +330,7 @@ impl ParquetWriter { let reader = input_file.reader().await?; let mut parquet_reader = ArrowFileReader::new(file_metadata, reader); - let parquet_metadata = parquet_reader.get_metadata().await.map_err(|err| { + let parquet_metadata = parquet_reader.get_metadata(None).await.map_err(|err| { Error::new( ErrorKind::DataInvalid, format!("Error reading Parquet metadata: {}", err), From a378abe9c73fc82917bc661dac3b62fc27c3d90a Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 12:01:03 +0200 Subject: [PATCH 04/68] cargo fmt --- crates/iceberg/src/arrow/delete_filter.rs | 6 +-- crates/iceberg/src/arrow/mod.rs | 1 - crates/playground/src/main.rs | 58 +++++++++++++++++------ 3 files changed, 46 insertions(+), 19 deletions(-) diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index b499887ac8..09781d8751 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -71,10 +71,8 @@ impl DeleteFilter { .and_then(|st| st.delete_vectors.get(delete_file_path).cloned()) } - pub(crate) fn with_read(&self, f: F) -> Result - where - F: FnOnce(&DeleteFileFilterState) -> Result, - { + pub(crate) fn with_read(&self, f: F) -> Result + where F: FnOnce(&DeleteFileFilterState) -> Result { let state = self.state.read().map_err(|e| { Error::new( ErrorKind::Unexpected, diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index a20514cf2d..f0a67fce87 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -35,7 +35,6 @@ mod value; mod incremental; pub use incremental::*; - pub use reader::*; pub use value::*; pub(crate) mod record_batch_partition_splitter; diff --git a/crates/playground/src/main.rs b/crates/playground/src/main.rs index f7dc65ae34..fcf5b67128 100644 --- a/crates/playground/src/main.rs +++ b/crates/playground/src/main.rs @@ -1,11 +1,14 @@ -use futures_util::TryStreamExt; -use tokio; use std::collections::HashMap; use std::env; + use dotenv::dotenv; -use iceberg::{NamespaceIdent, CatalogBuilder, Catalog}; +use futures_util::TryStreamExt; use iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, S3_SECRET_ACCESS_KEY}; -use iceberg_catalog_rest::{REST_CATALOG_PROP_URI, REST_CATALOG_PROP_WAREHOUSE, RestCatalogBuilder}; +use iceberg::{Catalog, CatalogBuilder, NamespaceIdent}; +use iceberg_catalog_rest::{ + REST_CATALOG_PROP_URI, REST_CATALOG_PROP_WAREHOUSE, RestCatalogBuilder, +}; +use tokio; #[tokio::main(flavor = "multi_thread")] async fn main() -> Result<(), Box> { @@ -13,7 +16,10 @@ async fn main() -> Result<(), Box> { let args: Vec = env::args().collect(); if args.len() < 5 { - eprintln!("Usage: {} ", args[0]); + eprintln!( + "Usage: {} ", + args[0] + ); return Err("Not enough arguments".into()); } let namespace = &args[1]; @@ -28,13 +34,31 @@ async fn main() -> Result<(), Box> { .load( "rest", HashMap::from([ - (REST_CATALOG_PROP_URI.to_string(), "http://localhost:8181".to_string()), - (REST_CATALOG_PROP_WAREHOUSE.to_string(), format!("s3://{}", bucket)), - (S3_ACCESS_KEY_ID.to_string(), env::var("AWS_ACCESS_KEY_ID").unwrap_or("admin".to_string())), - (S3_SECRET_ACCESS_KEY.to_string(), env::var("AWS_SECRET_ACCESS_KEY").unwrap_or("password".to_string())), - (S3_REGION.to_string(), env::var("AWS_REGION").unwrap_or("us-east-1".to_string())), - (S3_ENDPOINT.to_string(), env::var("AWS_ENDPOINT_URL").unwrap_or("http://localhost:9000".to_string())), - ]) + ( + REST_CATALOG_PROP_URI.to_string(), + "http://localhost:8181".to_string(), + ), + ( + REST_CATALOG_PROP_WAREHOUSE.to_string(), + format!("s3://{}", bucket), + ), + ( + S3_ACCESS_KEY_ID.to_string(), + env::var("AWS_ACCESS_KEY_ID").unwrap_or("admin".to_string()), + ), + ( + S3_SECRET_ACCESS_KEY.to_string(), + env::var("AWS_SECRET_ACCESS_KEY").unwrap_or("password".to_string()), + ), + ( + S3_REGION.to_string(), + env::var("AWS_REGION").unwrap_or("us-east-1".to_string()), + ), + ( + S3_ENDPOINT.to_string(), + env::var("AWS_ENDPOINT_URL").unwrap_or("http://localhost:9000".to_string()), + ), + ]), ) .await?; @@ -44,11 +68,17 @@ async fn main() -> Result<(), Box> { return Err("Namespace does not exist".into()); } - let table = catalog.load_table(&iceberg::TableIdent::new(namespace.clone(), table_name.clone())).await?; + let table = catalog + .load_table(&iceberg::TableIdent::new( + namespace.clone(), + table_name.clone(), + )) + .await?; let mut stream = table .incremental_scan(from_snapshot_id, to_snapshot_id) .build()? - .to_arrow().await?; + .to_arrow() + .await?; let mut rows = 0; while let Some((batch_type, batch)) = stream.try_next().await? { From 317b5c68fd6a40b44d43538170102e67f2161f72 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:16:52 +0200 Subject: [PATCH 05/68] Implement unzipped stream --- crates/iceberg/src/arrow/incremental.rs | 168 +++++++++++++----- crates/iceberg/src/arrow/reader.rs | 5 +- .../iceberg/src/scan/incremental/context.rs | 8 +- crates/iceberg/src/scan/incremental/mod.rs | 10 +- 4 files changed, 126 insertions(+), 65 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index f06263d95b..6cbe107113 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -3,12 +3,14 @@ use std::sync::Arc; use arrow_array::{RecordBatch, UInt64Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; -use futures::{Stream, StreamExt, TryStreamExt}; +use futures::stream::select; +use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; use roaring::RoaringTreemap; use crate::arrow::record_batch_transformer::RecordBatchTransformer; use crate::arrow::{ArrowBatchEmitter, ArrowReader}; use crate::io::FileIO; +use crate::runtime::spawn; use crate::scan::ArrowRecordBatchStream; use crate::scan::incremental::{ AppendedFileScanTask, IncrementalFileScanTask, IncrementalFileScanTaskStream, @@ -25,66 +27,134 @@ pub enum IncrementalBatchType { } /// The stream of incremental Arrow `RecordBatch`es with batch type. -pub type IncrementalArrowBatchRecordStream = +pub type CombinedIncrementalBatchRecordStream = Pin> + Send + 'static>>; -impl ArrowBatchEmitter +/// Stream type for obtaining a separate stream of appended and deleted record batches. +pub type UnzippedIncrementalBatchRecordStream = (ArrowRecordBatchStream, ArrowRecordBatchStream); + +impl ArrowBatchEmitter for IncrementalFileScanTaskStream { /// Take a stream of `IncrementalFileScanTasks` and reads all the files. Returns a /// stream of Arrow `RecordBatch`es containing the data from the files. - fn read(self, reader: ArrowReader) -> Result { - let file_io = reader.file_io.clone(); - let batch_size = reader.batch_size; - let concurrency_limit_data_files = reader.concurrency_limit_data_files; - - let stream = self - .map_ok(move |task| { - let file_io = file_io.clone(); + fn read(self, reader: ArrowReader) -> Result { + let (appends, deletes) = ArrowBatchEmitter::< + ArrowReader, + UnzippedIncrementalBatchRecordStream, + >::read(self, reader)?; - process_incremental_file_scan_task(task, batch_size, file_io) - }) - .map_err(|err| { - Error::new(ErrorKind::Unexpected, "file scan task generate failed").with_source(err) - }) - .try_buffer_unordered(concurrency_limit_data_files) - .try_flatten_unordered(concurrency_limit_data_files); + let left = appends.map(|res| res.map(|batch| (IncrementalBatchType::Append, batch))); + let right = deletes.map(|res| res.map(|batch| (IncrementalBatchType::Delete, batch))); - Ok(Box::pin(stream) as IncrementalArrowBatchRecordStream) + Ok(Box::pin(select(left, right)) as CombinedIncrementalBatchRecordStream) } } -async fn process_incremental_file_scan_task( - task: IncrementalFileScanTask, - batch_size: Option, - file_io: FileIO, -) -> Result { - match task { - IncrementalFileScanTask::Append(append_task) => { - process_incremental_append_task(append_task, batch_size, file_io) - .await - .map(|stream| { - // Map the stream to include the batch type - let typed_stream = stream.map(|batch_result| { - batch_result.map(|batch| (IncrementalBatchType::Append, batch)) - }); - Box::pin(typed_stream) as IncrementalArrowBatchRecordStream +impl ArrowBatchEmitter + for IncrementalFileScanTaskStream +{ + /// Take a stream of `IncrementalFileScanTasks` and reads all the files. Returns two + /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. + fn read(self, reader: ArrowReader) -> Result { + let (appends_tx, appends_rx) = + futures::channel::mpsc::channel(reader.concurrency_limit_data_files); + let (deletes_tx, deletes_rx) = + futures::channel::mpsc::channel(reader.concurrency_limit_data_files); + + let batch_size = reader.batch_size; + let concurrency_limit_data_files = reader.concurrency_limit_data_files; + let file_io = reader.file_io.clone(); + + spawn(async move { + let _ = self + .try_for_each_concurrent(concurrency_limit_data_files, |task| { + let file_io = file_io.clone(); + let mut appends_tx = appends_tx.clone(); + let mut deletes_tx = deletes_tx.clone(); + async move { + match task { + IncrementalFileScanTask::Append(append_task) => { + spawn(async move { + let record_batch_stream = process_incremental_append_task( + append_task, + batch_size, + file_io, + ) + .await; + + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = appends_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read appended record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = appends_tx.send(Err(e)).await; + } + } + }); + } + IncrementalFileScanTask::Delete(file_path, delete_vector) => { + spawn(async move { + // Clone the `RoaringTreemap` underlying the delete vector to take ownership. + let bit_map = { + let guard = delete_vector.lock().unwrap(); + guard.inner.clone() + }; + let record_batch_stream = process_incremental_delete_task( + file_path, bit_map, batch_size, + ); + + // Write a match block like above with the following while loop in the Ok arm + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = deletes_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read deleted record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = deletes_tx.send(Err(e)).await; + } + } + }); + } + }; + + Ok(()) + } }) - } - IncrementalFileScanTask::Delete(file_path, delete_vector) => { - // Clone the `RoaringTreemap` underlying the delete vector to take ownership. - let bit_map = { - let guard = delete_vector.lock().unwrap(); - guard.inner.clone() - }; - process_incremental_delete_task(file_path, bit_map, batch_size).map(|stream| { - // Map the stream to include the batch type - let typed_stream = stream.map(|batch_result| { - batch_result.map(|batch| (IncrementalBatchType::Delete, batch)) - }); - Box::pin(typed_stream) as IncrementalArrowBatchRecordStream - }) - } + .await; + }); + + return Ok(( + Box::pin(deletes_rx) as ArrowRecordBatchStream, + Box::pin(appends_rx) as ArrowRecordBatchStream, + )); } } diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 1846d2f82d..760ebaff5e 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -33,7 +33,6 @@ use arrow_string::like::starts_with; use bytes::Bytes; use fnv::FnvHashSet; use futures::future::BoxFuture; -use futures::stream::BoxStream; use futures::{FutureExt, StreamExt, TryFutureExt, TryStreamExt, try_join}; use parquet::arrow::arrow_reader::{ ArrowPredicateFn, ArrowReaderOptions, RowFilter, RowSelection, RowSelector, @@ -143,9 +142,9 @@ pub struct ArrowReader { } /// Trait indicating that Arrow batches can be read from an implementing type. -pub trait ArrowBatchEmitter { +pub trait ArrowBatchEmitter { /// Reads Arrow batches from the implementing type. - fn read(self, reader: R) -> Result>>; + fn read(self, reader: R) -> Result; } impl ArrowReader { diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index 5f4d795222..09614cc86f 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -7,8 +7,8 @@ use crate::Result; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; use crate::delete_file_index::DeleteFileIndex; use crate::io::object_cache::ObjectCache; +use crate::scan::ExpressionEvaluatorCache; use crate::scan::context::{ManifestEntryContext, ManifestEntryFilterFn, ManifestFileContext}; -use crate::scan::{ExpressionEvaluatorCache, ManifestEvaluatorCache, PartitionFilterCache}; use crate::spec::{ ManifestContentType, ManifestEntryRef, ManifestFile, Operation, SchemaRef, SnapshotRef, TableMetadataRef, @@ -34,12 +34,6 @@ pub(crate) struct IncrementalPlanContext { /// The field IDs to scan. pub field_ids: Arc>, - /// The partition filter cache to use for the scan. - pub partition_filter_cache: Arc, - - /// The manifest evaluator cache to use for the scan. - pub manifest_evaluator_cache: Arc, - /// The expression evaluator cache to use for the scan. pub expression_evaluator_cache: Arc, diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 5d3b73ebe8..523fb9bb3f 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -5,11 +5,11 @@ use std::sync::Arc; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; use crate::arrow::delete_filter::DeleteFilter; -use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, IncrementalArrowBatchRecordStream}; +use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, CombinedIncrementalBatchRecordStream}; use crate::delete_file_index::DeleteFileIndex; use crate::io::FileIO; use crate::scan::DeleteFileContext; -use crate::scan::cache::{ExpressionEvaluatorCache, ManifestEvaluatorCache, PartitionFilterCache}; +use crate::scan::cache::ExpressionEvaluatorCache; use crate::scan::context::ManifestEntryContext; use crate::spec::{DataContentType, ManifestStatus, Snapshot, SnapshotRef}; use crate::table::Table; @@ -219,8 +219,6 @@ impl<'a> IncrementalTableScanBuilder<'a> { to_snapshot_schema: schema, object_cache: self.table.object_cache().clone(), field_ids: Arc::new(field_ids), - partition_filter_cache: Arc::new(PartitionFilterCache::new()), - manifest_evaluator_cache: Arc::new(ManifestEvaluatorCache::new()), expression_evaluator_cache: Arc::new(ExpressionEvaluatorCache::new()), caching_delete_file_loader: CachingDeleteFileLoader::new( self.table.file_io().clone(), @@ -450,8 +448,8 @@ impl IncrementalTableScan { Ok(futures::stream::iter(tasks).map(|t| Ok(t)).boxed()) } - /// Returns an [`IncrementalArrowBatchRecordStream`] for this incremental table scan. - pub async fn to_arrow(&self) -> Result { + /// Returns an [`CombinedIncrementalBatchRecordStream`] for this incremental table scan. + pub async fn to_arrow(&self) -> Result { let file_scan_task_stream = self.plan_files().await?; let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) .with_data_file_concurrency_limit(self.concurrency_limit_data_files) From 06694a86ccb75437f1a6e3bd4227b668835d8698 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:20:06 +0200 Subject: [PATCH 06/68] Remove printlns --- crates/iceberg/src/arrow/incremental.rs | 2 -- crates/iceberg/src/scan/incremental/mod.rs | 17 ----------------- 2 files changed, 19 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 6cbe107113..d61029af5d 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -207,8 +207,6 @@ async fn process_incremental_append_task( // Build the batch stream and send all the RecordBatches that it generates // to the requester. - let file_path = task.data_file_path.clone(); - println!("Reading data file: {}", file_path); let record_batch_stream = record_batch_stream_builder .build()? .map(move |batch| match batch { diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 523fb9bb3f..052cd54cf2 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -426,23 +426,6 @@ impl IncrementalTableScan { Ok(()) })?; - for task in tasks.iter() { - match task { - IncrementalFileScanTask::Append(append_task) => { - println!( - "Planned incremental append file scan task: {:?}, deletes: {:?}", - append_task.data_file_path, append_task.positional_deletes, - ); - } - IncrementalFileScanTask::Delete(delete_path, _) => { - println!( - "Planned incremental delete file scan task: {:?}", - delete_path, - ); - } - } - } - // We actually would not need a stream here, but we can keep it compatible with // other scan types. Ok(futures::stream::iter(tasks).map(|t| Ok(t)).boxed()) From 4514695a0f8ee20eb37fe30945bd77384a31bc28 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:22:54 +0200 Subject: [PATCH 07/68] Add API method for unzipped stream --- crates/iceberg/src/scan/incremental/mod.rs | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 052cd54cf2..635a8404e8 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -5,7 +5,7 @@ use std::sync::Arc; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; use crate::arrow::delete_filter::DeleteFilter; -use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, CombinedIncrementalBatchRecordStream}; +use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, UnzippedIncrementalBatchRecordStream}; use crate::delete_file_index::DeleteFileIndex; use crate::io::FileIO; use crate::scan::DeleteFileContext; @@ -447,6 +447,22 @@ impl IncrementalTableScan { file_scan_task_stream.read(arrow_reader) } + /// Returns an [`UnzippedIncrementalBatchRecordStream`] for this incremental table scan. + pub async fn to_unzipped_arrow(&self) -> Result { + let file_scan_task_stream = self.plan_files().await?; + let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); + + if let Some(batch_size) = self.batch_size { + arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); + } + + let arrow_reader = arrow_reader_builder.build(); + file_scan_task_stream.read(arrow_reader) + } + async fn process_delete_manifest_entry( mut delete_file_ctx_tx: Sender, manifest_entry_context: ManifestEntryContext, From f56e068788b8bff59eae9bc7608d54652b921375 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:23:07 +0200 Subject: [PATCH 08/68] . --- crates/iceberg/src/scan/incremental/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 635a8404e8..be37a501bd 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -448,6 +448,7 @@ impl IncrementalTableScan { } /// Returns an [`UnzippedIncrementalBatchRecordStream`] for this incremental table scan. + /// This stream will yield separate streams for appended and deleted record batches. pub async fn to_unzipped_arrow(&self) -> Result { let file_scan_task_stream = self.plan_files().await?; let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) From 4125328d5ec635dcf18e414a3aeeda5573c931bd Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:45:57 +0200 Subject: [PATCH 09/68] Remove comment --- crates/iceberg/src/arrow/incremental.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index d61029af5d..5d1216129d 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -118,7 +118,6 @@ impl ArrowBatchEmitter file_path, bit_map, batch_size, ); - // Write a match block like above with the following while loop in the Ok arm match record_batch_stream { Ok(mut stream) => { while let Some(batch) = stream.next().await { From 9fa636000f2515582818161a6fc9235417e171c0 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:46:50 +0200 Subject: [PATCH 10/68] Rename var --- crates/iceberg/src/arrow/incremental.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 5d1216129d..1eb9f2689d 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -110,12 +110,12 @@ impl ArrowBatchEmitter IncrementalFileScanTask::Delete(file_path, delete_vector) => { spawn(async move { // Clone the `RoaringTreemap` underlying the delete vector to take ownership. - let bit_map = { + let treemap = { let guard = delete_vector.lock().unwrap(); guard.inner.clone() }; let record_batch_stream = process_incremental_delete_task( - file_path, bit_map, batch_size, + file_path, treemap, batch_size, ); match record_batch_stream { From 14dd4e9be1642583187210e98b7f0b649ba395ca Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:49:59 +0200 Subject: [PATCH 11/68] Add import --- crates/iceberg/src/arrow/incremental.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 1eb9f2689d..cfe1a3c3f1 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use arrow_array::{RecordBatch, UInt64Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; +use futures::channel::mpsc::channel; use futures::stream::select; use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; use roaring::RoaringTreemap; @@ -57,10 +58,8 @@ impl ArrowBatchEmitter /// Take a stream of `IncrementalFileScanTasks` and reads all the files. Returns two /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. fn read(self, reader: ArrowReader) -> Result { - let (appends_tx, appends_rx) = - futures::channel::mpsc::channel(reader.concurrency_limit_data_files); - let (deletes_tx, deletes_rx) = - futures::channel::mpsc::channel(reader.concurrency_limit_data_files); + let (appends_tx, appends_rx) = channel(reader.concurrency_limit_data_files); + let (deletes_tx, deletes_rx) = channel(reader.concurrency_limit_data_files); let batch_size = reader.batch_size; let concurrency_limit_data_files = reader.concurrency_limit_data_files; From 837947aca1e01ef8cc9243fcc565306492952da9 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:50:04 +0200 Subject: [PATCH 12/68] Measure time --- crates/playground/src/main.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/crates/playground/src/main.rs b/crates/playground/src/main.rs index fcf5b67128..ded6dc362e 100644 --- a/crates/playground/src/main.rs +++ b/crates/playground/src/main.rs @@ -81,10 +81,15 @@ async fn main() -> Result<(), Box> { .await?; let mut rows = 0; - while let Some((batch_type, batch)) = stream.try_next().await? { + let time = std::time::Instant::now(); + println!("Starting to read incremental scan..."); + while let Some((_, batch)) = stream.try_next().await? { rows += batch.num_rows(); - println!("Batch: {:?}, Type: {:?}", batch, batch_type); } + println!( + "Finished reading incremental scan in {:?}", + time.elapsed() + ); println!("Total rows: {:?}", rows); Ok(()) From 5a7a223a105355a93c2357819be124bff2ee000a Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:51:40 +0200 Subject: [PATCH 13/68] Fix typo --- crates/iceberg/src/arrow/incremental.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index cfe1a3c3f1..4a811d8949 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -37,7 +37,7 @@ pub type UnzippedIncrementalBatchRecordStream = (ArrowRecordBatchStream, ArrowRe impl ArrowBatchEmitter for IncrementalFileScanTaskStream { - /// Take a stream of `IncrementalFileScanTasks` and reads all the files. Returns a + /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a /// stream of Arrow `RecordBatch`es containing the data from the files. fn read(self, reader: ArrowReader) -> Result { let (appends, deletes) = ArrowBatchEmitter::< @@ -55,7 +55,7 @@ impl ArrowBatchEmitter impl ArrowBatchEmitter for IncrementalFileScanTaskStream { - /// Take a stream of `IncrementalFileScanTasks` and reads all the files. Returns two + /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. fn read(self, reader: ArrowReader) -> Result { let (appends_tx, appends_rx) = channel(reader.concurrency_limit_data_files); From 065c75414cfebe978a191f4321ee1866f9dca667 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 16:55:21 +0200 Subject: [PATCH 14/68] Undo some changes --- crates/iceberg/src/arrow/reader.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 760ebaff5e..0a27d45862 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -526,7 +526,7 @@ impl ArrowReader { }) } - pub(crate) fn build_field_id_set_and_map( + fn build_field_id_set_and_map( parquet_schema: &SchemaDescriptor, predicate: &BoundPredicate, ) -> Result<(HashSet, HashMap)> { @@ -544,7 +544,7 @@ impl ArrowReader { /// Insert the leaf field id into the field_ids using for projection. /// For nested type, it will recursively insert the leaf field id. - pub(crate) fn include_leaf_field_id(field: &NestedField, field_ids: &mut Vec) { + fn include_leaf_field_id(field: &NestedField, field_ids: &mut Vec) { match field.field_type.as_ref() { Type::Primitive(_) => { field_ids.push(field.id); @@ -1435,11 +1435,13 @@ impl AsyncFileReader for ArrowFileReader { fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { Box::pin( self.r - .read(range.start as u64..range.end as u64) + .read(range.start..range.end) .map_err(|err| parquet::errors::ParquetError::External(Box::new(err))), ) } + // TODO: currently we don't respect `ArrowReaderOptions` cause it don't expose any method to access the option field + // we will fix it after `v55.1.0` is released in https://github.com/apache/arrow-rs/issues/7393 fn get_metadata( &mut self, _options: Option<&'_ ArrowReaderOptions>, From f9ea05c32b0f8450f45497b55a6f0ecb68125fba Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 23 Oct 2025 17:00:10 +0200 Subject: [PATCH 15/68] Change type name --- crates/iceberg/src/arrow/incremental.rs | 8 ++++---- crates/iceberg/src/arrow/reader.rs | 8 ++++---- crates/iceberg/src/scan/incremental/mod.rs | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 4a811d8949..602e7b17c4 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -9,7 +9,7 @@ use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; use roaring::RoaringTreemap; use crate::arrow::record_batch_transformer::RecordBatchTransformer; -use crate::arrow::{ArrowBatchEmitter, ArrowReader}; +use crate::arrow::{StreamsInto, ArrowReader}; use crate::io::FileIO; use crate::runtime::spawn; use crate::scan::ArrowRecordBatchStream; @@ -34,13 +34,13 @@ pub type CombinedIncrementalBatchRecordStream = /// Stream type for obtaining a separate stream of appended and deleted record batches. pub type UnzippedIncrementalBatchRecordStream = (ArrowRecordBatchStream, ArrowRecordBatchStream); -impl ArrowBatchEmitter +impl StreamsInto for IncrementalFileScanTaskStream { /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a /// stream of Arrow `RecordBatch`es containing the data from the files. fn read(self, reader: ArrowReader) -> Result { - let (appends, deletes) = ArrowBatchEmitter::< + let (appends, deletes) = StreamsInto::< ArrowReader, UnzippedIncrementalBatchRecordStream, >::read(self, reader)?; @@ -52,7 +52,7 @@ impl ArrowBatchEmitter } } -impl ArrowBatchEmitter +impl StreamsInto for IncrementalFileScanTaskStream { /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 0a27d45862..297b6fd4d8 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -141,9 +141,10 @@ pub struct ArrowReader { pub(crate) row_selection_enabled: bool, } -/// Trait indicating that Arrow batches can be read from an implementing type. -pub trait ArrowBatchEmitter { - /// Reads Arrow batches from the implementing type. +/// Trait indicating that the implementing type streams into a stream of type `S` using +/// a reader of type `R`. +pub trait StreamsInto { + /// Read from the reader and produce a stream of type `S`. fn read(self, reader: R) -> Result; } @@ -356,7 +357,6 @@ impl ArrowReader { ArrowReaderOptions::new(), ) .await?; - Ok(record_batch_stream_builder) } diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index be37a501bd..2761fd972c 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -5,7 +5,7 @@ use std::sync::Arc; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; use crate::arrow::delete_filter::DeleteFilter; -use crate::arrow::{ArrowBatchEmitter, ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, UnzippedIncrementalBatchRecordStream}; +use crate::arrow::{StreamsInto, ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, UnzippedIncrementalBatchRecordStream}; use crate::delete_file_index::DeleteFileIndex; use crate::io::FileIO; use crate::scan::DeleteFileContext; From 06cba669b16a73c1e2036248f8309d0de4f4ad24 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Fri, 24 Oct 2025 09:05:16 +0200 Subject: [PATCH 16/68] Add comment header --- crates/iceberg/src/arrow/incremental.rs | 25 +++++++++++++++---- .../iceberg/src/scan/incremental/context.rs | 17 +++++++++++++ crates/iceberg/src/scan/incremental/mod.rs | 22 +++++++++++++++- crates/iceberg/src/scan/incremental/task.rs | 17 +++++++++++++ crates/playground/src/main.rs | 5 +--- 5 files changed, 76 insertions(+), 10 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 602e7b17c4..6225cdb4ae 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::pin::Pin; use std::sync::Arc; @@ -9,7 +26,7 @@ use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; use roaring::RoaringTreemap; use crate::arrow::record_batch_transformer::RecordBatchTransformer; -use crate::arrow::{StreamsInto, ArrowReader}; +use crate::arrow::{ArrowReader, StreamsInto}; use crate::io::FileIO; use crate::runtime::spawn; use crate::scan::ArrowRecordBatchStream; @@ -40,10 +57,8 @@ impl StreamsInto /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a /// stream of Arrow `RecordBatch`es containing the data from the files. fn read(self, reader: ArrowReader) -> Result { - let (appends, deletes) = StreamsInto::< - ArrowReader, - UnzippedIncrementalBatchRecordStream, - >::read(self, reader)?; + let (appends, deletes) = + StreamsInto::::read(self, reader)?; let left = appends.map(|res| res.map(|batch| (IncrementalBatchType::Append, batch))); let right = deletes.map(|res| res.map(|batch| (IncrementalBatchType::Delete, batch))); diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index 09614cc86f..e4512c9e86 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::collections::HashSet; use std::sync::Arc; diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 2761fd972c..2d436191ac 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -1,3 +1,20 @@ +// 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. + //! Incremental table scan implementation. use std::collections::HashSet; @@ -5,7 +22,10 @@ use std::sync::Arc; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; use crate::arrow::delete_filter::DeleteFilter; -use crate::arrow::{StreamsInto, ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, UnzippedIncrementalBatchRecordStream}; +use crate::arrow::{ + ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, StreamsInto, + UnzippedIncrementalBatchRecordStream, +}; use crate::delete_file_index::DeleteFileIndex; use crate::io::FileIO; use crate::scan::DeleteFileContext; diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs index 2ef6a63bd2..0fca8d1233 100644 --- a/crates/iceberg/src/scan/incremental/task.rs +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::sync::{Arc, Mutex}; use futures::stream::BoxStream; diff --git a/crates/playground/src/main.rs b/crates/playground/src/main.rs index ded6dc362e..b23ac7f197 100644 --- a/crates/playground/src/main.rs +++ b/crates/playground/src/main.rs @@ -86,10 +86,7 @@ async fn main() -> Result<(), Box> { while let Some((_, batch)) = stream.try_next().await? { rows += batch.num_rows(); } - println!( - "Finished reading incremental scan in {:?}", - time.elapsed() - ); + println!("Finished reading incremental scan in {:?}", time.elapsed()); println!("Total rows: {:?}", rows); Ok(()) From 2ddde41c129e7d5b716fc86004b54541651f15fe Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Fri, 24 Oct 2025 10:07:14 +0200 Subject: [PATCH 17/68] Fail when encountering equality deletes --- crates/iceberg/src/scan/incremental/mod.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 2d436191ac..a71abaeea7 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -493,12 +493,20 @@ impl IncrementalTableScan { return Ok(()); } - // Abort the plan if we encounter a manifest entry for a data file + // Abort the plan if we encounter a manifest entry for a data file or equality + // deletes. if manifest_entry_context.manifest_entry.content_type() == DataContentType::Data { return Err(Error::new( ErrorKind::FeatureUnsupported, "Encountered an entry for a data file in a delete file manifest", )); + } else if manifest_entry_context.manifest_entry.content_type() + == DataContentType::EqualityDeletes + { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Equality deletes are not supported yet in incremental scans", + )); } delete_file_ctx_tx From facbeeb07c0efdcff0d2b6355f043b0a07a614d8 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Fri, 24 Oct 2025 12:39:30 +0200 Subject: [PATCH 18/68] Add comments --- crates/iceberg/src/scan/incremental/mod.rs | 3 +++ crates/iceberg/src/scan/incremental/task.rs | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index a71abaeea7..83530d6ef5 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -423,8 +423,10 @@ impl IncrementalTableScan { } }); + // Collect all append tasks. let mut tasks = file_scan_task_rx.try_collect::>().await?; + // Compute those file paths that have been appended. let appended_files = tasks .iter() .filter_map(|task| match task { @@ -435,6 +437,7 @@ impl IncrementalTableScan { }) .collect::>(); + // Augment `tasks` with delete tasks. delete_filter.with_read(|state| { for (path, delete_vector) in state.delete_vectors().iter() { if !appended_files.contains::(path) { diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs index 0fca8d1233..7a30c13995 100644 --- a/crates/iceberg/src/scan/incremental/task.rs +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -71,7 +71,8 @@ pub type IncrementalFileScanTaskStream = BoxStream<'static, Result>), } From ae7663022e122497435d63438b3f50f970f4d0df Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Sat, 25 Oct 2025 12:22:56 +0200 Subject: [PATCH 19/68] Add some preliminary tests --- crates/iceberg/src/arrow/incremental.rs | 2 +- crates/iceberg/src/scan/incremental/mod.rs | 3 + crates/iceberg/src/scan/incremental/tests.rs | 805 +++++++++++++++++++ 3 files changed, 809 insertions(+), 1 deletion(-) create mode 100644 crates/iceberg/src/scan/incremental/tests.rs diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 6225cdb4ae..8252a2138d 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -165,8 +165,8 @@ impl StreamsInto }); return Ok(( - Box::pin(deletes_rx) as ArrowRecordBatchStream, Box::pin(appends_rx) as ArrowRecordBatchStream, + Box::pin(deletes_rx) as ArrowRecordBatchStream, )); } } diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 83530d6ef5..88bb0e9f03 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -548,3 +548,6 @@ impl IncrementalTableScan { Ok(()) } } + +#[cfg(test)] +mod tests; diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs new file mode 100644 index 0000000000..e8c58882a8 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -0,0 +1,805 @@ +use std::collections::HashMap; +use std::fs; +use std::fs::File; +use std::sync::Arc; + +use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use parquet::arrow::{ArrowWriter, PARQUET_FIELD_ID_META_KEY}; +use parquet::basic::Compression; +use parquet::file::properties::WriterProperties; +use tempfile::TempDir; +use uuid::Uuid; + +use crate::TableIdent; +use crate::io::{FileIO, OutputFile}; +use crate::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, ManifestEntry, ManifestListWriter, + ManifestStatus, ManifestWriterBuilder, PartitionSpec, Struct, TableMetadata, +}; +use crate::table::Table; + +/// Represents an operation to perform on a snapshot. +#[derive(Debug, Clone)] +pub enum Operation { + /// Add rows with the given `n` values and `data` values. Example: `Add(vec![1, 2, 3], + /// vec!["a", "b", "c"])` adds three rows with n=1,2,3 and data="a","b","c" + Add(Vec, Vec), + + /// Delete rows by their n values (uses positional deletes). + /// Example: `Delete(vec![2])` deletes the row where n=2 + Delete(Vec), +} + +/// Tracks the state of data files across snapshots +#[derive(Debug, Clone)] +struct DataFileInfo { + path: String, + snapshot_id: i64, + sequence_number: i64, + n_values: Vec, +} + +/// Test fixture that creates a table with custom snapshots based on operations. +/// +/// # Example +/// ``` +/// let fixture = IncrementalTestFixture::new(vec![ +/// Operation::Add(vec![], vec![]), // Empty snapshot +/// Operation::Add(vec![1, 2, 3], vec!["1", "2", "3"]), // Add 3 rows +/// Operation::Delete(vec![2]), // Delete row with n=2 +/// ]) +/// .await; +/// ``` +pub struct IncrementalTestFixture { + pub table_location: String, + pub table: Table, + _tmp_dir: TempDir, // Keep temp dir alive +} + +impl IncrementalTestFixture { + /// Create a new test fixture with the given operations. + pub async fn new(operations: Vec) -> Self { + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().join("incremental_test_table"); + + // Create directory structure + fs::create_dir_all(table_location.join("metadata")).unwrap(); + fs::create_dir_all(table_location.join("data")).unwrap(); + + let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) + .unwrap() + .build() + .unwrap(); + + let num_snapshots = operations.len(); + let current_snapshot_id = num_snapshots as i64; + let last_sequence_number = (num_snapshots - 1) as i64; + + // Build the snapshots JSON dynamically + let mut snapshots_json = Vec::new(); + let mut snapshot_log_json = Vec::new(); + let mut manifest_list_locations = Vec::new(); + + for (i, op) in operations.iter().enumerate() { + let snapshot_id = (i + 1) as i64; + let parent_id = if i == 0 { None } else { Some(i as i64) }; + let sequence_number = i as i64; + let timestamp = 1515100955770 + (i as i64 * 1000); + + let operation_type = match op { + Operation::Add(..) => "append", + Operation::Delete(..) => "delete", + }; + + let manifest_list_location = + table_location.join(format!("metadata/snap-{}-manifest-list.avro", snapshot_id)); + manifest_list_locations.push(manifest_list_location.clone()); + + let parent_str = if let Some(pid) = parent_id { + format!(r#""parent-snapshot-id": {},"#, pid) + } else { + String::new() + }; + + snapshots_json.push(format!( + r#" {{ + "snapshot-id": {}, + {} + "timestamp-ms": {}, + "sequence-number": {}, + "summary": {{"operation": "{}"}}, + "manifest-list": "{}", + "schema-id": 0 + }}"#, + snapshot_id, + parent_str, + timestamp, + sequence_number, + operation_type, + manifest_list_location.display() + )); + + snapshot_log_json.push(format!( + r#" {{"snapshot-id": {}, "timestamp-ms": {}}}"#, + snapshot_id, timestamp + )); + } + + let snapshots_str = snapshots_json.join(",\n"); + let snapshot_log_str = snapshot_log_json.join(",\n"); + + // Create the table metadata + let metadata_json = format!( + r#"{{ + "format-version": 2, + "table-uuid": "{}", + "location": "{}", + "last-sequence-number": {}, + "last-updated-ms": 1602638573590, + "last-column-id": 2, + "current-schema-id": 0, + "schemas": [ + {{ + "type": "struct", + "schema-id": 0, + "fields": [ + {{"id": 1, "name": "n", "required": true, "type": "int"}}, + {{"id": 2, "name": "data", "required": true, "type": "string"}} + ] + }} + ], + "default-spec-id": 0, + "partition-specs": [ + {{ + "spec-id": 0, + "fields": [] + }} + ], + "last-partition-id": 0, + "default-sort-order-id": 0, + "sort-orders": [ + {{ + "order-id": 0, + "fields": [] + }} + ], + "properties": {{}}, + "current-snapshot-id": {}, + "snapshots": [ +{} + ], + "snapshot-log": [ +{} + ], + "metadata-log": [] +}}"#, + Uuid::new_v4(), + table_location.display(), + last_sequence_number, + current_snapshot_id, + snapshots_str, + snapshot_log_str + ); + + let table_metadata_location = table_location.join("metadata/v1.json"); + let table_metadata = serde_json::from_str::(&metadata_json).unwrap(); + + let table = Table::builder() + .metadata(table_metadata) + .identifier(TableIdent::from_strs(["db", "incremental_test"]).unwrap()) + .file_io(file_io.clone()) + .metadata_location(table_metadata_location.as_os_str().to_str().unwrap()) + .build() + .unwrap(); + + let mut fixture = Self { + table_location: table_location.to_str().unwrap().to_string(), + table, + _tmp_dir: tmp_dir, + }; + + // Setup all snapshots based on operations + fixture.setup_snapshots(operations).await; + + fixture + } + + fn next_manifest_file(&self) -> OutputFile { + self.table + .file_io() + .new_output(format!( + "{}/metadata/manifest_{}.avro", + self.table_location, + Uuid::new_v4() + )) + .unwrap() + } + + async fn setup_snapshots(&mut self, operations: Vec) { + let current_schema = self + .table + .metadata() + .current_snapshot() + .unwrap() + .schema(self.table.metadata()) + .unwrap(); + let partition_spec = Arc::new(PartitionSpec::unpartition_spec()); + let empty_partition = Struct::empty(); + + // Track all data files and their contents across snapshots + let mut data_files: Vec = Vec::new(); + let mut delete_files: Vec<(String, i64, i64, Vec<(String, i64)>)> = Vec::new(); // (path, snapshot_id, sequence_number, [(data_file_path, position)]) + + for (snapshot_idx, operation) in operations.iter().enumerate() { + let snapshot_id = (snapshot_idx + 1) as i64; + let sequence_number = snapshot_idx as i64; + let parent_snapshot_id = if snapshot_idx == 0 { + None + } else { + Some(snapshot_idx as i64) + }; + + match operation { + Operation::Add(n_values, data_values) => { + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + // Add existing data files from previous snapshots + for data_file in &data_files { + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new data if not empty + if !n_values.is_empty() { + let data_file_path = + format!("{}/data/data-{}.parquet", &self.table_location, snapshot_id); + self.write_parquet_file(&data_file_path, n_values, data_values) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this data file + data_files.push(DataFileInfo { + path: data_file_path, + snapshot_id, + sequence_number, + n_values: n_values.clone(), + }); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Create delete manifest if there are any delete files + let mut manifests = vec![data_manifest]; + if !delete_files.is_empty() { + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + manifests.push(delete_writer.write_manifest_file().await.unwrap()); + } + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(manifests.into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + } + + Operation::Delete(n_values_to_delete) => { + // Find positions to delete from each data file + let mut deletes_by_file: HashMap> = HashMap::new(); + + for n_to_delete in n_values_to_delete { + for data_file in &data_files { + if let Some(pos) = + data_file.n_values.iter().position(|n| n == n_to_delete) + { + deletes_by_file + .entry(data_file.path.clone()) + .or_insert_with(Vec::new) + .push(pos as i64); + } + } + } + + // Create data manifest with existing data files + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + for data_file in &data_files { + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Create delete manifest + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + // Add existing delete files + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new delete files + for (data_file_path, positions) in deletes_by_file { + let delete_file_path = format!( + "{}/data/delete-{}-{}.parquet", + &self.table_location, + snapshot_id, + Uuid::new_v4() + ); + self.write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; + + delete_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(positions.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this delete file + delete_files.push(( + delete_file_path, + snapshot_id, + sequence_number, + positions + .into_iter() + .map(|pos| (data_file_path.clone(), pos)) + .collect(), + )); + } + + let delete_manifest = delete_writer.write_manifest_file().await.unwrap(); + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(vec![data_manifest, delete_manifest].into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + } + } + } + } + + async fn write_parquet_file(&self, path: &str, n_values: &[i32], data_values: &[String]) { + let schema = { + let fields = vec![ + arrow_schema::Field::new("n", arrow_schema::DataType::Int32, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ), + arrow_schema::Field::new("data", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let col_n = Arc::new(Int32Array::from(n_values.to_vec())) as ArrayRef; + let col_data = Arc::new(StringArray::from(data_values.to_vec())) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![col_n, col_data]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let file = File::create(path).unwrap(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + + async fn write_positional_delete_file( + &self, + path: &str, + data_file_path: &str, + positions: &[i64], + ) { + let schema = { + let fields = vec![ + arrow_schema::Field::new("file_path", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2147483546".to_string(), + )])), + arrow_schema::Field::new("pos", arrow_schema::DataType::Int64, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2147483545".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let file_paths: Vec<&str> = vec![data_file_path; positions.len()]; + let col_file_path = Arc::new(StringArray::from(file_paths)) as ArrayRef; + let col_pos = Arc::new(arrow_array::Int64Array::from(positions.to_vec())) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![col_file_path, col_pos]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let file = File::create(path).unwrap(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + + /// Verify incremental scan results. + /// + /// Verifies that the incremental scan contains the expected appended and deleted records. + pub async fn verify_incremental_scan( + &self, + from_snapshot_id: i64, + to_snapshot_id: i64, + expected_appends: Vec<(i32, &str)>, + expected_deletes: Vec<(i32, &str)>, + ) { + use arrow_array::cast::AsArray; + use arrow_select::concat::concat_batches; + use futures::TryStreamExt; + + let incremental_scan = self + .table + .incremental_scan(from_snapshot_id, to_snapshot_id) + .build() + .unwrap(); + + let stream = incremental_scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Separate appends and deletes + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + let delete_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Delete) + .map(|(_, b)| b.clone()) + .collect(); + + // Verify appended records + if !append_batches.is_empty() { + let append_batch = + concat_batches(&append_batches[0].schema(), append_batches.iter()).unwrap(); + + let n_array = append_batch + .column(0) + .as_primitive::(); + let data_array = append_batch.column(1).as_string::(); + + let mut appended_pairs: Vec<(i32, String)> = (0..append_batch.num_rows()) + .map(|i| (n_array.value(i), data_array.value(i).to_string())) + .collect(); + appended_pairs.sort(); + + let expected_appends: Vec<(i32, String)> = expected_appends + .into_iter() + .map(|(n, s)| (n, s.to_string())) + .collect(); + + assert_eq!(appended_pairs, expected_appends); + } else { + assert!(expected_appends.is_empty(), "Expected appends but got none"); + } + + // Verify deleted records + if !delete_batches.is_empty() { + let delete_batch = + concat_batches(&delete_batches[0].schema(), delete_batches.iter()).unwrap(); + + let n_array = delete_batch + .column(0) + .as_primitive::(); + let data_array = delete_batch.column(1).as_string::(); + + let mut deleted_pairs: Vec<(i32, String)> = (0..delete_batch.num_rows()) + .map(|i| (n_array.value(i), data_array.value(i).to_string())) + .collect(); + deleted_pairs.sort(); + + let expected_deletes: Vec<(i32, String)> = expected_deletes + .into_iter() + .map(|(n, s)| (n, s.to_string())) + .collect(); + + assert_eq!(deleted_pairs, expected_deletes); + } else { + assert!( + expected_deletes.is_empty(), + "Expected deletes but got none" + ); + } + } +} + +#[tokio::test] +async fn test_incremental_fixture_simple() { + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], vec![]), + Operation::Add(vec![1, 2, 3], vec![ + "1".to_string(), + "2".to_string(), + "3".to_string(), + ]), + Operation::Delete(vec![2]), + ]) + .await; + + // Verify we have 3 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 3); + + // Verify snapshot IDs + assert_eq!(snapshots[0].snapshot_id(), 1); + assert_eq!(snapshots[1].snapshot_id(), 2); + assert_eq!(snapshots[2].snapshot_id(), 3); + + // Verify parent relationships + assert_eq!(snapshots[0].parent_snapshot_id(), None); + assert_eq!(snapshots[1].parent_snapshot_id(), Some(1)); + assert_eq!(snapshots[2].parent_snapshot_id(), Some(2)); + + // Verify incremental scan from snapshot 1 to snapshot 3 + // Expected appends: snapshot 2 adds [1, 2, 3] + // Expected deletes: snapshot 3 deletes [2] + // In total we expect appends [1, 3] and deletes [] + fixture.verify_incremental_scan( + 1, + 3, + vec![(1, "1"), (3, "3")], + vec![], + ) + .await; +} + +#[tokio::test] +async fn test_incremental_fixture_complex() { + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], vec![]), // Snapshot 1: Empty + Operation::Add(vec![1, 2, 3, 4, 5], vec![ + "a".to_string(), + "b".to_string(), + "c".to_string(), + "d".to_string(), + "e".to_string(), + ]), // Snapshot 2: Add 5 rows + Operation::Delete(vec![2, 4]), // Snapshot 3: Delete rows with n=2,4 + Operation::Add(vec![6, 7], vec!["f".to_string(), "g".to_string()]), // Snapshot 4: Add 2 more rows + Operation::Delete(vec![1, 3, 5, 6, 7]), // Snapshot 5: Delete all remaining rows + ]) + .await; + + // Verify we have 5 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 5); + + // Verify parent chain + assert_eq!(snapshots[0].parent_snapshot_id(), None); + for i in 1..5 { + assert_eq!(snapshots[i].parent_snapshot_id(), Some(i as i64)); + } + + // Verify current snapshot + assert_eq!( + fixture + .table + .metadata() + .current_snapshot() + .unwrap() + .snapshot_id(), + 5 + ); + + // Verify incremental scan from snapshot 1 to snapshot 5 + // All data has been deleted, so we expect the empty result. + fixture.verify_incremental_scan( + 1, + 5, + vec![], + vec![], + ) + .await; +} From 27806347558431b981dad6ec843d660dbdf3f6e2 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Sat, 25 Oct 2025 12:23:10 +0200 Subject: [PATCH 20/68] Format --- crates/iceberg/src/scan/incremental/tests.rs | 23 +++++--------------- 1 file changed, 5 insertions(+), 18 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index e8c58882a8..97a9812d86 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -705,10 +705,7 @@ impl IncrementalTestFixture { assert_eq!(deleted_pairs, expected_deletes); } else { - assert!( - expected_deletes.is_empty(), - "Expected deletes but got none" - ); + assert!(expected_deletes.is_empty(), "Expected deletes but got none"); } } } @@ -745,13 +742,9 @@ async fn test_incremental_fixture_simple() { // Expected appends: snapshot 2 adds [1, 2, 3] // Expected deletes: snapshot 3 deletes [2] // In total we expect appends [1, 3] and deletes [] - fixture.verify_incremental_scan( - 1, - 3, - vec![(1, "1"), (3, "3")], - vec![], - ) - .await; + fixture + .verify_incremental_scan(1, 3, vec![(1, "1"), (3, "3")], vec![]) + .await; } #[tokio::test] @@ -795,11 +788,5 @@ async fn test_incremental_fixture_complex() { // Verify incremental scan from snapshot 1 to snapshot 5 // All data has been deleted, so we expect the empty result. - fixture.verify_incremental_scan( - 1, - 5, - vec![], - vec![], - ) - .await; + fixture.verify_incremental_scan(1, 5, vec![], vec![]).await; } From 536bbc4851856e5b14238d13e8aa5ce8308e657a Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Sat, 25 Oct 2025 13:51:57 +0200 Subject: [PATCH 21/68] Remove playground --- Cargo.toml | 1 - crates/playground/Cargo.toml | 15 ------ crates/playground/src/main.rs | 93 ----------------------------------- 3 files changed, 109 deletions(-) delete mode 100644 crates/playground/Cargo.toml delete mode 100644 crates/playground/src/main.rs diff --git a/Cargo.toml b/Cargo.toml index 7f9d4a9251..6167380594 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -25,7 +25,6 @@ members = [ "crates/integrations/*", "crates/sqllogictest", "crates/test_utils", - "crates/playground", ] resolver = "2" diff --git a/crates/playground/Cargo.toml b/crates/playground/Cargo.toml deleted file mode 100644 index d09970ba6c..0000000000 --- a/crates/playground/Cargo.toml +++ /dev/null @@ -1,15 +0,0 @@ -[package] -name = "playground" -edition.workspace = true -homepage.workspace = true -version.workspace = true -license.workspace = true -repository.workspace = true -rust-version.workspace = true - -[dependencies] -dotenv = "0.15.0" -futures-util = "0.3.31" -iceberg.workspace = true -iceberg-catalog-rest = { workspace = true } -tokio.workspace = true diff --git a/crates/playground/src/main.rs b/crates/playground/src/main.rs deleted file mode 100644 index b23ac7f197..0000000000 --- a/crates/playground/src/main.rs +++ /dev/null @@ -1,93 +0,0 @@ -use std::collections::HashMap; -use std::env; - -use dotenv::dotenv; -use futures_util::TryStreamExt; -use iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, S3_SECRET_ACCESS_KEY}; -use iceberg::{Catalog, CatalogBuilder, NamespaceIdent}; -use iceberg_catalog_rest::{ - REST_CATALOG_PROP_URI, REST_CATALOG_PROP_WAREHOUSE, RestCatalogBuilder, -}; -use tokio; - -#[tokio::main(flavor = "multi_thread")] -async fn main() -> Result<(), Box> { - dotenv().ok(); - - let args: Vec = env::args().collect(); - if args.len() < 5 { - eprintln!( - "Usage: {} ", - args[0] - ); - return Err("Not enough arguments".into()); - } - let namespace = &args[1]; - let table_name = &args[2]; - let from_snapshot_id: i64 = args[3].parse().unwrap(); - let to_snapshot_id: i64 = args[4].parse().unwrap(); - - let bucket = env::var("S3_BUCKET").expect("S3_BUCKET must be set"); - - // Create catalog - let catalog = RestCatalogBuilder::default() - .load( - "rest", - HashMap::from([ - ( - REST_CATALOG_PROP_URI.to_string(), - "http://localhost:8181".to_string(), - ), - ( - REST_CATALOG_PROP_WAREHOUSE.to_string(), - format!("s3://{}", bucket), - ), - ( - S3_ACCESS_KEY_ID.to_string(), - env::var("AWS_ACCESS_KEY_ID").unwrap_or("admin".to_string()), - ), - ( - S3_SECRET_ACCESS_KEY.to_string(), - env::var("AWS_SECRET_ACCESS_KEY").unwrap_or("password".to_string()), - ), - ( - S3_REGION.to_string(), - env::var("AWS_REGION").unwrap_or("us-east-1".to_string()), - ), - ( - S3_ENDPOINT.to_string(), - env::var("AWS_ENDPOINT_URL").unwrap_or("http://localhost:9000".to_string()), - ), - ]), - ) - .await?; - - let namespace = NamespaceIdent::new(namespace.clone()); - // Error if namespace doesn't exist - if !catalog.namespace_exists(&namespace).await? { - return Err("Namespace does not exist".into()); - } - - let table = catalog - .load_table(&iceberg::TableIdent::new( - namespace.clone(), - table_name.clone(), - )) - .await?; - let mut stream = table - .incremental_scan(from_snapshot_id, to_snapshot_id) - .build()? - .to_arrow() - .await?; - - let mut rows = 0; - let time = std::time::Instant::now(); - println!("Starting to read incremental scan..."); - while let Some((_, batch)) = stream.try_next().await? { - rows += batch.num_rows(); - } - println!("Finished reading incremental scan in {:?}", time.elapsed()); - - println!("Total rows: {:?}", rows); - Ok(()) -} From 5ac188793324b2af5caad332142578560b99f3cc Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 08:40:23 +0100 Subject: [PATCH 22/68] Add more tests --- Cargo.lock | 18 +----------------- crates/iceberg/src/scan/incremental/tests.rs | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b3eba50023..1e520bd448 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2698,12 +2698,6 @@ dependencies = [ "const-random", ] -[[package]] -name = "dotenv" -version = "0.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77c90badedccf4105eca100756a0b1289e191f6fcbdadd3cee1d2f614f97da8f" - [[package]] name = "dotenvy" version = "0.15.7" @@ -3789,6 +3783,7 @@ dependencies = [ "datafusion-cli", "dirs", "fs-err", + "home", "iceberg", "iceberg-catalog-rest", "iceberg-datafusion", @@ -5269,17 +5264,6 @@ version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" -[[package]] -name = "playground" -version = "0.7.0" -dependencies = [ - "dotenv", - "futures-util", - "iceberg", - "iceberg-catalog-rest", - "tokio", -] - [[package]] name = "polling" version = "3.11.0" diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 97a9812d86..a9d568d763 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::collections::HashMap; use std::fs; use std::fs::File; From 1f7202500b595512232be8384b1196730e90b1f8 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 08:51:42 +0100 Subject: [PATCH 23/68] Clippy --- crates/iceberg/src/arrow/incremental.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 8252a2138d..165da869ae 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -164,10 +164,10 @@ impl StreamsInto .await; }); - return Ok(( + Ok(( Box::pin(appends_rx) as ArrowRecordBatchStream, Box::pin(deletes_rx) as ArrowRecordBatchStream, - )); + )) } } @@ -238,7 +238,7 @@ fn process_incremental_delete_task( let stream = futures::stream::iter(delete_vector.into_iter()) .chunks(batch_size.unwrap_or(1024)) .map(move |chunk| { - let array = UInt64Array::from_iter(chunk.into_iter()); + let array = UInt64Array::from_iter(chunk); RecordBatch::try_new( Arc::new(ArrowSchema::new(vec![Field::new( "pos", From 6d72c8cd601b624057ab1ec72be088e87f2a418f Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 09:11:17 +0100 Subject: [PATCH 24/68] . --- crates/iceberg/src/arrow/incremental.rs | 2 +- crates/iceberg/src/scan/incremental/context.rs | 2 +- crates/iceberg/src/scan/incremental/mod.rs | 2 +- crates/iceberg/src/scan/incremental/task.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 165da869ae..5e7d954e88 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -235,7 +235,7 @@ fn process_incremental_delete_task( delete_vector: RoaringTreemap, batch_size: Option, ) -> Result { - let stream = futures::stream::iter(delete_vector.into_iter()) + let stream = futures::stream::iter(delete_vector) .chunks(batch_size.unwrap_or(1024)) .map(move |chunk| { let array = UInt64Array::from_iter(chunk); diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index e4512c9e86..4b6a2b3d62 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -82,7 +82,7 @@ impl IncrementalPlanContext { for snapshot in self.snapshots.iter() { let manifest_list = self .object_cache - .get_manifest_list(&snapshot, &self.table_metadata) + .get_manifest_list(snapshot, &self.table_metadata) .await?; for entry in manifest_list.entries() { if !snapshot_ids.contains(&entry.added_snapshot_id) { diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 88bb0e9f03..cab3175a75 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -451,7 +451,7 @@ impl IncrementalTableScan { // We actually would not need a stream here, but we can keep it compatible with // other scan types. - Ok(futures::stream::iter(tasks).map(|t| Ok(t)).boxed()) + Ok(futures::stream::iter(tasks).map(Ok).boxed()) } /// Returns an [`CombinedIncrementalBatchRecordStream`] for this incremental table scan. diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs index 7a30c13995..302f1a68a3 100644 --- a/crates/iceberg/src/scan/incremental/task.rs +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -91,7 +91,7 @@ impl IncrementalFileScanTask { data_file_format: manifest_entry_context.manifest_entry.file_format(), schema: manifest_entry_context.snapshot_schema.clone(), project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), - positional_deletes: delete_filter.get_delete_vector_for_path(&data_file_path), + positional_deletes: delete_filter.get_delete_vector_for_path(data_file_path), }) } From a024e60f4fa6e53b20aa6eadd35657c0d8bfc1aa Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 09:22:25 +0100 Subject: [PATCH 25/68] . --- crates/iceberg/src/scan/incremental/tests.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index a9d568d763..c84b502a59 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -245,6 +245,7 @@ impl IncrementalTestFixture { // Track all data files and their contents across snapshots let mut data_files: Vec = Vec::new(); + #[allow(clippy::type_complexity)] let mut delete_files: Vec<(String, i64, i64, Vec<(String, i64)>)> = Vec::new(); // (path, snapshot_id, sequence_number, [(data_file_path, position)]) for (snapshot_idx, operation) in operations.iter().enumerate() { @@ -412,7 +413,7 @@ impl IncrementalTestFixture { { deletes_by_file .entry(data_file.path.clone()) - .or_insert_with(Vec::new) + .or_default() .push(pos as i64); } } @@ -788,8 +789,8 @@ async fn test_incremental_fixture_complex() { // Verify parent chain assert_eq!(snapshots[0].parent_snapshot_id(), None); - for i in 1..5 { - assert_eq!(snapshots[i].parent_snapshot_id(), Some(i as i64)); + for (i, snapshot) in snapshots.iter().enumerate().take(5).skip(1) { + assert_eq!(snapshot.parent_snapshot_id(), Some(i as i64)); } // Verify current snapshot From 773a9d0b48ad151f083d931d823653b080c81193 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 11:08:08 +0100 Subject: [PATCH 26/68] Adapt tests --- crates/iceberg/src/scan/incremental/mod.rs | 10 ++-- crates/iceberg/src/scan/incremental/tests.rs | 63 +++++++++++--------- 2 files changed, 41 insertions(+), 32 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index cab3175a75..30d8a3dd31 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -174,10 +174,12 @@ impl<'a> IncrementalTableScanBuilder<'a> { ) .collect_vec(); - assert_eq!( - snapshots.first().map(|s| s.snapshot_id()), - Some(snapshot_to.snapshot_id()) - ); + if !snapshots.is_empty() { + assert_eq!( + snapshots.first().map(|s| s.snapshot_id()), + Some(snapshot_to.snapshot_id()) + ); + } let schema = snapshot_to.schema(self.table.metadata())?; diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index c84b502a59..f3733d989c 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -42,9 +42,10 @@ pub enum Operation { /// vec!["a", "b", "c"])` adds three rows with n=1,2,3 and data="a","b","c" Add(Vec, Vec), - /// Delete rows by their n values (uses positional deletes). - /// Example: `Delete(vec![2])` deletes the row where n=2 - Delete(Vec), + /// Delete rows by their global positions (uses positional deletes). + /// Positions are global indices across all data files in order of their creation. + /// Example: `Delete(vec![0, 1])` deletes the first and second rows from the table + Delete(Vec), } /// Tracks the state of data files across snapshots @@ -402,20 +403,26 @@ impl IncrementalTestFixture { manifest_list_write.close().await.unwrap(); } - Operation::Delete(n_values_to_delete) => { - // Find positions to delete from each data file + Operation::Delete(positions_to_delete) => { + // Map global positions to file-specific positions let mut deletes_by_file: HashMap> = HashMap::new(); - for n_to_delete in n_values_to_delete { + for global_pos in positions_to_delete { + // Find which data file contains this global position + let mut file_offset = 0i64; for data_file in &data_files { - if let Some(pos) = - data_file.n_values.iter().position(|n| n == n_to_delete) + let file_size = data_file.n_values.len() as i64; + if global_pos >= &file_offset && global_pos < &(file_offset + file_size) { + // This position belongs to this file + let local_pos = global_pos - file_offset; deletes_by_file .entry(data_file.path.clone()) .or_default() - .push(pos as i64); + .push(local_pos); + break; } + file_offset += file_size; } } @@ -648,7 +655,7 @@ impl IncrementalTestFixture { from_snapshot_id: i64, to_snapshot_id: i64, expected_appends: Vec<(i32, &str)>, - expected_deletes: Vec<(i32, &str)>, + expected_deletes: Vec, ) { use arrow_array::cast::AsArray; use arrow_select::concat::concat_batches; @@ -706,21 +713,13 @@ impl IncrementalTestFixture { let delete_batch = concat_batches(&delete_batches[0].schema(), delete_batches.iter()).unwrap(); - let n_array = delete_batch + let pos_array = delete_batch .column(0) - .as_primitive::(); - let data_array = delete_batch.column(1).as_string::(); + .as_primitive::(); - let mut deleted_pairs: Vec<(i32, String)> = (0..delete_batch.num_rows()) - .map(|i| (n_array.value(i), data_array.value(i).to_string())) - .collect(); + let mut deleted_pairs: Vec = pos_array.iter().filter_map(|v| v).collect(); deleted_pairs.sort(); - let expected_deletes: Vec<(i32, String)> = expected_deletes - .into_iter() - .map(|(n, s)| (n, s.to_string())) - .collect(); - assert_eq!(deleted_pairs, expected_deletes); } else { assert!(expected_deletes.is_empty(), "Expected deletes but got none"); @@ -737,7 +736,7 @@ async fn test_incremental_fixture_simple() { "2".to_string(), "3".to_string(), ]), - Operation::Delete(vec![2]), + Operation::Delete(vec![1]), // Delete position 1 (n=2, data="2") ]) .await; @@ -756,13 +755,21 @@ async fn test_incremental_fixture_simple() { assert_eq!(snapshots[1].parent_snapshot_id(), Some(1)); assert_eq!(snapshots[2].parent_snapshot_id(), Some(2)); - // Verify incremental scan from snapshot 1 to snapshot 3 + // Verify incremental scan from snapshot 1 to snapshot 3. // Expected appends: snapshot 2 adds [1, 2, 3] // Expected deletes: snapshot 3 deletes [2] // In total we expect appends [1, 3] and deletes [] fixture .verify_incremental_scan(1, 3, vec![(1, "1"), (3, "3")], vec![]) .await; + + // Verify incremental scan from snapshot 2 to snapshot 3. + fixture.verify_incremental_scan(2, 3, vec![], vec![1]).await; + + // Verify incremental scan from snapshot 1 to snapshot 1. + fixture + .verify_incremental_scan(1, 1, vec![], vec![]) + .await; } #[tokio::test] @@ -775,10 +782,10 @@ async fn test_incremental_fixture_complex() { "c".to_string(), "d".to_string(), "e".to_string(), - ]), // Snapshot 2: Add 5 rows - Operation::Delete(vec![2, 4]), // Snapshot 3: Delete rows with n=2,4 - Operation::Add(vec![6, 7], vec!["f".to_string(), "g".to_string()]), // Snapshot 4: Add 2 more rows - Operation::Delete(vec![1, 3, 5, 6, 7]), // Snapshot 5: Delete all remaining rows + ]), // Snapshot 2: Add 5 rows (positions 0-4) + Operation::Delete(vec![1, 3]), // Snapshot 3: Delete positions 1,3 (n=2,4; data=b,d) + Operation::Add(vec![6, 7], vec!["f".to_string(), "g".to_string()]), // Snapshot 4: Add 2 more rows (positions 5-6) + Operation::Delete(vec![0, 2, 4, 5, 6]), // Snapshot 5: Delete positions 0,2,4,5,6 (all remaining rows: n=1,3,5,6,7) ]) .await; @@ -804,7 +811,7 @@ async fn test_incremental_fixture_complex() { 5 ); - // Verify incremental scan from snapshot 1 to snapshot 5 + // Verify incremental scan from snapshot 1 to snapshot 5. // All data has been deleted, so we expect the empty result. fixture.verify_incremental_scan(1, 5, vec![], vec![]).await; } From 416a56d393c8d8f5a6ec8a3fae60fb2f28d1dc79 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 11:46:46 +0100 Subject: [PATCH 27/68] . --- crates/iceberg/src/scan/incremental/tests.rs | 164 ++++++++++++------- 1 file changed, 106 insertions(+), 58 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index f3733d989c..3ad4a5fd4b 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -38,14 +38,15 @@ use crate::table::Table; /// Represents an operation to perform on a snapshot. #[derive(Debug, Clone)] pub enum Operation { - /// Add rows with the given `n` values and `data` values. Example: `Add(vec![1, 2, 3], - /// vec!["a", "b", "c"])` adds three rows with n=1,2,3 and data="a","b","c" - Add(Vec, Vec), - - /// Delete rows by their global positions (uses positional deletes). - /// Positions are global indices across all data files in order of their creation. - /// Example: `Delete(vec![0, 1])` deletes the first and second rows from the table - Delete(Vec), + /// Add rows with the given (n, data) tuples, and write to the specified parquet file name. + /// Example: `Add(vec![(1, "a".to_string()), (2, "b".to_string())], "data-1.parquet".to_string())` + /// adds two rows with n=1,2 and data="a","b" to a file named "data-1.parquet" + Add(Vec<(i32, String)>, String), + + /// Delete rows by their positions within specific parquet files (uses positional deletes). + /// Takes a vector of (position, file_name) tuples specifying which position in which file to delete. + /// Example: `Delete(vec![(0, "data-1.parquet"), (1, "data-1.parquet")])` deletes positions 0 and 1 from data-1.parquet + Delete(Vec<(i64, String)>), } /// Tracks the state of data files across snapshots @@ -62,9 +63,16 @@ struct DataFileInfo { /// # Example /// ``` /// let fixture = IncrementalTestFixture::new(vec![ -/// Operation::Add(vec![], vec![]), // Empty snapshot -/// Operation::Add(vec![1, 2, 3], vec!["1", "2", "3"]), // Add 3 rows -/// Operation::Delete(vec![2]), // Delete row with n=2 +/// Operation::Add(vec![], "empty.parquet".to_string()), // Empty snapshot +/// Operation::Add( +/// vec![ +/// (1, "1".to_string()), +/// (2, "2".to_string()), +/// (3, "3".to_string()), +/// ], +/// "data-1.parquet".to_string(), +/// ), // Add 3 rows +/// Operation::Delete(vec![(1, "data-1.parquet".to_string())]), // Delete position 1 from data-1.parquet /// ]) /// .await; /// ``` @@ -259,7 +267,11 @@ impl IncrementalTestFixture { }; match operation { - Operation::Add(n_values, data_values) => { + Operation::Add(rows, file_name) => { + // Extract n_values and data_values from tuples + let n_values: Vec = rows.iter().map(|(n, _)| *n).collect(); + let data_values: Vec = rows.iter().map(|(_, d)| d.clone()).collect(); + // Create data manifest let mut data_writer = ManifestWriterBuilder::new( self.next_manifest_file(), @@ -299,9 +311,8 @@ impl IncrementalTestFixture { // Add new data if not empty if !n_values.is_empty() { - let data_file_path = - format!("{}/data/data-{}.parquet", &self.table_location, snapshot_id); - self.write_parquet_file(&data_file_path, n_values, data_values) + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + self.write_parquet_file(&data_file_path, &n_values, &data_values) .await; data_writer @@ -330,7 +341,7 @@ impl IncrementalTestFixture { path: data_file_path, snapshot_id, sequence_number, - n_values: n_values.clone(), + n_values, }); } @@ -404,26 +415,15 @@ impl IncrementalTestFixture { } Operation::Delete(positions_to_delete) => { - // Map global positions to file-specific positions + // Group deletes by file let mut deletes_by_file: HashMap> = HashMap::new(); - for global_pos in positions_to_delete { - // Find which data file contains this global position - let mut file_offset = 0i64; - for data_file in &data_files { - let file_size = data_file.n_values.len() as i64; - if global_pos >= &file_offset && global_pos < &(file_offset + file_size) - { - // This position belongs to this file - let local_pos = global_pos - file_offset; - deletes_by_file - .entry(data_file.path.clone()) - .or_default() - .push(local_pos); - break; - } - file_offset += file_size; - } + for (position, file_name) in positions_to_delete { + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + deletes_by_file + .entry(data_file_path) + .or_default() + .push(*position); } // Create data manifest with existing data files @@ -655,7 +655,7 @@ impl IncrementalTestFixture { from_snapshot_id: i64, to_snapshot_id: i64, expected_appends: Vec<(i32, &str)>, - expected_deletes: Vec, + expected_deletes: Vec<(u64, &str)>, ) { use arrow_array::cast::AsArray; use arrow_select::concat::concat_batches; @@ -717,9 +717,38 @@ impl IncrementalTestFixture { .column(0) .as_primitive::(); - let mut deleted_pairs: Vec = pos_array.iter().filter_map(|v| v).collect(); + // The file path column is a RunArray (Run-End Encoded), so we need to decode it + // RunArray stores repeated values efficiently. To access individual values, we use + // the Array trait which handles the run-length decoding automatically. + use arrow_array::Array; + let file_path_column = delete_batch.column(1); + + let mut deleted_pairs: Vec<(u64, String)> = (0..delete_batch.num_rows()) + .map(|i| { + let pos = pos_array.value(i); + // Use Array::to_data() to get the decoded data, then cast back + let file_path = { + // Get a slice of the run array for this single row + let slice = file_path_column.slice(i, 1); + // Cast the slice to a run array and get its values + let run_arr = slice + .as_any() + .downcast_ref::>() + .unwrap(); + let values = run_arr.values(); + let str_arr = values.as_string::(); + str_arr.value(0).to_string() + }; + (pos, file_path) + }) + .collect(); deleted_pairs.sort(); + let expected_deletes: Vec<(u64, String)> = expected_deletes + .into_iter() + .map(|(pos, file)| (pos, file.to_string())) + .collect(); + assert_eq!(deleted_pairs, expected_deletes); } else { assert!(expected_deletes.is_empty(), "Expected deletes but got none"); @@ -730,13 +759,16 @@ impl IncrementalTestFixture { #[tokio::test] async fn test_incremental_fixture_simple() { let fixture = IncrementalTestFixture::new(vec![ - Operation::Add(vec![], vec![]), - Operation::Add(vec![1, 2, 3], vec![ - "1".to_string(), - "2".to_string(), - "3".to_string(), - ]), - Operation::Delete(vec![1]), // Delete position 1 (n=2, data="2") + Operation::Add(vec![], "empty.parquet".to_string()), + Operation::Add( + vec![ + (1, "1".to_string()), + (2, "2".to_string()), + (3, "3".to_string()), + ], + "data-2.parquet".to_string(), + ), + Operation::Delete(vec![(1, "data-2.parquet".to_string())]), // Delete position 1 (n=2, data="2") ]) .await; @@ -764,28 +796,44 @@ async fn test_incremental_fixture_simple() { .await; // Verify incremental scan from snapshot 2 to snapshot 3. - fixture.verify_incremental_scan(2, 3, vec![], vec![1]).await; - - // Verify incremental scan from snapshot 1 to snapshot 1. + let data_file_path = format!("{}/data/data-2.parquet", fixture.table_location); fixture - .verify_incremental_scan(1, 1, vec![], vec![]) + .verify_incremental_scan(2, 3, vec![], vec![(1, &data_file_path)]) .await; + + // Verify incremental scan from snapshot 1 to snapshot 1. + fixture.verify_incremental_scan(1, 1, vec![], vec![]).await; } #[tokio::test] async fn test_incremental_fixture_complex() { let fixture = IncrementalTestFixture::new(vec![ - Operation::Add(vec![], vec![]), // Snapshot 1: Empty - Operation::Add(vec![1, 2, 3, 4, 5], vec![ - "a".to_string(), - "b".to_string(), - "c".to_string(), - "d".to_string(), - "e".to_string(), - ]), // Snapshot 2: Add 5 rows (positions 0-4) - Operation::Delete(vec![1, 3]), // Snapshot 3: Delete positions 1,3 (n=2,4; data=b,d) - Operation::Add(vec![6, 7], vec!["f".to_string(), "g".to_string()]), // Snapshot 4: Add 2 more rows (positions 5-6) - Operation::Delete(vec![0, 2, 4, 5, 6]), // Snapshot 5: Delete positions 0,2,4,5,6 (all remaining rows: n=1,3,5,6,7) + Operation::Add(vec![], "empty.parquet".to_string()), // Snapshot 1: Empty + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + (4, "d".to_string()), + (5, "e".to_string()), + ], + "data-2.parquet".to_string(), + ), // Snapshot 2: Add 5 rows (positions 0-4) + Operation::Delete(vec![ + (1, "data-2.parquet".to_string()), + (3, "data-2.parquet".to_string()), + ]), // Snapshot 3: Delete positions 1,3 (n=2,4; data=b,d) + Operation::Add( + vec![(6, "f".to_string()), (7, "g".to_string())], + "data-4.parquet".to_string(), + ), // Snapshot 4: Add 2 more rows (positions 5-6) + Operation::Delete(vec![ + (0, "data-2.parquet".to_string()), + (2, "data-2.parquet".to_string()), + (4, "data-2.parquet".to_string()), + (0, "data-4.parquet".to_string()), + (1, "data-4.parquet".to_string()), + ]), // Snapshot 5: Delete positions 0,2,4,5,6 (all remaining rows: n=1,3,5,6,7) ]) .await; From e06c118012007e448414e37f68769d258dc083a6 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 13:17:11 +0100 Subject: [PATCH 28/68] Add test --- crates/iceberg/src/scan/incremental/tests.rs | 26 ++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 3ad4a5fd4b..a2ac5dff35 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -862,4 +862,30 @@ async fn test_incremental_fixture_complex() { // Verify incremental scan from snapshot 1 to snapshot 5. // All data has been deleted, so we expect the empty result. fixture.verify_incremental_scan(1, 5, vec![], vec![]).await; + + // Verify incremental scan from snapshot 2 to snapshot 5. + // Snapshot 2 starts with: (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // Snapshot 5: Deletes positions 0,2,4 from data-2.parquet and 0,1 from data-4.parquet (n=1,3,5,6,7; data=a,c,e,f,g) + // + // The incremental scan computes the NET EFFECT between snapshot 2 and 5: + // - Files added in snapshot 4 were completely deleted in snapshot 5, so NO net appends + // - Net deletes from data-2.parquet: positions 0,1,2,3,4 (all 5 rows deleted across snapshots 3 and 5) + // - Since data-4 was added and deleted between 2 and 5, it doesn't appear in the scan + let data_2_path = format!("{}/data/data-2.parquet", fixture.table_location); + fixture + .verify_incremental_scan( + 2, + 5, + vec![], // No net appends (data-4 was added and fully deleted) + vec![ + (0, data_2_path.as_str()), // All 5 positions from data-2.parquet + (1, data_2_path.as_str()), + (2, data_2_path.as_str()), + (3, data_2_path.as_str()), + (4, data_2_path.as_str()), + ], + ) + .await; } From dc56ff7035b42080468d3c48615b225cd8b0279f Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 13:32:34 +0100 Subject: [PATCH 29/68] Add tests --- crates/iceberg/src/scan/incremental/tests.rs | 54 ++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index a2ac5dff35..076425ca78 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -888,4 +888,58 @@ async fn test_incremental_fixture_complex() { ], ) .await; + + // Verify incremental scan from snapshot 3 to snapshot 5. + // Snapshot 3 state: (1,a), (3,c), (5,e) remain in data-2.parquet at positions 0,2,4 + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // Snapshot 5: Deletes positions 0,2,4 from data-2.parquet (n=1,3,5) and 0,1 from data-4.parquet (n=6,7) + // + // Net effect from snapshot 3 to 5: + // - No net appends (data-4 was added and fully deleted between 3 and 5) + // - Net deletes from data-2.parquet: positions 0,2,4 (the three remaining rows deleted in snapshot 5) + fixture + .verify_incremental_scan( + 3, + 5, + vec![], // No net appends (data-4 was added and fully deleted) + vec![ + (0, data_2_path.as_str()), // Positions 0,2,4 from data-2.parquet + (2, data_2_path.as_str()), // (n=1,3,5; data=a,c,e) + (4, data_2_path.as_str()), + ], + ) + .await; + + // Verify incremental scan from snapshot 1 to snapshot 4. + // Snapshot 1: Empty + // Snapshot 2: Adds (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // + // Net effect from snapshot 1 to 4: + // - Net appends: (1,a), (3,c), (5,e), (6,f), (7,g) - all rows that exist at snapshot 4 + // - No deletes: rows deleted in snapshot 3 were added after snapshot 1, so they don't count as deletes + fixture + .verify_incremental_scan( + 1, + 4, + vec![(1, "a"), (3, "c"), (5, "e"), (6, "f"), (7, "g")], + vec![], // No deletes (deleted rows were added after snapshot 1) + ) + .await; + + // Verify incremental scan from snapshot 2 to snapshot 4. + // Snapshot 2: Has (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // + // Net effect from snapshot 2 to 4: + // - Net appends: (6,f), (7,g) from data-4.parquet + // - Net deletes: positions 1,3 from data-2.parquet (n=2,4; data=b,d) - existed at snapshot 2 and deleted in 3 + fixture + .verify_incremental_scan(2, 4, vec![(6, "f"), (7, "g")], vec![ + (1, data_2_path.as_str()), // Positions 1,3 from data-2.parquet + (3, data_2_path.as_str()), // (n=2,4; data=b,d) + ]) + .await; } From 3b2b6a022b514432d281ad0171ab323cc3be253e Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 13:48:53 +0100 Subject: [PATCH 30/68] Add tests --- crates/iceberg/src/scan/incremental/tests.rs | 142 +++++++++++++++++++ 1 file changed, 142 insertions(+) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 076425ca78..8bd5d881fb 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -943,3 +943,145 @@ async fn test_incremental_fixture_complex() { ]) .await; } + +#[tokio::test] +async fn test_incremental_scan_edge_cases() { + // This test covers several edge cases: + // 1. Multiple data files added in separate snapshots + // 2. Deletes spread across multiple data files + // 3. Partial deletes from multiple files + // 4. Cross-file delete operations in a single snapshot + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add file A with 3 rows + Operation::Add( + vec![(1, "a1".to_string()), (2, "a2".to_string()), (3, "a3".to_string())], + "file-a.parquet".to_string(), + ), + // Snapshot 3: Add file B with 4 rows + Operation::Add( + vec![ + (10, "b1".to_string()), + (20, "b2".to_string()), + (30, "b3".to_string()), + (40, "b4".to_string()), + ], + "file-b.parquet".to_string(), + ), + // Snapshot 4: Partial delete from file A (delete middle row n=2) + Operation::Delete(vec![(1, "file-a.parquet".to_string())]), + // Snapshot 5: Partial delete from file B (delete first and last rows n=10,40) + Operation::Delete(vec![ + (0, "file-b.parquet".to_string()), + (3, "file-b.parquet".to_string()), + ]), + // Snapshot 6: Add file C with 2 rows + Operation::Add( + vec![(100, "c1".to_string()), (200, "c2".to_string())], + "file-c.parquet".to_string(), + ), + // Snapshot 7: Delete from multiple files in one snapshot (cross-file deletes) + Operation::Delete(vec![ + (0, "file-a.parquet".to_string()), // n=1 + (1, "file-b.parquet".to_string()), // n=20 + (0, "file-c.parquet".to_string()), // n=100 + ]), + ]) + .await; + + // Verify we have 7 snapshots + let n_snapshots = fixture.table.metadata().snapshots().count(); + assert_eq!(n_snapshots, 7); + + let file_a_path = format!("{}/data/file-a.parquet", fixture.table_location); + let file_b_path = format!("{}/data/file-b.parquet", fixture.table_location); + + // Test 1: Scan from snapshot 1 to 4 + // Should see: file-a (1,2,3), file-b (10,20,30,40) added, then (2) deleted from file-a + // BUT: The row n=2 was added AFTER snapshot 1, so it won't show as a delete! + // Net: appends (1,3) from file-a (n=2 added then deleted = net zero), (10,20,30,40) from file-b + // No deletes (n=2 was added and deleted between snapshots 1 and 4) + fixture + .verify_incremental_scan( + 1, + 4, + vec![ + (1, "a1"), + (3, "a3"), + (10, "b1"), + (20, "b2"), + (30, "b3"), + (40, "b4"), + ], + vec![], // No deletes - n=2 was added and deleted between snapshots + ) + .await; + + // Test 2: Scan from snapshot 4 to 6 + // Snapshot 4: has file-a (1,3) and file-b (10,20,30,40) + // Snapshot 5: deletes positions 0,3 from file-b (n=10,40) + // Snapshot 6: adds file-c (100,200) + // Net: appends (100,200) from file-c; deletes pos 0,3 from file-b + fixture + .verify_incremental_scan( + 4, + 6, + vec![(100, "c1"), (200, "c2")], + vec![ + (0, file_b_path.as_str()), // n=10 + (3, file_b_path.as_str()), // n=40 + ], + ) + .await; + + // Test 3: Scan from snapshot 2 to 7 + // This tests the full lifecycle: multiple adds, partial deletes, more adds, cross-file deletes + // Starting at snapshot 2: file-a (1,2,3) exists + // File-b is added in snapshot 3 (after snapshot 2) + // File-c is added in snapshot 6 (after snapshot 2) + // By snapshot 7: file-a has (3) at position 2, file-b has (30), file-c has (200) + // + // Net appends: file-b (30) and file-c (200) were added after snapshot 2 + // Net deletes: positions 0,1 from file-a (n=1,2) existed at snapshot 2 and were deleted + // Note: (3) from file-a already existed at snapshot 2, so it's not a net append! + fixture + .verify_incremental_scan( + 2, + 7, + vec![(30, "b3"), (200, "c2")], + vec![ + (0, file_a_path.as_str()), // n=1 + (1, file_a_path.as_str()), // n=2 + ], + ) + .await; + + // Test 4: Scan from snapshot 5 to 6 + // Simple test: just adding a new file + // Snapshot 5 state: file-a (1,3), file-b (20,30) + // Snapshot 6: adds file-c (100,200) + // Net: appends (100,200) from file-c, no deletes + fixture + .verify_incremental_scan( + 5, + 6, + vec![(100, "c1"), (200, "c2")], + vec![], + ) + .await; + + // Test 5: Scan from snapshot 3 to 4 + // Tests a single delete operation + // State at 3: file-a (1,2,3), file-b (10,20,30,40) + // State at 4: file-a (1,3), file-b (10,20,30,40) + // Net: no appends, 1 delete (position 1, n=2) from file-a + fixture + .verify_incremental_scan( + 3, + 4, + vec![], + vec![(1, file_a_path.as_str())], // n=2 + ) + .await; +} From af28705df241d66ff2dd559c9b623cbd1988e496 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 15:10:52 +0100 Subject: [PATCH 31/68] Format --- crates/iceberg/src/scan/incremental/tests.rs | 39 +++++++------------- 1 file changed, 14 insertions(+), 25 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 8bd5d881fb..fe4b26b88c 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -956,7 +956,11 @@ async fn test_incremental_scan_edge_cases() { Operation::Add(vec![], "empty.parquet".to_string()), // Snapshot 2: Add file A with 3 rows Operation::Add( - vec![(1, "a1".to_string()), (2, "a2".to_string()), (3, "a3".to_string())], + vec![ + (1, "a1".to_string()), + (2, "a2".to_string()), + (3, "a3".to_string()), + ], "file-a.parquet".to_string(), ), // Snapshot 3: Add file B with 4 rows @@ -1024,15 +1028,10 @@ async fn test_incremental_scan_edge_cases() { // Snapshot 6: adds file-c (100,200) // Net: appends (100,200) from file-c; deletes pos 0,3 from file-b fixture - .verify_incremental_scan( - 4, - 6, - vec![(100, "c1"), (200, "c2")], - vec![ - (0, file_b_path.as_str()), // n=10 - (3, file_b_path.as_str()), // n=40 - ], - ) + .verify_incremental_scan(4, 6, vec![(100, "c1"), (200, "c2")], vec![ + (0, file_b_path.as_str()), // n=10 + (3, file_b_path.as_str()), // n=40 + ]) .await; // Test 3: Scan from snapshot 2 to 7 @@ -1046,15 +1045,10 @@ async fn test_incremental_scan_edge_cases() { // Net deletes: positions 0,1 from file-a (n=1,2) existed at snapshot 2 and were deleted // Note: (3) from file-a already existed at snapshot 2, so it's not a net append! fixture - .verify_incremental_scan( - 2, - 7, - vec![(30, "b3"), (200, "c2")], - vec![ - (0, file_a_path.as_str()), // n=1 - (1, file_a_path.as_str()), // n=2 - ], - ) + .verify_incremental_scan(2, 7, vec![(30, "b3"), (200, "c2")], vec![ + (0, file_a_path.as_str()), // n=1 + (1, file_a_path.as_str()), // n=2 + ]) .await; // Test 4: Scan from snapshot 5 to 6 @@ -1063,12 +1057,7 @@ async fn test_incremental_scan_edge_cases() { // Snapshot 6: adds file-c (100,200) // Net: appends (100,200) from file-c, no deletes fixture - .verify_incremental_scan( - 5, - 6, - vec![(100, "c1"), (200, "c2")], - vec![], - ) + .verify_incremental_scan(5, 6, vec![(100, "c1"), (200, "c2")], vec![]) .await; // Test 5: Scan from snapshot 3 to 4 From feb1cfefe2fab88cfb69d7e9e52d0d84b72cca4c Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:25:50 +0100 Subject: [PATCH 32/68] Add test --- crates/iceberg/src/scan/incremental/tests.rs | 255 ++++++++++++++++++- 1 file changed, 254 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index fe4b26b88c..ef6e3b382b 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -35,7 +35,8 @@ use crate::spec::{ }; use crate::table::Table; -/// Represents an operation to perform on a snapshot. +/// Represents an operation to perform on a snapshot of a table with schema (id: Int32, +/// data: String). #[derive(Debug, Clone)] pub enum Operation { /// Add rows with the given (n, data) tuples, and write to the specified parquet file name. @@ -1074,3 +1075,255 @@ async fn test_incremental_scan_edge_cases() { ) .await; } + +#[tokio::test] +async fn test_incremental_scan_builder_options() { + // This test demonstrates using the incremental scan builder API with various options: + // - Column projection (selecting specific columns) + // - Batch size configuration + // - Verifying the schema and batch structure + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add 10 rows to test batch size behavior + Operation::Add( + vec![ + (1, "data-1".to_string()), + (2, "data-2".to_string()), + (3, "data-3".to_string()), + (4, "data-4".to_string()), + (5, "data-5".to_string()), + (6, "data-6".to_string()), + (7, "data-7".to_string()), + (8, "data-8".to_string()), + (9, "data-9".to_string()), + (10, "data-10".to_string()), + ], + "data-2.parquet".to_string(), + ), + // Snapshot 3: Delete some rows + Operation::Delete(vec![ + (2, "data-2.parquet".to_string()), // n=3 + (5, "data-2.parquet".to_string()), // n=6 + (8, "data-2.parquet".to_string()), // n=9 + ]), + // Snapshot 4: Add more rows + Operation::Add( + vec![ + (20, "data-20".to_string()), + (21, "data-21".to_string()), + (22, "data-22".to_string()), + (23, "data-23".to_string()), + (24, "data-24".to_string()), + ], + "data-4.parquet".to_string(), + ), + ]) + .await; + + use arrow_array::cast::AsArray; + use futures::TryStreamExt; + + // Test 1: Column projection - select only the "n" column + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Verify we have both append and delete batches + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + assert!(!append_batches.is_empty(), "Should have append batches"); + + // Check schema - should only have "n" column + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 1, + "Should have only 1 column when projecting 'n'" + ); + assert_eq!( + batch.schema().field(0).name(), + "n", + "Projected column should be 'n'" + ); + } + + // Test 2: Column projection - select only the "data" column + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["data"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 1, + "Should have only 1 column when projecting 'data'" + ); + assert_eq!( + batch.schema().field(0).name(), + "data", + "Projected column should be 'data'" + ); + } + + // Test 3: Select both columns explicitly + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n", "data"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 2, + "Should have 2 columns when projecting both" + ); + assert_eq!(batch.schema().field(0).name(), "n"); + assert_eq!(batch.schema().field(1).name(), "data"); + } + + // Test 4: Batch size configuration + let scan = fixture + .table + .incremental_scan(1, 2) + .with_batch_size(Some(3)) // Small batch size to test batching + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in append_batches.iter() { + // Each batch should have at most 3 rows (except possibly the last) + assert!( + batch.num_rows() <= 3, + "Batch size should be <= 3 as configured" + ); + } + + // Test 5: Combining column projection and batch size + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n"]) + .with_batch_size(Some(4)) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in append_batches.iter() { + assert_eq!(batch.schema().fields().len(), 1, "Should project only 'n'"); + assert!(batch.num_rows() <= 4, "Batch size should be <= 4"); + } + + // Test 6: Verify actual data with column projection + let scan = fixture + .table + .incremental_scan(1, 2) + .select(vec!["n"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + if !append_batches.is_empty() { + use arrow_select::concat::concat_batches; + let combined_batch = + concat_batches(&append_batches[0].schema(), append_batches.iter()).unwrap(); + + let n_array = combined_batch + .column(0) + .as_primitive::(); + + let mut n_values: Vec = (0..combined_batch.num_rows()) + .map(|i| n_array.value(i)) + .collect(); + n_values.sort(); + + assert_eq!( + n_values, + vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10], + "Should have all 10 n values from snapshot 2" + ); + } + + // Test 7: Delete batches always have the same schema. + let scan = fixture.table.incremental_scan(2, 3).build().unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let delete_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Delete) + .map(|(_, b)| b.clone()) + .collect(); + + if !delete_batches.is_empty() { + for batch in &delete_batches { + // Delete batches should have "pos" and "_file" columns + assert!( + batch.schema().fields().len() == 2, + "Delete batch should have exactly position and file columns" + ); + assert_eq!( + batch.num_rows(), + 3, + "Should have 3 deleted positions from snapshot 3" + ); + } + } +} From dd1eec8d7d759f64f2f6d66e22b266ca9e3c7fa9 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:27:40 +0100 Subject: [PATCH 33/68] Format --- crates/iceberg/src/scan/incremental/tests.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index ef6e3b382b..26369949cc 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -20,7 +20,9 @@ use std::fs; use std::fs::File; use std::sync::Arc; +use arrow_array::cast::AsArray; use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; use parquet::arrow::{ArrowWriter, PARQUET_FIELD_ID_META_KEY}; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; @@ -1120,9 +1122,6 @@ async fn test_incremental_scan_builder_options() { ]) .await; - use arrow_array::cast::AsArray; - use futures::TryStreamExt; - // Test 1: Column projection - select only the "n" column let scan = fixture .table From ba5d3b14a41b56394b837e44f6b439e21272eab7 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:29:35 +0100 Subject: [PATCH 34/68] . --- crates/iceberg/src/scan/incremental/tests.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 26369949cc..ef1e91ea23 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -1224,13 +1224,12 @@ async fn test_incremental_scan_builder_options() { let stream = scan.to_arrow().await.unwrap(); let batches: Vec<_> = stream.try_collect().await.unwrap(); - let append_batches: Vec<_> = batches + let append_batches = batches .iter() .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) - .map(|(_, b)| b.clone()) - .collect(); + .map(|(_, b)| b.clone()); - for batch in append_batches.iter() { + for batch in append_batches { // Each batch should have at most 3 rows (except possibly the last) assert!( batch.num_rows() <= 3, @@ -1250,13 +1249,12 @@ async fn test_incremental_scan_builder_options() { let stream = scan.to_arrow().await.unwrap(); let batches: Vec<_> = stream.try_collect().await.unwrap(); - let append_batches: Vec<_> = batches + let append_batches = batches .iter() .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) - .map(|(_, b)| b.clone()) - .collect(); + .map(|(_, b)| b.clone()); - for batch in append_batches.iter() { + for batch in append_batches { assert_eq!(batch.schema().fields().len(), 1, "Should project only 'n'"); assert!(batch.num_rows() <= 4, "Batch size should be <= 4"); } From d8e6bc9bc341a86b79d0d2b2c486e1e770dc7873 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:31:04 +0100 Subject: [PATCH 35/68] Rm newline --- crates/iceberg/src/delete_file_index.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index 60acef31a1..3a3dfbb529 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -74,7 +74,6 @@ impl DeleteFileIndex { let mut guard = state.write().unwrap(); *guard = DeleteFileIndexState::Populated(populated_delete_file_index); } - notify.notify_waiters(); } }); From 32cf0605ec44a92361253c99ca94a305f80e5050 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:36:25 +0100 Subject: [PATCH 36/68] Rename trait function --- crates/iceberg/src/arrow/incremental.rs | 6 +++--- crates/iceberg/src/arrow/reader.rs | 4 ++-- crates/iceberg/src/scan/incremental/mod.rs | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 5e7d954e88..c18167a722 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -56,9 +56,9 @@ impl StreamsInto { /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a /// stream of Arrow `RecordBatch`es containing the data from the files. - fn read(self, reader: ArrowReader) -> Result { + fn stream(self, reader: ArrowReader) -> Result { let (appends, deletes) = - StreamsInto::::read(self, reader)?; + StreamsInto::::stream(self, reader)?; let left = appends.map(|res| res.map(|batch| (IncrementalBatchType::Append, batch))); let right = deletes.map(|res| res.map(|batch| (IncrementalBatchType::Delete, batch))); @@ -72,7 +72,7 @@ impl StreamsInto { /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. - fn read(self, reader: ArrowReader) -> Result { + fn stream(self, reader: ArrowReader) -> Result { let (appends_tx, appends_rx) = channel(reader.concurrency_limit_data_files); let (deletes_tx, deletes_rx) = channel(reader.concurrency_limit_data_files); diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 297b6fd4d8..7f07891e83 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -144,8 +144,8 @@ pub struct ArrowReader { /// Trait indicating that the implementing type streams into a stream of type `S` using /// a reader of type `R`. pub trait StreamsInto { - /// Read from the reader and produce a stream of type `S`. - fn read(self, reader: R) -> Result; + /// Stream from the reader and produce a stream of type `S`. + fn stream(self, reader: R) -> Result; } impl ArrowReader { diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 30d8a3dd31..76a3916980 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -469,7 +469,7 @@ impl IncrementalTableScan { } let arrow_reader = arrow_reader_builder.build(); - file_scan_task_stream.read(arrow_reader) + file_scan_task_stream.stream(arrow_reader) } /// Returns an [`UnzippedIncrementalBatchRecordStream`] for this incremental table scan. @@ -486,7 +486,7 @@ impl IncrementalTableScan { } let arrow_reader = arrow_reader_builder.build(); - file_scan_task_stream.read(arrow_reader) + file_scan_task_stream.stream(arrow_reader) } async fn process_delete_manifest_entry( From 3b57c884011ef8ce38e089d2266335df00a31c80 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:39:30 +0100 Subject: [PATCH 37/68] Reuse schema --- crates/iceberg/src/arrow/incremental.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index c18167a722..4eaf89dab9 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -235,16 +235,20 @@ fn process_incremental_delete_task( delete_vector: RoaringTreemap, batch_size: Option, ) -> Result { + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "pos", + DataType::UInt64, + false, + )])); + + let batch_size = batch_size.unwrap_or(1024); + let stream = futures::stream::iter(delete_vector) - .chunks(batch_size.unwrap_or(1024)) + .chunks(batch_size) .map(move |chunk| { let array = UInt64Array::from_iter(chunk); RecordBatch::try_new( - Arc::new(ArrowSchema::new(vec![Field::new( - "pos", - DataType::UInt64, - false, - )])), + Arc::clone(&schema), // Cheap Arc clone instead of full schema creation vec![Arc::new(array)], ) .map_err(|_| { From 07a43942c215e0d3f9555cd599f37df4faa0f6ab Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:40:51 +0100 Subject: [PATCH 38/68] . --- crates/iceberg/src/arrow/incremental.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 4eaf89dab9..5d03de88ee 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -78,12 +78,11 @@ impl StreamsInto let batch_size = reader.batch_size; let concurrency_limit_data_files = reader.concurrency_limit_data_files; - let file_io = reader.file_io.clone(); spawn(async move { let _ = self .try_for_each_concurrent(concurrency_limit_data_files, |task| { - let file_io = file_io.clone(); + let file_io = reader.file_io.clone(); let mut appends_tx = appends_tx.clone(); let mut deletes_tx = deletes_tx.clone(); async move { From 87587d6d5d34383bc69f521682aab4f66418bd88 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Mon, 27 Oct 2025 16:44:02 +0100 Subject: [PATCH 39/68] remove clone --- crates/iceberg/src/arrow/incremental.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 5d03de88ee..87deaee5cf 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -177,7 +177,7 @@ async fn process_incremental_append_task( ) -> Result { let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( &task.data_file_path, - file_io.clone(), + file_io, true, ) .await?; From 1e66b4b0ae88f8ddd8c211eb9dc28b8435091fcb Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 08:56:07 +0100 Subject: [PATCH 40/68] Add test for adding file_path column --- crates/iceberg/src/arrow/reader.rs | 261 +++++++++++++++++++++++++---- 1 file changed, 229 insertions(+), 32 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 7f07891e83..3f90421cf5 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -481,40 +481,57 @@ impl ArrowReader { let num_rows = batch.num_rows(); - // Use Run-End Encoded array for optimal memory efficiency - // For a constant value repeated num_rows times, this stores: - // - run_ends: [num_rows] (one i32) - // - values: [file_path] (one string) - let run_ends = Int32Array::from(vec![num_rows as i32]); - let values = StringArray::from(vec![file_path]); - // TODO @vustef L0: These may not be supported in Julia's Arrow.jl, see what alternatives we have... - let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to create RunArray for _file column", - ) - .with_source(e) - })?; - let mut columns = batch.columns().to_vec(); - columns.push(Arc::new(file_array) as ArrayRef); - let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); - // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE - // DataType is RunEndEncoded with Int32 run ends and Utf8 values - // Note: values field is nullable to match what StringArray::from() creates // TODO @vustef: Not sure why is that the case, fix it. - let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); - let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); - let file_field = Field::new( - RESERVED_COL_NAME_FILE, - DataType::RunEndEncoded(run_ends_field, values_field), - false, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_FILE.to_string(), - )])); - fields.push(Arc::new(file_field)); + + // Handle empty batches separately - use regular StringArray instead of RunEndEncoded + // since RunArray requires run_ends to be > 0 + if num_rows == 0 { + // Create an empty StringArray for the _file column + let file_array = StringArray::from(Vec::<&str>::new()); + columns.push(Arc::new(file_array) as ArrayRef); + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_FILE.to_string(), + )])); + fields.push(Arc::new(file_field)); + } else { + // Use Run-End Encoded array for optimal memory efficiency + // For a constant value repeated num_rows times, this stores: + // - run_ends: [num_rows] (one i32) + // - values: [file_path] (one string) + let run_ends = Int32Array::from(vec![num_rows as i32]); + let values = StringArray::from(vec![file_path]); + // TODO @vustef L0: These may not be supported in Julia's Arrow.jl, see what alternatives we have... + let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RunArray for _file column", + ) + .with_source(e) + })?; + + columns.push(Arc::new(file_array) as ArrayRef); + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + // DataType is RunEndEncoded with Int32 run ends and Utf8 values + // Note: values field is nullable to match what StringArray::from() creates // TODO @vustef: Not sure why is that the case, fix it. + let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); + let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); + let file_field = Field::new( + RESERVED_COL_NAME_FILE, + DataType::RunEndEncoded(run_ends_field, values_field), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_FILE.to_string(), + )])); + fields.push(Arc::new(file_field)); + } let schema = Arc::new(ArrowSchema::new(fields)); RecordBatch::try_new(schema, columns).map_err(|e| { @@ -2110,4 +2127,184 @@ message schema { assert!(col_b.is_null(1)); assert!(col_b.is_null(2)); } + + #[test] + fn test_add_file_path_column() { + use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; + + // Create a simple test batch with 2 columns and 3 rows + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let id_array = Int32Array::from(vec![1, 2, 3]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Charlie"]); + + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(id_array), + Arc::new(name_array), + ]) + .unwrap(); + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Add file path column + let file_path = "/path/to/data/file.parquet"; + let result = ArrowReader::add_file_path_column(batch, file_path); + assert!(result.is_ok(), "Should successfully add file path column"); + + let new_batch = result.unwrap(); + + // Verify the new batch has 3 columns + assert_eq!(new_batch.num_columns(), 3); + assert_eq!(new_batch.num_rows(), 3); + + // Verify schema has the _file column + let schema = new_batch.schema(); + assert_eq!(schema.fields().len(), 3); + + let file_field = schema.field(2); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + assert!(!file_field.is_nullable()); + + // Verify the field has the correct metadata + let metadata = file_field.metadata(); + assert_eq!( + metadata.get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the data type is RunEndEncoded + match file_field.data_type() { + DataType::RunEndEncoded(run_ends_field, values_field) => { + assert_eq!(run_ends_field.name(), "run_ends"); + assert_eq!(run_ends_field.data_type(), &DataType::Int32); + assert!(!run_ends_field.is_nullable()); + + assert_eq!(values_field.name(), "values"); + assert_eq!(values_field.data_type(), &DataType::Utf8); + } + _ => panic!("Expected RunEndEncoded data type for _file column"), + } + + // Verify the original columns are intact + let id_col = new_batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.values(), &[1, 2, 3]); + + let name_col = new_batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + + // Verify the file path column contains the correct value + // The _file column is a RunArray, so we need to decode it + let file_col = new_batch.column(2); + let run_array = file_col + .as_any() + .downcast_ref::>() + .expect("Expected RunArray for _file column"); + + // Check that all rows have the same file path + for i in 0..new_batch.num_rows() { + let slice = file_col.slice(i, 1); + let run_arr = slice + .as_any() + .downcast_ref::>() + .unwrap(); + let values = run_arr.values(); + let str_arr = values.as_string::(); + assert_eq!(str_arr.value(0), file_path); + } + + // Verify the run array structure (should be optimally encoded) + let run_ends = run_array.run_ends(); + assert_eq!(run_ends.values().len(), 1, "Should have only 1 run end"); + assert_eq!(run_ends.values()[0], 3, "Run end should be at position 3"); + + let values = run_array.values(); + let string_values = values.as_string::(); + assert_eq!(string_values.len(), 1, "Should have only 1 value"); + assert_eq!(string_values.value(0), file_path); + } + + #[test] + fn test_add_file_path_column_empty_batch() { + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; + + // Create an empty batch + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = arrow_array::Int32Array::from(Vec::::new()); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + assert_eq!(batch.num_rows(), 0); + + // Add file path column to empty batch + let file_path = "/empty/file.parquet"; + let result = ArrowReader::add_file_path_column(batch, file_path); + + // Should succeed with StringArray for empty batches + assert!(result.is_ok()); + let new_batch = result.unwrap(); + assert_eq!(new_batch.num_rows(), 0); + assert_eq!(new_batch.num_columns(), 2); + + // Verify the _file column exists with correct schema + let schema = new_batch.schema(); + let file_field = schema.field(1); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + + // For empty batches, should use StringArray (Utf8), not RunEndEncoded + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify metadata with reserved field ID + assert_eq!( + file_field.metadata().get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the file path column is empty but properly structured + let file_path_column = new_batch.column(1); + assert_eq!(file_path_column.len(), 0); + } + + #[test] + fn test_add_file_path_column_special_characters() { + use arrow_array::{Int32Array, RecordBatch}; + use arrow_schema::{DataType, Field, Schema}; + + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = Int32Array::from(vec![42]); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + // Test with file path containing special characters + let file_path = "/path/with spaces/and-dashes/file_name.parquet"; + let result = ArrowReader::add_file_path_column(batch, file_path); + assert!(result.is_ok()); + + let new_batch = result.unwrap(); + let file_col = new_batch.column(1); + + // Verify the file path is correctly stored + let slice = file_col.slice(0, 1); + let run_arr = slice + .as_any() + .downcast_ref::>() + .unwrap(); + let values = run_arr.values(); + let str_arr = values.as_string::(); + assert_eq!(str_arr.value(0), file_path); + } } From b77f37f8c2ae0f1fb22abfa608a90b70d1e3c5b1 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 09:18:42 +0100 Subject: [PATCH 41/68] Make `from_snapshot` mandatory --- .../iceberg/src/scan/incremental/context.rs | 2 +- crates/iceberg/src/scan/incremental/mod.rs | 40 +++++++++---------- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index 4b6a2b3d62..23ca62ad19 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -37,7 +37,7 @@ pub(crate) struct IncrementalPlanContext { pub snapshots: Vec, /// The snapshot to start the incremental scan from. - pub from_snapshot: Option, + pub from_snapshot: SnapshotRef, /// The metadata of the table being scanned. pub table_metadata: TableMetadataRef, diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 76a3916980..71bb9018cf 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -53,7 +53,7 @@ pub struct IncrementalTableScanBuilder<'a> { table: &'a Table, // Defaults to `None`, which means all columns. column_names: Option>, - from_snapshot_id: Option, + from_snapshot_id: i64, to_snapshot_id: i64, batch_size: Option, concurrency_limit_data_files: usize, @@ -67,7 +67,7 @@ impl<'a> IncrementalTableScanBuilder<'a> { Self { table, column_names: None, - from_snapshot_id: Some(from_snapshot_id), + from_snapshot_id, to_snapshot_id, batch_size: None, concurrency_limit_data_files: num_cpus, @@ -107,7 +107,7 @@ impl<'a> IncrementalTableScanBuilder<'a> { /// Set the `from_snapshot_id` for the incremental scan. pub fn from_snapshot_id(mut self, from_snapshot_id: i64) -> Self { - self.from_snapshot_id = Some(from_snapshot_id); + self.from_snapshot_id = from_snapshot_id; self } @@ -137,21 +137,17 @@ impl<'a> IncrementalTableScanBuilder<'a> { /// Build the incremental table scan. pub fn build(self) -> Result { - let snapshot_from: Option> = match self.from_snapshot_id { - Some(id) => Some( - self.table - .metadata() - .snapshot_by_id(id) - .ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - format!("Snapshot with id {} not found", id), - ) - })? - .clone(), - ), - None => None, - }; + let snapshot_from: Arc = self + .table + .metadata() + .snapshot_by_id(self.from_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", self.from_snapshot_id), + ) + })? + .clone(); let snapshot_to: Arc = self .table @@ -170,7 +166,7 @@ impl<'a> IncrementalTableScanBuilder<'a> { let snapshots = ancestors_between( &self.table.metadata_ref(), snapshot_to.snapshot_id(), - snapshot_from.as_ref().map(|s| s.snapshot_id()), + Some(snapshot_from.snapshot_id()), ) .collect_vec(); @@ -273,9 +269,9 @@ pub struct IncrementalTableScan { } impl IncrementalTableScan { - /// Returns the optional `from` snapshot of this incremental table scan. - pub fn snapshot_from(&self) -> Option<&SnapshotRef> { - self.plan_context.from_snapshot.as_ref() + /// Returns the `from` snapshot of this incremental table scan. + pub fn snapshot_from(&self) -> &SnapshotRef { + &self.plan_context.from_snapshot } /// Returns the snapshots involved in this incremental table scan. From 2204b5c264d232631abadb63519232951e004220 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 10:04:48 +0100 Subject: [PATCH 42/68] Error out if incremental scan encounters neither Append nor Delete --- .../iceberg/src/scan/incremental/context.rs | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index 23ca62ad19..dfecbf803f 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -65,16 +65,25 @@ impl IncrementalPlanContext { delete_file_idx: DeleteFileIndex, delete_file_tx: Sender, ) -> Result> + 'static>> { + // Validate that all snapshots are Append or Delete operations + for snapshot in self.snapshots.iter() { + let operation = &snapshot.summary().operation; + if !matches!(operation, Operation::Append | Operation::Delete) { + return Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + format!( + "Incremental scan only supports Append and Delete operations, but snapshot {} has operation {:?}", + snapshot.snapshot_id(), + operation + ), + )); + } + } + let (manifest_files, filter_fn) = { let snapshot_ids: HashSet = self .snapshots .iter() - .filter(|snapshot| { - matches!( - snapshot.summary().operation, - Operation::Append | Operation::Delete - ) - }) .map(|snapshot| snapshot.snapshot_id()) .collect(); From a281d7f5859c15bfe682caefd6c7c406e2d079c0 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 10:06:03 +0100 Subject: [PATCH 43/68] . --- .../iceberg/src/scan/incremental/context.rs | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index dfecbf803f..e92341eac4 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -65,28 +65,27 @@ impl IncrementalPlanContext { delete_file_idx: DeleteFileIndex, delete_file_tx: Sender, ) -> Result> + 'static>> { - // Validate that all snapshots are Append or Delete operations - for snapshot in self.snapshots.iter() { - let operation = &snapshot.summary().operation; - if !matches!(operation, Operation::Append | Operation::Delete) { - return Err(crate::Error::new( - crate::ErrorKind::FeatureUnsupported, - format!( - "Incremental scan only supports Append and Delete operations, but snapshot {} has operation {:?}", - snapshot.snapshot_id(), - operation - ), - )); + // Validate that all snapshots are Append or Delete operations and collect their IDs + let snapshot_ids: HashSet = { + let mut ids = HashSet::new(); + for snapshot in self.snapshots.iter() { + let operation = &snapshot.summary().operation; + if !matches!(operation, Operation::Append | Operation::Delete) { + return Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + format!( + "Incremental scan only supports Append and Delete operations, but snapshot {} has operation {:?}", + snapshot.snapshot_id(), + operation + ), + )); + } + ids.insert(snapshot.snapshot_id()); } - } + ids + }; let (manifest_files, filter_fn) = { - let snapshot_ids: HashSet = self - .snapshots - .iter() - .map(|snapshot| snapshot.snapshot_id()) - .collect(); - let mut manifest_files = HashSet::::new(); for snapshot in self.snapshots.iter() { let manifest_list = self From 72402490cf7dcde6e7e36cd3abbcf47187f17f0f Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 11:27:34 +0100 Subject: [PATCH 44/68] Add materialized variant of add_file_path_column --- crates/iceberg/src/arrow/reader.rs | 274 ++++++++++++++++++++++------- 1 file changed, 211 insertions(+), 63 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 3f90421cf5..0ad303cf69 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -473,7 +473,10 @@ impl ArrowReader { /// Adds a `_file` column to the RecordBatch containing the file path. /// Uses Run-End Encoding (RLE) for maximum memory efficiency when the same /// file path is repeated across all rows. - pub(crate) fn add_file_path_column(batch: RecordBatch, file_path: &str) -> Result { + pub(crate) fn add_file_path_column_rle( + batch: RecordBatch, + file_path: &str, + ) -> Result { use std::collections::HashMap; use arrow_array::{Int32Array, RunArray, StringArray}; @@ -484,54 +487,82 @@ impl ArrowReader { let mut columns = batch.columns().to_vec(); let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); - // Handle empty batches separately - use regular StringArray instead of RunEndEncoded - // since RunArray requires run_ends to be > 0 - if num_rows == 0 { - // Create an empty StringArray for the _file column - let file_array = StringArray::from(Vec::<&str>::new()); - columns.push(Arc::new(file_array) as ArrayRef); - - // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE - let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_FILE.to_string(), - )])); - fields.push(Arc::new(file_field)); + // Use Run-End Encoded array for optimal memory efficiency + // For a constant value repeated num_rows times, this stores: + // - run_ends: [num_rows] (one i32) for non-empty batches, or [] for empty batches + // - values: [file_path] (one string) for non-empty batches, or [] for empty batches + let run_ends = if num_rows == 0 { + Int32Array::from(Vec::::new()) } else { - // Use Run-End Encoded array for optimal memory efficiency - // For a constant value repeated num_rows times, this stores: - // - run_ends: [num_rows] (one i32) - // - values: [file_path] (one string) - let run_ends = Int32Array::from(vec![num_rows as i32]); - let values = StringArray::from(vec![file_path]); - // TODO @vustef L0: These may not be supported in Julia's Arrow.jl, see what alternatives we have... - let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to create RunArray for _file column", - ) - .with_source(e) - })?; - - columns.push(Arc::new(file_array) as ArrayRef); - - // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE - // DataType is RunEndEncoded with Int32 run ends and Utf8 values - // Note: values field is nullable to match what StringArray::from() creates // TODO @vustef: Not sure why is that the case, fix it. - let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); - let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); - let file_field = Field::new( - RESERVED_COL_NAME_FILE, - DataType::RunEndEncoded(run_ends_field, values_field), - false, + Int32Array::from(vec![num_rows as i32]) + }; + let values = if num_rows == 0 { + StringArray::from(Vec::<&str>::new()) + } else { + StringArray::from(vec![file_path]) + }; + // TODO @vustef L0: These may not be supported in Julia's Arrow.jl, see what alternatives we have... + let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RunArray for _file column", + ) + .with_source(e) + })?; + + columns.push(Arc::new(file_array) as ArrayRef); + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + // DataType is RunEndEncoded with Int32 run ends and Utf8 values + // Note: values field is nullable to match what StringArray::from() creates // TODO @vustef: Not sure why is that the case, fix it. + let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); + let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); + let file_field = Field::new( + RESERVED_COL_NAME_FILE, + DataType::RunEndEncoded(run_ends_field, values_field), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_FILE.to_string(), + )])); + fields.push(Arc::new(file_field)); + + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to add _file column to RecordBatch", ) - .with_metadata(HashMap::from([( + .with_source(e) + }) + } + + /// Adds a `_file` column to the RecordBatch containing the file path. + /// Materializes the file path string for each row (no compression). + pub(crate) fn add_file_path_column(batch: RecordBatch, file_path: &str) -> Result { + use std::collections::HashMap; + + use arrow_array::StringArray; + use arrow_schema::Field; + + let num_rows = batch.num_rows(); + + let mut columns = batch.columns().to_vec(); + let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); + + // Create a StringArray with the file path repeated num_rows times + let file_array = StringArray::from(vec![file_path; num_rows]); + columns.push(Arc::new(file_array) as ArrayRef); + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false).with_metadata( + HashMap::from([( PARQUET_FIELD_ID_META_KEY.to_string(), RESERVED_FIELD_ID_FILE.to_string(), - )])); - fields.push(Arc::new(file_field)); - } + )]), + ); + fields.push(Arc::new(file_field)); let schema = Arc::new(ArrowSchema::new(fields)); RecordBatch::try_new(schema, columns).map_err(|e| { @@ -2129,7 +2160,7 @@ message schema { } #[test] - fn test_add_file_path_column() { + fn test_add_file_path_column_rle() { use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; @@ -2153,9 +2184,9 @@ message schema { assert_eq!(batch.num_columns(), 2); assert_eq!(batch.num_rows(), 3); - // Add file path column + // Add file path column with RLE let file_path = "/path/to/data/file.parquet"; - let result = ArrowReader::add_file_path_column(batch, file_path); + let result = ArrowReader::add_file_path_column_rle(batch, file_path); assert!(result.is_ok(), "Should successfully add file path column"); let new_batch = result.unwrap(); @@ -2235,7 +2266,7 @@ message schema { } #[test] - fn test_add_file_path_column_empty_batch() { + fn test_add_file_path_column_rle_empty_batch() { use arrow_array::RecordBatch; use arrow_schema::{DataType, Field, Schema}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; @@ -2250,11 +2281,11 @@ message schema { assert_eq!(batch.num_rows(), 0); - // Add file path column to empty batch + // Add file path column to empty batch with RLE let file_path = "/empty/file.parquet"; - let result = ArrowReader::add_file_path_column(batch, file_path); + let result = ArrowReader::add_file_path_column_rle(batch, file_path); - // Should succeed with StringArray for empty batches + // Should succeed with empty RunArray for empty batches assert!(result.is_ok()); let new_batch = result.unwrap(); assert_eq!(new_batch.num_rows(), 0); @@ -2265,8 +2296,14 @@ message schema { let file_field = schema.field(1); assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); - // For empty batches, should use StringArray (Utf8), not RunEndEncoded - assert_eq!(file_field.data_type(), &DataType::Utf8); + // Should use RunEndEncoded even for empty batches + match file_field.data_type() { + DataType::RunEndEncoded(run_ends_field, values_field) => { + assert_eq!(run_ends_field.data_type(), &DataType::Int32); + assert_eq!(values_field.data_type(), &DataType::Utf8); + } + _ => panic!("Expected RunEndEncoded data type for _file column"), + } // Verify metadata with reserved field ID assert_eq!( @@ -2289,7 +2326,7 @@ message schema { let id_array = Int32Array::from(vec![42]); let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); - // Test with file path containing special characters + // Test with file path containing special characters (materialized version) let file_path = "/path/with spaces/and-dashes/file_name.parquet"; let result = ArrowReader::add_file_path_column(batch, file_path); assert!(result.is_ok()); @@ -2297,14 +2334,125 @@ message schema { let new_batch = result.unwrap(); let file_col = new_batch.column(1); - // Verify the file path is correctly stored - let slice = file_col.slice(0, 1); - let run_arr = slice - .as_any() - .downcast_ref::>() - .unwrap(); - let values = run_arr.values(); - let str_arr = values.as_string::(); + // Verify the file path is correctly stored as a materialized StringArray + let str_arr = file_col.as_string::(); assert_eq!(str_arr.value(0), file_path); } + + #[test] + fn test_add_file_path_column() { + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; + + // Create a simple test batch with 2 columns and 3 rows + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let id_array = Int32Array::from(vec![1, 2, 3]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Charlie"]); + + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(id_array), + Arc::new(name_array), + ]) + .unwrap(); + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Add file path column with materialization + let file_path = "/path/to/data/file.parquet"; + let result = ArrowReader::add_file_path_column(batch, file_path); + assert!(result.is_ok(), "Should successfully add file path column"); + + let new_batch = result.unwrap(); + + // Verify the new batch has 3 columns + assert_eq!(new_batch.num_columns(), 3); + assert_eq!(new_batch.num_rows(), 3); + + // Verify schema has the _file column + let schema = new_batch.schema(); + assert_eq!(schema.fields().len(), 3); + + let file_field = schema.field(2); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + assert!(!file_field.is_nullable()); + + // Verify the field has the correct metadata + let metadata = file_field.metadata(); + assert_eq!( + metadata.get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the data type is Utf8 (materialized strings) + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify the original columns are intact + let id_col = new_batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.values(), &[1, 2, 3]); + + let name_col = new_batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + + // Verify the file path column contains the correct value for all rows + let file_col = new_batch.column(2).as_string::(); + for i in 0..new_batch.num_rows() { + assert_eq!(file_col.value(i), file_path); + } + } + + #[test] + fn test_add_file_path_column_empty_batch() { + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; + + // Create an empty batch + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = arrow_array::Int32Array::from(Vec::::new()); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + assert_eq!(batch.num_rows(), 0); + + // Add file path column to empty batch (materialized version) + let file_path = "/empty/file.parquet"; + let result = ArrowReader::add_file_path_column(batch, file_path); + + // Should succeed with empty StringArray + assert!(result.is_ok()); + let new_batch = result.unwrap(); + assert_eq!(new_batch.num_rows(), 0); + assert_eq!(new_batch.num_columns(), 2); + + // Verify the _file column exists with correct schema + let schema = new_batch.schema(); + let file_field = schema.field(1); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + + // Should use Utf8 (materialized strings) + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify metadata with reserved field ID + assert_eq!( + file_field.metadata().get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the file path column is empty but properly structured + let file_path_column = new_batch.column(1); + assert_eq!(file_path_column.len(), 0); + } } From 170e0cff580bdf38428444f633885bef3932902b Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 11:27:58 +0100 Subject: [PATCH 45/68] . --- crates/iceberg/src/scan/incremental/tests.rs | 20 +++----------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index ef1e91ea23..e5bf8bf847 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -720,28 +720,14 @@ impl IncrementalTestFixture { .column(0) .as_primitive::(); - // The file path column is a RunArray (Run-End Encoded), so we need to decode it - // RunArray stores repeated values efficiently. To access individual values, we use - // the Array trait which handles the run-length decoding automatically. - use arrow_array::Array; + // The file path column is a StringArray with materialized values let file_path_column = delete_batch.column(1); + let file_path_array = file_path_column.as_string::(); let mut deleted_pairs: Vec<(u64, String)> = (0..delete_batch.num_rows()) .map(|i| { let pos = pos_array.value(i); - // Use Array::to_data() to get the decoded data, then cast back - let file_path = { - // Get a slice of the run array for this single row - let slice = file_path_column.slice(i, 1); - // Cast the slice to a run array and get its values - let run_arr = slice - .as_any() - .downcast_ref::>() - .unwrap(); - let values = run_arr.values(); - let str_arr = values.as_string::(); - str_arr.value(0).to_string() - }; + let file_path = file_path_array.value(i).to_string(); (pos, file_path) }) .collect(); From c02af9ebe683d0efe7f8d1b4214b1ab3641718c4 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 11:50:06 +0100 Subject: [PATCH 46/68] Allow dead code --- crates/iceberg/src/arrow/reader.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 0ad303cf69..a18ecac3fc 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -473,6 +473,7 @@ impl ArrowReader { /// Adds a `_file` column to the RecordBatch containing the file path. /// Uses Run-End Encoding (RLE) for maximum memory efficiency when the same /// file path is repeated across all rows. + #[allow(dead_code)] pub(crate) fn add_file_path_column_rle( batch: RecordBatch, file_path: &str, From 299e27440f8bde09cba0b37971d9ff720ce98418 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 13:47:01 +0100 Subject: [PATCH 47/68] Some PR comments --- crates/iceberg/src/arrow/reader.rs | 125 +++++++++----------- crates/iceberg/src/delete_vector.rs | 2 +- crates/iceberg/src/scan/incremental/task.rs | 3 +- 3 files changed, 57 insertions(+), 73 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index a18ecac3fc..9bff8c0885 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -23,11 +23,14 @@ use std::str::FromStr; use std::sync::Arc; use arrow_arith::boolean::{and, and_kleene, is_not_null, is_null, not, or, or_kleene}; -use arrow_array::{Array, ArrayRef, BooleanArray, Datum as ArrowDatum, RecordBatch, Scalar}; +use arrow_array::{ + Array, ArrayRef, BooleanArray, Datum as ArrowDatum, Int32Array, RecordBatch, RunArray, Scalar, + StringArray, +}; use arrow_cast::cast::cast; use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq}; use arrow_schema::{ - ArrowError, DataType, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, + ArrowError, DataType, Field, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, }; use arrow_string::like::starts_with; use bytes::Bytes; @@ -470,24 +473,49 @@ impl ArrowReader { Ok(results.into()) } + /// Helper function to add a `_file` column to a RecordBatch. + /// Takes the array and field to add, reducing code duplication. + fn add_file_path_column_impl( + batch: RecordBatch, + file_array: ArrayRef, + file_field: Field, + ) -> Result { + let mut columns = batch.columns().to_vec(); + columns.push(file_array); + + let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); + fields.push(Arc::new(file_field)); + + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to add _file column to RecordBatch", + ) + .with_source(e) + }) + } + + /// Creates the metadata for the `_file` field with the reserved field ID. + fn create_file_field_metadata() -> HashMap { + HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_FILE.to_string(), + )]) + } + /// Adds a `_file` column to the RecordBatch containing the file path. - /// Uses Run-End Encoding (RLE) for maximum memory efficiency when the same + /// Uses Run-End Encoding (REE) for maximum memory efficiency when the same /// file path is repeated across all rows. + /// Note: This is only used in tests for now, for production usage we use the + /// non-REE version as it is Julia-compatible. #[allow(dead_code)] - pub(crate) fn add_file_path_column_rle( + pub(crate) fn add_file_path_column_ree( batch: RecordBatch, file_path: &str, ) -> Result { - use std::collections::HashMap; - - use arrow_array::{Int32Array, RunArray, StringArray}; - use arrow_schema::Field; - let num_rows = batch.num_rows(); - let mut columns = batch.columns().to_vec(); - let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); - // Use Run-End Encoded array for optimal memory efficiency // For a constant value repeated num_rows times, this stores: // - run_ends: [num_rows] (one i32) for non-empty batches, or [] for empty batches @@ -502,7 +530,7 @@ impl ArrowReader { } else { StringArray::from(vec![file_path]) }; - // TODO @vustef L0: These may not be supported in Julia's Arrow.jl, see what alternatives we have... + let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { Error::new( ErrorKind::Unexpected, @@ -511,11 +539,9 @@ impl ArrowReader { .with_source(e) })?; - columns.push(Arc::new(file_array) as ArrayRef); - // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE // DataType is RunEndEncoded with Int32 run ends and Utf8 values - // Note: values field is nullable to match what StringArray::from() creates // TODO @vustef: Not sure why is that the case, fix it. + // Note: values field is nullable to match what StringArray::from() creates let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); let file_field = Field::new( @@ -523,56 +549,24 @@ impl ArrowReader { DataType::RunEndEncoded(run_ends_field, values_field), false, ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_FILE.to_string(), - )])); - fields.push(Arc::new(file_field)); + .with_metadata(Self::create_file_field_metadata()); - let schema = Arc::new(ArrowSchema::new(fields)); - RecordBatch::try_new(schema, columns).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to add _file column to RecordBatch", - ) - .with_source(e) - }) + Self::add_file_path_column_impl(batch, Arc::new(file_array), file_field) } /// Adds a `_file` column to the RecordBatch containing the file path. /// Materializes the file path string for each row (no compression). pub(crate) fn add_file_path_column(batch: RecordBatch, file_path: &str) -> Result { - use std::collections::HashMap; - - use arrow_array::StringArray; - use arrow_schema::Field; - let num_rows = batch.num_rows(); - let mut columns = batch.columns().to_vec(); - let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); - // Create a StringArray with the file path repeated num_rows times let file_array = StringArray::from(vec![file_path; num_rows]); - columns.push(Arc::new(file_array) as ArrayRef); // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE - let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false).with_metadata( - HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_FILE.to_string(), - )]), - ); - fields.push(Arc::new(file_field)); + let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false) + .with_metadata(Self::create_file_field_metadata()); - let schema = Arc::new(ArrowSchema::new(fields)); - RecordBatch::try_new(schema, columns).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to add _file column to RecordBatch", - ) - .with_source(e) - }) + Self::add_file_path_column_impl(batch, Arc::new(file_array), file_field) } fn build_field_id_set_and_map( @@ -2161,7 +2155,7 @@ message schema { } #[test] - fn test_add_file_path_column_rle() { + fn test_add_file_path_column_ree() { use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; @@ -2185,9 +2179,9 @@ message schema { assert_eq!(batch.num_columns(), 2); assert_eq!(batch.num_rows(), 3); - // Add file path column with RLE + // Add file path column with REE let file_path = "/path/to/data/file.parquet"; - let result = ArrowReader::add_file_path_column_rle(batch, file_path); + let result = ArrowReader::add_file_path_column_ree(batch, file_path); assert!(result.is_ok(), "Should successfully add file path column"); let new_batch = result.unwrap(); @@ -2243,23 +2237,12 @@ message schema { .downcast_ref::>() .expect("Expected RunArray for _file column"); - // Check that all rows have the same file path - for i in 0..new_batch.num_rows() { - let slice = file_col.slice(i, 1); - let run_arr = slice - .as_any() - .downcast_ref::>() - .unwrap(); - let values = run_arr.values(); - let str_arr = values.as_string::(); - assert_eq!(str_arr.value(0), file_path); - } - // Verify the run array structure (should be optimally encoded) let run_ends = run_array.run_ends(); assert_eq!(run_ends.values().len(), 1, "Should have only 1 run end"); assert_eq!(run_ends.values()[0], 3, "Run end should be at position 3"); + // Check that the single value in the RunArray is the expected file path let values = run_array.values(); let string_values = values.as_string::(); assert_eq!(string_values.len(), 1, "Should have only 1 value"); @@ -2267,7 +2250,7 @@ message schema { } #[test] - fn test_add_file_path_column_rle_empty_batch() { + fn test_add_file_path_column_ree_empty_batch() { use arrow_array::RecordBatch; use arrow_schema::{DataType, Field, Schema}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; @@ -2282,9 +2265,9 @@ message schema { assert_eq!(batch.num_rows(), 0); - // Add file path column to empty batch with RLE + // Add file path column to empty batch with REE let file_path = "/empty/file.parquet"; - let result = ArrowReader::add_file_path_column_rle(batch, file_path); + let result = ArrowReader::add_file_path_column_ree(batch, file_path); // Should succeed with empty RunArray for empty batches assert!(result.is_ok()); diff --git a/crates/iceberg/src/delete_vector.rs b/crates/iceberg/src/delete_vector.rs index db13e9aba0..1040796034 100644 --- a/crates/iceberg/src/delete_vector.rs +++ b/crates/iceberg/src/delete_vector.rs @@ -23,7 +23,7 @@ use roaring::treemap::BitmapIter; use crate::{Error, ErrorKind, Result}; -#[derive(Debug, Default, Clone, PartialEq)] +#[derive(Debug, Default, Clone)] pub struct DeleteVector { pub inner: RoaringTreemap, } diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs index 302f1a68a3..396cceb51c 100644 --- a/crates/iceberg/src/scan/incremental/task.rs +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -66,7 +66,8 @@ impl AppendedFileScanTask { /// The stream of incremental file scan tasks. pub type IncrementalFileScanTaskStream = BoxStream<'static, Result>; -/// An incremental file scan task, which can be either an appended data file. +/// An incremental file scan task, which can be either an appended data file or positional +/// deletes. #[derive(Debug, Clone)] pub enum IncrementalFileScanTask { /// An appended data file. From 2c77259d20027e40da2b6f56828ae20adbc76343 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 13:47:37 +0100 Subject: [PATCH 48/68] . --- crates/iceberg/src/scan/context.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 2290270b57..e35c260be4 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -196,7 +196,6 @@ impl PlanContext { ) -> Result> + 'static>> { let manifest_files = manifest_list.entries().iter(); - // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut filtered_mfcs = vec![]; for manifest_file in manifest_files { let tx = if manifest_file.content == ManifestContentType::Deletes { From 70683de1922cb7cb7420dfa814049ce89fba4c46 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 13:57:19 +0100 Subject: [PATCH 49/68] More PR comments --- crates/iceberg/src/scan/incremental/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 71bb9018cf..170c7b30bf 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -406,7 +406,7 @@ impl IncrementalTableScan { } else if manifest_entry_context.manifest_entry.status() == ManifestStatus::Deleted { - // TODO: Process deleted files + // TODO (RAI-43291): Process deleted files Ok(()) } else { Ok(()) From e4ad209ac205db984dc6884bc57aa8116a919f6e Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 13:59:56 +0100 Subject: [PATCH 50/68] . --- crates/iceberg/src/scan/incremental/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 170c7b30bf..fcb7036861 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -454,7 +454,6 @@ impl IncrementalTableScan { /// Returns an [`CombinedIncrementalBatchRecordStream`] for this incremental table scan. pub async fn to_arrow(&self) -> Result { - let file_scan_task_stream = self.plan_files().await?; let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) .with_data_file_concurrency_limit(self.concurrency_limit_data_files) .with_row_group_filtering_enabled(true) @@ -465,13 +464,13 @@ impl IncrementalTableScan { } let arrow_reader = arrow_reader_builder.build(); + let file_scan_task_stream = self.plan_files().await?; file_scan_task_stream.stream(arrow_reader) } /// Returns an [`UnzippedIncrementalBatchRecordStream`] for this incremental table scan. /// This stream will yield separate streams for appended and deleted record batches. pub async fn to_unzipped_arrow(&self) -> Result { - let file_scan_task_stream = self.plan_files().await?; let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) .with_data_file_concurrency_limit(self.concurrency_limit_data_files) .with_row_group_filtering_enabled(true) @@ -482,6 +481,7 @@ impl IncrementalTableScan { } let arrow_reader = arrow_reader_builder.build(); + let file_scan_task_stream = self.plan_files().await?; file_scan_task_stream.stream(arrow_reader) } From 279df445ffdba5442c633ce4365a30dec6df7db7 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 14:03:07 +0100 Subject: [PATCH 51/68] Add comments --- crates/iceberg/src/scan/incremental/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index fcb7036861..4a5a6d05de 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -335,7 +335,8 @@ impl IncrementalTableScan { let mut channel_for_data_manifest_entry_error = file_scan_task_tx.clone(); let mut channel_for_delete_manifest_entry_error = file_scan_task_tx.clone(); - // Process the delete file [`ManifestEntry`] stream in parallel + // Process the delete file [`ManifestEntry`] stream in parallel. Builds the delete + // index below. spawn(async move { let result = manifest_entry_delete_ctx_rx .map(|me_ctx| Ok((me_ctx, delete_file_tx.clone()))) @@ -371,6 +372,7 @@ impl IncrementalTableScan { ) .await; + // Build the delete filter from the loaded deletes. let delete_filter = match result { Ok(loaded_deletes) => loaded_deletes.unwrap(), Err(e) => { From 630d6233208bd3e3fa4314d518e88827418d9d0d Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Tue, 28 Oct 2025 14:08:06 +0100 Subject: [PATCH 52/68] Avoid cloning --- crates/iceberg/src/arrow/incremental.rs | 29 ++++++++++++++++--------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 87deaee5cf..a00ec89866 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -16,17 +16,17 @@ // under the License. use std::pin::Pin; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use arrow_array::{RecordBatch, UInt64Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use futures::channel::mpsc::channel; use futures::stream::select; use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; -use roaring::RoaringTreemap; use crate::arrow::record_batch_transformer::RecordBatchTransformer; use crate::arrow::{ArrowReader, StreamsInto}; +use crate::delete_vector::DeleteVector; use crate::io::FileIO; use crate::runtime::spawn; use crate::scan::ArrowRecordBatchStream; @@ -122,13 +122,10 @@ impl StreamsInto } IncrementalFileScanTask::Delete(file_path, delete_vector) => { spawn(async move { - // Clone the `RoaringTreemap` underlying the delete vector to take ownership. - let treemap = { - let guard = delete_vector.lock().unwrap(); - guard.inner.clone() - }; let record_batch_stream = process_incremental_delete_task( - file_path, treemap, batch_size, + file_path, + delete_vector, + batch_size, ); match record_batch_stream { @@ -231,7 +228,7 @@ async fn process_incremental_append_task( fn process_incremental_delete_task( file_path: String, - delete_vector: RoaringTreemap, + delete_vector: Arc>, batch_size: Option, ) -> Result { let schema = Arc::new(ArrowSchema::new(vec![Field::new( @@ -242,7 +239,19 @@ fn process_incremental_delete_task( let batch_size = batch_size.unwrap_or(1024); - let stream = futures::stream::iter(delete_vector) + // Try to take ownership of the DeleteVector without cloning + // If we're the only Arc holder, this succeeds and we avoid the clone + let treemap = match Arc::try_unwrap(delete_vector) { + Ok(mutex) => mutex.into_inner().unwrap().inner, + Err(arc) => { + // If there are other Arc holders, we have to clone + // This shouldn't happen in normal operation since each task gets its own Arc + let guard = arc.lock().unwrap(); + guard.inner.clone() + } + }; + + let stream = futures::stream::iter(treemap) .chunks(batch_size) .map(move |chunk| { let array = UInt64Array::from_iter(chunk); From 337c73e162092ac09c1ca4874afc65b3b868d2ed Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Wed, 29 Oct 2025 08:47:27 +0100 Subject: [PATCH 53/68] Add reference to PR --- crates/iceberg/src/util/snapshot.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/iceberg/src/util/snapshot.rs b/crates/iceberg/src/util/snapshot.rs index 62aa6769ec..513e23c110 100644 --- a/crates/iceberg/src/util/snapshot.rs +++ b/crates/iceberg/src/util/snapshot.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +// Taken from https://github.com/apache/iceberg-rust/pull/1470 + use crate::spec::{SnapshotRef, TableMetadataRef}; struct Ancestors { From e31f76ef65d00c096014ef427774e0d078c578c9 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Wed, 29 Oct 2025 13:08:50 +0100 Subject: [PATCH 54/68] Some PR comments --- crates/iceberg/src/arrow/incremental.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index a00ec89866..a88b478bbc 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -241,15 +241,15 @@ fn process_incremental_delete_task( // Try to take ownership of the DeleteVector without cloning // If we're the only Arc holder, this succeeds and we avoid the clone - let treemap = match Arc::try_unwrap(delete_vector) { - Ok(mutex) => mutex.into_inner().unwrap().inner, - Err(arc) => { - // If there are other Arc holders, we have to clone - // This shouldn't happen in normal operation since each task gets its own Arc - let guard = arc.lock().unwrap(); - guard.inner.clone() - } - }; + let treemap = Arc::try_unwrap(delete_vector) + .map_err(|_| { + Error::new(ErrorKind::Unexpected, "failed to unwrap DeleteVector Arc") + })? + .into_inner() + .map(|dv| dv.inner) + .map_err(|e| { + Error::new(ErrorKind::Unexpected, "failed to get DeleteVector inner").with_source(e) + })?; let stream = futures::stream::iter(treemap) .chunks(batch_size) From 02f9a815675501dcd51e5b7a42894e7b6b8a2190 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Wed, 29 Oct 2025 13:11:53 +0100 Subject: [PATCH 55/68] . --- crates/iceberg/src/arrow/reader.rs | 25 +++++++++---------------- 1 file changed, 9 insertions(+), 16 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index b0ab636526..5c5a3f3239 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -501,7 +501,7 @@ impl ArrowReader { /// Helper function to add a `_file` column to a RecordBatch. /// Takes the array and field to add, reducing code duplication. - fn add_file_path_column_impl( + fn create_file_field( batch: RecordBatch, file_array: ArrayRef, file_field: Field, @@ -510,7 +510,10 @@ impl ArrowReader { columns.push(file_array); let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); - fields.push(Arc::new(file_field)); + fields.push(Arc::new(file_field.with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_FILE.to_string(), + )])))); let schema = Arc::new(ArrowSchema::new(fields)); RecordBatch::try_new(schema, columns).map_err(|e| { @@ -522,14 +525,6 @@ impl ArrowReader { }) } - /// Creates the metadata for the `_file` field with the reserved field ID. - fn create_file_field_metadata() -> HashMap { - HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_FILE.to_string(), - )]) - } - /// Adds a `_file` column to the RecordBatch containing the file path. /// Uses Run-End Encoding (REE) for maximum memory efficiency when the same /// file path is repeated across all rows. @@ -574,10 +569,9 @@ impl ArrowReader { RESERVED_COL_NAME_FILE, DataType::RunEndEncoded(run_ends_field, values_field), false, - ) - .with_metadata(Self::create_file_field_metadata()); + ); - Self::add_file_path_column_impl(batch, Arc::new(file_array), file_field) + Self::create_file_field(batch, Arc::new(file_array), file_field) } /// Adds a `_file` column to the RecordBatch containing the file path. @@ -589,10 +583,9 @@ impl ArrowReader { let file_array = StringArray::from(vec![file_path; num_rows]); // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE - let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false) - .with_metadata(Self::create_file_field_metadata()); + let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false); - Self::add_file_path_column_impl(batch, Arc::new(file_array), file_field) + Self::create_file_field(batch, Arc::new(file_array), file_field) } fn build_field_id_set_and_map( From ec24817e12baeee6e0f05030326eddfff4f0df07 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Wed, 29 Oct 2025 13:12:59 +0100 Subject: [PATCH 56/68] format --- crates/iceberg/src/arrow/incremental.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index a88b478bbc..29d23c996d 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -242,9 +242,7 @@ fn process_incremental_delete_task( // Try to take ownership of the DeleteVector without cloning // If we're the only Arc holder, this succeeds and we avoid the clone let treemap = Arc::try_unwrap(delete_vector) - .map_err(|_| { - Error::new(ErrorKind::Unexpected, "failed to unwrap DeleteVector Arc") - })? + .map_err(|_| Error::new(ErrorKind::Unexpected, "failed to unwrap DeleteVector Arc"))? .into_inner() .map(|dv| dv.inner) .map_err(|e| { From efc57e823b876a46d6ac031b29ca22dc4d9b4c28 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Wed, 29 Oct 2025 13:16:12 +0100 Subject: [PATCH 57/68] Allow overwrite operation for now --- crates/iceberg/src/scan/incremental/context.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index e92341eac4..04db28a4c4 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -70,11 +70,14 @@ impl IncrementalPlanContext { let mut ids = HashSet::new(); for snapshot in self.snapshots.iter() { let operation = &snapshot.summary().operation; - if !matches!(operation, Operation::Append | Operation::Delete) { + if !matches!( + operation, + Operation::Append | Operation::Overwrite | Operation::Delete + ) { return Err(crate::Error::new( crate::ErrorKind::FeatureUnsupported, format!( - "Incremental scan only supports Append and Delete operations, but snapshot {} has operation {:?}", + "Incremental scan only supports Append, Overwrite and Delete operations, but snapshot {} has operation {:?}", snapshot.snapshot_id(), operation ), From f78fcca1157bc28f769348374d3e0734c0c3779c Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Wed, 29 Oct 2025 15:48:42 +0100 Subject: [PATCH 58/68] Fix file_path column --- crates/iceberg/src/arrow/incremental.rs | 13 ++++- crates/iceberg/src/arrow/reader.rs | 77 ++++++++++++++++++------- 2 files changed, 68 insertions(+), 22 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 29d23c996d..683ef6f282 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -25,7 +25,9 @@ use futures::stream::select; use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; use crate::arrow::record_batch_transformer::RecordBatchTransformer; -use crate::arrow::{ArrowReader, StreamsInto}; +use crate::arrow::{ + ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_FIELD_ID_FILE_PATH, StreamsInto, +}; use crate::delete_vector::DeleteVector; use crate::io::FileIO; use crate::runtime::spawn; @@ -263,7 +265,14 @@ fn process_incremental_delete_task( "Failed to create RecordBatch for DeleteVector", ) }) - .and_then(|batch| ArrowReader::add_file_path_column(batch, &file_path)) + .and_then(|batch| { + ArrowReader::add_file_path_column( + batch, + &file_path, + RESERVED_COL_NAME_FILE_PATH, + RESERVED_FIELD_ID_FILE_PATH, + ) + }) }); Ok(Box::pin(stream) as ArrowRecordBatchStream) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 5c5a3f3239..510b0b8929 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -63,11 +63,21 @@ use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; /// Reserved field ID for the file path (_file) column per Iceberg spec +/// This is dead code for now but will be used when we add the _file column support. +#[allow(dead_code)] pub(crate) const RESERVED_FIELD_ID_FILE: i32 = 2147483646; /// Column name for the file path metadata column per Iceberg spec +/// This is dead code for now but will be used when we add the _file column support. +#[allow(dead_code)] pub(crate) const RESERVED_COL_NAME_FILE: &str = "_file"; +/// Reserved field ID for the file path column used in delete file reading. +pub(crate) const RESERVED_FIELD_ID_FILE_PATH: i32 = 2147483546; + +/// Column name for the file path metadata column used in delete file reading. +pub(crate) const RESERVED_COL_NAME_FILE_PATH: &str = "file_path"; + /// Builder to create ArrowReader pub struct ArrowReaderBuilder { batch_size: Option, @@ -505,6 +515,7 @@ impl ArrowReader { batch: RecordBatch, file_array: ArrayRef, file_field: Field, + field_id: i32, ) -> Result { let mut columns = batch.columns().to_vec(); columns.push(file_array); @@ -512,7 +523,7 @@ impl ArrowReader { let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); fields.push(Arc::new(file_field.with_metadata(HashMap::from([( PARQUET_FIELD_ID_META_KEY.to_string(), - RESERVED_FIELD_ID_FILE.to_string(), + field_id.to_string(), )])))); let schema = Arc::new(ArrowSchema::new(fields)); @@ -534,6 +545,8 @@ impl ArrowReader { pub(crate) fn add_file_path_column_ree( batch: RecordBatch, file_path: &str, + field_name: &str, + field_id: i32, ) -> Result { let num_rows = batch.num_rows(); @@ -566,26 +579,31 @@ impl ArrowReader { let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); let file_field = Field::new( - RESERVED_COL_NAME_FILE, + field_name, DataType::RunEndEncoded(run_ends_field, values_field), false, ); - Self::create_file_field(batch, Arc::new(file_array), file_field) + Self::create_file_field(batch, Arc::new(file_array), file_field, field_id) } /// Adds a `_file` column to the RecordBatch containing the file path. /// Materializes the file path string for each row (no compression). - pub(crate) fn add_file_path_column(batch: RecordBatch, file_path: &str) -> Result { + pub(crate) fn add_file_path_column( + batch: RecordBatch, + file_path: &str, + field_name: &str, + field_id: i32, + ) -> Result { let num_rows = batch.num_rows(); // Create a StringArray with the file path repeated num_rows times let file_array = StringArray::from(vec![file_path; num_rows]); // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE - let file_field = Field::new(RESERVED_COL_NAME_FILE, DataType::Utf8, false); + let file_field = Field::new(field_name, DataType::Utf8, false); - Self::create_file_field(batch, Arc::new(file_array), file_field) + Self::create_file_field(batch, Arc::new(file_array), file_field, field_id) } fn build_field_id_set_and_map( @@ -1597,7 +1615,9 @@ mod tests { use crate::ErrorKind; use crate::arrow::reader::{CollectFieldIdVisitor, PARQUET_FIELD_ID_META_KEY}; - use crate::arrow::{ArrowReader, ArrowReaderBuilder}; + use crate::arrow::{ + ArrowReader, ArrowReaderBuilder, RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE, + }; use crate::delete_vector::DeleteVector; use crate::expr::visitors::bound_predicate_visitor::visit; use crate::expr::{Bind, Predicate, Reference}; @@ -2397,8 +2417,6 @@ message schema { use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; - use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; - // Create a simple test batch with 2 columns and 3 rows let schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), @@ -2419,7 +2437,12 @@ message schema { // Add file path column with REE let file_path = "/path/to/data/file.parquet"; - let result = ArrowReader::add_file_path_column_ree(batch, file_path); + let result = ArrowReader::add_file_path_column_ree( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); assert!(result.is_ok(), "Should successfully add file path column"); let new_batch = result.unwrap(); @@ -2493,8 +2516,6 @@ message schema { use arrow_schema::{DataType, Field, Schema}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; - use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; - // Create an empty batch let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); @@ -2505,7 +2526,12 @@ message schema { // Add file path column to empty batch with REE let file_path = "/empty/file.parquet"; - let result = ArrowReader::add_file_path_column_ree(batch, file_path); + let result = ArrowReader::add_file_path_column_ree( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); // Should succeed with empty RunArray for empty batches assert!(result.is_ok()); @@ -2550,7 +2576,12 @@ message schema { // Test with file path containing special characters (materialized version) let file_path = "/path/with spaces/and-dashes/file_name.parquet"; - let result = ArrowReader::add_file_path_column(batch, file_path); + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); assert!(result.is_ok()); let new_batch = result.unwrap(); @@ -2566,8 +2597,6 @@ message schema { use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; - use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; - // Create a simple test batch with 2 columns and 3 rows let schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), @@ -2588,7 +2617,12 @@ message schema { // Add file path column with materialization let file_path = "/path/to/data/file.parquet"; - let result = ArrowReader::add_file_path_column(batch, file_path); + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); assert!(result.is_ok(), "Should successfully add file path column"); let new_batch = result.unwrap(); @@ -2639,8 +2673,6 @@ message schema { use arrow_schema::{DataType, Field, Schema}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; - use crate::arrow::{RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE}; - // Create an empty batch let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); @@ -2651,7 +2683,12 @@ message schema { // Add file path column to empty batch (materialized version) let file_path = "/empty/file.parquet"; - let result = ArrowReader::add_file_path_column(batch, file_path); + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); // Should succeed with empty StringArray assert!(result.is_ok()); From 97efb4c37d527942c8feb2090b22465cc9017022 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 11:02:40 +0100 Subject: [PATCH 59/68] Add overwrite test --- crates/iceberg/src/scan/incremental/mod.rs | 5 +- crates/iceberg/src/scan/incremental/tests.rs | 391 +++++++++++++++++++ 2 files changed, 395 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 4a5a6d05de..323d6465b3 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -409,7 +409,10 @@ impl IncrementalTableScan { == ManifestStatus::Deleted { // TODO (RAI-43291): Process deleted files - Ok(()) + Err(Error::new( + ErrorKind::FeatureUnsupported, + "Processing deleted data files is not supported yet in incremental scans", + )) } else { Ok(()) } diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index e5bf8bf847..05724a54a4 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -50,6 +50,28 @@ pub enum Operation { /// Takes a vector of (position, file_name) tuples specifying which position in which file to delete. /// Example: `Delete(vec![(0, "data-1.parquet"), (1, "data-1.parquet")])` deletes positions 0 and 1 from data-1.parquet Delete(Vec<(i64, String)>), + + /// Overwrite operation that can append new rows, delete specific positions, and remove entire data files. + /// This is a combination of append and delete operations in a single atomic snapshot. + /// + /// Parameters: + /// 1. Rows to append: Vec<(n, data)> tuples and the filename to write them to + /// 2. Positions to delete: Vec<(position, file_name)> tuples for positional deletes + /// 3. Data files to delete: Vec of file names to completely remove + /// + /// All three parameters can be empty, allowing for various combinations: + /// - Pure append: `Overwrite((rows, "file.parquet"), vec![], vec![])` + /// - Pure positional delete: `Overwrite((vec![], ""), vec![(pos, "file")], vec![])` + /// - Pure file deletion: `Overwrite((vec![], ""), vec![], vec!["file.parquet"])` + /// - Delete entire files: `Overwrite((vec![], ""), vec![], vec!["old-file.parquet"])` + /// + /// Example: `Overwrite((vec![(1, "new".to_string())], "new.parquet"), vec![(0, "old.parquet")], vec!["remove.parquet"])` + /// This adds new data to "new.parquet", deletes position 0 from "old.parquet", and removes "remove.parquet" entirely. + Overwrite( + (Vec<(i32, String)>, String), + Vec<(i64, String)>, + Vec, + ), } /// Tracks the state of data files across snapshots @@ -88,6 +110,7 @@ pub struct IncrementalTestFixture { impl IncrementalTestFixture { /// Create a new test fixture with the given operations. pub async fn new(operations: Vec) -> Self { + // Use pwd let tmp_dir = TempDir::new().unwrap(); let table_location = tmp_dir.path().join("incremental_test_table"); @@ -118,6 +141,7 @@ impl IncrementalTestFixture { let operation_type = match op { Operation::Add(..) => "append", Operation::Delete(..) => "delete", + Operation::Overwrite(..) => "overwrite", }; let manifest_list_location = @@ -578,6 +602,259 @@ impl IncrementalTestFixture { .unwrap(); manifest_list_write.close().await.unwrap(); } + + Operation::Overwrite((rows, file_name), positions_to_delete, files_to_delete) => { + // Overwrite creates a single snapshot that can: + // 1. Add new data files + // 2. Delete positions from existing files + // 3. Remove entire data files + + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + // Determine which files to delete + let files_to_delete_set: std::collections::HashSet = files_to_delete + .iter() + .map(|f| format!("{}/data/{}", &self.table_location, f)) + .collect(); + + // Add existing data files (mark deleted ones as DELETED, others as EXISTING) + for data_file in &data_files { + if files_to_delete_set.contains(&data_file.path) { + // Mark file for deletion + data_writer + .add_delete_entry( + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } else { + // Keep existing file + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + } + + // Add new data file if rows provided + if !rows.is_empty() { + let n_values: Vec = rows.iter().map(|(n, _)| *n).collect(); + let data_values: Vec = + rows.iter().map(|(_, d)| d.clone()).collect(); + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + + self.write_parquet_file(&data_file_path, &n_values, &data_values) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this new data file + data_files.push(DataFileInfo { + path: data_file_path, + snapshot_id, + sequence_number, + n_values, + }); + } + + // Remove deleted files from tracking + data_files.retain(|df| !files_to_delete_set.contains(&df.path)); + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Handle positional deletes if any + let mut manifests = vec![data_manifest]; + + if !positions_to_delete.is_empty() || !delete_files.is_empty() { + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + // Add existing delete files + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new positional delete files + if !positions_to_delete.is_empty() { + // Group deletes by file + let mut deletes_by_file: HashMap> = HashMap::new(); + for (position, file_name) in positions_to_delete { + let data_file_path = + format!("{}/data/{}", &self.table_location, file_name); + deletes_by_file + .entry(data_file_path) + .or_default() + .push(*position); + } + + for (data_file_path, positions) in deletes_by_file { + let delete_file_path = format!( + "{}/data/delete-{}-{}.parquet", + &self.table_location, + snapshot_id, + Uuid::new_v4() + ); + self.write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; + + delete_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(positions.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this delete file + delete_files.push(( + delete_file_path, + snapshot_id, + sequence_number, + positions + .into_iter() + .map(|pos| (data_file_path.clone(), pos)) + .collect(), + )); + } + } + + manifests.push(delete_writer.write_manifest_file().await.unwrap()); + } + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(manifests.into_iter()) + .unwrap(); + + manifest_list_write.close().await.unwrap(); + } } } } @@ -1310,3 +1587,117 @@ async fn test_incremental_scan_builder_options() { } } } + +#[tokio::test] +async fn test_incremental_scan_with_deleted_files_errors() { + // This test verifies that incremental scans properly error out when entire data files + // are deleted (overwrite operation), since this is not yet supported. + // + // Test scenario: + // Snapshot 1: Add file-1.parquet with data + // Snapshot 2: Add file-2.parquet with data + // Snapshot 3: Overwrite - delete file-1.parquet entirely + // Snapshot 4: Add file-3.parquet with data + // + // Incremental scan from snapshot 1 to snapshot 3 should error because file-1 + // was completely removed in the overwrite operation. + + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Add file-1 with rows + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + ], + "file-1.parquet".to_string(), + ), + // Snapshot 2: Add file-2 with rows + Operation::Add( + vec![(10, "x".to_string()), (20, "y".to_string())], + "file-2.parquet".to_string(), + ), + // Snapshot 3: Overwrite - delete file-1 entirely + Operation::Overwrite( + (vec![], "".to_string()), // No new data to add + vec![], // No positional deletes + vec!["file-1.parquet".to_string()], // Delete file-1 completely + ), + // Snapshot 4: Add file-3 (to have more snapshots) + Operation::Add(vec![(100, "p".to_string())], "file-3.parquet".to_string()), + ]) + .await; + + // Test 1: Incremental scan from snapshot 1 to 3 should error when building the stream + // because file-1 was deleted entirely in snapshot 3 + let scan = fixture + .table + .incremental_scan(1, 3) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + match stream_result { + Err(error) => { + assert_eq!( + error.message(), + "Processing deleted data files is not supported yet in incremental scans", + "Error message should indicate that deleted files are not supported. Got: {}", + error + ); + } + Ok(_) => panic!( + "Expected error when building stream over a snapshot that deletes entire data files" + ), + } + + // Test 2: Incremental scan from snapshot 2 to 4 should also error + // because it includes snapshot 3 which deletes a file + let scan = fixture + .table + .incremental_scan(2, 4) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + match stream_result { + Err(_) => { + // Expected error + } + Ok(_) => panic!("Expected error when scan range includes a snapshot that deletes files"), + } + + // Test 3: Incremental scan from snapshot 1 to 2 should work fine + // (no files deleted) + let scan = fixture + .table + .incremental_scan(1, 2) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + assert!( + stream_result.is_ok(), + "Scan should succeed when no files are deleted. Error: {:?}", + stream_result.err() + ); + + // Test 4: Incremental scan from snapshot 3 to 4 should work + // (starting from after the deletion) + let scan = fixture + .table + .incremental_scan(3, 4) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + assert!( + stream_result.is_ok(), + "Scan should succeed when starting after the file deletion. Error: {:?}", + stream_result.err() + ); +} From 3aa40cbb5f274e7800861d16d6a3e7ef9a0d65d1 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 13:04:08 +0100 Subject: [PATCH 60/68] Unwrap delete vector --- crates/iceberg/src/arrow/incremental.rs | 14 +++----------- crates/iceberg/src/scan/incremental/mod.rs | 6 +++++- crates/iceberg/src/scan/incremental/task.rs | 2 +- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 683ef6f282..e7ca2521bf 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -16,7 +16,7 @@ // under the License. use std::pin::Pin; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use arrow_array::{RecordBatch, UInt64Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; @@ -230,7 +230,7 @@ async fn process_incremental_append_task( fn process_incremental_delete_task( file_path: String, - delete_vector: Arc>, + delete_vector: DeleteVector, batch_size: Option, ) -> Result { let schema = Arc::new(ArrowSchema::new(vec![Field::new( @@ -241,15 +241,7 @@ fn process_incremental_delete_task( let batch_size = batch_size.unwrap_or(1024); - // Try to take ownership of the DeleteVector without cloning - // If we're the only Arc holder, this succeeds and we avoid the clone - let treemap = Arc::try_unwrap(delete_vector) - .map_err(|_| Error::new(ErrorKind::Unexpected, "failed to unwrap DeleteVector Arc"))? - .into_inner() - .map(|dv| dv.inner) - .map_err(|e| { - Error::new(ErrorKind::Unexpected, "failed to get DeleteVector inner").with_source(e) - })?; + let treemap = delete_vector.inner; let stream = futures::stream::iter(treemap) .chunks(batch_size) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 323d6465b3..d4270f14cf 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -444,8 +444,12 @@ impl IncrementalTableScan { delete_filter.with_read(|state| { for (path, delete_vector) in state.delete_vectors().iter() { if !appended_files.contains::(path) { + let delete_vector_inner = { + let guard = delete_vector.lock().unwrap(); + guard.clone() + }; let delete_task = - IncrementalFileScanTask::Delete(path.clone(), delete_vector.clone()); + IncrementalFileScanTask::Delete(path.clone(), delete_vector_inner); tasks.push(delete_task); } } diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs index 396cceb51c..4b2f244410 100644 --- a/crates/iceberg/src/scan/incremental/task.rs +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -74,7 +74,7 @@ pub enum IncrementalFileScanTask { Append(AppendedFileScanTask), /// Deleted records of a data file. First argument is the file path, second the delete /// vector. - Delete(String, Arc>), + Delete(String, DeleteVector), } impl IncrementalFileScanTask { From 584a68439b75caf7e39ddc86be453cd74b20d405 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 13:09:21 +0100 Subject: [PATCH 61/68] . --- crates/iceberg/src/arrow/reader.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 510b0b8929..97d1db5609 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -2501,7 +2501,11 @@ message schema { // Verify the run array structure (should be optimally encoded) let run_ends = run_array.run_ends(); assert_eq!(run_ends.values().len(), 1, "Should have only 1 run end"); - assert_eq!(run_ends.values()[0], 3, "Run end should be at position 3"); + assert_eq!( + run_ends.values()[0], + new_batch.num_rows() as i32, + "Run end should equal number of rows" + ); // Check that the single value in the RunArray is the expected file path let values = run_array.values(); From ff8a38cc1cea670178cbe38d13d07bfb9c959f03 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 13:16:42 +0100 Subject: [PATCH 62/68] Add assertion --- crates/iceberg/src/arrow/reader.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 97d1db5609..3a6759e35f 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -1602,6 +1602,7 @@ mod tests { use arrow_array::cast::AsArray; use arrow_array::{ArrayRef, LargeStringArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema, TimeUnit}; + use as_any::Downcast; use futures::TryStreamExt; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::arrow::{ArrowWriter, ProjectionMask}; @@ -2512,6 +2513,8 @@ message schema { let string_values = values.as_string::(); assert_eq!(string_values.len(), 1, "Should have only 1 value"); assert_eq!(string_values.value(0), file_path); + + assert!(string_values.downcast_ref::().unwrap().iter().all(|v| v == Some(file_path))) } #[test] From f31bb64cdc047d539247315a969a6ecbc961e5ab Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 13:22:37 +0100 Subject: [PATCH 63/68] Avoid cloning the mutex guard --- crates/iceberg/src/arrow/delete_filter.rs | 16 +++++++ crates/iceberg/src/arrow/reader.rs | 8 +++- crates/iceberg/src/scan/incremental/mod.rs | 50 ++++++++++++++++------ 3 files changed, 60 insertions(+), 14 deletions(-) diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index 09781d8751..e029912bbe 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -44,6 +44,11 @@ impl DeleteFileFilterState { pub fn delete_vectors(&self) -> &HashMap>> { &self.delete_vectors } + + /// Remove and return the delete vector for the given data file path. + pub fn remove_delete_vector(&mut self, path: &str) -> Option>> { + self.delete_vectors.remove(path) + } } #[derive(Clone, Debug, Default)] @@ -82,6 +87,17 @@ impl DeleteFilter { f(&state) } + pub(crate) fn with_write(&self, f: F) -> Result + where F: FnOnce(&mut DeleteFileFilterState) -> Result { + let mut state = self.state.write().map_err(|e| { + Error::new( + ErrorKind::Unexpected, + format!("Failed to acquire write lock: {}", e), + ) + })?; + f(&mut state) + } + pub(crate) fn try_start_eq_del_load(&self, file_path: &str) -> Option> { let mut state = self.state.write().unwrap(); diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 3a6759e35f..4752674484 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -2514,7 +2514,13 @@ message schema { assert_eq!(string_values.len(), 1, "Should have only 1 value"); assert_eq!(string_values.value(0), file_path); - assert!(string_values.downcast_ref::().unwrap().iter().all(|v| v == Some(file_path))) + assert!( + string_values + .downcast_ref::() + .unwrap() + .iter() + .all(|v| v == Some(file_path)) + ) } #[test] diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index d4270f14cf..7aa2f348cb 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -441,21 +441,45 @@ impl IncrementalTableScan { .collect::>(); // Augment `tasks` with delete tasks. - delete_filter.with_read(|state| { - for (path, delete_vector) in state.delete_vectors().iter() { - if !appended_files.contains::(path) { - let delete_vector_inner = { - let guard = delete_vector.lock().unwrap(); - guard.clone() - }; - let delete_task = - IncrementalFileScanTask::Delete(path.clone(), delete_vector_inner); - tasks.push(delete_task); - } - } - Ok(()) + // First collect paths to process (paths that weren't appended in this scan range) + let delete_paths: Vec = delete_filter.with_read(|state| { + Ok(state + .delete_vectors() + .keys() + .filter(|path| !appended_files.contains::(path)) + .cloned() + .collect()) })?; + // Now remove and take ownership of each delete vector + for path in delete_paths { + let delete_vector_arc = delete_filter.with_write(|state| { + state.remove_delete_vector(&path).ok_or_else(|| { + Error::new( + ErrorKind::Unexpected, + format!("DeleteVector for path {} not found", path), + ) + }) + })?; + + // Try to unwrap the Arc to avoid cloning the DeleteVector + let delete_vector_inner = Arc::try_unwrap(delete_vector_arc) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "DeleteVector Arc has multiple references, cannot take ownership", + ) + })? + .into_inner() + .map_err(|e| { + Error::new(ErrorKind::Unexpected, "Failed to unwrap DeleteVector Mutex") + .with_source(e) + })?; + + let delete_task = IncrementalFileScanTask::Delete(path, delete_vector_inner); + tasks.push(delete_task); + } + // We actually would not need a stream here, but we can keep it compatible with // other scan types. Ok(futures::stream::iter(tasks).map(Ok).boxed()) From c6f5ab6d8a07714af2123c989d3c6d2d914c7146 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 13:57:10 +0100 Subject: [PATCH 64/68] Abort when encountering a deleted delete file --- crates/iceberg/src/scan/incremental/mod.rs | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 7aa2f348cb..eb60bf311f 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -522,11 +522,6 @@ impl IncrementalTableScan { mut delete_file_ctx_tx: Sender, manifest_entry_context: ManifestEntryContext, ) -> Result<()> { - // Skip processing this manifest entry if it has been marked as deleted. - if !manifest_entry_context.manifest_entry.is_alive() { - return Ok(()); - } - // Abort the plan if we encounter a manifest entry for a data file or equality // deletes. if manifest_entry_context.manifest_entry.content_type() == DataContentType::Data { @@ -543,6 +538,14 @@ impl IncrementalTableScan { )); } + // Abort if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Processing deleted delete files is not supported yet in incremental scans", + )); + } + delete_file_ctx_tx .send(DeleteFileContext { manifest_entry: manifest_entry_context.manifest_entry.clone(), From c0989b07544121267c4c01fb74ab3a7e2c00729c Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 14:40:30 +0100 Subject: [PATCH 65/68] Adjust comment --- crates/iceberg/src/arrow/reader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 4752674484..389a9c9358 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -575,7 +575,7 @@ impl ArrowReader { // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE // DataType is RunEndEncoded with Int32 run ends and Utf8 values - // Note: values field is nullable to match what StringArray::from() creates + // Note: values field is nullable to match what RecordBatch::try_new(..) expects. let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); let file_field = Field::new( From b95414c559a10ca40b30c6ec862f9784be3084fe Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Thu, 30 Oct 2025 14:44:50 +0100 Subject: [PATCH 66/68] Update crates/iceberg/src/arrow/reader.rs Co-authored-by: Vukasin Stefanovic --- crates/iceberg/src/arrow/reader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 389a9c9358..ebb5491a16 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -575,7 +575,7 @@ impl ArrowReader { // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE // DataType is RunEndEncoded with Int32 run ends and Utf8 values - // Note: values field is nullable to match what RecordBatch::try_new(..) expects. + // Note: values field is nullable to match what RunArray::try_new(..) expects. let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); let file_field = Field::new( From 87da883ae5c40fe49be92a1438f69dfe1c3b21c9 Mon Sep 17 00:00:00 2001 From: Gerald Berger Date: Thu, 30 Oct 2025 14:48:53 +0100 Subject: [PATCH 67/68] Add check --- crates/iceberg/src/scan/incremental/mod.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index eb60bf311f..2f8a0680e6 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -539,10 +539,13 @@ impl IncrementalTableScan { } // Abort if it has been marked as deleted. - if !manifest_entry_context.manifest_entry.is_alive() { + if !manifest_entry_context.manifest_entry.is_alive() + && manifest_entry_context.manifest_entry.content_type() + == DataContentType::PositionDeletes + { return Err(Error::new( ErrorKind::FeatureUnsupported, - "Processing deleted delete files is not supported yet in incremental scans", + "Processing deleted (position) delete files is not supported yet in incremental scans", )); } From 6c60b90901090634d6a3e6b3c37b4b1e0d71e193 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 30 Oct 2025 14:54:13 +0100 Subject: [PATCH 68/68] Update crates/iceberg/src/scan/incremental/mod.rs --- crates/iceberg/src/scan/incremental/mod.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 2f8a0680e6..1253353b4d 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -161,8 +161,6 @@ impl<'a> IncrementalTableScanBuilder<'a> { })? .clone(); - // TODO: What properties do we need to verify about the snapshots? What about - // schema changes? let snapshots = ancestors_between( &self.table.metadata_ref(), snapshot_to.snapshot_id(),