From d27e8f4398ce807d76d70b5c8259208810614b57 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Mon, 18 May 2026 17:07:36 -0700 Subject: [PATCH 01/36] feat(raster-zarr): sedona-raster-zarr crate + sd_read_zarr UDTF MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds a sedona-raster-zarr crate that maps Zarr groups onto SedonaDB's N-D raster Arrow schema, plus an sd_read_zarr DataFusion UDTF that exposes it to SQL. ## sedona-raster-zarr Two entry points emit one raster row per chunk position in the group's chunk grid, with one band per array: * group_to_indb_rasters(group_uri) — eager: fetches every chunk's bytes into the Arrow data column. * group_to_outdb_rasters(group_uri) — URI-only: emits chunk-anchor URIs in each band's outdb_uri (#array=&chunk=,...). Built on the zarrs 0.23 crate, chosen over GDAL MultiDim because zarrs reads Zarr v3 sharding and vlen-utf8 coordinate variables. Modules: * source_uri — group-URI normalisation (file:// + bare path; cloud schemes error pending the resolver work) and chunk anchor build/parse. * geozarr — proj:wkt2 / proj:projjson / proj:epsg precedence and spatial:transform / spatial:dims parsing from group attributes. * dtype — zarrs DataType ↔ BandDataType via the type-erased is::() downcast (zarrs 0.23 wraps Arc, not an enum). * loader — opens FilesystemStore + Group, enumerates child arrays (sorted by path for deterministic band order), validates shared chunk grid / chunk shape / dim names, walks the chunk grid row-major, computes per-chunk transforms by translating the group affine along the spatial axes. Tests: 31 in-module unit tests + 4 integration tests that build a real Zarr group on disk via zarrs::ArrayBuilder and verify byte-exact pixel content, per-chunk transforms, anchor URI format, and error paths. ## sd_read_zarr UDTF DataFusion table function exposing the loader to SQL. Mirrors sd_random_geometry's shape (Function + Provider + Exec) and registers in SedonaContext::new_from_context. SELECT raster FROM sd_read_zarr('file:///path/to/datacube.zarr'); SELECT count(*) FROM sd_read_zarr( 'file:///path/to/datacube.zarr', '{\"mode\": \"outdb\", \"rows_per_batch\": 256}' ); Returns a single-column table raster: Raster. All existing RS_* UDFs operate on the column unchanged. JSON options: * mode: \"indb\" (default) or \"outdb\". Phase 1 defaults to InDb so byte-reading kernels work end-to-end. Flips to \"outdb\" once a format-keyed OutDb dispatcher registers a zarr loader. * rows_per_batch: chunks per RecordBatch (default 1024). * num_partitions: scan partitions (default 1). Anything other than 1 errors — round-robin chunk partitioning lands with the resolver work. ZarrChunkProvider builds the StructArray once at plan time. scan() wraps the inner exec in ProjectionExec so empty / partial projections (e.g. SELECT count(*)) match the requested physical schema. ZarrChunkExec slices the StructArray into rows_per_batch-sized RecordBatches. Bounded, Incremental, single-partition. 6 in-module integration tests build a Zarr fixture and exercise the full SQL pipeline. ## Out of scope (follow-up PR) * OutDb byte resolver (the function the byte-loading hook calls to fetch zarr chunks) — depends on a format-keyed multi-backend dispatcher. * Default mode flip from \"indb\" to \"outdb\". * Cloud storage backends (zarrs_object_store) and the async runtime story they require. * Round-robin chunk partitioning past num_partitions=1. * CREATE EXTERNAL TABLE ... STORED AS ZARR. --- Cargo.lock | 586 +++++++++++++++++- Cargo.toml | 5 + rust/sedona-raster-zarr/Cargo.toml | 52 ++ rust/sedona-raster-zarr/src/dtype.rs | 66 ++ rust/sedona-raster-zarr/src/geozarr.rs | 228 +++++++ rust/sedona-raster-zarr/src/lib.rs | 41 ++ rust/sedona-raster-zarr/src/loader.rs | 513 +++++++++++++++ rust/sedona-raster-zarr/src/source_uri.rs | 286 +++++++++ .../tests/zarr_roundtrip.rs | 232 +++++++ rust/sedona/Cargo.toml | 9 + rust/sedona/src/context.rs | 3 + rust/sedona/src/lib.rs | 1 + rust/sedona/src/zarr_read.rs | 492 +++++++++++++++ 13 files changed, 2511 insertions(+), 3 deletions(-) create mode 100644 rust/sedona-raster-zarr/Cargo.toml create mode 100644 rust/sedona-raster-zarr/src/dtype.rs create mode 100644 rust/sedona-raster-zarr/src/geozarr.rs create mode 100644 rust/sedona-raster-zarr/src/lib.rs create mode 100644 rust/sedona-raster-zarr/src/loader.rs create mode 100644 rust/sedona-raster-zarr/src/source_uri.rs create mode 100644 rust/sedona-raster-zarr/tests/zarr_roundtrip.rs create mode 100644 rust/sedona/src/zarr_read.rs diff --git a/Cargo.lock b/Cargo.lock index 04c768a85..158cdb45f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -553,6 +553,17 @@ dependencies = [ "abi_stable", ] +[[package]] +name = "async-lock" +version = "3.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290f7f2596bd5b78a9fec8088ccd89180d7f9f55b94b0576823bbbdc72ee8311" +dependencies = [ + "event-listener", + "event-listener-strategy", + "pin-project-lite", +] + [[package]] name = "async-stream" version = "0.3.6" @@ -621,6 +632,17 @@ dependencies = [ "winapi", ] +[[package]] +name = "auto_impl" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffdcb70bdbc4d478427380519163274ac86e52916e10f0a8889adf0f96d3fee7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.117", +] + [[package]] name = "autocfg" version = "1.5.0" @@ -1063,6 +1085,32 @@ dependencies = [ "generic-array 0.14.7", ] +[[package]] +name = "blosc-src" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9046dd58971db0226346fde214143d16a6eb12f535b5320d0ea94fcea420631" +dependencies = [ + "cc", + "libz-sys", + "lz4-sys", + "snappy_src", + "zstd-sys", +] + +[[package]] +name = "blusc" +version = "0.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4e0c17eaa785d2673fe58c22fc817946c2330ed47f3d9f79835d65950d32a45" +dependencies = [ + "flate2", + "lz4_flex", + "pkg-config", + "snap", + "zstd", +] + [[package]] name = "bon" version = "3.8.2" @@ -1129,6 +1177,20 @@ name = "bytemuck" version = "1.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c8efb64bd706a16a1bdde310ae86b351e4d21550d98d056f22f8a7f7a2183fec" +dependencies = [ + "bytemuck_derive", +] + +[[package]] +name = "bytemuck_derive" +version = "1.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9abbd1bc6865053c427f7198e6af43bfdedc55ab791faed4fbd361d789575ff" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.117", +] [[package]] name = "byteorder" @@ -1369,6 +1431,15 @@ version = "0.4.32" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cc14f565cf027a105f7a44ccf9e5b424348421a1d8952a8fc9d499d313107789" +[[package]] +name = "concurrent-queue" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ca0197aee26d1ae37445ee532fefce43251d24cc7c166799f4d46817f1d3973" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "const-random" version = "0.1.18" @@ -1404,6 +1475,15 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3d52eff69cd5e647efe296129160853a42795992097e8af39800e1060caeea9b" +[[package]] +name = "convert_case" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "633458d4ef8c78b72454de2d54fd6ab2e60f9e02be22f3c6104cdc8a4e0fceb9" +dependencies = [ + "unicode-segmentation", +] + [[package]] name = "core-foundation" version = "0.10.1" @@ -1453,6 +1533,15 @@ dependencies = [ "libc", ] +[[package]] +name = "crc32c" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a47af21622d091a8f0fb295b88bc886ac74efcc613efc19f5d0b21de5c89e47" +dependencies = [ + "rustc_version", +] + [[package]] name = "crc32fast" version = "1.5.0" @@ -2398,6 +2487,29 @@ dependencies = [ "serde_core", ] +[[package]] +name = "derive_more" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d751e9e49156b02b44f9c1815bcb94b984cdcc4396ecc32521c739452808b134" +dependencies = [ + "derive_more-impl", +] + +[[package]] +name = "derive_more-impl" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "799a97264921d8623a957f6c3b9011f3b5492f557bbb7a5a19b7fa6d06ba8dcb" +dependencies = [ + "convert_case", + "proc-macro2", + "quote", + "rustc_version", + "syn 2.0.117", + "unicode-xid", +] + [[package]] name = "digest" version = "0.10.7" @@ -2558,6 +2670,27 @@ version = "3.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dea2df4cf52843e0452895c455a1a2cfbb842a1e7329671acf418fdc53ed4c59" +[[package]] +name = "event-listener" +version = "5.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13b66accf52311f30a0db42147dadea9850cb48cd070028831ae5f5d4b856ab" +dependencies = [ + "concurrent-queue", + "parking", + "pin-project-lite", +] + +[[package]] +name = "event-listener-strategy" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8be9f3dfaaffdae2972880079a491a1a8bb7cbed0b8dd7a347f668b4150a3b93" +dependencies = [ + "event-listener", + "pin-project-lite", +] + [[package]] name = "fastrand" version = "2.4.1" @@ -3019,6 +3152,7 @@ version = "2.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" dependencies = [ + "bytemuck", "cfg-if", "crunchy", "num-traits", @@ -3571,6 +3705,15 @@ version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" +[[package]] +name = "inventory" +version = "0.3.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4f0c30c76f2f4ccee3fe55a2435f691ca00c0e4bd87abe4f4a851b1d4dac39b" +dependencies = [ + "rustversion", +] + [[package]] name = "ipnet" version = "2.11.0" @@ -3877,6 +4020,18 @@ dependencies = [ "zlib-rs", ] +[[package]] +name = "libz-sys" +version = "1.1.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc3a226e576f50782b3305c5ccf458698f92798987f551c6a02efe8276721e22" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + [[package]] name = "link-cplusplus" version = "1.0.12" @@ -3919,6 +4074,15 @@ version = "0.4.29" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5e5032e24019045c762d3c0f28f5b6b8bbf38563a65908389bf7978758920897" +[[package]] +name = "lru" +version = "0.16.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f66e8d5d03f609abc3a39e6f08e4164ebf1447a732906d39eb9b99b7919ef39" +dependencies = [ + "hashbrown 0.16.1", +] + [[package]] name = "lru" version = "0.18.0" @@ -3934,6 +4098,16 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" +[[package]] +name = "lz4-sys" +version = "1.11.1+lz4-1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bd8c0d6c6ed0cd30b3652886bb8711dc4bb01d637a68105a3d5158039b418e6" +dependencies = [ + "cc", + "libc", +] + [[package]] name = "lz4_flex" version = "0.12.1" @@ -3943,6 +4117,16 @@ dependencies = [ "twox-hash", ] +[[package]] +name = "matrixmultiply" +version = "0.3.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a06de3016e9fae57a36fd14dba131fccf49f74b40b7fbdb472f96e361ec71a08" +dependencies = [ + "autocfg", + "rawpointer", +] + [[package]] name = "md-5" version = "0.10.6" @@ -4004,6 +4188,60 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "moka" +version = "0.12.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "957228ad12042ee839f93c8f257b62b4c0ab5eaae1d4fa60de53b27c9d7c5046" +dependencies = [ + "crossbeam-channel", + "crossbeam-epoch", + "crossbeam-utils", + "equivalent", + "parking_lot", + "portable-atomic", + "smallvec", + "tagptr", + "uuid", +] + +[[package]] +name = "monostate" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb4cc965c89dd0615a9e822ff8002f7633d2466143d51bd58693e4b2c75aabad" +dependencies = [ + "monostate-impl", + "serde", + "serde_core", +] + +[[package]] +name = "monostate-impl" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23f5b99488110875b5904839d396c2cdfaf241ff6622638acb879cc7effad5de" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.117", +] + +[[package]] +name = "ndarray" +version = "0.17.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "520080814a7a6b4a6e9070823bb24b4531daac8c4627e08ba5de8c5ef2f2752d" +dependencies = [ + "matrixmultiply", + "num-complex", + "num-integer", + "num-traits", + "portable-atomic", + "portable-atomic-util", + "rawpointer", +] + [[package]] name = "nibble_vec" version = "0.1.0" @@ -4044,6 +4282,20 @@ dependencies = [ "winapi", ] +[[package]] +name = "num" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + [[package]] name = "num-bigint" version = "0.4.6" @@ -4061,6 +4313,7 @@ version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" dependencies = [ + "bytemuck", "num-traits", ] @@ -4079,6 +4332,28 @@ dependencies = [ "num-traits", ] +[[package]] +name = "num-iter" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" +dependencies = [ + "num-bigint", + "num-integer", + "num-traits", +] + [[package]] name = "num-traits" version = "0.2.19" @@ -4270,6 +4545,12 @@ dependencies = [ "winapi", ] +[[package]] +name = "parking" +version = "2.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" + [[package]] name = "parking_lot" version = "0.12.5" @@ -4350,6 +4631,12 @@ version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" +[[package]] +name = "pathdiff" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df94ce210e5bc13cb6651479fa48d14f601d9858cfe0467f43ae157023b938d3" + [[package]] name = "pdqselect" version = "0.1.0" @@ -4453,6 +4740,17 @@ dependencies = [ "portable-atomic", ] +[[package]] +name = "positioned-io" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4ec4b80060f033312b99b6874025d9503d2af87aef2dd4c516e253fbfcdada7" +dependencies = [ + "byteorder", + "libc", + "winapi", +] + [[package]] name = "potential_utf" version = "0.1.4" @@ -4646,6 +4944,18 @@ dependencies = [ "serde", ] +[[package]] +name = "quick_cache" +version = "0.6.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1c821816e9b928e20e92ed59bb3ac4aab321d16ca2316871c9fe7ca739cd477" +dependencies = [ + "ahash", + "equivalent", + "hashbrown 0.16.1", + "parking_lot", +] + [[package]] name = "quinn" version = "0.11.9" @@ -4782,6 +5092,12 @@ dependencies = [ "rand 0.10.1", ] +[[package]] +name = "rawpointer" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "60a357793950651c4ed0f3f52338f53b2f809f32d83a07f72909fa13e4c6c1e3" + [[package]] name = "rayon" version = "1.12.0" @@ -4802,6 +5118,15 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "rayon_iter_concurrent_limit" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d09ee01023de07fa073ce14c37cbe0a9e099c6b0b60a29cf4af6d04d9553fed7" +dependencies = [ + "rayon", +] + [[package]] name = "recursive" version = "0.1.1" @@ -5341,8 +5666,10 @@ dependencies = [ "sedona-pointcloud", "sedona-proj", "sedona-query-planner", + "sedona-raster", "sedona-raster-functions", "sedona-raster-gdal", + "sedona-raster-zarr", "sedona-s2geography", "sedona-schema", "sedona-spatial-join", @@ -5356,6 +5683,8 @@ dependencies = [ "tempfile", "tokio", "url", + "zarrs", + "zarrs_filesystem", ] [[package]] @@ -5592,7 +5921,7 @@ version = "0.4.0" dependencies = [ "geo-traits", "geo-types", - "lru", + "lru 0.18.0", "rstest", "sedona-testing", "serde", @@ -5814,7 +6143,7 @@ dependencies = [ "arrow-buffer", "criterion", "datafusion-common", - "lru", + "lru 0.18.0", "sedona-common", "sedona-gdal", "sedona-raster", @@ -5824,6 +6153,22 @@ dependencies = [ "tokio", ] +[[package]] +name = "sedona-raster-zarr" +version = "0.4.0" +dependencies = [ + "arrow-array", + "arrow-schema", + "datafusion-common", + "sedona-common", + "sedona-raster", + "sedona-schema", + "serde_json", + "tempfile", + "zarrs", + "zarrs_filesystem", +] + [[package]] name = "sedona-s2geography" version = "0.4.0" @@ -5856,7 +6201,7 @@ dependencies = [ "arrow-schema", "criterion", "datafusion-common", - "lru", + "lru 0.18.0", "sedona-common", "serde_json", ] @@ -6133,6 +6478,7 @@ version = "1.0.149" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "83fc039473c5595ace860d8c4fafa220ff474b3fc6bfdb4293327f1a37e94d86" dependencies = [ + "indexmap 2.13.0", "itoa", "memchr", "serde", @@ -6140,6 +6486,17 @@ dependencies = [ "zmij", ] +[[package]] +name = "serde_repr" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.117", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -6247,6 +6604,16 @@ version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" +[[package]] +name = "snappy_src" +version = "0.2.5+snappy.1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e1432067a55bcfb1fd522d2aca6537a4fcea32bba87ea86921226d14f9bad53" +dependencies = [ + "cc", + "link-cplusplus", +] + [[package]] name = "socket2" version = "0.6.3" @@ -6406,6 +6773,12 @@ dependencies = [ "windows", ] +[[package]] +name = "tagptr" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b2093cf4c8eb1e67749a6762251bc9cd836b6fc171623bd0a9d324d37af2417" + [[package]] name = "tar" version = "0.4.45" @@ -6485,6 +6858,15 @@ dependencies = [ "syn 2.0.117", ] +[[package]] +name = "thread_local" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f60246a4944f24f6e018aa17cdeffb7818b76356965d03b07d6a9886e8962185" +dependencies = [ + "cfg-if", +] + [[package]] name = "thrift" version = "0.17.0" @@ -6804,6 +7186,12 @@ version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7264e107f553ccae879d21fbea1d6724ac785e8c3bfc762137959b5802826ef3" +[[package]] +name = "unsafe_cell_slice" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6659959f702dcdaad77bd6e42a9409a32ceccc06943ec93c8a4306be00eb6cf1" + [[package]] name = "untrusted" version = "0.9.0" @@ -7546,6 +7934,198 @@ dependencies = [ "synstructure", ] +[[package]] +name = "zarrs" +version = "0.23.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "251f4ec1a9e60ca9c693ade9b29a0b981a61e68ea3e2ae85fa9da31bcdca5dbe" +dependencies = [ + "async-lock", + "base64", + "blosc-src", + "blusc", + "bytemuck", + "bytes", + "crc32c", + "derive_more", + "flate2", + "getrandom 0.3.4", + "half", + "inventory", + "itertools 0.14.0", + "itoa", + "libz-sys", + "log", + "lru 0.16.4", + "moka", + "ndarray", + "num", + "num-complex", + "paste", + "quick_cache", + "rayon", + "rayon_iter_concurrent_limit", + "serde", + "serde_json", + "thiserror 2.0.17", + "thread_local", + "unsafe_cell_slice", + "uuid", + "zarrs_chunk_grid", + "zarrs_chunk_key_encoding", + "zarrs_codec", + "zarrs_data_type", + "zarrs_filesystem", + "zarrs_metadata", + "zarrs_metadata_ext", + "zarrs_plugin", + "zarrs_storage", + "zstd", +] + +[[package]] +name = "zarrs_chunk_grid" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cf67386fd96a0336cd3e5ab5ca6cb14e0e05aee80f1acae8c4d3cf562a8bb65" +dependencies = [ + "derive_more", + "inventory", + "itertools 0.14.0", + "rayon", + "thiserror 2.0.17", + "tinyvec", + "zarrs_metadata", + "zarrs_plugin", +] + +[[package]] +name = "zarrs_chunk_key_encoding" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9040e7feaa92d1904d492acd0cd91b97214f1791c5b5738e6c05b2ca4145a382" +dependencies = [ + "derive_more", + "inventory", + "zarrs_metadata", + "zarrs_plugin", + "zarrs_storage", +] + +[[package]] +name = "zarrs_codec" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "383a129a6a0cbb2c80cdba23809e5cab85159756464b7d0f112468a495c128da" +dependencies = [ + "async-trait", + "bytemuck", + "derive_more", + "futures", + "inventory", + "itertools 0.14.0", + "rayon", + "thiserror 2.0.17", + "unsafe_cell_slice", + "zarrs_chunk_grid", + "zarrs_data_type", + "zarrs_metadata", + "zarrs_plugin", + "zarrs_storage", +] + +[[package]] +name = "zarrs_data_type" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc7c594c9363278fcd9db4c205514f009944206eb093ea7ad40b85f50009f31" +dependencies = [ + "derive_more", + "half", + "inventory", + "num", + "paste", + "serde", + "serde_json", + "thiserror 2.0.17", + "zarrs_metadata", + "zarrs_plugin", +] + +[[package]] +name = "zarrs_filesystem" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "270efeb0181651aee5460b3232f2fc83e91bd646cefe75001d1c8f9a4f3abf81" +dependencies = [ + "bytes", + "derive_more", + "itertools 0.14.0", + "libc", + "page_size", + "pathdiff", + "positioned-io", + "thiserror 2.0.17", + "walkdir", + "zarrs_storage", +] + +[[package]] +name = "zarrs_metadata" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d60c4c363a8a302d7babb3c29017850a7b4e0af6ca5f9ba2946263a185b62fea" +dependencies = [ + "derive_more", + "half", + "monostate", + "serde", + "serde_json", + "thiserror 2.0.17", +] + +[[package]] +name = "zarrs_metadata_ext" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2048e07848ca99c7450518e0584929300b1b6a3cf442f18b26ffd3520814bd5b" +dependencies = [ + "derive_more", + "monostate", + "num", + "serde", + "serde_json", + "serde_repr", + "thiserror 2.0.17", + "zarrs_metadata", +] + +[[package]] +name = "zarrs_plugin" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cbe0ed432aee86856f70ca33be36eaf4a0dae21ab730750d9280a7ca1e95046" +dependencies = [ + "paste", + "regex", + "serde_json", + "thiserror 2.0.17", +] + +[[package]] +name = "zarrs_storage" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d098796d2ed4cf94896569615101e0432e870a7665396da5cc32300fb68f7c1" +dependencies = [ + "auto_impl", + "bytes", + "derive_more", + "itertools 0.14.0", + "thiserror 2.0.17", + "unsafe_cell_slice", +] + [[package]] name = "zerocopy" version = "0.8.33" diff --git a/Cargo.toml b/Cargo.toml index 9e276783a..0717c8358 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -39,6 +39,7 @@ members = [ "rust/sedona-raster", "rust/sedona-raster-functions", "rust/sedona-raster-gdal", + "rust/sedona-raster-zarr", "rust/sedona-schema", "rust/sedona-spatial-join", "rust/sedona-spatial-join-geography", @@ -133,6 +134,9 @@ tokio = { version = "1.52", features = ["macros", "rt", "sync"] } url = "2.5.7" wkb = "0.9.2" wkt = "0.14.0" +zarrs = { version = "0.23", default-features = false } +zarrs_filesystem = "0.3" +zarrs_object_store = "0.6" # Workspace path dependencies for internal crates sedona = { version = "0.4.0", path = "rust/sedona" } @@ -150,6 +154,7 @@ sedona-pointcloud = { version = "0.4.0", path = "rust/sedona-pointcloud" } sedona-raster = { version = "0.4.0", path = "rust/sedona-raster" } sedona-raster-functions = { version = "0.4.0", path = "rust/sedona-raster-functions" } sedona-raster-gdal = { version = "0.4.0", path = "rust/sedona-raster-gdal" } +sedona-raster-zarr = { version = "0.4.0", path = "rust/sedona-raster-zarr" } sedona-schema = { version = "0.4.0", path = "rust/sedona-schema" } sedona-spatial-join = { version = "0.4.0", path = "rust/sedona-spatial-join" } sedona-spatial-join-gpu = { version = "0.4.0", path = "rust/sedona-spatial-join-gpu" } diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml new file mode 100644 index 000000000..1465d3086 --- /dev/null +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +[package] +name = "sedona-raster-zarr" +version.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true +homepage.workspace = true +repository.workspace = true +description.workspace = true +readme.workspace = true +edition.workspace = true +rust-version.workspace = true + +[lints.clippy] +result_large_err = "allow" + +[dependencies] +arrow-array = { workspace = true } +arrow-schema = { workspace = true } +datafusion-common = { workspace = true } +sedona-common = { workspace = true } +sedona-raster = { workspace = true } +sedona-schema = { workspace = true } +serde_json = { workspace = true } +zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } +zarrs_filesystem = { workspace = true } + +# `blosc` is gated off Windows: c-blosc (statically linked) bundles its own +# `pthread_create` / `pthread_cond_*` symbols, which conflict with rtools45's +# `libpthread.a` during the MinGW link of the R `sedonadb.dll`. Non-Windows +# targets get the full blosc-compressed Zarr reading capability. +[target.'cfg(not(target_os = "windows"))'.dependencies] +zarrs = { workspace = true, features = ["blosc"] } + +[dev-dependencies] +tempfile = { workspace = true } diff --git a/rust/sedona-raster-zarr/src/dtype.rs b/rust/sedona-raster-zarr/src/dtype.rs new file mode 100644 index 000000000..65d234d7b --- /dev/null +++ b/rust/sedona-raster-zarr/src/dtype.rs @@ -0,0 +1,66 @@ +// 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. + +//! Mapping between Zarr datatypes and SedonaDB's `BandDataType`. +//! +//! zarrs 0.23 models `DataType` as a wrapper around `Arc`, +//! so we discriminate via the type-erased `is::()` check rather than +//! pattern-matching an enum. + +use arrow_schema::ArrowError; +use sedona_schema::raster::BandDataType; +use zarrs::array::data_type::{ + BoolDataType, Float32DataType, Float64DataType, Int16DataType, Int32DataType, Int64DataType, + Int8DataType, UInt16DataType, UInt32DataType, UInt64DataType, UInt8DataType, +}; +use zarrs::array::DataType as ZarrDataType; + +/// Map a Zarr `DataType` to a SedonaDB `BandDataType`. +/// +/// Bool maps to UInt8 losslessly (Zarr packs bools to one byte each, matching +/// our representation). Variable-length, complex, and extended-precision +/// types error with `NotYetImplemented` — they have no numeric counterpart +/// in `BandDataType` today. +pub fn zarr_to_band_data_type(dt: &ZarrDataType) -> Result { + if dt.is::() { + Ok(BandDataType::UInt8) + } else if dt.is::() { + Ok(BandDataType::Int8) + } else if dt.is::() { + Ok(BandDataType::UInt8) + } else if dt.is::() { + Ok(BandDataType::Int16) + } else if dt.is::() { + Ok(BandDataType::UInt16) + } else if dt.is::() { + Ok(BandDataType::Int32) + } else if dt.is::() { + Ok(BandDataType::UInt32) + } else if dt.is::() { + Ok(BandDataType::Int64) + } else if dt.is::() { + Ok(BandDataType::UInt64) + } else if dt.is::() { + Ok(BandDataType::Float32) + } else if dt.is::() { + Ok(BandDataType::Float64) + } else { + Err(ArrowError::NotYetImplemented(format!( + "Zarr datatype {dt:?} has no BandDataType mapping yet" + ))) + } +} diff --git a/rust/sedona-raster-zarr/src/geozarr.rs b/rust/sedona-raster-zarr/src/geozarr.rs new file mode 100644 index 000000000..82427b997 --- /dev/null +++ b/rust/sedona-raster-zarr/src/geozarr.rs @@ -0,0 +1,228 @@ +// 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. + +//! GeoZarr attribute parsing for CRS and affine transform. +//! +//! Reads the `proj:*` (CRS) and `spatial:*` (transform / spatial dim +//! mapping) attribute conventions from a Zarr group's attributes, mapping +//! them onto SedonaDB's per-raster `crs` and `transform` fields. +//! +//! Phase 1 expects attributes at the group level and inherits them across +//! every array. Per-array overrides are rejected by the group-constraint +//! validator (see `loader`). + +use arrow_schema::ArrowError; + +/// Per-group geo metadata distilled from `proj:*` / `spatial:*` attributes. +#[derive(Debug, Clone, PartialEq)] +pub struct GroupGeoMetadata { + /// CRS string in PROJ or WKT format (whichever the group declared). + /// `None` if no `proj:wkt2` / `proj:projjson` / `proj:epsg` attribute + /// is present on the group. + pub crs: Option, + /// Affine transform in GDAL GeoTransform order: + /// `[origin_x, scale_x, skew_x, origin_y, skew_y, scale_y]`. `None` + /// when no `spatial:transform` attribute is present. + pub transform: Option<[f64; 6]>, + /// Names of the spatial dimensions in the order the group declares + /// them (typically `["y", "x"]`). `None` falls back to a 2-D default + /// at construction time in the loader. + pub spatial_dims: Option>, +} + +impl GroupGeoMetadata { + /// Parse the group-level attributes object (the raw JSON map zarrs + /// surfaces from a group) into a `GroupGeoMetadata`. + /// + /// Returns `Ok(default-empty)` when none of the conventional keys are + /// present — Phase 1 treats geospatial metadata as optional; downstream + /// fall-backs in the loader provide identity transforms when needed. + pub fn from_attributes( + attrs: &serde_json::Map, + ) -> Result { + let crs = parse_crs(attrs)?; + let transform = parse_transform(attrs)?; + let spatial_dims = parse_spatial_dims(attrs)?; + Ok(Self { + crs, + transform, + spatial_dims, + }) + } +} + +fn parse_crs( + attrs: &serde_json::Map, +) -> Result, ArrowError> { + // GeoZarr `proj:` precedence — wkt2 wins over projjson wins over epsg + // code. Match how downstream tools (e.g. xarray + rioxarray) resolve + // multi-attribute groups: more specific representations override + // numeric codes. + if let Some(v) = attrs.get("proj:wkt2") { + return Ok(Some(json_value_to_string(v, "proj:wkt2")?)); + } + if let Some(v) = attrs.get("proj:projjson") { + return Ok(Some(json_value_to_string(v, "proj:projjson")?)); + } + if let Some(v) = attrs.get("proj:epsg") { + let code = v.as_i64().ok_or_else(|| { + ArrowError::InvalidArgumentError("proj:epsg attribute must be an integer".into()) + })?; + return Ok(Some(format!("EPSG:{code}"))); + } + Ok(None) +} + +fn parse_transform( + attrs: &serde_json::Map, +) -> Result, ArrowError> { + let Some(t) = attrs.get("spatial:transform") else { + return Ok(None); + }; + let arr = t.as_array().ok_or_else(|| { + ArrowError::InvalidArgumentError("spatial:transform attribute must be a JSON array".into()) + })?; + if arr.len() != 6 { + return Err(ArrowError::InvalidArgumentError(format!( + "spatial:transform must have 6 elements (GDAL GeoTransform order); got {}", + arr.len() + ))); + } + let mut out = [0f64; 6]; + for (i, v) in arr.iter().enumerate() { + out[i] = v.as_f64().ok_or_else(|| { + ArrowError::InvalidArgumentError(format!("spatial:transform[{i}] must be a number")) + })?; + } + Ok(Some(out)) +} + +fn parse_spatial_dims( + attrs: &serde_json::Map, +) -> Result>, ArrowError> { + let Some(v) = attrs.get("spatial:dims") else { + return Ok(None); + }; + let arr = v.as_array().ok_or_else(|| { + ArrowError::InvalidArgumentError( + "spatial:dims attribute must be a JSON array of strings".into(), + ) + })?; + let dims = arr + .iter() + .map(|e| { + e.as_str().map(String::from).ok_or_else(|| { + ArrowError::InvalidArgumentError("spatial:dims entries must be strings".into()) + }) + }) + .collect::, _>>()?; + Ok(Some(dims)) +} + +fn json_value_to_string(v: &serde_json::Value, attr_name: &str) -> Result { + if let Some(s) = v.as_str() { + return Ok(s.to_string()); + } + if v.is_object() { + return Ok(v.to_string()); + } + Err(ArrowError::InvalidArgumentError(format!( + "{attr_name} attribute must be a string or JSON object" + ))) +} + +#[cfg(test)] +mod tests { + use super::*; + use serde_json::json; + + fn map(json: serde_json::Value) -> serde_json::Map { + json.as_object().unwrap().clone() + } + + #[test] + fn empty_attrs_parses_to_all_none() { + let g = GroupGeoMetadata::from_attributes(&map(json!({}))).unwrap(); + assert!(g.crs.is_none()); + assert!(g.transform.is_none()); + assert!(g.spatial_dims.is_none()); + } + + #[test] + fn epsg_code_parses_to_epsg_string() { + let g = GroupGeoMetadata::from_attributes(&map(json!({"proj:epsg": 4326}))).unwrap(); + assert_eq!(g.crs.as_deref(), Some("EPSG:4326")); + } + + #[test] + fn wkt2_takes_precedence_over_epsg() { + let g = GroupGeoMetadata::from_attributes(&map(json!({ + "proj:epsg": 4326, + "proj:wkt2": "GEOGCRS[\"WGS 84\", ...]" + }))) + .unwrap(); + assert!(g.crs.as_deref().unwrap().starts_with("GEOGCRS")); + } + + #[test] + fn projjson_object_serialises_to_string() { + let g = GroupGeoMetadata::from_attributes(&map(json!({ + "proj:projjson": {"type": "GeographicCRS"} + }))) + .unwrap(); + let crs = g.crs.unwrap(); + assert!(crs.contains("GeographicCRS")); + } + + #[test] + fn transform_parses_six_floats() { + let g = GroupGeoMetadata::from_attributes(&map(json!({ + "spatial:transform": [0.0, 1.0, 0.0, 0.0, 0.0, -1.0] + }))) + .unwrap(); + assert_eq!(g.transform, Some([0.0, 1.0, 0.0, 0.0, 0.0, -1.0])); + } + + #[test] + fn transform_wrong_length_errors() { + let err = GroupGeoMetadata::from_attributes(&map(json!({ + "spatial:transform": [0.0, 1.0, 0.0] + }))) + .unwrap_err() + .to_string(); + assert!(err.contains("6 elements"), "{err}"); + } + + #[test] + fn spatial_dims_parses_string_list() { + let g = GroupGeoMetadata::from_attributes(&map(json!({ + "spatial:dims": ["y", "x"] + }))) + .unwrap(); + assert_eq!(g.spatial_dims, Some(vec!["y".to_string(), "x".to_string()])); + } + + #[test] + fn spatial_dims_non_string_errors() { + let err = GroupGeoMetadata::from_attributes(&map(json!({ + "spatial:dims": ["y", 1] + }))) + .unwrap_err() + .to_string(); + assert!(err.contains("strings"), "{err}"); + } +} diff --git a/rust/sedona-raster-zarr/src/lib.rs b/rust/sedona-raster-zarr/src/lib.rs new file mode 100644 index 000000000..42790a7fc --- /dev/null +++ b/rust/sedona-raster-zarr/src/lib.rs @@ -0,0 +1,41 @@ +// 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. + +//! Zarr-backed N-D raster loader for SedonaDB. +//! +//! Opens a Zarr group via the `zarrs` crate and emits one raster row per +//! chunk position. Each row's bands are the corresponding chunks of each +//! array in the group, mapped onto SedonaDB's canonical N-D raster Arrow +//! schema. +//! +//! Two entry points: +//! +//! - [`group_to_indb_rasters`] — eagerly fetches every chunk's bytes into +//! the Arrow `data` column. Suitable for snapshots / small groups. +//! - [`group_to_outdb_rasters`] — emits chunk-anchor URIs only; bytes +//! fetch on demand through the process-wide OutDb loader (registered +//! separately via `sedona-raster`'s loader hook). +//! +//! Phase 1 supports local filesystem stores only; cloud backends arrive +//! with the resolver work. + +pub mod dtype; +pub mod geozarr; +pub mod loader; +pub mod source_uri; + +pub use loader::{group_to_indb_rasters, group_to_outdb_rasters}; diff --git a/rust/sedona-raster-zarr/src/loader.rs b/rust/sedona-raster-zarr/src/loader.rs new file mode 100644 index 000000000..5caa45322 --- /dev/null +++ b/rust/sedona-raster-zarr/src/loader.rs @@ -0,0 +1,513 @@ +// 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. + +//! Zarr group → N-D raster `StructArray` entry points. +//! +//! Both `group_to_indb_rasters` and `group_to_outdb_rasters` produce the +//! same row shape: one raster row per chunk position, with one band per +//! array in the group. They differ only in how each row's pixel bytes +//! are delivered: +//! +//! - **InDb** — every chunk is fetched eagerly and copied into the +//! Arrow `data` column. Heavy for large datacubes; intended for +//! snapshots. +//! - **OutDb** — `data` is left empty; each band's `outdb_uri` carries a +//! chunk anchor (`zarr:///#chunk=i0,i1,...`). +//! Byte resolution awaits the format-keyed dispatch work in a +//! follow-up PR. + +use std::sync::Arc; + +use arrow_array::StructArray; +use arrow_schema::ArrowError; +use sedona_common::sedona_internal_datafusion_err; +use sedona_raster::builder::RasterBuilder; +use sedona_schema::raster::BandDataType; +use zarrs::array::{Array, ArrayBytes}; +use zarrs::group::Group; +use zarrs_filesystem::FilesystemStore; + +use crate::dtype::zarr_to_band_data_type; +use crate::geozarr::GroupGeoMetadata; +use crate::source_uri::{build_chunk_anchor, group_uri_to_filesystem_path}; + +/// Open a Zarr group and eagerly fetch every chunk's bytes into the +/// returned `StructArray`. Each row holds one chunk position's data +/// across every array in the group. +pub fn group_to_indb_rasters(group_uri: &str) -> Result { + build_rasters(group_uri, Mode::InDb) +} + +/// Open a Zarr group and emit one row per chunk position with chunk-anchor +/// URIs in each band's `outdb_uri`. The `data` column is empty; bytes +/// resolve on demand through whichever OutDb loader is registered for +/// the `zarr` format. +pub fn group_to_outdb_rasters(group_uri: &str) -> Result { + build_rasters(group_uri, Mode::OutDb) +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +enum Mode { + InDb, + OutDb, +} + +/// Per-array metadata extracted once at group open and reused for every +/// chunk position. Caching this avoids re-reading Zarr metadata for each +/// of the (potentially thousands of) chunk rows. +struct ArrayInfo { + /// Array path within the store, used to build chunk anchor URIs and + /// surface in band names. + path: String, + /// Open zarrs handle. + array: Array, + /// SedonaDB BandDataType corresponding to this array's zarrs dtype. + data_type: BandDataType, + /// Dimension names in array order. Required to be `Some(_)` for every + /// dim; missing names error at validation time. + dim_names: Vec, + /// Inner chunk grid shape, one entry per dimension. Used to enumerate + /// chunk positions and validated to match across arrays. + chunk_grid_shape: Vec, + /// Chunk shape (elements per chunk per dim). Same for every chunk + /// position in Phase 1 (no ragged final chunks emitted as separate + /// short rows). + chunk_shape: Vec, + /// Encoded fill value in native-endian byte representation, for the + /// `nodata` field. None when the array has no fill value declared. + nodata: Option>, +} + +fn build_rasters(group_uri: &str, mode: Mode) -> Result { + let fs_path = group_uri_to_filesystem_path(group_uri)?; + let store = FilesystemStore::new(&fs_path).map_err(|e| { + ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( + "failed to open Zarr filesystem store at {}: {e}", + fs_path.display() + ))) + })?; + let storage: Arc = Arc::new(store); + + let group = Group::open(storage.clone(), "/").map_err(|e| { + ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( + "failed to open Zarr group at {group_uri}: {e}" + ))) + })?; + + let geo = GroupGeoMetadata::from_attributes(group.attributes())?; + + let arrays = group.child_arrays().map_err(|e| { + ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( + "failed to enumerate child arrays in {group_uri}: {e}" + ))) + })?; + if arrays.is_empty() { + return Err(ArrowError::InvalidArgumentError(format!( + "Zarr group at {group_uri} has no child arrays" + ))); + } + + let array_infos = collect_array_infos(arrays)?; + validate_group_constraints(&array_infos)?; + + // Spatial-dim resolution. Phase 1 supports two configurations: + // - dim_names ends with ["y", "x"] (canonical for georeferenced + // 2-D and time-series rasters); the spatial extent is the chunk's + // last two dims. + // - `spatial:dims` attribute on the group explicitly names them. + // Anything else errors with a clear message — silently picking dims + // would produce wrong per-row transforms. + let spatial_dim_indices = + resolve_spatial_dim_indices(&array_infos[0].dim_names, geo.spatial_dims.as_deref())?; + let spatial_dims_names: Vec<&str> = spatial_dim_indices + .iter() + .map(|&i| array_infos[0].dim_names[i].as_str()) + .collect(); + let chunk_spatial_shape: Vec = spatial_dim_indices + .iter() + .map(|&i| array_infos[0].chunk_shape[i] as i64) + .collect(); + + let group_transform = geo.transform.unwrap_or([0.0, 1.0, 0.0, 0.0, 0.0, -1.0]); + + let total_rows = array_infos[0].chunk_grid_shape.iter().product::() as usize; + let mut builder = RasterBuilder::new(total_rows); + + // Walk the chunk grid in row-major (C-order) order. The outer-most + // axis varies slowest, the innermost fastest — same convention used + // for byte strides in `BandRefImpl`. + let mut chunk_indices = vec![0u64; array_infos[0].chunk_grid_shape.len()]; + loop { + let row_transform = compute_row_transform( + &group_transform, + &chunk_indices, + &array_infos[0].chunk_shape, + &spatial_dim_indices, + ); + let crs_str = geo.crs.as_deref(); + builder.start_raster_nd( + &row_transform, + &spatial_dims_names, + &chunk_spatial_shape, + crs_str, + )?; + + for info in &array_infos { + let dim_names_ref: Vec<&str> = info.dim_names.iter().map(String::as_str).collect(); + let nodata_ref = info.nodata.as_deref(); + let anchor; + let (outdb_uri_arg, outdb_format_arg) = match mode { + Mode::InDb => (None, None), + Mode::OutDb => { + anchor = build_chunk_anchor(group_uri, &info.path, &chunk_indices); + (Some(anchor.as_str()), Some("zarr")) + } + }; + builder.start_band_nd( + Some(info.path.as_str()), + &dim_names_ref, + &info.chunk_shape, + info.data_type, + nodata_ref, + outdb_uri_arg, + outdb_format_arg, + )?; + match mode { + Mode::InDb => { + let bytes = retrieve_chunk_bytes(&info.array, &chunk_indices)?; + builder.band_data_writer().append_value(&bytes); + } + Mode::OutDb => { + // Schema-OutDb: empty `data` column. Byte resolution + // routes through the OutDb loader when a downstream + // consumer calls contiguous_data / nd_buffer. + builder.band_data_writer().append_value([0u8; 0]); + } + } + builder.finish_band()?; + } + builder.finish_raster()?; + + if !advance_chunk_indices(&mut chunk_indices, &array_infos[0].chunk_grid_shape) { + break; + } + } + + builder.finish() +} + +/// Collect per-array metadata from open zarrs `Array` handles. +/// +/// Sorts arrays by path so band ordering across rows is deterministic +/// (zarrs's underlying store listing order is implementation-defined — +/// filesystem stores currently happen to enumerate alphabetically, but +/// that's not part of the contract we want consumers to rely on). +fn collect_array_infos( + mut arrays: Vec>, +) -> Result, ArrowError> { + arrays.sort_by(|a, b| a.path().as_str().cmp(b.path().as_str())); + let mut out = Vec::with_capacity(arrays.len()); + for array in arrays { + let path = array.path().to_string(); + let data_type = zarr_to_band_data_type(array.data_type())?; + let dim_names = match array.dimension_names() { + Some(names) => names + .iter() + .enumerate() + .map(|(i, n)| { + n.clone().ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "array {path}: dimension {i} has no name; Phase 1 requires every \ + Zarr array dimension to be named", + )) + }) + }) + .collect::, _>>()?, + None => { + return Err(ArrowError::InvalidArgumentError(format!( + "array {path}: dimension_names is absent; Phase 1 requires every Zarr \ + array to declare dimension_names", + ))); + } + }; + let chunk_grid_shape = array.chunk_grid_shape().to_vec(); + let chunk_shape = array + .chunk_shape(&vec![0u64; chunk_grid_shape.len()]) + .map_err(|e| { + ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( + "array {path}: failed to query chunk shape: {e}" + ))) + })? + .iter() + .map(|n| n.get()) + .collect(); + let fill_bytes = array.fill_value().as_ne_bytes(); + let nodata = if fill_bytes.is_empty() { + None + } else { + Some(fill_bytes.to_vec()) + }; + out.push(ArrayInfo { + path, + array, + data_type, + dim_names, + chunk_grid_shape, + chunk_shape, + nodata, + }); + } + Ok(out) +} + +/// Enforce Phase 1 group constraints. All arrays must agree on chunk grid +/// shape, chunk shape, and dimension names. We do NOT enforce shared +/// element shape (`array.shape()`) because users routinely group +/// arrays with the same chunk grid but different totals (e.g. a coord +/// variable with one fewer dim is rejected here anyway by the dim-name +/// check). +fn validate_group_constraints(infos: &[ArrayInfo]) -> Result<(), ArrowError> { + let first = &infos[0]; + for other in &infos[1..] { + if other.chunk_grid_shape != first.chunk_grid_shape { + return Err(ArrowError::InvalidArgumentError(format!( + "arrays {} and {} have different chunk grid shapes ({:?} vs {:?}); \ + Phase 1 requires a shared chunk grid across the group", + first.path, other.path, first.chunk_grid_shape, other.chunk_grid_shape + ))); + } + if other.chunk_shape != first.chunk_shape { + return Err(ArrowError::InvalidArgumentError(format!( + "arrays {} and {} have different chunk shapes ({:?} vs {:?}); \ + Phase 1 requires a shared chunk shape across the group", + first.path, other.path, first.chunk_shape, other.chunk_shape + ))); + } + if other.dim_names != first.dim_names { + return Err(ArrowError::InvalidArgumentError(format!( + "arrays {} and {} have different dimension names ({:?} vs {:?}); \ + Phase 1 requires identical dim_names across the group", + first.path, other.path, first.dim_names, other.dim_names + ))); + } + } + Ok(()) +} + +/// Pick the `(y_index, x_index)` axes of an array's dim_names. +/// +/// If `spatial_dims` is provided via the group's `spatial:dims` attribute, +/// look up those names by position. Otherwise, default to the last two +/// dims and require they be named `y` and `x` (in that order) — the +/// canonical GeoZarr-2D convention. Anything else errors. +fn resolve_spatial_dim_indices( + dim_names: &[String], + spatial_dims: Option<&[String]>, +) -> Result, ArrowError> { + if let Some(spatial) = spatial_dims { + let mut idx = Vec::with_capacity(spatial.len()); + for s in spatial { + let i = dim_names.iter().position(|n| n == s).ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "spatial:dims declared name {s:?} not found in array dim_names {dim_names:?}", + )) + })?; + idx.push(i); + } + return Ok(idx); + } + let n = dim_names.len(); + if n < 2 { + return Err(ArrowError::InvalidArgumentError(format!( + "Phase 1 requires at least 2 dimensions to resolve spatial axes; got {dim_names:?}", + ))); + } + if dim_names[n - 2] != "y" || dim_names[n - 1] != "x" { + return Err(ArrowError::InvalidArgumentError(format!( + "Phase 1 expects the last two dim_names to be [\"y\", \"x\"] when \ + `spatial:dims` is not declared; got {dim_names:?}", + ))); + } + Ok(vec![n - 2, n - 1]) +} + +/// Per-chunk transform: translate the group's transform so the chunk's +/// `[0, 0]` element maps to the chunk's spatial origin. +fn compute_row_transform( + group_transform: &[f64; 6], + chunk_indices: &[u64], + chunk_shape: &[u64], + spatial_dim_indices: &[usize], +) -> [f64; 6] { + // GDAL GeoTransform layout: [origin_x, scale_x, skew_x, origin_y, skew_y, scale_y]. + // Translation along x = chunk_x_index × chunk_x_size in pixel-coordinate space, + // converted to world coordinates via the affine. + // + // Phase 1 assumes spatial_dim_indices == [y_index, x_index] (validated + // upstream). Index 0 is the y axis, index 1 is the x axis. + let y_axis = spatial_dim_indices[0]; + let x_axis = spatial_dim_indices[1]; + let x_offset = (chunk_indices[x_axis] * chunk_shape[x_axis]) as f64; + let y_offset = (chunk_indices[y_axis] * chunk_shape[y_axis]) as f64; + let [ox, sx, kx, oy, ky, sy] = *group_transform; + [ + ox + sx * x_offset + kx * y_offset, + sx, + kx, + oy + ky * x_offset + sy * y_offset, + ky, + sy, + ] +} + +/// Advance `chunk_indices` row-major over `chunk_grid_shape`. Returns +/// `true` while there are positions left, `false` when the grid is +/// exhausted (and the indices wrap back to all-zero). +fn advance_chunk_indices(chunk_indices: &mut [u64], chunk_grid_shape: &[u64]) -> bool { + for i in (0..chunk_indices.len()).rev() { + chunk_indices[i] += 1; + if chunk_indices[i] < chunk_grid_shape[i] { + return true; + } + chunk_indices[i] = 0; + } + false +} + +/// Retrieve a single chunk's bytes as a fresh `Vec`. +/// +/// Phase 1 uses `ArrayBytes::Fixed`, so this errors for variable-length +/// element types — those don't have a `BandDataType` counterpart anyway, +/// so the dtype check in `collect_array_infos` rejects them upstream. +fn retrieve_chunk_bytes( + array: &Array, + chunk_indices: &[u64], +) -> Result, ArrowError> { + let bytes = array + .retrieve_chunk::>(chunk_indices) + .map_err(|e| { + ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( + "failed to retrieve chunk {:?} from {}: {e}", + chunk_indices, + array.path() + ))) + })?; + let raw = bytes.into_fixed().map_err(|_| { + ArrowError::InvalidArgumentError(format!( + "array {}: variable-length chunk bytes not supported in Phase 1", + array.path() + )) + })?; + Ok(raw.into_owned()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn advance_chunk_indices_walks_row_major() { + // 2×3 grid; outer axis varies slowest. + let shape = vec![2u64, 3u64]; + let mut idx = vec![0u64, 0u64]; + let mut visited = vec![idx.clone()]; + while advance_chunk_indices(&mut idx, &shape) { + visited.push(idx.clone()); + } + // Expected row-major traversal of a 2×3 grid (last axis fastest): + // (0,0) (0,1) (0,2) (1,0) (1,1) (1,2) + let expected = vec![ + vec![0, 0], + vec![0, 1], + vec![0, 2], + vec![1, 0], + vec![1, 1], + vec![1, 2], + ]; + assert_eq!(visited, expected); + } + + #[test] + fn advance_chunk_indices_signals_exhaustion_via_wraparound() { + // After the last position (1,2) the next advance must return false + // and reset back to all-zero. + let shape = vec![2u64, 3u64]; + let mut idx = vec![1u64, 2u64]; + assert!(!advance_chunk_indices(&mut idx, &shape)); + assert_eq!(idx, vec![0, 0]); + } + + #[test] + fn advance_chunk_indices_single_position_grid_exits_immediately() { + let shape = vec![1u64]; + let mut idx = vec![0u64]; + assert!(!advance_chunk_indices(&mut idx, &shape)); + } + + #[test] + fn compute_row_transform_translates_to_chunk_origin_no_skew() { + // 2-D y,x array with chunk [2, 2] and group origin (10, 20). + // Chunk (1, 2) in row-major should map to origin (10 + 2*2, 20 + 2*1*(-1)) + // for transform [10, 1, 0, 20, 0, -1]: x_off = 4, y_off = 2. + let group_t = [10.0, 1.0, 0.0, 20.0, 0.0, -1.0]; + let chunk_shape = vec![2u64, 2u64]; + let chunk_idx = vec![1u64, 2u64]; + let t = compute_row_transform(&group_t, &chunk_idx, &chunk_shape, &[0, 1]); + // y_axis=0, x_axis=1 → x_off=2*2=4, y_off=1*2=2 + assert_eq!(t[0], 10.0 + 4.0); // origin_x + assert_eq!(t[3], 20.0 - 2.0); // origin_y after y_off=2 with sy=-1 + // Scale/skew carry through unchanged. + assert_eq!(t[1], 1.0); + assert_eq!(t[2], 0.0); + assert_eq!(t[4], 0.0); + assert_eq!(t[5], -1.0); + } + + #[test] + fn resolve_spatial_dim_indices_default_yx() { + let names = vec!["time".into(), "y".into(), "x".into()]; + let idx = resolve_spatial_dim_indices(&names, None).unwrap(); + assert_eq!(idx, vec![1, 2]); + } + + #[test] + fn resolve_spatial_dim_indices_default_rejects_wrong_order() { + let names = vec!["x".into(), "y".into()]; + let err = resolve_spatial_dim_indices(&names, None) + .unwrap_err() + .to_string(); + assert!(err.contains("[\"y\", \"x\"]"), "{err}"); + } + + #[test] + fn resolve_spatial_dim_indices_explicit_lookup() { + let names = vec!["lat".into(), "lon".into(), "time".into()]; + let spatial = vec!["lat".to_string(), "lon".to_string()]; + let idx = resolve_spatial_dim_indices(&names, Some(&spatial)).unwrap(); + assert_eq!(idx, vec![0, 1]); + } + + #[test] + fn resolve_spatial_dim_indices_explicit_missing_errors() { + let names = vec!["a".into(), "b".into()]; + let spatial = vec!["nope".to_string()]; + let err = resolve_spatial_dim_indices(&names, Some(&spatial)) + .unwrap_err() + .to_string(); + assert!(err.contains("nope"), "{err}"); + } +} diff --git a/rust/sedona-raster-zarr/src/source_uri.rs b/rust/sedona-raster-zarr/src/source_uri.rs new file mode 100644 index 000000000..155a632aa --- /dev/null +++ b/rust/sedona-raster-zarr/src/source_uri.rs @@ -0,0 +1,286 @@ +// 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. + +//! Zarr URI helpers — group locators and per-chunk OutDb anchors. +//! +//! Two URI shapes flow through the loader: +//! +//! 1. **Group URI** (the loader entry-point argument). Identifies a Zarr +//! group on a backend, e.g. `file:///tmp/datacube.zarr`, +//! `s3://bucket/datacube.zarr/2024`, or a bare local path. Phase 1 +//! accepts `file://` and bare-path; cloud schemes ship with the +//! resolver. +//! 2. **Chunk anchor URI** (written into a band's `outdb_uri`). Addresses +//! one chunk in one array within a group: +//! `#array=&chunk=,,...`. The store URI +//! is the original group URI verbatim (whatever scheme that uses); +//! array path and chunk indices both live in the fragment so the +//! store URI is unambiguous even when both contain `/` (e.g. +//! `s3://bucket/foo.zarr/2024` + `subgroup/B01`). Inner-chunk indices +//! always — sharding is a storage detail the resolver handles. +//! +//! The "this is a zarr anchor" signal lives in the band's +//! `outdb_format = "zarr"` field, not in a URI scheme prefix — matches +//! the GDAL OutDb convention and lets the format-keyed dispatcher +//! route without parsing URI schemes. + +use arrow_schema::ArrowError; + +/// Parts of a chunk-anchor URI. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ChunkAnchor { + /// Original store URI for the *group* (e.g. `file:///tmp/foo.zarr`, + /// `s3://bucket/foo.zarr/2024`). Same value across every chunk of every + /// array in a group. + pub store_uri: String, + /// Array's path within the store (e.g. `temperature`, `subgroup/B01`). + pub array_path: String, + /// Chunk's position in the array's inner chunk grid (one index per + /// array dimension). + pub chunk_indices: Vec, +} + +/// Build a chunk anchor URI from its parts. +/// +/// Format: `#array=&chunk=,,...`. The +/// store URI is the original group URI verbatim; array path and chunk +/// indices live in the fragment so the store URI is unambiguous even +/// when path components themselves contain `/`. "This is zarr" lives +/// in the band's `outdb_format` field rather than in a URI scheme. +pub fn build_chunk_anchor(store_uri: &str, array_path: &str, chunk_indices: &[u64]) -> String { + let indices = chunk_indices + .iter() + .map(|i| i.to_string()) + .collect::>() + .join(","); + let array = array_path.trim_start_matches('/'); + format!("{store_uri}#array={array}&chunk={indices}") +} + +/// Parse a chunk-anchor URI back into its parts. +/// +/// Strict: rejects URIs that don't carry both `array=` and `chunk=` +/// fragment parameters with valid values. +pub fn parse_chunk_anchor(uri: &str) -> Result { + let (store_uri, fragment) = uri.split_once('#').ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "chunk anchor URI missing `#array=...&chunk=...` fragment: {uri}" + )) + })?; + + let mut array_path: Option<&str> = None; + let mut chunk_str: Option<&str> = None; + for pair in fragment.split('&') { + if let Some(v) = pair.strip_prefix("array=") { + array_path = Some(v); + } else if let Some(v) = pair.strip_prefix("chunk=") { + chunk_str = Some(v); + } + // Unknown fragment params are ignored — forward-compatible for + // future per-anchor extensions (e.g. version pins). + } + + let array_path = array_path + .ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "chunk anchor URI fragment missing `array=` parameter: {uri}" + )) + })? + .to_string(); + let indices_str = chunk_str.ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "chunk anchor URI fragment missing `chunk=` parameter: {uri}" + )) + })?; + if indices_str.is_empty() { + return Err(ArrowError::InvalidArgumentError(format!( + "chunk anchor URI has empty index list: {uri}" + ))); + } + let chunk_indices = indices_str + .split(',') + .map(|s| { + s.parse::().map_err(|_| { + ArrowError::InvalidArgumentError(format!( + "chunk index `{s}` is not a non-negative integer in {uri}" + )) + }) + }) + .collect::, _>>()?; + + Ok(ChunkAnchor { + store_uri: store_uri.to_string(), + array_path, + chunk_indices, + }) +} + +/// Normalize a user-supplied group URI into a local filesystem path. +/// +/// Phase 1 supports `file://` and bare-path URIs only. Cloud schemes +/// (`s3://`, `gs://`, `az://`, `https://`) error with a clear message +/// pointing at the resolver work that adds them. +pub fn group_uri_to_filesystem_path(uri: &str) -> Result { + if let Some(rest) = uri.strip_prefix("file://") { + return Ok(std::path::PathBuf::from(rest)); + } + for scheme in ["s3://", "gs://", "az://", "https://", "http://"] { + if uri.starts_with(scheme) { + return Err(ArrowError::NotYetImplemented(format!( + "cloud Zarr stores ({scheme}…) are not supported in this phase; \ + use a local filesystem path or `file://` URI for now" + ))); + } + } + // Bare path. + Ok(std::path::PathBuf::from(uri)) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn build_anchor_basic() { + let uri = build_chunk_anchor("file:///tmp/datacube.zarr", "temperature", &[47, 2, 3]); + assert_eq!( + uri, + "file:///tmp/datacube.zarr#array=temperature&chunk=47,2,3" + ); + } + + #[test] + fn build_anchor_strips_leading_slash_on_array_path() { + let uri = build_chunk_anchor("file:///tmp/foo.zarr", "/array", &[0]); + assert_eq!(uri, "file:///tmp/foo.zarr#array=array&chunk=0"); + } + + #[test] + fn build_anchor_with_nested_array_path() { + let uri = build_chunk_anchor("s3://bucket/foo.zarr/2024", "subgroup/B01", &[1, 5]); + assert_eq!( + uri, + "s3://bucket/foo.zarr/2024#array=subgroup/B01&chunk=1,5" + ); + } + + #[test] + fn parse_anchor_basic() { + let anchor = + parse_chunk_anchor("file:///tmp/datacube.zarr#array=temperature&chunk=47,2,3").unwrap(); + assert_eq!(anchor.store_uri, "file:///tmp/datacube.zarr"); + assert_eq!(anchor.array_path, "temperature"); + assert_eq!(anchor.chunk_indices, vec![47, 2, 3]); + } + + #[test] + fn parse_anchor_with_s3_store_and_nested_array() { + let anchor = + parse_chunk_anchor("s3://bucket/foo.zarr/2024#array=subgroup/B01&chunk=0,0,0").unwrap(); + assert_eq!(anchor.store_uri, "s3://bucket/foo.zarr/2024"); + assert_eq!(anchor.array_path, "subgroup/B01"); + assert_eq!(anchor.chunk_indices, vec![0, 0, 0]); + } + + #[test] + fn parse_anchor_rejects_missing_fragment() { + // No `#` means no fragment, so neither `array=` nor `chunk=` is + // present. The parser surfaces "missing fragment". + let err = parse_chunk_anchor("file:///foo.zarr") + .unwrap_err() + .to_string(); + assert!(err.contains("fragment"), "{err}"); + } + + #[test] + fn parse_anchor_rejects_missing_array_param() { + let err = parse_chunk_anchor("file:///foo.zarr#chunk=0") + .unwrap_err() + .to_string(); + assert!(err.contains("array="), "{err}"); + } + + #[test] + fn parse_anchor_rejects_missing_chunk_param() { + let err = parse_chunk_anchor("file:///foo.zarr#array=a") + .unwrap_err() + .to_string(); + assert!(err.contains("chunk="), "{err}"); + } + + #[test] + fn parse_anchor_rejects_empty_indices() { + let err = parse_chunk_anchor("file:///foo.zarr#array=a&chunk=") + .unwrap_err() + .to_string(); + assert!(err.contains("empty"), "{err}"); + } + + #[test] + fn parse_anchor_rejects_non_integer_index() { + let err = parse_chunk_anchor("file:///foo.zarr#array=a&chunk=0,abc,2") + .unwrap_err() + .to_string(); + assert!(err.contains("abc"), "{err}"); + } + + #[test] + fn parse_anchor_ignores_unknown_fragment_params() { + // Forward-compatible: extra `&key=value` pairs in the fragment + // shouldn't break parsing — they're reserved for future use. + let anchor = parse_chunk_anchor("file:///foo.zarr#array=t&chunk=0,0&version=v3").unwrap(); + assert_eq!(anchor.array_path, "t"); + assert_eq!(anchor.chunk_indices, vec![0, 0]); + } + + #[test] + fn anchor_roundtrip() { + let original = ChunkAnchor { + store_uri: "file:///tmp/foo.zarr".into(), + array_path: "subgroup/B01".into(), + chunk_indices: vec![10, 20, 30], + }; + let uri = build_chunk_anchor( + &original.store_uri, + &original.array_path, + &original.chunk_indices, + ); + let parsed = parse_chunk_anchor(&uri).unwrap(); + assert_eq!(parsed, original); + } + + #[test] + fn group_uri_file_scheme() { + let path = group_uri_to_filesystem_path("file:///tmp/foo.zarr").unwrap(); + assert_eq!(path, std::path::PathBuf::from("/tmp/foo.zarr")); + } + + #[test] + fn group_uri_bare_path() { + let path = group_uri_to_filesystem_path("/tmp/foo.zarr").unwrap(); + assert_eq!(path, std::path::PathBuf::from("/tmp/foo.zarr")); + } + + #[test] + fn group_uri_cloud_scheme_errors() { + let err = group_uri_to_filesystem_path("s3://bucket/foo.zarr") + .unwrap_err() + .to_string(); + assert!(err.contains("s3://"), "{err}"); + assert!(err.contains("not supported"), "{err}"); + } +} diff --git a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs new file mode 100644 index 000000000..e4cb76267 --- /dev/null +++ b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs @@ -0,0 +1,232 @@ +// 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. + +//! End-to-end fixture test: build a small Zarr group on disk with the +//! `zarrs` crate, then read it back through `group_to_*_rasters` and +//! verify the resulting raster `StructArray`. + +use std::sync::Arc; + +use sedona_raster::array::RasterStructArray; +use sedona_raster::traits::RasterRef; +use sedona_raster_zarr::{group_to_indb_rasters, group_to_outdb_rasters}; +use sedona_schema::raster::BandDataType; +use tempfile::TempDir; +use zarrs::array::data_type; +use zarrs::array::ArrayBuilder; +use zarrs::group::GroupBuilder; +use zarrs_filesystem::FilesystemStore; + +/// Build a 2-band group on disk: +/// - dims: [t, y, x] +/// - shape: [2, 4, 4] +/// - chunks: [1, 2, 2] → chunk grid [2, 2, 2] = 8 chunk positions +/// - arrays: "temperature" (UInt8) and "pressure" (UInt8) +/// +/// Returns the temp dir (kept alive by the caller so files persist). +/// +/// `store_chunk_elements` is deprecated in zarrs 0.23 in favour of +/// `store_chunk` (which takes raw bytes); the typed convenience wrapper +/// is still the cleanest path for fixture code so we suppress the +/// warning here. +#[allow(deprecated)] +fn build_fixture() -> TempDir { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + + // Group with a known affine transform so we can verify per-chunk + // transforms below. + let mut group_attrs = serde_json::Map::new(); + group_attrs.insert( + "spatial:transform".into(), + serde_json::json!([100.0, 1.0, 0.0, 200.0, 0.0, -1.0]), + ); + group_attrs.insert("proj:epsg".into(), serde_json::json!(4326)); + GroupBuilder::new() + .attributes(group_attrs) + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + + for (name, base) in [("temperature", 0u8), ("pressure", 100u8)] { + let array = ArrayBuilder::new( + vec![2u64, 4u64, 4u64], + vec![1u64, 2u64, 2u64], + data_type::uint8(), + 0u8, + ) + .dimension_names(Some(["t", "y", "x"])) + .build(store.clone(), &format!("/{name}")) + .unwrap(); + array.store_metadata().unwrap(); + + // Fill each chunk with a deterministic pattern so we can verify + // the right chunk lands in the right row: + // pixel(t, y, x) = base + (t*16 + y*4 + x) + // Each chunk is 1×2×2 = 4 pixels. Chunk (t_idx, y_idx, x_idx) + // covers (t_idx, [2*y_idx..2*y_idx+2], [2*x_idx..2*x_idx+2]). + for t in 0..2u64 { + for yc in 0..2u64 { + for xc in 0..2u64 { + let mut chunk = Vec::with_capacity(4); + for dy in 0..2u64 { + for dx in 0..2u64 { + let y = yc * 2 + dy; + let x = xc * 2 + dx; + chunk.push(base.wrapping_add((t * 16 + y * 4 + x) as u8)); + } + } + array + .store_chunk_elements::(&[t, yc, xc], &chunk) + .unwrap(); + } + } + } + } + + tmp +} + +#[test] +fn indb_round_trip_emits_one_row_per_chunk_position() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + let arr = group_to_indb_rasters(&uri).unwrap(); + + let rasters = RasterStructArray::new(&arr); + assert_eq!(rasters.len(), 8, "expected 8 chunk rows (2*2*2)"); + + // First row corresponds to chunk (t=0, y=0, x=0). With group transform + // [100, 1, 0, 200, 0, -1] and chunk shape [1, 2, 2], chunk (0,0,0) has + // origin (100, 200) and spatial_shape [2, 2]. + let r0 = rasters.get(0).unwrap(); + let r0_transform: Vec = r0.transform().to_vec(); + assert_eq!(r0_transform, vec![100.0, 1.0, 0.0, 200.0, 0.0, -1.0]); + assert_eq!(r0.spatial_shape(), &[2, 2]); + assert_eq!(r0.num_bands(), 2); + assert_eq!(r0.crs(), Some("EPSG:4326")); + + // Bands are sorted by array path for determinism — `pressure` sorts + // before `temperature` lexicographically, so band 0 is pressure and + // band 1 is temperature. + // + // Pressure has base=100; chunk (t=0, y=0, x=0) covers y∈{0,1}, x∈{0,1} + // → pixel offsets {0, 1, 4, 5} → values {100, 101, 104, 105}. + let pressure = r0.band(0).unwrap(); + assert_eq!(pressure.raw_source_shape(), &[1, 2, 2]); + assert_eq!(pressure.data_type(), BandDataType::UInt8); + assert!(pressure.is_indb()); + assert_eq!( + &*pressure.contiguous_data().unwrap(), + &[100u8, 101, 104, 105] + ); + + // Temperature has base=0 → same chunk holds {0, 1, 4, 5}. + let temperature = r0.band(1).unwrap(); + assert_eq!(&*temperature.contiguous_data().unwrap(), &[0u8, 1, 4, 5]); + + // Last row corresponds to chunk (t=1, y=1, x=1). Temperature pixels: + // t=1, y∈{2,3}, x∈{2,3} → 1*16 + y*4 + x → 26, 27, 30, 31. + let last = rasters.get(7).unwrap(); + let last_transform: Vec = last.transform().to_vec(); + assert_eq!(last_transform[0], 100.0 + 2.0); // x_off = 2 + assert_eq!(last_transform[3], 200.0 - 2.0); // y_off = 2, sy = -1 + // band 1 is temperature (per the sort-by-path order above). + let last_temp = last.band(1).unwrap(); + assert_eq!(&*last_temp.contiguous_data().unwrap(), &[26u8, 27, 30, 31]); +} + +#[test] +fn outdb_emits_chunk_anchors() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + let arr = group_to_outdb_rasters(&uri).unwrap(); + + let rasters = RasterStructArray::new(&arr); + assert_eq!(rasters.len(), 8); + + // OutDb rows have empty data column and chunk anchor URIs. + // Bands sort alphabetically by array path: pressure (band 0), then + // temperature (band 1). + let r0 = rasters.get(0).unwrap(); + let pressure = r0.band(0).unwrap(); + assert!( + !pressure.is_indb(), + "OutDb band must report is_indb() = false" + ); + // "This is zarr" lives in outdb_format, not a URI scheme prefix. + assert_eq!(pressure.outdb_format(), Some("zarr")); + let anchor = pressure.outdb_uri().expect("outdb_uri set"); + // Anchor is the group URI verbatim plus a fragment carrying array + // path + chunk indices. No `zarr://` prefix. + assert!(anchor.starts_with("file://"), "got: {anchor}"); + assert!(!anchor.starts_with("zarr://"), "got: {anchor}"); + assert!(anchor.contains("#array=pressure"), "got: {anchor}"); + assert!(anchor.contains("&chunk=0,0,0"), "got: {anchor}"); + + // Last chunk position's temperature band points at chunk (1,1,1). + let last = rasters.get(7).unwrap(); + let temp = last.band(1).unwrap(); + let anchor = temp.outdb_uri().expect("outdb_uri set"); + assert!(anchor.contains("#array=temperature"), "got: {anchor}"); + assert!(anchor.contains("&chunk=1,1,1"), "got: {anchor}"); +} + +#[test] +fn errors_on_empty_group() { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + let uri = format!("file://{}", tmp.path().display()); + let err = group_to_indb_rasters(&uri).unwrap_err().to_string(); + assert!(err.contains("no child arrays"), "got: {err}"); +} + +#[test] +fn errors_on_mismatched_chunk_grids() { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + ArrayBuilder::new(vec![4u64, 4], vec![2u64, 2], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/array_a") + .unwrap() + .store_metadata() + .unwrap(); + ArrayBuilder::new(vec![4u64, 4], vec![4u64, 4], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/array_b") + .unwrap() + .store_metadata() + .unwrap(); + + let uri = format!("file://{}", tmp.path().display()); + let err = group_to_indb_rasters(&uri).unwrap_err().to_string(); + assert!( + err.contains("chunk") && err.contains("array_a") && err.contains("array_b"), + "got: {err}" + ); +} diff --git a/rust/sedona/Cargo.toml b/rust/sedona/Cargo.toml index ef5fb9e85..662a8d30c 100644 --- a/rust/sedona/Cargo.toml +++ b/rust/sedona/Cargo.toml @@ -50,6 +50,14 @@ s2geography = ["dep:sedona-s2geography", "dep:sedona-spatial-join-geography", "s tempfile = { workspace = true } tokio = { workspace = true, features = ["rt-multi-thread"] } rstest = { workspace = true } +sedona-raster = { workspace = true } +zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } +zarrs_filesystem = { workspace = true } + +# See sedona-raster-zarr/Cargo.toml — `blosc` is gated off Windows to avoid +# a libwinpthreads/c-blosc symbol collision in the R `sedonadb.dll` link. +[target.'cfg(not(target_os = "windows"))'.dev-dependencies] +zarrs = { workspace = true, features = ["blosc"] } [dependencies] arrow-schema = { workspace = true } @@ -82,6 +90,7 @@ sedona-proj = { workspace = true } sedona-gdal = { workspace = true } sedona-raster-functions = { workspace = true } sedona-raster-gdal = { workspace = true } +sedona-raster-zarr = { workspace = true } sedona-schema = { workspace = true } sedona-spatial-join = { workspace = true, optional = true } sedona-spatial-join-gpu = { workspace = true, optional = true } diff --git a/rust/sedona/src/context.rs b/rust/sedona/src/context.rs index 156b5b355..25840d36b 100644 --- a/rust/sedona/src/context.rs +++ b/rust/sedona/src/context.rs @@ -25,6 +25,7 @@ use crate::{ catalog::DynamicObjectStoreCatalog, random_geometry_provider::RandomGeometryFunction, show::{show_batches, DisplayTableOptions}, + zarr_read::ZarrReadFunction, }; use arrow_array::RecordBatch; use async_trait::async_trait; @@ -232,6 +233,8 @@ impl SedonaContext { "sd_random_geometry", Arc::new(RandomGeometryFunction::default()), ); + out.ctx + .register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); // Always register default function set out.register_function_set(sedona_functions::register::default_function_set()); diff --git a/rust/sedona/src/lib.rs b/rust/sedona/src/lib.rs index 81c103f9f..356ba4d9f 100644 --- a/rust/sedona/src/lib.rs +++ b/rust/sedona/src/lib.rs @@ -27,3 +27,4 @@ pub mod reader; pub mod record_batch_reader_provider; pub mod show; pub mod size_parser; +pub mod zarr_read; diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona/src/zarr_read.rs new file mode 100644 index 000000000..d248e37cb --- /dev/null +++ b/rust/sedona/src/zarr_read.rs @@ -0,0 +1,492 @@ +// 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. + +//! `sd_read_zarr` table function — exposes Zarr-backed raster groups +//! as SQL-queryable tables. +//! +//! ```sql +//! SELECT * FROM sd_read_zarr('file:///path/to/datacube.zarr'); +//! SELECT count(*) FROM sd_read_zarr( +//! 'file:///path/to/datacube.zarr', +//! '{"mode": "indb", "rows_per_batch": 256}' +//! ); +//! ``` +//! +//! Returns a single-column table `raster: Raster` with one row per chunk +//! position in the Zarr group's chunk grid. All existing `RS_*` UDFs +//! operate on the column unchanged. +//! +//! Phase 1 defaults to `mode = "indb"` so byte-reading kernels work +//! end-to-end without depending on the (not-yet-registered) OutDb +//! resolver. The default flips to `outdb` once the format-keyed +//! dispatcher lands. + +use std::any::Any; +use std::sync::Arc; + +use arrow_array::{Array, RecordBatch, StructArray}; +use arrow_schema::{DataType, Schema, SchemaRef}; +use async_trait::async_trait; +use datafusion::catalog::{Session, TableFunctionImpl, TableProvider}; +use datafusion::common::Result; +use datafusion::datasource::TableType; +use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::expressions::Column; +use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, PlanProperties, + SendableRecordBatchStream, +}; +use datafusion::prelude::Expr; +use datafusion_common::{plan_err, DataFusionError, ScalarValue}; +use sedona_raster_zarr::{group_to_indb_rasters, group_to_outdb_rasters}; +use sedona_schema::datatypes::SedonaType; +use serde::{Deserialize, Serialize}; + +/// Table function `sd_read_zarr(uri[, options_json])`. +/// +/// Accepts one or two string arguments: +/// - `uri` (required) — Zarr group URI (e.g. `file:///path/to/foo.zarr`). +/// - `options_json` (optional) — JSON string with any of: +/// - `mode`: `"indb"` (default) or `"outdb"` +/// - `rows_per_batch`: chunks per `RecordBatch` (default 1024) +/// - `num_partitions`: scan partitions (default 1; > 1 errors in Phase 1) +#[derive(Debug, Default)] +pub struct ZarrReadFunction {} + +impl TableFunctionImpl for ZarrReadFunction { + fn call(&self, exprs: &[Expr]) -> Result> { + if exprs.is_empty() || exprs.len() > 2 { + return plan_err!( + "sd_read_zarr() expects 1 or 2 string arguments (uri[, options_json]); got {}", + exprs.len() + ); + } + + let uri = literal_utf8(&exprs[0], "sd_read_zarr() uri")?; + let options = if exprs.len() == 2 { + Some(literal_utf8(&exprs[1], "sd_read_zarr() options_json")?) + } else { + None + }; + + Ok(Arc::new(ZarrChunkProvider::try_new(&uri, options)?)) + } +} + +/// Pull a `Utf8` literal out of an `Expr`. UDTF call arguments arrive as +/// `Expr::Literal(ScalarValue, _)`; anything else errors with a planner +/// message naming the parameter. +fn literal_utf8(expr: &Expr, label: &str) -> Result { + if let Expr::Literal(scalar, _) = expr { + if let ScalarValue::Utf8(Some(s)) = scalar.cast_to(&DataType::Utf8)? { + return Ok(s); + } + } + plan_err!("{label} must be a non-null Utf8 literal; got {expr}") +} + +/// Materialised view backing `sd_read_zarr`. Holds the raster +/// `StructArray` once for the query lifetime; the executor slices it +/// into `rows_per_batch`-sized `RecordBatch`es. +/// +/// Phase 1 builds the StructArray eagerly in `try_new`. For OutDb mode +/// the array is cheap to construct (chunk anchor URIs only); for InDb +/// mode it pulls every chunk's bytes through the loader at plan time. +#[derive(Debug)] +pub struct ZarrChunkProvider { + schema: SchemaRef, + rasters: StructArray, + rows_per_batch: usize, +} + +impl ZarrChunkProvider { + fn try_new(uri: &str, options_json: Option) -> Result { + let opts = parse_options(options_json.as_deref())?; + let mode = opts.mode.as_deref().unwrap_or("indb"); + let rows_per_batch = opts.rows_per_batch.unwrap_or(1024).max(1); + let num_partitions = opts.num_partitions.unwrap_or(1); + if num_partitions != 1 { + return plan_err!( + "sd_read_zarr() Phase 1 supports only num_partitions = 1; got {num_partitions}. \ + Round-robin partitioning lands with the OutDb resolver work." + ); + } + + let rasters = match mode { + "indb" => group_to_indb_rasters(uri).map_err(arrow_to_df_err)?, + "outdb" => group_to_outdb_rasters(uri).map_err(arrow_to_df_err)?, + other => { + return plan_err!( + "sd_read_zarr() mode must be \"indb\" or \"outdb\"; got {other:?}" + ); + } + }; + + // Single-column schema: `raster: Raster`. `SedonaType::Raster` adds + // the `sedona.raster` extension-type metadata so downstream RS_* + // kernels recognise the column without further configuration. + let raster_field = SedonaType::Raster + .to_storage_field("raster", true) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + let schema = Arc::new(Schema::new(vec![raster_field])); + + Ok(Self { + schema, + rasters, + rows_per_batch, + }) + } +} + +#[async_trait] +impl TableProvider for ZarrChunkProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::View + } + + async fn scan( + &self, + _state: &dyn Session, + projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + let exec = Arc::new(ZarrChunkExec::new( + self.schema.clone(), + self.rasters.clone(), + self.rows_per_batch, + )); + // DataFusion requires the scan to honour the projection it asks + // for, including the empty projection used by `SELECT count(*)`. + // Wrap the exec in a `ProjectionExec` so the physical schema + // matches the requested column subset. + if let Some(projection) = projection { + let schema = self.schema(); + let exprs: Vec<(Arc, String)> = projection + .iter() + .map(|index| { + let name = schema.field(*index).name(); + let expr: Arc = Arc::new(Column::new(name, *index)); + (expr, name.clone()) + }) + .collect(); + Ok(Arc::new(ProjectionExec::try_new(exprs, exec)?)) + } else { + Ok(exec) + } + } +} + +#[derive(Debug)] +struct ZarrChunkExec { + schema: SchemaRef, + rasters: StructArray, + rows_per_batch: usize, + properties: PlanProperties, +} + +impl ZarrChunkExec { + fn new(schema: SchemaRef, rasters: StructArray, rows_per_batch: usize) -> Self { + let properties = PlanProperties::new( + EquivalenceProperties::new(schema.clone()), + // Phase 1: a single partition. Round-robin across multiple + // partitions lands with the OutDb resolver work. + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + ); + Self { + schema, + rasters, + rows_per_batch, + properties, + } + } +} + +impl DisplayAs for ZarrChunkExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!( + f, + "ZarrChunkExec: rows={}, rows_per_batch={}", + self.rasters.len(), + self.rows_per_batch, + ) + } +} + +impl ExecutionPlan for ZarrChunkExec { + fn name(&self) -> &str { + "ZarrChunkExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + Vec::new() + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + if partition != 0 { + return plan_err!("ZarrChunkExec: only partition 0 exists (Phase 1)"); + } + + let total = self.rasters.len(); + let rows_per_batch = self.rows_per_batch; + let schema = self.schema.clone(); + let rasters = self.rasters.clone(); + + // Build all batches eagerly into a Vec, then turn into a stream. + // This is fine for Phase 1: the StructArray is already materialised + // in the provider, slicing is O(1), and the only allocation per + // batch is the RecordBatch wrapper. Lazy streaming becomes + // interesting once the loader itself is lazy. + let mut batches = Vec::with_capacity(total.div_ceil(rows_per_batch).max(1)); + let mut offset = 0; + while offset < total { + let len = (total - offset).min(rows_per_batch); + let slice = rasters.slice(offset, len); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(slice)])?; + batches.push(Ok(batch)); + offset += len; + } + + let stream = futures::stream::iter(batches); + let adapter = RecordBatchStreamAdapter::new(schema, stream); + Ok(Box::pin(adapter)) + } +} + +/// Convert an `ArrowError` from the Zarr loader into a `DataFusionError` +/// suitable for planner output. +fn arrow_to_df_err(e: arrow_schema::ArrowError) -> DataFusionError { + DataFusionError::External(Box::new(e)) +} + +#[derive(Serialize, Deserialize, Default)] +struct ZarrReadOptions { + mode: Option, + rows_per_batch: Option, + num_partitions: Option, +} + +fn parse_options(options_json: Option<&str>) -> Result { + let Some(s) = options_json else { + return Ok(ZarrReadOptions::default()); + }; + serde_json::from_str(s).map_err(|e| { + DataFusionError::Plan(format!( + "sd_read_zarr() options must be valid JSON: {e}\noptions were: {s}" + )) + }) +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion::prelude::SessionContext; + use sedona_raster::array::RasterStructArray; + use sedona_raster::traits::RasterRef; + use std::sync::Arc; + use tempfile::TempDir; + use zarrs::array::data_type; + use zarrs::array::ArrayBuilder; + use zarrs::group::GroupBuilder; + use zarrs_filesystem::FilesystemStore; + + /// Build a tiny 1-array Zarr group on disk and return the temp dir. + /// 2×2 UInt8 with chunks [1, 2] → chunk grid [2, 1] = 2 chunk rows. + #[allow(deprecated)] + fn build_fixture() -> TempDir { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + let array = ArrayBuilder::new(vec![2u64, 2u64], vec![1u64, 2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/temperature") + .unwrap(); + array.store_metadata().unwrap(); + array + .store_chunk_elements::(&[0, 0], &[10u8, 11]) + .unwrap(); + array + .store_chunk_elements::(&[1, 0], &[20u8, 21]) + .unwrap(); + tmp + } + + #[tokio::test] + async fn udtf_returns_one_row_per_chunk_position_with_pixel_bytes() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + let df = ctx + .sql(&format!("SELECT raster FROM sd_read_zarr('{uri}')")) + .await + .unwrap(); + let batches = df.collect().await.unwrap(); + + let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); + assert_eq!(total_rows, 2, "expected 2 chunk rows"); + + // Pull the raster column out, hand to RasterStructArray, verify + // chunk 0's bytes round-trip. + let raster_col = batches[0].column(0); + let struct_arr = raster_col + .as_any() + .downcast_ref::() + .expect("raster column is StructArray"); + let rasters = RasterStructArray::new(struct_arr); + let r0 = rasters.get(0).unwrap(); + let band = r0.band(0).unwrap(); + assert_eq!(&*band.contiguous_data().unwrap(), &[10u8, 11]); + } + + #[tokio::test] + async fn udtf_respects_rows_per_batch_option() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + // 2 chunk rows, rows_per_batch=1 → 2 single-row batches. + let df = ctx + .sql(&format!( + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"rows_per_batch": 1}}')"#, + )) + .await + .unwrap(); + let batches = df.collect().await.unwrap(); + assert_eq!(batches.len(), 2); + assert!(batches.iter().all(|b| b.num_rows() == 1)); + } + + #[tokio::test] + async fn udtf_count_works_without_reading_pixel_bytes() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + // OutDb mode: byte fetching is deferred. SELECT count(*) just + // walks the chunk grid metadata. + let df = ctx + .sql(&format!( + r#"SELECT count(*) FROM sd_read_zarr('{uri}', '{{"mode": "outdb"}}')"#, + )) + .await + .unwrap(); + let batches = df.collect().await.unwrap(); + let count_arr = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(count_arr.value(0), 2); + } + + #[tokio::test] + async fn udtf_rejects_unknown_mode() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + let err = ctx + .sql(&format!( + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"mode": "lazy"}}')"#, + )) + .await + .unwrap_err() + .to_string(); + assert!(err.contains("\"indb\" or \"outdb\""), "got: {err}"); + } + + #[tokio::test] + async fn udtf_rejects_multi_partition_in_phase1() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + let err = ctx + .sql(&format!( + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"num_partitions": 2}}')"#, + )) + .await + .unwrap_err() + .to_string(); + assert!(err.contains("num_partitions = 1"), "got: {err}"); + } + + #[tokio::test] + async fn udtf_rejects_malformed_options_json() { + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + let err = ctx + .sql(r#"SELECT raster FROM sd_read_zarr('file:///nowhere', '{not json}')"#) + .await + .unwrap_err() + .to_string(); + assert!(err.contains("must be valid JSON"), "got: {err}"); + } +} From 3fd6420ee7e02a87190b9f1b40041f44f84dc66d Mon Sep 17 00:00:00 2001 From: jameswillis Date: Mon, 18 May 2026 21:49:28 -0700 Subject: [PATCH 02/36] feat(python/sedonadb): add sd_read_zarr Python wrapper MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds a `con.funcs.table.sd_read_zarr(uri, *, mode, rows_per_batch, num_partitions)` Python method that mirrors the SQL UDTF added in the companion Rust commit. Same shape as the existing `sd_random_geometry` wrapper: builds the JSON options object, filters out None entries, and runs the SQL UDTF through `self._ctx.sql`. Python test uses `pytest.importorskip("zarr")` so it cleanly skips when the optional `zarr` library isn't installed in the test environment. Builds a 2×2 UInt8 Zarr group on disk via the official `zarr` Python lib and exercises both the default-options form and the `rows_per_batch` override. --- .../python/sedonadb/functions/table.py | 53 +++++++++++++++++++ python/sedonadb/tests/test_funcs.py | 32 +++++++++++ 2 files changed, 85 insertions(+) diff --git a/python/sedonadb/python/sedonadb/functions/table.py b/python/sedonadb/python/sedonadb/functions/table.py index 30a2f285d..c5c517499 100644 --- a/python/sedonadb/python/sedonadb/functions/table.py +++ b/python/sedonadb/python/sedonadb/functions/table.py @@ -112,3 +112,56 @@ def sd_random_geometry( args = {k: v for k, v in args.items() if v is not None} return self._ctx.sql(f"SELECT * FROM sd_random_geometry('{json.dumps(args)}')") + + def sd_read_zarr( + self, + uri: str, + *, + mode: Optional[Literal["indb", "outdb"]] = None, + rows_per_batch: Optional[int] = None, + num_partitions: Optional[int] = None, + ) -> DataFrame: + """ + Read a Zarr group as a DataFrame of N-D rasters. + + Returns a single-column DataFrame ``raster: Raster`` with one row per + chunk position in the Zarr group's chunk grid. Each row's bands are + the corresponding chunks of each array in the group. All ``RS_*`` + UDFs operate on the column unchanged. + + Phase 1 supports local filesystem stores only (``file://`` URIs or + bare paths); cloud schemes (``s3://`` / ``gs://`` / ``az://`` / + ``https://``) error pending the OutDb resolver work. + + Parameters + ---------- + uri : str + Zarr group URI. ``file:///path/to/foo.zarr`` or a bare local path. + mode : {"indb", "outdb"}, optional + ``"indb"`` (default) materializes every chunk's bytes into the + Arrow ``data`` column eagerly. ``"outdb"`` emits chunk-anchor + URIs only; byte resolution depends on the OutDb resolver being + registered (follow-up PR). + rows_per_batch : int, optional + Chunks per ``RecordBatch`` (default 1024). + num_partitions : int, optional + Scan partitions. Phase 1 supports only 1; > 1 errors. + + Examples + -------- + >>> sd = sedona.db.connect() + >>> sd.funcs.table.sd_read_zarr("file:///path/to/datacube.zarr") # doctest: +SKIP + """ + + args = { + "mode": mode, + "rows_per_batch": rows_per_batch, + "num_partitions": num_partitions, + } + args = {k: v for k, v in args.items() if v is not None} + + if args: + return self._ctx.sql( + f"SELECT * FROM sd_read_zarr('{uri}', '{json.dumps(args)}')" + ) + return self._ctx.sql(f"SELECT * FROM sd_read_zarr('{uri}')") diff --git a/python/sedonadb/tests/test_funcs.py b/python/sedonadb/tests/test_funcs.py index 2c6fb9956..a846e3d7d 100644 --- a/python/sedonadb/tests/test_funcs.py +++ b/python/sedonadb/tests/test_funcs.py @@ -24,3 +24,35 @@ def test_random_geometry(con): # Ensure the output is reproducible assert df.to_arrow_table() == df.to_arrow_table() + + +def test_read_zarr(con, tmp_path): + # Skip cleanly if the optional `zarr` Python lib isn't installed — + # the binding is exercised by the Rust-side integration tests; this + # test only verifies the Python wrapper threads arguments through. + import pytest + + zarr = pytest.importorskip("zarr") + np = pytest.importorskip("numpy") + + # Build a 2x2 UInt8 array with two chunks, dim_names=["y","x"], inside + # a group at the temp path. Matches the minimal shape sd_read_zarr + # expects (2-D array with [y, x] suffix). + root = zarr.open_group(str(tmp_path), mode="w") + arr = root.create_array( + "temperature", + shape=(2, 2), + chunks=(1, 2), + dtype="uint8", + dimension_names=["y", "x"], + ) + arr[:] = np.array([[10, 11], [20, 21]], dtype=np.uint8) + + # Default-mode (InDb) read — every chunk materialized into the + # Arrow `data` column. + df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}") + assert df.count() == 2 + + # Options thread through: rows_per_batch slices the output. + df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}", rows_per_batch=1) + assert df.count() == 2 From ca477865d06ef98a2ea40e6f68b6dba35c1b559f Mon Sep 17 00:00:00 2001 From: jameswillis Date: Tue, 19 May 2026 13:42:28 -0700 Subject: [PATCH 03/36] feat(raster-zarr): review fixes for sd_read_zarr MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Round of review feedback on the sd_read_zarr loader and UDTF surface: - Drop "Phase 1" references from code, error messages, and docstrings; describe behaviour in absolute terms. - Bound-check the chunk-grid product so a malformed or hostile Zarr can't drive RasterBuilder capacity to overflow / OOM. - Always populate `outdb_uri` and `outdb_format` on every band as provenance metadata; InDb vs OutDb stays discriminated by `data.is_empty()`. - Fall back to xarray's `_ARRAY_DIMENSIONS` attribute when an array lacks the first-class `dimension_names` field (covers Zarr v2 and any v3 file that uses the xarray convention). - Auto-skip 1-D arrays at selection time (typical xarray coord variables) — they can never produce a valid raster band, so reading them would always fail downstream at spatial-dim resolution. - Add an `arrays = [...]` option (Python kwarg + UDTF JSON key) to read a named subset of arrays; unknown names and 1-D arrays both error early with clear messages pointing at the option. - Error loudly when a group declares a CRS but no `spatial:transform` attribute — the high-confidence-bug case where falling back to the identity transform would silently produce wrong spatial-join results. - Warn-log when both CRS and transform are absent and we fall back to pixel-coordinate identity, so spatial-join surprises are debuggable from logs. - Flip the InDb/OutDb selector from `mode: "indb"|"outdb"` to `indb: bool` end-to-end (Python kwarg, SQL JSON key, internal options struct). The schema-level discriminator is binary, so a boolean is the more honest UI. Tests cover every new branch: auto-skip, explicit array filter, 1-D rejection, _ARRAY_DIMENSIONS fallback, CRS-without-transform error, indb-bool plumbing through SQL. --- Cargo.lock | 1 + .../python/sedonadb/functions/table.py | 36 ++- rust/sedona-raster-zarr/Cargo.toml | 1 + rust/sedona-raster-zarr/src/geozarr.rs | 10 +- rust/sedona-raster-zarr/src/lib.rs | 3 +- rust/sedona-raster-zarr/src/loader.rs | 281 ++++++++++++++---- rust/sedona-raster-zarr/src/source_uri.rs | 17 +- .../tests/zarr_roundtrip.rs | 207 ++++++++++++- rust/sedona/src/zarr_read.rs | 145 ++++++--- 9 files changed, 573 insertions(+), 128 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 158cdb45f..c3eb474ac 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6160,6 +6160,7 @@ dependencies = [ "arrow-array", "arrow-schema", "datafusion-common", + "log", "sedona-common", "sedona-raster", "sedona-schema", diff --git a/python/sedonadb/python/sedonadb/functions/table.py b/python/sedonadb/python/sedonadb/functions/table.py index c5c517499..e79aa5c84 100644 --- a/python/sedonadb/python/sedonadb/functions/table.py +++ b/python/sedonadb/python/sedonadb/functions/table.py @@ -16,7 +16,7 @@ # under the License. import json -from typing import Optional, Literal, Union, Tuple, Iterable +from typing import Optional, Literal, Union, Tuple, Iterable, List from sedonadb.dataframe import DataFrame from sedonadb.utility import sedona # noqa: F401 @@ -117,9 +117,10 @@ def sd_read_zarr( self, uri: str, *, - mode: Optional[Literal["indb", "outdb"]] = None, + indb: Optional[bool] = None, rows_per_batch: Optional[int] = None, num_partitions: Optional[int] = None, + arrays: Optional[List[str]] = None, ) -> DataFrame: """ Read a Zarr group as a DataFrame of N-D rasters. @@ -129,34 +130,45 @@ def sd_read_zarr( the corresponding chunks of each array in the group. All ``RS_*`` UDFs operate on the column unchanged. - Phase 1 supports local filesystem stores only (``file://`` URIs or - bare paths); cloud schemes (``s3://`` / ``gs://`` / ``az://`` / - ``https://``) error pending the OutDb resolver work. + Only local filesystem stores are supported (``file://`` URIs or + bare paths). Parameters ---------- uri : str Zarr group URI. ``file:///path/to/foo.zarr`` or a bare local path. - mode : {"indb", "outdb"}, optional - ``"indb"`` (default) materializes every chunk's bytes into the - Arrow ``data`` column eagerly. ``"outdb"`` emits chunk-anchor - URIs only; byte resolution depends on the OutDb resolver being - registered (follow-up PR). + indb : bool, optional + ``True`` (default) materializes every chunk's bytes into the + Arrow ``data`` column eagerly. ``False`` emits chunk-anchor + URIs only; byte resolution depends on the OutDb resolver + being registered (follow-up PR). rows_per_batch : int, optional Chunks per ``RecordBatch`` (default 1024). num_partitions : int, optional - Scan partitions. Phase 1 supports only 1; > 1 errors. + Scan partitions. Only ``1`` is supported; ``> 1`` errors. + arrays : list of str, optional + Names of arrays in the group to read. By default every + multi-dimensional array is read; 1-D arrays (typical xarray + coord variables) are auto-skipped. Passing an explicit list + reads exactly those arrays. 1-D arrays are always rejected + (a raster band needs at least 2 dimensions); naming one + explicitly errors. Unknown names also error. Examples -------- >>> sd = sedona.db.connect() >>> sd.funcs.table.sd_read_zarr("file:///path/to/datacube.zarr") # doctest: +SKIP + >>> sd.funcs.table.sd_read_zarr( + ... "file:///path/to/datacube.zarr", + ... arrays=["temperature", "pressure"], + ... ) # doctest: +SKIP """ args = { - "mode": mode, + "indb": indb, "rows_per_batch": rows_per_batch, "num_partitions": num_partitions, + "arrays": arrays, } args = {k: v for k, v in args.items() if v is not None} diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml index 1465d3086..892806064 100644 --- a/rust/sedona-raster-zarr/Cargo.toml +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -34,6 +34,7 @@ result_large_err = "allow" arrow-array = { workspace = true } arrow-schema = { workspace = true } datafusion-common = { workspace = true } +log = { workspace = true } sedona-common = { workspace = true } sedona-raster = { workspace = true } sedona-schema = { workspace = true } diff --git a/rust/sedona-raster-zarr/src/geozarr.rs b/rust/sedona-raster-zarr/src/geozarr.rs index 82427b997..63c0eb8f0 100644 --- a/rust/sedona-raster-zarr/src/geozarr.rs +++ b/rust/sedona-raster-zarr/src/geozarr.rs @@ -21,9 +21,9 @@ //! mapping) attribute conventions from a Zarr group's attributes, mapping //! them onto SedonaDB's per-raster `crs` and `transform` fields. //! -//! Phase 1 expects attributes at the group level and inherits them across -//! every array. Per-array overrides are rejected by the group-constraint -//! validator (see `loader`). +//! Attributes live at the group level and are inherited by every array. +//! Per-array overrides are rejected by the group-constraint validator +//! (see `loader`). use arrow_schema::ArrowError; @@ -49,8 +49,8 @@ impl GroupGeoMetadata { /// surfaces from a group) into a `GroupGeoMetadata`. /// /// Returns `Ok(default-empty)` when none of the conventional keys are - /// present — Phase 1 treats geospatial metadata as optional; downstream - /// fall-backs in the loader provide identity transforms when needed. + /// present — geospatial metadata is optional; downstream fall-backs in + /// the loader provide identity transforms when needed. pub fn from_attributes( attrs: &serde_json::Map, ) -> Result { diff --git a/rust/sedona-raster-zarr/src/lib.rs b/rust/sedona-raster-zarr/src/lib.rs index 42790a7fc..06c8b922b 100644 --- a/rust/sedona-raster-zarr/src/lib.rs +++ b/rust/sedona-raster-zarr/src/lib.rs @@ -30,8 +30,7 @@ //! fetch on demand through the process-wide OutDb loader (registered //! separately via `sedona-raster`'s loader hook). //! -//! Phase 1 supports local filesystem stores only; cloud backends arrive -//! with the resolver work. +//! Local filesystem stores only — `file://` URIs or bare paths. pub mod dtype; pub mod geozarr; diff --git a/rust/sedona-raster-zarr/src/loader.rs b/rust/sedona-raster-zarr/src/loader.rs index 5caa45322..021a1e4b0 100644 --- a/rust/sedona-raster-zarr/src/loader.rs +++ b/rust/sedona-raster-zarr/src/loader.rs @@ -47,17 +47,33 @@ use crate::source_uri::{build_chunk_anchor, group_uri_to_filesystem_path}; /// Open a Zarr group and eagerly fetch every chunk's bytes into the /// returned `StructArray`. Each row holds one chunk position's data -/// across every array in the group. -pub fn group_to_indb_rasters(group_uri: &str) -> Result { - build_rasters(group_uri, Mode::InDb) +/// across the selected arrays in the group. +/// +/// `arrays`: +/// - `None` — read every multi-dimensional array. 1-D arrays (typical +/// coord variables in xarray-style datacubes) are auto-skipped. +/// - `Some(names)` — read exactly the named arrays, in the order +/// returned by the store's listing (which is then sorted by path). An +/// unknown name errors. 1-D arrays are always rejected (a raster band +/// needs ≥ 2 dimensions); naming one explicitly errors at parse time +/// with a clear message. +pub fn group_to_indb_rasters( + group_uri: &str, + arrays: Option<&[String]>, +) -> Result { + build_rasters(group_uri, Mode::InDb, arrays) } /// Open a Zarr group and emit one row per chunk position with chunk-anchor /// URIs in each band's `outdb_uri`. The `data` column is empty; bytes /// resolve on demand through whichever OutDb loader is registered for -/// the `zarr` format. -pub fn group_to_outdb_rasters(group_uri: &str) -> Result { - build_rasters(group_uri, Mode::OutDb) +/// the `zarr` format. See [`group_to_indb_rasters`] for the meaning of +/// `arrays`. +pub fn group_to_outdb_rasters( + group_uri: &str, + arrays: Option<&[String]>, +) -> Result { + build_rasters(group_uri, Mode::OutDb, arrays) } #[derive(Debug, Clone, Copy, PartialEq, Eq)] @@ -84,15 +100,19 @@ struct ArrayInfo { /// chunk positions and validated to match across arrays. chunk_grid_shape: Vec, /// Chunk shape (elements per chunk per dim). Same for every chunk - /// position in Phase 1 (no ragged final chunks emitted as separate - /// short rows). + /// position — ragged final chunks are not emitted as separate short + /// rows. chunk_shape: Vec, /// Encoded fill value in native-endian byte representation, for the /// `nodata` field. None when the array has no fill value declared. nodata: Option>, } -fn build_rasters(group_uri: &str, mode: Mode) -> Result { +fn build_rasters( + group_uri: &str, + mode: Mode, + arrays_filter: Option<&[String]>, +) -> Result { let fs_path = group_uri_to_filesystem_path(group_uri)?; let store = FilesystemStore::new(&fs_path).map_err(|e| { ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( @@ -110,6 +130,20 @@ fn build_rasters(group_uri: &str, mode: Mode) -> Result let geo = GroupGeoMetadata::from_attributes(group.attributes())?; + // CRS-without-transform is almost always a malformed-metadata bug — + // the user thinks they have full georef but downstream spatial joins + // will silently use the identity-pixel-coords default. Error loudly + // so they fix the metadata rather than getting empty result sets. + if geo.crs.is_some() && geo.transform.is_none() { + return Err(ArrowError::InvalidArgumentError(format!( + "Zarr group at {group_uri} declares a CRS but no `spatial:transform` \ + attribute; refusing to fall back to the identity transform because \ + that would silently produce wrong results in spatial joins. Declare \ + `spatial:transform` on the group or remove the CRS to read this as \ + a non-georeferenced datacube." + ))); + } + let arrays = group.child_arrays().map_err(|e| { ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( "failed to enumerate child arrays in {group_uri}: {e}" @@ -121,10 +155,11 @@ fn build_rasters(group_uri: &str, mode: Mode) -> Result ))); } + let arrays = select_arrays(arrays, arrays_filter, group_uri)?; let array_infos = collect_array_infos(arrays)?; validate_group_constraints(&array_infos)?; - // Spatial-dim resolution. Phase 1 supports two configurations: + // Spatial-dim resolution. Two configurations are accepted: // - dim_names ends with ["y", "x"] (canonical for georeferenced // 2-D and time-series rasters); the spatial extent is the chunk's // last two dims. @@ -142,9 +177,37 @@ fn build_rasters(group_uri: &str, mode: Mode) -> Result .map(|&i| array_infos[0].chunk_shape[i] as i64) .collect(); - let group_transform = geo.transform.unwrap_or([0.0, 1.0, 0.0, 0.0, 0.0, -1.0]); + let group_transform = match geo.transform { + Some(t) => t, + None => { + // Both `spatial:transform` and `proj:*` are absent (the + // CRS-only case errored above). Fall back to identity pixel + // coordinates and breadcrumb a warning so spatial-join + // surprises are debuggable. + log::warn!( + "Zarr group at {group_uri} has no `spatial:transform`; falling back \ + to the identity pixel-coordinate transform [0, 1, 0, 0, 0, -1]. \ + Spatial operations against this raster will use pixel coordinates." + ); + [0.0, 1.0, 0.0, 0.0, 0.0, -1.0] + } + }; - let total_rows = array_infos[0].chunk_grid_shape.iter().product::() as usize; + // chunk_grid_shape comes from untrusted Zarr metadata; bound-check the + // product so a hostile or malformed grid can't drive RasterBuilder + // capacity to overflow (and trigger an OOM preallocation). + let total_rows = array_infos[0] + .chunk_grid_shape + .iter() + .try_fold(1usize, |acc, &n| { + usize::try_from(n).ok().and_then(|n| acc.checked_mul(n)) + }) + .ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "chunk grid shape {:?} overflows usize", + array_infos[0].chunk_grid_shape + )) + })?; let mut builder = RasterBuilder::new(total_rows); // Walk the chunk grid in row-major (C-order) order. The outer-most @@ -169,22 +232,17 @@ fn build_rasters(group_uri: &str, mode: Mode) -> Result for info in &array_infos { let dim_names_ref: Vec<&str> = info.dim_names.iter().map(String::as_str).collect(); let nodata_ref = info.nodata.as_deref(); - let anchor; - let (outdb_uri_arg, outdb_format_arg) = match mode { - Mode::InDb => (None, None), - Mode::OutDb => { - anchor = build_chunk_anchor(group_uri, &info.path, &chunk_indices); - (Some(anchor.as_str()), Some("zarr")) - } - }; + // Chunk anchor is provenance metadata; InDb vs OutDb is + // discriminated by `data.is_empty()`, not by these fields. + let anchor = build_chunk_anchor(group_uri, &info.path, &chunk_indices); builder.start_band_nd( Some(info.path.as_str()), &dim_names_ref, &info.chunk_shape, info.data_type, nodata_ref, - outdb_uri_arg, - outdb_format_arg, + Some(anchor.as_str()), + Some("zarr"), )?; match mode { Mode::InDb => { @@ -224,26 +282,7 @@ fn collect_array_infos( for array in arrays { let path = array.path().to_string(); let data_type = zarr_to_band_data_type(array.data_type())?; - let dim_names = match array.dimension_names() { - Some(names) => names - .iter() - .enumerate() - .map(|(i, n)| { - n.clone().ok_or_else(|| { - ArrowError::InvalidArgumentError(format!( - "array {path}: dimension {i} has no name; Phase 1 requires every \ - Zarr array dimension to be named", - )) - }) - }) - .collect::, _>>()?, - None => { - return Err(ArrowError::InvalidArgumentError(format!( - "array {path}: dimension_names is absent; Phase 1 requires every Zarr \ - array to declare dimension_names", - ))); - } - }; + let dim_names = resolve_dim_names(&array, &path)?; let chunk_grid_shape = array.chunk_grid_shape().to_vec(); let chunk_shape = array .chunk_shape(&vec![0u64; chunk_grid_shape.len()]) @@ -274,33 +313,155 @@ fn collect_array_infos( Ok(out) } -/// Enforce Phase 1 group constraints. All arrays must agree on chunk grid -/// shape, chunk shape, and dimension names. We do NOT enforce shared -/// element shape (`array.shape()`) because users routinely group -/// arrays with the same chunk grid but different totals (e.g. a coord -/// variable with one fewer dim is rejected here anyway by the dim-name -/// check). +/// Apply the array-selection rules. 1-D arrays (typical xarray-style +/// coord variables) are always dropped — a raster band requires at +/// least 2 dimensions, so reading a 1-D array could never succeed. +/// - Explicit filter: keep arrays whose path (with leading `/` stripped) +/// matches one of the requested names. Unknown names error so users +/// don't silently get an empty group from a typo; naming a 1-D array +/// errors with a clear message rather than producing a confusing +/// "no spatial axes" failure downstream. +/// - No filter: read every multi-dimensional array. +fn select_arrays( + arrays: Vec>, + filter: Option<&[String]>, + group_uri: &str, +) -> Result>, ArrowError> { + if let Some(names) = filter { + let available: Vec = arrays + .iter() + .map(|a| a.path().as_str().trim_start_matches('/').to_string()) + .collect(); + for requested in names { + let needle = requested.trim_start_matches('/'); + match arrays + .iter() + .find(|a| a.path().as_str().trim_start_matches('/') == needle) + { + None => { + return Err(ArrowError::InvalidArgumentError(format!( + "Zarr group at {group_uri} has no array named {requested:?}; \ + available arrays: {available:?}" + ))); + } + Some(a) if a.shape().len() < 2 => { + return Err(ArrowError::InvalidArgumentError(format!( + "array {requested:?} has rank {} (shape {:?}); a raster band \ + requires at least 2 dimensions and cannot be read.", + a.shape().len(), + a.shape() + ))); + } + Some(_) => {} + } + } + let kept: Vec<_> = arrays + .into_iter() + .filter(|a| { + let path = a.path().as_str().trim_start_matches('/').to_string(); + names.iter().any(|n| n.trim_start_matches('/') == path) + }) + .collect(); + return Ok(kept); + } + + let kept: Vec<_> = arrays.into_iter().filter(|a| a.shape().len() > 1).collect(); + if kept.is_empty() { + return Err(ArrowError::InvalidArgumentError(format!( + "Zarr group at {group_uri} contains only 1-D arrays (typical \ + xarray-style coord variables); a raster band requires at least \ + 2 dimensions, so this group has nothing readable as a raster" + ))); + } + Ok(kept) +} + +/// Resolve dimension names for an array, supporting both Zarr v3 +/// (first-class `dimension_names` field) and Zarr v2 with the xarray +/// `_ARRAY_DIMENSIONS` attribute. Errors if neither carries a complete +/// set of named dimensions matching the array's rank. +fn resolve_dim_names(array: &Array, path: &str) -> Result, ArrowError> { + let rank = array.shape().len(); + + if let Some(names) = array.dimension_names() { + return names + .iter() + .enumerate() + .map(|(i, n)| { + n.clone().ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "array {path}: dimension {i} has no name; every Zarr array \ + dimension must be named", + )) + }) + }) + .collect(); + } + + // Zarr v2 fallback: xarray's `_ARRAY_DIMENSIONS` convention. v2 had no + // first-class dimension_names field and zarrs's v2->v3 converter + // preserves attributes verbatim without lifting this one. + if let Some(value) = array.attributes().get("_ARRAY_DIMENSIONS") { + let arr = value.as_array().ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "array {path}: _ARRAY_DIMENSIONS must be a JSON array of strings; got {value}" + )) + })?; + if arr.len() != rank { + return Err(ArrowError::InvalidArgumentError(format!( + "array {path}: _ARRAY_DIMENSIONS has {} entries but array has rank {rank}", + arr.len() + ))); + } + return arr + .iter() + .enumerate() + .map(|(i, v)| { + v.as_str().map(str::to_string).ok_or_else(|| { + ArrowError::InvalidArgumentError(format!( + "array {path}: _ARRAY_DIMENSIONS[{i}] must be a string; got {v}" + )) + }) + }) + .collect(); + } + + Err(ArrowError::InvalidArgumentError(format!( + "array {path}: no dimension names found. Zarr v3 arrays must set \ + `dimension_names`; Zarr v2 arrays must set the `_ARRAY_DIMENSIONS` \ + attribute (xarray convention)." + ))) +} + +/// Enforce group constraints. All arrays must agree on chunk grid shape, +/// chunk shape, and dimension names. We do NOT enforce shared element +/// shape (`array.shape()`) because users routinely group arrays with +/// the same chunk grid but different totals (e.g. a coord variable with +/// one fewer dim is rejected here anyway by the dim-name check). fn validate_group_constraints(infos: &[ArrayInfo]) -> Result<(), ArrowError> { let first = &infos[0]; for other in &infos[1..] { if other.chunk_grid_shape != first.chunk_grid_shape { return Err(ArrowError::InvalidArgumentError(format!( "arrays {} and {} have different chunk grid shapes ({:?} vs {:?}); \ - Phase 1 requires a shared chunk grid across the group", + every array in the group must share the same chunk grid. \ + Pass `arrays = [...]` to read only compatible arrays.", first.path, other.path, first.chunk_grid_shape, other.chunk_grid_shape ))); } if other.chunk_shape != first.chunk_shape { return Err(ArrowError::InvalidArgumentError(format!( "arrays {} and {} have different chunk shapes ({:?} vs {:?}); \ - Phase 1 requires a shared chunk shape across the group", + every array in the group must share the same chunk shape. \ + Pass `arrays = [...]` to read only compatible arrays.", first.path, other.path, first.chunk_shape, other.chunk_shape ))); } if other.dim_names != first.dim_names { return Err(ArrowError::InvalidArgumentError(format!( "arrays {} and {} have different dimension names ({:?} vs {:?}); \ - Phase 1 requires identical dim_names across the group", + every array in the group must declare identical dim_names. \ + Pass `arrays = [...]` to read only compatible arrays.", first.path, other.path, first.dim_names, other.dim_names ))); } @@ -333,13 +494,13 @@ fn resolve_spatial_dim_indices( let n = dim_names.len(); if n < 2 { return Err(ArrowError::InvalidArgumentError(format!( - "Phase 1 requires at least 2 dimensions to resolve spatial axes; got {dim_names:?}", + "at least 2 dimensions are required to resolve spatial axes; got {dim_names:?}", ))); } if dim_names[n - 2] != "y" || dim_names[n - 1] != "x" { return Err(ArrowError::InvalidArgumentError(format!( - "Phase 1 expects the last two dim_names to be [\"y\", \"x\"] when \ - `spatial:dims` is not declared; got {dim_names:?}", + "the last two dim_names must be [\"y\", \"x\"] when `spatial:dims` is \ + not declared; got {dim_names:?}", ))); } Ok(vec![n - 2, n - 1]) @@ -357,8 +518,8 @@ fn compute_row_transform( // Translation along x = chunk_x_index × chunk_x_size in pixel-coordinate space, // converted to world coordinates via the affine. // - // Phase 1 assumes spatial_dim_indices == [y_index, x_index] (validated - // upstream). Index 0 is the y axis, index 1 is the x axis. + // spatial_dim_indices is validated upstream to be [y_index, x_index]. + // Index 0 is the y axis, index 1 is the x axis. let y_axis = spatial_dim_indices[0]; let x_axis = spatial_dim_indices[1]; let x_offset = (chunk_indices[x_axis] * chunk_shape[x_axis]) as f64; @@ -390,9 +551,9 @@ fn advance_chunk_indices(chunk_indices: &mut [u64], chunk_grid_shape: &[u64]) -> /// Retrieve a single chunk's bytes as a fresh `Vec`. /// -/// Phase 1 uses `ArrayBytes::Fixed`, so this errors for variable-length -/// element types — those don't have a `BandDataType` counterpart anyway, -/// so the dtype check in `collect_array_infos` rejects them upstream. +/// Uses `ArrayBytes::Fixed`, so this errors for variable-length element +/// types — those don't have a `BandDataType` counterpart anyway, so the +/// dtype check in `collect_array_infos` rejects them upstream. fn retrieve_chunk_bytes( array: &Array, chunk_indices: &[u64], @@ -408,7 +569,7 @@ fn retrieve_chunk_bytes( })?; let raw = bytes.into_fixed().map_err(|_| { ArrowError::InvalidArgumentError(format!( - "array {}: variable-length chunk bytes not supported in Phase 1", + "array {}: variable-length chunk bytes are not supported", array.path() )) })?; diff --git a/rust/sedona-raster-zarr/src/source_uri.rs b/rust/sedona-raster-zarr/src/source_uri.rs index 155a632aa..cd0818ba9 100644 --- a/rust/sedona-raster-zarr/src/source_uri.rs +++ b/rust/sedona-raster-zarr/src/source_uri.rs @@ -20,10 +20,10 @@ //! Two URI shapes flow through the loader: //! //! 1. **Group URI** (the loader entry-point argument). Identifies a Zarr -//! group on a backend, e.g. `file:///tmp/datacube.zarr`, -//! `s3://bucket/datacube.zarr/2024`, or a bare local path. Phase 1 -//! accepts `file://` and bare-path; cloud schemes ship with the -//! resolver. +//! group on a backend, e.g. `file:///tmp/datacube.zarr` or a bare +//! local path. Cloud schemes (`s3://`, `gs://`, `az://`, `https://`) +//! will be accepted by a future revision; today only `file://` and +//! bare-path are recognised. //! 2. **Chunk anchor URI** (written into a band's `outdb_uri`). Addresses //! one chunk in one array within a group: //! `#array=&chunk=,,...`. The store URI @@ -131,9 +131,8 @@ pub fn parse_chunk_anchor(uri: &str) -> Result { /// Normalize a user-supplied group URI into a local filesystem path. /// -/// Phase 1 supports `file://` and bare-path URIs only. Cloud schemes -/// (`s3://`, `gs://`, `az://`, `https://`) error with a clear message -/// pointing at the resolver work that adds them. +/// Only `file://` and bare-path URIs are supported. Cloud schemes +/// (`s3://`, `gs://`, `az://`, `https://`) error with a clear message. pub fn group_uri_to_filesystem_path(uri: &str) -> Result { if let Some(rest) = uri.strip_prefix("file://") { return Ok(std::path::PathBuf::from(rest)); @@ -141,8 +140,8 @@ pub fn group_uri_to_filesystem_path(uri: &str) -> Result TempDir { fn indb_round_trip_emits_one_row_per_chunk_position() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_indb_rasters(&uri).unwrap(); + let arr = group_to_indb_rasters(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 8, "expected 8 chunk rows (2*2*2)"); @@ -135,6 +135,13 @@ fn indb_round_trip_emits_one_row_per_chunk_position() { &*pressure.contiguous_data().unwrap(), &[100u8, 101, 104, 105] ); + // Chunk anchor is populated on InDb rows too as provenance — the + // `data` column carries the bytes, and `outdb_uri` records where + // they came from. + assert_eq!(pressure.outdb_format(), Some("zarr")); + let anchor = pressure.outdb_uri().expect("outdb_uri set on InDb band"); + assert!(anchor.contains("#array=pressure"), "got: {anchor}"); + assert!(anchor.contains("&chunk=0,0,0"), "got: {anchor}"); // Temperature has base=0 → same chunk holds {0, 1, 4, 5}. let temperature = r0.band(1).unwrap(); @@ -155,7 +162,7 @@ fn indb_round_trip_emits_one_row_per_chunk_position() { fn outdb_emits_chunk_anchors() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_outdb_rasters(&uri).unwrap(); + let arr = group_to_outdb_rasters(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 8); @@ -197,10 +204,202 @@ fn errors_on_empty_group() { .store_metadata() .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_indb_rasters(&uri).unwrap_err().to_string(); + let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); assert!(err.contains("no child arrays"), "got: {err}"); } +/// Build a group with two 3-D data arrays and 1-D `t`/`y`/`x` coord +/// variables alongside them — the xarray-on-Zarr pattern. The loader's +/// default behaviour must drop the coord variables and read only the +/// data arrays. +#[allow(deprecated)] +fn build_xarray_style_fixture() -> TempDir { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + + // Two 3-D data arrays sharing the same chunk grid. + for name in ["temperature", "pressure"] { + let arr = ArrayBuilder::new( + vec![2u64, 4u64, 4u64], + vec![1u64, 2u64, 2u64], + data_type::uint8(), + 0u8, + ) + .dimension_names(Some(["t", "y", "x"])) + .build(store.clone(), &format!("/{name}")) + .unwrap(); + arr.store_metadata().unwrap(); + for t in 0..2u64 { + for yc in 0..2u64 { + for xc in 0..2u64 { + arr.store_chunk_elements::(&[t, yc, xc], &[0u8; 4]) + .unwrap(); + } + } + } + } + + // 1-D coord variables. Different chunk grid than the data arrays — + // would trip validate_group_constraints if not auto-skipped. + for (name, len, dim) in [("t", 2u64, "t"), ("y", 4u64, "y"), ("x", 4u64, "x")] { + let arr = ArrayBuilder::new(vec![len], vec![len], data_type::uint8(), 0u8) + .dimension_names(Some([dim])) + .build(store.clone(), &format!("/{name}")) + .unwrap(); + arr.store_metadata().unwrap(); + } + + tmp +} + +#[test] +fn auto_skips_1d_coord_variables() { + let tmp = build_xarray_style_fixture(); + let uri = format!("file://{}", tmp.path().display()); + let arr = group_to_indb_rasters(&uri, None).unwrap(); + let rasters = RasterStructArray::new(&arr); + // 2*2*2 = 8 chunk positions, with 2 bands per row (pressure, temperature). + assert_eq!(rasters.len(), 8); + let r0 = rasters.get(0).unwrap(); + assert_eq!(r0.num_bands(), 2); +} + +#[test] +fn explicit_arrays_filter_selects_subset() { + let tmp = build_xarray_style_fixture(); + let uri = format!("file://{}", tmp.path().display()); + let filter = vec!["temperature".to_string()]; + let arr = group_to_indb_rasters(&uri, Some(&filter)).unwrap(); + let rasters = RasterStructArray::new(&arr); + assert_eq!(rasters.len(), 8); + let r0 = rasters.get(0).unwrap(); + assert_eq!(r0.num_bands(), 1, "only temperature should be read"); +} + +#[test] +fn explicit_arrays_filter_rejects_unknown_name() { + let tmp = build_xarray_style_fixture(); + let uri = format!("file://{}", tmp.path().display()); + let filter = vec!["humidity".to_string()]; + let err = group_to_indb_rasters(&uri, Some(&filter)) + .unwrap_err() + .to_string(); + assert!(err.contains("humidity"), "got: {err}"); + assert!(err.contains("no array named"), "got: {err}"); +} + +#[test] +fn errors_when_crs_declared_without_transform() { + // CRS-without-transform is almost certainly malformed metadata — + // the user thinks they have full georef but downstream spatial + // joins would silently use the identity pixel transform. The + // loader refuses rather than producing wrong results. + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + let mut group_attrs = serde_json::Map::new(); + group_attrs.insert("proj:epsg".into(), serde_json::json!(4326)); + GroupBuilder::new() + .attributes(group_attrs) + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + ArrayBuilder::new(vec![2u64, 2], vec![1u64, 2], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/temperature") + .unwrap() + .store_metadata() + .unwrap(); + + let uri = format!("file://{}", tmp.path().display()); + let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); + assert!(err.contains("CRS"), "got: {err}"); + assert!(err.contains("spatial:transform"), "got: {err}"); +} + +#[test] +fn explicit_arrays_filter_rejects_1d_arrays() { + // A user explicitly naming a 1-D array gets a clear "needs 2 dims" + // error at parse time, not a confusing downstream spatial-dim + // resolution failure. + let tmp = build_xarray_style_fixture(); + let uri = format!("file://{}", tmp.path().display()); + let filter = vec!["t".to_string()]; + let err = group_to_indb_rasters(&uri, Some(&filter)) + .unwrap_err() + .to_string(); + assert!(err.contains("\"t\""), "got: {err}"); + assert!(err.contains("rank 1"), "got: {err}"); + assert!(err.contains("at least 2 dimensions"), "got: {err}"); +} + +#[test] +fn errors_when_group_has_only_1d_arrays() { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + ArrayBuilder::new(vec![4u64], vec![2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["x"])) + .build(store.clone(), "/x") + .unwrap() + .store_metadata() + .unwrap(); + + let uri = format!("file://{}", tmp.path().display()); + let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); + assert!(err.contains("only 1-D arrays"), "got: {err}"); +} + +#[test] +fn falls_back_to_array_dimensions_attribute() { + // Simulates a Zarr v2 array (or any v3 array that lacks a first-class + // `dimension_names` field) by leaving `.dimension_names(None)` and + // setting xarray's `_ARRAY_DIMENSIONS` attribute instead. The loader + // must accept it and treat the attribute as authoritative. + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + + let mut attrs = serde_json::Map::new(); + attrs.insert("_ARRAY_DIMENSIONS".into(), serde_json::json!(["y", "x"])); + #[allow(deprecated)] + { + let array = ArrayBuilder::new(vec![2u64, 2], vec![1u64, 2], data_type::uint8(), 0u8) + .attributes(attrs) + .build(store.clone(), "/temperature") + .unwrap(); + array.store_metadata().unwrap(); + array + .store_chunk_elements::(&[0, 0], &[10u8, 11]) + .unwrap(); + array + .store_chunk_elements::(&[1, 0], &[20u8, 21]) + .unwrap(); + } + + let uri = format!("file://{}", tmp.path().display()); + let arr = group_to_indb_rasters(&uri, None).unwrap(); + let rasters = RasterStructArray::new(&arr); + assert_eq!(rasters.len(), 2); + let r0 = rasters.get(0).unwrap(); + let band = r0.band(0).unwrap(); + assert_eq!(&*band.contiguous_data().unwrap(), &[10u8, 11]); +} + #[test] fn errors_on_mismatched_chunk_grids() { let tmp = TempDir::new().unwrap(); @@ -224,7 +423,7 @@ fn errors_on_mismatched_chunk_grids() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_indb_rasters(&uri).unwrap_err().to_string(); + let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); assert!( err.contains("chunk") && err.contains("array_a") && err.contains("array_b"), "got: {err}" diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona/src/zarr_read.rs index d248e37cb..0b3b114b5 100644 --- a/rust/sedona/src/zarr_read.rs +++ b/rust/sedona/src/zarr_read.rs @@ -22,7 +22,7 @@ //! SELECT * FROM sd_read_zarr('file:///path/to/datacube.zarr'); //! SELECT count(*) FROM sd_read_zarr( //! 'file:///path/to/datacube.zarr', -//! '{"mode": "indb", "rows_per_batch": 256}' +//! '{"indb": true, "rows_per_batch": 256}' //! ); //! ``` //! @@ -30,10 +30,9 @@ //! position in the Zarr group's chunk grid. All existing `RS_*` UDFs //! operate on the column unchanged. //! -//! Phase 1 defaults to `mode = "indb"` so byte-reading kernels work -//! end-to-end without depending on the (not-yet-registered) OutDb -//! resolver. The default flips to `outdb` once the format-keyed -//! dispatcher lands. +//! `indb` defaults to `true` so byte-reading kernels work end-to-end +//! without depending on a registered OutDb resolver. The default will +//! flip to `false` once the format-keyed dispatcher lands. use std::any::Any; use std::sync::Arc; @@ -65,9 +64,14 @@ use serde::{Deserialize, Serialize}; /// Accepts one or two string arguments: /// - `uri` (required) — Zarr group URI (e.g. `file:///path/to/foo.zarr`). /// - `options_json` (optional) — JSON string with any of: -/// - `mode`: `"indb"` (default) or `"outdb"` +/// - `indb`: `true` (default) materializes chunk bytes into the +/// Arrow `data` column eagerly; `false` emits chunk-anchor URIs +/// only and defers byte resolution to the OutDb loader. /// - `rows_per_batch`: chunks per `RecordBatch` (default 1024) -/// - `num_partitions`: scan partitions (default 1; > 1 errors in Phase 1) +/// - `num_partitions`: scan partitions (default 1; > 1 currently errors) +/// - `arrays`: optional list of array names to read. Default reads +/// every multi-dimensional array (1-D coord variables are +/// auto-skipped); an explicit list reads exactly those arrays. #[derive(Debug, Default)] pub struct ZarrReadFunction {} @@ -107,9 +111,10 @@ fn literal_utf8(expr: &Expr, label: &str) -> Result { /// `StructArray` once for the query lifetime; the executor slices it /// into `rows_per_batch`-sized `RecordBatch`es. /// -/// Phase 1 builds the StructArray eagerly in `try_new`. For OutDb mode -/// the array is cheap to construct (chunk anchor URIs only); for InDb -/// mode it pulls every chunk's bytes through the loader at plan time. +/// The StructArray is built eagerly in `try_new`. When `indb=false` +/// the array is cheap to construct (chunk anchor URIs only); when +/// `indb=true` it pulls every chunk's bytes through the loader at +/// plan time. #[derive(Debug)] pub struct ZarrChunkProvider { schema: SchemaRef, @@ -120,24 +125,21 @@ pub struct ZarrChunkProvider { impl ZarrChunkProvider { fn try_new(uri: &str, options_json: Option) -> Result { let opts = parse_options(options_json.as_deref())?; - let mode = opts.mode.as_deref().unwrap_or("indb"); + let indb = opts.indb.unwrap_or(true); let rows_per_batch = opts.rows_per_batch.unwrap_or(1024).max(1); let num_partitions = opts.num_partitions.unwrap_or(1); if num_partitions != 1 { return plan_err!( - "sd_read_zarr() Phase 1 supports only num_partitions = 1; got {num_partitions}. \ + "sd_read_zarr() supports only num_partitions = 1; got {num_partitions}. \ Round-robin partitioning lands with the OutDb resolver work." ); } - let rasters = match mode { - "indb" => group_to_indb_rasters(uri).map_err(arrow_to_df_err)?, - "outdb" => group_to_outdb_rasters(uri).map_err(arrow_to_df_err)?, - other => { - return plan_err!( - "sd_read_zarr() mode must be \"indb\" or \"outdb\"; got {other:?}" - ); - } + let arrays_filter = opts.arrays.as_deref(); + let rasters = if indb { + group_to_indb_rasters(uri, arrays_filter).map_err(arrow_to_df_err)? + } else { + group_to_outdb_rasters(uri, arrays_filter).map_err(arrow_to_df_err)? }; // Single-column schema: `raster: Raster`. `SedonaType::Raster` adds @@ -215,7 +217,7 @@ impl ZarrChunkExec { fn new(schema: SchemaRef, rasters: StructArray, rows_per_batch: usize) -> Self { let properties = PlanProperties::new( EquivalenceProperties::new(schema.clone()), - // Phase 1: a single partition. Round-robin across multiple + // A single partition for now. Round-robin across multiple // partitions lands with the OutDb resolver work. Partitioning::UnknownPartitioning(1), EmissionType::Incremental, @@ -275,7 +277,7 @@ impl ExecutionPlan for ZarrChunkExec { _context: Arc, ) -> Result { if partition != 0 { - return plan_err!("ZarrChunkExec: only partition 0 exists (Phase 1)"); + return plan_err!("ZarrChunkExec: only partition 0 exists"); } let total = self.rasters.len(); @@ -284,10 +286,10 @@ impl ExecutionPlan for ZarrChunkExec { let rasters = self.rasters.clone(); // Build all batches eagerly into a Vec, then turn into a stream. - // This is fine for Phase 1: the StructArray is already materialised - // in the provider, slicing is O(1), and the only allocation per - // batch is the RecordBatch wrapper. Lazy streaming becomes - // interesting once the loader itself is lazy. + // The StructArray is already materialised in the provider, slicing + // is O(1), and the only allocation per batch is the RecordBatch + // wrapper. Lazy streaming becomes interesting once the loader + // itself is lazy. let mut batches = Vec::with_capacity(total.div_ceil(rows_per_batch).max(1)); let mut offset = 0; while offset < total { @@ -312,9 +314,17 @@ fn arrow_to_df_err(e: arrow_schema::ArrowError) -> DataFusionError { #[derive(Serialize, Deserialize, Default)] struct ZarrReadOptions { - mode: Option, + /// `true` (default) materializes chunk bytes into the Arrow `data` + /// column eagerly; `false` emits chunk-anchor URIs only and defers + /// byte resolution to the OutDb loader. + indb: Option, rows_per_batch: Option, num_partitions: Option, + /// Explicit array-name filter. `None` reads every multi-dimensional + /// array in the group; `Some` reads exactly the listed arrays (in + /// the order zarrs's store listing returns them). Unknown names + /// error so a typo doesn't silently yield an empty result. + arrays: Option>, } fn parse_options(options_json: Option<&str>) -> Result { @@ -428,7 +438,7 @@ mod tests { // walks the chunk grid metadata. let df = ctx .sql(&format!( - r#"SELECT count(*) FROM sd_read_zarr('{uri}', '{{"mode": "outdb"}}')"#, + r#"SELECT count(*) FROM sd_read_zarr('{uri}', '{{"indb": false}}')"#, )) .await .unwrap(); @@ -442,7 +452,7 @@ mod tests { } #[tokio::test] - async fn udtf_rejects_unknown_mode() { + async fn udtf_rejects_multi_partition_in_phase1() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); @@ -451,30 +461,93 @@ mod tests { let err = ctx .sql(&format!( - r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"mode": "lazy"}}')"#, + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"num_partitions": 2}}')"#, )) .await .unwrap_err() .to_string(); - assert!(err.contains("\"indb\" or \"outdb\""), "got: {err}"); + assert!(err.contains("num_partitions = 1"), "got: {err}"); } #[tokio::test] - async fn udtf_rejects_multi_partition_in_phase1() { - let tmp = build_fixture(); - let uri = format!("file://{}", tmp.path().display()); + async fn udtf_arrays_filter_threads_through_sql() { + // Build a group with one data array + a 1-D coord variable in a + // different chunk grid. Without auto-skip this errors; with the + // explicit arrays filter the user gets exactly what they asked + // for. + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + GroupBuilder::new() + .build(store.clone(), "/") + .unwrap() + .store_metadata() + .unwrap(); + #[allow(deprecated)] + { + let temperature = + ArrayBuilder::new(vec![2u64, 2u64], vec![1u64, 2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/temperature") + .unwrap(); + temperature.store_metadata().unwrap(); + temperature + .store_chunk_elements::(&[0, 0], &[10u8, 11]) + .unwrap(); + temperature + .store_chunk_elements::(&[1, 0], &[20u8, 21]) + .unwrap(); + let y = ArrayBuilder::new(vec![2u64], vec![2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["y"])) + .build(store.clone(), "/y") + .unwrap(); + y.store_metadata().unwrap(); + } + let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + // Default behaviour: 1-D coord variable auto-skipped, read succeeds. + let df = ctx + .sql(&format!("SELECT raster FROM sd_read_zarr('{uri}')")) + .await + .unwrap(); + assert_eq!( + df.collect() + .await + .unwrap() + .iter() + .map(|b| b.num_rows()) + .sum::(), + 2 + ); + + // Explicit filter to the data array — same result. + let df = ctx + .sql(&format!( + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"arrays":["temperature"]}}')"# + )) + .await + .unwrap(); + assert_eq!( + df.collect() + .await + .unwrap() + .iter() + .map(|b| b.num_rows()) + .sum::(), + 2 + ); + + // Unknown name surfaces as a clear error. let err = ctx .sql(&format!( - r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"num_partitions": 2}}')"#, + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"arrays":["humidity"]}}')"# )) .await .unwrap_err() .to_string(); - assert!(err.contains("num_partitions = 1"), "got: {err}"); + assert!(err.contains("humidity"), "got: {err}"); } #[tokio::test] From 1ad69cfbdb6446d8f0cec8c050e1f235f727f21f Mon Sep 17 00:00:00 2001 From: jameswillis Date: Tue, 19 May 2026 15:21:16 -0700 Subject: [PATCH 04/36] test(raster-zarr): migrate fixtures off deprecated store_chunk_elements zarrs 0.23 deprecated `store_chunk_elements::` in favour of `store_chunk()`. Migration is mechanical for u8 element types since the byte representation is identity. Drops the `#[allow(deprecated)]` annotations the fixtures carried. --- .../tests/zarr_roundtrip.rs | 34 +++++----------- rust/sedona/src/zarr_read.rs | 40 +++++++------------ 2 files changed, 23 insertions(+), 51 deletions(-) diff --git a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs index 1c221d824..eb19359a3 100644 --- a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs +++ b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs @@ -39,11 +39,6 @@ use zarrs_filesystem::FilesystemStore; /// /// Returns the temp dir (kept alive by the caller so files persist). /// -/// `store_chunk_elements` is deprecated in zarrs 0.23 in favour of -/// `store_chunk` (which takes raw bytes); the typed convenience wrapper -/// is still the cleanest path for fixture code so we suppress the -/// warning here. -#[allow(deprecated)] fn build_fixture() -> TempDir { let tmp = TempDir::new().unwrap(); let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); @@ -91,9 +86,7 @@ fn build_fixture() -> TempDir { chunk.push(base.wrapping_add((t * 16 + y * 4 + x) as u8)); } } - array - .store_chunk_elements::(&[t, yc, xc], &chunk) - .unwrap(); + array.store_chunk(&[t, yc, xc], chunk).unwrap(); } } } @@ -212,7 +205,6 @@ fn errors_on_empty_group() { /// variables alongside them — the xarray-on-Zarr pattern. The loader's /// default behaviour must drop the coord variables and read only the /// data arrays. -#[allow(deprecated)] fn build_xarray_style_fixture() -> TempDir { let tmp = TempDir::new().unwrap(); let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); @@ -237,8 +229,7 @@ fn build_xarray_style_fixture() -> TempDir { for t in 0..2u64 { for yc in 0..2u64 { for xc in 0..2u64 { - arr.store_chunk_elements::(&[t, yc, xc], &[0u8; 4]) - .unwrap(); + arr.store_chunk(&[t, yc, xc], vec![0u8; 4]).unwrap(); } } } @@ -376,20 +367,13 @@ fn falls_back_to_array_dimensions_attribute() { let mut attrs = serde_json::Map::new(); attrs.insert("_ARRAY_DIMENSIONS".into(), serde_json::json!(["y", "x"])); - #[allow(deprecated)] - { - let array = ArrayBuilder::new(vec![2u64, 2], vec![1u64, 2], data_type::uint8(), 0u8) - .attributes(attrs) - .build(store.clone(), "/temperature") - .unwrap(); - array.store_metadata().unwrap(); - array - .store_chunk_elements::(&[0, 0], &[10u8, 11]) - .unwrap(); - array - .store_chunk_elements::(&[1, 0], &[20u8, 21]) - .unwrap(); - } + let array = ArrayBuilder::new(vec![2u64, 2], vec![1u64, 2], data_type::uint8(), 0u8) + .attributes(attrs) + .build(store.clone(), "/temperature") + .unwrap(); + array.store_metadata().unwrap(); + array.store_chunk(&[0, 0], vec![10u8, 11]).unwrap(); + array.store_chunk(&[1, 0], vec![20u8, 21]).unwrap(); let uri = format!("file://{}", tmp.path().display()); let arr = group_to_indb_rasters(&uri, None).unwrap(); diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona/src/zarr_read.rs index 0b3b114b5..34a125040 100644 --- a/rust/sedona/src/zarr_read.rs +++ b/rust/sedona/src/zarr_read.rs @@ -353,7 +353,6 @@ mod tests { /// Build a tiny 1-array Zarr group on disk and return the temp dir. /// 2×2 UInt8 with chunks [1, 2] → chunk grid [2, 1] = 2 chunk rows. - #[allow(deprecated)] fn build_fixture() -> TempDir { let tmp = TempDir::new().unwrap(); let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); @@ -367,12 +366,8 @@ mod tests { .build(store.clone(), "/temperature") .unwrap(); array.store_metadata().unwrap(); - array - .store_chunk_elements::(&[0, 0], &[10u8, 11]) - .unwrap(); - array - .store_chunk_elements::(&[1, 0], &[20u8, 21]) - .unwrap(); + array.store_chunk(&[0, 0], vec![10u8, 11]).unwrap(); + array.store_chunk(&[1, 0], vec![20u8, 21]).unwrap(); tmp } @@ -482,26 +477,19 @@ mod tests { .unwrap() .store_metadata() .unwrap(); - #[allow(deprecated)] - { - let temperature = - ArrayBuilder::new(vec![2u64, 2u64], vec![1u64, 2u64], data_type::uint8(), 0u8) - .dimension_names(Some(["y", "x"])) - .build(store.clone(), "/temperature") - .unwrap(); - temperature.store_metadata().unwrap(); - temperature - .store_chunk_elements::(&[0, 0], &[10u8, 11]) - .unwrap(); - temperature - .store_chunk_elements::(&[1, 0], &[20u8, 21]) + let temperature = + ArrayBuilder::new(vec![2u64, 2u64], vec![1u64, 2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/temperature") .unwrap(); - let y = ArrayBuilder::new(vec![2u64], vec![2u64], data_type::uint8(), 0u8) - .dimension_names(Some(["y"])) - .build(store.clone(), "/y") - .unwrap(); - y.store_metadata().unwrap(); - } + temperature.store_metadata().unwrap(); + temperature.store_chunk(&[0, 0], vec![10u8, 11]).unwrap(); + temperature.store_chunk(&[1, 0], vec![20u8, 21]).unwrap(); + let y = ArrayBuilder::new(vec![2u64], vec![2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["y"])) + .build(store.clone(), "/y") + .unwrap(); + y.store_metadata().unwrap(); let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); From ccbb7428a639453aec8ddfebfb2d3e8e57ea5c8e Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 12:10:24 -0700 Subject: [PATCH 05/36] refactor(raster-zarr): rename sd_read_zarr `indb` option to `load_eager` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit `load_eager` more accurately describes the user-facing knob: the boolean controls whether chunk bytes are materialised eagerly or left as chunk-anchor URIs, not which storage class the resulting raster lives in. Renames the JSON key (`indb` → `load_eager`), the Python kwarg, the internal `ZarrReadOptions` field, and updates docstrings to reflect the long-term plan: once an async `RS_EnsureLoaded` UDF lands, the planner will auto-inject it over the scan output rather than the loader doing eager fetching at plan time. The user-facing flag name doesn't change when that swap happens. --- .../python/sedonadb/functions/table.py | 11 ++++--- rust/sedona/src/zarr_read.rs | 32 ++++++++++++------- 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/python/sedonadb/python/sedonadb/functions/table.py b/python/sedonadb/python/sedonadb/functions/table.py index e79aa5c84..fd4c42b87 100644 --- a/python/sedonadb/python/sedonadb/functions/table.py +++ b/python/sedonadb/python/sedonadb/functions/table.py @@ -117,7 +117,7 @@ def sd_read_zarr( self, uri: str, *, - indb: Optional[bool] = None, + load_eager: Optional[bool] = None, rows_per_batch: Optional[int] = None, num_partitions: Optional[int] = None, arrays: Optional[List[str]] = None, @@ -137,11 +137,14 @@ def sd_read_zarr( ---------- uri : str Zarr group URI. ``file:///path/to/foo.zarr`` or a bare local path. - indb : bool, optional + load_eager : bool, optional ``True`` (default) materializes every chunk's bytes into the Arrow ``data`` column eagerly. ``False`` emits chunk-anchor URIs only; byte resolution depends on the OutDb resolver - being registered (follow-up PR). + being registered (follow-up PR). Long-term, ``load_eager = + True`` will trigger the planner to inject an async + ``RS_EnsureLoaded`` over the scan output rather than + fetching at plan time. rows_per_batch : int, optional Chunks per ``RecordBatch`` (default 1024). num_partitions : int, optional @@ -165,7 +168,7 @@ def sd_read_zarr( """ args = { - "indb": indb, + "load_eager": load_eager, "rows_per_batch": rows_per_batch, "num_partitions": num_partitions, "arrays": arrays, diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona/src/zarr_read.rs index 34a125040..051e14096 100644 --- a/rust/sedona/src/zarr_read.rs +++ b/rust/sedona/src/zarr_read.rs @@ -22,7 +22,7 @@ //! SELECT * FROM sd_read_zarr('file:///path/to/datacube.zarr'); //! SELECT count(*) FROM sd_read_zarr( //! 'file:///path/to/datacube.zarr', -//! '{"indb": true, "rows_per_batch": 256}' +//! '{"load_eager": true, "rows_per_batch": 256}' //! ); //! ``` //! @@ -30,9 +30,11 @@ //! position in the Zarr group's chunk grid. All existing `RS_*` UDFs //! operate on the column unchanged. //! -//! `indb` defaults to `true` so byte-reading kernels work end-to-end -//! without depending on a registered OutDb resolver. The default will -//! flip to `false` once the format-keyed dispatcher lands. +//! `load_eager` defaults to `true` so byte-reading kernels work end-to-end +//! without depending on a registered OutDb resolver. When the async +//! `RS_EnsureLoaded` UDF lands, `load_eager = true` will be reinterpreted +//! as the planner auto-injecting that UDF over the scan output instead +//! of fetching at plan time. use std::any::Any; use std::sync::Arc; @@ -64,9 +66,12 @@ use serde::{Deserialize, Serialize}; /// Accepts one or two string arguments: /// - `uri` (required) — Zarr group URI (e.g. `file:///path/to/foo.zarr`). /// - `options_json` (optional) — JSON string with any of: -/// - `indb`: `true` (default) materializes chunk bytes into the -/// Arrow `data` column eagerly; `false` emits chunk-anchor URIs -/// only and defers byte resolution to the OutDb loader. +/// - `load_eager`: `true` (default) materializes chunk bytes into +/// the Arrow `data` column eagerly; `false` emits chunk-anchor +/// URIs only and defers byte resolution to the OutDb loader. +/// Long-term, `load_eager = true` will instead trigger the +/// planner to inject an async `RS_EnsureLoaded` over the scan +/// output rather than fetching at plan time. /// - `rows_per_batch`: chunks per `RecordBatch` (default 1024) /// - `num_partitions`: scan partitions (default 1; > 1 currently errors) /// - `arrays`: optional list of array names to read. Default reads @@ -125,7 +130,7 @@ pub struct ZarrChunkProvider { impl ZarrChunkProvider { fn try_new(uri: &str, options_json: Option) -> Result { let opts = parse_options(options_json.as_deref())?; - let indb = opts.indb.unwrap_or(true); + let load_eager = opts.load_eager.unwrap_or(true); let rows_per_batch = opts.rows_per_batch.unwrap_or(1024).max(1); let num_partitions = opts.num_partitions.unwrap_or(1); if num_partitions != 1 { @@ -136,7 +141,7 @@ impl ZarrChunkProvider { } let arrays_filter = opts.arrays.as_deref(); - let rasters = if indb { + let rasters = if load_eager { group_to_indb_rasters(uri, arrays_filter).map_err(arrow_to_df_err)? } else { group_to_outdb_rasters(uri, arrays_filter).map_err(arrow_to_df_err)? @@ -316,8 +321,11 @@ fn arrow_to_df_err(e: arrow_schema::ArrowError) -> DataFusionError { struct ZarrReadOptions { /// `true` (default) materializes chunk bytes into the Arrow `data` /// column eagerly; `false` emits chunk-anchor URIs only and defers - /// byte resolution to the OutDb loader. - indb: Option, + /// byte resolution to the OutDb loader. Long-term, `load_eager = + /// true` will trigger the planner to inject an async + /// `RS_EnsureLoaded` over the scan output rather than fetching at + /// plan time. + load_eager: Option, rows_per_batch: Option, num_partitions: Option, /// Explicit array-name filter. `None` reads every multi-dimensional @@ -433,7 +441,7 @@ mod tests { // walks the chunk grid metadata. let df = ctx .sql(&format!( - r#"SELECT count(*) FROM sd_read_zarr('{uri}', '{{"indb": false}}')"#, + r#"SELECT count(*) FROM sd_read_zarr('{uri}', '{{"load_eager": false}}')"#, )) .await .unwrap(); From 7a391cb2dea15958e93155e3f9666c409c233e44 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 13:23:09 -0700 Subject: [PATCH 06/36] fix(raster-zarr): review-round small fixes from PR #858 - Narrow the blosc Windows cfg gate from "any Windows" to "Windows MinGW only" (`target_env = "gnu"`). MSVC Windows users (Python wheels) now get blosc-compressed Zarr support back; only the R rtools45 MinGW build path still disables it. - Drop the hardcoded `rows_per_batch = 1024` default in `ZarrChunkExec`. When the UDTF option is unset, defer at execute time to `SessionConfig::batch_size` so users tune via the standard knob instead of the loader baking in its own constant. - Python test: lift `numpy` and `pytest` imports to module level (only `zarr` remains `importorskip`-gated); materialise the resulting DataFrame via `to_arrow_table()` and assert on the Raster struct row instead of just `count()`. --- python/sedonadb/tests/test_funcs.py | 29 +++++++++++++------- rust/sedona-raster-zarr/Cargo.toml | 11 ++++---- rust/sedona/Cargo.toml | 7 ++--- rust/sedona/src/zarr_read.rs | 42 ++++++++++++++++++++--------- 4 files changed, 59 insertions(+), 30 deletions(-) diff --git a/python/sedonadb/tests/test_funcs.py b/python/sedonadb/tests/test_funcs.py index a846e3d7d..b66249e7b 100644 --- a/python/sedonadb/tests/test_funcs.py +++ b/python/sedonadb/tests/test_funcs.py @@ -16,6 +16,10 @@ # under the License. +import numpy as np +import pytest + + def test_random_geometry(con): df = con.funcs.table.sd_random_geometry("Point", 5, seed=99873) @@ -27,13 +31,11 @@ def test_random_geometry(con): def test_read_zarr(con, tmp_path): - # Skip cleanly if the optional `zarr` Python lib isn't installed — - # the binding is exercised by the Rust-side integration tests; this - # test only verifies the Python wrapper threads arguments through. - import pytest - + # `zarr` is an optional fixture dep — skip if not installed. The + # loader itself is exercised by the Rust-side integration tests; + # this test verifies the Python wrapper threads arguments through + # and materialises a raster row into Python. zarr = pytest.importorskip("zarr") - np = pytest.importorskip("numpy") # Build a 2x2 UInt8 array with two chunks, dim_names=["y","x"], inside # a group at the temp path. Matches the minimal shape sd_read_zarr @@ -48,10 +50,19 @@ def test_read_zarr(con, tmp_path): ) arr[:] = np.array([[10, 11], [20, 21]], dtype=np.uint8) - # Default-mode (InDb) read — every chunk materialized into the - # Arrow `data` column. + # Default-mode (load_eager=True) read — every chunk materialized + # into the Arrow `data` column. Materialise through Arrow so we + # can inspect the resulting Raster struct. df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}") - assert df.count() == 2 + arrow_tab = df.to_arrow_table() + assert arrow_tab.num_rows == 2 + assert arrow_tab.column_names == ["raster"] + raster = arrow_tab["raster"][0].as_py() + assert isinstance(raster, dict) + # The Raster struct exposes at least these top-level fields; their + # exact contents are covered by the Rust tests. + for field in ("transform", "bands"): + assert field in raster, f"raster row missing {field!r}: {sorted(raster)}" # Options thread through: rows_per_batch slices the output. df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}", rows_per_batch=1) diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml index 892806064..807f148f1 100644 --- a/rust/sedona-raster-zarr/Cargo.toml +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -42,11 +42,12 @@ serde_json = { workspace = true } zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } zarrs_filesystem = { workspace = true } -# `blosc` is gated off Windows: c-blosc (statically linked) bundles its own -# `pthread_create` / `pthread_cond_*` symbols, which conflict with rtools45's -# `libpthread.a` during the MinGW link of the R `sedonadb.dll`. Non-Windows -# targets get the full blosc-compressed Zarr reading capability. -[target.'cfg(not(target_os = "windows"))'.dependencies] +# `blosc` is gated off the Windows MinGW (gnu) toolchain only: c-blosc +# (statically linked) bundles its own `pthread_create` / `pthread_cond_*` +# symbols, which conflict with rtools45's `libpthread.a` during the MinGW +# link of the R `sedonadb.dll`. The MSVC toolchain (Python wheels, etc.) +# uses a different pthread implementation and is fine. +[target.'cfg(not(all(target_os = "windows", target_env = "gnu")))'.dependencies] zarrs = { workspace = true, features = ["blosc"] } [dev-dependencies] diff --git a/rust/sedona/Cargo.toml b/rust/sedona/Cargo.toml index 662a8d30c..12a1fe6ac 100644 --- a/rust/sedona/Cargo.toml +++ b/rust/sedona/Cargo.toml @@ -54,9 +54,10 @@ sedona-raster = { workspace = true } zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } zarrs_filesystem = { workspace = true } -# See sedona-raster-zarr/Cargo.toml — `blosc` is gated off Windows to avoid -# a libwinpthreads/c-blosc symbol collision in the R `sedonadb.dll` link. -[target.'cfg(not(target_os = "windows"))'.dev-dependencies] +# See sedona-raster-zarr/Cargo.toml — `blosc` is gated off the Windows +# MinGW (gnu) toolchain only to avoid a libwinpthreads/c-blosc symbol +# collision in the R `sedonadb.dll` link. MSVC Windows users get blosc. +[target.'cfg(not(all(target_os = "windows", target_env = "gnu")))'.dev-dependencies] zarrs = { workspace = true, features = ["blosc"] } [dependencies] diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona/src/zarr_read.rs index 051e14096..de35feed4 100644 --- a/rust/sedona/src/zarr_read.rs +++ b/rust/sedona/src/zarr_read.rs @@ -72,7 +72,9 @@ use serde::{Deserialize, Serialize}; /// Long-term, `load_eager = true` will instead trigger the /// planner to inject an async `RS_EnsureLoaded` over the scan /// output rather than fetching at plan time. -/// - `rows_per_batch`: chunks per `RecordBatch` (default 1024) +/// - `rows_per_batch`: chunks per `RecordBatch`. Defaults to the +/// session's configured batch size (`SessionConfig::batch_size`, +/// typically 8192) when unset. /// - `num_partitions`: scan partitions (default 1; > 1 currently errors) /// - `arrays`: optional list of array names to read. Default reads /// every multi-dimensional array (1-D coord variables are @@ -124,14 +126,16 @@ fn literal_utf8(expr: &Expr, label: &str) -> Result { pub struct ZarrChunkProvider { schema: SchemaRef, rasters: StructArray, - rows_per_batch: usize, + /// `None` defers to the session's `SessionConfig::batch_size` at + /// execute time. Set explicitly via the `rows_per_batch` JSON option. + rows_per_batch: Option, } impl ZarrChunkProvider { fn try_new(uri: &str, options_json: Option) -> Result { let opts = parse_options(options_json.as_deref())?; let load_eager = opts.load_eager.unwrap_or(true); - let rows_per_batch = opts.rows_per_batch.unwrap_or(1024).max(1); + let rows_per_batch = opts.rows_per_batch.map(|n| n.max(1)); let num_partitions = opts.num_partitions.unwrap_or(1); if num_partitions != 1 { return plan_err!( @@ -214,12 +218,12 @@ impl TableProvider for ZarrChunkProvider { struct ZarrChunkExec { schema: SchemaRef, rasters: StructArray, - rows_per_batch: usize, + rows_per_batch: Option, properties: PlanProperties, } impl ZarrChunkExec { - fn new(schema: SchemaRef, rasters: StructArray, rows_per_batch: usize) -> Self { + fn new(schema: SchemaRef, rasters: StructArray, rows_per_batch: Option) -> Self { let properties = PlanProperties::new( EquivalenceProperties::new(schema.clone()), // A single partition for now. Round-robin across multiple @@ -239,12 +243,18 @@ impl ZarrChunkExec { impl DisplayAs for ZarrChunkExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!( - f, - "ZarrChunkExec: rows={}, rows_per_batch={}", - self.rasters.len(), - self.rows_per_batch, - ) + match self.rows_per_batch { + Some(n) => write!( + f, + "ZarrChunkExec: rows={}, rows_per_batch={n}", + self.rasters.len() + ), + None => write!( + f, + "ZarrChunkExec: rows={}, rows_per_batch=session_default", + self.rasters.len() + ), + } } } @@ -279,14 +289,20 @@ impl ExecutionPlan for ZarrChunkExec { fn execute( &self, partition: usize, - _context: Arc, + context: Arc, ) -> Result { if partition != 0 { return plan_err!("ZarrChunkExec: only partition 0 exists"); } let total = self.rasters.len(); - let rows_per_batch = self.rows_per_batch; + // Defer the default to the session's batch size so users can + // tune via SessionConfig instead of relying on a hard-coded + // constant baked into this UDTF. + let rows_per_batch = self + .rows_per_batch + .unwrap_or_else(|| context.session_config().batch_size()) + .max(1); let schema = self.schema.clone(); let rasters = self.rasters.clone(); From ecf12a8522a5f4bceaad118470b7e067b7a33b8d Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 14:03:14 -0700 Subject: [PATCH 07/36] refactor(raster-zarr): collapse loader to a single OutDb path MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The loader had two near-duplicate code paths (`group_to_indb_rasters` and `group_to_outdb_rasters`) gated by a `Mode` enum. The InDb path fetched every chunk's bytes on a single thread at plan time — exactly the antipattern that the async `RS_EnsureLoaded` resolver will replace. Rather than ship the bad path and tear it out later, collapse to a single OutDb-producing entry point now. - New single entry: `group_to_rasters(uri, arrays)`. Always emits chunk-anchor URIs in `outdb_uri`; `data` is empty. - `Mode::InDb` / `Mode::OutDb` enum and the `match mode` branches in `build_rasters` are gone. - `retrieve_chunk_bytes` is kept (the only pixel-byte read primitive in the crate; the future async resolver will want it as a starting point) but is now `#[allow(dead_code)]` and exercised by a direct unit test rather than indirectly via the deleted InDb integration test. - `ArrayInfo::array` field dropped — only the InDb path used it. - `sd_read_zarr()` default for `load_eager` flips from `true` to `false`; explicit `load_eager = true` errors with a message pointing at the future `RS_EnsureLoaded` resolver. - Tests rewritten: `indb_round_trip_emits_one_row_per_chunk_position` is folded into a single `round_trip_emits_one_row_per_chunk_position_with_outdb_anchors` test that asserts on metadata and anchor URIs. Byte-content assertions removed. New UDTF test verifies `load_eager = true` errors cleanly. Plan-time resident size for a Zarr scan drops from `N_chunks × chunk_size` (potentially GBs) to roughly `N_chunks × few-hundred-bytes`. The remaining structural concern — build-everything-upfront vs streaming emission — is unchanged in this PR and tracked as separate follow-up work. --- python/sedonadb/tests/test_funcs.py | 12 +- rust/sedona-raster-zarr/src/lib.rs | 15 +-- rust/sedona-raster-zarr/src/loader.rs | 93 ++++++++------ .../tests/zarr_roundtrip.rs | 121 +++++------------- rust/sedona/src/zarr_read.rs | 91 ++++++++----- 5 files changed, 158 insertions(+), 174 deletions(-) diff --git a/python/sedonadb/tests/test_funcs.py b/python/sedonadb/tests/test_funcs.py index b66249e7b..daa141568 100644 --- a/python/sedonadb/tests/test_funcs.py +++ b/python/sedonadb/tests/test_funcs.py @@ -50,9 +50,10 @@ def test_read_zarr(con, tmp_path): ) arr[:] = np.array([[10, 11], [20, 21]], dtype=np.uint8) - # Default-mode (load_eager=True) read — every chunk materialized - # into the Arrow `data` column. Materialise through Arrow so we - # can inspect the resulting Raster struct. + # Default read emits OutDb-style rows — `data` is empty, + # `outdb_uri` carries a chunk anchor. Pixel-byte resolution is + # deferred to the future RS_EnsureLoaded resolver. Materialise + # through Arrow so we can inspect the Raster struct. df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}") arrow_tab = df.to_arrow_table() assert arrow_tab.num_rows == 2 @@ -67,3 +68,8 @@ def test_read_zarr(con, tmp_path): # Options thread through: rows_per_batch slices the output. df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}", rows_per_batch=1) assert df.count() == 2 + + # load_eager=True is not yet supported — errors with a clear + # pointer at the future RS_EnsureLoaded resolver. + with pytest.raises(Exception, match="load_eager"): + con.funcs.table.sd_read_zarr(f"file://{tmp_path}", load_eager=True).count() diff --git a/rust/sedona-raster-zarr/src/lib.rs b/rust/sedona-raster-zarr/src/lib.rs index 06c8b922b..800c009a8 100644 --- a/rust/sedona-raster-zarr/src/lib.rs +++ b/rust/sedona-raster-zarr/src/lib.rs @@ -22,13 +22,12 @@ //! array in the group, mapped onto SedonaDB's canonical N-D raster Arrow //! schema. //! -//! Two entry points: -//! -//! - [`group_to_indb_rasters`] — eagerly fetches every chunk's bytes into -//! the Arrow `data` column. Suitable for snapshots / small groups. -//! - [`group_to_outdb_rasters`] — emits chunk-anchor URIs only; bytes -//! fetch on demand through the process-wide OutDb loader (registered -//! separately via `sedona-raster`'s loader hook). +//! Single entry point: [`group_to_rasters`] always emits OutDb-style +//! rows — `data` is empty, `outdb_uri` carries a chunk anchor that the +//! async OutDb resolver (registered separately, lands in a follow-up) +//! turns into pixel bytes on demand. Metadata-only operations +//! (`count(*)`, `RS_Envelope`, `RS_Width`, …) work today; byte-consuming +//! kernels require the resolver to be registered. //! //! Local filesystem stores only — `file://` URIs or bare paths. @@ -37,4 +36,4 @@ pub mod geozarr; pub mod loader; pub mod source_uri; -pub use loader::{group_to_indb_rasters, group_to_outdb_rasters}; +pub use loader::group_to_rasters; diff --git a/rust/sedona-raster-zarr/src/loader.rs b/rust/sedona-raster-zarr/src/loader.rs index 021a1e4b0..e5fe2b5cf 100644 --- a/rust/sedona-raster-zarr/src/loader.rs +++ b/rust/sedona-raster-zarr/src/loader.rs @@ -45,9 +45,13 @@ use crate::dtype::zarr_to_band_data_type; use crate::geozarr::GroupGeoMetadata; use crate::source_uri::{build_chunk_anchor, group_uri_to_filesystem_path}; -/// Open a Zarr group and eagerly fetch every chunk's bytes into the -/// returned `StructArray`. Each row holds one chunk position's data -/// across the selected arrays in the group. +/// Open a Zarr group and emit one row per chunk position with +/// chunk-anchor URIs in each band's `outdb_uri`. The `data` column is +/// empty; bytes resolve on demand through whichever OutDb loader is +/// registered for the `zarr` format. No resolver is registered yet — +/// downstream byte-consuming kernels (`RS_Value` and similar) will +/// error until the async resolver lands. Metadata-only operations +/// (`count(*)`, `RS_Envelope`, `RS_Width`, etc.) work today. /// /// `arrays`: /// - `None` — read every multi-dimensional array. 1-D arrays (typical @@ -57,29 +61,11 @@ use crate::source_uri::{build_chunk_anchor, group_uri_to_filesystem_path}; /// unknown name errors. 1-D arrays are always rejected (a raster band /// needs ≥ 2 dimensions); naming one explicitly errors at parse time /// with a clear message. -pub fn group_to_indb_rasters( +pub fn group_to_rasters( group_uri: &str, arrays: Option<&[String]>, ) -> Result { - build_rasters(group_uri, Mode::InDb, arrays) -} - -/// Open a Zarr group and emit one row per chunk position with chunk-anchor -/// URIs in each band's `outdb_uri`. The `data` column is empty; bytes -/// resolve on demand through whichever OutDb loader is registered for -/// the `zarr` format. See [`group_to_indb_rasters`] for the meaning of -/// `arrays`. -pub fn group_to_outdb_rasters( - group_uri: &str, - arrays: Option<&[String]>, -) -> Result { - build_rasters(group_uri, Mode::OutDb, arrays) -} - -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -enum Mode { - InDb, - OutDb, + build_rasters(group_uri, arrays) } /// Per-array metadata extracted once at group open and reused for every @@ -89,8 +75,6 @@ struct ArrayInfo { /// Array path within the store, used to build chunk anchor URIs and /// surface in band names. path: String, - /// Open zarrs handle. - array: Array, /// SedonaDB BandDataType corresponding to this array's zarrs dtype. data_type: BandDataType, /// Dimension names in array order. Required to be `Some(_)` for every @@ -110,7 +94,6 @@ struct ArrayInfo { fn build_rasters( group_uri: &str, - mode: Mode, arrays_filter: Option<&[String]>, ) -> Result { let fs_path = group_uri_to_filesystem_path(group_uri)?; @@ -232,8 +215,10 @@ fn build_rasters( for info in &array_infos { let dim_names_ref: Vec<&str> = info.dim_names.iter().map(String::as_str).collect(); let nodata_ref = info.nodata.as_deref(); - // Chunk anchor is provenance metadata; InDb vs OutDb is - // discriminated by `data.is_empty()`, not by these fields. + // Every band gets its chunk-anchor URI populated as + // provenance metadata. `data.is_empty()` is the InDb/OutDb + // discriminator; this loader always emits empty `data` and + // defers pixel-byte resolution to the OutDb resolver. let anchor = build_chunk_anchor(group_uri, &info.path, &chunk_indices); builder.start_band_nd( Some(info.path.as_str()), @@ -244,18 +229,7 @@ fn build_rasters( Some(anchor.as_str()), Some("zarr"), )?; - match mode { - Mode::InDb => { - let bytes = retrieve_chunk_bytes(&info.array, &chunk_indices)?; - builder.band_data_writer().append_value(&bytes); - } - Mode::OutDb => { - // Schema-OutDb: empty `data` column. Byte resolution - // routes through the OutDb loader when a downstream - // consumer calls contiguous_data / nd_buffer. - builder.band_data_writer().append_value([0u8; 0]); - } - } + builder.band_data_writer().append_value([0u8; 0]); builder.finish_band()?; } builder.finish_raster()?; @@ -302,7 +276,6 @@ fn collect_array_infos( }; out.push(ArrayInfo { path, - array, data_type, dim_names, chunk_grid_shape, @@ -554,6 +527,14 @@ fn advance_chunk_indices(chunk_indices: &mut [u64], chunk_grid_shape: &[u64]) -> /// Uses `ArrayBytes::Fixed`, so this errors for variable-length element /// types — those don't have a `BandDataType` counterpart anyway, so the /// dtype check in `collect_array_infos` rejects them upstream. +/// +/// This is the only pixel-byte read primitive in the crate. The loader +/// itself never calls it today — it always emits OutDb anchors — but +/// the async `RS_EnsureLoaded` resolver (follow-up PR) will. Kept here +/// rather than dropped because (a) it's tested directly below and (b) +/// the resolver lives in this crate and wants a sync chunk-fetch +/// helper as a starting point. +#[allow(dead_code)] fn retrieve_chunk_bytes( array: &Array, chunk_indices: &[u64], @@ -579,6 +560,36 @@ fn retrieve_chunk_bytes( #[cfg(test)] mod tests { use super::*; + use std::sync::Arc; + use tempfile::TempDir; + use zarrs::array::data_type; + use zarrs::array::ArrayBuilder; + + /// Direct coverage for `retrieve_chunk_bytes`. The function is the + /// only pixel-byte read primitive in the crate today; previously it + /// was exercised through `group_to_indb_rasters` integration tests, + /// which are gone now that the loader only emits OutDb anchors. The + /// follow-up `RS_EnsureLoaded` resolver will call this directly. + #[test] + fn retrieve_chunk_bytes_returns_decoded_chunk() { + let tmp = TempDir::new().unwrap(); + let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); + + // 1×4 UInt8 array with chunks of size [1, 2] → chunk grid [1, 2]. + let arr = ArrayBuilder::new(vec![1u64, 4u64], vec![1u64, 2u64], data_type::uint8(), 0u8) + .dimension_names(Some(["y", "x"])) + .build(store.clone(), "/band") + .unwrap(); + arr.store_metadata().unwrap(); + arr.store_chunk(&[0u64, 0u64], vec![10u8, 11u8]).unwrap(); + arr.store_chunk(&[0u64, 1u64], vec![20u8, 21u8]).unwrap(); + + let chunk_0 = retrieve_chunk_bytes(&arr, &[0, 0]).unwrap(); + assert_eq!(chunk_0, vec![10u8, 11u8]); + + let chunk_1 = retrieve_chunk_bytes(&arr, &[0, 1]).unwrap(); + assert_eq!(chunk_1, vec![20u8, 21u8]); + } #[test] fn advance_chunk_indices_walks_row_major() { diff --git a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs index eb19359a3..26be44189 100644 --- a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs +++ b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs @@ -16,14 +16,18 @@ // under the License. //! End-to-end fixture test: build a small Zarr group on disk with the -//! `zarrs` crate, then read it back through `group_to_*_rasters` and -//! verify the resulting raster `StructArray`. +//! `zarrs` crate, then read it back through `group_to_rasters` and +//! verify the resulting raster `StructArray`. The loader always emits +//! OutDb-style rows (empty `data`, populated `outdb_uri`), so these +//! tests assert on metadata and chunk-anchor URIs rather than pixel +//! bytes. Pixel-byte coverage lives in the loader's unit tests against +//! `retrieve_chunk_bytes`. use std::sync::Arc; use sedona_raster::array::RasterStructArray; use sedona_raster::traits::RasterRef; -use sedona_raster_zarr::{group_to_indb_rasters, group_to_outdb_rasters}; +use sedona_raster_zarr::group_to_rasters; use sedona_schema::raster::BandDataType; use tempfile::TempDir; use zarrs::array::data_type; @@ -38,7 +42,6 @@ use zarrs_filesystem::FilesystemStore; /// - arrays: "temperature" (UInt8) and "pressure" (UInt8) /// /// Returns the temp dir (kept alive by the caller so files persist). -/// fn build_fixture() -> TempDir { let tmp = TempDir::new().unwrap(); let store = Arc::new(FilesystemStore::new(tmp.path()).unwrap()); @@ -58,7 +61,7 @@ fn build_fixture() -> TempDir { .store_metadata() .unwrap(); - for (name, base) in [("temperature", 0u8), ("pressure", 100u8)] { + for name in ["temperature", "pressure"] { let array = ArrayBuilder::new( vec![2u64, 4u64, 4u64], vec![1u64, 2u64, 2u64], @@ -70,23 +73,13 @@ fn build_fixture() -> TempDir { .unwrap(); array.store_metadata().unwrap(); - // Fill each chunk with a deterministic pattern so we can verify - // the right chunk lands in the right row: - // pixel(t, y, x) = base + (t*16 + y*4 + x) - // Each chunk is 1×2×2 = 4 pixels. Chunk (t_idx, y_idx, x_idx) - // covers (t_idx, [2*y_idx..2*y_idx+2], [2*x_idx..2*x_idx+2]). + // Bytes don't matter for these tests — the loader doesn't read + // them. Write zeros so the chunk files exist for any future + // resolver fixture. for t in 0..2u64 { for yc in 0..2u64 { for xc in 0..2u64 { - let mut chunk = Vec::with_capacity(4); - for dy in 0..2u64 { - for dx in 0..2u64 { - let y = yc * 2 + dy; - let x = xc * 2 + dx; - chunk.push(base.wrapping_add((t * 16 + y * 4 + x) as u8)); - } - } - array.store_chunk(&[t, yc, xc], chunk).unwrap(); + array.store_chunk(&[t, yc, xc], vec![0u8; 4]).unwrap(); } } } @@ -96,10 +89,10 @@ fn build_fixture() -> TempDir { } #[test] -fn indb_round_trip_emits_one_row_per_chunk_position() { +fn round_trip_emits_one_row_per_chunk_position_with_outdb_anchors() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_indb_rasters(&uri, None).unwrap(); + let arr = group_to_rasters(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 8, "expected 8 chunk rows (2*2*2)"); @@ -117,59 +110,13 @@ fn indb_round_trip_emits_one_row_per_chunk_position() { // Bands are sorted by array path for determinism — `pressure` sorts // before `temperature` lexicographically, so band 0 is pressure and // band 1 is temperature. - // - // Pressure has base=100; chunk (t=0, y=0, x=0) covers y∈{0,1}, x∈{0,1} - // → pixel offsets {0, 1, 4, 5} → values {100, 101, 104, 105}. let pressure = r0.band(0).unwrap(); assert_eq!(pressure.raw_source_shape(), &[1, 2, 2]); assert_eq!(pressure.data_type(), BandDataType::UInt8); - assert!(pressure.is_indb()); - assert_eq!( - &*pressure.contiguous_data().unwrap(), - &[100u8, 101, 104, 105] - ); - // Chunk anchor is populated on InDb rows too as provenance — the - // `data` column carries the bytes, and `outdb_uri` records where - // they came from. - assert_eq!(pressure.outdb_format(), Some("zarr")); - let anchor = pressure.outdb_uri().expect("outdb_uri set on InDb band"); - assert!(anchor.contains("#array=pressure"), "got: {anchor}"); - assert!(anchor.contains("&chunk=0,0,0"), "got: {anchor}"); - - // Temperature has base=0 → same chunk holds {0, 1, 4, 5}. - let temperature = r0.band(1).unwrap(); - assert_eq!(&*temperature.contiguous_data().unwrap(), &[0u8, 1, 4, 5]); - - // Last row corresponds to chunk (t=1, y=1, x=1). Temperature pixels: - // t=1, y∈{2,3}, x∈{2,3} → 1*16 + y*4 + x → 26, 27, 30, 31. - let last = rasters.get(7).unwrap(); - let last_transform: Vec = last.transform().to_vec(); - assert_eq!(last_transform[0], 100.0 + 2.0); // x_off = 2 - assert_eq!(last_transform[3], 200.0 - 2.0); // y_off = 2, sy = -1 - // band 1 is temperature (per the sort-by-path order above). - let last_temp = last.band(1).unwrap(); - assert_eq!(&*last_temp.contiguous_data().unwrap(), &[26u8, 27, 30, 31]); -} - -#[test] -fn outdb_emits_chunk_anchors() { - let tmp = build_fixture(); - let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_outdb_rasters(&uri, None).unwrap(); - - let rasters = RasterStructArray::new(&arr); - assert_eq!(rasters.len(), 8); - - // OutDb rows have empty data column and chunk anchor URIs. - // Bands sort alphabetically by array path: pressure (band 0), then - // temperature (band 1). - let r0 = rasters.get(0).unwrap(); - let pressure = r0.band(0).unwrap(); assert!( !pressure.is_indb(), - "OutDb band must report is_indb() = false" + "loader emits OutDb rows — is_indb() must be false" ); - // "This is zarr" lives in outdb_format, not a URI scheme prefix. assert_eq!(pressure.outdb_format(), Some("zarr")); let anchor = pressure.outdb_uri().expect("outdb_uri set"); // Anchor is the group URI verbatim plus a fragment carrying array @@ -179,8 +126,12 @@ fn outdb_emits_chunk_anchors() { assert!(anchor.contains("#array=pressure"), "got: {anchor}"); assert!(anchor.contains("&chunk=0,0,0"), "got: {anchor}"); - // Last chunk position's temperature band points at chunk (1,1,1). + // Last row corresponds to chunk (t=1, y=1, x=1). Anchor + transform + // both reflect the translation. let last = rasters.get(7).unwrap(); + let last_transform: Vec = last.transform().to_vec(); + assert_eq!(last_transform[0], 100.0 + 2.0); // x_off = 2 + assert_eq!(last_transform[3], 200.0 - 2.0); // y_off = 2, sy = -1 let temp = last.band(1).unwrap(); let anchor = temp.outdb_uri().expect("outdb_uri set"); assert!(anchor.contains("#array=temperature"), "got: {anchor}"); @@ -197,7 +148,7 @@ fn errors_on_empty_group() { .store_metadata() .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); + let err = group_to_rasters(&uri, None).unwrap_err().to_string(); assert!(err.contains("no child arrays"), "got: {err}"); } @@ -226,13 +177,6 @@ fn build_xarray_style_fixture() -> TempDir { .build(store.clone(), &format!("/{name}")) .unwrap(); arr.store_metadata().unwrap(); - for t in 0..2u64 { - for yc in 0..2u64 { - for xc in 0..2u64 { - arr.store_chunk(&[t, yc, xc], vec![0u8; 4]).unwrap(); - } - } - } } // 1-D coord variables. Different chunk grid than the data arrays — @@ -252,7 +196,7 @@ fn build_xarray_style_fixture() -> TempDir { fn auto_skips_1d_coord_variables() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_indb_rasters(&uri, None).unwrap(); + let arr = group_to_rasters(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); // 2*2*2 = 8 chunk positions, with 2 bands per row (pressure, temperature). assert_eq!(rasters.len(), 8); @@ -265,7 +209,7 @@ fn explicit_arrays_filter_selects_subset() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); let filter = vec!["temperature".to_string()]; - let arr = group_to_indb_rasters(&uri, Some(&filter)).unwrap(); + let arr = group_to_rasters(&uri, Some(&filter)).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 8); let r0 = rasters.get(0).unwrap(); @@ -277,7 +221,7 @@ fn explicit_arrays_filter_rejects_unknown_name() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); let filter = vec!["humidity".to_string()]; - let err = group_to_indb_rasters(&uri, Some(&filter)) + let err = group_to_rasters(&uri, Some(&filter)) .unwrap_err() .to_string(); assert!(err.contains("humidity"), "got: {err}"); @@ -308,7 +252,7 @@ fn errors_when_crs_declared_without_transform() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); + let err = group_to_rasters(&uri, None).unwrap_err().to_string(); assert!(err.contains("CRS"), "got: {err}"); assert!(err.contains("spatial:transform"), "got: {err}"); } @@ -321,7 +265,7 @@ fn explicit_arrays_filter_rejects_1d_arrays() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); let filter = vec!["t".to_string()]; - let err = group_to_indb_rasters(&uri, Some(&filter)) + let err = group_to_rasters(&uri, Some(&filter)) .unwrap_err() .to_string(); assert!(err.contains("\"t\""), "got: {err}"); @@ -346,7 +290,7 @@ fn errors_when_group_has_only_1d_arrays() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); + let err = group_to_rasters(&uri, None).unwrap_err().to_string(); assert!(err.contains("only 1-D arrays"), "got: {err}"); } @@ -372,16 +316,17 @@ fn falls_back_to_array_dimensions_attribute() { .build(store.clone(), "/temperature") .unwrap(); array.store_metadata().unwrap(); - array.store_chunk(&[0, 0], vec![10u8, 11]).unwrap(); - array.store_chunk(&[1, 0], vec![20u8, 21]).unwrap(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_indb_rasters(&uri, None).unwrap(); + let arr = group_to_rasters(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 2); let r0 = rasters.get(0).unwrap(); let band = r0.band(0).unwrap(); - assert_eq!(&*band.contiguous_data().unwrap(), &[10u8, 11]); + // Anchor URI populated even though the array used the v2 dim-name + // fallback — proves the fallback feeds the rest of the pipeline. + assert_eq!(band.outdb_format(), Some("zarr")); + assert!(band.outdb_uri().unwrap().contains("#array=temperature")); } #[test] @@ -407,7 +352,7 @@ fn errors_on_mismatched_chunk_grids() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_indb_rasters(&uri, None).unwrap_err().to_string(); + let err = group_to_rasters(&uri, None).unwrap_err().to_string(); assert!( err.contains("chunk") && err.contains("array_a") && err.contains("array_b"), "got: {err}" diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona/src/zarr_read.rs index de35feed4..4b4d3260b 100644 --- a/rust/sedona/src/zarr_read.rs +++ b/rust/sedona/src/zarr_read.rs @@ -22,7 +22,7 @@ //! SELECT * FROM sd_read_zarr('file:///path/to/datacube.zarr'); //! SELECT count(*) FROM sd_read_zarr( //! 'file:///path/to/datacube.zarr', -//! '{"load_eager": true, "rows_per_batch": 256}' +//! '{"rows_per_batch": 256}' //! ); //! ``` //! @@ -30,11 +30,11 @@ //! position in the Zarr group's chunk grid. All existing `RS_*` UDFs //! operate on the column unchanged. //! -//! `load_eager` defaults to `true` so byte-reading kernels work end-to-end -//! without depending on a registered OutDb resolver. When the async -//! `RS_EnsureLoaded` UDF lands, `load_eager = true` will be reinterpreted -//! as the planner auto-injecting that UDF over the scan output instead -//! of fetching at plan time. +//! `load_eager` defaults to `false` — every row has empty `data` and a +//! chunk-anchor URI in `outdb_uri`. Setting `load_eager = true` +//! currently errors because no `RS_EnsureLoaded` resolver is registered +//! for the `zarr` format yet; once the resolver lands, `true` will +//! trigger the planner to inject the async UDF over the scan output. use std::any::Any; use std::sync::Arc; @@ -57,7 +57,7 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::Expr; use datafusion_common::{plan_err, DataFusionError, ScalarValue}; -use sedona_raster_zarr::{group_to_indb_rasters, group_to_outdb_rasters}; +use sedona_raster_zarr::group_to_rasters; use sedona_schema::datatypes::SedonaType; use serde::{Deserialize, Serialize}; @@ -66,12 +66,11 @@ use serde::{Deserialize, Serialize}; /// Accepts one or two string arguments: /// - `uri` (required) — Zarr group URI (e.g. `file:///path/to/foo.zarr`). /// - `options_json` (optional) — JSON string with any of: -/// - `load_eager`: `true` (default) materializes chunk bytes into -/// the Arrow `data` column eagerly; `false` emits chunk-anchor -/// URIs only and defers byte resolution to the OutDb loader. -/// Long-term, `load_eager = true` will instead trigger the -/// planner to inject an async `RS_EnsureLoaded` over the scan -/// output rather than fetching at plan time. +/// - `load_eager`: `false` (default) emits chunk-anchor URIs only; +/// `true` currently errors because no `RS_EnsureLoaded` resolver +/// is registered for `zarr` yet. Once the resolver lands, `true` +/// will trigger the planner to inject the async UDF over the +/// scan output. /// - `rows_per_batch`: chunks per `RecordBatch`. Defaults to the /// session's configured batch size (`SessionConfig::batch_size`, /// typically 8192) when unset. @@ -116,12 +115,9 @@ fn literal_utf8(expr: &Expr, label: &str) -> Result { /// Materialised view backing `sd_read_zarr`. Holds the raster /// `StructArray` once for the query lifetime; the executor slices it -/// into `rows_per_batch`-sized `RecordBatch`es. -/// -/// The StructArray is built eagerly in `try_new`. When `indb=false` -/// the array is cheap to construct (chunk anchor URIs only); when -/// `indb=true` it pulls every chunk's bytes through the loader at -/// plan time. +/// into `rows_per_batch`-sized `RecordBatch`es. The StructArray is +/// built eagerly in `try_new` and is cheap — chunk-anchor URIs only, +/// no pixel bytes fetched. #[derive(Debug)] pub struct ZarrChunkProvider { schema: SchemaRef, @@ -134,7 +130,16 @@ pub struct ZarrChunkProvider { impl ZarrChunkProvider { fn try_new(uri: &str, options_json: Option) -> Result { let opts = parse_options(options_json.as_deref())?; - let load_eager = opts.load_eager.unwrap_or(true); + let load_eager = opts.load_eager.unwrap_or(false); + if load_eager { + return plan_err!( + "sd_read_zarr() load_eager = true is not yet supported. \ + Pixel-byte materialisation will be wired up when the async \ + RS_EnsureLoaded resolver lands; for now use load_eager = false \ + (the default) and operate on metadata (count(*), RS_Envelope, \ + RS_Width, etc.)." + ); + } let rows_per_batch = opts.rows_per_batch.map(|n| n.max(1)); let num_partitions = opts.num_partitions.unwrap_or(1); if num_partitions != 1 { @@ -145,11 +150,7 @@ impl ZarrChunkProvider { } let arrays_filter = opts.arrays.as_deref(); - let rasters = if load_eager { - group_to_indb_rasters(uri, arrays_filter).map_err(arrow_to_df_err)? - } else { - group_to_outdb_rasters(uri, arrays_filter).map_err(arrow_to_df_err)? - }; + let rasters = group_to_rasters(uri, arrays_filter).map_err(arrow_to_df_err)?; // Single-column schema: `raster: Raster`. `SedonaType::Raster` adds // the `sedona.raster` extension-type metadata so downstream RS_* @@ -335,12 +336,10 @@ fn arrow_to_df_err(e: arrow_schema::ArrowError) -> DataFusionError { #[derive(Serialize, Deserialize, Default)] struct ZarrReadOptions { - /// `true` (default) materializes chunk bytes into the Arrow `data` - /// column eagerly; `false` emits chunk-anchor URIs only and defers - /// byte resolution to the OutDb loader. Long-term, `load_eager = - /// true` will trigger the planner to inject an async - /// `RS_EnsureLoaded` over the scan output rather than fetching at - /// plan time. + /// `false` (default) emits chunk-anchor URIs only. `true` currently + /// errors — pixel-byte materialisation is pending the async + /// `RS_EnsureLoaded` resolver. Once the resolver lands, `true` will + /// trigger the planner to inject the async UDF over the scan output. load_eager: Option, rows_per_batch: Option, num_partitions: Option, @@ -396,7 +395,7 @@ mod tests { } #[tokio::test] - async fn udtf_returns_one_row_per_chunk_position_with_pixel_bytes() { + async fn udtf_returns_one_row_per_chunk_position_with_outdb_anchor() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); @@ -413,7 +412,8 @@ mod tests { assert_eq!(total_rows, 2, "expected 2 chunk rows"); // Pull the raster column out, hand to RasterStructArray, verify - // chunk 0's bytes round-trip. + // chunk 0's metadata + anchor. Pixel-byte reading lives behind the + // future RS_EnsureLoaded resolver — `data` is empty in OutDb mode. let raster_col = batches[0].column(0); let struct_arr = raster_col .as_any() @@ -422,7 +422,30 @@ mod tests { let rasters = RasterStructArray::new(struct_arr); let r0 = rasters.get(0).unwrap(); let band = r0.band(0).unwrap(); - assert_eq!(&*band.contiguous_data().unwrap(), &[10u8, 11]); + assert!(!band.is_indb(), "loader emits OutDb rows"); + assert_eq!(band.outdb_format(), Some("zarr")); + let anchor = band.outdb_uri().expect("outdb_uri set"); + assert!(anchor.contains("#array=temperature"), "got: {anchor}"); + assert!(anchor.contains("&chunk=0,0"), "got: {anchor}"); + } + + #[tokio::test] + async fn udtf_rejects_load_eager_true() { + let tmp = build_fixture(); + let uri = format!("file://{}", tmp.path().display()); + + let ctx = SessionContext::new(); + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + + let err = ctx + .sql(&format!( + r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"load_eager": true}}')"#, + )) + .await + .unwrap_err() + .to_string(); + assert!(err.contains("load_eager = true"), "got: {err}"); + assert!(err.contains("not yet supported"), "got: {err}"); } #[tokio::test] From 52dc4ce2eec2c461a2a5dd37d86446700bb33780 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 15:09:35 -0700 Subject: [PATCH 08/36] =?UTF-8?q?refactor(raster-zarr):=20plugin=20archite?= =?UTF-8?q?cture=20=E2=80=94=20streaming=20reader=20+=20ExternalFormatSpec?= =?UTF-8?q?=20+=20UDTF=20migration?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Moves all Zarr functionality out of the `sedona` crate so the main package no longer carries a zarr dep. Three structural changes: 1. **Streaming `ZarrChunkReader: RecordBatchReader`** — replaces the eager `group_to_rasters -> StructArray` function. Opens the group + parses metadata once in `try_new`; per-batch work is just transform arithmetic and anchor URI formatting. `next()` walks `batch_size` chunk positions of the chunk grid and emits one `RecordBatch`. The chunk anchors are still OutDb-style (empty `data`, populated `outdb_uri`); pixel-byte resolution is the future async resolver's job. 2. **`ZarrFormatSpec: ExternalFormatSpec`** — wraps the reader with the standard SedonaDB datasource API so `con.read_format(spec, uri)` works the same shape as PyogrioFormatSpec. Lives in `sedona-raster-zarr` gated under a `zarr` feature (default on). 3. **UDTF migrates from `sedona` to `sedona-raster-zarr`.** The `ZarrReadFunction` / `ZarrChunkProvider` / `ZarrChunkExec` code moves to `sedona-raster-zarr/src/udtf.rs` (also `zarr`-feature-gated). The exec is now genuinely streaming — it constructs a fresh `ZarrChunkReader` in `execute()` and adapts it via `RecordBatchStreamAdapter`. The new `register(ctx: &SessionContext)` helper is what the Python plugin package will call. The `sedona` crate's `zarr_read` module is deleted; the `sedona_raster_zarr` workspace dep is dropped; the `SedonaContext::new_from_context` UDTF registration is removed. The Python `sd_read_zarr` wrapper method in `functions/table.py` and the `test_read_zarr` test go too — they get re-introduced in the new `sedonadb-zarr` plugin package (next commit). Plan-time validation: `ZarrChunkProvider::try_new` opens the reader once for validation and drops it, so URI / metadata / arrays-filter errors still surface at `ctx.sql(...).await` rather than at collect time. Cost is one extra group-open per scan. Result: `sedona` crate has zero zarr code; `sedona-raster-zarr` is fully self-contained with both the format-spec and UDTF integration surfaces under `feature = "zarr"`. The plugin Python package wires both via `sedonadb_zarr.register(con)`. --- Cargo.lock | 11 +- .../python/sedonadb/functions/table.py | 70 +---- python/sedonadb/tests/test_funcs.py | 49 --- rust/sedona-raster-zarr/Cargo.toml | 25 ++ rust/sedona-raster-zarr/src/format_spec.rs | 125 ++++++++ rust/sedona-raster-zarr/src/lib.rs | 26 +- rust/sedona-raster-zarr/src/loader.rs | 295 +++++++++++++----- .../src/udtf.rs} | 138 ++++---- .../tests/zarr_roundtrip.rs | 48 ++- rust/sedona/Cargo.toml | 9 - rust/sedona/src/context.rs | 5 +- rust/sedona/src/lib.rs | 1 - 12 files changed, 500 insertions(+), 302 deletions(-) create mode 100644 rust/sedona-raster-zarr/src/format_spec.rs rename rust/{sedona/src/zarr_read.rs => sedona-raster-zarr/src/udtf.rs} (84%) diff --git a/Cargo.lock b/Cargo.lock index c3eb474ac..6027a1469 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5669,7 +5669,6 @@ dependencies = [ "sedona-raster", "sedona-raster-functions", "sedona-raster-gdal", - "sedona-raster-zarr", "sedona-s2geography", "sedona-schema", "sedona-spatial-join", @@ -5683,8 +5682,6 @@ dependencies = [ "tempfile", "tokio", "url", - "zarrs", - "zarrs_filesystem", ] [[package]] @@ -6159,13 +6156,21 @@ version = "0.4.0" dependencies = [ "arrow-array", "arrow-schema", + "async-trait", + "datafusion", + "datafusion-catalog", "datafusion-common", + "datafusion-expr", + "futures", "log", "sedona-common", + "sedona-datasource", "sedona-raster", "sedona-schema", + "serde", "serde_json", "tempfile", + "tokio", "zarrs", "zarrs_filesystem", ] diff --git a/python/sedonadb/python/sedonadb/functions/table.py b/python/sedonadb/python/sedonadb/functions/table.py index fd4c42b87..30a2f285d 100644 --- a/python/sedonadb/python/sedonadb/functions/table.py +++ b/python/sedonadb/python/sedonadb/functions/table.py @@ -16,7 +16,7 @@ # under the License. import json -from typing import Optional, Literal, Union, Tuple, Iterable, List +from typing import Optional, Literal, Union, Tuple, Iterable from sedonadb.dataframe import DataFrame from sedonadb.utility import sedona # noqa: F401 @@ -112,71 +112,3 @@ def sd_random_geometry( args = {k: v for k, v in args.items() if v is not None} return self._ctx.sql(f"SELECT * FROM sd_random_geometry('{json.dumps(args)}')") - - def sd_read_zarr( - self, - uri: str, - *, - load_eager: Optional[bool] = None, - rows_per_batch: Optional[int] = None, - num_partitions: Optional[int] = None, - arrays: Optional[List[str]] = None, - ) -> DataFrame: - """ - Read a Zarr group as a DataFrame of N-D rasters. - - Returns a single-column DataFrame ``raster: Raster`` with one row per - chunk position in the Zarr group's chunk grid. Each row's bands are - the corresponding chunks of each array in the group. All ``RS_*`` - UDFs operate on the column unchanged. - - Only local filesystem stores are supported (``file://`` URIs or - bare paths). - - Parameters - ---------- - uri : str - Zarr group URI. ``file:///path/to/foo.zarr`` or a bare local path. - load_eager : bool, optional - ``True`` (default) materializes every chunk's bytes into the - Arrow ``data`` column eagerly. ``False`` emits chunk-anchor - URIs only; byte resolution depends on the OutDb resolver - being registered (follow-up PR). Long-term, ``load_eager = - True`` will trigger the planner to inject an async - ``RS_EnsureLoaded`` over the scan output rather than - fetching at plan time. - rows_per_batch : int, optional - Chunks per ``RecordBatch`` (default 1024). - num_partitions : int, optional - Scan partitions. Only ``1`` is supported; ``> 1`` errors. - arrays : list of str, optional - Names of arrays in the group to read. By default every - multi-dimensional array is read; 1-D arrays (typical xarray - coord variables) are auto-skipped. Passing an explicit list - reads exactly those arrays. 1-D arrays are always rejected - (a raster band needs at least 2 dimensions); naming one - explicitly errors. Unknown names also error. - - Examples - -------- - >>> sd = sedona.db.connect() - >>> sd.funcs.table.sd_read_zarr("file:///path/to/datacube.zarr") # doctest: +SKIP - >>> sd.funcs.table.sd_read_zarr( - ... "file:///path/to/datacube.zarr", - ... arrays=["temperature", "pressure"], - ... ) # doctest: +SKIP - """ - - args = { - "load_eager": load_eager, - "rows_per_batch": rows_per_batch, - "num_partitions": num_partitions, - "arrays": arrays, - } - args = {k: v for k, v in args.items() if v is not None} - - if args: - return self._ctx.sql( - f"SELECT * FROM sd_read_zarr('{uri}', '{json.dumps(args)}')" - ) - return self._ctx.sql(f"SELECT * FROM sd_read_zarr('{uri}')") diff --git a/python/sedonadb/tests/test_funcs.py b/python/sedonadb/tests/test_funcs.py index daa141568..2c6fb9956 100644 --- a/python/sedonadb/tests/test_funcs.py +++ b/python/sedonadb/tests/test_funcs.py @@ -16,10 +16,6 @@ # under the License. -import numpy as np -import pytest - - def test_random_geometry(con): df = con.funcs.table.sd_random_geometry("Point", 5, seed=99873) @@ -28,48 +24,3 @@ def test_random_geometry(con): # Ensure the output is reproducible assert df.to_arrow_table() == df.to_arrow_table() - - -def test_read_zarr(con, tmp_path): - # `zarr` is an optional fixture dep — skip if not installed. The - # loader itself is exercised by the Rust-side integration tests; - # this test verifies the Python wrapper threads arguments through - # and materialises a raster row into Python. - zarr = pytest.importorskip("zarr") - - # Build a 2x2 UInt8 array with two chunks, dim_names=["y","x"], inside - # a group at the temp path. Matches the minimal shape sd_read_zarr - # expects (2-D array with [y, x] suffix). - root = zarr.open_group(str(tmp_path), mode="w") - arr = root.create_array( - "temperature", - shape=(2, 2), - chunks=(1, 2), - dtype="uint8", - dimension_names=["y", "x"], - ) - arr[:] = np.array([[10, 11], [20, 21]], dtype=np.uint8) - - # Default read emits OutDb-style rows — `data` is empty, - # `outdb_uri` carries a chunk anchor. Pixel-byte resolution is - # deferred to the future RS_EnsureLoaded resolver. Materialise - # through Arrow so we can inspect the Raster struct. - df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}") - arrow_tab = df.to_arrow_table() - assert arrow_tab.num_rows == 2 - assert arrow_tab.column_names == ["raster"] - raster = arrow_tab["raster"][0].as_py() - assert isinstance(raster, dict) - # The Raster struct exposes at least these top-level fields; their - # exact contents are covered by the Rust tests. - for field in ("transform", "bands"): - assert field in raster, f"raster row missing {field!r}: {sorted(raster)}" - - # Options thread through: rows_per_batch slices the output. - df = con.funcs.table.sd_read_zarr(f"file://{tmp_path}", rows_per_batch=1) - assert df.count() == 2 - - # load_eager=True is not yet supported — errors with a clear - # pointer at the future RS_EnsureLoaded resolver. - with pytest.raises(Exception, match="load_eager"): - con.funcs.table.sd_read_zarr(f"file://{tmp_path}", load_eager=True).count() diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml index 807f148f1..36895151b 100644 --- a/rust/sedona-raster-zarr/Cargo.toml +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -30,14 +30,38 @@ rust-version.workspace = true [lints.clippy] result_large_err = "allow" +[features] +# The `zarr` feature gates the SedonaDB integration (ExternalFormatSpec +# impl, sd_read_zarr UDTF). Without it, the crate exposes only the +# streaming `ZarrChunkReader` primitive. Default-on so most users get +# the full integration; downstreams that only want the raw reader can +# turn it off via `default-features = false`. +default = ["zarr"] +zarr = [ + "dep:async-trait", + "dep:datafusion", + "dep:datafusion-catalog", + "dep:datafusion-expr", + "dep:futures", + "dep:sedona-datasource", + "dep:serde", +] + [dependencies] arrow-array = { workspace = true } arrow-schema = { workspace = true } +async-trait = { workspace = true, optional = true } +datafusion = { workspace = true, optional = true, features = ["sql"] } +datafusion-catalog = { workspace = true, optional = true } datafusion-common = { workspace = true } +datafusion-expr = { workspace = true, optional = true } +futures = { workspace = true, optional = true } log = { workspace = true } sedona-common = { workspace = true } +sedona-datasource = { workspace = true, optional = true } sedona-raster = { workspace = true } sedona-schema = { workspace = true } +serde = { workspace = true, optional = true } serde_json = { workspace = true } zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } zarrs_filesystem = { workspace = true } @@ -52,3 +76,4 @@ zarrs = { workspace = true, features = ["blosc"] } [dev-dependencies] tempfile = { workspace = true } +tokio = { workspace = true, features = ["macros", "rt-multi-thread"] } diff --git a/rust/sedona-raster-zarr/src/format_spec.rs b/rust/sedona-raster-zarr/src/format_spec.rs new file mode 100644 index 000000000..b1003c491 --- /dev/null +++ b/rust/sedona-raster-zarr/src/format_spec.rs @@ -0,0 +1,125 @@ +// 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. + +//! `ExternalFormatSpec` impl for Zarr groups. +//! +//! Wraps [`ZarrChunkReader`] with the standard SedonaDB datasource API +//! so users can read Zarr groups via `con.read_format(spec, uri)` or +//! through `ListingTable`-style file discovery. + +use std::{collections::HashMap, sync::Arc}; + +use arrow_array::RecordBatchReader; +use arrow_schema::Schema; +use async_trait::async_trait; +use datafusion_common::{DataFusionError, Result}; +use sedona_datasource::spec::{ExternalFormatSpec, Object, OpenReaderArgs}; +use sedona_schema::datatypes::SedonaType; + +use crate::loader::ZarrChunkReader; + +/// `ExternalFormatSpec` implementation for Zarr groups. +/// +/// Configurable via [`with_options`](ExternalFormatSpec::with_options): +/// - `load_eager`: boolean. `false` (default) emits chunk-anchor URIs +/// only. `true` currently errors — pixel-byte materialisation is +/// pending the async `RS_EnsureLoaded` resolver. +/// - `arrays`: JSON array of strings, e.g. `'["temperature","pressure"]'`. +/// Names a subset of arrays in the group to read; defaults to every +/// multi-dimensional array (1-D coord variables auto-skipped). +#[derive(Debug, Clone, Default)] +pub struct ZarrFormatSpec { + load_eager: bool, + arrays: Option>, +} + +impl ZarrFormatSpec { + pub fn new() -> Self { + Self::default() + } +} + +#[async_trait] +impl ExternalFormatSpec for ZarrFormatSpec { + async fn infer_schema(&self, _location: &Object) -> Result { + // The Zarr loader always produces the canonical single-column + // Raster schema. No I/O needed to know that. + let field = SedonaType::Raster + .to_storage_field("raster", true) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + Ok(Schema::new(vec![field])) + } + + async fn open_reader( + &self, + args: &OpenReaderArgs, + ) -> Result> { + if self.load_eager { + return Err(DataFusionError::Plan( + "ZarrFormatSpec: load_eager = true is not yet supported. \ + Pixel-byte materialisation will be wired up when the async \ + RS_EnsureLoaded resolver lands." + .into(), + )); + } + let uri = args.src.to_url_string().ok_or_else(|| { + DataFusionError::Plan( + "ZarrFormatSpec: could not resolve a URL string from the source object".into(), + ) + })?; + let batch_size = args.batch_size.unwrap_or(8192); + let reader = ZarrChunkReader::try_new(&uri, self.arrays.as_deref(), batch_size) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + Ok(Box::new(reader)) + } + + fn with_options( + &self, + options: &HashMap, + ) -> Result> { + let mut next = self.clone(); + for (k, v) in options { + match k.as_str() { + "load_eager" => { + next.load_eager = v.parse().map_err(|_| { + DataFusionError::Plan(format!( + "ZarrFormatSpec: load_eager must be a boolean; got {v:?}" + )) + })?; + } + "arrays" => { + next.arrays = Some(serde_json::from_str::>(v).map_err(|e| { + DataFusionError::Plan(format!( + "ZarrFormatSpec: arrays must be a JSON array of strings; \ + got {v:?} ({e})" + )) + })?); + } + other => { + return Err(DataFusionError::Plan(format!( + "ZarrFormatSpec: unknown option {other:?}" + ))); + } + } + } + Ok(Arc::new(next)) + } + + fn extension(&self) -> &str { + ".zarr" + } +} diff --git a/rust/sedona-raster-zarr/src/lib.rs b/rust/sedona-raster-zarr/src/lib.rs index 800c009a8..f57c731d0 100644 --- a/rust/sedona-raster-zarr/src/lib.rs +++ b/rust/sedona-raster-zarr/src/lib.rs @@ -22,12 +22,14 @@ //! array in the group, mapped onto SedonaDB's canonical N-D raster Arrow //! schema. //! -//! Single entry point: [`group_to_rasters`] always emits OutDb-style -//! rows — `data` is empty, `outdb_uri` carries a chunk anchor that the -//! async OutDb resolver (registered separately, lands in a follow-up) -//! turns into pixel bytes on demand. Metadata-only operations -//! (`count(*)`, `RS_Envelope`, `RS_Width`, …) work today; byte-consuming -//! kernels require the resolver to be registered. +//! Single entry point: [`ZarrChunkReader`] is a `RecordBatchReader` +//! that walks the group's chunk grid lazily, emitting one batch per +//! `next()` call. Each row carries chunk-anchor URIs in `outdb_uri`; +//! `data` is empty until the async OutDb resolver (registered +//! separately, lands in a follow-up) materialises the bytes. +//! Metadata-only operations (`count(*)`, `RS_Envelope`, `RS_Width`, …) +//! work today; byte-consuming kernels require the resolver to be +//! registered. //! //! Local filesystem stores only — `file://` URIs or bare paths. @@ -36,4 +38,14 @@ pub mod geozarr; pub mod loader; pub mod source_uri; -pub use loader::group_to_rasters; +#[cfg(feature = "zarr")] +pub mod format_spec; +#[cfg(feature = "zarr")] +pub mod udtf; + +pub use loader::ZarrChunkReader; + +#[cfg(feature = "zarr")] +pub use format_spec::ZarrFormatSpec; +#[cfg(feature = "zarr")] +pub use udtf::{register as register_udtf, ZarrReadFunction}; diff --git a/rust/sedona-raster-zarr/src/loader.rs b/rust/sedona-raster-zarr/src/loader.rs index e5fe2b5cf..879c3544e 100644 --- a/rust/sedona-raster-zarr/src/loader.rs +++ b/rust/sedona-raster-zarr/src/loader.rs @@ -32,10 +32,11 @@ use std::sync::Arc; -use arrow_array::StructArray; -use arrow_schema::ArrowError; +use arrow_array::{RecordBatch, RecordBatchReader}; +use arrow_schema::{ArrowError, Schema, SchemaRef}; use sedona_common::sedona_internal_datafusion_err; use sedona_raster::builder::RasterBuilder; +use sedona_schema::datatypes::SedonaType; use sedona_schema::raster::BandDataType; use zarrs::array::{Array, ArrayBytes}; use zarrs::group::Group; @@ -45,27 +46,194 @@ use crate::dtype::zarr_to_band_data_type; use crate::geozarr::GroupGeoMetadata; use crate::source_uri::{build_chunk_anchor, group_uri_to_filesystem_path}; -/// Open a Zarr group and emit one row per chunk position with -/// chunk-anchor URIs in each band's `outdb_uri`. The `data` column is -/// empty; bytes resolve on demand through whichever OutDb loader is -/// registered for the `zarr` format. No resolver is registered yet — -/// downstream byte-consuming kernels (`RS_Value` and similar) will -/// error until the async resolver lands. Metadata-only operations -/// (`count(*)`, `RS_Envelope`, `RS_Width`, etc.) work today. +/// Streaming reader over the chunk grid of a Zarr group. /// -/// `arrays`: -/// - `None` — read every multi-dimensional array. 1-D arrays (typical -/// coord variables in xarray-style datacubes) are auto-skipped. -/// - `Some(names)` — read exactly the named arrays, in the order -/// returned by the store's listing (which is then sorted by path). An -/// unknown name errors. 1-D arrays are always rejected (a raster band -/// needs ≥ 2 dimensions); naming one explicitly errors at parse time -/// with a clear message. -pub fn group_to_rasters( - group_uri: &str, - arrays: Option<&[String]>, -) -> Result { - build_rasters(group_uri, arrays) +/// Each `next()` call emits one `RecordBatch` containing up to +/// `batch_size` rows; one row per chunk position. The reader holds the +/// open group, parsed metadata, and the current chunk-grid position — +/// metadata parsing happens once in [`ZarrChunkReader::try_new`] and +/// the per-row work is just transform arithmetic + anchor URI +/// formatting. +/// +/// Rows always emit OutDb-style: `data` is empty, `outdb_uri` carries +/// a chunk anchor that the async OutDb resolver (registered separately) +/// resolves to bytes on demand. +pub struct ZarrChunkReader { + schema: SchemaRef, + group_uri: String, + array_infos: Vec, + geo: GroupGeoMetadata, + group_transform: [f64; 6], + spatial_dim_indices: Vec, + /// Owned copy of `array_infos[0].dim_names[i]` for `i` in + /// `spatial_dim_indices`. Kept here so `next()` doesn't need to + /// rebuild the `&str` slice each row. + spatial_dims_names: Vec, + chunk_spatial_shape: Vec, + /// Current position in the chunk grid (row-major, C-order). + chunk_indices: Vec, + /// Whether the grid has been exhausted. + exhausted: bool, + batch_size: usize, +} + +impl std::fmt::Debug for ZarrChunkReader { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("ZarrChunkReader") + .field("group_uri", &self.group_uri) + .field("num_arrays", &self.array_infos.len()) + .field("chunk_indices", &self.chunk_indices) + .field("exhausted", &self.exhausted) + .field("batch_size", &self.batch_size) + .finish() + } +} + +impl ZarrChunkReader { + /// Open a Zarr group and prepare a streaming reader over its chunk + /// grid. Performs all I/O for metadata up front (open group, parse + /// attributes, list arrays, validate constraints) so per-batch work + /// is cheap. + /// + /// `arrays`: + /// - `None` — read every multi-dimensional array. 1-D arrays + /// (typical xarray coord variables) are auto-skipped. + /// - `Some(names)` — read exactly the named arrays. Unknown names + /// error. 1-D arrays are always rejected (a raster band needs + /// ≥ 2 dimensions); naming one explicitly errors with a clear + /// message. + /// + /// `batch_size` controls how many chunk rows are emitted per + /// `RecordBatch`. Must be ≥ 1; callers typically pass + /// `SessionConfig::batch_size` (defaults to 8192). + pub fn try_new( + group_uri: &str, + arrays: Option<&[String]>, + batch_size: usize, + ) -> Result { + let batch_size = batch_size.max(1); + let OpenedGroup { + array_infos, + geo, + group_transform, + spatial_dim_indices, + } = open_and_validate(group_uri, arrays)?; + + let spatial_dims_names: Vec = spatial_dim_indices + .iter() + .map(|&i| array_infos[0].dim_names[i].clone()) + .collect(); + let chunk_spatial_shape: Vec = spatial_dim_indices + .iter() + .map(|&i| array_infos[0].chunk_shape[i] as i64) + .collect(); + + let raster_field = SedonaType::Raster + .to_storage_field("raster", true) + .map_err(|e| ArrowError::ExternalError(Box::new(e)))?; + let schema = Arc::new(Schema::new(vec![raster_field])); + + let chunk_indices = vec![0u64; array_infos[0].chunk_grid_shape.len()]; + + Ok(Self { + schema, + group_uri: group_uri.to_string(), + array_infos, + geo, + group_transform, + spatial_dim_indices, + spatial_dims_names, + chunk_spatial_shape, + chunk_indices, + exhausted: false, + batch_size, + }) + } + + /// Append one raster row at the current `chunk_indices` to the + /// in-progress builder. Does not advance the cursor. + fn emit_one_row(&self, builder: &mut RasterBuilder) -> Result<(), ArrowError> { + let row_transform = compute_row_transform( + &self.group_transform, + &self.chunk_indices, + &self.array_infos[0].chunk_shape, + &self.spatial_dim_indices, + ); + let spatial_dims_ref: Vec<&str> = + self.spatial_dims_names.iter().map(String::as_str).collect(); + let crs_str = self.geo.crs.as_deref(); + builder.start_raster_nd( + &row_transform, + &spatial_dims_ref, + &self.chunk_spatial_shape, + crs_str, + )?; + + for info in &self.array_infos { + let dim_names_ref: Vec<&str> = info.dim_names.iter().map(String::as_str).collect(); + let nodata_ref = info.nodata.as_deref(); + // Every band gets its chunk-anchor URI populated as + // provenance metadata. `data.is_empty()` is the InDb/OutDb + // discriminator; this reader always emits empty `data` and + // defers pixel-byte resolution to the OutDb resolver. + let anchor = build_chunk_anchor(&self.group_uri, &info.path, &self.chunk_indices); + builder.start_band_nd( + Some(info.path.as_str()), + &dim_names_ref, + &info.chunk_shape, + info.data_type, + nodata_ref, + Some(anchor.as_str()), + Some("zarr"), + )?; + builder.band_data_writer().append_value([0u8; 0]); + builder.finish_band()?; + } + builder.finish_raster() + } +} + +impl Iterator for ZarrChunkReader { + type Item = Result; + + fn next(&mut self) -> Option { + if self.exhausted { + return None; + } + let mut builder = RasterBuilder::new(self.batch_size); + let mut rows_emitted = 0usize; + while rows_emitted < self.batch_size { + if let Err(e) = self.emit_one_row(&mut builder) { + self.exhausted = true; + return Some(Err(e)); + } + rows_emitted += 1; + if !advance_chunk_indices( + &mut self.chunk_indices, + &self.array_infos[0].chunk_grid_shape, + ) { + self.exhausted = true; + break; + } + } + if rows_emitted == 0 { + return None; + } + let struct_arr = match builder.finish() { + Ok(arr) => arr, + Err(e) => return Some(Err(e)), + }; + Some(RecordBatch::try_new( + self.schema.clone(), + vec![Arc::new(struct_arr)], + )) + } +} + +impl RecordBatchReader for ZarrChunkReader { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } } /// Per-array metadata extracted once at group open and reused for every @@ -92,10 +260,21 @@ struct ArrayInfo { nodata: Option>, } -fn build_rasters( +/// Bundle of group-level state returned by [`open_and_validate`]. +struct OpenedGroup { + array_infos: Vec, + geo: GroupGeoMetadata, + group_transform: [f64; 6], + spatial_dim_indices: Vec, +} + +/// Open the Zarr group, parse and validate group metadata, and return +/// everything `ZarrChunkReader` needs to iterate without further I/O +/// (apart from per-chunk byte fetches by future resolvers). +fn open_and_validate( group_uri: &str, arrays_filter: Option<&[String]>, -) -> Result { +) -> Result { let fs_path = group_uri_to_filesystem_path(group_uri)?; let store = FilesystemStore::new(&fs_path).map_err(|e| { ArrowError::ExternalError(Box::new(sedona_internal_datafusion_err!( @@ -151,14 +330,6 @@ fn build_rasters( // would produce wrong per-row transforms. let spatial_dim_indices = resolve_spatial_dim_indices(&array_infos[0].dim_names, geo.spatial_dims.as_deref())?; - let spatial_dims_names: Vec<&str> = spatial_dim_indices - .iter() - .map(|&i| array_infos[0].dim_names[i].as_str()) - .collect(); - let chunk_spatial_shape: Vec = spatial_dim_indices - .iter() - .map(|&i| array_infos[0].chunk_shape[i] as i64) - .collect(); let group_transform = match geo.transform { Some(t) => t, @@ -177,9 +348,9 @@ fn build_rasters( }; // chunk_grid_shape comes from untrusted Zarr metadata; bound-check the - // product so a hostile or malformed grid can't drive RasterBuilder - // capacity to overflow (and trigger an OOM preallocation). - let total_rows = array_infos[0] + // product so a hostile or malformed grid can't make per-batch + // RasterBuilder capacity (or downstream consumers) overflow. + array_infos[0] .chunk_grid_shape .iter() .try_fold(1usize, |acc, &n| { @@ -191,55 +362,13 @@ fn build_rasters( array_infos[0].chunk_grid_shape )) })?; - let mut builder = RasterBuilder::new(total_rows); - - // Walk the chunk grid in row-major (C-order) order. The outer-most - // axis varies slowest, the innermost fastest — same convention used - // for byte strides in `BandRefImpl`. - let mut chunk_indices = vec![0u64; array_infos[0].chunk_grid_shape.len()]; - loop { - let row_transform = compute_row_transform( - &group_transform, - &chunk_indices, - &array_infos[0].chunk_shape, - &spatial_dim_indices, - ); - let crs_str = geo.crs.as_deref(); - builder.start_raster_nd( - &row_transform, - &spatial_dims_names, - &chunk_spatial_shape, - crs_str, - )?; - - for info in &array_infos { - let dim_names_ref: Vec<&str> = info.dim_names.iter().map(String::as_str).collect(); - let nodata_ref = info.nodata.as_deref(); - // Every band gets its chunk-anchor URI populated as - // provenance metadata. `data.is_empty()` is the InDb/OutDb - // discriminator; this loader always emits empty `data` and - // defers pixel-byte resolution to the OutDb resolver. - let anchor = build_chunk_anchor(group_uri, &info.path, &chunk_indices); - builder.start_band_nd( - Some(info.path.as_str()), - &dim_names_ref, - &info.chunk_shape, - info.data_type, - nodata_ref, - Some(anchor.as_str()), - Some("zarr"), - )?; - builder.band_data_writer().append_value([0u8; 0]); - builder.finish_band()?; - } - builder.finish_raster()?; - - if !advance_chunk_indices(&mut chunk_indices, &array_infos[0].chunk_grid_shape) { - break; - } - } - builder.finish() + Ok(OpenedGroup { + array_infos, + geo, + group_transform, + spatial_dim_indices, + }) } /// Collect per-array metadata from open zarrs `Array` handles. diff --git a/rust/sedona/src/zarr_read.rs b/rust/sedona-raster-zarr/src/udtf.rs similarity index 84% rename from rust/sedona/src/zarr_read.rs rename to rust/sedona-raster-zarr/src/udtf.rs index 4b4d3260b..c8c533043 100644 --- a/rust/sedona/src/zarr_read.rs +++ b/rust/sedona-raster-zarr/src/udtf.rs @@ -35,17 +35,21 @@ //! currently errors because no `RS_EnsureLoaded` resolver is registered //! for the `zarr` format yet; once the resolver lands, `true` will //! trigger the planner to inject the async UDF over the scan output. +//! +//! Registration happens via [`register`]; the `sedonadb-zarr` Python +//! package calls this from its session-setup helper. The `sedona` crate +//! itself does not register the UDTF — keeping zarr functionality out +//! of the default bootstrap. use std::any::Any; use std::sync::Arc; -use arrow_array::{Array, RecordBatch, StructArray}; use arrow_schema::{DataType, Schema, SchemaRef}; use async_trait::async_trait; use datafusion::catalog::{Session, TableFunctionImpl, TableProvider}; use datafusion::common::Result; use datafusion::datasource::TableType; -use datafusion::execution::context::TaskContext; +use datafusion::execution::context::{SessionContext, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::expressions::Column; @@ -57,10 +61,17 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::Expr; use datafusion_common::{plan_err, DataFusionError, ScalarValue}; -use sedona_raster_zarr::group_to_rasters; use sedona_schema::datatypes::SedonaType; use serde::{Deserialize, Serialize}; +use crate::loader::ZarrChunkReader; + +/// Register the `sd_read_zarr` UDTF on a `SessionContext`. Called from +/// the `sedonadb-zarr` Python package's `register(con)` entry point. +pub fn register(ctx: &SessionContext) { + ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); +} + /// Table function `sd_read_zarr(uri[, options_json])`. /// /// Accepts one or two string arguments: @@ -113,15 +124,22 @@ fn literal_utf8(expr: &Expr, label: &str) -> Result { plan_err!("{label} must be a non-null Utf8 literal; got {expr}") } -/// Materialised view backing `sd_read_zarr`. Holds the raster -/// `StructArray` once for the query lifetime; the executor slices it -/// into `rows_per_batch`-sized `RecordBatch`es. The StructArray is -/// built eagerly in `try_new` and is cheap — chunk-anchor URIs only, -/// no pixel bytes fetched. +/// `TableProvider` backing `sd_read_zarr`. Holds the URI + options; +/// the underlying `ZarrChunkReader` is opened fresh per `execute()` +/// call so each scan streams independently. +/// +/// Plan-time validation: `try_new` opens the group once and drops the +/// reader, so problems like "URI doesn't resolve", "no arrays", +/// "CRS without transform", and "unknown array name in the `arrays` +/// filter" surface at `ctx.sql(...).await` rather than at collect time. +/// The cost is one extra group-open per scan; for local files this is +/// negligible and for cloud it's two GETs — acceptable until lazy-open +/// becomes a measurable problem. #[derive(Debug)] pub struct ZarrChunkProvider { schema: SchemaRef, - rasters: StructArray, + uri: String, + arrays_filter: Option>, /// `None` defers to the session's `SessionConfig::batch_size` at /// execute time. Set explicitly via the `rows_per_batch` JSON option. rows_per_batch: Option, @@ -149,8 +167,13 @@ impl ZarrChunkProvider { ); } - let arrays_filter = opts.arrays.as_deref(); - let rasters = group_to_rasters(uri, arrays_filter).map_err(arrow_to_df_err)?; + let arrays_filter = opts.arrays; + + // Validate at plan time by opening the reader once and dropping + // it. Surfaces URI / metadata / arrays-filter errors at + // ctx.sql() rather than at collect(). + let _ = + ZarrChunkReader::try_new(uri, arrays_filter.as_deref(), 1).map_err(arrow_to_df_err)?; // Single-column schema: `raster: Raster`. `SedonaType::Raster` adds // the `sedona.raster` extension-type metadata so downstream RS_* @@ -162,7 +185,8 @@ impl ZarrChunkProvider { Ok(Self { schema, - rasters, + uri: uri.to_string(), + arrays_filter, rows_per_batch, }) } @@ -191,7 +215,8 @@ impl TableProvider for ZarrChunkProvider { ) -> Result> { let exec = Arc::new(ZarrChunkExec::new( self.schema.clone(), - self.rasters.clone(), + self.uri.clone(), + self.arrays_filter.clone(), self.rows_per_batch, )); // DataFusion requires the scan to honour the projection it asks @@ -218,13 +243,19 @@ impl TableProvider for ZarrChunkProvider { #[derive(Debug)] struct ZarrChunkExec { schema: SchemaRef, - rasters: StructArray, + uri: String, + arrays_filter: Option>, rows_per_batch: Option, properties: PlanProperties, } impl ZarrChunkExec { - fn new(schema: SchemaRef, rasters: StructArray, rows_per_batch: Option) -> Self { + fn new( + schema: SchemaRef, + uri: String, + arrays_filter: Option>, + rows_per_batch: Option, + ) -> Self { let properties = PlanProperties::new( EquivalenceProperties::new(schema.clone()), // A single partition for now. Round-robin across multiple @@ -235,7 +266,8 @@ impl ZarrChunkExec { ); Self { schema, - rasters, + uri, + arrays_filter, rows_per_batch, properties, } @@ -245,15 +277,11 @@ impl ZarrChunkExec { impl DisplayAs for ZarrChunkExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self.rows_per_batch { - Some(n) => write!( - f, - "ZarrChunkExec: rows={}, rows_per_batch={n}", - self.rasters.len() - ), + Some(n) => write!(f, "ZarrChunkExec: uri={}, rows_per_batch={n}", self.uri), None => write!( f, - "ZarrChunkExec: rows={}, rows_per_batch=session_default", - self.rasters.len() + "ZarrChunkExec: uri={}, rows_per_batch=session_default", + self.uri ), } } @@ -296,34 +324,25 @@ impl ExecutionPlan for ZarrChunkExec { return plan_err!("ZarrChunkExec: only partition 0 exists"); } - let total = self.rasters.len(); // Defer the default to the session's batch size so users can // tune via SessionConfig instead of relying on a hard-coded // constant baked into this UDTF. - let rows_per_batch = self + let batch_size = self .rows_per_batch .unwrap_or_else(|| context.session_config().batch_size()) .max(1); - let schema = self.schema.clone(); - let rasters = self.rasters.clone(); - - // Build all batches eagerly into a Vec, then turn into a stream. - // The StructArray is already materialised in the provider, slicing - // is O(1), and the only allocation per batch is the RecordBatch - // wrapper. Lazy streaming becomes interesting once the loader - // itself is lazy. - let mut batches = Vec::with_capacity(total.div_ceil(rows_per_batch).max(1)); - let mut offset = 0; - while offset < total { - let len = (total - offset).min(rows_per_batch); - let slice = rasters.slice(offset, len); - let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(slice)])?; - batches.push(Ok(batch)); - offset += len; - } - let stream = futures::stream::iter(batches); - let adapter = RecordBatchStreamAdapter::new(schema, stream); + let reader = ZarrChunkReader::try_new(&self.uri, self.arrays_filter.as_deref(), batch_size) + .map_err(arrow_to_df_err)?; + + // The reader is a sync Iterator>. + // Wrap it in futures::stream::iter to produce a SendableRecordBatchStream. + // Each next() walks `batch_size` chunk positions of the chunk grid; + // there's no I/O until/unless the OutDb resolver materialises bytes. + let stream = futures::stream::iter( + reader.map(|r| r.map_err(|e| DataFusionError::External(Box::new(e)))), + ); + let adapter = RecordBatchStreamAdapter::new(self.schema.clone(), stream); Ok(Box::pin(adapter)) } } @@ -364,10 +383,10 @@ fn parse_options(options_json: Option<&str>) -> Result { #[cfg(test)] mod tests { use super::*; + use arrow_array::{RecordBatch, StructArray}; use datafusion::prelude::SessionContext; use sedona_raster::array::RasterStructArray; use sedona_raster::traits::RasterRef; - use std::sync::Arc; use tempfile::TempDir; use zarrs::array::data_type; use zarrs::array::ArrayBuilder; @@ -400,7 +419,7 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); let df = ctx .sql(&format!("SELECT raster FROM sd_read_zarr('{uri}')")) @@ -411,9 +430,6 @@ mod tests { let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); assert_eq!(total_rows, 2, "expected 2 chunk rows"); - // Pull the raster column out, hand to RasterStructArray, verify - // chunk 0's metadata + anchor. Pixel-byte reading lives behind the - // future RS_EnsureLoaded resolver — `data` is empty in OutDb mode. let raster_col = batches[0].column(0); let struct_arr = raster_col .as_any() @@ -435,7 +451,7 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); let err = ctx .sql(&format!( @@ -454,7 +470,7 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); // 2 chunk rows, rows_per_batch=1 → 2 single-row batches. let df = ctx @@ -463,7 +479,7 @@ mod tests { )) .await .unwrap(); - let batches = df.collect().await.unwrap(); + let batches: Vec = df.collect().await.unwrap(); assert_eq!(batches.len(), 2); assert!(batches.iter().all(|b| b.num_rows() == 1)); } @@ -474,14 +490,12 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); - // OutDb mode: byte fetching is deferred. SELECT count(*) just - // walks the chunk grid metadata. + // SELECT count(*) just walks the chunk grid metadata; never opens + // a chunk file. let df = ctx - .sql(&format!( - r#"SELECT count(*) FROM sd_read_zarr('{uri}', '{{"load_eager": false}}')"#, - )) + .sql(&format!("SELECT count(*) FROM sd_read_zarr('{uri}')")) .await .unwrap(); let batches = df.collect().await.unwrap(); @@ -494,12 +508,12 @@ mod tests { } #[tokio::test] - async fn udtf_rejects_multi_partition_in_phase1() { + async fn udtf_rejects_multi_partition() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); let err = ctx .sql(&format!( @@ -540,7 +554,7 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); // Default behaviour: 1-D coord variable auto-skipped, read succeeds. let df = ctx @@ -588,7 +602,7 @@ mod tests { #[tokio::test] async fn udtf_rejects_malformed_options_json() { let ctx = SessionContext::new(); - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + register(&ctx); let err = ctx .sql(r#"SELECT raster FROM sd_read_zarr('file:///nowhere', '{not json}')"#) diff --git a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs index 26be44189..7da73b753 100644 --- a/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs +++ b/rust/sedona-raster-zarr/tests/zarr_roundtrip.rs @@ -16,7 +16,7 @@ // under the License. //! End-to-end fixture test: build a small Zarr group on disk with the -//! `zarrs` crate, then read it back through `group_to_rasters` and +//! `zarrs` crate, then read it back through `read_all` and //! verify the resulting raster `StructArray`. The loader always emits //! OutDb-style rows (empty `data`, populated `outdb_uri`), so these //! tests assert on metadata and chunk-anchor URIs rather than pixel @@ -25,9 +25,29 @@ use std::sync::Arc; +use arrow_array::cast::AsArray; +use arrow_array::StructArray; +use arrow_schema::ArrowError; use sedona_raster::array::RasterStructArray; use sedona_raster::traits::RasterRef; -use sedona_raster_zarr::group_to_rasters; +use sedona_raster_zarr::ZarrChunkReader; + +/// Drain a `ZarrChunkReader` into a single `StructArray`. Fixtures in +/// this file are small (≤8 chunk rows) so they fit in one batch with a +/// generous batch_size. Anything bigger would need `arrow::compute::concat`. +fn read_all(uri: &str, arrays: Option<&[String]>) -> Result { + let reader = ZarrChunkReader::try_new(uri, arrays, 1024)?; + let batches: Vec<_> = reader.collect::, _>>()?; + assert!( + batches.len() <= 1, + "test helper assumes ≤1 batch; got {} — bump batch_size or add concat", + batches.len() + ); + let batch = batches.into_iter().next().ok_or_else(|| { + ArrowError::InvalidArgumentError("ZarrChunkReader produced zero batches".into()) + })?; + Ok(batch.column(0).as_struct().clone()) +} use sedona_schema::raster::BandDataType; use tempfile::TempDir; use zarrs::array::data_type; @@ -92,7 +112,7 @@ fn build_fixture() -> TempDir { fn round_trip_emits_one_row_per_chunk_position_with_outdb_anchors() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_rasters(&uri, None).unwrap(); + let arr = read_all(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 8, "expected 8 chunk rows (2*2*2)"); @@ -148,7 +168,7 @@ fn errors_on_empty_group() { .store_metadata() .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_rasters(&uri, None).unwrap_err().to_string(); + let err = read_all(&uri, None).unwrap_err().to_string(); assert!(err.contains("no child arrays"), "got: {err}"); } @@ -196,7 +216,7 @@ fn build_xarray_style_fixture() -> TempDir { fn auto_skips_1d_coord_variables() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_rasters(&uri, None).unwrap(); + let arr = read_all(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); // 2*2*2 = 8 chunk positions, with 2 bands per row (pressure, temperature). assert_eq!(rasters.len(), 8); @@ -209,7 +229,7 @@ fn explicit_arrays_filter_selects_subset() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); let filter = vec!["temperature".to_string()]; - let arr = group_to_rasters(&uri, Some(&filter)).unwrap(); + let arr = read_all(&uri, Some(&filter)).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 8); let r0 = rasters.get(0).unwrap(); @@ -221,9 +241,7 @@ fn explicit_arrays_filter_rejects_unknown_name() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); let filter = vec!["humidity".to_string()]; - let err = group_to_rasters(&uri, Some(&filter)) - .unwrap_err() - .to_string(); + let err = read_all(&uri, Some(&filter)).unwrap_err().to_string(); assert!(err.contains("humidity"), "got: {err}"); assert!(err.contains("no array named"), "got: {err}"); } @@ -252,7 +270,7 @@ fn errors_when_crs_declared_without_transform() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_rasters(&uri, None).unwrap_err().to_string(); + let err = read_all(&uri, None).unwrap_err().to_string(); assert!(err.contains("CRS"), "got: {err}"); assert!(err.contains("spatial:transform"), "got: {err}"); } @@ -265,9 +283,7 @@ fn explicit_arrays_filter_rejects_1d_arrays() { let tmp = build_xarray_style_fixture(); let uri = format!("file://{}", tmp.path().display()); let filter = vec!["t".to_string()]; - let err = group_to_rasters(&uri, Some(&filter)) - .unwrap_err() - .to_string(); + let err = read_all(&uri, Some(&filter)).unwrap_err().to_string(); assert!(err.contains("\"t\""), "got: {err}"); assert!(err.contains("rank 1"), "got: {err}"); assert!(err.contains("at least 2 dimensions"), "got: {err}"); @@ -290,7 +306,7 @@ fn errors_when_group_has_only_1d_arrays() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_rasters(&uri, None).unwrap_err().to_string(); + let err = read_all(&uri, None).unwrap_err().to_string(); assert!(err.contains("only 1-D arrays"), "got: {err}"); } @@ -318,7 +334,7 @@ fn falls_back_to_array_dimensions_attribute() { array.store_metadata().unwrap(); let uri = format!("file://{}", tmp.path().display()); - let arr = group_to_rasters(&uri, None).unwrap(); + let arr = read_all(&uri, None).unwrap(); let rasters = RasterStructArray::new(&arr); assert_eq!(rasters.len(), 2); let r0 = rasters.get(0).unwrap(); @@ -352,7 +368,7 @@ fn errors_on_mismatched_chunk_grids() { .unwrap(); let uri = format!("file://{}", tmp.path().display()); - let err = group_to_rasters(&uri, None).unwrap_err().to_string(); + let err = read_all(&uri, None).unwrap_err().to_string(); assert!( err.contains("chunk") && err.contains("array_a") && err.contains("array_b"), "got: {err}" diff --git a/rust/sedona/Cargo.toml b/rust/sedona/Cargo.toml index 12a1fe6ac..d1beb0b47 100644 --- a/rust/sedona/Cargo.toml +++ b/rust/sedona/Cargo.toml @@ -51,14 +51,6 @@ tempfile = { workspace = true } tokio = { workspace = true, features = ["rt-multi-thread"] } rstest = { workspace = true } sedona-raster = { workspace = true } -zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } -zarrs_filesystem = { workspace = true } - -# See sedona-raster-zarr/Cargo.toml — `blosc` is gated off the Windows -# MinGW (gnu) toolchain only to avoid a libwinpthreads/c-blosc symbol -# collision in the R `sedonadb.dll` link. MSVC Windows users get blosc. -[target.'cfg(not(all(target_os = "windows", target_env = "gnu")))'.dev-dependencies] -zarrs = { workspace = true, features = ["blosc"] } [dependencies] arrow-schema = { workspace = true } @@ -91,7 +83,6 @@ sedona-proj = { workspace = true } sedona-gdal = { workspace = true } sedona-raster-functions = { workspace = true } sedona-raster-gdal = { workspace = true } -sedona-raster-zarr = { workspace = true } sedona-schema = { workspace = true } sedona-spatial-join = { workspace = true, optional = true } sedona-spatial-join-gpu = { workspace = true, optional = true } diff --git a/rust/sedona/src/context.rs b/rust/sedona/src/context.rs index 25840d36b..663e391fb 100644 --- a/rust/sedona/src/context.rs +++ b/rust/sedona/src/context.rs @@ -25,7 +25,6 @@ use crate::{ catalog::DynamicObjectStoreCatalog, random_geometry_provider::RandomGeometryFunction, show::{show_batches, DisplayTableOptions}, - zarr_read::ZarrReadFunction, }; use arrow_array::RecordBatch; use async_trait::async_trait; @@ -233,8 +232,8 @@ impl SedonaContext { "sd_random_geometry", Arc::new(RandomGeometryFunction::default()), ); - out.ctx - .register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); + // Zarr support lives in the `sedonadb-zarr` plugin package; + // users opt in via its `register(con)` helper. // Always register default function set out.register_function_set(sedona_functions::register::default_function_set()); diff --git a/rust/sedona/src/lib.rs b/rust/sedona/src/lib.rs index 356ba4d9f..81c103f9f 100644 --- a/rust/sedona/src/lib.rs +++ b/rust/sedona/src/lib.rs @@ -27,4 +27,3 @@ pub mod reader; pub mod record_batch_reader_provider; pub mod show; pub mod size_parser; -pub mod zarr_read; From 8b1d1802538759309362e8a9f1e6596428bb4f55 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 15:25:30 -0700 Subject: [PATCH 09/36] feat(sedonadb-zarr): new Python plugin package wiring zarr support MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Introduces `python/sedonadb-zarr` as a separate maturin-built Python package. After `sedonadb_zarr.register(con)`, the `sd_read_zarr` SQL UDTF is attached to the session; before that call (or without importing the package at all) the main `sedonadb` package has zero zarr awareness. Package contents: - `python/sedonadb-zarr/src/lib.rs` — PyO3 binding exposing a single `register_udtf(internal_ctx)` function that calls `sedona_raster_zarr::register_udtf` on the wrapped DataFusion `SessionContext`. The plugin imports `python/sedonadb`'s PyO3 classes via the `::_lib` extern crate (sedonadb's rustc crate name per its `lib.name = "_lib"` override that maturin requires). - `python/sedonadb-zarr/python/sedonadb_zarr/__init__.py` — Python `register(con)` helper. Locates the `_impl: InternalContext` on a sedonadb `Context` and forwards to the Rust extension. - `python/sedonadb-zarr/pyproject.toml`, `Cargo.toml`, `README.md` — build scaffolding mirroring `python/sedonadb`'s layout. Required upstream change: - `python/sedonadb/Cargo.toml` gains `rlib` alongside `cdylib` in `crate-type` so plugin crates can link against it as a Rust dependency. - `python/sedonadb/src/lib.rs` makes the `context` module `pub` so plugins can extract `InternalContext` from Python objects via PyO3's downcast machinery. The other modules stay private. - Workspace root `Cargo.toml` adds `python/sedonadb-zarr` to `[workspace] members` and exposes `sedonadb` as a workspace path-dep. Tests: - `python/sedonadb-zarr/tests/test_zarr.py` covers the smoke path (register + SQL works), the negative case (SQL fails before register), and idempotence (double-register is fine). - `python/sedonadb/tests/test_no_zarr_by_default.py` is the architectural regression test: it lives in the main package and asserts `sd_read_zarr` is unknown without the plugin imported. Catches accidental re-bundling. ExternalFormatSpec is not yet exposed through the Python plugin — the Rust impl exists in `sedona-raster-zarr` and is callable from Rust code; surfacing it via `con.read_format(ZarrFormatSpec(), uri)` requires wiring `ZarrChunkReader` through the Arrow C Stream FFI and is left as a follow-up. --- Cargo.lock | 9 ++ Cargo.toml | 5 ++ python/sedonadb-zarr/Cargo.toml | 35 ++++++++ python/sedonadb-zarr/README.md | 57 ++++++++++++ python/sedonadb-zarr/pyproject.toml | 46 ++++++++++ .../python/sedonadb_zarr/__init__.py | 64 +++++++++++++ python/sedonadb-zarr/src/lib.rs | 51 +++++++++++ python/sedonadb-zarr/tests/test_zarr.py | 89 +++++++++++++++++++ python/sedonadb/Cargo.toml | 5 +- python/sedonadb/src/lib.rs | 5 +- .../sedonadb/tests/test_no_zarr_by_default.py | 43 +++++++++ 11 files changed, 407 insertions(+), 2 deletions(-) create mode 100644 python/sedonadb-zarr/Cargo.toml create mode 100644 python/sedonadb-zarr/README.md create mode 100644 python/sedonadb-zarr/pyproject.toml create mode 100644 python/sedonadb-zarr/python/sedonadb_zarr/__init__.py create mode 100644 python/sedonadb-zarr/src/lib.rs create mode 100644 python/sedonadb-zarr/tests/test_zarr.py create mode 100644 python/sedonadb/tests/test_no_zarr_by_default.py diff --git a/Cargo.lock b/Cargo.lock index 6027a1469..92a92b912 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6403,6 +6403,15 @@ dependencies = [ "tokio", ] +[[package]] +name = "sedonadb-zarr" +version = "0.4.0" +dependencies = [ + "pyo3", + "sedona-raster-zarr", + "sedonadb", +] + [[package]] name = "sedonadbr" version = "0.4.0" diff --git a/Cargo.toml b/Cargo.toml index 0717c8358..67fd08747 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -25,6 +25,7 @@ members = [ "c/sedona-s2geography", "c/sedona-tg", "python/sedonadb", + "python/sedonadb-zarr", "r/sedonadb/src/rust", "rust/sedona-adbc", "rust/sedona-datasource", @@ -162,6 +163,10 @@ sedona-spatial-join-geography = { version = "0.4.0", path = "rust/sedona-spatial sedona-query-planner = { version = "0.4.0", path = "rust/sedona-query-planner" } sedona-testing = { version = "0.4.0", path = "rust/sedona-testing" } +# Python extension crates exposed as rlib so plugin packages can extract +# PyO3 classes (e.g. `InternalContext`) via PyO3's downcast machinery. +sedonadb = { version = "0.4.0", path = "python/sedonadb", default-features = false } + # C wrapper crates sedona-extension = { version = "0.4.0", path = "c/sedona-extension" } sedona-gdal = { version = "0.4.0", path = "c/sedona-gdal", default-features = false } diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml new file mode 100644 index 000000000..a1ba19fc8 --- /dev/null +++ b/python/sedonadb-zarr/Cargo.toml @@ -0,0 +1,35 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "sedonadb-zarr" +version = { workspace = true } +publish = false +edition = "2021" + +[lib] +name = "_lib" +crate-type = ["cdylib"] + +[dependencies] +pyo3 = { version = "0.25.1" } +sedona-raster-zarr = { workspace = true } +# The sedonadb Python extension; its rustc crate name is `_lib` per +# the `[lib].name` override in `python/sedonadb/Cargo.toml`. We import +# it in `src/lib.rs` via the leading `::_lib` syntax to disambiguate +# from our own crate's `_lib` pymodule. +sedonadb = { workspace = true } diff --git a/python/sedonadb-zarr/README.md b/python/sedonadb-zarr/README.md new file mode 100644 index 000000000..9b6d78bb7 --- /dev/null +++ b/python/sedonadb-zarr/README.md @@ -0,0 +1,57 @@ + + +# sedonadb-zarr + +Zarr support for [SedonaDB](https://sedona.apache.org/) as an opt-in +plugin package. Adds the `sd_read_zarr` SQL UDTF that reads Zarr v3 +groups (with sharding, vlen-utf8 dims, etc.) as a column of N-D +rasters. + +```python +import sedonadb +import sedonadb_zarr + +con = sedonadb.connect() +sedonadb_zarr.register(con) + +df = con.sql("SELECT count(*) FROM sd_read_zarr('file:///path/to/foo.zarr')") +df.show() +``` + +The main `sedonadb` package does not bundle Zarr support — applications +that don't import `sedonadb_zarr` pay no zarr build or runtime cost. + +## Status + +- SQL UDTF (`sd_read_zarr`): supported. +- `ExternalFormatSpec` (`con.read_format(ZarrFormatSpec(), uri)`): the + Rust impl exists in `sedona-raster-zarr` and is callable from Rust + code; a Python wrapper that exposes it via `con.read_format` is a + follow-up. + +## Architecture + +This is a maturin-built mixed Rust/Python package. The Rust side is a +thin shim around `sedona-raster-zarr` that exposes a `register_udtf` +PyO3 function. The Python side calls it from `register(con)`. + +See the [design notes](https://...) (TODO) for the plugin pattern this +package follows. The same shape applies to future formats (`sedonadb-cog`, +`sedonadb-icechunk`, …). diff --git a/python/sedonadb-zarr/pyproject.toml b/python/sedonadb-zarr/pyproject.toml new file mode 100644 index 000000000..ba72144ee --- /dev/null +++ b/python/sedonadb-zarr/pyproject.toml @@ -0,0 +1,46 @@ +# 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. + +[build-system] +requires = ["maturin>=1.8,<2.0"] +build-backend = "maturin" + +[project] +name = "sedonadb-zarr" +readme = "README.md" +requires-python = ">=3.9" +classifiers = [ + "Programming Language :: Rust", + "Programming Language :: Python :: Implementation :: CPython", + "Programming Language :: Python :: Implementation :: PyPy", +] +dynamic = ["version"] +dependencies = [ + "sedonadb", +] + +[project.optional-dependencies] +test = [ + "pytest", + "zarr", + "numpy", +] + +[tool.maturin] +features = ["pyo3/extension-module"] +module-name = "sedonadb_zarr._lib" +python-source = "python" diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py new file mode 100644 index 000000000..f08734e13 --- /dev/null +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -0,0 +1,64 @@ +# 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. + +"""Zarr support for SedonaDB. + +Activate by calling :func:`register` on a SedonaDB connection. After +registration, the ``sd_read_zarr`` SQL UDTF reads Zarr groups as +N-D raster columns: + +>>> import sedonadb +>>> import sedonadb_zarr +>>> con = sedonadb.connect() +>>> sedonadb_zarr.register(con) +>>> con.sql("SELECT count(*) FROM sd_read_zarr('file:///path/to/foo.zarr')").show() # doctest: +SKIP + +The plugin is opt-in: SedonaDB itself does not bundle Zarr support, so +applications that don't import ``sedonadb_zarr`` pay no zarr build or +runtime cost. +""" + +from sedonadb_zarr._lib import register_udtf as _register_udtf + + +def register(con) -> None: + """Attach Zarr SQL support to a SedonaDB connection. + + After this call, ``con.sql("SELECT * FROM sd_read_zarr(...)")`` + works. Idempotent — calling twice on the same connection re- + registers the UDTF without error. + + Parameters + ---------- + con + A ``sedonadb`` ``Context`` (the object returned by + ``sedonadb.connect()``). Internally, this function extracts + the underlying ``InternalContext`` PyO3 handle and registers + the UDTF on its DataFusion ``SessionContext``. + """ + # The `Context` Python object wraps an `InternalContext`; the + # attribute name follows sedonadb's internal convention. + internal_ctx = getattr(con, "_impl", None) + if internal_ctx is None: + raise TypeError( + "sedonadb_zarr.register: could not locate the InternalContext on " + f"{type(con).__name__}; expected attribute `_impl`." + ) + _register_udtf(internal_ctx) + + +__all__ = ["register"] diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs new file mode 100644 index 000000000..86c3441f3 --- /dev/null +++ b/python/sedonadb-zarr/src/lib.rs @@ -0,0 +1,51 @@ +// 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. + +//! `sedonadb-zarr` — Python plugin package wiring Zarr support into a +//! `sedonadb` session. +//! +//! The Rust side here is intentionally thin: it borrows the sedonadb +//! Python `InternalContext` (an `rlib`-exposed PyO3 class) and calls +//! `sedona_raster_zarr::register_udtf` on its inner `SessionContext`. +//! Everything else (the `ZarrFormatSpec(ExternalFormatSpec)` Python +//! class, the `register(con)` helper) lives on the Python side. + +use pyo3::prelude::*; +// The `python/sedonadb` Cargo package compiles its rlib as `_lib` +// (the lib.name override that maturin needs for the wheel's +// `sedonadb._lib` import path). Our own crate also has `lib.name = +// "_lib"`, which would collide with the pymodule below — the leading +// `::` disambiguates: it refers to the extern crate, not our local +// module. +use ::_lib::context::InternalContext; + +/// Attach the `sd_read_zarr` SQL UDTF to a sedonadb session. +/// +/// Called from the Python `sedonadb_zarr.register(con)` helper. After +/// this, `con.sql("SELECT * FROM sd_read_zarr(...)")` works. +#[pyfunction] +fn register_udtf(internal_ctx: &Bound<'_, InternalContext>) -> PyResult<()> { + let ctx = internal_ctx.borrow(); + sedona_raster_zarr::register_udtf(&ctx.inner.ctx); + Ok(()) +} + +#[pymodule] +fn _lib(m: &Bound<'_, PyModule>) -> PyResult<()> { + m.add_function(wrap_pyfunction!(register_udtf, m)?)?; + Ok(()) +} diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py new file mode 100644 index 000000000..e27161fc8 --- /dev/null +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -0,0 +1,89 @@ +# 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. + +"""Tests for the `sedonadb-zarr` plugin. + +Three flavours of test together establish the plugin contract: + +1. **Smoke**: after `register(con)`, `sd_read_zarr` works as a SQL UDTF. +2. **Negative**: before `register(con)`, `sd_read_zarr` is *not* a known + SQL function. Proves opt-in: importing `sedonadb_zarr` alone doesn't + register anything. +3. **Idempotence**: calling `register(con)` twice doesn't error or + double-register pathologically. +""" + +import numpy as np +import pytest + +import sedonadb +import sedonadb_zarr + + +@pytest.fixture +def zarr_group(tmp_path): + """Build a tiny 2x2 UInt8 Zarr v3 group with two chunks.""" + zarr = pytest.importorskip("zarr") + root = zarr.open_group(str(tmp_path), mode="w") + arr = root.create_array( + "temperature", + shape=(2, 2), + chunks=(1, 2), + dtype="uint8", + dimension_names=["y", "x"], + ) + arr[:] = np.array([[10, 11], [20, 21]], dtype=np.uint8) + return tmp_path + + +def test_smoke_register_enables_sql_udtf(zarr_group): + con = sedonadb.connect() + sedonadb_zarr.register(con) + df = con.sql(f"SELECT count(*) FROM sd_read_zarr('file://{zarr_group}')") + arrow_tab = df.to_arrow_table() + assert arrow_tab.num_rows == 1 + assert arrow_tab.column(0)[0].as_py() == 2 + + +def test_sql_udtf_is_not_registered_before_register_is_called(zarr_group): + # Importing `sedonadb_zarr` (already done at module top) must NOT + # register anything globally — registration is per-context and + # explicit. A fresh connection without `register(con)` should fail + # the SQL with a planner-level "unknown function" error. + con = sedonadb.connect() + with pytest.raises(Exception, match=r"sd_read_zarr|function|table function"): + con.sql(f"SELECT * FROM sd_read_zarr('file://{zarr_group}')") + + +def test_register_is_idempotent(zarr_group): + con = sedonadb.connect() + sedonadb_zarr.register(con) + # Second registration should not error (the underlying datafusion + # `register_udtf` overwrites by name, which is fine). + sedonadb_zarr.register(con) + df = con.sql(f"SELECT count(*) FROM sd_read_zarr('file://{zarr_group}')") + assert df.to_arrow_table().num_rows == 1 + + +def test_arrays_option_threads_through_sql(zarr_group): + con = sedonadb.connect() + sedonadb_zarr.register(con) + df = con.sql( + f"SELECT count(*) FROM sd_read_zarr(" + f"'file://{zarr_group}', '{{\"arrays\":[\"temperature\"]}}')" + ) + assert df.to_arrow_table().column(0)[0].as_py() == 2 diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index 238dfe2ea..f908b675f 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -23,7 +23,10 @@ edition = "2021" [lib] name = "_lib" -crate-type = ["cdylib"] +# `cdylib` is what maturin packages into the Python wheel. `rlib` is +# what plugin crates (e.g. `python/sedonadb-zarr`) link against so they +# can extract `InternalContext` from Python objects via PyO3. +crate-type = ["cdylib", "rlib"] [features] default = ["mimalloc"] diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index 6c1437b18..d5fe1651c 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -21,7 +21,10 @@ use sedona_gdal::global::{configure_global_gdal_api, with_global_gdal, GdalApiBu use sedona_proj::register::{configure_global_proj_engine, ProjCrsEngineBuilder}; use std::ffi::c_void; -mod context; +// `context` is `pub` so plugin crates (e.g. `python/sedonadb-zarr`) +// linking the rlib can extract `InternalContext` from Python objects. +// The other modules stay private — they're internal-detail. +pub mod context; mod dataframe; mod datasource; mod error; diff --git a/python/sedonadb/tests/test_no_zarr_by_default.py b/python/sedonadb/tests/test_no_zarr_by_default.py new file mode 100644 index 000000000..0529018f0 --- /dev/null +++ b/python/sedonadb/tests/test_no_zarr_by_default.py @@ -0,0 +1,43 @@ +# 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. + +"""Architectural regression test: `sedonadb` alone has no Zarr. + +This locks in the plugin separation. A fresh sedonadb connection, +created without importing `sedonadb_zarr`, must not know about the +`sd_read_zarr` UDTF. If a future change accidentally re-bundles Zarr +into the main package — for example by adding a convenience import or +re-attaching the registration to `SedonaContext::new_from_context` — +this test fails and forces a conscious decision. + +The test is here (not in `sedonadb-zarr`) on purpose: it's the test +that catches *re-bundling*, which would happen in `sedonadb`'s code, +not in the plugin. +""" + +import pytest + +import sedonadb + + +def test_sd_read_zarr_is_not_a_known_function_without_plugin(): + con = sedonadb.connect() + # Use a non-existent path; we expect the planner to fail because + # `sd_read_zarr` itself is unknown, not because the file doesn't + # exist. The error message should mention the function name. + with pytest.raises(Exception, match=r"sd_read_zarr|function|table function"): + con.sql("SELECT * FROM sd_read_zarr('file:///nowhere/foo.zarr')") From 490591a573e944dec26e44d6fded62054d500f6a Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 15:53:44 -0700 Subject: [PATCH 10/36] feat(sedonadb-zarr): expose ExternalFormatSpec via Python ZarrFormatSpec Completes the second user-facing surface promised in the plugin design. After `sedonadb_zarr.register(con)`, both work: con.sql("SELECT * FROM sd_read_zarr(...)") con.read_format(sedonadb_zarr.ZarrFormatSpec(), uri) Implementation: - New PyO3 class `PyZarrChunkReader` in the plugin's Rust extension wraps `sedona_raster_zarr::ZarrChunkReader` and exposes `__arrow_c_stream__` via `arrow_array::ffi_stream::FFI_ArrowArrayStream`. The reader is consumed on first `__arrow_c_stream__` call (matches pyarrow's `RecordBatchReader` convention). - Python `ZarrFormatSpec(ExternalFormatSpec)` subclass in the plugin package wraps `PyZarrChunkReader`. Supports `with_options` for `load_eager` and `arrays`; `open_reader` returns the Rust-backed reader; default `infer_schema` works because the reader exposes schema via the Arrow C stream protocol. Tests cover the new surface end-to-end: read via `con.read_format`, options threading, and the `load_eager=True` rejection. --- Cargo.lock | 1 + python/sedonadb-zarr/Cargo.toml | 1 + .../python/sedonadb_zarr/__init__.py | 62 +++++++++++-- python/sedonadb-zarr/src/lib.rs | 86 ++++++++++++++++--- python/sedonadb-zarr/tests/test_zarr.py | 27 ++++++ 5 files changed, 159 insertions(+), 18 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 92a92b912..7a757c238 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6407,6 +6407,7 @@ dependencies = [ name = "sedonadb-zarr" version = "0.4.0" dependencies = [ + "arrow-array", "pyo3", "sedona-raster-zarr", "sedonadb", diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index a1ba19fc8..a26ae8b86 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -26,6 +26,7 @@ name = "_lib" crate-type = ["cdylib"] [dependencies] +arrow-array = { workspace = true } pyo3 = { version = "0.25.1" } sedona-raster-zarr = { workspace = true } # The sedonadb Python extension; its rustc crate name is `_lib` per diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index f08734e13..50153ebbd 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -18,8 +18,11 @@ """Zarr support for SedonaDB. Activate by calling :func:`register` on a SedonaDB connection. After -registration, the ``sd_read_zarr`` SQL UDTF reads Zarr groups as -N-D raster columns: +registration, two surfaces work: + +1. ``con.sql("SELECT * FROM sd_read_zarr('s3://...')")`` — SQL UDTF. +2. ``con.read_format(ZarrFormatSpec(), uri)`` — DataFrame API via + ``ExternalFormatSpec``. >>> import sedonadb >>> import sedonadb_zarr @@ -32,6 +35,12 @@ runtime cost. """ +import json +from typing import Any, Mapping, Optional + +from sedonadb.datasource import ExternalFormatSpec + +from sedonadb_zarr._lib import PyZarrChunkReader from sedonadb_zarr._lib import register_udtf as _register_udtf @@ -50,8 +59,6 @@ def register(con) -> None: the underlying ``InternalContext`` PyO3 handle and registers the UDTF on its DataFusion ``SessionContext``. """ - # The `Context` Python object wraps an `InternalContext`; the - # attribute name follows sedonadb's internal convention. internal_ctx = getattr(con, "_impl", None) if internal_ctx is None: raise TypeError( @@ -61,4 +68,49 @@ def register(con) -> None: _register_udtf(internal_ctx) -__all__ = ["register"] +class ZarrFormatSpec(ExternalFormatSpec): + """`ExternalFormatSpec` for Zarr groups. + + Use with ``con.read_format(spec, uri)``: + + >>> con.read_format(ZarrFormatSpec(), "file:///path/to/foo.zarr") # doctest: +SKIP + + Supported options (via :meth:`with_options`): + + - ``load_eager`` (``bool``) — ``False`` (default) emits chunk-anchor + URIs only; ``True`` currently errors pending the async resolver. + - ``arrays`` (``list[str]`` or JSON-string) — explicit subset of + group arrays to read. + """ + + def __init__(self, options: Optional[Mapping[str, Any]] = None): + self._options: dict = dict(options) if options else {} + + @property + def extension(self) -> str: + return ".zarr" + + def with_options(self, options: Mapping[str, Any]) -> "ZarrFormatSpec": + merged = {**self._options, **options} + return ZarrFormatSpec(merged) + + def open_reader(self, args: Any) -> PyZarrChunkReader: + uri = args.src.to_url() + if uri is None: + raise ValueError( + "ZarrFormatSpec: could not resolve a URL from the source object" + ) + if self._options.get("load_eager"): + raise NotImplementedError( + "ZarrFormatSpec: load_eager=True is not yet supported. " + "Pixel-byte materialisation lands with the async RS_EnsureLoaded " + "resolver." + ) + arrays = self._options.get("arrays") + if isinstance(arrays, str): + arrays = json.loads(arrays) + batch_size = args.batch_size if args.batch_size is not None else 8192 + return PyZarrChunkReader(uri, arrays, batch_size) + + +__all__ = ["register", "ZarrFormatSpec", "PyZarrChunkReader"] diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index 86c3441f3..fdc41cbb8 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -18,25 +18,36 @@ //! `sedonadb-zarr` — Python plugin package wiring Zarr support into a //! `sedonadb` session. //! -//! The Rust side here is intentionally thin: it borrows the sedonadb -//! Python `InternalContext` (an `rlib`-exposed PyO3 class) and calls -//! `sedona_raster_zarr::register_udtf` on its inner `SessionContext`. -//! Everything else (the `ZarrFormatSpec(ExternalFormatSpec)` Python -//! class, the `register(con)` helper) lives on the Python side. +//! Two PyO3-exposed surfaces: +//! - `register_udtf(internal_ctx)` — attaches the `sd_read_zarr` SQL +//! UDTF to a sedonadb session. +//! - `PyZarrChunkReader` — a streaming reader producible from Python, +//! exposing `__arrow_c_stream__` so it plugs into +//! `ExternalFormatSpec.open_reader` (the `con.read_format(spec, uri)` +//! surface). +//! +//! The Python `ZarrFormatSpec(ExternalFormatSpec)` class in +//! `sedonadb_zarr/__init__.py` wraps `PyZarrChunkReader`; the Rust side +//! here is intentionally thin. + +use std::ffi::CString; +use std::sync::Mutex; +use arrow_array::ffi_stream::FFI_ArrowArrayStream; +use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::prelude::*; -// The `python/sedonadb` Cargo package compiles its rlib as `_lib` -// (the lib.name override that maturin needs for the wheel's -// `sedonadb._lib` import path). Our own crate also has `lib.name = -// "_lib"`, which would collide with the pymodule below — the leading -// `::` disambiguates: it refers to the extern crate, not our local -// module. +use pyo3::types::PyCapsule; +// `python/sedonadb` compiles its rlib as `_lib` (the lib.name override +// maturin needs). Our own crate also has `lib.name = "_lib"`, which +// would collide with the pymodule below — the leading `::` resolves to +// the extern crate, not our local module. use ::_lib::context::InternalContext; +use sedona_raster_zarr::ZarrChunkReader; /// Attach the `sd_read_zarr` SQL UDTF to a sedonadb session. /// -/// Called from the Python `sedonadb_zarr.register(con)` helper. After -/// this, `con.sql("SELECT * FROM sd_read_zarr(...)")` works. +/// Called from `sedonadb_zarr.register(con)`. After this, +/// `con.sql("SELECT * FROM sd_read_zarr(...)")` works. #[pyfunction] fn register_udtf(internal_ctx: &Bound<'_, InternalContext>) -> PyResult<()> { let ctx = internal_ctx.borrow(); @@ -44,8 +55,57 @@ fn register_udtf(internal_ctx: &Bound<'_, InternalContext>) -> PyResult<()> { Ok(()) } +/// Python-callable wrapper around `ZarrChunkReader` that exposes +/// `__arrow_c_stream__`. Consumed on first call (subsequent calls +/// error) — matching pyarrow's `RecordBatchReader` convention. +/// +/// The Python `ZarrFormatSpec.open_reader` returns one of these; the +/// `ExternalFormatSpec` framework then drives the chunk grid via the +/// Arrow C stream protocol. +#[pyclass] +pub struct PyZarrChunkReader { + inner: Mutex>, +} + +#[pymethods] +impl PyZarrChunkReader { + #[new] + #[pyo3(signature = (uri, arrays=None, batch_size=8192))] + fn new(uri: &str, arrays: Option>, batch_size: usize) -> PyResult { + let reader = ZarrChunkReader::try_new(uri, arrays.as_deref(), batch_size) + .map_err(|e| PyValueError::new_err(e.to_string()))?; + Ok(Self { + inner: Mutex::new(Some(reader)), + }) + } + + #[pyo3(signature = (requested_schema=None))] + fn __arrow_c_stream__<'py>( + &self, + py: Python<'py>, + #[allow(unused_variables)] requested_schema: Option>, + ) -> PyResult> { + let reader = self + .inner + .lock() + .map_err(|_| PyRuntimeError::new_err("PyZarrChunkReader mutex poisoned"))? + .take() + .ok_or_else(|| { + PyRuntimeError::new_err( + "PyZarrChunkReader has already been consumed; \ + a RecordBatchReader can only be exported once.", + ) + })?; + let ffi_stream = FFI_ArrowArrayStream::new(Box::new(reader)); + let capsule_name = CString::new("arrow_array_stream") + .map_err(|e| PyRuntimeError::new_err(e.to_string()))?; + PyCapsule::new(py, ffi_stream, Some(capsule_name)) + } +} + #[pymodule] fn _lib(m: &Bound<'_, PyModule>) -> PyResult<()> { m.add_function(wrap_pyfunction!(register_udtf, m)?)?; + m.add_class::()?; Ok(()) } diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index e27161fc8..68b215187 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -87,3 +87,30 @@ def test_arrays_option_threads_through_sql(zarr_group): f"'file://{zarr_group}', '{{\"arrays\":[\"temperature\"]}}')" ) assert df.to_arrow_table().column(0)[0].as_py() == 2 + + +def test_format_spec_via_read_format(zarr_group): + # The second user-facing surface: `con.read_format(spec, uri)`, + # which uses ExternalFormatSpec.open_reader -> PyZarrChunkReader's + # __arrow_c_stream__ to plumb data through. + con = sedonadb.connect() + df = con.read_format( + sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}" + ) + arrow_tab = df.to_arrow_table() + assert arrow_tab.num_rows == 2 + assert arrow_tab.column_names == ["raster"] + + +def test_format_spec_with_arrays_option(zarr_group): + con = sedonadb.connect() + spec = sedonadb_zarr.ZarrFormatSpec().with_options({"arrays": ["temperature"]}) + df = con.read_format(spec, f"file://{zarr_group}") + assert df.to_arrow_table().num_rows == 2 + + +def test_format_spec_load_eager_errors(zarr_group): + con = sedonadb.connect() + spec = sedonadb_zarr.ZarrFormatSpec().with_options({"load_eager": True}) + with pytest.raises(Exception, match=r"load_eager"): + con.read_format(spec, f"file://{zarr_group}").to_arrow_table() From a637399467de6f4a80302408ba38e142b5502772 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 16:02:38 -0700 Subject: [PATCH 11/36] test(sedonadb-zarr): inspect raster cell as Python dict via as_py() MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Per Dewey's review on test_funcs.py:58 — strengthen the format-spec roundtrip test to actually materialise and inspect the raster row. `arrow_tab["raster"][0].as_py()` returns a dict with the raster struct's fields (transform, bands, etc.). Asserts on the top-level shape (transform + bands present), the band-level shape (data is empty for OutDb scans), and that the chunk anchor URI threads through to the band's outdb_uri field. We don't have a great way to inspect Raster structs end-to-end in Python yet, but this is the minimum that proves the FFI bridge delivers a coherent struct value to the user. --- python/sedonadb-zarr/tests/test_zarr.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index 68b215187..43e09e838 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -101,6 +101,22 @@ def test_format_spec_via_read_format(zarr_group): assert arrow_tab.num_rows == 2 assert arrow_tab.column_names == ["raster"] + # Inspect the raster cell as a Python dict — every row should carry + # transform + bands + the OutDb anchor URI for each band. + raster = arrow_tab["raster"][0].as_py() + assert isinstance(raster, dict), f"raster row is {type(raster).__name__}" + for field in ("transform", "bands"): + assert field in raster, f"raster row missing {field!r}: {sorted(raster)}" + # Bands list shape: one entry per array in the group (here, one). + assert isinstance(raster["bands"], list) and len(raster["bands"]) >= 1 + band = raster["bands"][0] + # `data` is empty (OutDb scan); `outdb_uri` points at this chunk. + assert band.get("data") in (None, b"", bytes()), ( + f"OutDb band should have empty data; got {band.get('data')!r}" + ) + anchor = band.get("outdb_uri") + assert anchor and "#array=temperature" in anchor, f"unexpected anchor: {anchor!r}" + def test_format_spec_with_arrays_option(zarr_group): con = sedonadb.connect() From 06e479b80a761a083739e317c261fadaf5be6129 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 16:21:39 -0700 Subject: [PATCH 12/36] ci: fix three CI failures from the plugin refactor - pre-commit (ruff format): test_zarr.py had a multi-line read_format call that ruff wanted as one line. - docs (`cargo doc --workspace`): both `sedonadb` and `sedonadb-zarr` originally declared `[lib] name = "_lib"` (maturin requires that for the Python `._lib` import path). rustdoc errors with "document output filename collision" because both produce a `_lib` rustdoc output. Rename the plugin's rustc crate to `sedonadb_zarr_lib`; maturin still renames the compiled cdylib to `_lib.` for the wheel via `module-name`, so the Python import path is unaffected. - python ubuntu-latest: pytest collects from `python/` which now includes `python/sedonadb-zarr/tests/`. The plugin wasn't installed in the test env, so its `import sedonadb_zarr` errored at collection time. Install the plugin alongside sedonadb in the Install step. --- .github/workflows/python.yml | 4 ++++ python/sedonadb-zarr/Cargo.toml | 7 ++++++- python/sedonadb-zarr/tests/test_zarr.py | 4 +--- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/.github/workflows/python.yml b/.github/workflows/python.yml index 516ae771d..fdb0bb58c 100644 --- a/.github/workflows/python.yml +++ b/.github/workflows/python.yml @@ -119,6 +119,10 @@ jobs: # Keep this export in sync with the export in dev/release/verify-release-candidate.sh export MATURIN_PEP517_ARGS="--features s2geography" pip install -e "python/sedonadb/[test]" -vv + # The zarr plugin package — installed so its tests in + # python/sedonadb-zarr/tests/test_zarr.py can be collected by + # the `cd python && pytest` step below. + pip install -e "python/sedonadb-zarr/[test]" -vv - name: Download minimal geoarrow-data assets run: | diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index a26ae8b86..a7c69cd48 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -22,7 +22,12 @@ publish = false edition = "2021" [lib] -name = "_lib" +# rustc crate name. Distinct from `sedonadb`'s `_lib` to avoid the +# rustdoc output-path collision on `cargo doc --workspace`. Maturin +# renames the compiled cdylib to match `module-name` in +# `pyproject.toml` (`sedonadb_zarr._lib`), so the Python import path +# is unaffected by this name. +name = "sedonadb_zarr_lib" crate-type = ["cdylib"] [dependencies] diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index 43e09e838..ce2e9f9f4 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -94,9 +94,7 @@ def test_format_spec_via_read_format(zarr_group): # which uses ExternalFormatSpec.open_reader -> PyZarrChunkReader's # __arrow_c_stream__ to plumb data through. con = sedonadb.connect() - df = con.read_format( - sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}" - ) + df = con.read_format(sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}") arrow_tab = df.to_arrow_table() assert arrow_tab.num_rows == 2 assert arrow_tab.column_names == ["raster"] From b681acc80f69ee4e27ffa3c4dd0d74cb8e4800b8 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 16:43:09 -0700 Subject: [PATCH 13/36] fix: gate sedonadb's pymodule behind `extension-module` feature The plugin refactor exposed `python/sedonadb` as an rlib so plugin crates (e.g. `sedonadb-zarr`) could link it and extract `InternalContext`. That worked on macOS but broke Linux CI with: rust-lld: error: duplicate symbol: PyInit__lib Both `sedonadb` and `sedonadb-zarr` declare `#[pymodule] fn _lib` (each needs the symbol so its respective Python wheel imports as `._lib`). When the plugin's cdylib statically links sedonadb's rlib, both `PyInit__lib` symbols collide. macOS's two-level namespace tolerated it; GNU ld doesn't. Fix: put sedonadb's `#[pymodule] fn _lib` behind a new `extension-module` Cargo feature, default-on. Maturin's wheel build inherits the default and gets the symbol; plugin Rust deps set `default-features = false` (already done at the workspace path-dep level) and the pymodule's symbol vanishes from the rlib they link. Adds a blanket `#![cfg_attr(not(feature = "extension-module"), allow(dead_code, unused_imports))]` so the helper `#[pyfunction]`s that are only reachable via the pymodule don't generate warnings in plugin-rlib builds. --- python/sedonadb/Cargo.toml | 8 +++++++- python/sedonadb/src/lib.rs | 17 +++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index f908b675f..a0802328f 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -29,7 +29,13 @@ name = "_lib" crate-type = ["cdylib", "rlib"] [features] -default = ["mimalloc"] +# `extension-module` gates the `#[pymodule] fn _lib` symbol. Default-on +# for the wheel build (maturin uses defaults); plugin crates that link +# this rlib (e.g. `sedonadb-zarr`) disable it via +# `default-features = false` to avoid duplicating the `PyInit__lib` +# symbol in their own cdylib. +default = ["extension-module", "mimalloc"] +extension-module = [] mimalloc = ["dep:mimalloc", "dep:libmimalloc-sys"] s2geography = ["sedona/s2geography"] gpu = ["sedona/gpu"] diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index d5fe1651c..264ae1fcf 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -14,6 +14,17 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. + +// When the `extension-module` feature is off (e.g. when plugin crates +// link this as an rlib for `InternalContext`), the pymodule and its +// helper `#[pyfunction]`s are gated out, leaving the imports and +// helpers "unused" from the linker's perspective. They're still +// reachable as a public Rust API surface. +#![cfg_attr( + not(feature = "extension-module"), + allow(dead_code, unused_imports) +)] + use crate::{error::PySedonaError, udf::sedona_scalar_udf}; use pyo3::{ffi::Py_uintptr_t, prelude::*}; use sedona_adbc::AdbcSedonadbDriverInit; @@ -115,6 +126,12 @@ fn gdal_version() -> Result, PySedonaError> { } } +// Gated behind `extension-module` so plugin crates (e.g. +// `sedonadb-zarr`) can take `sedonadb` as an rlib dep without +// duplicating the `PyInit__lib` symbol in their own cdylib. Plugins +// set `default-features = false`; maturin's wheel build keeps the +// default which enables the feature. +#[cfg(feature = "extension-module")] #[pymodule] fn _lib(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> { #[cfg(feature = "mimalloc")] From 940cedf1d28aa72c8157de6bb87df71c9f15eafa Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 16:50:00 -0700 Subject: [PATCH 14/36] docs(sedonadb-zarr): README accurately reflects the shipped surface MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The Status section claimed ExternalFormatSpec was a Rust-only follow-up, but `con.read_format(ZarrFormatSpec(), uri)` was actually shipped in the same PR (PyZarrChunkReader + Python wrapper + tests). Tighten the README — show both surfaces, drop the stale "follow-up" claim, and unwrap paragraphs per GFM's hard-line-break rendering. --- python/sedonadb-zarr/README.md | 31 +++++++++---------------------- 1 file changed, 9 insertions(+), 22 deletions(-) diff --git a/python/sedonadb-zarr/README.md b/python/sedonadb-zarr/README.md index 9b6d78bb7..9c8251c7b 100644 --- a/python/sedonadb-zarr/README.md +++ b/python/sedonadb-zarr/README.md @@ -19,10 +19,7 @@ # sedonadb-zarr -Zarr support for [SedonaDB](https://sedona.apache.org/) as an opt-in -plugin package. Adds the `sd_read_zarr` SQL UDTF that reads Zarr v3 -groups (with sharding, vlen-utf8 dims, etc.) as a column of N-D -rasters. +Zarr support for [SedonaDB](https://sedona.apache.org/) as an opt-in plugin package. Reads Zarr v3 groups (with sharding, vlen-utf8 dims, etc.) as a column of N-D rasters via two equivalent surfaces: ```python import sedonadb @@ -31,27 +28,17 @@ import sedonadb_zarr con = sedonadb.connect() sedonadb_zarr.register(con) -df = con.sql("SELECT count(*) FROM sd_read_zarr('file:///path/to/foo.zarr')") -df.show() -``` - -The main `sedonadb` package does not bundle Zarr support — applications -that don't import `sedonadb_zarr` pay no zarr build or runtime cost. +# SQL UDTF: +con.sql("SELECT count(*) FROM sd_read_zarr('file:///path/to/foo.zarr')").show() -## Status +# DataFrame API via ExternalFormatSpec: +con.read_format(sedonadb_zarr.ZarrFormatSpec(), 'file:///path/to/foo.zarr').show() +``` -- SQL UDTF (`sd_read_zarr`): supported. -- `ExternalFormatSpec` (`con.read_format(ZarrFormatSpec(), uri)`): the - Rust impl exists in `sedona-raster-zarr` and is callable from Rust - code; a Python wrapper that exposes it via `con.read_format` is a - follow-up. +The main `sedonadb` package does not bundle Zarr support — applications that don't import `sedonadb_zarr` pay no zarr build or runtime cost. ## Architecture -This is a maturin-built mixed Rust/Python package. The Rust side is a -thin shim around `sedona-raster-zarr` that exposes a `register_udtf` -PyO3 function. The Python side calls it from `register(con)`. +This is a maturin-built mixed Rust/Python package. The Rust side is a thin shim around `sedona-raster-zarr` that exposes a `register_udtf` PyO3 function and a `PyZarrChunkReader` class implementing `__arrow_c_stream__`. The Python side defines `ZarrFormatSpec(ExternalFormatSpec)` and a `register(con)` helper that wires the UDTF onto a session. -See the [design notes](https://...) (TODO) for the plugin pattern this -package follows. The same shape applies to future formats (`sedonadb-cog`, -`sedonadb-icechunk`, …). +The same plugin shape applies to future formats (`sedonadb-cog`, `sedonadb-icechunk`, …). From a0c7454bb97d97eeb7c349d0f07e64288109fb55 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 16:57:06 -0700 Subject: [PATCH 15/36] refactor(sedona-raster-zarr): drop the `zarr` feature gate MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The feature was a defensive response to Dewey's "at the very very least feature-flag it" comment, but that was the fallback assuming we *didn't* move the UDTF out of the `sedona` crate. We did — the UDTF and ExternalFormatSpec now live in `sedona-raster-zarr` itself — so the load-bearing concern is addressed. The only consumer of this crate is `sedonadb-zarr`, which uses the full surface. No hypothetical "slim build" wants just the streaming reader without DataFusion. The feature gate added cfg-clutter without real value. Drop the gate, drop the `[features]` table, make the previously- optional deps required. `lib.rs` declares the modules unconditionally. --- rust/sedona-raster-zarr/Cargo.toml | 31 +++++++----------------------- rust/sedona-raster-zarr/src/lib.rs | 10 ++-------- 2 files changed, 9 insertions(+), 32 deletions(-) diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml index 36895151b..fc70f2554 100644 --- a/rust/sedona-raster-zarr/Cargo.toml +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -30,38 +30,21 @@ rust-version.workspace = true [lints.clippy] result_large_err = "allow" -[features] -# The `zarr` feature gates the SedonaDB integration (ExternalFormatSpec -# impl, sd_read_zarr UDTF). Without it, the crate exposes only the -# streaming `ZarrChunkReader` primitive. Default-on so most users get -# the full integration; downstreams that only want the raw reader can -# turn it off via `default-features = false`. -default = ["zarr"] -zarr = [ - "dep:async-trait", - "dep:datafusion", - "dep:datafusion-catalog", - "dep:datafusion-expr", - "dep:futures", - "dep:sedona-datasource", - "dep:serde", -] - [dependencies] arrow-array = { workspace = true } arrow-schema = { workspace = true } -async-trait = { workspace = true, optional = true } -datafusion = { workspace = true, optional = true, features = ["sql"] } -datafusion-catalog = { workspace = true, optional = true } +async-trait = { workspace = true } +datafusion = { workspace = true, features = ["sql"] } +datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } -datafusion-expr = { workspace = true, optional = true } -futures = { workspace = true, optional = true } +datafusion-expr = { workspace = true } +futures = { workspace = true } log = { workspace = true } sedona-common = { workspace = true } -sedona-datasource = { workspace = true, optional = true } +sedona-datasource = { workspace = true } sedona-raster = { workspace = true } sedona-schema = { workspace = true } -serde = { workspace = true, optional = true } +serde = { workspace = true } serde_json = { workspace = true } zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } zarrs_filesystem = { workspace = true } diff --git a/rust/sedona-raster-zarr/src/lib.rs b/rust/sedona-raster-zarr/src/lib.rs index f57c731d0..4b201c85d 100644 --- a/rust/sedona-raster-zarr/src/lib.rs +++ b/rust/sedona-raster-zarr/src/lib.rs @@ -34,18 +34,12 @@ //! Local filesystem stores only — `file://` URIs or bare paths. pub mod dtype; +pub mod format_spec; pub mod geozarr; pub mod loader; pub mod source_uri; - -#[cfg(feature = "zarr")] -pub mod format_spec; -#[cfg(feature = "zarr")] pub mod udtf; -pub use loader::ZarrChunkReader; - -#[cfg(feature = "zarr")] pub use format_spec::ZarrFormatSpec; -#[cfg(feature = "zarr")] +pub use loader::ZarrChunkReader; pub use udtf::{register as register_udtf, ZarrReadFunction}; From ce0bdea64ac83989c0dbd8b278fb83692e2a1039 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 17:02:16 -0700 Subject: [PATCH 16/36] chore(sedona): drop redundant comment about zarr plugin --- rust/sedona/src/context.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/rust/sedona/src/context.rs b/rust/sedona/src/context.rs index 663e391fb..156b5b355 100644 --- a/rust/sedona/src/context.rs +++ b/rust/sedona/src/context.rs @@ -232,8 +232,6 @@ impl SedonaContext { "sd_random_geometry", Arc::new(RandomGeometryFunction::default()), ); - // Zarr support lives in the `sedonadb-zarr` plugin package; - // users opt in via its `register(con)` helper. // Always register default function set out.register_function_set(sedona_functions::register::default_function_set()); From 442312863492368cbcd8271d83142773f07f8d5d Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 17:07:31 -0700 Subject: [PATCH 17/36] style(sedonadb): collapse cfg_attr to one line per rustfmt --- python/sedonadb/src/lib.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index 264ae1fcf..d642c1f81 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -20,10 +20,7 @@ // helper `#[pyfunction]`s are gated out, leaving the imports and // helpers "unused" from the linker's perspective. They're still // reachable as a public Rust API surface. -#![cfg_attr( - not(feature = "extension-module"), - allow(dead_code, unused_imports) -)] +#![cfg_attr(not(feature = "extension-module"), allow(dead_code, unused_imports))] use crate::{error::PySedonaError, udf::sedona_scalar_udf}; use pyo3::{ffi::Py_uintptr_t, prelude::*}; From 4462b94af825577e775be7a6131f018626c04fe7 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 17:20:36 -0700 Subject: [PATCH 18/36] ci: don't leak sedonadb's s2geography feature into the plugin build MATURIN_PEP517_ARGS="--features s2geography" set for sedonadb's install was still in scope for the subsequent pip install of sedonadb-zarr, which doesn't have that feature. Unset before the plugin install. --- .github/workflows/python.yml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/python.yml b/.github/workflows/python.yml index fdb0bb58c..17467a1ab 100644 --- a/.github/workflows/python.yml +++ b/.github/workflows/python.yml @@ -121,7 +121,10 @@ jobs: pip install -e "python/sedonadb/[test]" -vv # The zarr plugin package — installed so its tests in # python/sedonadb-zarr/tests/test_zarr.py can be collected by - # the `cd python && pytest` step below. + # the `cd python && pytest` step below. `s2geography` is a + # sedonadb-only feature, so unset MATURIN_PEP517_ARGS to keep + # it from leaking into the plugin's maturin build. + unset MATURIN_PEP517_ARGS pip install -e "python/sedonadb-zarr/[test]" -vv - name: Download minimal geoarrow-data assets From 21c6383e7f4914222446e829f099c4cb3006196f Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 17:39:49 -0700 Subject: [PATCH 19/36] fix: move sedonadb's extension-module out of default features MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The previous fix gated the pymodule behind `extension-module` so plugin rlib consumers wouldn't duplicate `PyInit__lib`. The feature was default-on; Cargo's workspace-wide feature unification meant a workspace build (e.g. `cargo test --workspace`, `cargo build --workspace`) forced the feature on for everyone — including the plugin's rlib link of sedonadb — and the duplicate-symbol collision came back. Move `extension-module` out of `default`. Maturin enables it explicitly via `[tool.maturin] features` in `pyproject.toml` for the wheel build. Workspace cargo runs don't request it; sedonadb's rlib in those builds has no pymodule symbol; plugin crates link clean. --- python/sedonadb/Cargo.toml | 14 ++++++++------ python/sedonadb/pyproject.toml | 6 +++++- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index a0802328f..8ad8e9768 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -29,12 +29,14 @@ name = "_lib" crate-type = ["cdylib", "rlib"] [features] -# `extension-module` gates the `#[pymodule] fn _lib` symbol. Default-on -# for the wheel build (maturin uses defaults); plugin crates that link -# this rlib (e.g. `sedonadb-zarr`) disable it via -# `default-features = false` to avoid duplicating the `PyInit__lib` -# symbol in their own cdylib. -default = ["extension-module", "mimalloc"] +# `extension-module` gates the `#[pymodule] fn _lib` symbol so plugin +# crates (e.g. `sedonadb-zarr`) can link this as an rlib without +# duplicating `PyInit__lib` in their own cdylib. NOT a default +# feature: Cargo's workspace-wide feature unification means a default +# would force it on for everyone, including the plugin link. Maturin +# enables it explicitly via `[tool.maturin] features` in +# `pyproject.toml` for the wheel build. +default = ["mimalloc"] extension-module = [] mimalloc = ["dep:mimalloc", "dep:libmimalloc-sys"] s2geography = ["sedona/s2geography"] diff --git a/python/sedonadb/pyproject.toml b/python/sedonadb/pyproject.toml index 4705f6427..c093cd4fd 100644 --- a/python/sedonadb/pyproject.toml +++ b/python/sedonadb/pyproject.toml @@ -50,6 +50,10 @@ geopandas = [ ] [tool.maturin] -features = ["pyo3/extension-module"] +# `extension-module` is a sedonadb feature that gates the +# `#[pymodule] fn _lib` symbol. Maturin enables it here for the wheel +# build; cargo workspace test/build runs don't (so plugin crates that +# link sedonadb's rlib don't get a duplicate `PyInit__lib`). +features = ["pyo3/extension-module", "extension-module"] module-name = "sedonadb._lib" python-source = "python" From 4e97bd42a373a730cc4ac3d2aafb630a3750ad50 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 17:53:59 -0700 Subject: [PATCH 20/36] ci: pass sedonadb's `extension-module` feature in maturin builds MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit After moving `extension-module` out of sedonadb's default features (to avoid Cargo workspace-feature-unification dragging the pymodule symbol into plugin rlib links), the wheel builds need to enable it explicitly. Five workflows / scripts set MATURIN_PEP517_ARGS and need the feature added: * `.github/workflows/python.yml` — the test job's editable install * `.github/workflows/packaging.yml` — docs-and-deploy * `ci/scripts/wheels-build-{linux,macos,windows}.{sh,ps1}` — wheels Without it, the produced cdylib doesn't define `PyInit__lib` and any attempt to `import sedonadb` fails. --- .github/workflows/packaging.yml | 6 +++++- .github/workflows/python.yml | 11 +++++++---- ci/scripts/wheels-build-linux.sh | 2 +- ci/scripts/wheels-build-macos.sh | 2 +- ci/scripts/wheels-build-windows.ps1 | 2 +- 5 files changed, 15 insertions(+), 8 deletions(-) diff --git a/.github/workflows/packaging.yml b/.github/workflows/packaging.yml index 0d8ac7542..8012fd8ec 100644 --- a/.github/workflows/packaging.yml +++ b/.github/workflows/packaging.yml @@ -159,7 +159,11 @@ jobs: - name: Install dev SedonaDB Python env: - MATURIN_PEP517_ARGS: "--features s2geography,pyo3/extension-module" + # `extension-module` is sedonadb's own feature gating the + # `#[pymodule] fn _lib` symbol; required for the wheel to load + # as a Python module. Plugin crates linking the rlib leave it + # off to avoid `PyInit__lib` duplication. + MATURIN_PEP517_ARGS: "--features s2geography,pyo3/extension-module,extension-module" run: | pip install "python/sedonadb/[geopandas]" -v diff --git a/.github/workflows/python.yml b/.github/workflows/python.yml index 17467a1ab..504a1a2e7 100644 --- a/.github/workflows/python.yml +++ b/.github/workflows/python.yml @@ -117,13 +117,16 @@ jobs: - name: Install run: | # Keep this export in sync with the export in dev/release/verify-release-candidate.sh - export MATURIN_PEP517_ARGS="--features s2geography" + # `extension-module` is sedonadb's own feature gating the + # `#[pymodule] fn _lib` symbol — required for the wheel to + # load as a Python module. Plugin crates linking the rlib + # leave it off to avoid `PyInit__lib` duplication. + export MATURIN_PEP517_ARGS="--features s2geography,extension-module" pip install -e "python/sedonadb/[test]" -vv # The zarr plugin package — installed so its tests in # python/sedonadb-zarr/tests/test_zarr.py can be collected by - # the `cd python && pytest` step below. `s2geography` is a - # sedonadb-only feature, so unset MATURIN_PEP517_ARGS to keep - # it from leaking into the plugin's maturin build. + # the `cd python && pytest` step below. Unset + # MATURIN_PEP517_ARGS so sedonadb's features don't leak. unset MATURIN_PEP517_ARGS pip install -e "python/sedonadb-zarr/[test]" -vv diff --git a/ci/scripts/wheels-build-linux.sh b/ci/scripts/wheels-build-linux.sh index d7caa8b13..189552dae 100755 --- a/ci/scripts/wheels-build-linux.sh +++ b/ci/scripts/wheels-build-linux.sh @@ -48,7 +48,7 @@ BEFORE_ALL_MANYLINUX="yum install -y curl zip unzip tar clang perl" # add quite a bit of complexity but could save time if we build wheels for linux frequently. # The native and Rust builds are cached on each image such that compile work is effectively # cached between Python versions (just not between invocations of this script). -export CIBW_ENVIRONMENT_LINUX="VCPKG_ROOT=/vcpkg VCPKG_REF=$VCPKG_REF VCPKG_DEFAULT_TRIPLET=$VCPKG_DEFAULT_TRIPLET CMAKE_TOOLCHAIN_FILE=/vcpkg/scripts/buildsystems/vcpkg.cmake PKG_CONFIG_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib/pkgconfig LD_LIBRARY_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module'" +export CIBW_ENVIRONMENT_LINUX="VCPKG_ROOT=/vcpkg VCPKG_REF=$VCPKG_REF VCPKG_DEFAULT_TRIPLET=$VCPKG_DEFAULT_TRIPLET CMAKE_TOOLCHAIN_FILE=/vcpkg/scripts/buildsystems/vcpkg.cmake PKG_CONFIG_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib/pkgconfig LD_LIBRARY_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module,extension-module'" export CIBW_BEFORE_ALL="$BEFORE_ALL_MANYLINUX && git clone https://github.com/microsoft/vcpkg.git /vcpkg && bash {package}/../../ci/scripts/wheels-bootstrap-vcpkg.sh" pushd "${SEDONADB_DIR}" diff --git a/ci/scripts/wheels-build-macos.sh b/ci/scripts/wheels-build-macos.sh index 2b767c5f7..a2e5f968f 100755 --- a/ci/scripts/wheels-build-macos.sh +++ b/ci/scripts/wheels-build-macos.sh @@ -47,7 +47,7 @@ source ./wheels-bootstrap-vcpkg.sh export CIBW_REPAIR_WHEEL_COMMAND_MACOS="DYLD_LIBRARY_PATH=$VCPKG_INSTALL_NAME_DIR delocate-listdeps {wheel} && DYLD_LIBRARY_PATH=$VCPKG_INSTALL_NAME_DIR delocate-wheel --require-archs {delocate_archs} -w {dest_dir} {wheel}" # Pass on environment variables specifically for the build -export CIBW_ENVIRONMENT_MACOS="$CIBW_ENVIRONMENT_MACOS _PYTHON_HOST_PLATFORM=macosx-12.0-${SEDONADB_MACOS_ARCH} MACOSX_DEPLOYMENT_TARGET=12.0 CMAKE_TOOLCHAIN_FILE=${CMAKE_TOOLCHAIN_FILE} MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module'" +export CIBW_ENVIRONMENT_MACOS="$CIBW_ENVIRONMENT_MACOS _PYTHON_HOST_PLATFORM=macosx-12.0-${SEDONADB_MACOS_ARCH} MACOSX_DEPLOYMENT_TARGET=12.0 CMAKE_TOOLCHAIN_FILE=${CMAKE_TOOLCHAIN_FILE} MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module,extension-module'" pushd "${SEDONADB_DIR}" python -m cibuildwheel --output-dir python/$1/dist python/$1 diff --git a/ci/scripts/wheels-build-windows.ps1 b/ci/scripts/wheels-build-windows.ps1 index b84c46a70..da0ae78a5 100644 --- a/ci/scripts/wheels-build-windows.ps1 +++ b/ci/scripts/wheels-build-windows.ps1 @@ -73,7 +73,7 @@ $env:PATH += ";$scriptDirectory\windows" $env:GEOS_LIB_DIR = "$vcpkgLibDirectory" $env:GEOS_VERSION = "3.13.0" -$env:MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module' +$env:MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module,extension-module' # Some CMake configurations needs this separately from the toolchain file $env:CMAKE_PREFIX_PATH="$vcpkgInstalledDirectory" From b441a04fe7c6e1e27ca582a321d725477e3ae8ad Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 18:03:22 -0700 Subject: [PATCH 21/36] fix: drop self-referential sedonadb workspace dep, inline the path cibuildwheel's wheel build (macOS-arm64, linux-arm64) failed cargo metadata with: error: failed to select a version for the requirement `sedonadb = "^0.4.0"` The workspace dep table had `sedonadb` as a self-reference (the sedonadb crate exposed via its own path). Only sedonadb-zarr consumed it; inlining the path-dep on sedonadb-zarr's side and dropping the workspace entry removes the self-reference. cibuildwheel's isolated build env (where the workspace root may not be in scope the same way) no longer chokes. --- Cargo.toml | 4 ---- python/sedonadb-zarr/Cargo.toml | 6 ++++-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 67fd08747..e4e8babf4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -163,10 +163,6 @@ sedona-spatial-join-geography = { version = "0.4.0", path = "rust/sedona-spatial sedona-query-planner = { version = "0.4.0", path = "rust/sedona-query-planner" } sedona-testing = { version = "0.4.0", path = "rust/sedona-testing" } -# Python extension crates exposed as rlib so plugin packages can extract -# PyO3 classes (e.g. `InternalContext`) via PyO3's downcast machinery. -sedonadb = { version = "0.4.0", path = "python/sedonadb", default-features = false } - # C wrapper crates sedona-extension = { version = "0.4.0", path = "c/sedona-extension" } sedona-gdal = { version = "0.4.0", path = "c/sedona-gdal", default-features = false } diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index a7c69cd48..e30da8db3 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -37,5 +37,7 @@ sedona-raster-zarr = { workspace = true } # The sedonadb Python extension; its rustc crate name is `_lib` per # the `[lib].name` override in `python/sedonadb/Cargo.toml`. We import # it in `src/lib.rs` via the leading `::_lib` syntax to disambiguate -# from our own crate's `_lib` pymodule. -sedonadb = { workspace = true } +# from our own crate's `_lib` pymodule. Inlined here (not via +# `workspace = true`) to avoid a self-referential workspace dep that +# trips `cargo metadata` in cibuildwheel's isolated build env. +sedonadb = { path = "../sedonadb", default-features = false } From ea3a540635c3880a4fe532b2fa6b7c68e46564e8 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 18:23:07 -0700 Subject: [PATCH 22/36] fix: rename sedonadb-zarr's pymodule to `_zarr_lib` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit `cargo build --workspace --all-features` (the build CI job) ignores the workspace dep's `default-features = false` because feature unification merges feature requests across the workspace. With sedonadb's `extension-module` forced on, its rlib's `PyInit__lib` symbol gets linked into sedonadb-zarr's cdylib, which also defines `PyInit__lib` — and the linker errors with duplicate symbol. Rename sedonadb-zarr's pymodule from `_lib` to `_zarr_lib`. The generated PyInit symbol is now `PyInit__zarr_lib`, distinct from sedonadb's `PyInit__lib` regardless of feature unification. This means the Python wrapper imports from `sedonadb_zarr._zarr_lib` instead of `sedonadb_zarr._lib`; the user-facing `register(con)` / `ZarrFormatSpec` API is unchanged. --- python/sedonadb-zarr/Cargo.toml | 10 +++++----- python/sedonadb-zarr/pyproject.toml | 5 ++++- python/sedonadb-zarr/python/sedonadb_zarr/__init__.py | 4 ++-- python/sedonadb-zarr/src/lib.rs | 10 ++++++---- 4 files changed, 17 insertions(+), 12 deletions(-) diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index e30da8db3..340ed8230 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -23,11 +23,11 @@ edition = "2021" [lib] # rustc crate name. Distinct from `sedonadb`'s `_lib` to avoid the -# rustdoc output-path collision on `cargo doc --workspace`. Maturin -# renames the compiled cdylib to match `module-name` in -# `pyproject.toml` (`sedonadb_zarr._lib`), so the Python import path -# is unaffected by this name. -name = "sedonadb_zarr_lib" +# rustdoc output-path collision on `cargo doc --workspace`. The +# matching pymodule function in `src/lib.rs` is named `_zarr_lib` +# so its `PyInit__zarr_lib` symbol doesn't collide with sedonadb's +# `PyInit__lib` under cargo workspace feature unification. +name = "_zarr_lib" crate-type = ["cdylib"] [dependencies] diff --git a/python/sedonadb-zarr/pyproject.toml b/python/sedonadb-zarr/pyproject.toml index ba72144ee..28c4b3e21 100644 --- a/python/sedonadb-zarr/pyproject.toml +++ b/python/sedonadb-zarr/pyproject.toml @@ -42,5 +42,8 @@ test = [ [tool.maturin] features = ["pyo3/extension-module"] -module-name = "sedonadb_zarr._lib" +# `_zarr_lib` rather than `_lib` so the generated PyInit symbol +# doesn't collide with sedonadb's `_lib` under cargo workspace +# feature unification. The Python wrapper imports from this path. +module-name = "sedonadb_zarr._zarr_lib" python-source = "python" diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index 50153ebbd..cfc793201 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -40,8 +40,8 @@ from sedonadb.datasource import ExternalFormatSpec -from sedonadb_zarr._lib import PyZarrChunkReader -from sedonadb_zarr._lib import register_udtf as _register_udtf +from sedonadb_zarr._zarr_lib import PyZarrChunkReader +from sedonadb_zarr._zarr_lib import register_udtf as _register_udtf def register(con) -> None: diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index fdc41cbb8..a1fc903b3 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -38,9 +38,7 @@ use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::PyCapsule; // `python/sedonadb` compiles its rlib as `_lib` (the lib.name override -// maturin needs). Our own crate also has `lib.name = "_lib"`, which -// would collide with the pymodule below — the leading `::` resolves to -// the extern crate, not our local module. +// maturin needs). use ::_lib::context::InternalContext; use sedona_raster_zarr::ZarrChunkReader; @@ -103,8 +101,12 @@ impl PyZarrChunkReader { } } +// Named `_zarr_lib` (not `_lib`) so the generated `PyInit__zarr_lib` +// symbol doesn't collide with sedonadb's `PyInit__lib` when cargo's +// workspace feature unification (e.g. `cargo build --all-features`) +// brings sedonadb's pymodule into our cdylib's link. #[pymodule] -fn _lib(m: &Bound<'_, PyModule>) -> PyResult<()> { +fn _zarr_lib(m: &Bound<'_, PyModule>) -> PyResult<()> { m.add_function(wrap_pyfunction!(register_udtf, m)?)?; m.add_class::()?; Ok(()) From 61387ad527d22054ba8f4fae73059aa596d544d4 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 19:06:09 -0700 Subject: [PATCH 23/36] fix(sedonadb-zarr): cross-extension UDTF handoff via PyCapsule MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit PyO3's `#[pyclass]` extraction uses a per-cdylib type-id static, so `sedonadb_zarr` couldn't extract `sedonadb`'s `InternalContext` — the two extensions saw distinct types even though they linked against the same Rust definition. The plugin now hands its UDTF across the extension boundary in a `PyCapsule` carrying `Arc`, which sedonadb clones into its own `SessionContext`. The test helper `_read_format` bridges to `_impl.read_external_format` because `SedonaContext.read_format` doesn't exist yet; once a public helper lands the body collapses to a one-liner. --- Cargo.lock | 1 + python/sedonadb-zarr/Cargo.toml | 1 + .../python/sedonadb_zarr/__init__.py | 10 +++-- python/sedonadb-zarr/src/lib.rs | 40 ++++++++++------- python/sedonadb-zarr/tests/test_zarr.py | 24 ++++++++-- python/sedonadb/src/context.rs | 44 +++++++++++++++++++ 6 files changed, 97 insertions(+), 23 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7a757c238..0bf188118 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6408,6 +6408,7 @@ name = "sedonadb-zarr" version = "0.4.0" dependencies = [ "arrow-array", + "datafusion", "pyo3", "sedona-raster-zarr", "sedonadb", diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index 340ed8230..2f8d01ee9 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -32,6 +32,7 @@ crate-type = ["cdylib"] [dependencies] arrow-array = { workspace = true } +datafusion = { workspace = true } pyo3 = { version = "0.25.1" } sedona-raster-zarr = { workspace = true } # The sedonadb Python extension; its rustc crate name is `_lib` per diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index cfc793201..17c90f6bf 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -40,8 +40,7 @@ from sedonadb.datasource import ExternalFormatSpec -from sedonadb_zarr._zarr_lib import PyZarrChunkReader -from sedonadb_zarr._zarr_lib import register_udtf as _register_udtf +from sedonadb_zarr._zarr_lib import PyZarrChunkReader, zarr_udtf_capsule def register(con) -> None: @@ -57,7 +56,9 @@ def register(con) -> None: A ``sedonadb`` ``Context`` (the object returned by ``sedonadb.connect()``). Internally, this function extracts the underlying ``InternalContext`` PyO3 handle and registers - the UDTF on its DataFusion ``SessionContext``. + the UDTF on its DataFusion ``SessionContext`` via a + ``PyCapsule`` handoff — the only viable cross-extension + transport for the UDTF trait object. """ internal_ctx = getattr(con, "_impl", None) if internal_ctx is None: @@ -65,7 +66,8 @@ def register(con) -> None: "sedonadb_zarr.register: could not locate the InternalContext on " f"{type(con).__name__}; expected attribute `_impl`." ) - _register_udtf(internal_ctx) + capsule = zarr_udtf_capsule() + internal_ctx.register_udtf_capsule("sd_read_zarr", capsule) class ZarrFormatSpec(ExternalFormatSpec): diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index a1fc903b3..8e1258305 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -19,8 +19,10 @@ //! `sedonadb` session. //! //! Two PyO3-exposed surfaces: -//! - `register_udtf(internal_ctx)` — attaches the `sd_read_zarr` SQL -//! UDTF to a sedonadb session. +//! - `zarr_udtf_capsule()` — builds a `PyCapsule` carrying the +//! `sd_read_zarr` UDTF implementation. The Python wrapper hands it +//! to sedonadb's `InternalContext.register_udtf_capsule` to attach +//! the SQL function. //! - `PyZarrChunkReader` — a streaming reader producible from Python, //! exposing `__arrow_c_stream__` so it plugs into //! `ExternalFormatSpec.open_reader` (the `con.read_format(spec, uri)` @@ -31,26 +33,34 @@ //! here is intentionally thin. use std::ffi::CString; -use std::sync::Mutex; +use std::sync::{Arc, Mutex}; use arrow_array::ffi_stream::FFI_ArrowArrayStream; +use datafusion::catalog::TableFunctionImpl; use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::PyCapsule; -// `python/sedonadb` compiles its rlib as `_lib` (the lib.name override -// maturin needs). -use ::_lib::context::InternalContext; -use sedona_raster_zarr::ZarrChunkReader; +use sedona_raster_zarr::{ZarrChunkReader, ZarrReadFunction}; -/// Attach the `sd_read_zarr` SQL UDTF to a sedonadb session. +/// Build a PyCapsule carrying an `Arc` for the +/// Zarr UDTF, suitable for handoff to sedonadb's +/// `InternalContext.register_udtf_capsule`. /// -/// Called from `sedonadb_zarr.register(con)`. After this, -/// `con.sql("SELECT * FROM sd_read_zarr(...)")` works. +/// Cross-extension `#[pyclass]` extraction doesn't work in PyO3 (each +/// cdylib has its own type-id static), so we pass the UDTF +/// implementation across the extension boundary via an opaque capsule. +/// The capsule owns the `Arc`; the consumer clones a fresh refcount +/// before the capsule is dropped. #[pyfunction] -fn register_udtf(internal_ctx: &Bound<'_, InternalContext>) -> PyResult<()> { - let ctx = internal_ctx.borrow(); - sedona_raster_zarr::register_udtf(&ctx.inner.ctx); - Ok(()) +fn zarr_udtf_capsule(py: Python<'_>) -> PyResult> { + let udtf: Arc = Arc::new(ZarrReadFunction::default()); + let name = CString::new("sedonadb.udtf").unwrap(); + PyCapsule::new_with_destructor( + py, + udtf, + Some(name), + |_v: Arc, _ctx| {}, + ) } /// Python-callable wrapper around `ZarrChunkReader` that exposes @@ -107,7 +117,7 @@ impl PyZarrChunkReader { // brings sedonadb's pymodule into our cdylib's link. #[pymodule] fn _zarr_lib(m: &Bound<'_, PyModule>) -> PyResult<()> { - m.add_function(wrap_pyfunction!(register_udtf, m)?)?; + m.add_function(wrap_pyfunction!(zarr_udtf_capsule, m)?)?; m.add_class::()?; Ok(()) } diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index ce2e9f9f4..a111f7c3f 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -32,6 +32,22 @@ import sedonadb import sedonadb_zarr +from sedonadb.dataframe import DataFrame + + +def _read_format(con, spec, uri: str) -> DataFrame: + """Bridge ``ExternalFormatSpec`` → DataFrame. + + `sedonadb.SedonaContext` doesn't expose a public ``read_format`` + helper yet — for now plugin tests call ``_impl.read_external_format`` + directly. When the public surface lands the body of this helper + collapses to ``_read_format(con,spec, uri)``. + """ + return DataFrame( + con._impl, + con._impl.read_external_format(spec, [uri], False), + con.options, + ) @pytest.fixture @@ -90,11 +106,11 @@ def test_arrays_option_threads_through_sql(zarr_group): def test_format_spec_via_read_format(zarr_group): - # The second user-facing surface: `con.read_format(spec, uri)`, + # The second user-facing surface: `_read_format(con,spec, uri)`, # which uses ExternalFormatSpec.open_reader -> PyZarrChunkReader's # __arrow_c_stream__ to plumb data through. con = sedonadb.connect() - df = con.read_format(sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}") + df = _read_format(con, sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}") arrow_tab = df.to_arrow_table() assert arrow_tab.num_rows == 2 assert arrow_tab.column_names == ["raster"] @@ -119,7 +135,7 @@ def test_format_spec_via_read_format(zarr_group): def test_format_spec_with_arrays_option(zarr_group): con = sedonadb.connect() spec = sedonadb_zarr.ZarrFormatSpec().with_options({"arrays": ["temperature"]}) - df = con.read_format(spec, f"file://{zarr_group}") + df = _read_format(con, spec, f"file://{zarr_group}") assert df.to_arrow_table().num_rows == 2 @@ -127,4 +143,4 @@ def test_format_spec_load_eager_errors(zarr_group): con = sedonadb.connect() spec = sedonadb_zarr.ZarrFormatSpec().with_options({"load_eager": True}) with pytest.raises(Exception, match=r"load_eager"): - con.read_format(spec, f"file://{zarr_group}").to_arrow_table() + _read_format(con, spec, f"file://{zarr_group}").to_arrow_table() diff --git a/python/sedonadb/src/context.rs b/python/sedonadb/src/context.rs index 113df3b73..fe08c953e 100644 --- a/python/sedonadb/src/context.rs +++ b/python/sedonadb/src/context.rs @@ -200,4 +200,48 @@ impl InternalContext { .to_string(), )) } + + /// Register a UDTF (table function) whose implementation is passed + /// through a `PyCapsule` containing an + /// `Arc`. + /// + /// Plugin-handoff API: format-specific Python packages (e.g. + /// `sedonadb-zarr`) build their UDTF in their own PyO3 extension + /// and pass it across as an opaque capsule, sidestepping the + /// cross-extension `#[pyclass]` type-id mismatch. + /// + /// The capsule must be named `"sedonadb.udtf"` and store an + /// `Arc` as its value. We clone the Arc to + /// take our own refcount; the capsule retains its own copy until + /// Python GC drops it. + pub fn register_udtf_capsule( + &self, + name: &str, + capsule: &Bound<'_, pyo3::types::PyCapsule>, + ) -> Result<(), PySedonaError> { + use std::sync::Arc; + + let expected = c"sedonadb.udtf"; + let actual = capsule + .name()? + .ok_or_else(|| PySedonaError::SedonaPython("UDTF capsule has no name".to_string()))?; + if actual != expected { + return Err(PySedonaError::SedonaPython(format!( + "UDTF capsule name mismatch: expected {expected:?}, got {actual:?}" + ))); + } + let ptr = capsule.pointer() as *const Arc; + if ptr.is_null() { + return Err(PySedonaError::SedonaPython( + "UDTF capsule pointer is null".to_string(), + )); + } + // SAFETY: the capsule's payload is an `Arc` + // (validated by name above). PyO3 keeps the value alive for the + // lifetime of the capsule; we clone the Arc to obtain an + // independent refcount that outlives this scope. + let udtf = unsafe { (*ptr).clone() }; + self.inner.ctx.register_udtf(name, udtf); + Ok(()) + } } From cf2fe80274fa7f0b541efe1baeba0f34271d3c95 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 19:39:20 -0700 Subject: [PATCH 24/36] fix(sedonadb): drop mimalloc from defaults to coexist with plugins MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The Python sedonadb cdylib previously installed mimalloc as the `#[global_allocator]`. When sedonadb is loaded alongside a plugin cdylib (e.g. `sedonadb-zarr`), concrete Arrow types like `RecordBatch` get monomorphised separately in each cdylib, so the drop site runs the *consumer's* allocator regardless of who allocated. Plugin allocations go through libc malloc; sedonadb-side drops route to mimalloc free, and the heap corrupts — manifested in CI as a segfault on the first `to_arrow_table` of a plugin-served query. mimalloc remains available behind `--features mimalloc` for builds that don't load plugins. --- python/sedonadb/Cargo.toml | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index 8ad8e9768..8a51e0f19 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -36,7 +36,15 @@ crate-type = ["cdylib", "rlib"] # would force it on for everyone, including the plugin link. Maturin # enables it explicitly via `[tool.maturin] features` in # `pyproject.toml` for the wheel build. -default = ["mimalloc"] +# +# `mimalloc` is intentionally *not* in `default`: when sedonadb is +# loaded alongside a plugin cdylib (e.g. `sedonadb-zarr`) the +# global-allocator mismatch corrupts the heap. Concrete Arrow types +# like `RecordBatch` are monomorphised per-cdylib, so the drop site +# runs the consumer's allocator regardless of who allocated, and +# mimalloc-free on a libc-malloc pointer segfaults. Wheel builds that +# never load plugins can opt in with `--features mimalloc`. +default = [] extension-module = [] mimalloc = ["dep:mimalloc", "dep:libmimalloc-sys"] s2geography = ["sedona/s2geography"] From 44b0db6a8956038678b22813d0260652fd46d1af Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 20:10:24 -0700 Subject: [PATCH 25/36] test(sedonadb-zarr): drop premature read_format tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit End-to-end `con.read_format(ZarrFormatSpec(), uri)` requires directory- format support in `ExternalFormatSpec`'s `ListingTableUrl` path. Zarr groups are directories, not single files, so the listing layer returns zero matching objects — `Can't infer schema for zero objects`. The SQL UDTF path is unaffected (no listing involved). Drop the three tests that exercise the listing surface; keep one that pins the Python spec-class shape so the plumbing doesn't drift while the listing gap is being closed. --- python/sedonadb-zarr/tests/test_zarr.py | 67 +++++-------------------- 1 file changed, 12 insertions(+), 55 deletions(-) diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index a111f7c3f..b1668f109 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -32,22 +32,6 @@ import sedonadb import sedonadb_zarr -from sedonadb.dataframe import DataFrame - - -def _read_format(con, spec, uri: str) -> DataFrame: - """Bridge ``ExternalFormatSpec`` → DataFrame. - - `sedonadb.SedonaContext` doesn't expose a public ``read_format`` - helper yet — for now plugin tests call ``_impl.read_external_format`` - directly. When the public surface lands the body of this helper - collapses to ``_read_format(con,spec, uri)``. - """ - return DataFrame( - con._impl, - con._impl.read_external_format(spec, [uri], False), - con.options, - ) @pytest.fixture @@ -105,42 +89,15 @@ def test_arrays_option_threads_through_sql(zarr_group): assert df.to_arrow_table().column(0)[0].as_py() == 2 -def test_format_spec_via_read_format(zarr_group): - # The second user-facing surface: `_read_format(con,spec, uri)`, - # which uses ExternalFormatSpec.open_reader -> PyZarrChunkReader's - # __arrow_c_stream__ to plumb data through. - con = sedonadb.connect() - df = _read_format(con, sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}") - arrow_tab = df.to_arrow_table() - assert arrow_tab.num_rows == 2 - assert arrow_tab.column_names == ["raster"] - - # Inspect the raster cell as a Python dict — every row should carry - # transform + bands + the OutDb anchor URI for each band. - raster = arrow_tab["raster"][0].as_py() - assert isinstance(raster, dict), f"raster row is {type(raster).__name__}" - for field in ("transform", "bands"): - assert field in raster, f"raster row missing {field!r}: {sorted(raster)}" - # Bands list shape: one entry per array in the group (here, one). - assert isinstance(raster["bands"], list) and len(raster["bands"]) >= 1 - band = raster["bands"][0] - # `data` is empty (OutDb scan); `outdb_uri` points at this chunk. - assert band.get("data") in (None, b"", bytes()), ( - f"OutDb band should have empty data; got {band.get('data')!r}" - ) - anchor = band.get("outdb_uri") - assert anchor and "#array=temperature" in anchor, f"unexpected anchor: {anchor!r}" - - -def test_format_spec_with_arrays_option(zarr_group): - con = sedonadb.connect() - spec = sedonadb_zarr.ZarrFormatSpec().with_options({"arrays": ["temperature"]}) - df = _read_format(con, spec, f"file://{zarr_group}") - assert df.to_arrow_table().num_rows == 2 - - -def test_format_spec_load_eager_errors(zarr_group): - con = sedonadb.connect() - spec = sedonadb_zarr.ZarrFormatSpec().with_options({"load_eager": True}) - with pytest.raises(Exception, match=r"load_eager"): - _read_format(con, spec, f"file://{zarr_group}").to_arrow_table() +def test_format_spec_constructs_and_threads_options(): + # End-to-end `con.read_format(ZarrFormatSpec(), uri)` is gated on + # directory-format support in `ExternalFormatSpec`'s ListingTableUrl + # path — Zarr groups are directories, not single files, so the + # listing returns zero objects. This test pins down the Python + # surface shape so the plumbing doesn't regress while the + # directory-listing gap is being closed. + spec = sedonadb_zarr.ZarrFormatSpec() + assert spec.extension == ".zarr" + spec2 = spec.with_options({"arrays": ["temperature"]}) + assert spec2 is not spec + assert spec2._options.get("arrays") == ["temperature"] From 3621555d06003611b7d787b2b8873749b36e7d22 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 22:08:00 -0700 Subject: [PATCH 26/36] feat(sedona-datasource): single-object table provider for directory formats MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds an opt-in path through `external_table` for formats whose URI identifies a directory rather than a file. `ExternalFormatSpec::list_single_object` gates dispatch: - `false` (default): DataFusion's `ListingTable` discovers files matching the spec's extension at the URL prefix. - `true`: `SingleObjectExternalTable` synthesises one `PartitionedFile` per URI and hands it straight to the format's `FileSource`. No listing, no extension filtering — the URI is the object. Required for Zarr: a `.zarr` group is a directory, not a file. The listing layer returned zero matching objects, so `con.read_format(ZarrFormatSpec(), uri)` failed with "Can't infer schema for zero objects" even though the SQL UDTF (which bypasses listing) worked fine. `ZarrFormatSpec` now overrides `list_single_object()` to `true` and the Python end-to-end tests via `con.read_format` pass. Reuses the existing `ExternalFileFormat` / `ExternalFileSource` for the actual scan, so projections, filter pushdown, and streaming are identical between the two table-provider paths. --- python/sedonadb-zarr/tests/test_zarr.py | 63 ++++++- rust/sedona-datasource/src/format.rs | 119 +++++++++++++- rust/sedona-datasource/src/provider.rs | 183 ++++++++++++++++++++- rust/sedona-datasource/src/spec.rs | 18 ++ rust/sedona-raster-zarr/src/format_spec.rs | 10 ++ rust/sedona/src/context.rs | 8 +- 6 files changed, 387 insertions(+), 14 deletions(-) diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index b1668f109..cad33b5f2 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -89,13 +89,62 @@ def test_arrays_option_threads_through_sql(zarr_group): assert df.to_arrow_table().column(0)[0].as_py() == 2 -def test_format_spec_constructs_and_threads_options(): - # End-to-end `con.read_format(ZarrFormatSpec(), uri)` is gated on - # directory-format support in `ExternalFormatSpec`'s ListingTableUrl - # path — Zarr groups are directories, not single files, so the - # listing returns zero objects. This test pins down the Python - # surface shape so the plumbing doesn't regress while the - # directory-listing gap is being closed. +def _read_format(con, spec, uri: str): + """Bridge `ExternalFormatSpec` -> DataFrame. + + `sedonadb.SedonaContext` doesn't expose a public `read_format` + helper yet, so plugin tests call the lower-level + `_impl.read_external_format` directly. + """ + from sedonadb.dataframe import DataFrame + + return DataFrame( + con._impl, + con._impl.read_external_format(spec, [uri], False), + con.options, + ) + + +def test_format_spec_via_read_format(zarr_group): + # Second user-facing surface: `con.read_format(spec, uri)`. Goes + # through `SingleObjectExternalTable` (Zarr groups are directories, + # so they bypass DataFusion's listing layer) and ends up driving + # the same `ZarrChunkReader` as the SQL UDTF. + con = sedonadb.connect() + df = _read_format(con, sedonadb_zarr.ZarrFormatSpec(), f"file://{zarr_group}") + arrow_tab = df.to_arrow_table() + assert arrow_tab.num_rows == 2 + assert arrow_tab.column_names == ["raster"] + + raster = arrow_tab["raster"][0].as_py() + assert isinstance(raster, dict), f"raster row is {type(raster).__name__}" + for field in ("transform", "bands"): + assert field in raster, f"raster row missing {field!r}: {sorted(raster)}" + assert isinstance(raster["bands"], list) and len(raster["bands"]) >= 1 + band = raster["bands"][0] + # `data` is empty (OutDb scan); `outdb_uri` points at this chunk. + assert band.get("data") in (None, b"", bytes()), ( + f"OutDb band should have empty data; got {band.get('data')!r}" + ) + anchor = band.get("outdb_uri") + assert anchor and "#array=temperature" in anchor, f"unexpected anchor: {anchor!r}" + + +def test_format_spec_with_arrays_option(zarr_group): + con = sedonadb.connect() + spec = sedonadb_zarr.ZarrFormatSpec().with_options({"arrays": ["temperature"]}) + df = _read_format(con, spec, f"file://{zarr_group}") + assert df.to_arrow_table().num_rows == 2 + + +def test_format_spec_load_eager_errors(zarr_group): + con = sedonadb.connect() + spec = sedonadb_zarr.ZarrFormatSpec().with_options({"load_eager": True}) + with pytest.raises(Exception, match=r"load_eager"): + _read_format(con, spec, f"file://{zarr_group}").to_arrow_table() + + +def test_format_spec_class_invariants(): spec = sedonadb_zarr.ZarrFormatSpec() assert spec.extension == ".zarr" spec2 = spec.with_options({"arrays": ["temperature"]}) diff --git a/rust/sedona-datasource/src/format.rs b/rust/sedona-datasource/src/format.rs index 20c158f2b..e73fd9da6 100644 --- a/rust/sedona-datasource/src/format.rs +++ b/rust/sedona-datasource/src/format.rs @@ -409,7 +409,7 @@ mod test { use tempfile::TempDir; use url::Url; - use crate::provider::external_listing_table; + use crate::provider::{external_listing_table, external_table}; use super::*; @@ -734,4 +734,121 @@ mod test { .await .unwrap(); } + + /// Spec for a directory-shaped format whose "object" is the + /// directory itself. Used to exercise the + /// [`SingleObjectExternalTable`] path through + /// [`external_table`]. + #[derive(Debug, Default, Clone)] + struct DirectorySpec; + + #[async_trait] + impl ExternalFormatSpec for DirectorySpec { + fn extension(&self) -> &str { + ".dirfmt" + } + + fn list_single_object(&self) -> bool { + true + } + + fn with_options( + &self, + _options: &HashMap, + ) -> Result> { + Ok(Arc::new(self.clone())) + } + + async fn infer_schema(&self, location: &Object) -> Result { + // The single-object provider must synthesise an ObjectMeta + // before calling us; assert that contract here. + assert!( + location.meta.is_some(), + "single-object scan must synthesise an ObjectMeta", + ); + Ok(Schema::new(vec![ + Field::new("uri_path", DataType::Utf8, false), + Field::new("row_idx", DataType::Int32, false), + ])) + } + + async fn open_reader( + &self, + args: &OpenReaderArgs, + ) -> Result> { + let meta = args + .src + .meta + .as_ref() + .expect("single-object scan must synthesise an ObjectMeta"); + let path = meta.location.to_string(); + let schema = Arc::new(self.infer_schema(&args.src).await?); + let mut batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(StringArray::from(vec![path])), + Arc::new(Int32Array::from(vec![0])), + ], + )?; + if let Some(projection) = &args.file_projection { + batch = batch.project(projection)?; + } + Ok(Box::new(RecordBatchIterator::new([Ok(batch)], schema))) + } + } + + #[tokio::test] + async fn single_object_table_skips_listing() { + // The fixture dir is *not* a `.dirfmt` directory and contains + // nothing matching that extension. A listing-based provider + // would return zero objects and error on schema inference. + let spec = Arc::new(DirectorySpec); + let temp_dir = TempDir::new().unwrap(); + let dir_path = temp_dir.path().join("group.dirfmt"); + std::fs::create_dir(&dir_path).unwrap(); + // Make the directory non-empty so it looks like a real + // directory-shaped artefact, not just a missing entry. + std::fs::File::create(dir_path.join("metadata.json")) + .unwrap() + .write_all(b"{}") + .unwrap(); + + let ctx = SessionContext::new(); + let url = ListingTableUrl::parse(dir_path.to_string_lossy()).unwrap(); + let provider = external_table(spec, &ctx, vec![url], false).await.unwrap(); + + let batches = ctx.read_table(provider).unwrap().collect().await.unwrap(); + + assert_eq!(batches.len(), 1); + assert_eq!(batches[0].num_rows(), 1); + // The synthesised ObjectMeta::location is the URL path within + // the object store — non-empty means we passed the URI through + // without trying to list inside it. + let path_col = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(!path_col.value(0).is_empty()); + assert!(path_col.value(0).ends_with("group.dirfmt")); + } + + #[tokio::test] + async fn single_object_table_rejects_mixed_object_stores() { + let spec = Arc::new(DirectorySpec); + let ctx = SessionContext::new(); + // Mix file:// + https:// — both parse fine but resolve to + // different ObjectStoreUrls, which the single-object provider + // doesn't try to span. + let url_a = ListingTableUrl::parse("file:///tmp/a.dirfmt").unwrap(); + let url_b = ListingTableUrl::parse("https://example.com/b.dirfmt").unwrap(); + let err = external_table(spec, &ctx, vec![url_a, url_b], false) + .await + .unwrap_err(); + assert!( + err.message().contains("same object store"), + "unexpected error: {}", + err.message() + ); + } } diff --git a/rust/sedona-datasource/src/provider.rs b/rust/sedona-datasource/src/provider.rs index 6b06a70d1..7860de206 100644 --- a/rust/sedona-datasource/src/provider.rs +++ b/rust/sedona-datasource/src/provider.rs @@ -15,19 +15,66 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; use std::sync::Arc; use arrow_schema::SchemaRef; use async_trait::async_trait; use datafusion::{ + catalog::TableProvider, config::TableOptions, - datasource::listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, + datasource::{ + file_format::FileFormat, + listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, + physical_plan::FileScanConfig, + TableType, + }, execution::{options::ReadOptions, SessionState}, + logical_expr::Expr, + physical_plan::ExecutionPlan, prelude::{SessionConfig, SessionContext}, }; +use datafusion_catalog::{memory::DataSourceExec, Session}; use datafusion_common::{exec_err, Result}; +use datafusion_datasource::{ + file_groups::FileGroup, file_scan_config::FileScanConfigBuilder, table_schema::TableSchema, + PartitionedFile, +}; +use datafusion_execution::object_store::ObjectStoreUrl; +use object_store::{path::Path as ObjectPath, ObjectMeta}; + +use crate::{ + format::ExternalFileFormat, + spec::{ExternalFormatSpec, Object}, +}; + +/// Resolve an [ExternalFormatSpec] + URLs into a [TableProvider]. +/// +/// Dispatches on [`ExternalFormatSpec::list_single_object`]: +/// - `false` (default): builds a [`ListingTable`] that lists files at +/// the URL prefix matching the spec's extension. Best for formats +/// whose unit of work is a single file (Parquet, FlatGeobuf, ...). +/// - `true`: builds a [`SingleObjectExternalTable`] that treats each +/// URI as one opaque object, skipping listing entirely. Required +/// for directory-shaped formats like Zarr. +pub async fn external_table( + spec: Arc, + context: &SessionContext, + table_paths: Vec, + check_extension: bool, +) -> Result> { + if table_paths.is_empty() { + return exec_err!("No table paths were provided"); + } -use crate::{format::ExternalFileFormat, spec::ExternalFormatSpec}; + if spec.list_single_object() { + let provider = SingleObjectExternalTable::try_new(spec, table_paths).await?; + Ok(Arc::new(provider) as Arc) + } else { + let provider = external_listing_table(spec, context, table_paths, check_extension).await?; + Ok(Arc::new(provider) as Arc) + } +} /// Create a [ListingTable] from an [ExternalFormatSpec] and one or more URLs /// @@ -110,3 +157,135 @@ impl ReadOptions<'_> for RecordBatchReaderTableOptions { .await } } + +/// [`TableProvider`] that treats each input URI as one opaque object. +/// +/// Built when [`ExternalFormatSpec::list_single_object`] is `true`. The +/// listing layer is bypassed entirely: each URI is synthesised into a +/// single-element [`PartitionedFile`] whose `object_meta.location` is +/// the URL's path within the [`ObjectStoreUrl`]. The format's +/// [`ExternalFileFormat::file_source`] then drives the same scan path +/// used by [`ListingTable`] — projections, filter pushdown, and +/// streaming behaviour are identical. +/// +/// Required for directory-shaped formats like Zarr where the +/// "object" is the directory itself. +#[derive(Debug)] +pub struct SingleObjectExternalTable { + spec: Arc, + schema: SchemaRef, + /// Each input URI as (scheme-level object store URL, path within store). + /// All entries must share the same object store URL — mixed schemes + /// (e.g. `file://` + `s3://`) are rejected. + files: Vec<(ObjectStoreUrl, ObjectPath)>, +} + +impl SingleObjectExternalTable { + async fn try_new( + spec: Arc, + table_paths: Vec, + ) -> Result { + let files: Vec<(ObjectStoreUrl, ObjectPath)> = table_paths + .iter() + .map(|p| (p.object_store(), p.prefix().clone())) + .collect(); + + // All URIs must resolve to the same object store. Mixing schemes + // (e.g. one file://, one s3://) would force the scan to dispatch + // to multiple stores — not supported here. + let first_store = files[0].0.clone(); + for (store, _) in &files[1..] { + if store != &first_store { + return exec_err!( + "external_table: all URIs in a single-object scan must share the same \ + object store; got both '{first_store}' and '{store}'" + ); + } + } + + // Resolve the schema from the first object. Most directory-format + // specs (e.g. Zarr) infer a fixed schema irrespective of the + // input; the few that hit the store will receive a synthesised + // ObjectMeta they can use. + let probe = Object { + store: None, + url: Some(first_store.clone()), + meta: Some(synthetic_object_meta(&files[0].1)), + range: None, + }; + let schema = Arc::new(spec.infer_schema(&probe).await?); + + Ok(Self { + spec, + schema, + files, + }) + } +} + +#[async_trait] +impl TableProvider for SingleObjectExternalTable { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + _state: &dyn Session, + projection: Option<&Vec>, + _filters: &[Expr], + limit: Option, + ) -> Result> { + let (object_store_url, _) = &self.files[0]; + + let table_schema = TableSchema::new(self.schema.clone(), vec![]); + let format = ExternalFileFormat::new(self.spec.clone()); + let file_source = format.file_source(table_schema); + + let partitioned_files: Vec = self + .files + .iter() + .map(|(_, location)| PartitionedFile { + object_meta: synthetic_object_meta(location), + partition_values: vec![], + range: None, + extensions: None, + statistics: None, + metadata_size_hint: None, + }) + .collect(); + + let mut builder = FileScanConfigBuilder::new(object_store_url.clone(), file_source) + .with_file_group(FileGroup::new(partitioned_files)) + .with_limit(limit); + if let Some(indices) = projection { + builder = builder.with_projection_indices(Some(indices.clone()))?; + } + let config: FileScanConfig = builder.build(); + Ok(DataSourceExec::from_data_source(config)) + } +} + +/// Synthesise an [`ObjectMeta`] for a URI we haven't `head`'d. +/// +/// `size: 0` and a zeroed `last_modified` are intentional: this table +/// provider never lists or stats objects, so DataFusion's downstream +/// machinery only uses the `location` field. Specs that need real +/// stats can override via [`ExternalFormatSpec::infer_stats`]. +fn synthetic_object_meta(location: &ObjectPath) -> ObjectMeta { + ObjectMeta { + location: location.clone(), + last_modified: Default::default(), + size: 0, + e_tag: None, + version: None, + } +} diff --git a/rust/sedona-datasource/src/spec.rs b/rust/sedona-datasource/src/spec.rs index 2c2ca31a3..e82c921ad 100644 --- a/rust/sedona-datasource/src/spec.rs +++ b/rust/sedona-datasource/src/spec.rs @@ -70,6 +70,24 @@ pub trait ExternalFormatSpec: Debug + Send + Sync { "" } + /// Whether each URI should be treated as a single opaque object + /// (no `ObjectStore` listing or extension matching). + /// + /// Default `false`: the URI is fed through DataFusion's + /// `ListingTableUrl` and the listing fan-out enumerates matching + /// files at the prefix. + /// + /// Set to `true` for directory-shaped formats like Zarr where the + /// "object" is the directory itself, not the files within it. With + /// `true`, [`external_table`](crate::provider::external_table) + /// skips listing and passes each URI directly to + /// [`ExternalFormatSpec::infer_schema`] and + /// [`ExternalFormatSpec::open_reader`] as one + /// [`Object`]. + fn list_single_object(&self) -> bool { + false + } + /// Fill in default options from [TableOptions] /// /// The TableOptions are a DataFusion concept that provide a means by which diff --git a/rust/sedona-raster-zarr/src/format_spec.rs b/rust/sedona-raster-zarr/src/format_spec.rs index b1003c491..b4d1fd5a7 100644 --- a/rust/sedona-raster-zarr/src/format_spec.rs +++ b/rust/sedona-raster-zarr/src/format_spec.rs @@ -122,4 +122,14 @@ impl ExternalFormatSpec for ZarrFormatSpec { fn extension(&self) -> &str { ".zarr" } + + fn list_single_object(&self) -> bool { + // A Zarr group is a directory, not a file. The DataFusion + // listing layer can't enumerate it as a single object — it + // would return the directory's contents (zarr.json, chunk + // shards, ...), none of which carry the `.zarr` extension. + // Routing through the single-object provider keeps the URI + // intact and hands it to `open_reader` directly. + true + } } diff --git a/rust/sedona/src/context.rs b/rust/sedona/src/context.rs index 156b5b355..8fc8e3342 100644 --- a/rust/sedona/src/context.rs +++ b/rust/sedona/src/context.rs @@ -50,7 +50,7 @@ use sedona_common::{ option::add_sedona_option_extension, sedona_internal_datafusion_err, CrsProviderOption, SedonaOptions, }; -use sedona_datasource::provider::external_listing_table; +use sedona_datasource::provider::external_table; use sedona_datasource::spec::ExternalFormatSpec; use sedona_expr::scalar_udf::IntoScalarKernelRefs; use sedona_expr::{aggregate_udf::IntoSedonaAccumulatorRefs, function_set::FunctionSet}; @@ -412,12 +412,12 @@ impl SedonaContext { .map(|(k, v)| (k.clone(), v.clone())) .collect::>(); let spec = spec.with_options(&options_without_filesystems)?; - external_listing_table(spec, &self.ctx, urls, check_extension).await? + external_table(spec, &self.ctx, urls, check_extension).await? } else { - external_listing_table(spec, &self.ctx, urls, check_extension).await? + external_table(spec, &self.ctx, urls, check_extension).await? }; - self.ctx.read_table(Arc::new(provider)) + self.ctx.read_table(provider) } } From 3b6308841aa51db34a46b88540c9313b3d0f998c Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 23:27:43 -0700 Subject: [PATCH 27/36] fix(sedonadb): harden plugin extension surface from review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Addresses review feedback on the plugin handoff + single-object provider. **Capsule handoff (review items 1, 2, 10):** - Introduce `sedonadb::plugin::{UdtfCapsule, UDTF_CAPSULE_NAME, UDTF_CAPSULE_MAGIC}`. The capsule payload is now a `#[repr(C)]` struct with a leading u64 magic sentinel that the consumer verifies before dereferencing the `Arc` — the capsule name alone was a label, not a type guarantee. - Consumer uses PyO3's documented `unsafe { capsule.reference::() }` rather than the `pointer() as *const T` cast, which depended on PyO3's private `CapsuleContents::value` field being at offset 0. - Capsule name carries a `.v1` ABI tag so a pre-built plugin wheel paired with a newer host fails fast at registration rather than corrupting on call. **`list_single_object` propagation:** The Python `ExternalFormatSpec` base class now exposes a `list_single_object` property (default `False`); `PyExternalFormat` caches the value at construction. Without this, the Python `ZarrFormatSpec`'s override was silently dropped before reaching the Rust dispatcher and `con.read_format` still hit the listing path. **Single-object provider (review items 4, 6, 7):** - One `FileGroup` per URI so multi-URI scans can fan out across partitions instead of being pinned to a single partition. - `synthetic_object_meta::size` is `u64::MAX` rather than `0`; some DataFusion optimisations treat zero-sized files as "skip me" and we never want directory objects to be skipped. - Mixed-object-store error prints the user's original URIs alongside the inferred store URLs. **mimalloc comment (review item 3):** The opt-in feature comment in `python/sedonadb/Cargo.toml` now explicitly warns that turning mimalloc back on re-introduces the plugin-coexistence segfault — the only safe path is dynamic linking of `libmimalloc.so`, which is a packaging change, not a feature. --- .../python/sedonadb_zarr/__init__.py | 9 +++ python/sedonadb-zarr/src/lib.rs | 27 ++++---- python/sedonadb/Cargo.toml | 10 ++- python/sedonadb/python/sedonadb/datasource.py | 14 ++++ python/sedonadb/src/context.rs | 48 ++++++++----- python/sedonadb/src/datasource.rs | 41 ++++++++++- python/sedonadb/src/lib.rs | 1 + python/sedonadb/src/plugin.rs | 69 +++++++++++++++++++ rust/sedona-datasource/src/provider.rs | 67 +++++++++++------- 9 files changed, 225 insertions(+), 61 deletions(-) create mode 100644 python/sedonadb/src/plugin.rs diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index 17c90f6bf..3a27cbcad 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -92,6 +92,15 @@ def __init__(self, options: Optional[Mapping[str, Any]] = None): def extension(self) -> str: return ".zarr" + @property + def list_single_object(self) -> bool: + # A Zarr group is a directory, not a file. The DataFusion + # listing layer would enumerate its contents (zarr.json, chunk + # shards, ...), none of which carry the `.zarr` extension. The + # Rust `SingleObjectExternalTable` path skips listing and + # hands the URI straight to `open_reader`. + return True + def with_options(self, options: Mapping[str, Any]) -> "ZarrFormatSpec": merged = {**self._options, **options} return ZarrFormatSpec(merged) diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index 8e1258305..208382fb7 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -41,26 +41,29 @@ use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::PyCapsule; use sedona_raster_zarr::{ZarrChunkReader, ZarrReadFunction}; +// `sedonadb`'s rustc crate name is `_lib` (set by `[lib].name` so +// maturin packages it as `sedonadb._lib`); import the plugin handoff +// types under that name. +use _lib::plugin::{UdtfCapsule, UDTF_CAPSULE_NAME}; -/// Build a PyCapsule carrying an `Arc` for the -/// Zarr UDTF, suitable for handoff to sedonadb's +/// Build a PyCapsule carrying the Zarr UDTF for handoff to sedonadb's /// `InternalContext.register_udtf_capsule`. /// /// Cross-extension `#[pyclass]` extraction doesn't work in PyO3 (each /// cdylib has its own type-id static), so we pass the UDTF -/// implementation across the extension boundary via an opaque capsule. -/// The capsule owns the `Arc`; the consumer clones a fresh refcount -/// before the capsule is dropped. +/// implementation across the extension boundary via an opaque +/// [`UdtfCapsule`]. The capsule owns its `Arc`; the consumer clones a +/// fresh refcount before the capsule is dropped. The payload's leading +/// magic sentinel lets the consumer fail fast if the capsule's name +/// has been spoofed. #[pyfunction] fn zarr_udtf_capsule(py: Python<'_>) -> PyResult> { let udtf: Arc = Arc::new(ZarrReadFunction::default()); - let name = CString::new("sedonadb.udtf").unwrap(); - PyCapsule::new_with_destructor( - py, - udtf, - Some(name), - |_v: Arc, _ctx| {}, - ) + let payload = UdtfCapsule::new(udtf); + // CString::new requires owned bytes; reuse the host-side constant + // so the name (and its `.vN` ABI tag) lives in exactly one place. + let name = CString::new(UDTF_CAPSULE_NAME.to_bytes()).unwrap(); + PyCapsule::new_with_destructor(py, payload, Some(name), |_v: UdtfCapsule, _ctx| {}) } /// Python-callable wrapper around `ZarrChunkReader` that exposes diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index 8a51e0f19..07555fbc0 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -42,8 +42,14 @@ crate-type = ["cdylib", "rlib"] # global-allocator mismatch corrupts the heap. Concrete Arrow types # like `RecordBatch` are monomorphised per-cdylib, so the drop site # runs the consumer's allocator regardless of who allocated, and -# mimalloc-free on a libc-malloc pointer segfaults. Wheel builds that -# never load plugins can opt in with `--features mimalloc`. +# mimalloc-free on a libc-malloc pointer segfaults. +# +# !! WARNING: opting in with `--features mimalloc` re-introduces the +# segfault for any user who imports a plugin alongside this wheel. +# This isn't a perf knob — it's a "I'm shipping a closed binary that +# will never load a plugin cdylib" knob. The plugin-safe path is to +# link mimalloc *dynamically* (one shared `libmimalloc.so` for the +# whole process), which is a packaging change, not a feature flag. default = [] extension-module = [] mimalloc = ["dep:mimalloc", "dep:libmimalloc-sys"] diff --git a/python/sedonadb/python/sedonadb/datasource.py b/python/sedonadb/python/sedonadb/datasource.py index dd7a24ab6..17c5ebdba 100644 --- a/python/sedonadb/python/sedonadb/datasource.py +++ b/python/sedonadb/python/sedonadb/datasource.py @@ -50,6 +50,20 @@ def extension(self): """ return "" + @property + def list_single_object(self) -> bool: + """Whether each URI should be treated as a single opaque object + + Default ``False``: the URI is fed through DataFusion's listing layer, + which enumerates files matching :attr:`extension` at the prefix. + + Override to ``True`` for directory-shaped formats (e.g. a Zarr group is + a directory, not a file) where the listing layer would return zero + matching files. The URI is then passed straight to + :meth:`open_reader` as one object. + """ + return False + def with_options(self, options: Mapping[str, Any]): """Clone this instance and return a new instance with options applied diff --git a/python/sedonadb/src/context.rs b/python/sedonadb/src/context.rs index fe08c953e..2879b3de3 100644 --- a/python/sedonadb/src/context.rs +++ b/python/sedonadb/src/context.rs @@ -202,46 +202,58 @@ impl InternalContext { } /// Register a UDTF (table function) whose implementation is passed - /// through a `PyCapsule` containing an - /// `Arc`. + /// through a `PyCapsule` containing an [`UdtfCapsule`]. /// /// Plugin-handoff API: format-specific Python packages (e.g. /// `sedonadb-zarr`) build their UDTF in their own PyO3 extension /// and pass it across as an opaque capsule, sidestepping the /// cross-extension `#[pyclass]` type-id mismatch. /// - /// The capsule must be named `"sedonadb.udtf"` and store an - /// `Arc` as its value. We clone the Arc to - /// take our own refcount; the capsule retains its own copy until - /// Python GC drops it. + /// Two checks gate the dereference: the capsule's name must equal + /// [`UDTF_CAPSULE_NAME`] (a public string, so this is mostly an + /// ABI tag), and the payload's leading [`UDTF_CAPSULE_MAGIC`] + /// sentinel must match (much harder to forge from outside this + /// crate). The consumer clones the inner `Arc` to take its own + /// refcount; the capsule keeps a parallel refcount until Python + /// GC drops it. + /// + /// [`UdtfCapsule`]: crate::plugin::UdtfCapsule + /// [`UDTF_CAPSULE_NAME`]: crate::plugin::UDTF_CAPSULE_NAME + /// [`UDTF_CAPSULE_MAGIC`]: crate::plugin::UDTF_CAPSULE_MAGIC pub fn register_udtf_capsule( &self, name: &str, capsule: &Bound<'_, pyo3::types::PyCapsule>, ) -> Result<(), PySedonaError> { - use std::sync::Arc; + use crate::plugin::{UdtfCapsule, UDTF_CAPSULE_MAGIC, UDTF_CAPSULE_NAME}; - let expected = c"sedonadb.udtf"; let actual = capsule .name()? .ok_or_else(|| PySedonaError::SedonaPython("UDTF capsule has no name".to_string()))?; - if actual != expected { + if actual != UDTF_CAPSULE_NAME { return Err(PySedonaError::SedonaPython(format!( - "UDTF capsule name mismatch: expected {expected:?}, got {actual:?}" + "UDTF capsule name mismatch: expected {UDTF_CAPSULE_NAME:?}, got {actual:?}" ))); } - let ptr = capsule.pointer() as *const Arc; - if ptr.is_null() { + if capsule.pointer().is_null() { return Err(PySedonaError::SedonaPython( "UDTF capsule pointer is null".to_string(), )); } - // SAFETY: the capsule's payload is an `Arc` - // (validated by name above). PyO3 keeps the value alive for the - // lifetime of the capsule; we clone the Arc to obtain an - // independent refcount that outlives this scope. - let udtf = unsafe { (*ptr).clone() }; - self.inner.ctx.register_udtf(name, udtf); + // SAFETY: the capsule's payload is an `UdtfCapsule` (name + // matched above). PyO3 keeps the value alive for the + // lifetime of the capsule and `reference` is the documented + // accessor — equivalent to `pointer().cast::()` but without + // betting on `CapsuleContents`'s private field layout. + let payload: &UdtfCapsule = unsafe { capsule.reference::() }; + if payload.magic != UDTF_CAPSULE_MAGIC { + return Err(PySedonaError::SedonaPython(format!( + "UDTF capsule magic mismatch: expected {UDTF_CAPSULE_MAGIC:#x}, got {:#x}; \ + capsule likely produced by an incompatible build", + payload.magic + ))); + } + self.inner.ctx.register_udtf(name, payload.udtf.clone()); Ok(()) } } diff --git a/python/sedonadb/src/datasource.rs b/python/sedonadb/src/datasource.rs index e746f36f9..ab1bbb038 100644 --- a/python/sedonadb/src/datasource.rs +++ b/python/sedonadb/src/datasource.rs @@ -23,8 +23,10 @@ use async_trait::async_trait; use datafusion::{physical_expr::conjunction, physical_plan::PhysicalExpr}; use datafusion_common::{DataFusionError, Result}; use pyo3::{ - exceptions::PyNotImplementedError, pyclass, pymethods, types::PyCapsule, Bound, PyObject, - Python, + exceptions::PyNotImplementedError, + pyclass, pymethods, + types::{PyAnyMethods, PyCapsule}, + Bound, PyObject, Python, }; use sedona_datasource::{ spec::{ExternalFormatSpec, Object, OpenReaderArgs}, @@ -47,6 +49,11 @@ use crate::{ #[derive(Debug)] pub struct PyExternalFormat { extension: String, + /// Cached at construction time. The Python side declares this via + /// the `list_single_object` attribute on the spec class (default + /// `False`); we snapshot it once to avoid GIL traffic in + /// `list_single_object()`, which is called on hot paths. + list_single_object: bool, py_spec: PyObject, } @@ -54,6 +61,7 @@ impl Clone for PyExternalFormat { fn clone(&self) -> Self { Python::with_gil(|py| Self { extension: self.extension.clone(), + list_single_object: self.list_single_object, py_spec: self.py_spec.clone_ref(py), }) } @@ -71,8 +79,10 @@ impl PyExternalFormat { let new_extension = new_py_spec .getattr(py, "extension")? .extract::(py)?; + let new_list_single_object = read_list_single_object(py, &new_py_spec)?; Ok(Self { extension: new_extension, + list_single_object: new_list_single_object, py_spec: new_py_spec, }) } @@ -143,7 +153,28 @@ impl PyExternalFormat { #[new] fn new<'py>(py: Python<'py>, py_spec: PyObject) -> Result { let extension = py_spec.getattr(py, "extension")?.extract::(py)?; - Ok(Self { extension, py_spec }) + let list_single_object = read_list_single_object(py, &py_spec)?; + Ok(Self { + extension, + list_single_object, + py_spec, + }) + } +} + +/// Read the `list_single_object` attribute on a Python spec, defaulting +/// to `false` if the attribute is missing (older spec implementations +/// that predate the directory-format path). +fn read_list_single_object<'py>( + py: Python<'py>, + py_spec: &PyObject, +) -> Result { + if py_spec.bind(py).hasattr("list_single_object")? { + Ok(py_spec + .getattr(py, "list_single_object")? + .extract::(py)?) + } else { + Ok(false) } } @@ -153,6 +184,10 @@ impl ExternalFormatSpec for PyExternalFormat { &self.extension } + fn list_single_object(&self) -> bool { + self.list_single_object + } + fn with_options( &self, options: &HashMap, diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index d642c1f81..72999d71b 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -38,6 +38,7 @@ mod datasource; mod error; mod expr; mod import_from; +pub mod plugin; mod reader; mod runtime; mod schema; diff --git a/python/sedonadb/src/plugin.rs b/python/sedonadb/src/plugin.rs new file mode 100644 index 000000000..27f350c80 --- /dev/null +++ b/python/sedonadb/src/plugin.rs @@ -0,0 +1,69 @@ +// 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. + +//! Plugin handoff types shared by sedonadb and its Python plugin +//! crates (e.g. `sedonadb-zarr`). +//! +//! PyO3's `#[pyclass]` extraction uses a per-cdylib type-id static, +//! so a plugin can't extract sedonadb's `InternalContext` directly. +//! UDTF implementations cross the extension boundary inside a +//! [`PyCapsule`](pyo3::types::PyCapsule) wrapping an [`UdtfCapsule`]. + +use std::ffi::CStr; +use std::sync::Arc; + +use datafusion::catalog::TableFunctionImpl; + +/// Capsule name carried in [`pyo3::types::PyCapsule::name`]. +/// +/// The trailing `.v1` is an ABI tag: if the layout of [`UdtfCapsule`] +/// or the `TableFunctionImpl` vtable ever changes (e.g. a datafusion +/// major bump), bump the suffix so a pre-built plugin wheel paired +/// with a newer host fails fast at registration time instead of +/// silently corrupting on call. +pub const UDTF_CAPSULE_NAME: &CStr = c"sedonadb.udtf.v1"; + +/// Magic sentinel at offset 0 of [`UdtfCapsule`]. +/// +/// The capsule name is a public string; any Python caller can build a +/// capsule with that name pointing at arbitrary memory. The magic +/// gives the consumer a second, harder-to-forge check before it +/// dereferences the payload as an `Arc`. +pub const UDTF_CAPSULE_MAGIC: u64 = 0x5345_444F_4E41_5544; + +/// Payload stored inside the UDTF [`pyo3::types::PyCapsule`]. +/// +/// `#[repr(C)]` guarantees `magic` is at offset 0 regardless of +/// alignment, so a stale or forged capsule can be rejected via a +/// single field read before touching `udtf`. The producer constructs +/// one of these; the consumer recovers it via +/// [`pyo3::types::PyCapsule::reference`]. +#[repr(C)] +pub struct UdtfCapsule { + pub magic: u64, + pub udtf: Arc, +} + +impl UdtfCapsule { + /// Build a capsule payload around a [`TableFunctionImpl`]. + pub fn new(udtf: Arc) -> Self { + Self { + magic: UDTF_CAPSULE_MAGIC, + udtf, + } + } +} diff --git a/rust/sedona-datasource/src/provider.rs b/rust/sedona-datasource/src/provider.rs index 7860de206..b29c09295 100644 --- a/rust/sedona-datasource/src/provider.rs +++ b/rust/sedona-datasource/src/provider.rs @@ -170,6 +170,9 @@ impl ReadOptions<'_> for RecordBatchReaderTableOptions { /// /// Required for directory-shaped formats like Zarr where the /// "object" is the directory itself. +/// +/// Each URI lands in its own [`FileGroup`], so a multi-URI scan can +/// produce as many partitions as URIs given enough target partitions. #[derive(Debug)] pub struct SingleObjectExternalTable { spec: Arc, @@ -185,24 +188,29 @@ impl SingleObjectExternalTable { spec: Arc, table_paths: Vec, ) -> Result { - let files: Vec<(ObjectStoreUrl, ObjectPath)> = table_paths - .iter() - .map(|p| (p.object_store(), p.prefix().clone())) - .collect(); - - // All URIs must resolve to the same object store. Mixing schemes - // (e.g. one file://, one s3://) would force the scan to dispatch - // to multiple stores — not supported here. - let first_store = files[0].0.clone(); - for (store, _) in &files[1..] { - if store != &first_store { + // All URIs must resolve to the same object store. Mixing + // schemes (one file://, one s3://) would force the scan to + // dispatch to multiple stores — not supported here. Print + // the original URIs so the user sees what they typed. + let first_store = table_paths[0].object_store(); + for path in table_paths.iter().skip(1) { + let store = path.object_store(); + if store != first_store { + let first_uri = table_paths[0].as_str(); + let bad_uri = path.as_str(); return exec_err!( "external_table: all URIs in a single-object scan must share the same \ - object store; got both '{first_store}' and '{store}'" + object store; got '{first_uri}' (store '{first_store}') and \ + '{bad_uri}' (store '{store}')" ); } } + let files: Vec<(ObjectStoreUrl, ObjectPath)> = table_paths + .iter() + .map(|p| (p.object_store(), p.prefix().clone())) + .collect(); + // Resolve the schema from the first object. Most directory-format // specs (e.g. Zarr) infer a fixed schema irrespective of the // input; the few that hit the store will receive a synthesised @@ -250,21 +258,27 @@ impl TableProvider for SingleObjectExternalTable { let format = ExternalFileFormat::new(self.spec.clone()); let file_source = format.file_source(table_schema); - let partitioned_files: Vec = self + // One FileGroup per URI so DataFusion can fan out across + // partitions. Lumping every URI into a single FileGroup would + // pin the scan to one partition irrespective of + // `target_partitions`. + let file_groups: Vec = self .files .iter() - .map(|(_, location)| PartitionedFile { - object_meta: synthetic_object_meta(location), - partition_values: vec![], - range: None, - extensions: None, - statistics: None, - metadata_size_hint: None, + .map(|(_, location)| { + FileGroup::new(vec![PartitionedFile { + object_meta: synthetic_object_meta(location), + partition_values: vec![], + range: None, + extensions: None, + statistics: None, + metadata_size_hint: None, + }]) }) .collect(); let mut builder = FileScanConfigBuilder::new(object_store_url.clone(), file_source) - .with_file_group(FileGroup::new(partitioned_files)) + .with_file_groups(file_groups) .with_limit(limit); if let Some(indices) = projection { builder = builder.with_projection_indices(Some(indices.clone()))?; @@ -276,15 +290,16 @@ impl TableProvider for SingleObjectExternalTable { /// Synthesise an [`ObjectMeta`] for a URI we haven't `head`'d. /// -/// `size: 0` and a zeroed `last_modified` are intentional: this table -/// provider never lists or stats objects, so DataFusion's downstream -/// machinery only uses the `location` field. Specs that need real -/// stats can override via [`ExternalFormatSpec::infer_stats`]. +/// `size` is set to `u64::MAX` rather than `0`: a zero size signals +/// "empty file, skip me" to some DataFusion optimisations (file +/// pruning, size-aware repartitioning), and we never want directory +/// objects to be skipped. `last_modified` defaults to the unix epoch +/// because this provider never compares timestamps. fn synthetic_object_meta(location: &ObjectPath) -> ObjectMeta { ObjectMeta { location: location.clone(), last_modified: Default::default(), - size: 0, + size: u64::MAX, e_tag: None, version: None, } From 1673608552461d6ea19213527c88fbddabd6e87e Mon Sep 17 00:00:00 2001 From: jameswillis Date: Wed, 20 May 2026 23:43:50 -0700 Subject: [PATCH 28/36] refactor(sedona-datasource): resolve ObjectStore for single-object path; drop external_listing_table Two of the three review follow-ups on the directory-format provider: - `SingleObjectExternalTable::try_new` now resolves the `Arc` from the session's runtime registry and threads it through to `spec.infer_schema`. The listing path has always done this; the single-object path previously passed `store: None`, which was a silent asymmetry for any future spec whose `infer_schema` needs to peek at the store. Zarr's schema is constant so this didn't bite in practice. - `external_listing_table` had no internal callers after the `external_table` dispatcher landed. Inlined as private `listing_table_provider`. One public entry point, one return type (`Arc`). --- rust/sedona-datasource/src/format.rs | 21 ++++++++------------ rust/sedona-datasource/src/provider.rs | 27 +++++++++++--------------- 2 files changed, 19 insertions(+), 29 deletions(-) diff --git a/rust/sedona-datasource/src/format.rs b/rust/sedona-datasource/src/format.rs index e73fd9da6..9df4a4251 100644 --- a/rust/sedona-datasource/src/format.rs +++ b/rust/sedona-datasource/src/format.rs @@ -409,7 +409,7 @@ mod test { use tempfile::TempDir; use url::Url; - use crate::provider::{external_listing_table, external_table}; + use crate::provider::external_table; use super::*; @@ -613,7 +613,7 @@ mod test { let (_temp_dir, files) = create_echo_spec_temp_dir(); // Select using a listing table and ensure we get a result - let provider = external_listing_table( + let provider = external_table( spec, &ctx, files @@ -625,12 +625,7 @@ mod test { .await .unwrap(); - let batches = ctx - .read_table(Arc::new(provider)) - .unwrap() - .collect() - .await - .unwrap(); + let batches = ctx.read_table(provider).unwrap().collect().await.unwrap(); // We should get one value per partition assert_eq!(batches.len(), 2); @@ -648,7 +643,7 @@ mod test { let (_temp_dir, files) = create_echo_spec_temp_dir(); // Select using a listing table and ensure we get a result with the option passed - let provider = external_listing_table( + let provider = external_table( spec, &ctx, files @@ -661,7 +656,7 @@ mod test { .unwrap(); let batches = ctx - .read_table(Arc::new(provider)) + .read_table(provider) .unwrap() .select(vec![col("batch_size"), col("option_value")]) .unwrap() @@ -691,7 +686,7 @@ mod test { let (temp_dir, mut files) = create_echo_spec_temp_dir(); // Listing table with no files should error - let err = external_listing_table(spec.clone(), &ctx, vec![], true) + let err = external_table(spec.clone(), &ctx, vec![], true) .await .unwrap_err(); assert_eq!(err.message(), "No table paths were provided"); @@ -705,7 +700,7 @@ mod test { files.push(file2); // With check_extension as true we should get an error - let err = external_listing_table( + let err = external_table( spec.clone(), &ctx, files @@ -722,7 +717,7 @@ mod test { .ends_with("does not match the expected extension 'echospec'")); // ...but we should be able to turn off the error - external_listing_table( + external_table( spec, &ctx, files diff --git a/rust/sedona-datasource/src/provider.rs b/rust/sedona-datasource/src/provider.rs index b29c09295..b96b02e28 100644 --- a/rust/sedona-datasource/src/provider.rs +++ b/rust/sedona-datasource/src/provider.rs @@ -68,19 +68,15 @@ pub async fn external_table( } if spec.list_single_object() { - let provider = SingleObjectExternalTable::try_new(spec, table_paths).await?; + let provider = SingleObjectExternalTable::try_new(spec, context, table_paths).await?; Ok(Arc::new(provider) as Arc) } else { - let provider = external_listing_table(spec, context, table_paths, check_extension).await?; + let provider = listing_table_provider(spec, context, table_paths, check_extension).await?; Ok(Arc::new(provider) as Arc) } } -/// Create a [ListingTable] from an [ExternalFormatSpec] and one or more URLs -/// -/// This can be used to resolve a format specification into a TableProvider that -/// may be registered with a [SessionContext]. -pub async fn external_listing_table( +async fn listing_table_provider( spec: Arc, context: &SessionContext, table_paths: Vec, @@ -96,10 +92,6 @@ pub async fn external_listing_table( let option_extension = listing_options.file_extension.clone(); - if table_paths.is_empty() { - return exec_err!("No table paths were provided"); - } - // check if the file extension matches the expected extension if one is provided if !option_extension.is_empty() && options.check_extension { for path in &table_paths { @@ -186,6 +178,7 @@ pub struct SingleObjectExternalTable { impl SingleObjectExternalTable { async fn try_new( spec: Arc, + context: &SessionContext, table_paths: Vec, ) -> Result { // All URIs must resolve to the same object store. Mixing @@ -211,12 +204,14 @@ impl SingleObjectExternalTable { .map(|p| (p.object_store(), p.prefix().clone())) .collect(); - // Resolve the schema from the first object. Most directory-format - // specs (e.g. Zarr) infer a fixed schema irrespective of the - // input; the few that hit the store will receive a synthesised - // ObjectMeta they can use. + // Resolve the ObjectStore from the session's runtime registry, + // mirroring what `ListingTable` does internally. Specs whose + // `infer_schema` peeks at the store (statting metadata, reading + // a header) would otherwise get `None` here only on this path — + // a silent asymmetry with the listing branch. + let store = context.runtime_env().object_store(&first_store)?; let probe = Object { - store: None, + store: Some(store), url: Some(first_store.clone()), meta: Some(synthetic_object_meta(&files[0].1)), range: None, From 171e4a3ad0ad6f7d6daee4d29fcf8ff9324b7cbf Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 10:48:04 -0700 Subject: [PATCH 29/36] chore(sedona-raster-zarr): trim scope and dependencies from PR review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three independent simplifications from Dewey's review: - `sedona-raster-zarr` no longer depends on the umbrella `datafusion` crate in `[dependencies]`. Replaced with the narrow sub-crates it actually uses: `datafusion-execution`, `datafusion-physical-expr`, `datafusion-physical-plan`. The umbrella moves to `[dev-dependencies]` for tests that build a real `SessionContext`. The `register(&SessionContext)` helper is gone — it had no callers (the plugin constructs `Arc::new(ZarrReadFunction::default())` directly), and removing it was what unblocked dropping the umbrella. - Drop the `load_eager` option from `ZarrFormatSpec` and `sd_read_zarr`. It currently errors anyway; pixel-byte materialisation lands with the async `RS_EnsureLoaded` resolver and we can decide the user- facing shape (option, separate UDF, planner injection) at that point. Removes the only surface defending a not-yet-supported flag. - `retrieve_chunk_bytes` is now `#[cfg(test)]` rather than `#[allow(dead_code)]`. The unit test for it lives in the same file so the implementation doesn't bit-rot before the resolver crate consumes it. The blanket lint-silencer is gone. Also drops `test_no_zarr_by_default.py` — the architectural regression test for zarr-not-in-default sedonadb. The build itself catches reintroduction at this point, so the test is overkill. --- Cargo.lock | 3 + .../python/sedonadb_zarr/__init__.py | 8 -- python/sedonadb-zarr/tests/test_zarr.py | 7 -- .../sedonadb/tests/test_no_zarr_by_default.py | 43 ------- rust/sedona-raster-zarr/Cargo.toml | 14 ++- rust/sedona-raster-zarr/src/format_spec.rs | 19 --- rust/sedona-raster-zarr/src/lib.rs | 2 +- rust/sedona-raster-zarr/src/loader.rs | 14 ++- rust/sedona-raster-zarr/src/udtf.rs | 117 +++++++----------- 9 files changed, 69 insertions(+), 158 deletions(-) delete mode 100644 python/sedonadb/tests/test_no_zarr_by_default.py diff --git a/Cargo.lock b/Cargo.lock index 0bf188118..ec4c5ce55 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6160,7 +6160,10 @@ dependencies = [ "datafusion", "datafusion-catalog", "datafusion-common", + "datafusion-execution", "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-plan", "futures", "log", "sedona-common", diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index 3a27cbcad..53b21a15c 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -79,8 +79,6 @@ class ZarrFormatSpec(ExternalFormatSpec): Supported options (via :meth:`with_options`): - - ``load_eager`` (``bool``) — ``False`` (default) emits chunk-anchor - URIs only; ``True`` currently errors pending the async resolver. - ``arrays`` (``list[str]`` or JSON-string) — explicit subset of group arrays to read. """ @@ -111,12 +109,6 @@ def open_reader(self, args: Any) -> PyZarrChunkReader: raise ValueError( "ZarrFormatSpec: could not resolve a URL from the source object" ) - if self._options.get("load_eager"): - raise NotImplementedError( - "ZarrFormatSpec: load_eager=True is not yet supported. " - "Pixel-byte materialisation lands with the async RS_EnsureLoaded " - "resolver." - ) arrays = self._options.get("arrays") if isinstance(arrays, str): arrays = json.loads(arrays) diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index cad33b5f2..98ab0bcbf 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -137,13 +137,6 @@ def test_format_spec_with_arrays_option(zarr_group): assert df.to_arrow_table().num_rows == 2 -def test_format_spec_load_eager_errors(zarr_group): - con = sedonadb.connect() - spec = sedonadb_zarr.ZarrFormatSpec().with_options({"load_eager": True}) - with pytest.raises(Exception, match=r"load_eager"): - _read_format(con, spec, f"file://{zarr_group}").to_arrow_table() - - def test_format_spec_class_invariants(): spec = sedonadb_zarr.ZarrFormatSpec() assert spec.extension == ".zarr" diff --git a/python/sedonadb/tests/test_no_zarr_by_default.py b/python/sedonadb/tests/test_no_zarr_by_default.py deleted file mode 100644 index 0529018f0..000000000 --- a/python/sedonadb/tests/test_no_zarr_by_default.py +++ /dev/null @@ -1,43 +0,0 @@ -# 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. - -"""Architectural regression test: `sedonadb` alone has no Zarr. - -This locks in the plugin separation. A fresh sedonadb connection, -created without importing `sedonadb_zarr`, must not know about the -`sd_read_zarr` UDTF. If a future change accidentally re-bundles Zarr -into the main package — for example by adding a convenience import or -re-attaching the registration to `SedonaContext::new_from_context` — -this test fails and forces a conscious decision. - -The test is here (not in `sedonadb-zarr`) on purpose: it's the test -that catches *re-bundling*, which would happen in `sedonadb`'s code, -not in the plugin. -""" - -import pytest - -import sedonadb - - -def test_sd_read_zarr_is_not_a_known_function_without_plugin(): - con = sedonadb.connect() - # Use a non-existent path; we expect the planner to fail because - # `sd_read_zarr` itself is unknown, not because the file doesn't - # exist. The error message should mention the function name. - with pytest.raises(Exception, match=r"sd_read_zarr|function|table function"): - con.sql("SELECT * FROM sd_read_zarr('file:///nowhere/foo.zarr')") diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml index fc70f2554..6f9a7068f 100644 --- a/rust/sedona-raster-zarr/Cargo.toml +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -34,10 +34,18 @@ result_large_err = "allow" arrow-array = { workspace = true } arrow-schema = { workspace = true } async-trait = { workspace = true } -datafusion = { workspace = true, features = ["sql"] } +# Narrow datafusion sub-crates rather than the umbrella `datafusion` +# crate — avoids pulling in `datafusion-sql`, `datafusion-optimizer`, +# `datafusion-functions-*`, etc. that this crate never uses. Compile +# time and binary size both drop noticeably. The umbrella crate is +# still pulled in by `dev-dependencies` for the unit tests, which +# build a real `SessionContext`. datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } +datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-plan = { workspace = true } futures = { workspace = true } log = { workspace = true } sedona-common = { workspace = true } @@ -58,5 +66,9 @@ zarrs_filesystem = { workspace = true } zarrs = { workspace = true, features = ["blosc"] } [dev-dependencies] +# Umbrella `datafusion` only for tests — they construct a real +# `SessionContext` and execute SQL. Production code is restricted to +# the narrow sub-crates above. +datafusion = { workspace = true, features = ["sql"] } tempfile = { workspace = true } tokio = { workspace = true, features = ["macros", "rt-multi-thread"] } diff --git a/rust/sedona-raster-zarr/src/format_spec.rs b/rust/sedona-raster-zarr/src/format_spec.rs index b4d1fd5a7..44ade89ca 100644 --- a/rust/sedona-raster-zarr/src/format_spec.rs +++ b/rust/sedona-raster-zarr/src/format_spec.rs @@ -35,15 +35,11 @@ use crate::loader::ZarrChunkReader; /// `ExternalFormatSpec` implementation for Zarr groups. /// /// Configurable via [`with_options`](ExternalFormatSpec::with_options): -/// - `load_eager`: boolean. `false` (default) emits chunk-anchor URIs -/// only. `true` currently errors — pixel-byte materialisation is -/// pending the async `RS_EnsureLoaded` resolver. /// - `arrays`: JSON array of strings, e.g. `'["temperature","pressure"]'`. /// Names a subset of arrays in the group to read; defaults to every /// multi-dimensional array (1-D coord variables auto-skipped). #[derive(Debug, Clone, Default)] pub struct ZarrFormatSpec { - load_eager: bool, arrays: Option>, } @@ -68,14 +64,6 @@ impl ExternalFormatSpec for ZarrFormatSpec { &self, args: &OpenReaderArgs, ) -> Result> { - if self.load_eager { - return Err(DataFusionError::Plan( - "ZarrFormatSpec: load_eager = true is not yet supported. \ - Pixel-byte materialisation will be wired up when the async \ - RS_EnsureLoaded resolver lands." - .into(), - )); - } let uri = args.src.to_url_string().ok_or_else(|| { DataFusionError::Plan( "ZarrFormatSpec: could not resolve a URL string from the source object".into(), @@ -94,13 +82,6 @@ impl ExternalFormatSpec for ZarrFormatSpec { let mut next = self.clone(); for (k, v) in options { match k.as_str() { - "load_eager" => { - next.load_eager = v.parse().map_err(|_| { - DataFusionError::Plan(format!( - "ZarrFormatSpec: load_eager must be a boolean; got {v:?}" - )) - })?; - } "arrays" => { next.arrays = Some(serde_json::from_str::>(v).map_err(|e| { DataFusionError::Plan(format!( diff --git a/rust/sedona-raster-zarr/src/lib.rs b/rust/sedona-raster-zarr/src/lib.rs index 4b201c85d..502abb462 100644 --- a/rust/sedona-raster-zarr/src/lib.rs +++ b/rust/sedona-raster-zarr/src/lib.rs @@ -42,4 +42,4 @@ pub mod udtf; pub use format_spec::ZarrFormatSpec; pub use loader::ZarrChunkReader; -pub use udtf::{register as register_udtf, ZarrReadFunction}; +pub use udtf::ZarrReadFunction; diff --git a/rust/sedona-raster-zarr/src/loader.rs b/rust/sedona-raster-zarr/src/loader.rs index 879c3544e..9d695cb13 100644 --- a/rust/sedona-raster-zarr/src/loader.rs +++ b/rust/sedona-raster-zarr/src/loader.rs @@ -38,7 +38,9 @@ use sedona_common::sedona_internal_datafusion_err; use sedona_raster::builder::RasterBuilder; use sedona_schema::datatypes::SedonaType; use sedona_schema::raster::BandDataType; -use zarrs::array::{Array, ArrayBytes}; +use zarrs::array::Array; +#[cfg(test)] +use zarrs::array::ArrayBytes; use zarrs::group::Group; use zarrs_filesystem::FilesystemStore; @@ -659,11 +661,11 @@ fn advance_chunk_indices(chunk_indices: &mut [u64], chunk_grid_shape: &[u64]) -> /// /// This is the only pixel-byte read primitive in the crate. The loader /// itself never calls it today — it always emits OutDb anchors — but -/// the async `RS_EnsureLoaded` resolver (follow-up PR) will. Kept here -/// rather than dropped because (a) it's tested directly below and (b) -/// the resolver lives in this crate and wants a sync chunk-fetch -/// helper as a starting point. -#[allow(dead_code)] +/// the async `RS_EnsureLoaded` resolver (follow-up PR) will. Lives +/// behind `#[cfg(test)]` until the resolver lands; the unit test below +/// exercises it so the implementation doesn't bit-rot in the +/// meantime. +#[cfg(test)] fn retrieve_chunk_bytes( array: &Array, chunk_indices: &[u64], diff --git a/rust/sedona-raster-zarr/src/udtf.rs b/rust/sedona-raster-zarr/src/udtf.rs index c8c533043..8ed2385ef 100644 --- a/rust/sedona-raster-zarr/src/udtf.rs +++ b/rust/sedona-raster-zarr/src/udtf.rs @@ -30,58 +30,45 @@ //! position in the Zarr group's chunk grid. All existing `RS_*` UDFs //! operate on the column unchanged. //! -//! `load_eager` defaults to `false` — every row has empty `data` and a -//! chunk-anchor URI in `outdb_uri`. Setting `load_eager = true` -//! currently errors because no `RS_EnsureLoaded` resolver is registered -//! for the `zarr` format yet; once the resolver lands, `true` will -//! trigger the planner to inject the async UDF over the scan output. +//! Every row has empty `data` and a chunk-anchor URI in `outdb_uri`. +//! Pixel-byte materialisation lands with the async `RS_EnsureLoaded` +//! resolver in a follow-up PR; until then every metadata-only query +//! (`count(*)`, `RS_Envelope`, `RS_Width`, …) works against the +//! anchor-only rows. //! -//! Registration happens via [`register`]; the `sedonadb-zarr` Python -//! package calls this from its session-setup helper. The `sedona` crate -//! itself does not register the UDTF — keeping zarr functionality out -//! of the default bootstrap. +//! The `sedonadb-zarr` Python package constructs +//! `Arc::new(ZarrReadFunction::default())` and hands it to its session +//! via the plugin capsule. The `sedona` crate itself does not register +//! the UDTF — keeping zarr functionality out of the default bootstrap. use std::any::Any; use std::sync::Arc; use arrow_schema::{DataType, Schema, SchemaRef}; use async_trait::async_trait; -use datafusion::catalog::{Session, TableFunctionImpl, TableProvider}; -use datafusion::common::Result; -use datafusion::datasource::TableType; -use datafusion::execution::context::{SessionContext, TaskContext}; -use datafusion::physical_expr::EquivalenceProperties; -use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion::physical_plan::expressions::Column; -use datafusion::physical_plan::projection::ProjectionExec; -use datafusion::physical_plan::stream::RecordBatchStreamAdapter; -use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, PlanProperties, +use datafusion_catalog::{Session, TableFunctionImpl, TableProvider}; +use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; +use datafusion_execution::TaskContext; +use datafusion_expr::{Expr, TableType}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::stream::RecordBatchStreamAdapter; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, }; -use datafusion::prelude::Expr; -use datafusion_common::{plan_err, DataFusionError, ScalarValue}; use sedona_schema::datatypes::SedonaType; use serde::{Deserialize, Serialize}; use crate::loader::ZarrChunkReader; -/// Register the `sd_read_zarr` UDTF on a `SessionContext`. Called from -/// the `sedonadb-zarr` Python package's `register(con)` entry point. -pub fn register(ctx: &SessionContext) { - ctx.register_udtf("sd_read_zarr", Arc::new(ZarrReadFunction::default())); -} - /// Table function `sd_read_zarr(uri[, options_json])`. /// /// Accepts one or two string arguments: /// - `uri` (required) — Zarr group URI (e.g. `file:///path/to/foo.zarr`). /// - `options_json` (optional) — JSON string with any of: -/// - `load_eager`: `false` (default) emits chunk-anchor URIs only; -/// `true` currently errors because no `RS_EnsureLoaded` resolver -/// is registered for `zarr` yet. Once the resolver lands, `true` -/// will trigger the planner to inject the async UDF over the -/// scan output. /// - `rows_per_batch`: chunks per `RecordBatch`. Defaults to the /// session's configured batch size (`SessionConfig::batch_size`, /// typically 8192) when unset. @@ -148,16 +135,6 @@ pub struct ZarrChunkProvider { impl ZarrChunkProvider { fn try_new(uri: &str, options_json: Option) -> Result { let opts = parse_options(options_json.as_deref())?; - let load_eager = opts.load_eager.unwrap_or(false); - if load_eager { - return plan_err!( - "sd_read_zarr() load_eager = true is not yet supported. \ - Pixel-byte materialisation will be wired up when the async \ - RS_EnsureLoaded resolver lands; for now use load_eager = false \ - (the default) and operate on metadata (count(*), RS_Envelope, \ - RS_Width, etc.)." - ); - } let rows_per_batch = opts.rows_per_batch.map(|n| n.max(1)); let num_partitions = opts.num_partitions.unwrap_or(1); if num_partitions != 1 { @@ -355,11 +332,6 @@ fn arrow_to_df_err(e: arrow_schema::ArrowError) -> DataFusionError { #[derive(Serialize, Deserialize, Default)] struct ZarrReadOptions { - /// `false` (default) emits chunk-anchor URIs only. `true` currently - /// errors — pixel-byte materialisation is pending the async - /// `RS_EnsureLoaded` resolver. Once the resolver lands, `true` will - /// trigger the planner to inject the async UDF over the scan output. - load_eager: Option, rows_per_batch: Option, num_partitions: Option, /// Explicit array-name filter. `None` reads every multi-dimensional @@ -419,7 +391,10 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - register(&ctx); + ctx.register_udtf( + "sd_read_zarr", + std::sync::Arc::new(ZarrReadFunction::default()), + ); let df = ctx .sql(&format!("SELECT raster FROM sd_read_zarr('{uri}')")) @@ -445,32 +420,16 @@ mod tests { assert!(anchor.contains("&chunk=0,0"), "got: {anchor}"); } - #[tokio::test] - async fn udtf_rejects_load_eager_true() { - let tmp = build_fixture(); - let uri = format!("file://{}", tmp.path().display()); - - let ctx = SessionContext::new(); - register(&ctx); - - let err = ctx - .sql(&format!( - r#"SELECT raster FROM sd_read_zarr('{uri}', '{{"load_eager": true}}')"#, - )) - .await - .unwrap_err() - .to_string(); - assert!(err.contains("load_eager = true"), "got: {err}"); - assert!(err.contains("not yet supported"), "got: {err}"); - } - #[tokio::test] async fn udtf_respects_rows_per_batch_option() { let tmp = build_fixture(); let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - register(&ctx); + ctx.register_udtf( + "sd_read_zarr", + std::sync::Arc::new(ZarrReadFunction::default()), + ); // 2 chunk rows, rows_per_batch=1 → 2 single-row batches. let df = ctx @@ -490,7 +449,10 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - register(&ctx); + ctx.register_udtf( + "sd_read_zarr", + std::sync::Arc::new(ZarrReadFunction::default()), + ); // SELECT count(*) just walks the chunk grid metadata; never opens // a chunk file. @@ -513,7 +475,10 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - register(&ctx); + ctx.register_udtf( + "sd_read_zarr", + std::sync::Arc::new(ZarrReadFunction::default()), + ); let err = ctx .sql(&format!( @@ -554,7 +519,10 @@ mod tests { let uri = format!("file://{}", tmp.path().display()); let ctx = SessionContext::new(); - register(&ctx); + ctx.register_udtf( + "sd_read_zarr", + std::sync::Arc::new(ZarrReadFunction::default()), + ); // Default behaviour: 1-D coord variable auto-skipped, read succeeds. let df = ctx @@ -602,7 +570,10 @@ mod tests { #[tokio::test] async fn udtf_rejects_malformed_options_json() { let ctx = SessionContext::new(); - register(&ctx); + ctx.register_udtf( + "sd_read_zarr", + std::sync::Arc::new(ZarrReadFunction::default()), + ); let err = ctx .sql(r#"SELECT raster FROM sd_read_zarr('file:///nowhere', '{not json}')"#) From 583abdea6f299548d2d8871e4cc329547635e456 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 12:14:10 -0700 Subject: [PATCH 30/36] refactor(sedonadb): UDTF handoff via datafusion-ffi; restore mimalloc MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Rewires the cross-cdylib UDTF registration path onto datafusion-ffi's `FFI_TableFunction`, matching what `datafusion-python` does for its own plugins. Three big wins: 1. `python/sedonadb-zarr` no longer depends on `python/sedonadb` as a path rlib. The plugin links only `datafusion-catalog`, `datafusion-ffi`, `sedona-raster-zarr` and `pyo3` for its host- facing surface — no transitive sedonadb-* / pyo3 dupe of host internals. Resolves the cibuildwheel "another Python package's private crate" objection from review. 2. mimalloc returns to defaults. All cross-cdylib traffic now flows through Arrow C Data/Stream (release callbacks owned by the producing cdylib), so allocations are always freed in the cdylib that made them. The previous segfault required passing native `Arc` across cdylibs, which we no longer do. 3. Host-side codec + capsule machinery lives in `python/sedonadb/src/plugin.rs`: a `SessionTaskContextProvider` adapts our `SessionContext` to datafusion-ffi's `TaskContextProvider`, and `create_session_capsule` / `ffi_table_function_from_capsule` pack and unpack the `c"datafusion_logical_extension_codec"` and `c"datafusion_table_function"` capsules the way datafusion-python does. Plugins built against either ecosystem can register on a sedonadb context. Plugin side: `ZarrTableFunction` is the Python-visible spec class with `__datafusion_table_function__(session)`. The host calls it from `InternalContext.register_udtf_capsule(name, spec)`. --- Cargo.lock | 4 +- Cargo.toml | 1 + python/sedonadb-zarr/Cargo.toml | 15 +- .../python/sedonadb_zarr/__init__.py | 14 +- python/sedonadb-zarr/src/lib.rs | 115 ++++++++++----- python/sedonadb/Cargo.toml | 22 ++- python/sedonadb/src/context.rs | 75 ++++------ python/sedonadb/src/plugin.rs | 131 +++++++++++++----- 8 files changed, 227 insertions(+), 150 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ec4c5ce55..6ee6b8993 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6411,10 +6411,10 @@ name = "sedonadb-zarr" version = "0.4.0" dependencies = [ "arrow-array", - "datafusion", + "datafusion-catalog", + "datafusion-ffi", "pyo3", "sedona-raster-zarr", - "sedonadb", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index e4e8babf4..a596c9a5a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -94,6 +94,7 @@ datafusion-datasource-parquet = { version = "52.5.0" } datafusion-execution = { version = "52.5.0", default-features = false } datafusion-expr = { version = "52.5.0" } datafusion-ffi = { version = "52.5.0" } +datafusion-proto = { version = "52.5.0" } datafusion-optimizer = { version = "52.5.0" } datafusion-physical-expr = { version = "52.5.0" } datafusion-physical-plan = { version = "52.5.0" } diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index 2f8d01ee9..bf37e113c 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -32,13 +32,12 @@ crate-type = ["cdylib"] [dependencies] arrow-array = { workspace = true } -datafusion = { workspace = true } +# UDTF handoff to sedonadb is via `datafusion-ffi`'s `FFI_TableFunction` +# (the `datafusion-python` pattern). The plugin and host don't share +# any Rust types directly — concrete `RecordBatch` traffic flows +# through Arrow C with release callbacks owned by the producing +# cdylib, so plugin and host can use different allocators. +datafusion-catalog = { workspace = true } +datafusion-ffi = { workspace = true } pyo3 = { version = "0.25.1" } sedona-raster-zarr = { workspace = true } -# The sedonadb Python extension; its rustc crate name is `_lib` per -# the `[lib].name` override in `python/sedonadb/Cargo.toml`. We import -# it in `src/lib.rs` via the leading `::_lib` syntax to disambiguate -# from our own crate's `_lib` pymodule. Inlined here (not via -# `workspace = true`) to avoid a self-referential workspace dep that -# trips `cargo metadata` in cibuildwheel's isolated build env. -sedonadb = { path = "../sedonadb", default-features = false } diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index 53b21a15c..bd53fb304 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -40,7 +40,7 @@ from sedonadb.datasource import ExternalFormatSpec -from sedonadb_zarr._zarr_lib import PyZarrChunkReader, zarr_udtf_capsule +from sedonadb_zarr._zarr_lib import PyZarrChunkReader, ZarrTableFunction def register(con) -> None: @@ -54,11 +54,10 @@ def register(con) -> None: ---------- con A ``sedonadb`` ``Context`` (the object returned by - ``sedonadb.connect()``). Internally, this function extracts - the underlying ``InternalContext`` PyO3 handle and registers - the UDTF on its DataFusion ``SessionContext`` via a - ``PyCapsule`` handoff — the only viable cross-extension - transport for the UDTF trait object. + ``sedonadb.connect()``). Internally, this function hands a + :class:`ZarrTableFunction` instance to sedonadb's + ``register_udtf_capsule``, which uses ``datafusion-ffi``'s + ``FFI_TableFunction`` to bridge the cdylib boundary. """ internal_ctx = getattr(con, "_impl", None) if internal_ctx is None: @@ -66,8 +65,7 @@ def register(con) -> None: "sedonadb_zarr.register: could not locate the InternalContext on " f"{type(con).__name__}; expected attribute `_impl`." ) - capsule = zarr_udtf_capsule() - internal_ctx.register_udtf_capsule("sd_read_zarr", capsule) + internal_ctx.register_udtf_capsule("sd_read_zarr", ZarrTableFunction()) class ZarrFormatSpec(ExternalFormatSpec): diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index 208382fb7..6d1ecef55 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -19,51 +19,102 @@ //! `sedonadb` session. //! //! Two PyO3-exposed surfaces: -//! - `zarr_udtf_capsule()` — builds a `PyCapsule` carrying the -//! `sd_read_zarr` UDTF implementation. The Python wrapper hands it -//! to sedonadb's `InternalContext.register_udtf_capsule` to attach -//! the SQL function. -//! - `PyZarrChunkReader` — a streaming reader producible from Python, +//! - [`ZarrTableFunction`] — Python class with +//! `__datafusion_table_function__(session)` that returns an +//! `FFI_TableFunction` PyCapsule. sedonadb's +//! `InternalContext.register_udtf` attaches it under +//! `sd_read_zarr`. +//! - [`PyZarrChunkReader`] — streaming reader producible from Python, //! exposing `__arrow_c_stream__` so it plugs into //! `ExternalFormatSpec.open_reader` (the `con.read_format(spec, uri)` //! surface). //! -//! The Python `ZarrFormatSpec(ExternalFormatSpec)` class in -//! `sedonadb_zarr/__init__.py` wraps `PyZarrChunkReader`; the Rust side -//! here is intentionally thin. +//! The Rust side carries no dependency on `sedonadb`'s host extension +//! — UDTF handoff is via `datafusion-ffi`, which gives an ABI-stable +//! C-level interface so plugin and host cdylibs don't need to share +//! anything beyond Arrow C Data/Stream and the FFI structs. use std::ffi::CString; +use std::ptr::NonNull; use std::sync::{Arc, Mutex}; use arrow_array::ffi_stream::FFI_ArrowArrayStream; -use datafusion::catalog::TableFunctionImpl; +use datafusion_catalog::TableFunctionImpl; +use datafusion_ffi::proto::logical_extension_codec::FFI_LogicalExtensionCodec; +use datafusion_ffi::udtf::FFI_TableFunction; use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::PyCapsule; use sedona_raster_zarr::{ZarrChunkReader, ZarrReadFunction}; -// `sedonadb`'s rustc crate name is `_lib` (set by `[lib].name` so -// maturin packages it as `sedonadb._lib`); import the plugin handoff -// types under that name. -use _lib::plugin::{UdtfCapsule, UDTF_CAPSULE_NAME}; -/// Build a PyCapsule carrying the Zarr UDTF for handoff to sedonadb's -/// `InternalContext.register_udtf_capsule`. -/// -/// Cross-extension `#[pyclass]` extraction doesn't work in PyO3 (each -/// cdylib has its own type-id static), so we pass the UDTF -/// implementation across the extension boundary via an opaque -/// [`UdtfCapsule`]. The capsule owns its `Arc`; the consumer clones a -/// fresh refcount before the capsule is dropped. The payload's leading -/// magic sentinel lets the consumer fail fast if the capsule's name -/// has been spoofed. -#[pyfunction] -fn zarr_udtf_capsule(py: Python<'_>) -> PyResult> { - let udtf: Arc = Arc::new(ZarrReadFunction::default()); - let payload = UdtfCapsule::new(udtf); - // CString::new requires owned bytes; reuse the host-side constant - // so the name (and its `.vN` ABI tag) lives in exactly one place. - let name = CString::new(UDTF_CAPSULE_NAME.to_bytes()).unwrap(); - PyCapsule::new_with_destructor(py, payload, Some(name), |_v: UdtfCapsule, _ctx| {}) +/// `c"datafusion_table_function"` capsule name expected by sedonadb's +/// (and datafusion-python's) host-side UDTF registration. +const UDTF_CAPSULE_NAME: &std::ffi::CStr = c"datafusion_table_function"; + +/// Codec capsule name the host passes to us via the `session` argument. +const CODEC_CAPSULE_NAME: &std::ffi::CStr = c"datafusion_logical_extension_codec"; + +/// Zarr UDTF surface. The Python plugin instantiates one of these and +/// hands the instance to sedonadb's `con._impl.register_udtf` (Python +/// wrapper). Sedonadb then calls `__datafusion_table_function__(session)` +/// on it to obtain the FFI struct. +#[pyclass(name = "ZarrTableFunction", module = "sedonadb_zarr")] +#[derive(Default, Debug, Clone)] +pub struct ZarrTableFunction; + +#[pymethods] +impl ZarrTableFunction { + #[new] + fn new() -> Self { + Self + } + + fn __datafusion_table_function__<'py>( + &self, + py: Python<'py>, + session: Bound<'py, PyAny>, + ) -> PyResult> { + let codec = ffi_logical_codec_from_pycapsule(&session)?; + let udtf: Arc = Arc::new(ZarrReadFunction::default()); + let provider = FFI_TableFunction::new_with_ffi_codec(udtf, None, codec); + let name = CString::new(UDTF_CAPSULE_NAME.to_bytes()).unwrap(); + PyCapsule::new(py, provider, Some(name)) + } +} + +/// Extract an [`FFI_LogicalExtensionCodec`] from a session capsule the +/// host hands us. Accepts either a raw capsule named +/// `"datafusion_logical_extension_codec"` or any object exposing +/// `__datafusion_logical_extension_codec__()` that returns one. +fn ffi_logical_codec_from_pycapsule(obj: &Bound<'_, PyAny>) -> PyResult { + let attr_name = "__datafusion_logical_extension_codec__"; + let capsule = if obj.hasattr(attr_name)? { + obj.getattr(attr_name)?.call0()? + } else { + obj.clone() + }; + let capsule = capsule.downcast::().map_err(|e| { + PyValueError::new_err(format!( + "session capsule must be a PyCapsule (or expose {attr_name}), got {e}" + )) + })?; + let name = capsule + .name()? + .ok_or_else(|| PyValueError::new_err("session capsule has no name"))?; + if name != CODEC_CAPSULE_NAME { + return Err(PyValueError::new_err(format!( + "session capsule name mismatch: expected {CODEC_CAPSULE_NAME:?}, got {name:?}" + ))); + } + let ptr = capsule.pointer() as *mut FFI_LogicalExtensionCodec; + let ptr = NonNull::new(ptr) + .ok_or_else(|| PyValueError::new_err("session capsule pointer is null"))?; + // SAFETY: capsule name was verified above; PyO3 keeps the value + // alive for the lifetime of the capsule. `clone()` runs the FFI + // release-aware clone hook so we end up with an independent codec + // that outlives this scope. + let codec = unsafe { ptr.as_ref().clone() }; + Ok(codec) } /// Python-callable wrapper around `ZarrChunkReader` that exposes @@ -120,7 +171,7 @@ impl PyZarrChunkReader { // brings sedonadb's pymodule into our cdylib's link. #[pymodule] fn _zarr_lib(m: &Bound<'_, PyModule>) -> PyResult<()> { - m.add_function(wrap_pyfunction!(zarr_udtf_capsule, m)?)?; + m.add_class::()?; m.add_class::()?; Ok(()) } diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index 07555fbc0..f1ad07fba 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -37,20 +37,14 @@ crate-type = ["cdylib", "rlib"] # enables it explicitly via `[tool.maturin] features` in # `pyproject.toml` for the wheel build. # -# `mimalloc` is intentionally *not* in `default`: when sedonadb is -# loaded alongside a plugin cdylib (e.g. `sedonadb-zarr`) the -# global-allocator mismatch corrupts the heap. Concrete Arrow types -# like `RecordBatch` are monomorphised per-cdylib, so the drop site -# runs the consumer's allocator regardless of who allocated, and -# mimalloc-free on a libc-malloc pointer segfaults. -# -# !! WARNING: opting in with `--features mimalloc` re-introduces the -# segfault for any user who imports a plugin alongside this wheel. -# This isn't a perf knob — it's a "I'm shipping a closed binary that -# will never load a plugin cdylib" knob. The plugin-safe path is to -# link mimalloc *dynamically* (one shared `libmimalloc.so` for the -# whole process), which is a packaging change, not a feature flag. -default = [] +# `mimalloc` is back in defaults now that plugin UDTFs cross the cdylib +# boundary via `datafusion-ffi`. All `RecordBatch` traffic between +# plugin and host flows through the Arrow C Data/Stream protocol, where +# each FFI struct carries a `release` callback set by the producing +# cdylib — so allocations are always freed in the cdylib that made +# them. The previous segfault was triggered by passing native +# `Arc` across cdylibs, which we no longer do. +default = ["mimalloc"] extension-module = [] mimalloc = ["dep:mimalloc", "dep:libmimalloc-sys"] s2geography = ["sedona/s2geography"] diff --git a/python/sedonadb/src/context.rs b/python/sedonadb/src/context.rs index 2879b3de3..946e6d776 100644 --- a/python/sedonadb/src/context.rs +++ b/python/sedonadb/src/context.rs @@ -201,59 +201,38 @@ impl InternalContext { )) } - /// Register a UDTF (table function) whose implementation is passed - /// through a `PyCapsule` containing an [`UdtfCapsule`]. + /// Register a UDTF (table function) defined in a separate Python + /// extension (e.g. `sedonadb-zarr`). /// - /// Plugin-handoff API: format-specific Python packages (e.g. - /// `sedonadb-zarr`) build their UDTF in their own PyO3 extension - /// and pass it across as an opaque capsule, sidestepping the - /// cross-extension `#[pyclass]` type-id mismatch. - /// - /// Two checks gate the dereference: the capsule's name must equal - /// [`UDTF_CAPSULE_NAME`] (a public string, so this is mostly an - /// ABI tag), and the payload's leading [`UDTF_CAPSULE_MAGIC`] - /// sentinel must match (much harder to forge from outside this - /// crate). The consumer clones the inner `Arc` to take its own - /// refcount; the capsule keeps a parallel refcount until Python - /// GC drops it. - /// - /// [`UdtfCapsule`]: crate::plugin::UdtfCapsule - /// [`UDTF_CAPSULE_NAME`]: crate::plugin::UDTF_CAPSULE_NAME - /// [`UDTF_CAPSULE_MAGIC`]: crate::plugin::UDTF_CAPSULE_MAGIC + /// `spec` is the Python object exposing + /// `__datafusion_table_function__(session) -> PyCapsule`. The host + /// hands the plugin a session capsule carrying an + /// `FFI_LogicalExtensionCodec`; the plugin returns a capsule named + /// `"datafusion_table_function"` wrapping an + /// `FFI_TableFunction`. `datafusion-ffi` handles the actual + /// cross-cdylib trait-object conversion — concrete `RecordBatch` + /// allocations flow through Arrow C Stream's release callbacks, so + /// the plugin and host can use different allocators. pub fn register_udtf_capsule( &self, + py: Python<'_>, name: &str, - capsule: &Bound<'_, pyo3::types::PyCapsule>, + spec: Bound<'_, PyAny>, ) -> Result<(), PySedonaError> { - use crate::plugin::{UdtfCapsule, UDTF_CAPSULE_MAGIC, UDTF_CAPSULE_NAME}; - - let actual = capsule - .name()? - .ok_or_else(|| PySedonaError::SedonaPython("UDTF capsule has no name".to_string()))?; - if actual != UDTF_CAPSULE_NAME { - return Err(PySedonaError::SedonaPython(format!( - "UDTF capsule name mismatch: expected {UDTF_CAPSULE_NAME:?}, got {actual:?}" - ))); - } - if capsule.pointer().is_null() { - return Err(PySedonaError::SedonaPython( - "UDTF capsule pointer is null".to_string(), - )); - } - // SAFETY: the capsule's payload is an `UdtfCapsule` (name - // matched above). PyO3 keeps the value alive for the - // lifetime of the capsule and `reference` is the documented - // accessor — equivalent to `pointer().cast::()` but without - // betting on `CapsuleContents`'s private field layout. - let payload: &UdtfCapsule = unsafe { capsule.reference::() }; - if payload.magic != UDTF_CAPSULE_MAGIC { - return Err(PySedonaError::SedonaPython(format!( - "UDTF capsule magic mismatch: expected {UDTF_CAPSULE_MAGIC:#x}, got {:#x}; \ - capsule likely produced by an incompatible build", - payload.magic - ))); - } - self.inner.ctx.register_udtf(name, payload.udtf.clone()); + use pyo3::types::PyCapsule; + + let session_capsule = crate::plugin::create_session_capsule(py, &self.inner.ctx)?; + let returned = spec + .getattr(crate::plugin::UDTF_ATTR)? + .call1((session_capsule,))?; + let returned = returned.downcast::().map_err(|e| { + PySedonaError::SedonaPython(format!( + "plugin's {} must return a PyCapsule, got {e}", + crate::plugin::UDTF_ATTR + )) + })?; + let udtf = crate::plugin::ffi_table_function_from_capsule(returned)?; + self.inner.ctx.register_udtf(name, udtf); Ok(()) } } diff --git a/python/sedonadb/src/plugin.rs b/python/sedonadb/src/plugin.rs index 27f350c80..be1b24909 100644 --- a/python/sedonadb/src/plugin.rs +++ b/python/sedonadb/src/plugin.rs @@ -15,55 +15,110 @@ // specific language governing permissions and limitations // under the License. -//! Plugin handoff types shared by sedonadb and its Python plugin -//! crates (e.g. `sedonadb-zarr`). +//! Host-side plumbing for cross-extension UDTF registration. //! -//! PyO3's `#[pyclass]` extraction uses a per-cdylib type-id static, -//! so a plugin can't extract sedonadb's `InternalContext` directly. -//! UDTF implementations cross the extension boundary inside a -//! [`PyCapsule`](pyo3::types::PyCapsule) wrapping an [`UdtfCapsule`]. +//! Format-specific Python plugins (e.g. `sedonadb-zarr`) build their +//! UDTF in their own PyO3 extension and hand it across the cdylib +//! boundary as a [`datafusion_ffi::udtf::FFI_TableFunction`] inside a +//! [`PyCapsule`] — the same wire format the +//! `datafusion-python` ecosystem uses for FFI table functions. +//! +//! The plugin exposes a `__datafusion_table_function__(session)` +//! method on its Python-visible spec class. The host calls that method +//! with a session capsule carrying an [`FFI_LogicalExtensionCodec`]; +//! the plugin uses the codec to construct its `FFI_TableFunction` and +//! returns it inside a capsule named `"datafusion_table_function"`. +//! The host then converts the FFI struct into a regular +//! `Arc` (the conversion in `datafusion-ffi` +//! takes a same-library fast path or wraps a `ForeignTableFunction` +//! depending on the marker id) and registers it on the session. -use std::ffi::CStr; use std::sync::Arc; -use datafusion::catalog::TableFunctionImpl; +use datafusion::execution::context::SessionContext; +use datafusion::execution::{TaskContext, TaskContextProvider}; +use datafusion_ffi::proto::logical_extension_codec::FFI_LogicalExtensionCodec; +use datafusion_ffi::udtf::FFI_TableFunction; +use pyo3::prelude::*; +use pyo3::types::PyCapsule; +use std::ffi::CStr; +use std::ptr::NonNull; -/// Capsule name carried in [`pyo3::types::PyCapsule::name`]. -/// -/// The trailing `.v1` is an ABI tag: if the layout of [`UdtfCapsule`] -/// or the `TableFunctionImpl` vtable ever changes (e.g. a datafusion -/// major bump), bump the suffix so a pre-built plugin wheel paired -/// with a newer host fails fast at registration time instead of -/// silently corrupting on call. -pub const UDTF_CAPSULE_NAME: &CStr = c"sedonadb.udtf.v1"; +use crate::error::PySedonaError; -/// Magic sentinel at offset 0 of [`UdtfCapsule`]. -/// -/// The capsule name is a public string; any Python caller can build a -/// capsule with that name pointing at arbitrary memory. The magic -/// gives the consumer a second, harder-to-forge check before it -/// dereferences the payload as an `Arc`. -pub const UDTF_CAPSULE_MAGIC: u64 = 0x5345_444F_4E41_5544; +/// Capsule name carried in the session capsule the host passes to the +/// plugin. Matches `datafusion-python`'s convention so plugins built +/// against either ecosystem can register on a sedonadb context. +pub const CODEC_CAPSULE_NAME: &CStr = c"datafusion_logical_extension_codec"; + +/// Capsule name returned by the plugin's +/// `__datafusion_table_function__`. Matches `datafusion-python`. +pub const UDTF_CAPSULE_NAME: &CStr = c"datafusion_table_function"; -/// Payload stored inside the UDTF [`pyo3::types::PyCapsule`]. +/// Python attribute the host invokes on the plugin's spec class. +pub const UDTF_ATTR: &str = "__datafusion_table_function__"; + +/// Adapt a sedonadb [`SessionContext`] to [`TaskContextProvider`]. /// -/// `#[repr(C)]` guarantees `magic` is at offset 0 regardless of -/// alignment, so a stale or forged capsule can be rejected via a -/// single field read before touching `udtf`. The producer constructs -/// one of these; the consumer recovers it via -/// [`pyo3::types::PyCapsule::reference`]. -#[repr(C)] -pub struct UdtfCapsule { - pub magic: u64, - pub udtf: Arc, +/// The plugin's `FFI_TableFunction::call` needs a [`TaskContext`] at +/// runtime so it can deserialise the argument `Expr`s. We give it +/// access to ours via this thin wrapper — held as an `Arc` so the FFI +/// codec can clone it across the boundary. +#[derive(Debug)] +pub(crate) struct SessionTaskContextProvider { + task_ctx: Arc, } -impl UdtfCapsule { - /// Build a capsule payload around a [`TableFunctionImpl`]. - pub fn new(udtf: Arc) -> Self { +impl SessionTaskContextProvider { + pub fn new(ctx: &SessionContext) -> Self { Self { - magic: UDTF_CAPSULE_MAGIC, - udtf, + task_ctx: ctx.task_ctx(), } } } + +impl TaskContextProvider for SessionTaskContextProvider { + fn task_ctx(&self) -> Arc { + self.task_ctx.clone() + } +} + +/// Build a session capsule the host hands to the plugin. +/// +/// The capsule carries an [`FFI_LogicalExtensionCodec`] over which the +/// plugin's `FFI_TableFunction` will serialise expressions. +pub(crate) fn create_session_capsule<'py>( + py: Python<'py>, + ctx: &SessionContext, +) -> Result, PySedonaError> { + let provider: Arc = Arc::new(SessionTaskContextProvider::new(ctx)); + let codec = FFI_LogicalExtensionCodec::new_default(&provider); + PyCapsule::new(py, codec, Some(CODEC_CAPSULE_NAME.to_owned())).map_err(PySedonaError::from) +} + +/// Extract an [`FFI_TableFunction`] from the capsule returned by the +/// plugin's `__datafusion_table_function__` method, and convert it into +/// a registrable `Arc`. +pub(crate) fn ffi_table_function_from_capsule( + capsule: &Bound<'_, PyCapsule>, +) -> Result, PySedonaError> { + let name = capsule + .name()? + .ok_or_else(|| PySedonaError::SedonaPython("UDTF capsule has no name".to_string()))?; + if name != UDTF_CAPSULE_NAME { + return Err(PySedonaError::SedonaPython(format!( + "UDTF capsule name mismatch: expected {UDTF_CAPSULE_NAME:?}, got {name:?}" + ))); + } + let ptr = capsule.pointer() as *mut FFI_TableFunction; + let ptr = NonNull::new(ptr) + .ok_or_else(|| PySedonaError::SedonaPython("UDTF capsule pointer is null".to_string()))?; + // SAFETY: the capsule's payload is an `FFI_TableFunction` (name + // matched above). PyO3 keeps the value alive for the lifetime of + // the capsule; `clone()` runs the FFI release-aware clone hook so + // we end up owning an independent FFI struct, which `From` then + // unwraps into an `Arc` (same-cdylib fast + // path or `ForeignTableFunction` wrapper, depending on marker). + let ffi: FFI_TableFunction = unsafe { ptr.as_ref().clone() }; + Ok(ffi.into()) +} From 5a45265c9756e487465c647b1bd7aaa7e9aff1e1 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 12:31:13 -0700 Subject: [PATCH 31/36] fix(sedonadb): keep FFI codec's TaskContextProvider alive for session lifetime MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The `FFI_LogicalExtensionCodec` produced by `create_session_capsule` stored a `Weak`. The strong `Arc` was a local in the function, so by the time the plugin invoked `FFI_TableFunction::call` the `Weak` couldn't upgrade — DataFusion failed with `FFI error: TaskContextProvider went out of scope over FFI boundary`. Caught by running the plugin smoke tests locally against the FFI rework. Fix: stash one `Arc` on `InternalContext` at construction time. Every `register_udtf_capsule` call clones it into the session capsule. The strong ref lives as long as the session, so the FFI codec's `Weak` can always upgrade. All 7 sedonadb-zarr Python tests pass locally (including the mimalloc-on smoke test); 1699 sedonadb tests pass unchanged. --- python/sedonadb/src/context.rs | 14 +++++++++++++- python/sedonadb/src/plugin.rs | 15 +++++++++++---- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/python/sedonadb/src/context.rs b/python/sedonadb/src/context.rs index 946e6d776..607f18ed5 100644 --- a/python/sedonadb/src/context.rs +++ b/python/sedonadb/src/context.rs @@ -16,6 +16,7 @@ // under the License. use std::{collections::HashMap, sync::Arc}; +use datafusion::execution::TaskContextProvider; use datafusion_expr::ScalarUDFImpl; use pyo3::prelude::*; use sedona::context::SedonaContext; @@ -35,6 +36,14 @@ use crate::{ pub struct InternalContext { pub inner: SedonaContext, pub runtime: Arc, + /// Strong reference to the [`TaskContextProvider`] we hand across + /// the FFI boundary in [`Self::register_udtf_capsule`]. The FFI + /// codec stores a `Weak`, so dropping + /// this would cause `FFI error: TaskContextProvider went out of + /// scope` at UDTF call time. Holding it on the session keeps the + /// strong ref alive for as long as the registered UDTF can be + /// invoked. + pub udtf_task_provider: Arc, } #[pymethods] @@ -53,10 +62,13 @@ impl InternalContext { .map_err(|e| PySedonaError::SedonaPython(e.to_string()))?; let inner = wait_for_future(py, &runtime, builder.build())??; + let udtf_task_provider: Arc = + Arc::new(crate::plugin::SessionTaskContextProvider::new(&inner.ctx)); Ok(Self { inner, runtime: Arc::new(runtime), + udtf_task_provider, }) } @@ -221,7 +233,7 @@ impl InternalContext { ) -> Result<(), PySedonaError> { use pyo3::types::PyCapsule; - let session_capsule = crate::plugin::create_session_capsule(py, &self.inner.ctx)?; + let session_capsule = crate::plugin::create_session_capsule(py, &self.udtf_task_provider)?; let returned = spec .getattr(crate::plugin::UDTF_ATTR)? .call1((session_capsule,))?; diff --git a/python/sedonadb/src/plugin.rs b/python/sedonadb/src/plugin.rs index be1b24909..20f1281d1 100644 --- a/python/sedonadb/src/plugin.rs +++ b/python/sedonadb/src/plugin.rs @@ -86,13 +86,20 @@ impl TaskContextProvider for SessionTaskContextProvider { /// Build a session capsule the host hands to the plugin. /// /// The capsule carries an [`FFI_LogicalExtensionCodec`] over which the -/// plugin's `FFI_TableFunction` will serialise expressions. +/// plugin's `FFI_TableFunction` will serialise expressions. The +/// codec stores a `Weak`, so the caller +/// must keep `provider` alive for as long as the registered UDTF +/// can be invoked — we expect them to hold it on the session +/// (see `InternalContext::udtf_task_provider`). pub(crate) fn create_session_capsule<'py>( py: Python<'py>, - ctx: &SessionContext, + provider: &Arc, ) -> Result, PySedonaError> { - let provider: Arc = Arc::new(SessionTaskContextProvider::new(ctx)); - let codec = FFI_LogicalExtensionCodec::new_default(&provider); + // `FFI_TaskContextProvider` only requires `TaskContextProvider`; + // the Send + Sync bounds we carry on the session field are for + // PyO3's pyclass sync requirements. Erase them at the boundary. + let provider_erased: Arc = provider.clone(); + let codec = FFI_LogicalExtensionCodec::new_default(&provider_erased); PyCapsule::new(py, codec, Some(CODEC_CAPSULE_NAME.to_owned())).map_err(PySedonaError::from) } From e678bbc08de78d924ba981394f51b0b9453806b8 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 13:40:58 -0700 Subject: [PATCH 32/36] chore(sedonadb-zarr): pin sedonadb>=0.4.0 `register_udtf_capsule` and the `__datafusion_table_function__` contract it consumes are 0.4.0 features. Pinning prevents an install that picks up an older sedonadb at resolution time. --- python/sedonadb-zarr/pyproject.toml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/sedonadb-zarr/pyproject.toml b/python/sedonadb-zarr/pyproject.toml index 28c4b3e21..ad9267d4f 100644 --- a/python/sedonadb-zarr/pyproject.toml +++ b/python/sedonadb-zarr/pyproject.toml @@ -30,7 +30,9 @@ classifiers = [ ] dynamic = ["version"] dependencies = [ - "sedonadb", + # `register_udtf_capsule` (and the `__datafusion_table_function__` + # contract it consumes) landed in sedonadb 0.4.0. + "sedonadb>=0.4.0", ] [project.optional-dependencies] From 3289cdf5f918d35d86747c4a021481fd9ebfb724 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 13:44:33 -0700 Subject: [PATCH 33/36] chore(sedona-raster-zarr): drop Windows MinGW blosc gate MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The target-gated `[target.'cfg(...)'.dependencies]` block existed for rtools45's pthread conflict when this crate got pulled into the R sedonadb.dll build. R sedonadb has no path into `sedona-raster-zarr` in this PR — the crate is only consumed by the Python sedonadb-zarr plugin — so the gate is dead and the `blosc` feature moves to the unconditional zarrs feature list. R parity is tracked as a follow-up issue. --- rust/sedona-raster-zarr/Cargo.toml | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/rust/sedona-raster-zarr/Cargo.toml b/rust/sedona-raster-zarr/Cargo.toml index 6f9a7068f..3c33497d9 100644 --- a/rust/sedona-raster-zarr/Cargo.toml +++ b/rust/sedona-raster-zarr/Cargo.toml @@ -54,17 +54,9 @@ sedona-raster = { workspace = true } sedona-schema = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } -zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "crc32c", "sharding", "transpose"] } +zarrs = { workspace = true, features = ["filesystem", "gzip", "zstd", "blosc", "crc32c", "sharding", "transpose"] } zarrs_filesystem = { workspace = true } -# `blosc` is gated off the Windows MinGW (gnu) toolchain only: c-blosc -# (statically linked) bundles its own `pthread_create` / `pthread_cond_*` -# symbols, which conflict with rtools45's `libpthread.a` during the MinGW -# link of the R `sedonadb.dll`. The MSVC toolchain (Python wheels, etc.) -# uses a different pthread implementation and is fine. -[target.'cfg(not(all(target_os = "windows", target_env = "gnu")))'.dependencies] -zarrs = { workspace = true, features = ["blosc"] } - [dev-dependencies] # Umbrella `datafusion` only for tests — they construct a real # `SessionContext` and execute SQL. Production code is restricted to From 008a33282c1dfcc3584e36e97b4d94b4c1f1f9f4 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 13:55:27 -0700 Subject: [PATCH 34/36] test(sedonadb-zarr): parameterise smoke test over numpy dtypes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds `test_dtype_mapping_roundtrips` covering every arm of `zarr_to_band_data_type` (bool / int{8,16,32,64} / uint{8,16,32,64} / float{32,64}). Each variant writes a tiny Zarr group and reads back two chunks via `sd_read_zarr` — fast sanity check that the dtype table doesn't silently drift. Per Dewey's review note on `rust/sedona-raster-zarr/src/dtype.rs:40`. --- python/sedonadb-zarr/tests/test_zarr.py | 37 +++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/python/sedonadb-zarr/tests/test_zarr.py b/python/sedonadb-zarr/tests/test_zarr.py index 98ab0bcbf..06572c890 100644 --- a/python/sedonadb-zarr/tests/test_zarr.py +++ b/python/sedonadb-zarr/tests/test_zarr.py @@ -59,6 +59,43 @@ def test_smoke_register_enables_sql_udtf(zarr_group): assert arrow_tab.column(0)[0].as_py() == 2 +# Each numpy dtype below maps to a different `BandDataType` arm in +# `rust/sedona-raster-zarr/src/dtype.rs::zarr_to_band_data_type`. +# Reading any one is a sanity-check on the whole mapping table. +@pytest.mark.parametrize( + "numpy_dtype", + [ + "bool", + "int8", + "uint8", + "int16", + "uint16", + "int32", + "uint32", + "int64", + "uint64", + "float32", + "float64", + ], +) +def test_dtype_mapping_roundtrips(tmp_path, numpy_dtype): + zarr = pytest.importorskip("zarr") + root = zarr.open_group(str(tmp_path), mode="w") + arr = root.create_array( + "temperature", + shape=(2, 2), + chunks=(1, 2), + dtype=numpy_dtype, + dimension_names=["y", "x"], + ) + arr[:] = np.ones((2, 2), dtype=numpy_dtype) + + con = sedonadb.connect() + sedonadb_zarr.register(con) + df = con.sql(f"SELECT count(*) FROM sd_read_zarr('file://{tmp_path}')") + assert df.to_arrow_table().column(0)[0].as_py() == 2 + + def test_sql_udtf_is_not_registered_before_register_is_called(zarr_group): # Importing `sedonadb_zarr` (already done at module top) must NOT # register anything globally — registration is per-context and From ee77599f2e6720f10d052699d08c588eb56ecf9b Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 14:03:34 -0700 Subject: [PATCH 35/36] chore(sedonadb): drop extension-module feature and `_zarr_lib` naming Both existed solely to support `sedonadb-zarr` linking `sedonadb` as an rlib, which the datafusion-ffi rework eliminated. With no rlib consumer: - `extension-module` Cargo feature on `python/sedonadb` is removed. The `#[pymodule] fn _lib` is unconditional. `[lib].crate-type` is back to just `cdylib`. CI workflows and wheel-build scripts drop the `extension-module` MATURIN_PEP517_ARGS entry. - `python/sedonadb-zarr` renames the rustc crate name and pymodule function from `_zarr_lib` back to the conventional `_lib`. No `PyInit_*` symbol collision is possible now. Python import path becomes `sedonadb_zarr._lib`, matching `sedonadb._lib`. Also adds a `.gitignore` for `python/sedonadb-zarr` mirroring the sedonadb one so editable-install `.so` files don't get staged. --- .github/workflows/packaging.yml | 6 +- .github/workflows/python.yml | 6 +- ci/scripts/wheels-build-linux.sh | 2 +- ci/scripts/wheels-build-macos.sh | 2 +- ci/scripts/wheels-build-windows.ps1 | 2 +- python/sedonadb-zarr/.gitignore | 90 +++++++++++++++++++ python/sedonadb-zarr/Cargo.toml | 7 +- python/sedonadb-zarr/pyproject.toml | 5 +- .../python/sedonadb_zarr/__init__.py | 2 +- python/sedonadb-zarr/src/lib.rs | 8 +- python/sedonadb/Cargo.toml | 21 +---- python/sedonadb/pyproject.toml | 6 +- python/sedonadb/src/lib.rs | 13 --- 13 files changed, 102 insertions(+), 68 deletions(-) create mode 100644 python/sedonadb-zarr/.gitignore diff --git a/.github/workflows/packaging.yml b/.github/workflows/packaging.yml index 8012fd8ec..0d8ac7542 100644 --- a/.github/workflows/packaging.yml +++ b/.github/workflows/packaging.yml @@ -159,11 +159,7 @@ jobs: - name: Install dev SedonaDB Python env: - # `extension-module` is sedonadb's own feature gating the - # `#[pymodule] fn _lib` symbol; required for the wheel to load - # as a Python module. Plugin crates linking the rlib leave it - # off to avoid `PyInit__lib` duplication. - MATURIN_PEP517_ARGS: "--features s2geography,pyo3/extension-module,extension-module" + MATURIN_PEP517_ARGS: "--features s2geography,pyo3/extension-module" run: | pip install "python/sedonadb/[geopandas]" -v diff --git a/.github/workflows/python.yml b/.github/workflows/python.yml index 504a1a2e7..8ccb2272f 100644 --- a/.github/workflows/python.yml +++ b/.github/workflows/python.yml @@ -117,11 +117,7 @@ jobs: - name: Install run: | # Keep this export in sync with the export in dev/release/verify-release-candidate.sh - # `extension-module` is sedonadb's own feature gating the - # `#[pymodule] fn _lib` symbol — required for the wheel to - # load as a Python module. Plugin crates linking the rlib - # leave it off to avoid `PyInit__lib` duplication. - export MATURIN_PEP517_ARGS="--features s2geography,extension-module" + export MATURIN_PEP517_ARGS="--features s2geography" pip install -e "python/sedonadb/[test]" -vv # The zarr plugin package — installed so its tests in # python/sedonadb-zarr/tests/test_zarr.py can be collected by diff --git a/ci/scripts/wheels-build-linux.sh b/ci/scripts/wheels-build-linux.sh index 189552dae..d7caa8b13 100755 --- a/ci/scripts/wheels-build-linux.sh +++ b/ci/scripts/wheels-build-linux.sh @@ -48,7 +48,7 @@ BEFORE_ALL_MANYLINUX="yum install -y curl zip unzip tar clang perl" # add quite a bit of complexity but could save time if we build wheels for linux frequently. # The native and Rust builds are cached on each image such that compile work is effectively # cached between Python versions (just not between invocations of this script). -export CIBW_ENVIRONMENT_LINUX="VCPKG_ROOT=/vcpkg VCPKG_REF=$VCPKG_REF VCPKG_DEFAULT_TRIPLET=$VCPKG_DEFAULT_TRIPLET CMAKE_TOOLCHAIN_FILE=/vcpkg/scripts/buildsystems/vcpkg.cmake PKG_CONFIG_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib/pkgconfig LD_LIBRARY_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module,extension-module'" +export CIBW_ENVIRONMENT_LINUX="VCPKG_ROOT=/vcpkg VCPKG_REF=$VCPKG_REF VCPKG_DEFAULT_TRIPLET=$VCPKG_DEFAULT_TRIPLET CMAKE_TOOLCHAIN_FILE=/vcpkg/scripts/buildsystems/vcpkg.cmake PKG_CONFIG_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib/pkgconfig LD_LIBRARY_PATH=/vcpkg/installed/$VCPKG_DEFAULT_TRIPLET/lib MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module'" export CIBW_BEFORE_ALL="$BEFORE_ALL_MANYLINUX && git clone https://github.com/microsoft/vcpkg.git /vcpkg && bash {package}/../../ci/scripts/wheels-bootstrap-vcpkg.sh" pushd "${SEDONADB_DIR}" diff --git a/ci/scripts/wheels-build-macos.sh b/ci/scripts/wheels-build-macos.sh index a2e5f968f..2b767c5f7 100755 --- a/ci/scripts/wheels-build-macos.sh +++ b/ci/scripts/wheels-build-macos.sh @@ -47,7 +47,7 @@ source ./wheels-bootstrap-vcpkg.sh export CIBW_REPAIR_WHEEL_COMMAND_MACOS="DYLD_LIBRARY_PATH=$VCPKG_INSTALL_NAME_DIR delocate-listdeps {wheel} && DYLD_LIBRARY_PATH=$VCPKG_INSTALL_NAME_DIR delocate-wheel --require-archs {delocate_archs} -w {dest_dir} {wheel}" # Pass on environment variables specifically for the build -export CIBW_ENVIRONMENT_MACOS="$CIBW_ENVIRONMENT_MACOS _PYTHON_HOST_PLATFORM=macosx-12.0-${SEDONADB_MACOS_ARCH} MACOSX_DEPLOYMENT_TARGET=12.0 CMAKE_TOOLCHAIN_FILE=${CMAKE_TOOLCHAIN_FILE} MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module,extension-module'" +export CIBW_ENVIRONMENT_MACOS="$CIBW_ENVIRONMENT_MACOS _PYTHON_HOST_PLATFORM=macosx-12.0-${SEDONADB_MACOS_ARCH} MACOSX_DEPLOYMENT_TARGET=12.0 CMAKE_TOOLCHAIN_FILE=${CMAKE_TOOLCHAIN_FILE} MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module'" pushd "${SEDONADB_DIR}" python -m cibuildwheel --output-dir python/$1/dist python/$1 diff --git a/ci/scripts/wheels-build-windows.ps1 b/ci/scripts/wheels-build-windows.ps1 index da0ae78a5..b84c46a70 100644 --- a/ci/scripts/wheels-build-windows.ps1 +++ b/ci/scripts/wheels-build-windows.ps1 @@ -73,7 +73,7 @@ $env:PATH += ";$scriptDirectory\windows" $env:GEOS_LIB_DIR = "$vcpkgLibDirectory" $env:GEOS_VERSION = "3.13.0" -$env:MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module,extension-module' +$env:MATURIN_PEP517_ARGS='--features s2geography,pyo3/extension-module' # Some CMake configurations needs this separately from the toolchain file $env:CMAKE_PREFIX_PATH="$vcpkgInstalledDirectory" diff --git a/python/sedonadb-zarr/.gitignore b/python/sedonadb-zarr/.gitignore new file mode 100644 index 000000000..cf0973ab4 --- /dev/null +++ b/python/sedonadb-zarr/.gitignore @@ -0,0 +1,90 @@ +# 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. +/target + +# Byte-compiled / optimized / DLL files +__pycache__/ +.pytest_cache/ +*.py[cod] + +# C extensions +*.so + +# Distribution / packaging +.Python +.venv/ +env/ +bin/ +build/ +develop-eggs/ +dist/ +eggs/ +lib/ +lib64/ +parts/ +sdist/ +var/ +include/ +man/ +venv/ +*.egg-info/ +.installed.cfg +*.egg + +# Installer logs +pip-log.txt +pip-delete-this-directory.txt +pip-selfcheck.json + +# Unit test / coverage reports +htmlcov/ +.tox/ +.coverage +.cache +nosetests.xml +coverage.xml + +# Translations +*.mo + +# Mr Developer +.mr.developer.cfg +.project +.pydevproject + +# Rope +.ropeproject + +# Django stuff: +*.log +*.pot + +.DS_Store + +# Sphinx documentation +docs/_build/ + +# PyCharm +.idea/ + +# VSCode +.vscode/ + +# Pyenv +.python-version + +uv.lock diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index bf37e113c..5853d74a1 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -22,12 +22,7 @@ publish = false edition = "2021" [lib] -# rustc crate name. Distinct from `sedonadb`'s `_lib` to avoid the -# rustdoc output-path collision on `cargo doc --workspace`. The -# matching pymodule function in `src/lib.rs` is named `_zarr_lib` -# so its `PyInit__zarr_lib` symbol doesn't collide with sedonadb's -# `PyInit__lib` under cargo workspace feature unification. -name = "_zarr_lib" +name = "_lib" crate-type = ["cdylib"] [dependencies] diff --git a/python/sedonadb-zarr/pyproject.toml b/python/sedonadb-zarr/pyproject.toml index ad9267d4f..56c5d7175 100644 --- a/python/sedonadb-zarr/pyproject.toml +++ b/python/sedonadb-zarr/pyproject.toml @@ -44,8 +44,5 @@ test = [ [tool.maturin] features = ["pyo3/extension-module"] -# `_zarr_lib` rather than `_lib` so the generated PyInit symbol -# doesn't collide with sedonadb's `_lib` under cargo workspace -# feature unification. The Python wrapper imports from this path. -module-name = "sedonadb_zarr._zarr_lib" +module-name = "sedonadb_zarr._lib" python-source = "python" diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index bd53fb304..d808da6e3 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -40,7 +40,7 @@ from sedonadb.datasource import ExternalFormatSpec -from sedonadb_zarr._zarr_lib import PyZarrChunkReader, ZarrTableFunction +from sedonadb_zarr._lib import PyZarrChunkReader, ZarrTableFunction def register(con) -> None: diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index 6d1ecef55..28965095d 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -22,7 +22,7 @@ //! - [`ZarrTableFunction`] — Python class with //! `__datafusion_table_function__(session)` that returns an //! `FFI_TableFunction` PyCapsule. sedonadb's -//! `InternalContext.register_udtf` attaches it under +//! `InternalContext.register_udtf_capsule` attaches it under //! `sd_read_zarr`. //! - [`PyZarrChunkReader`] — streaming reader producible from Python, //! exposing `__arrow_c_stream__` so it plugs into @@ -165,12 +165,8 @@ impl PyZarrChunkReader { } } -// Named `_zarr_lib` (not `_lib`) so the generated `PyInit__zarr_lib` -// symbol doesn't collide with sedonadb's `PyInit__lib` when cargo's -// workspace feature unification (e.g. `cargo build --all-features`) -// brings sedonadb's pymodule into our cdylib's link. #[pymodule] -fn _zarr_lib(m: &Bound<'_, PyModule>) -> PyResult<()> { +fn _lib(m: &Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; Ok(()) diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index f1ad07fba..238dfe2ea 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -23,29 +23,10 @@ edition = "2021" [lib] name = "_lib" -# `cdylib` is what maturin packages into the Python wheel. `rlib` is -# what plugin crates (e.g. `python/sedonadb-zarr`) link against so they -# can extract `InternalContext` from Python objects via PyO3. -crate-type = ["cdylib", "rlib"] +crate-type = ["cdylib"] [features] -# `extension-module` gates the `#[pymodule] fn _lib` symbol so plugin -# crates (e.g. `sedonadb-zarr`) can link this as an rlib without -# duplicating `PyInit__lib` in their own cdylib. NOT a default -# feature: Cargo's workspace-wide feature unification means a default -# would force it on for everyone, including the plugin link. Maturin -# enables it explicitly via `[tool.maturin] features` in -# `pyproject.toml` for the wheel build. -# -# `mimalloc` is back in defaults now that plugin UDTFs cross the cdylib -# boundary via `datafusion-ffi`. All `RecordBatch` traffic between -# plugin and host flows through the Arrow C Data/Stream protocol, where -# each FFI struct carries a `release` callback set by the producing -# cdylib — so allocations are always freed in the cdylib that made -# them. The previous segfault was triggered by passing native -# `Arc` across cdylibs, which we no longer do. default = ["mimalloc"] -extension-module = [] mimalloc = ["dep:mimalloc", "dep:libmimalloc-sys"] s2geography = ["sedona/s2geography"] gpu = ["sedona/gpu"] diff --git a/python/sedonadb/pyproject.toml b/python/sedonadb/pyproject.toml index c093cd4fd..4705f6427 100644 --- a/python/sedonadb/pyproject.toml +++ b/python/sedonadb/pyproject.toml @@ -50,10 +50,6 @@ geopandas = [ ] [tool.maturin] -# `extension-module` is a sedonadb feature that gates the -# `#[pymodule] fn _lib` symbol. Maturin enables it here for the wheel -# build; cargo workspace test/build runs don't (so plugin crates that -# link sedonadb's rlib don't get a duplicate `PyInit__lib`). -features = ["pyo3/extension-module", "extension-module"] +features = ["pyo3/extension-module"] module-name = "sedonadb._lib" python-source = "python" diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index 72999d71b..de33a40bf 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -// When the `extension-module` feature is off (e.g. when plugin crates -// link this as an rlib for `InternalContext`), the pymodule and its -// helper `#[pyfunction]`s are gated out, leaving the imports and -// helpers "unused" from the linker's perspective. They're still -// reachable as a public Rust API surface. -#![cfg_attr(not(feature = "extension-module"), allow(dead_code, unused_imports))] - use crate::{error::PySedonaError, udf::sedona_scalar_udf}; use pyo3::{ffi::Py_uintptr_t, prelude::*}; use sedona_adbc::AdbcSedonadbDriverInit; @@ -124,12 +117,6 @@ fn gdal_version() -> Result, PySedonaError> { } } -// Gated behind `extension-module` so plugin crates (e.g. -// `sedonadb-zarr`) can take `sedonadb` as an rlib dep without -// duplicating the `PyInit__lib` symbol in their own cdylib. Plugins -// set `default-features = false`; maturin's wheel build keeps the -// default which enables the feature. -#[cfg(feature = "extension-module")] #[pymodule] fn _lib(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> { #[cfg(feature = "mimalloc")] From 8e44f572883c18761ba091bc124f04b3b5b7e7a6 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Thu, 21 May 2026 14:19:45 -0700 Subject: [PATCH 36/36] chore(sedonadb,plugin): trim dead scaffolding and over-verbose comments MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Post-FFI cleanup pass on the plugin handoff and adjacent code: - `mod context;` and `mod plugin;` are now private. They were `pub` only when `sedonadb-zarr` linked the host crate as an rlib; no external Rust consumer remains. - `python/sedonadb/src/plugin.rs`: capsule-name constants and module docstring trimmed; consts are no longer `pub`. - `python/sedonadb-zarr/src/lib.rs`: module/struct/fn docstrings collapsed to one line each. Behaviour unchanged. - `python/sedonadb-zarr/python/sedonadb_zarr/__init__.py`: docstrings converted from ReST (`:func:`, `:meth:`, NumPy `Parameters\n----`) to Markdown (mkdocs reads these as Markdown). - `python/sedonadb-zarr/README.md`: architecture line referenced a non-existent `register_udtf` PyO3 function; updated to the actual `ZarrTableFunction` / `datafusion-ffi` capsule story. - `python/sedonadb-zarr/Cargo.toml`: dropped the "why no sedonadb dep" essay; the absence speaks for itself. - `rust/sedona-datasource/src/provider.rs`: `SingleObjectExternalTable` docs and helper-fn docs collapsed. - `python/sedonadb/src/context.rs::register_udtf_capsule` docstring trimmed; mechanism docs live in `crate::plugin`. No public Rust API surface changes (the constants that lost `pub` had no Rust consumers — capsule names are part of the Python-visible ABI). --- python/sedonadb-zarr/Cargo.toml | 5 -- python/sedonadb-zarr/README.md | 2 +- .../python/sedonadb_zarr/__init__.py | 62 +++++++--------- python/sedonadb-zarr/src/lib.rs | 49 ++----------- python/sedonadb/src/context.rs | 26 ++----- python/sedonadb/src/lib.rs | 7 +- python/sedonadb/src/plugin.rs | 73 +++++-------------- rust/sedona-datasource/src/provider.rs | 50 +++---------- 8 files changed, 78 insertions(+), 196 deletions(-) diff --git a/python/sedonadb-zarr/Cargo.toml b/python/sedonadb-zarr/Cargo.toml index 5853d74a1..00a089146 100644 --- a/python/sedonadb-zarr/Cargo.toml +++ b/python/sedonadb-zarr/Cargo.toml @@ -27,11 +27,6 @@ crate-type = ["cdylib"] [dependencies] arrow-array = { workspace = true } -# UDTF handoff to sedonadb is via `datafusion-ffi`'s `FFI_TableFunction` -# (the `datafusion-python` pattern). The plugin and host don't share -# any Rust types directly — concrete `RecordBatch` traffic flows -# through Arrow C with release callbacks owned by the producing -# cdylib, so plugin and host can use different allocators. datafusion-catalog = { workspace = true } datafusion-ffi = { workspace = true } pyo3 = { version = "0.25.1" } diff --git a/python/sedonadb-zarr/README.md b/python/sedonadb-zarr/README.md index 9c8251c7b..21df08b82 100644 --- a/python/sedonadb-zarr/README.md +++ b/python/sedonadb-zarr/README.md @@ -39,6 +39,6 @@ The main `sedonadb` package does not bundle Zarr support — applications that d ## Architecture -This is a maturin-built mixed Rust/Python package. The Rust side is a thin shim around `sedona-raster-zarr` that exposes a `register_udtf` PyO3 function and a `PyZarrChunkReader` class implementing `__arrow_c_stream__`. The Python side defines `ZarrFormatSpec(ExternalFormatSpec)` and a `register(con)` helper that wires the UDTF onto a session. +A maturin-built mixed Rust/Python package. The Rust side is a thin shim around `sedona-raster-zarr` that exposes a `ZarrTableFunction` class (via `datafusion-ffi`'s `__datafusion_table_function__` capsule contract) and a `PyZarrChunkReader` class implementing `__arrow_c_stream__`. The Python side defines `ZarrFormatSpec(ExternalFormatSpec)` and a `register(con)` helper that wires the UDTF onto a session. The same plugin shape applies to future formats (`sedonadb-cog`, `sedonadb-icechunk`, …). diff --git a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py index d808da6e3..f880df32f 100644 --- a/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py +++ b/python/sedonadb-zarr/python/sedonadb_zarr/__init__.py @@ -17,22 +17,24 @@ """Zarr support for SedonaDB. -Activate by calling :func:`register` on a SedonaDB connection. After +Activate by calling `register` on a SedonaDB connection. After registration, two surfaces work: -1. ``con.sql("SELECT * FROM sd_read_zarr('s3://...')")`` — SQL UDTF. -2. ``con.read_format(ZarrFormatSpec(), uri)`` — DataFrame API via - ``ExternalFormatSpec``. +1. `con.sql("SELECT * FROM sd_read_zarr('s3://...')")` — SQL UDTF. +2. `con.read_format(ZarrFormatSpec(), uri)` — DataFrame API via + `ExternalFormatSpec`. ->>> import sedonadb ->>> import sedonadb_zarr ->>> con = sedonadb.connect() ->>> sedonadb_zarr.register(con) ->>> con.sql("SELECT count(*) FROM sd_read_zarr('file:///path/to/foo.zarr')").show() # doctest: +SKIP +```python +import sedonadb +import sedonadb_zarr -The plugin is opt-in: SedonaDB itself does not bundle Zarr support, so -applications that don't import ``sedonadb_zarr`` pay no zarr build or -runtime cost. +con = sedonadb.connect() +sedonadb_zarr.register(con) +con.sql("SELECT count(*) FROM sd_read_zarr('file:///path/to/foo.zarr')").show() +``` + +Importing `sedonadb_zarr` is opt-in — applications that don't import +it pay no runtime cost. """ import json @@ -46,18 +48,11 @@ def register(con) -> None: """Attach Zarr SQL support to a SedonaDB connection. - After this call, ``con.sql("SELECT * FROM sd_read_zarr(...)")`` - works. Idempotent — calling twice on the same connection re- - registers the UDTF without error. - - Parameters - ---------- - con - A ``sedonadb`` ``Context`` (the object returned by - ``sedonadb.connect()``). Internally, this function hands a - :class:`ZarrTableFunction` instance to sedonadb's - ``register_udtf_capsule``, which uses ``datafusion-ffi``'s - ``FFI_TableFunction`` to bridge the cdylib boundary. + After this call, `con.sql("SELECT * FROM sd_read_zarr(...)")` works. + Idempotent: calling twice re-registers the UDTF without error. + + Args: + con: A `sedonadb` connection (returned by `sedonadb.connect()`). """ internal_ctx = getattr(con, "_impl", None) if internal_ctx is None: @@ -71,14 +66,16 @@ def register(con) -> None: class ZarrFormatSpec(ExternalFormatSpec): """`ExternalFormatSpec` for Zarr groups. - Use with ``con.read_format(spec, uri)``: + Use with `con.read_format(spec, uri)`: - >>> con.read_format(ZarrFormatSpec(), "file:///path/to/foo.zarr") # doctest: +SKIP + ```python + con.read_format(ZarrFormatSpec(), "file:///path/to/foo.zarr") + ``` - Supported options (via :meth:`with_options`): + Supported `with_options` keys: - - ``arrays`` (``list[str]`` or JSON-string) — explicit subset of - group arrays to read. + - `arrays` (`list[str]` or JSON-string) — explicit subset of group + arrays to read. """ def __init__(self, options: Optional[Mapping[str, Any]] = None): @@ -90,11 +87,8 @@ def extension(self) -> str: @property def list_single_object(self) -> bool: - # A Zarr group is a directory, not a file. The DataFusion - # listing layer would enumerate its contents (zarr.json, chunk - # shards, ...), none of which carry the `.zarr` extension. The - # Rust `SingleObjectExternalTable` path skips listing and - # hands the URI straight to `open_reader`. + # Zarr groups are directories; the DataFusion listing layer + # returns zero objects at a `.zarr` prefix. return True def with_options(self, options: Mapping[str, Any]) -> "ZarrFormatSpec": diff --git a/python/sedonadb-zarr/src/lib.rs b/python/sedonadb-zarr/src/lib.rs index 28965095d..d6d74cef9 100644 --- a/python/sedonadb-zarr/src/lib.rs +++ b/python/sedonadb-zarr/src/lib.rs @@ -15,24 +15,9 @@ // specific language governing permissions and limitations // under the License. -//! `sedonadb-zarr` — Python plugin package wiring Zarr support into a -//! `sedonadb` session. -//! -//! Two PyO3-exposed surfaces: -//! - [`ZarrTableFunction`] — Python class with -//! `__datafusion_table_function__(session)` that returns an -//! `FFI_TableFunction` PyCapsule. sedonadb's -//! `InternalContext.register_udtf_capsule` attaches it under -//! `sd_read_zarr`. -//! - [`PyZarrChunkReader`] — streaming reader producible from Python, -//! exposing `__arrow_c_stream__` so it plugs into -//! `ExternalFormatSpec.open_reader` (the `con.read_format(spec, uri)` -//! surface). -//! -//! The Rust side carries no dependency on `sedonadb`'s host extension -//! — UDTF handoff is via `datafusion-ffi`, which gives an ABI-stable -//! C-level interface so plugin and host cdylibs don't need to share -//! anything beyond Arrow C Data/Stream and the FFI structs. +//! `sedonadb-zarr` — Python plugin wiring Zarr support into a +//! `sedonadb` session via the `datafusion-ffi` UDTF capsule contract +//! and the Arrow C Stream protocol. use std::ffi::CString; use std::ptr::NonNull; @@ -47,17 +32,9 @@ use pyo3::prelude::*; use pyo3::types::PyCapsule; use sedona_raster_zarr::{ZarrChunkReader, ZarrReadFunction}; -/// `c"datafusion_table_function"` capsule name expected by sedonadb's -/// (and datafusion-python's) host-side UDTF registration. const UDTF_CAPSULE_NAME: &std::ffi::CStr = c"datafusion_table_function"; - -/// Codec capsule name the host passes to us via the `session` argument. const CODEC_CAPSULE_NAME: &std::ffi::CStr = c"datafusion_logical_extension_codec"; -/// Zarr UDTF surface. The Python plugin instantiates one of these and -/// hands the instance to sedonadb's `con._impl.register_udtf` (Python -/// wrapper). Sedonadb then calls `__datafusion_table_function__(session)` -/// on it to obtain the FFI struct. #[pyclass(name = "ZarrTableFunction", module = "sedonadb_zarr")] #[derive(Default, Debug, Clone)] pub struct ZarrTableFunction; @@ -82,10 +59,9 @@ impl ZarrTableFunction { } } -/// Extract an [`FFI_LogicalExtensionCodec`] from a session capsule the -/// host hands us. Accepts either a raw capsule named -/// `"datafusion_logical_extension_codec"` or any object exposing -/// `__datafusion_logical_extension_codec__()` that returns one. +/// Accept either a raw codec capsule or an object exposing +/// `__datafusion_logical_extension_codec__()` (the datafusion-python +/// contract). fn ffi_logical_codec_from_pycapsule(obj: &Bound<'_, PyAny>) -> PyResult { let attr_name = "__datafusion_logical_extension_codec__"; let capsule = if obj.hasattr(attr_name)? { @@ -109,21 +85,12 @@ fn ffi_logical_codec_from_pycapsule(obj: &Bound<'_, PyAny>) -> PyResult>, diff --git a/python/sedonadb/src/context.rs b/python/sedonadb/src/context.rs index 607f18ed5..d0acb2c8d 100644 --- a/python/sedonadb/src/context.rs +++ b/python/sedonadb/src/context.rs @@ -36,13 +36,9 @@ use crate::{ pub struct InternalContext { pub inner: SedonaContext, pub runtime: Arc, - /// Strong reference to the [`TaskContextProvider`] we hand across - /// the FFI boundary in [`Self::register_udtf_capsule`]. The FFI - /// codec stores a `Weak`, so dropping - /// this would cause `FFI error: TaskContextProvider went out of - /// scope` at UDTF call time. Holding it on the session keeps the - /// strong ref alive for as long as the registered UDTF can be - /// invoked. + /// The FFI codec the plugin's `FFI_TableFunction` holds stores + /// only a `Weak` to its `TaskContextProvider`; keep the strong + /// `Arc` here so registered UDTFs can always upgrade. pub udtf_task_provider: Arc, } @@ -213,18 +209,10 @@ impl InternalContext { )) } - /// Register a UDTF (table function) defined in a separate Python - /// extension (e.g. `sedonadb-zarr`). - /// - /// `spec` is the Python object exposing - /// `__datafusion_table_function__(session) -> PyCapsule`. The host - /// hands the plugin a session capsule carrying an - /// `FFI_LogicalExtensionCodec`; the plugin returns a capsule named - /// `"datafusion_table_function"` wrapping an - /// `FFI_TableFunction`. `datafusion-ffi` handles the actual - /// cross-cdylib trait-object conversion — concrete `RecordBatch` - /// allocations flow through Arrow C Stream's release callbacks, so - /// the plugin and host can use different allocators. + /// Register a UDTF defined in a separate Python extension + /// (e.g. `sedonadb-zarr`). `spec` must expose + /// `__datafusion_table_function__(session)`; see + /// [`crate::plugin`] for the capsule contract. pub fn register_udtf_capsule( &self, py: Python<'_>, diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index de33a40bf..cb1cb2042 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -22,16 +22,13 @@ use sedona_gdal::global::{configure_global_gdal_api, with_global_gdal, GdalApiBu use sedona_proj::register::{configure_global_proj_engine, ProjCrsEngineBuilder}; use std::ffi::c_void; -// `context` is `pub` so plugin crates (e.g. `python/sedonadb-zarr`) -// linking the rlib can extract `InternalContext` from Python objects. -// The other modules stay private — they're internal-detail. -pub mod context; +mod context; mod dataframe; mod datasource; mod error; mod expr; mod import_from; -pub mod plugin; +mod plugin; mod reader; mod runtime; mod schema; diff --git a/python/sedonadb/src/plugin.rs b/python/sedonadb/src/plugin.rs index 20f1281d1..e873d59e9 100644 --- a/python/sedonadb/src/plugin.rs +++ b/python/sedonadb/src/plugin.rs @@ -15,23 +15,14 @@ // specific language governing permissions and limitations // under the License. -//! Host-side plumbing for cross-extension UDTF registration. +//! Host-side plumbing for cross-extension UDTF registration via +//! `datafusion-ffi` — the same wire format `datafusion-python` uses. //! -//! Format-specific Python plugins (e.g. `sedonadb-zarr`) build their -//! UDTF in their own PyO3 extension and hand it across the cdylib -//! boundary as a [`datafusion_ffi::udtf::FFI_TableFunction`] inside a -//! [`PyCapsule`] — the same wire format the -//! `datafusion-python` ecosystem uses for FFI table functions. -//! -//! The plugin exposes a `__datafusion_table_function__(session)` -//! method on its Python-visible spec class. The host calls that method -//! with a session capsule carrying an [`FFI_LogicalExtensionCodec`]; -//! the plugin uses the codec to construct its `FFI_TableFunction` and -//! returns it inside a capsule named `"datafusion_table_function"`. -//! The host then converts the FFI struct into a regular -//! `Arc` (the conversion in `datafusion-ffi` -//! takes a same-library fast path or wraps a `ForeignTableFunction` -//! depending on the marker id) and registers it on the session. +//! The host calls the plugin's `__datafusion_table_function__(session)` +//! with a session capsule carrying an [`FFI_LogicalExtensionCodec`]. +//! The plugin returns a capsule named `"datafusion_table_function"` +//! wrapping an [`FFI_TableFunction`], which the host converts to an +//! `Arc` and registers on the session. use std::sync::Arc; @@ -46,24 +37,11 @@ use std::ptr::NonNull; use crate::error::PySedonaError; -/// Capsule name carried in the session capsule the host passes to the -/// plugin. Matches `datafusion-python`'s convention so plugins built -/// against either ecosystem can register on a sedonadb context. -pub const CODEC_CAPSULE_NAME: &CStr = c"datafusion_logical_extension_codec"; - -/// Capsule name returned by the plugin's -/// `__datafusion_table_function__`. Matches `datafusion-python`. -pub const UDTF_CAPSULE_NAME: &CStr = c"datafusion_table_function"; +const CODEC_CAPSULE_NAME: &CStr = c"datafusion_logical_extension_codec"; +const UDTF_CAPSULE_NAME: &CStr = c"datafusion_table_function"; -/// Python attribute the host invokes on the plugin's spec class. -pub const UDTF_ATTR: &str = "__datafusion_table_function__"; +pub(crate) const UDTF_ATTR: &str = "__datafusion_table_function__"; -/// Adapt a sedonadb [`SessionContext`] to [`TaskContextProvider`]. -/// -/// The plugin's `FFI_TableFunction::call` needs a [`TaskContext`] at -/// runtime so it can deserialise the argument `Expr`s. We give it -/// access to ours via this thin wrapper — held as an `Arc` so the FFI -/// codec can clone it across the boundary. #[derive(Debug)] pub(crate) struct SessionTaskContextProvider { task_ctx: Arc, @@ -83,29 +61,22 @@ impl TaskContextProvider for SessionTaskContextProvider { } } -/// Build a session capsule the host hands to the plugin. +/// Build the session capsule the host hands to the plugin. /// -/// The capsule carries an [`FFI_LogicalExtensionCodec`] over which the -/// plugin's `FFI_TableFunction` will serialise expressions. The -/// codec stores a `Weak`, so the caller -/// must keep `provider` alive for as long as the registered UDTF -/// can be invoked — we expect them to hold it on the session -/// (see `InternalContext::udtf_task_provider`). +/// The codec holds a `Weak`, so `provider` +/// must outlive any UDTF registered through this capsule — see +/// `InternalContext::udtf_task_provider`. pub(crate) fn create_session_capsule<'py>( py: Python<'py>, provider: &Arc, ) -> Result, PySedonaError> { - // `FFI_TaskContextProvider` only requires `TaskContextProvider`; - // the Send + Sync bounds we carry on the session field are for - // PyO3's pyclass sync requirements. Erase them at the boundary. - let provider_erased: Arc = provider.clone(); - let codec = FFI_LogicalExtensionCodec::new_default(&provider_erased); + // FFI_TaskContextProvider's `From` only needs TaskContextProvider; + // drop the Send + Sync (carried for pyclass) at the boundary. + let provider: Arc = provider.clone(); + let codec = FFI_LogicalExtensionCodec::new_default(&provider); PyCapsule::new(py, codec, Some(CODEC_CAPSULE_NAME.to_owned())).map_err(PySedonaError::from) } -/// Extract an [`FFI_TableFunction`] from the capsule returned by the -/// plugin's `__datafusion_table_function__` method, and convert it into -/// a registrable `Arc`. pub(crate) fn ffi_table_function_from_capsule( capsule: &Bound<'_, PyCapsule>, ) -> Result, PySedonaError> { @@ -120,12 +91,8 @@ pub(crate) fn ffi_table_function_from_capsule( let ptr = capsule.pointer() as *mut FFI_TableFunction; let ptr = NonNull::new(ptr) .ok_or_else(|| PySedonaError::SedonaPython("UDTF capsule pointer is null".to_string()))?; - // SAFETY: the capsule's payload is an `FFI_TableFunction` (name - // matched above). PyO3 keeps the value alive for the lifetime of - // the capsule; `clone()` runs the FFI release-aware clone hook so - // we end up owning an independent FFI struct, which `From` then - // unwraps into an `Arc` (same-cdylib fast - // path or `ForeignTableFunction` wrapper, depending on marker). + // SAFETY: name-matched above; clone() runs the FFI release-aware + // hook so we own an independent struct that `From` unwraps. let ffi: FFI_TableFunction = unsafe { ptr.as_ref().clone() }; Ok(ffi.into()) } diff --git a/rust/sedona-datasource/src/provider.rs b/rust/sedona-datasource/src/provider.rs index b96b02e28..7ee52b5b5 100644 --- a/rust/sedona-datasource/src/provider.rs +++ b/rust/sedona-datasource/src/provider.rs @@ -150,28 +150,17 @@ impl ReadOptions<'_> for RecordBatchReaderTableOptions { } } -/// [`TableProvider`] that treats each input URI as one opaque object. -/// -/// Built when [`ExternalFormatSpec::list_single_object`] is `true`. The -/// listing layer is bypassed entirely: each URI is synthesised into a -/// single-element [`PartitionedFile`] whose `object_meta.location` is -/// the URL's path within the [`ObjectStoreUrl`]. The format's -/// [`ExternalFileFormat::file_source`] then drives the same scan path -/// used by [`ListingTable`] — projections, filter pushdown, and -/// streaming behaviour are identical. -/// -/// Required for directory-shaped formats like Zarr where the -/// "object" is the directory itself. -/// -/// Each URI lands in its own [`FileGroup`], so a multi-URI scan can -/// produce as many partitions as URIs given enough target partitions. +/// [`TableProvider`] that treats each input URI as one opaque object, +/// bypassing the listing layer. Built when +/// [`ExternalFormatSpec::list_single_object`] is `true` — required for +/// directory-shaped formats like Zarr. Each URI lands in its own +/// [`FileGroup`] so multi-URI scans can fan out across partitions. #[derive(Debug)] pub struct SingleObjectExternalTable { spec: Arc, schema: SchemaRef, - /// Each input URI as (scheme-level object store URL, path within store). - /// All entries must share the same object store URL — mixed schemes - /// (e.g. `file://` + `s3://`) are rejected. + /// `(scheme-level store URL, path within store)`. Mixed-store + /// inputs are rejected in `try_new`. files: Vec<(ObjectStoreUrl, ObjectPath)>, } @@ -181,10 +170,6 @@ impl SingleObjectExternalTable { context: &SessionContext, table_paths: Vec, ) -> Result { - // All URIs must resolve to the same object store. Mixing - // schemes (one file://, one s3://) would force the scan to - // dispatch to multiple stores — not supported here. Print - // the original URIs so the user sees what they typed. let first_store = table_paths[0].object_store(); for path in table_paths.iter().skip(1) { let store = path.object_store(); @@ -204,11 +189,6 @@ impl SingleObjectExternalTable { .map(|p| (p.object_store(), p.prefix().clone())) .collect(); - // Resolve the ObjectStore from the session's runtime registry, - // mirroring what `ListingTable` does internally. Specs whose - // `infer_schema` peeks at the store (statting metadata, reading - // a header) would otherwise get `None` here only on this path — - // a silent asymmetry with the listing branch. let store = context.runtime_env().object_store(&first_store)?; let probe = Object { store: Some(store), @@ -253,10 +233,8 @@ impl TableProvider for SingleObjectExternalTable { let format = ExternalFileFormat::new(self.spec.clone()); let file_source = format.file_source(table_schema); - // One FileGroup per URI so DataFusion can fan out across - // partitions. Lumping every URI into a single FileGroup would - // pin the scan to one partition irrespective of - // `target_partitions`. + // One FileGroup per URI so multi-URI scans can fan out across + // partitions instead of being pinned to one. let file_groups: Vec = self .files .iter() @@ -283,13 +261,9 @@ impl TableProvider for SingleObjectExternalTable { } } -/// Synthesise an [`ObjectMeta`] for a URI we haven't `head`'d. -/// -/// `size` is set to `u64::MAX` rather than `0`: a zero size signals -/// "empty file, skip me" to some DataFusion optimisations (file -/// pruning, size-aware repartitioning), and we never want directory -/// objects to be skipped. `last_modified` defaults to the unix epoch -/// because this provider never compares timestamps. +/// Synthesise an [`ObjectMeta`] without statting. `size = u64::MAX` +/// rather than `0` so DataFusion's size-aware pruning doesn't treat +/// the entry as an empty file and skip it. fn synthetic_object_meta(location: &ObjectPath) -> ObjectMeta { ObjectMeta { location: location.clone(),