diff --git a/.github/workflows/extended.yml b/.github/workflows/extended.yml index 8ceffb1e2077c..a143cb49fd35b 100644 --- a/.github/workflows/extended.yml +++ b/.github/workflows/extended.yml @@ -63,7 +63,7 @@ jobs: runs-on: ${{ vars.USE_RUNS_ON == 'true' && format('runs-on={0},family=m8a+m7a+c8a,cpu=32,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion', github.run_id) || 'ubuntu-latest' }} # note: do not use amd/rust container to preserve disk space steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: ref: ${{ github.event.inputs.pr_head_sha }} # will be empty if triggered by push @@ -110,7 +110,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: ref: ${{ github.event.inputs.pr_head_sha }} # will be empty if triggered by push @@ -132,7 +132,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: ref: ${{ github.event.inputs.pr_head_sha }} # will be empty if triggered by push diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 4e88d0aadf825..3dcdf165907b1 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -50,7 +50,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 - name: Setup Rust toolchain uses: ./.github/actions/setup-builder @@ -141,7 +141,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 - name: Setup Rust toolchain uses: ./.github/actions/setup-builder @@ -173,7 +173,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 - name: Setup Rust toolchain uses: ./.github/actions/setup-builder @@ -276,7 +276,7 @@ jobs: volumes: - /usr/local:/host/usr/local steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -322,7 +322,7 @@ jobs: needs: linux-build-lib runs-on: ${{ vars.USE_RUNS_ON == 'true' && format('runs-on={0},family=m8a+m7a+c8a,cpu=16,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion', github.run_id) || 'ubuntu-latest' }} steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -354,7 +354,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -385,7 +385,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -407,7 +407,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 - name: Setup Rust toolchain uses: ./.github/actions/setup-builder @@ -448,7 +448,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -496,7 +496,7 @@ jobs: --health-timeout 5s --health-retries 5 steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -521,7 +521,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -657,7 +657,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true @@ -704,7 +704,7 @@ jobs: container: image: amd64/rust steps: - - uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0 + - uses: runs-on/action@d141ef83eb66d096ce8afc767e09115a65c63b60 # v2.1.2 - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: submodules: true diff --git a/Cargo.lock b/Cargo.lock index 40d920459719c..1e5d650c4615a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -156,8 +156,7 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "378530e55cd479eda3c14eb345310799717e6f76d0c332041e8487022166b471" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-arith", "arrow-array", @@ -179,8 +178,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0ab212d2c1886e802f51c5212d78ebbcbb0bec980fff9dadc1eb8d45cd0b738" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -193,8 +191,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfd33d3e92f207444098c75b42de99d329562be0cf686b307b097cc52b4e999e" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "ahash", "arrow-buffer", @@ -236,8 +233,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6cd424c2693bcdbc150d843dc9d4d137dd2de4782ce6df491ad11a3a0416c0" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "bytes", "half", @@ -248,8 +244,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c5aefb56a2c02e9e2b30746241058b85f8983f0fcff2ba0c6d09006e1cded7f" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -270,8 +265,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94e8cf7e517657a52b91ea1263acf38c4ca62a84655d72458a3359b12ab97de" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-cast", @@ -285,8 +279,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c88210023a2bfee1896af366309a3028fc3bcbd6515fa29a7990ee1baa08ee0" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-buffer", "arrow-schema", @@ -298,8 +291,7 @@ dependencies = [ [[package]] name = "arrow-flight" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28abfe8bf9f124e5fc83b334af4fa58f8d0323ad25312ccb2d1da50178415704" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-arith", "arrow-array", @@ -326,8 +318,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "238438f0834483703d88896db6fe5a7138b2230debc31b34c0336c2996e3c64f" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -342,8 +333,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "205ca2119e6d679d5c133c6f30e68f027738d95ed948cf77677ea69c7800036b" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -367,8 +357,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bffd8fd2579286a5d63bac898159873e5094a79009940bcb42bbfce4f19f1d0" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -380,8 +369,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bab5994731204603c73ba69267616c50f80780774c6bb0476f1f830625115e0c" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -393,8 +381,7 @@ dependencies = [ [[package]] name = "arrow-schema" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f633dbfdf39c039ada1bf9e34c694816eb71fbb7dc78f613993b7245e078a1ed" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "bitflags", "serde", @@ -405,8 +392,7 @@ dependencies = [ [[package]] name = "arrow-select" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cd065c54172ac787cf3f2f8d4107e0d3fdc26edba76fdf4f4cc170258942222" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "ahash", "arrow-array", @@ -419,8 +405,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29dd7cda3ab9692f43a2e4acc444d760cc17b12bb6d8232ddf64e9bab7c06b42" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -2136,6 +2121,7 @@ dependencies = [ "arrow-schema", "async-ffi", "async-trait", + "chrono", "datafusion", "datafusion-catalog", "datafusion-common", @@ -3603,9 +3589,9 @@ dependencies = [ [[package]] name = "idna_adapter" -version = "1.2.1" +version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3acae9609540aa318d1bc588455225fb2085b9ed0c4f6bd0d9d5bcd86f1a0344" +checksum = "cb68373c0d6620ef8105e855e7745e18b0d00d3bdb07fb532e434244cdb9a714" dependencies = [ "icu_normalizer", "icu_properties", @@ -4352,8 +4338,7 @@ dependencies = [ [[package]] name = "parquet" version = "58.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dafa7d01085b62a47dd0c1829550a0a36710ea9c4fe358a05a85477cec8a908" +source = "git+https://github.com/pydantic/arrow-rs.git?branch=adaptive-strategy-swap#b75d300b660e732062fc2462ef97fd5617ce7f3c" dependencies = [ "ahash", "arrow-array", @@ -4533,18 +4518,18 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.12" +version = "1.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbf0d9e68100b3a7989b4901972f265cd542e560a3a8a724e1e20322f4d06ce9" +checksum = "f1749c7ed4bcaf4c3d0a3efc28538844fb29bcdd7d2b67b2be7e20ba861ff517" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.12" +version = "1.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a990e22f43e84855daf260dded30524ef4a9021cc7541c26540500a50b624389" +checksum = "d9b20ed30f105399776b9c883e68e536ef602a16ae6f596d2c473591d6ad64c6" dependencies = [ "proc-macro2", "quote", @@ -4740,7 +4725,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "343d3bd7056eda839b03204e68deff7d1b13aba7af2b2fd16890697274262ee7" dependencies = [ "heck", - "itertools 0.14.0", + "itertools 0.13.0", "log", "multimap", "petgraph", @@ -4759,7 +4744,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "27c6023962132f4b30eb4c172c91ce92d933da334c59c23cddee82358ddafb0b" dependencies = [ "anyhow", - "itertools 0.14.0", + "itertools 0.13.0", "proc-macro2", "quote", "syn 2.0.117", @@ -6149,9 +6134,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.52.3" +version = "1.52.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fc7f01b389ac15039e4dc9531aa973a135d7a4135281b12d7c1bc79fd57fffe" +checksum = "b67dee974fe86fd92cc45b7a95fdd2f99a36a6d7b0d431a231178d3d670bbcc6" dependencies = [ "bytes", "libc", @@ -6289,9 +6274,9 @@ checksum = "756daf9b1013ebe47a8776667b466417e2d4c5679d441c26230efd9ef78692db" [[package]] name = "tonic" -version = "0.14.6" +version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac2a5518c70fa84342385732db33fb3f44bc4cc748936eb5833d2df34d6445ef" +checksum = "fec7c61a0695dc1887c1b53952990f3ad2e3a31453e1f49f10e75424943a93ec" dependencies = [ "async-trait", "axum", @@ -6884,9 +6869,9 @@ dependencies = [ [[package]] name = "whoami" -version = "2.1.1" +version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6a5b12f9df4f978d2cfdb1bd3bac52433f44393342d7ee9c25f5a1c14c0f45d" +checksum = "998767ef88740d1f5b0682a9c53c24431453923962269c2db68ee43788c5a40d" dependencies = [ "libc", "libredox", diff --git a/Cargo.toml b/Cargo.toml index 78c271d524fb8..428b1cfa1f5d1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -204,6 +204,34 @@ url = "2.5.7" uuid = "1.23" zstd = { version = "0.13", default-features = false } +# Override arrow / parquet to the `adaptive-strategy-swap` branch on +# pydantic's fork of arrow-rs, which adds the `swap_strategy` API on +# `ParquetPushDecoder` that the in-decoder adaptive filter scheduling +# depends on. +# +# The full set of arrow-rs workspace crates is listed so transitive +# deps (e.g. `arrow-cast` pulled in via `arrow`) resolve to the patched +# version and we don't link two copies into one binary. +# +# Branch: https://github.com/pydantic/arrow-rs/tree/adaptive-strategy-swap +[patch.crates-io] +arrow = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-arith = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-array = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-buffer = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-cast = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-csv = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-data = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-flight = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-ipc = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-json = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-ord = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-row = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-schema = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-select = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +arrow-string = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } +parquet = { git = "https://github.com/pydantic/arrow-rs.git", branch = "adaptive-strategy-swap" } + [workspace.lints.clippy] # Detects large stack-allocated futures that may cause stack overflow crashes (see threshold in clippy.toml) large_futures = "warn" diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 6bfe1160ecdd6..9cf340f88a285 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -616,9 +616,9 @@ mod tests { +-----------------------------------+-----------------+---------------------+------+------------------+ | filename | file_size_bytes | metadata_size_bytes | hits | extra | +-----------------------------------+-----------------+---------------------+------+------------------+ - | alltypes_plain.parquet | 1851 | 8882 | 2 | page_index=false | - | alltypes_tiny_pages.parquet | 454233 | 269074 | 2 | page_index=true | - | lz4_raw_compressed_larger.parquet | 380836 | 1339 | 2 | page_index=false | + | alltypes_plain.parquet | 1851 | 8794 | 2 | page_index=false | + | alltypes_tiny_pages.parquet | 454233 | 268970 | 2 | page_index=true | + | lz4_raw_compressed_larger.parquet | 380836 | 1331 | 2 | page_index=false | +-----------------------------------+-----------------+---------------------+------+------------------+ "); @@ -647,9 +647,9 @@ mod tests { +-----------------------------------+-----------------+---------------------+------+------------------+ | filename | file_size_bytes | metadata_size_bytes | hits | extra | +-----------------------------------+-----------------+---------------------+------+------------------+ - | alltypes_plain.parquet | 1851 | 8882 | 5 | page_index=false | - | alltypes_tiny_pages.parquet | 454233 | 269074 | 2 | page_index=true | - | lz4_raw_compressed_larger.parquet | 380836 | 1339 | 3 | page_index=false | + | alltypes_plain.parquet | 1851 | 8794 | 5 | page_index=false | + | alltypes_tiny_pages.parquet | 454233 | 268970 | 2 | page_index=true | + | lz4_raw_compressed_larger.parquet | 380836 | 1331 | 3 | page_index=false | +-----------------------------------+-----------------+---------------------+------+------------------+ "); diff --git a/datafusion-examples/examples/data_io/json_shredding.rs b/datafusion-examples/examples/data_io/json_shredding.rs index 72fbb56773123..3bef09cb07771 100644 --- a/datafusion-examples/examples/data_io/json_shredding.rs +++ b/datafusion-examples/examples/data_io/json_shredding.rs @@ -92,6 +92,16 @@ pub async fn json_shredding() -> Result<()> { // Set up query execution let mut cfg = SessionConfig::new(); cfg.options_mut().execution.parquet.pushdown_filters = true; + // Force every filter to row-level so the example's + // `pushdown_rows_pruned=1` assertion is deterministic. The default + // adaptive scheduler keeps small-file filters on the post-scan path + // (via the byte-ratio heuristic), where `pushdown_rows_pruned` stays + // 0; setting `filter_pushdown_min_bytes_per_sec = 0` disables that + // heuristic. + cfg.options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; let ctx = SessionContext::new_with_config(cfg); ctx.runtime_env().register_object_store( ObjectStoreUrl::parse("memory://")?.as_ref(), diff --git a/datafusion/catalog-listing/src/helpers.rs b/datafusion/catalog-listing/src/helpers.rs index c6305c30008ce..2abe2a49a947a 100644 --- a/datafusion/catalog-listing/src/helpers.rs +++ b/datafusion/catalog-listing/src/helpers.rs @@ -267,14 +267,15 @@ fn populate_partition_values<'a>( match op { Operator::Eq => match (left.as_ref(), right.as_ref()) { (Expr::Column(Column { name, .. }), Expr::Literal(val, _)) - | (Expr::Literal(val, _), Expr::Column(Column { name, .. })) => { + | (Expr::Literal(val, _), Expr::Column(Column { name, .. })) if partition_values .insert(name, PartitionValue::Single(val.to_string())) - .is_some() - { - partition_values.insert(name, PartitionValue::Multi); - } + .is_some() => + { + partition_values.insert(name, PartitionValue::Multi); } + (Expr::Column(Column { .. }), Expr::Literal(_, _)) + | (Expr::Literal(_, _), Expr::Column(Column { .. })) => {} _ => {} }, Operator::And => { diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 7d32f2a88fd9c..d39848c5cabe0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -919,6 +919,29 @@ config_namespace! { /// parquet reader setting. 0 means no caching. pub max_predicate_cache_size: Option, default = None + /// (reading) Minimum throughput, in bytes per second, that an adaptive + /// row-level filter must sustain to remain at row-level. Filters that + /// drop below this threshold (with statistical confidence — see + /// `filter_confidence_z`) are demoted to post-scan, or dropped entirely + /// if they were optional (e.g. a hash-join build-side dynamic filter). + /// Set to `0` to force every filter to row-level (skip the threshold + /// check); set to `f64::INFINITY` to keep every filter post-scan. + pub filter_pushdown_min_bytes_per_sec: f64, default = 100.0 * 1024.0 * 1024.0 + + /// (reading) Initial-placement heuristic for adaptive filters: when a + /// filter is first observed, place it at row-level if its column bytes + /// are this fraction or less of the total projection's column bytes. + /// Above this ratio, the filter starts as post-scan and only gets + /// promoted later if measured throughput crosses + /// `filter_pushdown_min_bytes_per_sec`. + pub filter_collecting_byte_ratio_threshold: f64, default = 0.20 + + /// (reading) Z-score for the one-sided confidence interval the adaptive + /// filter scheduler uses when promoting / demoting / dropping filters. + /// Default `2.0` (≈ 97.5%) keeps strategy moves conservative; lower the + /// value for snappier adaptation, raise it for more stable placements. + pub filter_confidence_z: f64, default = 2.0 + // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index eaf5a1642e8e2..3715e1699cc64 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -157,8 +157,8 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { } if let Some(bloom_filter_ndv) = options.bloom_filter_ndv { - builder = - builder.set_column_bloom_filter_ndv(path.clone(), bloom_filter_ndv); + builder = builder + .set_column_bloom_filter_max_ndv(path.clone(), bloom_filter_ndv); } } @@ -210,6 +210,10 @@ impl ParquetOptions { coerce_int96: _, // not used for writer props skip_arrow_metadata: _, max_predicate_cache_size: _, + // Read-time adaptive filter knobs; not used for writer props. + filter_pushdown_min_bytes_per_sec: _, + filter_collecting_byte_ratio_threshold: _, + filter_confidence_z: _, } = self; let mut builder = WriterProperties::builder() @@ -234,7 +238,7 @@ impl ParquetOptions { builder = builder.set_bloom_filter_fpp(*bloom_filter_fpp); }; if let Some(bloom_filter_ndv) = bloom_filter_ndv { - builder = builder.set_bloom_filter_ndv(*bloom_filter_ndv); + builder = builder.set_bloom_filter_max_ndv(*bloom_filter_ndv); }; if let Some(dictionary_enabled) = dictionary_enabled { builder = builder.set_dictionary_enabled(*dictionary_enabled); @@ -483,6 +487,10 @@ mod tests { skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, + filter_pushdown_min_bytes_per_sec: defaults.filter_pushdown_min_bytes_per_sec, + filter_collecting_byte_ratio_threshold: defaults + .filter_collecting_byte_ratio_threshold, + filter_confidence_z: defaults.filter_confidence_z, use_content_defined_chunking: defaults.use_content_defined_chunking.clone(), } } @@ -600,6 +608,11 @@ mod tests { binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, coerce_int96: None, + filter_pushdown_min_bytes_per_sec: global_options_defaults + .filter_pushdown_min_bytes_per_sec, + filter_collecting_byte_ratio_threshold: global_options_defaults + .filter_collecting_byte_ratio_threshold, + filter_confidence_z: global_options_defaults.filter_confidence_z, use_content_defined_chunking: props.content_defined_chunking().map(|c| { CdcOptions { min_chunk_size: c.min_chunk_size, @@ -900,7 +913,7 @@ mod tests { // the WriterProperties::default, with only ndv set let default_writer_props = WriterProperties::builder() .set_bloom_filter_enabled(true) - .set_bloom_filter_ndv(42) + .set_bloom_filter_max_ndv(42) .build(); assert_eq!( diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 9be0941b5d575..06e567cb12672 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -115,6 +115,7 @@ pub type HashMap = hashbrown::HashMap; pub type HashSet = hashbrown::HashSet; pub mod hash_map { pub use hashbrown::hash_map::Entry; + pub use hashbrown::hash_map::EntryRef; } pub mod hash_set { pub use hashbrown::hash_set::Entry; diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index d726b5c94016f..644ed1085d742 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -592,38 +592,39 @@ impl PartialOrd for ScalarValue { // any newly added enum variant will require editing this list // or else face a compile error match (self, other) { - (Decimal32(v1, p1, s1), Decimal32(v2, p2, s2)) => { - if p1.eq(p2) && s1.eq(s2) { + (Decimal32(v1, _, s1), Decimal32(v2, _, s2)) => { + if s1.eq(s2) { + // Same scale means the underlying integer values share + // a common interpretation regardless of declared + // precision (arithmetic such as `add_checked` widens + // precision by 1 but does not change the numeric + // meaning). v1.partial_cmp(v2) } else { - // Two decimal values can be compared if they have the same precision and scale. None } } (Decimal32(_, _, _), _) => None, - (Decimal64(v1, p1, s1), Decimal64(v2, p2, s2)) => { - if p1.eq(p2) && s1.eq(s2) { + (Decimal64(v1, _, s1), Decimal64(v2, _, s2)) => { + if s1.eq(s2) { v1.partial_cmp(v2) } else { - // Two decimal values can be compared if they have the same precision and scale. None } } (Decimal64(_, _, _), _) => None, - (Decimal128(v1, p1, s1), Decimal128(v2, p2, s2)) => { - if p1.eq(p2) && s1.eq(s2) { + (Decimal128(v1, _, s1), Decimal128(v2, _, s2)) => { + if s1.eq(s2) { v1.partial_cmp(v2) } else { - // Two decimal values can be compared if they have the same precision and scale. None } } (Decimal128(_, _, _), _) => None, - (Decimal256(v1, p1, s1), Decimal256(v2, p2, s2)) => { - if p1.eq(p2) && s1.eq(s2) { + (Decimal256(v1, _, s1), Decimal256(v2, _, s2)) => { + if s1.eq(s2) { v1.partial_cmp(v2) } else { - // Two decimal values can be compared if they have the same precision and scale. None } } diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 29b9c36c0a7ea..320fd43751025 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -553,10 +553,12 @@ impl Statistics { num_rows: Precision::Inexact(nr), .. } => { - // Here, the inexact case gives us an upper bound on the number of rows. + // Here, the inexact case gives us an estimate of the number of rows. if nr <= skip { - // All input data will be skipped: - Precision::Exact(0) + // All input data will be skipped. Preserve the exactness of + // the input estimate: if the input was inexact, the + // resulting zero is also inexact. + check_num_rows(Some(0), self.num_rows.is_exact().unwrap()) } else if nr <= fetch_val && skip == 0 { // If the input does not reach the `fetch` globally, and `skip` // is zero (meaning the input and output are identical), return @@ -2336,6 +2338,22 @@ mod tests { assert_eq!(result.total_byte_size, Precision::Inexact(0)); } + #[test] + fn test_with_fetch_skip_all_rows_inexact() { + // When the input num_rows is Inexact (an upper-bound estimate), an + // `nr <= skip` outcome must remain Inexact: the estimate could be + // wrong, so we cannot promote 0 to Exact. + let original_stats = Statistics { + num_rows: Precision::Inexact(0), + total_byte_size: Precision::Inexact(0), + column_statistics: vec![col_stats_i64(10)], + }; + + let result = original_stats.clone().with_fetch(None, 0, 1).unwrap(); + + assert_eq!(result.num_rows, Precision::Inexact(0)); + } + #[test] fn test_with_fetch_no_limit() { // Test when fetch is None and skip is 0 (no limit applied) diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index acee7b7a84b02..0c667b17c3fd9 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -31,12 +31,12 @@ use arrow::array::{ cast::AsArray, }; use arrow::array::{ - ArrowPrimitiveType, Datum, GenericListArray, Int32Array, Int64Array, + ArrowPrimitiveType, BooleanArray, Datum, GenericListArray, Int32Array, Int64Array, MutableArrayData, PrimitiveArray, make_array, }; use arrow::array::{LargeListViewArray, ListViewArray}; use arrow::buffer::{OffsetBuffer, ScalarBuffer}; -use arrow::compute::kernels::cmp::neq; +use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::length::length; use arrow::compute::{SortColumn, SortOptions, partition}; use arrow::datatypes::{ @@ -1129,6 +1129,7 @@ pub fn remove_list_null_values(array: &ArrayRef) -> Result { } } +/// Create a new list array where all the nulls point to empty lists fn truncate_list_nulls( list: &GenericListArray, ) -> Result> { @@ -1142,17 +1143,18 @@ fn truncate_list_nulls( &Int64Array::new_scalar(0) }; - let not_empty = neq(&lengths, zero)?; - let null_and_non_empty = &!nulls.inner() & not_empty.values(); + let (mut valid_or_empty, _nulls) = eq(&lengths, zero)?.into_parts(); + valid_or_empty |= nulls.inner(); + let valid_or_empty = BooleanArray::from(valid_or_empty); - if null_and_non_empty.count_set_bits() > 0 { + if valid_or_empty.has_false() { let array_data = list.values().to_data(); let offsets = list.offsets(); let capacity = offsets[offsets.len() - 1] - offsets[0]; let mut mutable_array_data = MutableArrayData::new(vec![&array_data], false, capacity.as_usize()); - let valid_or_empty = nulls.inner() | &!not_empty.values(); + let (valid_or_empty, _nulls) = valid_or_empty.into_parts(); for (start, end) in valid_or_empty.set_slices() { mutable_array_data.extend( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 6a8f7ab999757..c977deab32aa4 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -68,7 +68,7 @@ pub(crate) mod test_util { // Each batch writes to their own file let files: Vec<_> = batches .into_iter() - .zip(tmp_files.into_iter()) + .zip(tmp_files) .map(|(batch, mut output)| { let mut builder = parquet::file::properties::WriterProperties::builder(); if multi_page { diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 6f38df46e3d2e..2b2454b729ae3 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -166,13 +166,19 @@ mod tests { source = source.with_predicate(predicate); } + // The adaptive selectivity tracker subsumes the static + // `reorder_filters` flag. To keep these row-filter-pushdown + // assertions deterministic regardless of the byte-ratio + // heuristic, force every filter to row-level by setting + // `filter_pushdown_min_bytes_per_sec = 0` (the + // "always-row-level" sentinel). The promote/demote behavior + // exercised by other tests is irrelevant here. if self.pushdown_predicate { - source = source - .with_pushdown_filters(true) - .with_reorder_filters(true); - } else { - source = source.with_pushdown_filters(false); + let mut opts = TableParquetOptions::default(); + opts.global.filter_pushdown_min_bytes_per_sec = 0.0; + source = source.with_table_parquet_options(opts); } + source = source.with_pushdown_filters(self.pushdown_predicate); if self.page_index_predicate { source = source.with_enable_page_index(true); diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index c53495421307b..59bac91ef0af9 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -70,6 +70,18 @@ impl ParquetScanOptions { config.execution.parquet.pushdown_filters = self.pushdown_filters; config.execution.parquet.reorder_filters = self.reorder_filters; config.execution.parquet.enable_page_index = self.enable_page_index; + // When pushdown is requested in a test, force the adaptive + // selectivity tracker to promote every filter to row-level + // immediately. The tracker's production default + // (min_bytes_per_sec = INFINITY) sends every filter to + // post-scan until enough bytes-saved-per-sec evidence + // accumulates, which is the right behaviour at scale but + // keeps the row-level path / predicate cache / row-group + // pruning unexercised inside short, deterministic test + // queries. + if self.pushdown_filters { + config.execution.parquet.filter_pushdown_min_bytes_per_sec = 0.0; + } config.into() } } diff --git a/datafusion/core/tests/datasource/object_store_access.rs b/datafusion/core/tests/datasource/object_store_access.rs index 83b84f6f9284e..25150ae284cc0 100644 --- a/datafusion/core/tests/datasource/object_store_access.rs +++ b/datafusion/core/tests/datasource/object_store_access.rs @@ -904,7 +904,7 @@ async fn query_single_parquet_file_with_single_predicate() { RequestCountingObjectStore() Total Requests: 2 - GET (opts) path=parquet_table.parquet head=true - - GET (ranges) path=parquet_table.parquet ranges=1064-1481,1481-1594,1594-2011,2011-2124 + - GET (ranges) path=parquet_table.parquet ranges=1064-1594,1594-2124 " ); } @@ -928,8 +928,8 @@ async fn query_single_parquet_file_multi_row_groups_multiple_predicates() { RequestCountingObjectStore() Total Requests: 3 - GET (opts) path=parquet_table.parquet head=true - - GET (ranges) path=parquet_table.parquet ranges=4-421,421-534,534-951,951-1064 - - GET (ranges) path=parquet_table.parquet ranges=1064-1481,1481-1594,1594-2011,2011-2124 + - GET (ranges) path=parquet_table.parquet ranges=4-534,534-1064 + - GET (ranges) path=parquet_table.parquet ranges=1064-1594,1594-2124 " ); } diff --git a/datafusion/core/tests/parquet/filter_pushdown.rs b/datafusion/core/tests/parquet/filter_pushdown.rs index e6266b2c088d7..f811f2efb6d61 100644 --- a/datafusion/core/tests/parquet/filter_pushdown.rs +++ b/datafusion/core/tests/parquet/filter_pushdown.rs @@ -633,6 +633,13 @@ async fn predicate_cache_default() -> datafusion_common::Result<()> { async fn predicate_cache_pushdown_default() -> datafusion_common::Result<()> { let mut config = SessionConfig::new(); config.options_mut().execution.parquet.pushdown_filters = true; + // Force row-level placement so the predicate cache is exercised; + // see ParquetScanOptions::config for context. + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; let ctx = SessionContext::new_with_config(config); // The cache is on by default, and used when filter pushdown is enabled PredicateCacheTest { @@ -647,6 +654,11 @@ async fn predicate_cache_pushdown_default() -> datafusion_common::Result<()> { async fn predicate_cache_stats_issue_19561() -> datafusion_common::Result<()> { let mut config = SessionConfig::new(); config.options_mut().execution.parquet.pushdown_filters = true; + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; // force to get multiple batches to trigger repeated metric compound bug config.options_mut().execution.batch_size = 1; let ctx = SessionContext::new_with_config(config); @@ -664,6 +676,11 @@ async fn predicate_cache_pushdown_default_selections_only() -> datafusion_common::Result<()> { let mut config = SessionConfig::new(); config.options_mut().execution.parquet.pushdown_filters = true; + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; // forcing filter selections minimizes the number of rows read from the cache config .options_mut() diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 0e936a79ebe9f..e61b4ef26c82d 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -305,6 +305,17 @@ impl ContextWithParquet { Unit::RowGroup(row_per_group) => { config = config.with_parquet_bloom_filter_pruning(true); config.options_mut().execution.parquet.pushdown_filters = true; + // Force the adaptive selectivity tracker to promote + // every filter to row-level on first encounter; otherwise + // the row-group / bloom-filter pruning these tests assert + // on never runs (filters default to post-scan and only + // promote once enough bytes-saved-per-sec evidence + // accumulates — by which point the test has finished). + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; make_test_file_rg(scenario, row_per_group, custom_schema, custom_batches) .await } diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 4ff1fad8f52b9..47348feb1037b 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -2952,14 +2952,24 @@ async fn test_discover_dynamic_filters_via_expressions_api() { use datafusion_common::JoinType; use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr; + use datafusion_physical_expr_common::physical_expr::OptionalFilterPhysicalExpr; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; fn count_dynamic_filters(plan: &Arc) -> usize { let mut count = 0; - // Check expressions from this node using apply_expressions + // Check expressions from this node using apply_expressions. The hash + // join wraps the pushed-down dynamic filter in + // `OptionalFilterPhysicalExpr`, so peek inside that wrapper too. let _ = plan.apply_expressions(&mut |expr| { - if let Some(_df) = expr.downcast_ref::() { + if expr.downcast_ref::().is_some() { + count += 1; + } else if let Some(opt) = expr.downcast_ref::() + && opt + .inner() + .downcast_ref::() + .is_some() + { count += 1; } Ok(TreeNodeRecursion::Continue) diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 6635220cf2028..6f88e01059fc9 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -225,7 +225,7 @@ fn test_update_matching_exprs() -> Result<()> { .iter() .map(|(expr, alias)| ProjectionExpr::new(expr.clone(), alias.clone())) .collect(); - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs) { assert!( update_expr(&expr, &child_exprs, true)? .unwrap() @@ -366,7 +366,7 @@ fn test_update_projected_exprs() -> Result<()> { .iter() .map(|(expr, alias)| ProjectionExpr::new(expr.clone(), alias.clone())) .collect(); - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs) { assert!( update_expr(&expr, &proj_exprs, false)? .unwrap() @@ -579,8 +579,7 @@ fn test_streaming_table_after_projection() -> Result<()> { options: SortOptions::default(), }] .into(), - ] - .into_iter(), + ], true, None, )?; diff --git a/datafusion/datasource-parquet/Cargo.toml b/datafusion/datasource-parquet/Cargo.toml index a5855af17a536..d1837f10cb145 100644 --- a/datafusion/datasource-parquet/Cargo.toml +++ b/datafusion/datasource-parquet/Cargo.toml @@ -86,3 +86,7 @@ harness = false [[bench]] name = "parquet_struct_filter_pushdown" harness = false + +[[bench]] +name = "selectivity_tracker" +harness = false diff --git a/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs b/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs index 02137b5a1d288..cd3d2da56be53 100644 --- a/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs +++ b/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs @@ -24,6 +24,7 @@ use arrow::array::{ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use criterion::{Criterion, Throughput, criterion_group, criterion_main}; use datafusion_common::ScalarValue; +use datafusion_datasource_parquet::selectivity::SelectivityTracker; use datafusion_datasource_parquet::{ParquetFileMetrics, build_row_filter}; use datafusion_expr::{Expr, col}; use datafusion_functions_nested::expr_fn::array_has; @@ -115,9 +116,17 @@ fn scan_with_predicate( let file_metrics = ParquetFileMetrics::new(0, &path.display().to_string(), &metrics); let builder = if pushdown { - if let Some(row_filter) = - build_row_filter(predicate, file_schema, &metadata, false, &file_metrics)? - { + let tracker = Arc::new(SelectivityTracker::new()); + let filters = vec![(0usize, Arc::clone(predicate))]; + let (maybe_row_filter, _unbuildable) = build_row_filter( + &filters, + file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + )?; + if let Some(row_filter) = maybe_row_filter { builder.with_row_filter(row_filter) } else { builder diff --git a/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs b/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs index b52408d4222d8..cfc326d84fb6b 100644 --- a/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs +++ b/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs @@ -50,6 +50,7 @@ use arrow::array::{BooleanArray, Int32Array, RecordBatch, StringBuilder, StructA use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use criterion::{Criterion, Throughput, criterion_group, criterion_main}; use datafusion_common::ScalarValue; +use datafusion_datasource_parquet::selectivity::SelectivityTracker; use datafusion_datasource_parquet::{ParquetFileMetrics, build_row_filter}; use datafusion_expr::{Expr, col}; use datafusion_physical_expr::planner::logical2physical; @@ -210,9 +211,17 @@ fn scan( let mut filter_applied = false; let builder = if pushdown { - if let Some(row_filter) = - build_row_filter(predicate, file_schema, &metadata, false, &file_metrics)? - { + let tracker = Arc::new(SelectivityTracker::new()); + let filters = vec![(0usize, Arc::clone(predicate))]; + let (maybe_row_filter, _unbuildable) = build_row_filter( + &filters, + file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + )?; + if let Some(row_filter) = maybe_row_filter { filter_applied = true; builder.with_row_filter(row_filter) } else { diff --git a/datafusion/datasource-parquet/benches/selectivity_tracker.rs b/datafusion/datasource-parquet/benches/selectivity_tracker.rs new file mode 100644 index 0000000000000..45fed92d1db91 --- /dev/null +++ b/datafusion/datasource-parquet/benches/selectivity_tracker.rs @@ -0,0 +1,333 @@ +// 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. + +//! Microbenchmarks for [`SelectivityTracker`] hot paths. +//! +//! These benches isolate the tracker from decoder/IO so we can iterate on +//! its data structures independently. The scenarios model the load a +//! ClickBench-style partitioned query puts on the tracker: +//! +//! - a file is opened and each of its row-group morsels asks the tracker +//! where to place each user filter (`partition_filters`); +//! - inside each morsel the decoder hands us one `RecordBatch` at a time +//! and each batch feeds selectivity stats to the tracker (`update`). +//! +//! With the default ClickBench-partitioned workload (100 files × ~2–3 +//! row-group morsels × ~125 batches-per-morsel × ~1–3 filters-per-query), +//! the `update` path fires tens of thousands of times per query and +//! `partition_filters` fires hundreds — both on the scan critical path. +//! +//! Each bench reports the cost of a single representative operation so +//! the per-query overhead follows by simple multiplication. + +use std::sync::Arc; + +use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; +use datafusion_datasource_parquet::selectivity::{ + FilterId, SelectivityTracker, TrackerConfig, +}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::expressions::Column; +use parquet::basic::{LogicalType, Type as PhysicalType}; +use parquet::file::metadata::{ + ColumnChunkMetaData, FileMetaData, ParquetMetaData, RowGroupMetaData, +}; +use parquet::schema::types::{SchemaDescPtr, SchemaDescriptor, Type as SchemaType}; + +/// How many files a ClickBench-partitioned query typically opens. +const NUM_FILES: usize = 100; +/// Morsels per file — two full-row-group chunks is typical for hits_partitioned. +const MORSELS_PER_FILE: usize = 3; +/// Batches per morsel (row_group_rows / batch_size ≈ 500k / 8k). +const BATCHES_PER_MORSEL: usize = 60; +/// Filters per query — matches the worst regressed ClickBench queries. +const FILTERS_PER_QUERY: usize = 3; + +fn build_columns(n: usize) -> SchemaDescPtr { + let fields: Vec<_> = (0..n) + .map(|i| { + let name = format!("c{i}"); + SchemaType::primitive_type_builder(&name, PhysicalType::BYTE_ARRAY) + .with_logical_type(Some(LogicalType::String)) + .build() + .unwrap() + .into() + }) + .collect(); + let group = SchemaType::group_type_builder("schema") + .with_fields(fields) + .build() + .unwrap(); + Arc::new(SchemaDescriptor::new(Arc::new(group))) +} + +/// One file with `rg_count` row groups, each nominally `rows_per_rg` rows, +/// `bytes_per_col` compressed bytes per column. +fn build_metadata( + rg_count: usize, + rows_per_rg: i64, + num_cols: usize, + bytes_per_col: i64, +) -> ParquetMetaData { + let schema = build_columns(num_cols); + let row_groups: Vec<_> = (0..rg_count) + .map(|_| { + let cols = (0..num_cols) + .map(|c| { + ColumnChunkMetaData::builder(schema.column(c)) + .set_num_values(rows_per_rg) + .set_total_compressed_size(bytes_per_col) + .build() + .unwrap() + }) + .collect(); + RowGroupMetaData::builder(schema.clone()) + .set_num_rows(rows_per_rg) + .set_column_metadata(cols) + .build() + .unwrap() + }) + .collect(); + let total_rows = rg_count as i64 * rows_per_rg; + let file_meta = FileMetaData::new(1, total_rows, None, None, schema, None); + ParquetMetaData::new(file_meta, row_groups) +} + +/// Produce `F` user filters, each referencing a single column. Column 0 is +/// shared by filter 0 and the projection (filter-in-projection shape, as in +/// ClickBench Q14 `WHERE SearchPhrase <> ''`); the rest sit on columns +/// outside the projection. +fn make_filters(n: usize) -> Vec<(FilterId, Arc)> { + (0..n) + .map(|i| { + let expr: Arc = Arc::new(Column::new(&format!("c{i}"), i)); + (i as FilterId, expr) + }) + .collect() +} + +/// Shared setup: tracker pre-warmed with one `partition_filters` call so +/// the filter stats / state entries exist. Models "second morsel onwards". +fn warm_tracker( + config: TrackerConfig, + filters: &[(FilterId, Arc)], + metadata: &ParquetMetaData, +) -> Arc { + let tracker = Arc::new(config.build()); + // Seed with a round-trip so HashMap entries exist; otherwise the first + // bench iteration pays the "new filter" insertion cost and later ones + // don't. + let _ = tracker.partition_filters_for_test( + filters.to_vec(), + &std::collections::HashSet::new(), + 1_000_000, + metadata, + ); + tracker +} + +/// Per-batch `update` cost. This is the tightest loop — it fires once per +/// decoded batch per active filter. At ClickBench scale that's +/// NUM_FILES × MORSELS_PER_FILE × BATCHES_PER_MORSEL × FILTERS = +/// 54,000 calls per query, so every nanosecond here matters. +fn bench_update(c: &mut Criterion) { + let metadata = build_metadata(2, 500_000, 4, 10_000_000); + let filters = make_filters(FILTERS_PER_QUERY); + let tracker = warm_tracker(TrackerConfig::new(), &filters, &metadata); + + let mut group = c.benchmark_group("selectivity_tracker/update"); + group.throughput(criterion::Throughput::Elements(1)); + group.bench_function("single_call", |b| { + let id = filters[0].0; + b.iter(|| { + tracker.update( + std::hint::black_box(id), + std::hint::black_box(4_096), + std::hint::black_box(8_192), + std::hint::black_box(50_000), + std::hint::black_box(65_536), + ); + }) + }); + + // A realistic per-batch hit: we update every active filter for this + // batch. Mirrors `apply_post_scan_filters_with_stats` calling + // `tracker.update` once per filter per batch. + group.bench_function("per_batch_all_filters", |b| { + b.iter(|| { + for (id, _) in &filters { + tracker.update( + std::hint::black_box(*id), + std::hint::black_box(4_096), + std::hint::black_box(8_192), + std::hint::black_box(50_000), + std::hint::black_box(65_536), + ); + } + }) + }); + group.finish(); +} + +/// Per-morsel `partition_filters` cost. Fires once per row-group morsel, +/// so NUM_FILES × MORSELS_PER_FILE ≈ 300 per query. We measure both the +/// "cold" (first) call and the "warm" (re-partition) case. +fn bench_partition_filters(c: &mut Criterion) { + let metadata = build_metadata(2, 500_000, 4, 10_000_000); + let filters = make_filters(FILTERS_PER_QUERY); + let projection_bytes = 40_000_000usize; + + let mut group = c.benchmark_group("selectivity_tracker/partition_filters"); + group.bench_function("cold_first_call", |b| { + b.iter_batched( + || Arc::new(TrackerConfig::new().build()), + |tracker| { + std::hint::black_box(tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + projection_bytes, + &metadata, + )); + }, + criterion::BatchSize::SmallInput, + ) + }); + + // Warm case: tracker already has state for every filter, matches the + // per-morsel path after morsel 0 of any file. + let warm = warm_tracker(TrackerConfig::new(), &filters, &metadata); + group.bench_function("warm_repeat_call", |b| { + b.iter(|| { + std::hint::black_box(warm.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + projection_bytes, + &metadata, + )); + }) + }); + + // Same warm case but after realistic stats have accumulated — this is + // the path that also evaluates the confidence-bound promote/demote + // branches. Seed the tracker with a credible number of `update` calls + // before measuring. + let promoted = warm_tracker(TrackerConfig::new(), &filters, &metadata); + for _ in 0..500 { + for (id, _) in &filters { + promoted.update(*id, 3_000, 8_192, 50_000, 65_536); + } + } + group.bench_function("warm_with_accumulated_stats", |b| { + b.iter(|| { + std::hint::black_box(promoted.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + projection_bytes, + &metadata, + )); + }) + }); + group.finish(); +} + +/// End-to-end "one file open" cost: one `partition_filters` per morsel +/// plus `update` per batch per filter. This matches what a single +/// ClickBench-partitioned file inflicts on the tracker and lets us read +/// the combined improvement from any optimization in one number. +fn bench_file_scan_simulation(c: &mut Criterion) { + let metadata = build_metadata(2, 500_000, 4, 10_000_000); + let filters = make_filters(FILTERS_PER_QUERY); + let projection_bytes = 40_000_000usize; + let warm = warm_tracker(TrackerConfig::new(), &filters, &metadata); + + let mut group = c.benchmark_group("selectivity_tracker/file_scan"); + group.throughput(criterion::Throughput::Elements( + (MORSELS_PER_FILE * BATCHES_PER_MORSEL * FILTERS_PER_QUERY) as u64, + )); + group.bench_function("one_file", |b| { + b.iter(|| { + for _morsel in 0..MORSELS_PER_FILE { + std::hint::black_box(warm.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + projection_bytes, + &metadata, + )); + for _batch in 0..BATCHES_PER_MORSEL { + for (id, _) in &filters { + warm.update(*id, 3_000, 8_192, 50_000, 65_536); + } + } + } + }) + }); + group.finish(); +} + +/// Full-query simulation: [`NUM_FILES`] sequential file scans on a single +/// tracker instance. Closest approximation to the per-query tracker cost +/// a ClickBench user sees. +/// +/// Parameterised on morsels-per-file so we can see how sensitive the +/// total cost is to the morsel-split fan-out. +fn bench_query_simulation(c: &mut Criterion) { + let metadata = build_metadata(2, 500_000, 4, 10_000_000); + let filters = make_filters(FILTERS_PER_QUERY); + let projection_bytes = 40_000_000usize; + + let mut group = c.benchmark_group("selectivity_tracker/query"); + group.sample_size(20); + for morsels in [1usize, 2, 3, 5] { + group.bench_with_input( + BenchmarkId::from_parameter(morsels), + &morsels, + |b, &morsels_per_file| { + b.iter_batched( + || Arc::new(TrackerConfig::new().build()), + |tracker| { + for _file in 0..NUM_FILES { + for _morsel in 0..morsels_per_file { + std::hint::black_box(tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + projection_bytes, + &metadata, + )); + for _batch in 0..BATCHES_PER_MORSEL { + for (id, _) in &filters { + tracker.update(*id, 3_000, 8_192, 50_000, 65_536); + } + } + } + } + }, + criterion::BatchSize::SmallInput, + ) + }, + ); + } + group.finish(); +} + +criterion_group!( + benches, + bench_update, + bench_partition_filters, + bench_file_scan_simulation, + bench_query_simulation, +); +criterion_main!(benches); diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index ca4d097c37a44..295665055b849 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -91,6 +91,12 @@ use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData}; pub struct ParquetAccessPlan { /// How to access the i-th row group row_groups: Vec, + /// Whether all rows in the i-th row group are known to match the predicate. + /// + /// This is tracked separately from [`RowGroupAccess`] because it describes + /// whether row-level filter evaluation can be skipped, not which rows should + /// be read. + fully_matched: Vec, } /// Describes how the parquet reader will access a row group @@ -104,6 +110,24 @@ pub enum RowGroupAccess { Selection(RowSelection), } +/// A consecutive set of row groups that share the same row filter requirement. +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct RowGroupRun { + /// True if this run needs row filter evaluation. + pub(crate) needs_filter: bool, + /// The access plan for this run. + pub(crate) access_plan: ParquetAccessPlan, +} + +impl RowGroupRun { + fn new(needs_filter: bool, access_plan: ParquetAccessPlan) -> Self { + Self { + needs_filter, + access_plan, + } + } +} + impl RowGroupAccess { /// Return true if this row group should be scanned pub fn should_scan(&self) -> bool { @@ -119,6 +143,7 @@ impl ParquetAccessPlan { pub fn new_all(row_group_count: usize) -> Self { Self { row_groups: vec![RowGroupAccess::Scan; row_group_count], + fully_matched: vec![false; row_group_count], } } @@ -126,17 +151,26 @@ impl ParquetAccessPlan { pub fn new_none(row_group_count: usize) -> Self { Self { row_groups: vec![RowGroupAccess::Skip; row_group_count], + fully_matched: vec![false; row_group_count], } } /// Create a new `ParquetAccessPlan` from the specified [`RowGroupAccess`]es pub fn new(row_groups: Vec) -> Self { - Self { row_groups } + let row_group_count = row_groups.len(); + Self { + row_groups, + fully_matched: vec![false; row_group_count], + } } /// Set the i-th row group to the specified [`RowGroupAccess`] pub fn set(&mut self, idx: usize, access: RowGroupAccess) { + let should_scan = access.should_scan(); self.row_groups[idx] = access; + if !should_scan { + self.fully_matched[idx] = false; + } } /// skips the i-th row group (should not be scanned) @@ -154,6 +188,32 @@ impl ParquetAccessPlan { self.row_groups[idx].should_scan() } + /// Marks the i-th row group as fully matched. + /// + /// Fully matched row groups are still read according to their + /// [`RowGroupAccess`], but row-level filter evaluation can be skipped. + pub(crate) fn mark_fully_matched(&mut self, idx: usize) { + if self.should_scan(idx) { + self.fully_matched[idx] = true; + } + } + + /// Return true if the i-th row group is fully matched and scanned. + pub(crate) fn is_fully_matched(&self, idx: usize) -> bool { + self.should_scan(idx) && self.fully_matched[idx] + } + + /// Returns the fully matched row group flags. + pub(crate) fn fully_matched(&self) -> &Vec { + &self.fully_matched + } + + /// Return true if any scanned row group is fully matched. + fn has_fully_matched(&self) -> bool { + self.row_group_index_iter() + .any(|idx| self.is_fully_matched(idx)) + } + /// Set to scan only the [`RowSelection`] in the specified row group. /// /// Behavior is different depending on the existing access @@ -339,6 +399,54 @@ impl ParquetAccessPlan { self.row_groups } + /// Split this plan into consecutive row group runs that share the same row + /// filter requirement. + pub(crate) fn split_runs(self, needs_filter: bool) -> Vec { + if !needs_filter || !self.has_fully_matched() { + return vec![RowGroupRun::new(needs_filter, self)]; + } + + let num_row_groups = self.row_groups.len(); + let row_groups = self.row_groups; + let fully_matched = self.fully_matched; + let mut runs: Vec = Vec::new(); + + for (idx, (access, fully_matched)) in + row_groups.into_iter().zip(fully_matched).enumerate() + { + if !access.should_scan() { + continue; + } + + let row_group_needs_filter = !fully_matched; + if let Some(run) = runs + .last_mut() + .filter(|run| run.needs_filter == row_group_needs_filter) + { + run.access_plan.set(idx, access); + if fully_matched { + run.access_plan.mark_fully_matched(idx); + } + } else { + let mut run_plan = ParquetAccessPlan::new_none(num_row_groups); + run_plan.set(idx, access); + if fully_matched { + run_plan.mark_fully_matched(idx); + } + runs.push(RowGroupRun::new(row_group_needs_filter, run_plan)); + } + } + + if runs.is_empty() { + vec![RowGroupRun::new( + needs_filter, + ParquetAccessPlan::new_none(num_row_groups), + )] + } else { + runs + } + } + /// Prepare this plan and resolve to the final `PreparedAccessPlan` pub(crate) fn prepare( self, diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 5676ee940c7b8..175c268daf4f3 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -502,6 +502,12 @@ impl FileFormat for ParquetFormat { ) -> Result> { let mut metadata_size_hint = None; + let filter_pushdown_min_bytes_per_sec = state + .config_options() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec; + if let Some(metadata) = self.metadata_size_hint() { metadata_size_hint = Some(metadata); } @@ -511,7 +517,10 @@ impl FileFormat for ParquetFormat { .downcast_ref::() .cloned() .ok_or_else(|| internal_datafusion_err!("Expected ParquetSource"))?; - source = source.with_table_parquet_options(self.options.clone()); + let mut options = self.options.clone(); + options.global.filter_pushdown_min_bytes_per_sec = + filter_pushdown_min_bytes_per_sec; + source = source.with_table_parquet_options(options); // Use the CachedParquetFileReaderFactory let metadata_cache = state.runtime_env().cache_manager.get_file_metadata_cache(); diff --git a/datafusion/datasource-parquet/src/metrics.rs b/datafusion/datasource-parquet/src/metrics.rs index 8eb5912b919da..e5b21e140216b 100644 --- a/datafusion/datasource-parquet/src/metrics.rs +++ b/datafusion/datasource-parquet/src/metrics.rs @@ -91,6 +91,8 @@ pub struct ParquetFileMetrics { /// number of rows that were stored in the cache after evaluating predicates /// reused for the output. pub predicate_cache_records: Gauge, + //// Time spent applying filters + pub filter_apply_time: Time, } impl ParquetFileMetrics { @@ -192,6 +194,10 @@ impl ParquetFileMetrics { .with_category(MetricCategory::Rows) .gauge("predicate_cache_records", partition); + let filter_apply_time = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .subset_time("filter_apply_time", partition); + Self { files_ranges_pruned_statistics, predicate_evaluation_errors, @@ -211,6 +217,31 @@ impl ParquetFileMetrics { scan_efficiency_ratio, predicate_cache_inner_records, predicate_cache_records, + filter_apply_time, } } + + /// Record pages whose page-index pruning was skipped because the containing + /// row group was fully matched by row-group statistics. + /// + /// The counter is only registered when there is a non-zero value. This keeps + /// [`ParquetFileMetrics::new`] from cloning the filename and metrics set for + /// files that never use this metric. + pub(crate) fn add_page_index_pages_skipped_by_fully_matched( + metrics: &ExecutionPlanMetricsSet, + partition: usize, + filename: &str, + n: usize, + ) { + if n == 0 { + return; + } + + let count = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .with_type(MetricType::Summary) + .with_category(MetricCategory::Rows) + .counter("page_index_pages_skipped_by_fully_matched", partition); + count.add(n); + } } diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 9a907f4118a86..eb81383a93ca7 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -33,6 +33,7 @@ mod page_filter; mod reader; mod row_filter; mod row_group_filter; +pub mod selectivity; mod sort; pub mod source; mod supported_predicates; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index bbbd298687ab5..e6a818e0a0b01 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -17,6 +17,7 @@ //! [`ParquetMorselizer`] state machines for opening Parquet files +use crate::access_plan::PreparedAccessPlan; use crate::page_filter::PagePruningAccessPlanFilter; use crate::row_filter::build_projection_read_plan; use crate::row_group_filter::{BloomFilterStatistics, RowGroupAccessPlanFilter}; @@ -30,7 +31,7 @@ use datafusion_datasource::morsel::{Morsel, MorselPlan, MorselPlanner, Morselize use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr_adapter::replace_columns_with_literals; -use std::collections::HashMap; +use std::collections::{HashMap, VecDeque}; use std::fmt; use std::future::Future; use std::mem; @@ -68,14 +69,17 @@ use parquet::DecodeResult; use parquet::arrow::ParquetRecordBatchStreamBuilder; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; use parquet::arrow::arrow_reader::{ - ArrowReaderMetadata, ArrowReaderOptions, RowSelectionPolicy, + ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, + RowSelectionPolicy, }; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::parquet_column; -use parquet::arrow::push_decoder::{ParquetPushDecoder, ParquetPushDecoderBuilder}; +use parquet::arrow::push_decoder::{ + ParquetPushDecoder, ParquetPushDecoderBuilder, StrategySwap, +}; use parquet::basic::Type; use parquet::bloom_filter::Sbbf; -use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader}; +use parquet::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; /// Stateless Parquet morselizer implementation. /// @@ -94,8 +98,16 @@ pub(super) struct ParquetMorselizer { pub(crate) limit: Option, /// If should keep the output rows in order pub preserve_order: bool, - /// Optional predicate to apply during the scan - pub predicate: Option>, + /// Optional predicate conjuncts for row filtering during the scan. + /// Each conjunct is tagged with a stable `FilterId` for the adaptive + /// [`crate::selectivity::SelectivityTracker`] so per-filter stats + /// accumulate across files. + pub predicate_conjuncts: + Option)>>, + /// Adaptive selectivity tracker shared across files. Each opener feeds + /// per-batch stats and asks for the current optimal split between + /// row-level and post-scan placement at row-group boundaries. + pub selectivity_tracker: Arc, /// Table schema, including partition columns. pub table_schema: TableSchema, /// Optional hint for how large the initial request to read parquet metadata @@ -108,8 +120,6 @@ pub(super) struct ParquetMorselizer { /// Should the filters be evaluated during the parquet scan using /// [`DataFusionArrowPredicate`](row_filter::DatafusionArrowPredicate)? pub pushdown_filters: bool, - /// Should the filters be reordered to optimize the scan? - pub reorder_filters: bool, /// Should we force the reader to use RowSelections for filtering pub force_filter_selections: bool, /// Should the page index be read from parquet files, if present, to skip @@ -273,8 +283,15 @@ struct PreparedParquetOpen { physical_file_schema: SchemaRef, output_schema: SchemaRef, projection: ProjectionExprs, - predicate: Option>, - reorder_predicates: bool, + /// Predicate conjuncts with stable `FilterId`s for the adaptive + /// selectivity tracker. Carried forward from + /// [`ParquetMorselizer::predicate_conjuncts`]. The combined predicate + /// (used for pruning and `FilePruner`) is recomputed on demand from + /// these conjuncts. + predicate_conjuncts: + Option)>>, + /// Shared adaptive selectivity tracker. + selectivity_tracker: Arc, pushdown_filters: bool, force_filter_selections: bool, enable_page_index: bool, @@ -316,6 +333,12 @@ struct FiltersPreparedParquetOpen { struct RowGroupsPrunedParquetOpen { prepared: FiltersPreparedParquetOpen, row_groups: RowGroupAccessPlanFilter, + /// Per-conjunct row-group pruning rates surfaced as a side-effect of + /// `prune_by_statistics_with_per_conjunct_stats`. Threaded into the + /// adaptive scheduler's `pruning_rates` map alongside the + /// page-pruning rates so the initial-placement prior gets the + /// strongest available signal per FilterId. + row_group_per_conjunct: Vec, } /// State of [`ParquetOpenState`] @@ -597,22 +620,41 @@ impl ParquetMorselizer { )); let mut projection = self.projection.clone(); - let mut predicate = self.predicate.clone(); + let mut predicate_conjuncts = self.predicate_conjuncts.clone(); if !literal_columns.is_empty() { projection = projection.try_map_exprs(|expr| { replace_columns_with_literals(Arc::clone(&expr), &literal_columns) })?; - predicate = predicate - .map(|p| replace_columns_with_literals(p, &literal_columns)) - .transpose()?; + // Rewrite each conjunct individually so per-conjunct FilterIds + // remain stable and continue to refer to the same expression + // across files (modulo literal substitution). + if let Some(ref mut conjuncts) = predicate_conjuncts { + for (_id, expr) in conjuncts.iter_mut() { + *expr = replace_columns_with_literals( + Arc::clone(expr), + &literal_columns, + )?; + } + } } let predicate_creation_errors = MetricBuilder::new(&self.metrics) .with_category(MetricCategory::Rows) .global_counter("num_predicate_creation_errors"); + // Combine conjuncts into a single AND-ed predicate for the file-level + // pruner and for early statistics-driven elimination. The file + // pruner does not need per-conjunct identities — only a boolean + // expression over file-level columns and partition values. + let combined_predicate: Option> = + predicate_conjuncts.as_ref().map(|conjuncts| { + datafusion_physical_expr::conjunction( + conjuncts.iter().map(|(_, e)| Arc::clone(e)), + ) + }); + // Apply literal replacements to projection and predicate - let file_pruner = predicate + let file_pruner = combined_predicate .as_ref() .filter(|p| is_dynamic_physical_expr(p) || partitioned_file.has_statistics()) .and_then(|p| { @@ -642,8 +684,8 @@ impl ParquetMorselizer { physical_file_schema: logical_file_schema, output_schema, projection, - predicate, - reorder_predicates: self.reorder_filters, + predicate_conjuncts, + selectivity_tracker: Arc::clone(&self.selectivity_tracker), pushdown_filters: self.pushdown_filters, force_filter_selections: self.force_filter_selections, enable_page_index: self.enable_page_index, @@ -663,6 +705,17 @@ impl ParquetMorselizer { } impl PreparedParquetOpen { + /// Reconstruct a single AND-ed predicate from the per-conjunct list. + /// Used for pruning, page-index setup, and `FilePruner` construction — + /// callers that don't care about the per-conjunct `FilterId` identities. + fn combined_predicate(&self) -> Option> { + self.predicate_conjuncts.as_ref().map(|conjuncts| { + datafusion_physical_expr::conjunction( + conjuncts.iter().map(|(_, e)| Arc::clone(e)), + ) + }) + } + /// Attempt file-level pruning before any metadata is loaded. /// /// Returns `None` if the file can be skipped completely. @@ -804,7 +857,7 @@ impl MetadataLoadedParquetOpen { // columns are appended after file columns in the table schema), // types are the same, and there are no missing columns. Skip the // tree walk entirely in that case. - let needs_rewrite = prepared.predicate.is_some() + let needs_rewrite = prepared.predicate_conjuncts.is_some() || prepared.logical_file_schema != physical_file_schema; if needs_rewrite { let rewriter = prepared.expr_adapter_factory.create( @@ -812,29 +865,73 @@ impl MetadataLoadedParquetOpen { Arc::clone(&physical_file_schema), )?; let simplifier = PhysicalExprSimplifier::new(&physical_file_schema); - prepared.predicate = prepared - .predicate - .map(|p| simplifier.simplify(rewriter.rewrite(p)?)) - .transpose()?; + // Rewrite each conjunct individually so per-conjunct FilterIds + // remain stable across files. + if let Some(ref mut conjuncts) = prepared.predicate_conjuncts { + for (_, expr) in conjuncts.iter_mut() { + *expr = simplifier.simplify(rewriter.rewrite(Arc::clone(expr))?)?; + } + } prepared.projection = prepared .projection .try_map_exprs(|p| simplifier.simplify(rewriter.rewrite(p)?))?; } prepared.physical_file_schema = Arc::clone(&physical_file_schema); - // Build predicates for this specific file - let pruning_predicate = build_pruning_predicates( - prepared.predicate.as_ref(), - &physical_file_schema, - &prepared.predicate_creation_errors, - ); + // Combined AND-ed predicate is only used for pruning / page-index + // setup; the conjunct-level identities are preserved separately for + // the adaptive selectivity tracker. + let combined_predicate = prepared.combined_predicate(); + + // Build predicates for this specific file. When we have + // FilterId-tagged conjuncts available, use the tagged + // constructor so the row-group pruning pass surfaces + // per-FilterId pruning rates as a side-effect of the same + // iteration that produces the row-group prune decision (see + // `PruningPredicate::prune_per_conjunct`). Otherwise fall back + // to the existing combined-predicate path. + let pruning_predicate = if let Some(conjuncts) = + prepared.predicate_conjuncts.as_ref() + && !conjuncts.is_empty() + { + match PruningPredicate::try_new_tagged_conjuncts( + conjuncts.as_slice(), + Arc::clone(&physical_file_schema), + ) { + Ok(p) if !p.always_true() => Some(Arc::new(p)), + _ => None, + } + } else { + build_pruning_predicates( + combined_predicate.as_ref(), + &physical_file_schema, + &prepared.predicate_creation_errors, + ) + }; - // Only build page pruning predicate if page index is enabled + // Only build page pruning predicate if page index is enabled. + // Prefer the *tagged* constructor when we have FilterId-tagged + // conjuncts available: this lets `prune_plan_with_per_conjunct_stats` + // surface per-FilterId pruning rates as a side-effect of the + // pruning iteration the opener was going to run anyway. The + // adaptive scheduler then consumes those rates as its initial- + // placement prior — no extra pruning passes. let page_pruning_predicate = if prepared.enable_page_index { - prepared.predicate.as_ref().and_then(|predicate| { - let p = build_page_pruning_predicate(predicate, &physical_file_schema); + if let Some(conjuncts) = prepared.predicate_conjuncts.as_ref() + && !conjuncts.is_empty() + { + let p = Arc::new(PagePruningAccessPlanFilter::new_tagged( + conjuncts.as_slice(), + &physical_file_schema, + )); (p.filter_number() > 0).then_some(p) - }) + } else { + combined_predicate.as_ref().and_then(|predicate| { + let p = + build_page_pruning_predicate(predicate, &physical_file_schema); + (p.filter_number() > 0).then_some(p) + }) + } } else { None }; @@ -894,15 +991,18 @@ impl FiltersPreparedParquetOpen { } // If there is a predicate that can be evaluated against the metadata + let mut row_group_per_conjunct: Vec = + Vec::new(); if let Some(predicate) = self.pruning_predicate.as_ref().map(|p| p.as_ref()) { if prepared.enable_row_group_stats_pruning { - row_groups.prune_by_statistics( - &prepared.physical_file_schema, - loaded.reader_metadata.parquet_schema(), - rg_metadata, - predicate, - &prepared.file_metrics, - ); + row_group_per_conjunct = row_groups + .prune_by_statistics_with_per_conjunct_stats( + &prepared.physical_file_schema, + loaded.reader_metadata.parquet_schema(), + rg_metadata, + predicate, + &prepared.file_metrics, + ); } else { // Update metrics: statistics unavailable, so all row groups are // matched (not pruned) @@ -936,6 +1036,7 @@ impl FiltersPreparedParquetOpen { Ok(RowGroupsPrunedParquetOpen { prepared: self, row_groups, + row_group_per_conjunct, }) } } @@ -1043,11 +1144,39 @@ impl BloomFiltersLoadedParquetOpen { && self.prepared.prepared.loaded.prepared.enable_bloom_filter && !self.prepared.row_groups.is_empty() { - self.prepared.row_groups.prune_by_bloom_filters( - predicate, - &self.prepared.prepared.loaded.prepared.file_metrics, - &self.row_group_bloom_filters, - ); + // Capture per-conjunct bloom-filter rates; merge into the + // RowGroupsPrunedParquetOpen accumulator alongside the + // row-group-stats rates. Bloom filters often catch + // string-equality / IN predicates that min/max stats miss. + let bloom_per_conjunct = self + .prepared + .row_groups + .prune_by_bloom_filters_with_per_conjunct_stats( + predicate, + &self.prepared.prepared.loaded.prepared.file_metrics, + &self.row_group_bloom_filters, + ); + // Merge into the row_group_per_conjunct accumulator. For + // each FilterId, keep the strongest signal (max + // pruning_rate) seen across the two sources. + for bloom in bloom_per_conjunct { + if let Some(rate) = bloom.pruning_rate() { + if let Some(existing) = self + .prepared + .row_group_per_conjunct + .iter_mut() + .find(|s| s.tag == bloom.tag) + { + let existing_rate = existing.pruning_rate().unwrap_or(0.0); + if rate > existing_rate { + existing.containers_seen = bloom.containers_seen; + existing.containers_pruned = bloom.containers_pruned; + } + } else { + self.prepared.row_group_per_conjunct.push(bloom); + } + } + } } self.prepared @@ -1060,6 +1189,7 @@ impl RowGroupsPrunedParquetOpen { let RowGroupsPrunedParquetOpen { prepared, mut row_groups, + row_group_per_conjunct, } = self; let FiltersPreparedParquetOpen { loaded, @@ -1075,129 +1205,389 @@ impl RowGroupsPrunedParquetOpen { let file_metadata = Arc::clone(reader_metadata.metadata()); let rg_metadata = file_metadata.row_groups(); - // Filter pushdown: evaluate predicates during scan - let row_filter = if let Some(predicate) = prepared - .pushdown_filters - .then_some(prepared.predicate.clone()) - .flatten() - { - let row_filter = row_filter::build_row_filter( - &predicate, - &prepared.physical_file_schema, - file_metadata.as_ref(), - prepared.reorder_predicates, - &prepared.file_metrics, - ); - - match row_filter { - Ok(Some(filter)) => Some(filter), - Ok(None) => None, - Err(e) => { - debug!("Ignoring error building row filter for '{predicate:?}': {e}"); - None - } - } - } else { - None - }; + // Adaptive filter placement at file open. Ask the shared + // `SelectivityTracker` to split predicate conjuncts (already adapted + // to `physical_file_schema`) into row-level and post-scan buckets + // based on stats accumulated across earlier files. The same split + // is re-evaluated mid-stream at row-group boundaries via + // `AdaptiveParquetStream::maybe_swap_strategy`. + // + // The set of leaf-column indices in the user projection — passed + // to the tracker so its byte-ratio heuristic only counts filter + // columns *not already in the projection* (a column that's in + // the projection costs zero extra I/O to push down). + let projection_columns: std::collections::HashSet = + datafusion_physical_expr::utils::collect_columns( + &datafusion_physical_expr::conjunction(prepared.projection.expr_iter()), + ) + .iter() + .map(|c| c.index()) + .collect(); + let projection_compressed_bytes = row_filter::total_compressed_bytes( + &projection_columns.iter().copied().collect::>(), + file_metadata.as_ref(), + ); + // === Run row-group + page pruning FIRST so we can capture + // per-conjunct page-pruning rates and feed them into + // `partition_filters` as the initial-placement prior. === // Prune by limit if limit is set and limit order is not sensitive if let (Some(limit), false) = (prepared.limit, prepared.preserve_order) { row_groups.prune_by_limit(limit, rg_metadata, &prepared.file_metrics); } - // Page index pruning: if all data on individual pages can - // be ruled using page metadata, rows from other columns - // with that range can be skipped as well. + // Initial-placement prior map: per-FilterId pruning rates fed + // to `partition_filters` so it can decide row-level vs + // post-scan based on real selectivity stats from the prunings + // the opener already ran. + // + // Two sources, both side-effects of work that happens regardless + // of this experiment: + // 1. Row-group min/max pruning + // (see `prune_by_statistics_with_per_conjunct_stats`). + // 2. Page-index pruning (see + // `prune_plan_with_per_conjunct_stats`). Page-level is + // strictly finer-grained; when both produce a rate for + // the same FilterId, the page-level rate wins (it's + // written second). let mut access_plan = row_groups.build(); + let mut page_pruning_rates: HashMap = + HashMap::new(); + // Source 1: row-group rates + for stats in &row_group_per_conjunct { + if let Some(tag) = stats.tag + && let Some(rate) = stats.pruning_rate() + { + page_pruning_rates.insert(tag, rate); + } + } + // Source 2: page-index rates (override row-group when both are present) if prepared.enable_page_index && !access_plan.is_empty() - && let Some(page_pruning_predicate) = page_pruning_predicate + && let Some(page_pruning_predicate) = page_pruning_predicate.as_ref() + { + let (new_plan, per_conjunct, pages_skipped_by_fully_matched) = + page_pruning_predicate.prune_plan_with_per_conjunct_stats( + access_plan, + &prepared.physical_file_schema, + reader_metadata.parquet_schema(), + file_metadata.as_ref(), + &prepared.file_metrics, + ); + access_plan = new_plan; + for stats in per_conjunct { + if let Some(tag) = stats.tag + && let Some(rate) = stats.pruning_rate() + { + page_pruning_rates.insert(tag, rate); + } + } + ParquetFileMetrics::add_page_index_pages_skipped_by_fully_matched( + &prepared.metrics, + prepared.partition_index, + &prepared.file_name, + pages_skipped_by_fully_matched, + ); + } + + let (row_filter_conjuncts, mut post_scan_conjuncts) = if prepared.pushdown_filters + && let Some(conjuncts) = prepared.predicate_conjuncts.clone() + && !conjuncts.is_empty() { - access_plan = page_pruning_predicate.prune_plan_with_page_index( - access_plan, + let partitioned = prepared.selectivity_tracker.partition_filters( + conjuncts, + &projection_columns, + projection_compressed_bytes, + file_metadata.as_ref(), &prepared.physical_file_schema, reader_metadata.parquet_schema(), + &page_pruning_rates, + ); + (partitioned.row_filters, partitioned.post_scan) + } else { + (Vec::new(), Vec::new()) + }; + + // Build row-level `ArrowPredicate`s for the row_filters bucket. Any + // conjunct that `build_row_filter` reports as `unbuildable` falls + // through to the post-scan bucket so we never silently drop a + // filter — dropping would relax the user's predicate and return + // wrong results. + // Capture the row-filter id set before any potential move into + // `post_scan_conjuncts` on the error fall-through below, so the + // adaptive stream can detect placement changes against this baseline. + let initial_row_filter_ids: std::collections::BTreeSet< + crate::selectivity::FilterId, + > = row_filter_conjuncts.iter().map(|(id, _)| *id).collect(); + + let row_filter = if !row_filter_conjuncts.is_empty() { + match row_filter::build_row_filter( + &row_filter_conjuncts, + &prepared.physical_file_schema, file_metadata.as_ref(), + projection_compressed_bytes, + &prepared.selectivity_tracker, &prepared.file_metrics, - ); - } + ) { + Ok((row_filter, unbuildable)) => { + post_scan_conjuncts.extend(unbuildable); + row_filter + } + Err(e) => { + debug!( + "Error building row filter for {row_filter_conjuncts:?}: {e}; \ + falling all row-filter candidates through to post-scan" + ); + post_scan_conjuncts.extend(row_filter_conjuncts.clone()); + None + } + } + } else { + None + }; - // Prepare the access plan (extract row groups and row selection) - let mut prepared_plan = access_plan.prepare(rg_metadata)?; + // If the build above failed and dropped every row-filter candidate + // into post-scan, treat the active set as empty so the first + // mid-stream swap will rebuild from scratch using whatever the + // tracker decides next. + let active_row_filter_ids = if row_filter.is_some() { + initial_row_filter_ids + } else { + std::collections::BTreeSet::new() + }; - // Potentially reverse the access plan for performance. - // See `ParquetSource::try_pushdown_sort` for the rationale. - if prepared.reverse_row_groups { - prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; - } + // (prune_by_limit + page-index pruning ran above so we could + // pass the per-conjunct page rates into `partition_filters`.) let arrow_reader_metrics = ArrowReaderMetrics::enabled(); - let read_plan = build_projection_read_plan( - prepared.projection.expr_iter(), + + // Build the decoder's projection over (user projection ∪ + // initial post-scan filter columns). Row-level filter columns + // live in the RowFilter's per-predicate masks, so they don't + // need to be in the decoder's output stream batch. + // + // The mask is NOT fixed for the file's life — `maybe_swap_strategy` + // can grow or shrink it at row-group boundaries when the optimal + // mask cols change (e.g. a filter promotes to row-level so its + // cols leave the mask, or a previously-placeholder dynamic filter + // wakes up and its cols enter the mask via a post-scan placement). + // Each mask change triggers a rebuild of `stream_schema`, + // `projector`, and the post-scan filter rebase; arrow-rs's + // `StrategySwap::with_projection` then installs the new mask + // before the next row group is read. + // Keep the *unrebased* projection (against physical_file_schema) so + // dynamic-mask changes can re-rebase it against a new stream_schema. + let original_projection = prepared.projection.clone(); + + // Build initial decoder projection state (stream_schema, projection + // mask, projector, rebased post-scan filters). Same helper is used + // for mid-stream mask swaps in `maybe_swap_strategy`. + let proj_state = build_decoder_projection_state( + &original_projection, + &post_scan_conjuncts, + &projection_columns, &prepared.physical_file_schema, reader_metadata.parquet_schema(), - ); + &prepared.output_schema, + )?; + + // Split the access plan into consecutive runs of row groups that share + // the same filter requirement. Fully-matched runs (where row-group/page + // statistics already proved every row satisfies the predicate) skip the + // RowFilter *and* the post-scan filters — both are no-ops there and + // wasted CPU/IO. Other runs use the adaptive setup computed above. + // + // `has_filter_work` reflects whether any conjunct survived adaptive + // partitioning; if not, `split_runs` returns a single no-filter run. + let has_filter_work = row_filter.is_some() || !post_scan_conjuncts.is_empty(); + let mut runs = access_plan.split_runs(has_filter_work); + if prepared.reverse_row_groups { + runs.reverse(); + } + let run_count = runs.len(); + + // Decoder-level limit can only be pushed in when: + // * there's a single run (otherwise later runs would never see the + // limit), AND + // * that run has no post-scan filters (otherwise the decoder may + // short-circuit before post-scan would have rejected enough rows). + // For multi-run files, we enforce the limit at the stream level via + // `remaining_limit`. + let single_no_post_scan = + run_count == 1 && (!runs[0].needs_filter || post_scan_conjuncts.is_empty()); + let decoder_limit = prepared.limit.filter(|_| single_no_post_scan); + let remaining_limit = prepared.limit.filter(|_| !single_no_post_scan); + + // Pre-compute a stripped projection state for any !needs_filter runs, + // since they don't need post-scan-filter columns in the decoder mask. + // Built lazily on first use to avoid the cost when all runs need + // the filter. + let mut no_filter_proj_state: Option = None; + + // Build one `RunDecoder` per run. The first run becomes the active + // state of the stream; the rest queue up as `pending_runs`. + let mut run_decoders: VecDeque = VecDeque::with_capacity(run_count); + // Hand the eagerly-built first row_filter into the first needs_filter + // run; subsequent needs_filter runs rebuild it from the same partition + // (RowFilter is `!Clone`). + let mut first_row_filter = row_filter; + let mut first_filter_run_consumed = false; + for run in runs { + let prepared_access_plan = { + let mut plan = run.access_plan.prepare(rg_metadata)?; + if prepared.reverse_row_groups { + plan = plan.reverse(file_metadata.as_ref())?; + } + plan + }; - let mut decoder_builder = - ParquetPushDecoderBuilder::new_with_metadata(reader_metadata) - .with_projection(read_plan.projection_mask) - .with_batch_size(prepared.batch_size) - .with_metrics(arrow_reader_metrics.clone()); + let run_decoder = if run.needs_filter { + // Adaptive setup. The first needs_filter run reuses the + // already-built row_filter; later ones rebuild it. + // + // `post_scan_conjuncts` and `active_row_filter_ids` were + // finalized above when the first `build_row_filter` ran — + // either with the success path's `unbuildable` folded in, or + // with the error path's full fall-through. Because + // `build_row_filter` is deterministic given the same inputs, + // re-running it for later runs produces an identical + // success/failure partition, so we reuse those vars verbatim + // and only need a fresh `RowFilter` (it is `!Clone`). + let (row_filter, post_scan_for_run, active_ids_for_run) = + if !first_filter_run_consumed { + first_filter_run_consumed = true; + ( + first_row_filter.take(), + post_scan_conjuncts.clone(), + active_row_filter_ids.clone(), + ) + } else { + let rf = if !row_filter_conjuncts.is_empty() { + match row_filter::build_row_filter( + &row_filter_conjuncts, + &prepared.physical_file_schema, + file_metadata.as_ref(), + projection_compressed_bytes, + &prepared.selectivity_tracker, + &prepared.file_metrics, + ) { + Ok((rf, _unbuildable)) => rf, + Err(e) => { + debug!( + "Error rebuilding row filter for next run: {e}; \ + falling all row-filter candidates through to post-scan" + ); + None + } + } + } else { + None + }; + ( + rf, + post_scan_conjuncts.clone(), + active_row_filter_ids.clone(), + ) + }; - if let Some(row_filter) = row_filter { - decoder_builder = decoder_builder.with_row_filter(row_filter); - } - if prepared.force_filter_selections { - decoder_builder = - decoder_builder.with_row_selection_policy(RowSelectionPolicy::Selectors); - } - if let Some(row_selection) = prepared_plan.row_selection { - decoder_builder = decoder_builder.with_row_selection(row_selection); - } - decoder_builder = - decoder_builder.with_row_groups(prepared_plan.row_group_indexes); - if let Some(limit) = prepared.limit { - decoder_builder = decoder_builder.with_limit(limit); - } - if let Some(max_predicate_cache_size) = prepared.max_predicate_cache_size { - decoder_builder = - decoder_builder.with_max_predicate_cache_size(max_predicate_cache_size); + build_run_decoder( + /*needs_filter=*/ true, + prepared_access_plan, + reader_metadata.clone(), + row_filter, + &post_scan_for_run, + active_ids_for_run, + &proj_state, + projection_compressed_bytes, + file_metadata.as_ref(), + arrow_reader_metrics.clone(), + prepared.batch_size, + prepared.force_filter_selections, + prepared.max_predicate_cache_size, + decoder_limit, + )? + } else { + // Fully-matched run: no row filter, no post-scan filters, + // narrower mask (no post-scan-filter cols). Build the + // stripped projection state once and reuse. + if no_filter_proj_state.is_none() { + no_filter_proj_state = Some(build_decoder_projection_state( + &original_projection, + &[], + &projection_columns, + &prepared.physical_file_schema, + reader_metadata.parquet_schema(), + &prepared.output_schema, + )?); + } + let no_filter_state = no_filter_proj_state.as_ref().unwrap(); + + build_run_decoder( + /*needs_filter=*/ false, + prepared_access_plan, + reader_metadata.clone(), + None, + &[], + std::collections::BTreeSet::new(), + no_filter_state, + projection_compressed_bytes, + file_metadata.as_ref(), + arrow_reader_metrics.clone(), + prepared.batch_size, + prepared.force_filter_selections, + prepared.max_predicate_cache_size, + decoder_limit, + )? + }; + run_decoders.push_back(run_decoder); } - let decoder = decoder_builder.build()?; + let initial = run_decoders + .pop_front() + .expect("split_runs always yields at least one run"); + let pending_runs = run_decoders; let predicate_cache_inner_records = prepared.file_metrics.predicate_cache_inner_records.clone(); let predicate_cache_records = prepared.file_metrics.predicate_cache_records.clone(); + let filter_apply_time = prepared.file_metrics.filter_apply_time.clone(); - // Check if we need to replace the schema to handle things like differing nullability or metadata. - // See note below about file vs. output schema. - let stream_schema = read_plan.projected_schema; - let replace_schema = stream_schema != prepared.output_schema; - - // Rebase column indices to match the narrowed stream schema. - // The projection expressions have indices based on physical_file_schema, - // but the stream only contains the columns selected by the ProjectionMask. - let projection = prepared - .projection - .try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))?; - let projector = projection.make_projector(&stream_schema)?; let output_schema = Arc::clone(&prepared.output_schema); let files_ranges_pruned_statistics = prepared.file_metrics.files_ranges_pruned_statistics.clone(); let stream = futures::stream::unfold( - PushDecoderStreamState { - decoder, + AdaptiveParquetStream { + decoder: initial.decoder, reader: prepared.async_file_reader, - projector, + active_reader: None, + file_metadata: Arc::clone(&file_metadata), + parquet_schema: file_metadata.file_metadata().schema_descr_ptr(), + physical_file_schema: Arc::clone(&prepared.physical_file_schema), + stream_schema: initial.stream_schema, + file_metrics: prepared.file_metrics.clone(), + tracker: Arc::clone(&prepared.selectivity_tracker), + all_conjuncts: prepared.predicate_conjuncts.unwrap_or_default(), + projection_columns, + projection_compressed_bytes, + active_row_filter_ids: initial.active_row_filter_ids, + post_scan_filters: initial.post_scan_filters, + post_scan_other_bytes_per_row: initial.post_scan_other_bytes_per_row, + filter_apply_time, + projector: initial.projector, + original_projection, + current_mask_cols: initial.current_mask_cols, output_schema, - replace_schema, + replace_schema: initial.replace_schema, arrow_reader_metrics, predicate_cache_inner_records, predicate_cache_records, baseline_metrics: prepared.baseline_metrics, + pushdown_filters: prepared.pushdown_filters, + page_pruning_rates, + current_run_needs_filter: initial.needs_filter, + pending_runs, + remaining_limit, }, |state| async move { state.transition().await }, ) @@ -1218,75 +1608,626 @@ impl RowGroupsPrunedParquetOpen { } } -/// State for a stream that decodes a single Parquet file using a push-based decoder. +/// Bundles the per-mask decoder projection state. Produced at file open +/// from the initial filter partition and again at any row-group boundary +/// where `maybe_swap_strategy` decides the mask cols changed. Holding +/// everything together keeps the two code paths in sync — the same +/// expressions go through the same `build_projection_read_plan` -> +/// `reassign_expr_columns` -> `make_projector` chain in both cases. +struct DecoderProjectionState { + /// Schema of batches yielded by the decoder (post-mask, pre-projector). + stream_schema: SchemaRef, + /// Mask installed on the decoder / passed to `StrategySwap`. + projection_mask: parquet::arrow::ProjectionMask, + /// Projector that maps stream batches to the user-visible output. + projector: Projector, + /// Post-scan filter exprs rebased against `stream_schema`. + rebased_post_scan: Vec<(crate::selectivity::FilterId, Arc)>, + /// True when `stream_schema != output_schema` and `project_batch` + /// must replace the schema metadata before yielding. + replace_schema: bool, + /// Leaf column indices the mask covers, kept for cheap change + /// detection on the next swap. + mask_cols: std::collections::BTreeSet, +} + +/// Build a fresh [`DecoderProjectionState`] for the given partition of +/// post-scan filters. Used at file open with the initial partition, and +/// again on every mask-changing swap. +/// +/// The "mask cols" are derived from `(user projection ∪ post_scan_conjuncts +/// columns)`. Row-level filter columns are *not* in this set — arrow-rs's +/// per-predicate masks decode them separately from the output stream +/// batch. +fn build_decoder_projection_state( + original_projection: &ProjectionExprs, + post_scan_conjuncts: &[(crate::selectivity::FilterId, Arc)], + projection_columns: &std::collections::HashSet, + physical_file_schema: &SchemaRef, + parquet_schema: &parquet::schema::types::SchemaDescriptor, + output_schema: &SchemaRef, +) -> Result { + let read_plan = build_projection_read_plan( + original_projection + .expr_iter() + .chain(post_scan_conjuncts.iter().map(|(_, expr)| Arc::clone(expr))), + physical_file_schema, + parquet_schema, + ); + let stream_schema = Arc::clone(&read_plan.projected_schema); + let replace_schema = stream_schema != *output_schema; + + let rebased_projection = original_projection + .clone() + .try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))?; + let projector = rebased_projection.make_projector(&stream_schema)?; + + let rebased_post_scan = post_scan_conjuncts + .iter() + .map(|(id, expr)| { + reassign_expr_columns(Arc::clone(expr), &stream_schema).map(|e| (*id, e)) + }) + .collect::>>()?; + + let mut mask_cols: std::collections::BTreeSet = + projection_columns.iter().copied().collect(); + for (_, expr) in post_scan_conjuncts { + for c in datafusion_physical_expr::utils::collect_columns(expr) { + mask_cols.insert(c.index()); + } + } + + Ok(DecoderProjectionState { + stream_schema, + projection_mask: read_plan.projection_mask, + projector, + rebased_post_scan, + replace_schema, + mask_cols, + }) +} + +/// Per-run decoder bundle. One [`AdaptiveParquetStream`] holds one of these +/// as its active state and queues the rest in `pending_runs`. Each run gets +/// its own decoder with run-specific filter configuration: +/// +/// * "Fully matched" runs (`needs_filter == false`) carry no row filter, +/// no post-scan filters, and use a stripped projection mask containing +/// only the user-projection columns — neither row-level pushdown nor +/// post-scan filtering does any work that the row-group statistics +/// haven't already proved redundant. +/// * "Needs filter" runs carry the adaptive `(row_filter, post_scan)` +/// placement decided at file open. Within such a run, +/// [`AdaptiveParquetStream::maybe_swap_strategy`] can still re-place +/// filters at row-group boundaries; placement is *not* re-evaluated when +/// crossing into a different run (the next run's setup is fixed at file +/// open and the decoder is rebuilt fresh). +struct RunDecoder { + decoder: ParquetPushDecoder, + /// True if this run still needs filter evaluation (row-level + post-scan + /// as adaptively partitioned). False for fully-matched runs that skip + /// filtering entirely. + needs_filter: bool, + /// Schema of batches yielded by this run's decoder. + stream_schema: SchemaRef, + /// Projector mapping this run's stream batches to the user-visible + /// output. Run-specific because the stream_schema can differ across + /// runs (no-filter runs use a narrower mask). + projector: Projector, + /// Post-scan filters for this run, rebased to its stream_schema. + /// Empty for fully-matched runs. + post_scan_filters: Vec<(crate::selectivity::FilterId, Arc)>, + /// Per-post-scan-filter "other-bytes-per-row" cost metric. Empty for + /// fully-matched runs. + post_scan_other_bytes_per_row: Vec, + /// Leaf-column indices in this run's decoder projection mask. + current_mask_cols: std::collections::BTreeSet, + /// FilterIds currently applied as row-level predicates inside this + /// run's decoder. Empty for fully-matched runs and for needs_filter + /// runs whose `build_row_filter` returned `None`. + active_row_filter_ids: std::collections::BTreeSet, + /// True when `stream_schema != output_schema` for this run. + replace_schema: bool, +} + +/// Build a single [`RunDecoder`] for one row-group run. Used both for the +/// initial decoder built at file open and for queued decoders that the +/// stream switches into when the active decoder finishes. +/// +/// `proj_state` carries the projection state appropriate for this run — +/// the full adaptive state for `needs_filter` runs, or the stripped +/// user-projection-only state for fully-matched runs. The caller is +/// responsible for picking the right `proj_state` *and* `post_scan_for_run` +/// (which must match the state used to build `proj_state`). +#[expect(clippy::too_many_arguments)] +fn build_run_decoder( + needs_filter: bool, + prepared_access_plan: PreparedAccessPlan, + reader_metadata: ArrowReaderMetadata, + row_filter: Option, + post_scan_for_run: &[(crate::selectivity::FilterId, Arc)], + active_row_filter_ids: std::collections::BTreeSet, + proj_state: &DecoderProjectionState, + projection_compressed_bytes: usize, + file_metadata: &ParquetMetaData, + arrow_reader_metrics: ArrowReaderMetrics, + batch_size: usize, + force_filter_selections: bool, + max_predicate_cache_size: Option, + decoder_limit: Option, +) -> Result { + let mut builder = ParquetPushDecoderBuilder::new_with_metadata(reader_metadata) + .with_projection(proj_state.projection_mask.clone()) + .with_batch_size(batch_size) + .with_metrics(arrow_reader_metrics); + + if let Some(row_filter) = row_filter { + builder = builder.with_row_filter(row_filter); + } + if force_filter_selections { + builder = builder.with_row_selection_policy(RowSelectionPolicy::Selectors); + } + if let Some(row_selection) = prepared_access_plan.row_selection { + builder = builder.with_row_selection(row_selection); + } + builder = builder.with_row_groups(prepared_access_plan.row_group_indexes); + if let Some(limit) = decoder_limit { + builder = builder.with_limit(limit); + } + if let Some(max_predicate_cache_size) = max_predicate_cache_size { + builder = builder.with_max_predicate_cache_size(max_predicate_cache_size); + } + + let decoder = builder.build()?; + + // Per-post-scan-filter "other-bytes-per-row" cost, used by the tracker + // to compare promote/demote utility across row-level and post-scan + // candidates on the same axis. + let total_rows: i64 = file_metadata + .row_groups() + .iter() + .map(|rg| rg.num_rows()) + .sum(); + let post_scan_other_bytes_per_row: Vec = post_scan_for_run + .iter() + .map(|(_, expr)| { + let filter_cols: Vec = + datafusion_physical_expr::utils::collect_columns(expr) + .iter() + .map(|c| c.index()) + .collect(); + let filter_compressed = + row_filter::total_compressed_bytes(&filter_cols, file_metadata); + if total_rows > 0 { + projection_compressed_bytes.saturating_sub(filter_compressed) as f64 + / total_rows as f64 + } else { + 0.0 + } + }) + .collect(); + + Ok(RunDecoder { + decoder, + needs_filter, + stream_schema: Arc::clone(&proj_state.stream_schema), + projector: proj_state.projector.clone(), + post_scan_filters: proj_state.rebased_post_scan.clone(), + post_scan_other_bytes_per_row, + current_mask_cols: proj_state.mask_cols.clone(), + active_row_filter_ids, + replace_schema: proj_state.replace_schema, + }) +} + +/// State for a stream that decodes a single Parquet file with adaptive +/// filter scheduling. +/// +/// The [`transition`](Self::transition) method drives one row group at a +/// time: /// -/// The [`transition`](Self::transition) method drives the decoder in a loop: it requests -/// byte ranges from the [`AsyncFileReader`], pushes the fetched data into the -/// [`ParquetPushDecoder`], and yields projected [`RecordBatch`]es until the file is -/// fully consumed. -struct PushDecoderStreamState { +/// 1. Pull a [`ParquetRecordBatchReader`] for the next row group via +/// [`ParquetPushDecoder::try_next_reader`], fetching ranges as needed. +/// 2. Iterate the reader synchronously; each batch goes through any +/// post-scan filters (which feed per-filter stats into the shared +/// [`SelectivityTracker`](crate::selectivity::SelectivityTracker)) and +/// then through the projector. +/// 3. When the reader exhausts, ask the tracker to re-partition filters +/// based on accumulated stats. If the placement changed, build a new +/// `RowFilter` and call [`ParquetPushDecoder::swap_strategy`] before +/// requesting the next row group. +/// +/// Why one decoder per file (vs the chunk-per-row-group split in PR #9): +/// - Reuses arrow-rs `PushBuffers` across row groups so already-fetched +/// bytes that survive a strategy swap aren't re-requested. +/// - Avoids per-chunk reader minting and per-chunk `RowFilter` rebuild +/// (`RowFilter` is `!Clone`). +/// - One [`EarlyStoppingStream`] wrap covers the whole file — no +/// chunk-0-only special case for the non-`Clone` `FilePruner`. +struct AdaptiveParquetStream { decoder: ParquetPushDecoder, reader: Box, + /// Active row-group reader. `None` between row groups (when a swap + /// can be applied) and at start. + active_reader: Option, + /// Parquet metadata for the file. Used by the tracker to size filter + /// vs projection bytes when re-partitioning. + file_metadata: Arc, + /// Parquet `SchemaDescriptor`, used by the page-pruning prior in the + /// tracker to construct `RowGroupPruningStatistics`. + parquet_schema: Arc, + /// Schema used for filter expressions before rebase. + physical_file_schema: SchemaRef, + /// Wide schema the decoder yields — including post-scan-filter columns + /// not in the user projection. Stable across the file even when a + /// strategy swap moves filters around. + stream_schema: SchemaRef, + file_metrics: ParquetFileMetrics, + tracker: Arc, + /// Full set of predicate conjuncts for this file (with stable + /// FilterIds), re-fed to `partition_filters` at every row-group + /// boundary. + all_conjuncts: Vec<(crate::selectivity::FilterId, Arc)>, + /// Leaf-column indices in the user projection — passed to the tracker + /// so its byte-ratio heuristic can subtract overlap with the + /// projection (a filter column already in the projection costs no + /// extra I/O at row-level). + projection_columns: std::collections::HashSet, + /// Total compressed bytes for the user projection. Constant across + /// the file; reused at every swap decision. + projection_compressed_bytes: usize, + /// Set of FilterIds currently applied as row-level predicates inside + /// the decoder. A change in this set drives the swap. + active_row_filter_ids: std::collections::BTreeSet, + /// Post-scan filters expressed against `stream_schema`. Updated on + /// swap. + post_scan_filters: Vec<(crate::selectivity::FilterId, Arc)>, + /// Per-post-scan-filter "other-bytes-per-row" cost metric — bytes of + /// projection columns *not* referenced by this filter, amortised. + /// Same units as the row-filter path's `other_projected_bytes_per_row` + /// so promote/demote rankings compare on a single axis. + post_scan_other_bytes_per_row: Vec, + filter_apply_time: datafusion_physical_plan::metrics::Time, projector: Projector, + /// The user projection expressed against `physical_file_schema`, + /// kept here so the projector can be rebuilt against a new + /// `stream_schema` whenever the decoder mask changes. + original_projection: ProjectionExprs, + /// Leaf column indices currently covered by the decoder's projection + /// mask. Maintained in lockstep with the decoder; updated on every + /// successful `swap_strategy(with_projection(...))`. A change in this + /// set triggers the projector / stream_schema rebuild. + current_mask_cols: std::collections::BTreeSet, output_schema: Arc, replace_schema: bool, arrow_reader_metrics: ArrowReaderMetrics, predicate_cache_inner_records: Gauge, predicate_cache_records: Gauge, baseline_metrics: BaselineMetrics, + /// Whether filter pushdown is enabled for this file. When `false`, + /// `swap_strategy` is never called and `post_scan_filters` is empty. + pushdown_filters: bool, + /// Per-FilterId page-pruning rates collected as a side-effect of + /// the page-index pruning the opener already ran on this file. + /// Empty when page index wasn't loaded or `predicate_conjuncts` was + /// not tagged. Threaded into every `partition_filters` call so the + /// initial-placement prior can use real selectivity stats from the + /// already-completed pruning instead of re-evaluating. + page_pruning_rates: HashMap, + /// True when the currently-active decoder is processing a run that + /// needs filter evaluation. False for fully-matched runs whose row + /// groups are known to entirely satisfy the predicate from row-group + /// (and possibly page-index) statistics — those runs skip both the + /// row-level RowFilter and any post-scan filtering, and `maybe_swap_strategy` + /// becomes a no-op for them. + current_run_needs_filter: bool, + /// Additional decoders queued for subsequent row-group runs. Used when + /// the access plan was split because consecutive row groups have + /// differing filter requirements (e.g. a stripe of fully-matched row + /// groups in the middle of an otherwise filtered scan). Each entry + /// carries the full per-run state (decoder, projector, filters, mask) + /// because runs may use different masks / filters. + pending_runs: VecDeque, + /// Global remaining row limit across all decoder runs. + /// + /// Decoder-local limits are only safe for single-run scans without + /// post-scan filters (see file-open setup). When the scan is split + /// across multiple decoders, or when the active run has post-scan + /// filters that may reject more rows, the combined stream limit is + /// enforced here by slicing batches. + remaining_limit: Option, } -impl PushDecoderStreamState { - /// Advances the decoder state machine until the next [`RecordBatch`] is - /// produced, the file is fully consumed, or an error occurs. +impl AdaptiveParquetStream { + /// Advances the state machine until the next batch is produced, the + /// file is fully consumed, or an error occurs. Drives one row group + /// at a time, swapping filter strategy at row-group boundaries. /// - /// On each iteration the decoder is polled via [`ParquetPushDecoder::try_decode`]: - /// - [`NeedsData`](DecodeResult::NeedsData) – the requested byte ranges are - /// fetched from the [`AsyncFileReader`] and fed back into the decoder. - /// - [`Data`](DecodeResult::Data) – a decoded batch is projected and returned. - /// - [`Finished`](DecodeResult::Finished) – signals end-of-stream (`None`). - /// - /// Takes `self` by value (rather than `&mut self`) so the generated future - /// owns the state directly. This avoids a Stacked Borrows violation under - /// miri where `&mut self` creates a single opaque borrow that conflicts - /// with `unfold`'s ownership across yield points. + /// Takes `self` by value so the generated future owns the state + /// directly — same rationale as the previous `PushDecoderStreamState`: + /// `&mut self` creates a Stacked Borrows conflict with `unfold`'s + /// ownership across yield points under miri. async fn transition(mut self) -> Option<(Result, Self)> { loop { - match self.decoder.try_decode() { - Ok(DecodeResult::NeedsData(ranges)) => { - let data = self - .reader - .get_byte_ranges(ranges.clone()) - .await - .map_err(DataFusionError::from); - match data { - Ok(data) => { - if let Err(e) = self.decoder.push_ranges(ranges, data) { - return Some((Err(DataFusionError::from(e)), self)); + if self.remaining_limit == Some(0) { + return None; + } + // Step 1: ensure we have a reader for the current row group. + if self.active_reader.is_none() { + // Re-evaluate filter placement at every row-group boundary. + // Skipped for fully-matched runs (no filter to place) and + // when pushdown is disabled. + if self.pushdown_filters + && self.current_run_needs_filter + && let Err(e) = self.maybe_swap_strategy() + { + return Some((Err(e), self)); + } + // Pull the next reader, fetching data as needed. + loop { + match self.decoder.try_next_reader() { + Ok(DecodeResult::NeedsData(ranges)) => { + let n_ranges = ranges.len(); + let started = datafusion_common::instant::Instant::now(); + match self.reader.get_byte_ranges(ranges.clone()).await { + Ok(data) => { + let elapsed = started.elapsed().as_nanos() as u64; + self.tracker.record_fetch(n_ranges, elapsed); + if let Err(e) = self.decoder.push_ranges(ranges, data) + { + return Some(( + Err(DataFusionError::from(e)), + self, + )); + } + } + Err(e) => { + return Some((Err(DataFusionError::from(e)), self)); + } } } - Err(e) => return Some((Err(e), self)), + Ok(DecodeResult::Data(reader)) => { + self.active_reader = Some(reader); + break; + } + Ok(DecodeResult::Finished) => { + // Current run finished. If another run is queued, + // swap its state in and continue. + if let Some(next) = self.pending_runs.pop_front() { + self.decoder = next.decoder; + self.current_run_needs_filter = next.needs_filter; + self.stream_schema = next.stream_schema; + self.projector = next.projector; + self.post_scan_filters = next.post_scan_filters; + self.post_scan_other_bytes_per_row = + next.post_scan_other_bytes_per_row; + self.current_mask_cols = next.current_mask_cols; + self.active_row_filter_ids = next.active_row_filter_ids; + self.replace_schema = next.replace_schema; + continue; + } + return None; + } + Err(e) => return Some((Err(DataFusionError::from(e)), self)), } } - Ok(DecodeResult::Data(batch)) => { - let mut timer = self.baseline_metrics.elapsed_compute().timer(); - self.copy_arrow_reader_metrics(); - let result = self.project_batch(&batch); + } + + // Step 2: pull the next batch out of the active reader. Reader + // iteration is synchronous because all bytes for the row group + // were already pushed before the reader was constructed. + let batch_result = self + .active_reader + .as_mut() + .expect("active_reader set above") + .next(); + let batch = match batch_result { + Some(Ok(batch)) => batch, + Some(Err(e)) => return Some((Err(DataFusionError::from(e)), self)), + None => { + // Row group exhausted — drop the reader so the next + // iteration goes back to step 1 and considers a swap. + self.active_reader = None; + continue; + } + }; + + // Step 3: post-scan filters + projector + schema replacement. + // + // Fully-matched runs (`!current_run_needs_filter`) carry an empty + // `post_scan_filters` set by construction, so the post-scan + // branch is effectively skipped without an extra flag check. + let mut timer = self.baseline_metrics.elapsed_compute().timer(); + self.copy_arrow_reader_metrics(); + let filtered = if self.post_scan_filters.is_empty() { + Ok(batch) + } else { + let start = datafusion_common::instant::Instant::now(); + let r = apply_post_scan_filters_with_stats( + batch, + &self.post_scan_filters, + &self.post_scan_other_bytes_per_row, + &self.tracker, + ); + self.filter_apply_time.add_elapsed(start); + r + }; + match filtered { + // Post-scan may filter every row in a batch. Skip empty + // outputs so the consumer doesn't see noise batches. + Ok(b) if b.num_rows() == 0 => { + timer.stop(); + continue; + } + Ok(b) => { + // Apply the global cross-run row limit when enforced at + // the stream level. Decoder-local limits handle the + // single-run no-post-scan case at file open. + let b = if let Some(remaining) = self.remaining_limit { + if b.num_rows() > remaining { + self.remaining_limit = Some(0); + b.slice(0, remaining) + } else { + self.remaining_limit = Some(remaining - b.num_rows()); + b + } + } else { + b + }; + let result = self.project_batch(&b); timer.stop(); - // Release the borrow on baseline_metrics before moving self drop(timer); return Some((result, self)); } - Ok(DecodeResult::Finished) => { - return None; - } Err(e) => { - return Some((Err(DataFusionError::from(e)), self)); + timer.stop(); + drop(timer); + return Some((Err(e), self)); } } } } + /// Re-evaluate filter placement at a row-group boundary. The + /// resulting `StrategySwap` may install: + /// + /// - A new `RowFilter` (if the row-level filter set changed), and/or + /// - A new `ProjectionMask` (if the optimal mask cols changed — + /// shrinks when a filter promotes out of post-scan, grows when a + /// filter newly enters post-scan, e.g. a dynamic placeholder that + /// woke up). + /// + /// When the mask changes we rebuild `stream_schema`, `projector`, + /// and re-rebase post-scan filter exprs against the new schema. The + /// invariant: `post_scan_filters` are always expressed in terms of + /// `stream_schema`, and `stream_schema` always matches the decoder's + /// current projection mask. + /// + /// No-op when the decoder isn't at a swap point or there are no + /// conjuncts. + fn maybe_swap_strategy(&mut self) -> Result<()> { + if !self.decoder.can_swap_strategy() || self.all_conjuncts.is_empty() { + return Ok(()); + } + let partitioned = self.tracker.partition_filters( + self.all_conjuncts.clone(), + &self.projection_columns, + self.projection_compressed_bytes, + self.file_metadata.as_ref(), + &self.physical_file_schema, + self.parquet_schema.as_ref(), + &self.page_pruning_rates, + ); + + let new_ids: std::collections::BTreeSet = + partitioned.row_filters.iter().map(|(id, _)| *id).collect(); + + // Cheap pre-check: if the row-filter set AND the mask cols would + // both be identical, skip the rest of the work. The mask check + // uses just the heuristic post-scan list (no unbuildable yet); + // unbuildable can only grow the col set, so if we see equality + // here we're guaranteed equality after merging — safe to bail. + if new_ids == self.active_row_filter_ids { + let mut tentative_mask_cols: std::collections::BTreeSet = + self.projection_columns.iter().copied().collect(); + for (_, expr) in &partitioned.post_scan { + for c in datafusion_physical_expr::utils::collect_columns(expr) { + tentative_mask_cols.insert(c.index()); + } + } + if tentative_mask_cols == self.current_mask_cols { + // Placement unchanged. Post-scan and dropped filters can + // change with stats but they don't need a decoder-level + // swap — `apply_post_scan_filters_with_stats` already + // consults `tracker.is_filter_skipped` per batch. + return Ok(()); + } + } + + // Rebuild the row filter from the new row-level set. + let (row_filter, unbuildable) = row_filter::build_row_filter( + &partitioned.row_filters, + &self.physical_file_schema, + self.file_metadata.as_ref(), + self.projection_compressed_bytes, + &self.tracker, + &self.file_metrics, + )?; + + // Combine post-scan + unbuildable into the new post-scan set. + // Unbuildable filters may reference cols outside the heuristic + // post-scan list, so the mask check has to wait until after we + // merge them. + let mut post_scan = partitioned.post_scan; + post_scan.extend(unbuildable); + + // Rebuild the per-mask decoder state. The helper computes + // `mask_cols` from `(projection_columns ∪ post_scan cols)` so we + // pick up any unbuildable cols that just joined post-scan. + let new_state = build_decoder_projection_state( + &self.original_projection, + &post_scan, + &self.projection_columns, + &self.physical_file_schema, + self.parquet_schema.as_ref(), + &self.output_schema, + )?; + let mask_will_change = new_state.mask_cols != self.current_mask_cols; + + let total_rows: i64 = self + .file_metadata + .row_groups() + .iter() + .map(|rg| rg.num_rows()) + .sum(); + + let post_scan_other_bytes_per_row: Vec = post_scan + .iter() + .map(|(_, expr)| { + let cols: Vec = + datafusion_physical_expr::utils::collect_columns(expr) + .iter() + .map(|c| c.index()) + .collect(); + let filter_compressed = row_filter::total_compressed_bytes( + &cols, + self.file_metadata.as_ref(), + ); + if total_rows > 0 { + self.projection_compressed_bytes + .saturating_sub(filter_compressed) as f64 + / total_rows as f64 + } else { + 0.0 + } + }) + .collect(); + + // Build the StrategySwap; only include `with_projection` when + // the mask actually changed (avoid unnecessary arrow-rs internal + // rebuilds when only the RowFilter set changed). + let mut swap = StrategySwap::new().with_filter(row_filter); + if mask_will_change { + swap = swap.with_projection(new_state.projection_mask.clone()); + } + + self.decoder + .swap_strategy(swap) + .map_err(DataFusionError::from)?; + + // Decoder accepted the swap; commit state changes. + self.active_row_filter_ids = new_ids; + self.post_scan_filters = new_state.rebased_post_scan; + self.post_scan_other_bytes_per_row = post_scan_other_bytes_per_row; + if mask_will_change { + self.stream_schema = new_state.stream_schema; + self.projector = new_state.projector; + self.replace_schema = new_state.replace_schema; + self.current_mask_cols = new_state.mask_cols; + } + Ok(()) + } + /// Copies metrics from ArrowReaderMetrics (the metrics collected by the /// arrow-rs parquet reader) to the parquet file metrics for DataFusion fn copy_arrow_reader_metrics(&self) { @@ -1301,14 +2242,6 @@ impl PushDecoderStreamState { fn project_batch(&self, batch: &RecordBatch) -> Result { let mut batch = self.projector.project_batch(batch)?; if self.replace_schema { - // Ensure the output batch has the expected schema. - // This handles things like schema level and field level metadata, which may not be present - // in the physical file schema. - // It is also possible for nullability to differ; some writers create files with - // OPTIONAL fields even when there are no nulls in the data. - // In these cases it may make sense for the logical schema to be `NOT NULL`. - // RecordBatch::try_new_with_options checks that if the schema is NOT NULL - // the array cannot contain nulls, amongst other checks. let (_stream_schema, arrays, num_rows) = batch.into_parts(); let options = RecordBatchOptions::new().with_row_count(Some(num_rows)); batch = RecordBatch::try_new_with_options( @@ -1321,6 +2254,79 @@ impl PushDecoderStreamState { } } +/// Apply a list of post-scan filters to a batch in order, AND-ing their +/// boolean masks. Each filter's evaluation reports stats to the shared +/// [`SelectivityTracker`](crate::selectivity::SelectivityTracker) in the +/// same units as the row-filter path so promote/demote decisions can +/// compare row-level and post-scan filter effectiveness on one axis. +/// +/// `other_bytes_per_row[i]` is the bytes-per-row of the projection columns +/// *not* referenced by `filters[i]` — i.e. the late-materialization saving +/// per pruned row. +fn apply_post_scan_filters_with_stats( + batch: RecordBatch, + filters: &[(crate::selectivity::FilterId, Arc)], + other_bytes_per_row: &[f64], + tracker: &crate::selectivity::SelectivityTracker, +) -> Result { + use arrow::array::BooleanArray; + use arrow::compute::{and, filter_record_batch}; + use datafusion_common::cast::as_boolean_array; + + if batch.num_rows() == 0 { + return Ok(batch); + } + + let input_rows = batch.num_rows() as u64; + let mut combined_mask: Option = None; + + for (i, (id, expr)) in filters.iter().enumerate() { + // Mid-stream skip: the tracker sets this flag on + // `OptionalFilterPhysicalExpr` whose CI upper bound has fallen + // below `min_bytes_per_sec`. Correctness is preserved because the + // originating join independently enforces the predicate. We do + // not update the tracker for a skipped batch. + if tracker.is_filter_skipped(*id) { + continue; + } + + // Per-batch tracker bookkeeping. We measure every batch (no + // sampling): the `Instant + tracker.update` path is hot, but + // skipping samples delays first-promotion by N× and that + // dominates the steady-state lock contention on + // strongly-selective queries (Q22 / Q23 / Q24). The Welford + // accumulator converges within the first row group either way. + let start = datafusion_common::instant::Instant::now(); + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let bool_arr = as_boolean_array(result.as_ref())?; + let nanos = start.elapsed().as_nanos() as u64; + let num_matched = bool_arr.true_count() as u64; + + // Convert the raw "all the non-filter projection bytes for + // this batch" into a *scatter-aware* skippable count: only + // the sub-windows of the bool array with zero survivors + // represent decode work that late-materialization would + // actually skip. A 50% filter on uniform data scores 0 + // here; a 50% filter on contiguous data scores ~0.5. + let total_other_bytes = (other_bytes_per_row[i] * input_rows as f64) as u64; + let skippable_bytes = + crate::selectivity::count_skippable_bytes(bool_arr, total_other_bytes); + tracker.update(*id, num_matched, input_rows, nanos, skippable_bytes); + + if num_matched < input_rows { + combined_mask = Some(match combined_mask { + Some(prev) => and(&prev, bool_arr)?, + None => bool_arr.clone(), + }); + } + } + + match combined_mask { + Some(mask) => Ok(filter_record_batch(&batch, &mask)?), + None => Ok(batch), + } +} + type ConstantColumns = HashMap; /// Extract constant column values from statistics, keyed by column name in the logical file schema. @@ -1661,7 +2667,6 @@ mod test { metadata_size_hint: Option, metrics: ExecutionPlanMetricsSet, pushdown_filters: bool, - reorder_filters: bool, force_filter_selections: bool, enable_page_index: bool, enable_bloom_filter: bool, @@ -1687,7 +2692,6 @@ mod test { metadata_size_hint: None, metrics: ExecutionPlanMetricsSet::new(), pushdown_filters: false, - reorder_filters: false, force_filter_selections: false, enable_page_index: false, enable_bloom_filter: false, @@ -1735,12 +2739,6 @@ mod test { self } - /// Enable filter reordering. - fn with_reorder_filters(mut self, enable: bool) -> Self { - self.reorder_filters = enable; - self - } - /// Enable row group stats pruning. fn with_row_group_stats_pruning(mut self, enable: bool) -> Self { self.enable_row_group_stats_pruning = enable; @@ -1783,13 +2781,28 @@ mod test { ProjectionExprs::from_indices(&all_indices, &file_schema) }; + // Split the test-supplied AND-of-conjuncts predicate into the + // tagged-conjunct shape `ParquetMorselizer` now expects. Tests + // continue to pass a single `Arc` for + // ergonomics. + let predicate_conjuncts = self.predicate.as_ref().map(|p| { + datafusion_physical_expr::split_conjunction(p) + .into_iter() + .enumerate() + .map(|(id, expr)| (id, Arc::clone(expr))) + .collect::>() + }); + ParquetMorselizer { partition_index: self.partition_index, projection, batch_size: self.batch_size, limit: self.limit, preserve_order: self.preserve_order, - predicate: self.predicate, + predicate_conjuncts, + selectivity_tracker: Arc::new( + crate::selectivity::SelectivityTracker::default(), + ), table_schema, metadata_size_hint: self.metadata_size_hint, metrics: self.metrics, @@ -1797,7 +2810,6 @@ mod test { DefaultParquetFileReaderFactory::new(store), ), pushdown_filters: self.pushdown_filters, - reorder_filters: self.reorder_filters, force_filter_selections: self.force_filter_selections, enable_page_index: self.enable_page_index, enable_bloom_filter: self.enable_bloom_filter, @@ -2235,7 +3247,6 @@ mod test { .with_projection_indices(&[0]) .with_predicate(predicate) .with_pushdown_filters(true) // note that this is true! - .with_reorder_filters(true) .build() }; diff --git a/datafusion/datasource-parquet/src/page_filter.rs b/datafusion/datasource-parquet/src/page_filter.rs index bc6551c977003..076fffc248718 100644 --- a/datafusion/datasource-parquet/src/page_filter.rs +++ b/datafusion/datasource-parquet/src/page_filter.rs @@ -113,6 +113,37 @@ pub struct PagePruningAccessPlanFilter { /// single column predicates (e.g. (`col = 5`) extracted from the overall /// predicate. Must all be true for a row to be included in the result. predicates: Vec, + /// Per-predicate tag (caller-supplied id, typically a `FilterId`). + /// `None` when the filter was constructed without tagging via + /// [`Self::new`]; `Some` when constructed via [`Self::new_tagged`]. + /// The vector has the same length as `predicates`. + tags: Option>, +} + +/// Per-conjunct accumulators surfaced by +/// [`PagePruningAccessPlanFilter::prune_plan_with_per_conjunct_stats`]. +/// One entry per kept predicate (in the same order as `predicates`). +#[derive(Clone, Debug, Default)] +pub struct PerConjunctPageStats { + /// Caller tag (e.g. FilterId) — `None` when the filter was built + /// untagged via [`PagePruningAccessPlanFilter::new`]. + pub tag: Option, + /// Total rows in row groups where this conjunct was evaluated. + pub rows_seen: u64, + /// Rows the page index proved this conjunct alone would skip. + pub rows_skipped: u64, +} + +impl PerConjunctPageStats { + /// Returns the per-conjunct page-pruning rate, or `None` when no + /// rows were evaluated (e.g. the file has no page index for this + /// column, or the predicate's converter couldn't be built). + pub fn pruning_rate(&self) -> Option { + if self.rows_seen == 0 { + return None; + } + Some(self.rows_skipped as f64 / self.rows_seen as f64) + } } impl PagePruningAccessPlanFilter { @@ -148,7 +179,50 @@ impl PagePruningAccessPlanFilter { Some(pp) }) .collect::>(); - Self { predicates } + Self { + predicates, + tags: None, + } + } + + /// Variant of [`Self::new`] that takes already-split conjuncts each + /// carrying a caller tag (usually a `FilterId`). Predicates that + /// fail the same single-column / non-trivial filtering as `new` + /// are dropped, but tags survive for the conjuncts that make it + /// through. Subsequent calls to + /// [`Self::prune_plan_with_per_conjunct_stats`] return per-conjunct + /// pruning stats keyed by tag. + pub fn new_tagged( + conjuncts: &[(usize, Arc)], + schema: &SchemaRef, + ) -> Self { + let mut predicates = Vec::with_capacity(conjuncts.len()); + let mut tags = Vec::with_capacity(conjuncts.len()); + for (id, expr) in conjuncts { + let pp = match PruningPredicate::try_new(Arc::clone(expr), Arc::clone(schema)) + { + Ok(pp) => pp, + Err(e) => { + debug!( + "Ignoring error creating tagged page pruning predicate \ + for filter id {id}: {e}" + ); + continue; + } + }; + if pp.always_true() { + continue; + } + if pp.required_columns().single_column().is_none() { + continue; + } + predicates.push(pp); + tags.push(*id); + } + Self { + predicates, + tags: Some(tags), + } } /// Returns an updated [`ParquetAccessPlan`] by applying predicates to the @@ -160,18 +234,18 @@ impl PagePruningAccessPlanFilter { parquet_schema: &SchemaDescriptor, parquet_metadata: &ParquetMetaData, file_metrics: &ParquetFileMetrics, - ) -> ParquetAccessPlan { + ) -> (ParquetAccessPlan, usize) { // scoped timer updates on drop let _timer_guard = file_metrics.page_index_eval_time.timer(); if self.predicates.is_empty() { - return access_plan; + return (access_plan, 0); } let page_index_predicates = &self.predicates; let groups = parquet_metadata.row_groups(); if groups.is_empty() { - return access_plan; + return (access_plan, 0); } if parquet_metadata.offset_index().is_none() @@ -182,7 +256,7 @@ impl PagePruningAccessPlanFilter { parquet_metadata.offset_index().is_some(), parquet_metadata.column_index().is_some() ); - return access_plan; + return (access_plan, 0); }; // track the total number of rows that should be skipped @@ -193,10 +267,22 @@ impl PagePruningAccessPlanFilter { let mut total_pages_skip = 0; // track the total number of pages that should not be skipped let mut total_pages_select = 0; + // track pages for which page-index pruning was skipped because the + // containing row group was already proven fully matched by statistics + let mut total_pages_skipped_by_fully_matched = 0; // for each row group specified in the access plan let row_group_indexes = access_plan.row_group_indexes(); for row_group_index in row_group_indexes { + // Skip page pruning for fully matched row groups: all rows are + // known to satisfy the predicate, so page-level pruning is wasted work. + if access_plan.is_fully_matched(row_group_index) { + let page_count = + fully_matched_page_count(row_group_index, parquet_metadata); + total_pages_skipped_by_fully_matched += page_count; + + continue; + } // The selection for this particular row group let mut overall_selection = None; @@ -211,10 +297,13 @@ impl PagePruningAccessPlanFilter { HashSet::from_iter(0..total_pages_in_group); for predicate in page_index_predicates { - let column = predicate - .required_columns() - .single_column() - .expect("Page pruning requires single column predicates"); + let Some(column) = predicate.required_columns().single_column() else { + debug!( + "Ignoring multi-column page pruning predicate: {:?}", + predicate.predicate_expr() + ); + continue; + }; let converter = StatisticsConverter::try_new( column.name(), @@ -314,13 +403,197 @@ impl PagePruningAccessPlanFilter { file_metrics .page_index_pages_pruned .add_matched(total_pages_select); - access_plan + (access_plan, total_pages_skipped_by_fully_matched) } /// Returns the number of filters in the [`PagePruningAccessPlanFilter`] pub fn filter_number(&self) -> usize { self.predicates.len() } + + /// Like [`Self::prune_plan_with_page_index`] but also surfaces, as a + /// side-effect of the pruning iteration that already runs, a + /// per-conjunct accumulator with the rows that conjunct alone + /// would have proven skippable. Callers use this to seed a + /// per-FilterId selectivity prior without doing any extra pruning + /// work — every page-index lookup that would have happened in + /// `prune_plan_with_page_index` happens exactly once here too. + pub fn prune_plan_with_per_conjunct_stats( + &self, + mut access_plan: ParquetAccessPlan, + arrow_schema: &Schema, + parquet_schema: &SchemaDescriptor, + parquet_metadata: &ParquetMetaData, + file_metrics: &ParquetFileMetrics, + ) -> (ParquetAccessPlan, Vec, usize) { + // scoped timer updates on drop + let _timer_guard = file_metrics.page_index_eval_time.timer(); + + let mut per_conjunct: Vec = (0..self.predicates.len()) + .map(|i| PerConjunctPageStats { + tag: self.tags.as_ref().and_then(|t| t.get(i).copied()), + rows_seen: 0, + rows_skipped: 0, + }) + .collect(); + + if self.predicates.is_empty() { + return (access_plan, per_conjunct, 0); + } + + let groups = parquet_metadata.row_groups(); + if groups.is_empty() { + return (access_plan, per_conjunct, 0); + } + + if parquet_metadata.offset_index().is_none() + || parquet_metadata.column_index().is_none() + { + return (access_plan, per_conjunct, 0); + } + + // Same accumulators as the untagged path, plus per-conjunct. + let mut total_skip = 0; + let mut total_select = 0; + let mut total_pages_skip = 0; + let mut total_pages_select = 0; + // Pages we skipped pruning for because row-group stats already + // proved the row group is fully matched — wasted work avoided, + // surfaced as a metric. + let mut total_pages_skipped_by_fully_matched = 0; + + let row_group_indexes = access_plan.row_group_indexes(); + for row_group_index in row_group_indexes { + // Skip page pruning for fully matched row groups: all rows are + // known to satisfy the predicate, so page-level pruning is + // wasted work. Still feed the rows into `rows_seen` per + // conjunct so per-FilterId pruning rates reflect the file's + // full row count rather than just the non-fully-matched part. + if access_plan.is_fully_matched(row_group_index) { + let page_count = + fully_matched_page_count(row_group_index, parquet_metadata); + total_pages_skipped_by_fully_matched += page_count; + let rg_rows = groups[row_group_index].num_rows() as u64; + for stats in per_conjunct.iter_mut() { + stats.rows_seen = stats.rows_seen.saturating_add(rg_rows); + } + continue; + } + let rg_rows = groups[row_group_index].num_rows() as u64; + let mut overall_selection = None; + + let total_pages_in_group = + parquet_metadata.offset_index().map_or(0, |offset_index| { + offset_index[row_group_index] + .first() + .map_or(0, |column| column.page_locations.len()) + }); + // Intersection of per-conjunct matched pages, matching the + // untagged path's behavior so the page-level metric reflects + // the AND of all predicates rather than a per-conjunct sum. + let mut matched_pages_in_group: HashSet = + HashSet::from_iter(0..total_pages_in_group); + + for (i, predicate) in self.predicates.iter().enumerate() { + per_conjunct[i].rows_seen = + per_conjunct[i].rows_seen.saturating_add(rg_rows); + + let column = predicate + .required_columns() + .single_column() + .expect("Page pruning requires single column predicates"); + + let converter = match StatisticsConverter::try_new( + column.name(), + arrow_schema, + parquet_schema, + ) { + Ok(c) => c, + Err(e) => { + debug!( + "Could not create statistics converter for column {}: {e}", + column.name() + ); + continue; + } + }; + + let selection = prune_pages_in_one_row_group( + row_group_index, + predicate, + converter, + parquet_metadata, + file_metrics, + ); + + let Some((selection, page_match_flags)) = selection else { + continue; + }; + let matched_pages_indexes: HashSet<_> = page_match_flags + .into_iter() + .enumerate() + .filter(|x| x.1) + .map(|x| x.0) + .collect(); + matched_pages_in_group.retain(|x| matched_pages_indexes.contains(x)); + + // Per-conjunct skipped rows for this row group: anything + // the predicate's selection didn't include is something + // this conjunct alone proved skippable. + let kept_rows_for_conjunct = selection.row_count() as u64; + let skipped_rows_for_conjunct = + rg_rows.saturating_sub(kept_rows_for_conjunct); + per_conjunct[i].rows_skipped = per_conjunct[i] + .rows_skipped + .saturating_add(skipped_rows_for_conjunct); + + overall_selection = update_selection(overall_selection, selection); + + let selects_any = overall_selection + .as_ref() + .map(|sel| sel.selects_any()) + .unwrap_or(true); + if !selects_any { + break; + } + } + + let pages_matched = matched_pages_in_group.len(); + total_pages_select += pages_matched; + total_pages_skip += total_pages_in_group - pages_matched; + + if let Some(overall_selection) = overall_selection { + let rows_selected = overall_selection.row_count(); + if rows_selected > 0 { + let rows_skipped = overall_selection.skipped_row_count(); + total_skip += rows_skipped; + total_select += rows_selected; + access_plan.scan_selection(row_group_index, overall_selection); + } else { + let rows_skipped = groups[row_group_index].num_rows() as usize; + access_plan.skip(row_group_index); + total_skip += rows_skipped; + } + } + } + + file_metrics.page_index_rows_pruned.add_pruned(total_skip); + file_metrics + .page_index_rows_pruned + .add_matched(total_select); + file_metrics + .page_index_pages_pruned + .add_pruned(total_pages_skip); + file_metrics + .page_index_pages_pruned + .add_matched(total_pages_select); + + ( + access_plan, + per_conjunct, + total_pages_skipped_by_fully_matched, + ) + } } fn update_selection( @@ -333,6 +606,19 @@ fn update_selection( } } +/// Returns the number of pages for which page-index pruning is skipped because +/// the containing row group is fully matched by row-group statistics. +fn fully_matched_page_count( + row_group_index: usize, + parquet_metadata: &ParquetMetaData, +) -> usize { + parquet_metadata.offset_index().map_or(0, |offset_index| { + offset_index[row_group_index] + .first() + .map_or(0, |column| column.page_locations.len()) + }) +} + /// Returns a [`RowSelection`] for the rows in this row group to scan, in addition to a vec of /// booleans that state if each page was matched (true) or not (false). /// diff --git a/datafusion/datasource-parquet/src/row_filter.rs b/datafusion/datasource-parquet/src/row_filter.rs index 6dfaa731ae7f9..f7d070b9e062e 100644 --- a/datafusion/datasource-parquet/src/row_filter.rs +++ b/datafusion/datasource-parquet/src/row_filter.rs @@ -65,6 +65,7 @@ //! - `WHERE s['value'] > 5` — pushed down (accesses a primitive leaf) //! - `WHERE s IS NOT NULL` — not pushed down (references the whole struct) +use log::debug; use std::collections::BTreeSet; use std::sync::Arc; @@ -81,10 +82,10 @@ use parquet::schema::types::SchemaDescriptor; use datafusion_common::Result; use datafusion_common::cast::as_boolean_array; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::ScalarFunctionExpr; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::{collect_columns, reassign_expr_columns}; -use datafusion_physical_expr::{PhysicalExpr, split_conjunction}; use datafusion_physical_plan::metrics; @@ -119,18 +120,49 @@ pub(crate) struct DatafusionArrowPredicate { rows_matched: metrics::Count, /// how long was spent evaluating this predicate time: metrics::Time, + /// Stable id used by the adaptive selectivity tracker to key per-filter + /// statistics across files. + filter_id: crate::selectivity::FilterId, + /// Shared handle to the adaptive selectivity tracker. Per-batch stats + /// are reported through `update()` after each `evaluate()` call. + tracker: Arc, + /// Estimated *late-materialization savings* per row for this filter: + /// the compressed bytes of projection columns that the filter does + /// NOT reference, amortised across the file's rows. When a pruned + /// row is dropped by the filter, these are the bytes the reader + /// avoids decoding further along the pipeline — the quantity the + /// adaptive tracker needs in order to rank filters by "cost avoided + /// per unit evaluation time". This MUST match the metric the + /// post-scan path reports in `apply_post_scan_filters_with_stats` + /// (see `opener.rs::post_scan_other_bytes_per_row`); if the two + /// paths disagreed, the tracker would rank row-filter and post-scan + /// candidates on incomparable axes and wrongly promote or demote them. + other_projected_bytes_per_row: f64, + /// Mid-stream "drop" flag, shared with the + /// [`crate::selectivity::SelectivityTracker`]. The tracker flips this + /// when an `OptionalFilterPhysicalExpr` proves CPU-dominated and + /// ineffective; once set, [`Self::evaluate`] returns an all-true mask + /// without invoking `physical_expr`. Filter columns are still decoded + /// (the parquet decoder cannot be reconfigured mid-scan), so this only + /// reclaims CPU, not I/O. Flagged only for filters known to be + /// optional, so correctness is preserved by the join itself. + skip_flag: Arc, } impl DatafusionArrowPredicate { - /// Create a new `DatafusionArrowPredicate` from a `FilterCandidate` + /// Create a new `DatafusionArrowPredicate` from a `FilterCandidate`. pub fn try_new( candidate: FilterCandidate, rows_pruned: metrics::Count, rows_matched: metrics::Count, time: metrics::Time, + filter_id: crate::selectivity::FilterId, + tracker: Arc, + other_projected_bytes_per_row: f64, ) -> Result { let physical_expr = reassign_expr_columns(candidate.expr, &candidate.read_plan.projected_schema)?; + let skip_flag = tracker.skip_flag(filter_id); Ok(Self { physical_expr, @@ -138,6 +170,10 @@ impl DatafusionArrowPredicate { rows_pruned, rows_matched, time, + filter_id, + tracker, + other_projected_bytes_per_row, + skip_flag, }) } } @@ -148,10 +184,27 @@ impl ArrowPredicate for DatafusionArrowPredicate { } fn evaluate(&mut self, batch: RecordBatch) -> ArrowResult { + // Mid-stream drop: the tracker has decided this optional filter is + // pulling its weight no longer. Return an all-true mask to bypass + // expression evaluation entirely. We still bump `rows_matched` so + // the per-predicate count stays consistent with input rows; the + // tracker is intentionally NOT updated for skipped batches because + // (a) we have nothing meaningful to report and (b) flooding it + // with zero-cost samples would mask the underlying effectiveness + // signal if the flag is ever cleared. + if self.skip_flag.load(std::sync::atomic::Ordering::Acquire) { + let rows_in_batch = batch.num_rows(); + self.rows_matched.add(rows_in_batch); + return Ok(BooleanArray::from(vec![true; rows_in_batch])); + } + // scoped timer updates on drop let mut timer = self.time.timer(); + let start_nanos = datafusion_common::instant::Instant::now(); - self.physical_expr + let rows_in_batch = batch.num_rows(); + let result = self + .physical_expr .evaluate(&batch) .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { @@ -161,13 +214,42 @@ impl ArrowPredicate for DatafusionArrowPredicate { self.rows_pruned.add(num_pruned); self.rows_matched.add(num_matched); timer.stop(); - Ok(bool_arr) + Ok((bool_arr, num_matched)) }) .map_err(|e| { ArrowError::ComputeError(format!( "Error evaluating filter predicate: {e:?}" )) - }) + }); + + match result { + Ok((bool_arr, num_matched)) => { + let eval_nanos = start_nanos.elapsed().as_nanos() as u64; + // Scatter-aware skippable bytes: same units as the + // post-scan path (see `apply_post_scan_filters_with_stats`). + // At row-level this is a conservative *measurement* of + // what the decoder skipped — it counts only fully-empty + // sub-windows and ignores the additional savings from + // within-window RowSelection narrowing, which biases + // the demote-or-not decision in the safe direction. + let total_other_bytes = (rows_in_batch as f64 + * self.other_projected_bytes_per_row) + .round() as u64; + let skippable_bytes = crate::selectivity::count_skippable_bytes( + &bool_arr, + total_other_bytes, + ); + self.tracker.update( + self.filter_id, + num_matched as u64, + rows_in_batch as u64, + eval_nanos, + skippable_bytes, + ); + Ok(bool_arr) + } + Err(e) => Err(e), + } } } @@ -991,95 +1073,167 @@ fn size_of_columns(columns: &[usize], metadata: &ParquetMetaData) -> Result)>; + +/// Build row-level filters for the row-filter partition chosen by the +/// adaptive selectivity tracker. /// -/// # Arguments -/// * `expr` - The filter predicate, already adapted to reference columns in `file_schema` -/// * `file_schema` - The Arrow schema of the parquet file (the result of converting -/// the parquet schema to Arrow, potentially with type coercions applied) -/// * `metadata` - Parquet file metadata used for cost estimation -/// * `reorder_predicates` - If true, reorder predicates to minimize I/O -/// * `file_metrics` - Metrics for tracking filter performance +/// Each input filter keeps its stable filter id so the resulting +/// `ArrowPredicate`s can report per-batch statistics back to the tracker on +/// each `evaluate()` call, driving future promote/demote decisions. /// -/// # Returns -/// * `Ok(Some(row_filter))` if the expression can be used as a RowFilter -/// * `Ok(None)` if the expression cannot be used as a RowFilter -/// * `Err(e)` if an error occurs while building the filter +/// Filters that cannot be represented as an `ArrowPredicate` (e.g. whole +/// struct references or other unsupported patterns) are returned in the +/// second element of the returned tuple so the opener can apply them +/// post-scan instead of silently dropping them. /// -/// Note: The returned `RowFilter` may not contain all conjuncts from the original -/// expression. Conjuncts that cannot be evaluated as an `ArrowPredicate` are ignored. +/// # Arguments +/// * `filters` — The candidate filters paired with their stable ids. Assumed +/// to already be adapted to reference columns in `file_schema`. +/// * `file_schema` — The Arrow schema of the parquet file. +/// * `metadata` — Parquet file metadata used for cost estimation. +/// * `projection_compressed_bytes` — Total compressed bytes the user +/// projection reads across the file. Used to derive the per-filter +/// *late-materialization savings* reported to the tracker, so that +/// row-filter and post-scan candidates are ranked on a single common +/// axis. +/// * `tracker` — Shared adaptive selectivity tracker. +/// * `file_metrics` — Metrics for tracking filter performance. /// -/// For example, if the expression is `a = 1 AND b = 2 AND c = 3` and `b = 2` -/// cannot be evaluated for some reason, the returned `RowFilter` will contain -/// only `a = 1` and `c = 3`. +/// # Returns +/// * `Ok((Some(row_filter), unbuildable))` when at least one filter could be +/// represented as a row-level predicate. +/// * `Ok((None, unbuildable))` when no filters could be represented as row +/// filters; all are returned in `unbuildable`. +/// * `Err(e)` if an error occurs while building the filter. pub fn build_row_filter( - expr: &Arc, + filters: &[(crate::selectivity::FilterId, Arc)], file_schema: &SchemaRef, metadata: &ParquetMetaData, - reorder_predicates: bool, + projection_compressed_bytes: usize, + tracker: &Arc, file_metrics: &ParquetFileMetrics, -) -> Result> { +) -> Result<(Option, UnbuildableFilters)> { let rows_pruned = &file_metrics.pushdown_rows_pruned; let rows_matched = &file_metrics.pushdown_rows_matched; let time = &file_metrics.row_pushdown_eval_time; - // Split into conjuncts: - // `a = 1 AND b = 2 AND c = 3` -> [`a = 1`, `b = 2`, `c = 3`] - let predicates = split_conjunction(expr); - - // Determine which conjuncts can be evaluated as ArrowPredicates, if any - let mut candidates: Vec = predicates - .into_iter() - .map(|expr| { - FilterCandidateBuilder::new(Arc::clone(expr), Arc::clone(file_schema)) - .build(metadata) - }) - .collect::, _>>()? - .into_iter() - .flatten() - .collect(); - - // no candidates - if candidates.is_empty() { - return Ok(None); + // Total rows in the file, used to amortise compressed-byte totals + // over rows. We floor at 1 so empty files don't divide by zero; any + // effectiveness contribution in that degenerate case is irrelevant + // because there are no batches to track anyway. + let total_rows: i64 = metadata.row_groups().iter().map(|rg| rg.num_rows()).sum(); + let total_rows_f = total_rows.max(1) as f64; + + // Try to build a candidate for each filter independently. Any filter + // that can't be represented as an `ArrowPredicate`, for *any* reason + // (the candidate builder returned `None`, the builder returned an + // `Err`, or the `DatafusionArrowPredicate` constructor failed below), + // falls through into `unbuildable` so the caller can apply it + // post-scan. Silently dropping any conjunct here would relax the + // user's predicate and return wrong results — see the + // `post_scan_conjuncts` fallthrough in + // `ParquetOpener::build_stream`. + let mut buildable: Vec<(crate::selectivity::FilterId, FilterCandidate)> = + Vec::with_capacity(filters.len()); + let mut unbuildable: UnbuildableFilters = Vec::new(); + for (id, expr) in filters { + match FilterCandidateBuilder::new(Arc::clone(expr), Arc::clone(file_schema)) + .build(metadata) + { + Ok(Some(c)) => buildable.push((*id, c)), + Ok(None) => unbuildable.push((*id, Arc::clone(expr))), + Err(e) => { + debug!( + "failed to build row-filter candidate for {id}: {e}; falling through to post-scan" + ); + unbuildable.push((*id, Arc::clone(expr))); + } + } } - if reorder_predicates { - candidates.sort_unstable_by_key(|c| c.required_bytes); + if buildable.is_empty() { + return Ok((None, unbuildable)); } // To avoid double-counting metrics when multiple predicates are used: - // - All predicates should count rows_pruned (cumulative pruned rows) - // - Only the last predicate should count rows_matched (final result) - // This ensures: rows_matched + rows_pruned = total rows processed - let total_candidates = candidates.len(); - - candidates - .into_iter() - .enumerate() - .map(|(idx, candidate)| { - let is_last = idx == total_candidates - 1; - - // All predicates share the pruned counter (cumulative) - let predicate_rows_pruned = rows_pruned.clone(); - - // Only the last predicate tracks matched rows (final result) - let predicate_rows_matched = if is_last { - rows_matched.clone() - } else { - metrics::Count::new() - }; + // - All predicates share the cumulative rows_pruned counter + // - Only the last predicate writes to rows_matched (final pass count) + // This preserves the invariant: rows_matched + rows_pruned = total rows. + let total_candidates = buildable.len(); + + let mut predicates: Vec> = + Vec::with_capacity(total_candidates); + for (idx, (filter_id, candidate)) in buildable.into_iter().enumerate() { + let is_last = idx == total_candidates - 1; + let predicate_rows_pruned = rows_pruned.clone(); + let predicate_rows_matched = if is_last { + rows_matched.clone() + } else { + metrics::Count::new() + }; + // Late-materialization savings: bytes of the *non-filter* portion + // of the projection, per row. When the filter prunes a row, the + // decoder avoids decoding these bytes further downstream — that + // is the quantity the tracker needs as `batch_bytes` so its + // effectiveness metric (bytes-saved / eval-time) ranks filters + // by actual savings rather than by their own read cost. Match the + // post-scan path's formula in + // `opener.rs::post_scan_other_bytes_per_row`. + let other_projected_bytes_per_row = + projection_compressed_bytes.saturating_sub(candidate.required_bytes) as f64 + / total_rows_f; + // Remember the original expression before we move `candidate` into + // `try_new`, so that a failed predicate construction can fall back + // into `unbuildable` rather than being silently dropped. + let original_expr = Arc::clone(&candidate.expr); + match DatafusionArrowPredicate::try_new( + candidate, + predicate_rows_pruned, + predicate_rows_matched, + time.clone(), + filter_id, + Arc::clone(tracker), + other_projected_bytes_per_row, + ) { + Ok(pred) => predicates.push(Box::new(pred) as _), + Err(e) => { + debug!( + "failed to construct ArrowPredicate for filter {filter_id}: {e}; \ + falling through to post-scan" + ); + unbuildable.push((filter_id, original_expr)); + } + } + } - DatafusionArrowPredicate::try_new( - candidate, - predicate_rows_pruned, - predicate_rows_matched, - time.clone(), - ) - .map(|pred| Box::new(pred) as _) - }) - .collect::, _>>() - .map(|filters| Some(RowFilter::new(filters))) + if predicates.is_empty() { + Ok((None, unbuildable)) + } else { + Ok((Some(RowFilter::new(predicates)), unbuildable)) + } +} + +/// Estimate the total on-disk (compressed) byte cost of reading the given +/// leaf column indices across every row group in the file. Used by the +/// adaptive [`crate::selectivity::SelectivityTracker`] as a cheap proxy for +/// filter evaluation cost before runtime stats are available. +pub(crate) fn total_compressed_bytes( + column_indices: &[usize], + metadata: &ParquetMetaData, +) -> usize { + let mut total: i64 = 0; + for rg in metadata.row_groups() { + for &idx in column_indices { + if let Some(col) = rg.columns().get(idx) { + total += col.compressed_size(); + } + } + } + total.max(0) as usize } #[cfg(test)] @@ -1183,11 +1337,15 @@ mod test { .expect("building candidate") .expect("candidate expected"); + let test_tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); let mut row_filter = DatafusionArrowPredicate::try_new( candidate, Count::new(), Count::new(), Time::new(), + 0, + Arc::clone(&test_tracker), + 0.0, ) .expect("creating filter predicate"); @@ -1222,11 +1380,15 @@ mod test { .expect("building candidate") .expect("candidate expected"); + let test_tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); let mut row_filter = DatafusionArrowPredicate::try_new( candidate, Count::new(), Count::new(), Time::new(), + 0, + Arc::clone(&test_tracker), + 0.0, ) .expect("creating filter predicate"); @@ -1371,10 +1533,18 @@ mod test { let file_metrics = ParquetFileMetrics::new(0, &format!("{func_name}.parquet"), &metrics); - let row_filter = - build_row_filter(&expr, &file_schema, &metadata, false, &file_metrics) - .expect("building row filter") - .expect("row filter should exist"); + let tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); + let filters = vec![(0usize, expr)]; + let (row_filter, _unbuildable) = build_row_filter( + &filters, + &file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + ) + .expect("building row filter"); + let row_filter = row_filter.expect("row filter should exist"); let reader = parquet_reader_builder .with_row_filter(row_filter) @@ -1949,10 +2119,18 @@ mod test { let metrics = ExecutionPlanMetricsSet::new(); let file_metrics = ParquetFileMetrics::new(0, "struct_e2e.parquet", &metrics); - let row_filter = - build_row_filter(&expr, &file_schema, &metadata, false, &file_metrics) - .expect("building row filter") - .expect("row filter should exist"); + let tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); + let filters = vec![(0usize, expr)]; + let (row_filter, _unbuildable) = build_row_filter( + &filters, + &file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + ) + .expect("building row filter"); + let row_filter = row_filter.expect("row filter should exist"); let reader = parquet_reader_builder .with_row_filter(row_filter) diff --git a/datafusion/datasource-parquet/src/row_group_filter.rs b/datafusion/datasource-parquet/src/row_group_filter.rs index 9eb9024516ebe..ab0eec2f4eb3b 100644 --- a/datafusion/datasource-parquet/src/row_group_filter.rs +++ b/datafusion/datasource-parquet/src/row_group_filter.rs @@ -45,20 +45,13 @@ use parquet::{bloom_filter::Sbbf, file::metadata::RowGroupMetaData}; pub struct RowGroupAccessPlanFilter { /// which row groups should be accessed access_plan: ParquetAccessPlan, - /// Row groups where ALL rows are known to match the pruning predicate - /// (the predicate does not filter any rows) - is_fully_matched: Vec, } impl RowGroupAccessPlanFilter { /// Create a new `RowGroupPlanBuilder` for pruning out the groups to scan /// based on metadata and statistics pub fn new(access_plan: ParquetAccessPlan) -> Self { - let num_row_groups = access_plan.len(); - Self { - access_plan, - is_fully_matched: vec![false; num_row_groups], - } + Self { access_plan } } /// Return true if there are no row groups @@ -76,14 +69,14 @@ impl RowGroupAccessPlanFilter { self.access_plan.row_group_index_iter() } - /// Returns the inner access plan + /// Returns the inner access plan. pub fn build(self) -> ParquetAccessPlan { self.access_plan } - /// Returns the is_fully_matched vector + /// Returns the is_fully_matched vector. pub fn is_fully_matched(&self) -> &Vec { - &self.is_fully_matched + self.access_plan.fully_matched() } /// Prunes the access plan based on the limit and fully contained row groups. @@ -186,7 +179,7 @@ impl RowGroupAccessPlanFilter { // Iterate through the currently accessible row groups and try to // find a set of matching row groups that can satisfy the limit for &idx in self.access_plan.row_group_indexes().iter() { - if self.is_fully_matched[idx] { + if self.access_plan.is_fully_matched(idx) { let row_group_row_count = rg_metadata[idx].num_rows() as usize; fully_matched_row_group_indexes.push(idx); fully_matched_rows_count += row_group_row_count; @@ -209,6 +202,7 @@ impl RowGroupAccessPlanFilter { let mut new_access_plan = ParquetAccessPlan::new_none(rg_metadata.len()); for &idx in &fully_matched_row_group_indexes { new_access_plan.scan(idx); + new_access_plan.mark_fully_matched(idx); } self.access_plan = new_access_plan; } @@ -259,6 +253,28 @@ impl RowGroupAccessPlanFilter { predicate: &PruningPredicate, metrics: &ParquetFileMetrics, ) { + self.prune_by_statistics_with_per_conjunct_stats( + arrow_schema, + parquet_schema, + groups, + predicate, + metrics, + ); + } + + /// Variant of [`Self::prune_by_statistics`] that also returns + /// per-conjunct pruning stats produced by + /// [`PruningPredicate::prune_per_conjunct`]. Returns an empty + /// `Vec` when the predicate was not constructed with tagged + /// conjuncts, so callers can ignore it on the untagged path. + pub fn prune_by_statistics_with_per_conjunct_stats( + &mut self, + arrow_schema: &Schema, + parquet_schema: &SchemaDescriptor, + groups: &[RowGroupMetaData], + predicate: &PruningPredicate, + metrics: &ParquetFileMetrics, + ) -> Vec { // scoped timer updates on drop let _timer_guard = metrics.statistics_eval_time.timer(); @@ -281,9 +297,14 @@ impl RowGroupAccessPlanFilter { missing_null_counts_as_zero: true, }; - // try to prune the row groups in a single call - match predicate.prune(&pruning_stats) { - Ok(values) => { + let mut per_conjunct: Vec = Vec::new(); + + // try to prune the row groups in a single call (now also captures + // per-conjunct rates when the predicate was built with + // `try_new_tagged_conjuncts`). + match predicate.prune_per_conjunct(&pruning_stats) { + Ok((values, stats)) => { + per_conjunct = stats; let mut fully_contained_candidates_original_idx: Vec = Vec::new(); for (idx, &value) in row_group_indexes.iter().zip(values.iter()) { if !value { @@ -311,6 +332,8 @@ impl RowGroupAccessPlanFilter { metrics.predicate_evaluation_errors.add(1); } } + + per_conjunct } /// Identifies row groups that are fully matched by the predicate. @@ -397,7 +420,7 @@ impl RowGroupAccessPlanFilter { // If the inverted predicate *also* prunes this row group (meaning inverted_values[i] is false), // it implies that *all* rows in this group satisfy the original predicate. if !inverted_values[i] { - self.is_fully_matched[original_row_group_idx] = true; + self.access_plan.mark_fully_matched(original_row_group_idx); metrics.row_groups_pruned_statistics.add_fully_matched(1); } } @@ -408,28 +431,53 @@ impl RowGroupAccessPlanFilter { /// /// Updates this set with row groups that should not be scanned. /// `row_group_bloom_filters[idx]` contains the bloom filters for the - /// parquet row group at index `idx`. + /// parquet row group at index `idx`. Surfaces per-conjunct + /// bloom-filter pruning stats aggregated across row groups, + /// populated when the predicate was built via + /// `PruningPredicate::try_new_tagged_conjuncts`. Empty `Vec` on the + /// untagged path. /// /// # Panics /// if `row_group_bloom_filters` does not have the same number of row groups as this set - pub(crate) fn prune_by_bloom_filters( + pub(crate) fn prune_by_bloom_filters_with_per_conjunct_stats( &mut self, predicate: &PruningPredicate, metrics: &ParquetFileMetrics, row_group_bloom_filters: &[BloomFilterStatistics], - ) { + ) -> Vec { // scoped timer updates on drop let _timer_guard = metrics.bloom_filter_eval_time.timer(); assert_eq!(row_group_bloom_filters.len(), self.access_plan.len()); + + // Per-conjunct accumulators. Indexed by conjunct position in + // the predicate's `sub_predicates` order. + let mut tags: Vec> = Vec::new(); + let mut seen: Vec = Vec::new(); + let mut pruned: Vec = Vec::new(); + for (idx, stats) in row_group_bloom_filters.iter().enumerate() { if !self.access_plan.should_scan(idx) { continue; } // Can this group be pruned? - let prune_group = match predicate.prune(stats) { - Ok(values) => !values[0], + let prune_group = match predicate.prune_per_conjunct(stats) { + Ok((values, per_conjunct)) => { + // Initialize tags/accumulators on first row group. + if tags.is_empty() && !per_conjunct.is_empty() { + tags = per_conjunct.iter().map(|s| s.tag).collect(); + seen = vec![0; per_conjunct.len()]; + pruned = vec![0; per_conjunct.len()]; + } + for (i, sub) in per_conjunct.iter().enumerate() { + if i < seen.len() { + seen[i] += sub.containers_seen; + pruned[i] += sub.containers_pruned; + } + } + !values[0] + } Err(e) => { log::debug!( "Error evaluating row group predicate on bloom filter: {e}" @@ -446,6 +494,15 @@ impl RowGroupAccessPlanFilter { metrics.row_groups_pruned_bloom_filter.add_matched(1); } } + + tags.into_iter() + .zip(seen.into_iter().zip(pruned)) + .map(|(tag, (s, p))| datafusion_pruning::PerConjunctPruneStats { + tag, + containers_seen: s, + containers_pruned: p, + }) + .collect() } } @@ -613,11 +670,11 @@ impl PruningStatistics for BloomFilterStatistics { } /// Wraps a slice of [`RowGroupMetaData`] in a way that implements [`PruningStatistics`] -struct RowGroupPruningStatistics<'a> { - parquet_schema: &'a SchemaDescriptor, - row_group_metadatas: Vec<&'a RowGroupMetaData>, - arrow_schema: &'a Schema, - missing_null_counts_as_zero: bool, +pub(crate) struct RowGroupPruningStatistics<'a> { + pub(crate) parquet_schema: &'a SchemaDescriptor, + pub(crate) row_group_metadatas: Vec<&'a RowGroupMetaData>, + pub(crate) arrow_schema: &'a Schema, + pub(crate) missing_null_counts_as_zero: bool, } impl<'a> RowGroupPruningStatistics<'a> { @@ -1918,7 +1975,7 @@ mod tests { } row_group_bloom_filters[idx] = BloomFilterStatistics { column_sbbf }; } - pruned_row_groups.prune_by_bloom_filters( + pruned_row_groups.prune_by_bloom_filters_with_per_conjunct_stats( pruning_predicate, &file_metrics, &row_group_bloom_filters, diff --git a/datafusion/datasource-parquet/src/selectivity.rs b/datafusion/datasource-parquet/src/selectivity.rs new file mode 100644 index 0000000000000..81a5fe518eb05 --- /dev/null +++ b/datafusion/datasource-parquet/src/selectivity.rs @@ -0,0 +1,2610 @@ +// 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. + +//! Adaptive filter selectivity tracking for Parquet row filters. +//! +//! See [`SelectivityTracker`] for the main entry point, `FilterState` for the +//! per-filter lifecycle, `PartitionedFilters` for the output consumed by +//! `ParquetOpener::open`, and [`FilterId`] for stable filter identification. + +use arrow::array::BooleanArray; +use arrow::datatypes::SchemaRef; +use log::debug; +use parking_lot::{Mutex, RwLock}; +use parquet::file::metadata::ParquetMetaData; +use parquet::schema::types::SchemaDescriptor; +use std::collections::HashMap; +use std::sync::Arc; +use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; + +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr_common::physical_expr::{ + OptionalFilterPhysicalExpr, PhysicalExpr, snapshot_generation, +}; + +/// Window size for the per-batch scatter analysis fed to +/// [`count_skippable_bytes`]. Approximates a parquet data page so that +/// "windows with zero survivors" tracks "pages a row-level decoder +/// could skip". Hardcoded for now; making this configurable (or +/// deriving it from per-row-group page metadata) is a natural follow-up. +pub(crate) const SKIP_WINDOW_ROWS: usize = 8192; + +/// Compute the bytes that late-materialization can plausibly skip for a +/// batch given the predicate output `bool_arr` and the total non-filter +/// projection bytes for that batch. +/// +/// Splits `bool_arr` into [`SKIP_WINDOW_ROWS`]-sized windows; each window +/// with zero survivors represents a page-sized chunk whose +/// other-projection columns the row-level decoder can skip outright. +/// Returns `total_other_bytes × (empty_windows / total_windows)` — +/// scatter-discounted skippable bytes. +/// +/// Interpretation depends on which side calls this: +/// +/// - **Post-scan path**: a *prediction* of bytes-saved-per-sec the +/// row-level path would achieve. The bool_arr we see is over the wide +/// batch in the same row order the decoder would emit, so for single- +/// predicate filters the prediction is faithful (modulo `W` matching +/// the actual parquet page size). +/// +/// - **Row-level path**: a conservative *measurement* of what the +/// decoder actually skipped — within-window RowSelection narrowing is +/// an additional uncounted bonus. So at row-level this is a *lower +/// bound* of real savings, which is the safe direction for the +/// demote-or-not decision. +pub(crate) fn count_skippable_bytes( + bool_arr: &BooleanArray, + total_other_bytes: u64, +) -> u64 { + let n = bool_arr.len(); + if n == 0 || total_other_bytes == 0 { + return 0; + } + // Short-circuit on the two extremes: avoids a redundant per-window + // SIMD scan over the same buffer when the answer is already + // determined by the batch-level total. The whole helper otherwise + // costs ~2× per-batch `true_count` for nothing. + let total_matched = bool_arr.true_count(); + if total_matched == 0 { + // Every window empty: full skippable. + return total_other_bytes; + } + if total_matched == n { + // No window empty: nothing skippable. + return 0; + } + let total_windows = n.div_ceil(SKIP_WINDOW_ROWS); + if total_windows == 1 { + // One-window batch with mixed matches → not skippable. Avoids + // a wasted slice+`true_count`. + return 0; + } + let mut empty_windows: u64 = 0; + for i in 0..total_windows { + let start = i * SKIP_WINDOW_ROWS; + let len = SKIP_WINDOW_ROWS.min(n - start); + if bool_arr.slice(start, len).true_count() == 0 { + empty_windows += 1; + } + } + ((total_other_bytes as f64 * empty_windows as f64) / total_windows as f64) as u64 +} + +/// Stable identifier for a filter conjunct, assigned by `ParquetSource::with_predicate`. +pub type FilterId = usize; + +/// Per-filter lifecycle state in the adaptive filter system. +/// +/// State transitions: +/// - **(unseen)** → [`RowFilter`](Self::RowFilter) or [`PostScan`](Self::PostScan) +/// on first encounter in [`SelectivityTracker::partition_filters`]. +/// - [`PostScan`](Self::PostScan) → [`RowFilter`](Self::RowFilter) when +/// effectiveness ≥ `min_bytes_per_sec` and enough rows have been observed. +/// - [`RowFilter`](Self::RowFilter) → [`PostScan`](Self::PostScan) when +/// effectiveness is below threshold (mandatory filter). +/// - [`RowFilter`](Self::RowFilter) → [`Dropped`](Self::Dropped) when +/// effectiveness is below threshold and the filter is optional +/// ([`OptionalFilterPhysicalExpr`]). +/// - [`RowFilter`](Self::RowFilter) → [`PostScan`](Self::PostScan)/[`Dropped`](Self::Dropped) +/// on periodic re-evaluation if effectiveness drops below threshold after +/// CI upper bound drops below threshold. +/// - **Any state** → re-evaluated when a dynamic filter's +/// `snapshot_generation` changes. +#[derive(Debug, Clone, Copy, PartialEq)] +pub(crate) enum FilterState { + /// Currently a row filter. + RowFilter, + /// Currently a post-scan filter. + PostScan, + /// Dropped entirely (insufficient throughput and optional). + Dropped, +} + +/// Result of partitioning filters into row filters vs post-scan. +/// +/// Produced by [`SelectivityTracker::partition_filters`], consumed by +/// `ParquetOpener::open` to build row-level predicates and post-scan filters. +/// +/// Filters are partitioned based on their effectiveness threshold. +/// +/// This type is `pub` to support the [selectivity tracker benchmark +/// harness](../../benches/selectivity_tracker.rs); treat the layout as +/// unstable from outside the crate. +#[derive(Debug, Clone, Default)] +#[doc(hidden)] +pub struct PartitionedFilters { + /// Filters promoted past collection — individual chained ArrowPredicates + pub row_filters: Vec<(FilterId, Arc)>, + /// Filters demoted to post-scan (fast path only) + pub post_scan: Vec<(FilterId, Arc)>, +} + +/// Tracks selectivity statistics for a single filter expression. +#[derive(Debug, Clone, Default, Copy, PartialEq)] +struct SelectivityStats { + /// Number of rows that matched (passed) the filter + rows_matched: u64, + /// Total number of rows evaluated + rows_total: u64, + /// Cumulative evaluation time in nanoseconds + eval_nanos: u64, + /// Cumulative bytes across batches this filter has been evaluated on + bytes_seen: u64, + /// Welford's online algorithm: number of per-batch effectiveness samples + sample_count: u64, + /// Welford's online algorithm: running mean of per-batch effectiveness + eff_mean: f64, + /// Welford's online algorithm: running sum of squared deviations (M2) + eff_m2: f64, + /// Whether the underlying expression is wrapped in + /// `OptionalFilterPhysicalExpr`. Cached here (rather than re-checked + /// via [`is_optional_filter`] on every batch) so the per-batch hot + /// path in [`SelectivityTracker::update`] can skip the + /// SKIP_FLAG/CI-bound work entirely for non-optional filters with a + /// single field load on the already-held stats lock — no extra + /// HashMap or `RwLock::read()` per batch. + is_optional: bool, +} + +impl SelectivityStats { + /// Returns the cumulative effectiveness as an opaque ordering score + /// (higher = run first). + /// + /// Computed from `eff_mean` so it matches the Welford-tracked metric + /// fed to CI bounds: per-batch scatter-aware bytes-saved-per-second. + /// Callers should not assume the unit. + fn effectiveness(&self) -> Option { + if self.sample_count == 0 { + return None; + } + Some(self.eff_mean) + } + + /// Returns the lower bound of a confidence interval on mean effectiveness. + /// + /// Uses Welford's online variance to compute a one-sided CI: + /// `mean - z * stderr`. Returns `None` if fewer than 2 samples. + fn confidence_lower_bound(&self, confidence_z: f64) -> Option { + if self.sample_count < 2 { + return None; + } + let variance = self.eff_m2 / (self.sample_count - 1) as f64; + let stderr = (variance / self.sample_count as f64).sqrt(); + Some(self.eff_mean - confidence_z * stderr) + } + + /// Returns the upper bound of a confidence interval on mean effectiveness. + /// + /// Uses Welford's online variance: `mean + z * stderr`. + /// Returns `None` if fewer than 2 samples. + fn confidence_upper_bound(&self, confidence_z: f64) -> Option { + if self.sample_count < 2 { + return None; + } + let variance = self.eff_m2 / (self.sample_count - 1) as f64; + let stderr = (variance / self.sample_count as f64).sqrt(); + Some(self.eff_mean + confidence_z * stderr) + } + + /// Update stats with new observations. + /// + /// `skippable_bytes` is the caller's already-computed estimate of + /// non-filter projection bytes that late-materialization would + /// actually save for this batch — see [`count_skippable_bytes`] for + /// the windowed scatter calculation. The Welford accumulator tracks + /// `skippable_bytes × 1e9 / eval_nanos` (= scatter-aware + /// bytes-saved-per-second), which is what the promote/demote + /// gates compare against `min_bytes_per_sec`. + fn update( + &mut self, + matched: u64, + total: u64, + eval_nanos: u64, + skippable_bytes: u64, + ) { + self.rows_matched += matched; + self.rows_total += total; + self.eval_nanos += eval_nanos; + self.bytes_seen += skippable_bytes; + + if total > 0 && eval_nanos > 0 { + let batch_eff = skippable_bytes as f64 * 1e9 / eval_nanos as f64; + + self.sample_count += 1; + let delta = batch_eff - self.eff_mean; + self.eff_mean += delta / self.sample_count as f64; + let delta2 = batch_eff - self.eff_mean; + self.eff_m2 += delta * delta2; + } + } +} + +/// Immutable configuration for a [`SelectivityTracker`]. +/// +/// Use the builder methods to customise, then call [`build()`](TrackerConfig::build) +/// to produce a ready-to-use tracker. +#[doc(hidden)] +pub struct TrackerConfig { + /// Minimum bytes/sec throughput for promoting a filter (default: INFINITY = disabled). + pub min_bytes_per_sec: f64, + /// Byte-ratio threshold for initial filter placement (row-level vs post-scan). + /// Computed as `filter_compressed_bytes / projection_compressed_bytes`. + /// When low, the filter columns are small relative to the projection, + /// so row-level placement enables large late-materialization savings. + /// When high, the filter columns dominate the projection, so there's + /// little benefit from late materialization. + /// Default is 0.20. + pub byte_ratio_threshold: f64, + /// Z-score for confidence intervals on filter effectiveness. + /// Lower values (e.g. 1.0 or 0.0) will make the tracker more aggressive about promotion/demotion based on limited data. + /// Higher values (e.g. 3.0) will require more confidence before changing filter states. + /// Default is 2.0, corresponding to ~97.5% one-sided confidence. + /// Set to <= 0.0 to disable confidence intervals and promote/demote based on point estimates alone (not recommended). + /// Set to INFINITY to disable promotion entirely (overrides `min_bytes_per_sec`). + pub confidence_z: f64, + /// Initial-placement prior threshold: if per-conjunct row-group + /// statistics pruning prunes ≥ this fraction of the file's row + /// groups, place the filter at row-level on first encounter. Set + /// to >1.0 to disable the prior. Default 0.5. + pub prior_promote_threshold: f64, + /// Initial-placement prior threshold: if per-conjunct row-group + /// statistics pruning prunes ≤ this fraction of the file's row + /// groups, place the filter at post-scan on first encounter. Set + /// to <0.0 to disable the prior. Default 0.05. + pub prior_demote_threshold: f64, + /// Per-fetch latency baseline in milliseconds — at this average + /// per-fetch RTT the tracker uses the unmodified `confidence_z`. + /// Above this, `confidence_z` is shrunk proportionally so the + /// tracker becomes more aggressive about state changes when + /// per-request cost is high. 0.0 disables. Default 5.0. + pub latency_z_baseline_ms: f64, + /// Maximum scale factor for the latency-aware z shrink. Default 8.0. + pub latency_z_max_scale: f64, +} + +impl TrackerConfig { + pub fn new() -> Self { + Self { + min_bytes_per_sec: f64::INFINITY, + byte_ratio_threshold: 0.20, + confidence_z: 2.0, + prior_promote_threshold: 0.5, + prior_demote_threshold: 0.05, + latency_z_baseline_ms: 5.0, + latency_z_max_scale: 8.0, + } + } + + pub fn with_min_bytes_per_sec(mut self, v: f64) -> Self { + self.min_bytes_per_sec = v; + self + } + + pub fn with_byte_ratio_threshold(mut self, v: f64) -> Self { + self.byte_ratio_threshold = v; + self + } + + pub fn with_confidence_z(mut self, v: f64) -> Self { + self.confidence_z = v; + self + } + + pub fn with_prior_promote_threshold(mut self, v: f64) -> Self { + self.prior_promote_threshold = v; + self + } + + pub fn with_prior_demote_threshold(mut self, v: f64) -> Self { + self.prior_demote_threshold = v; + self + } + + pub fn with_latency_z_baseline_ms(mut self, v: f64) -> Self { + self.latency_z_baseline_ms = v; + self + } + + pub fn with_latency_z_max_scale(mut self, v: f64) -> Self { + self.latency_z_max_scale = v; + self + } + + pub fn build(self) -> SelectivityTracker { + SelectivityTracker { + config: self, + filter_stats: RwLock::new(HashMap::new()), + skip_flags: RwLock::new(HashMap::new()), + inner: Mutex::new(SelectivityTrackerInner::new()), + total_fetch_ns: AtomicU64::new(0), + total_fetches: AtomicU64::new(0), + } + } +} + +impl Default for TrackerConfig { + fn default() -> Self { + Self::new() + } +} + +/// Cross-file adaptive system that measures filter effectiveness and decides +/// which filters are promoted to row-level predicates (pushed into the Parquet +/// reader) vs. applied post-scan (demoted) or dropped entirely. +/// +/// # Locking design +/// +/// All locks are **private** to this struct — external callers cannot hold a +/// guard across expensive work, and all lock-holding code paths are auditable +/// in this file alone. +/// +/// State is split across two independent locks to minimise contention between +/// the hot per-batch `update()` path and the cold per-file-open +/// `partition_filters()` path: +/// +/// - **`filter_stats`** (`RwLock>>`) +/// — `update()` acquires a *shared read* lock on the outer map, then a +/// per-filter `Mutex` to increment counters. Multiple threads updating +/// *different* filters never contend at all; threads updating the *same* +/// filter serialize only on the cheap per-filter `Mutex` (~100 ns). +/// `partition_filters()` also takes a read lock here when it needs to +/// inspect stats for promotion/demotion decisions, so it never blocks +/// `update()` callers. The write lock is taken only briefly in Phase 2 +/// of `partition_filters()` to insert entries for newly-seen filter IDs. +/// +/// - **`inner`** (`Mutex`) — holds the filter +/// state-machine (`filter_states`) and dynamic-filter generation tracking. +/// Only `partition_filters()` acquires this lock (once per file open), so +/// concurrent `update()` calls are completely unaffected. +/// +/// ## Lock ordering (deadlock-free) +/// +/// Locks are always acquired in the order `inner` → `filter_stats` → +/// per-filter `Mutex`. Because `update()` never acquires `inner`, no +/// cycle is possible. +/// +/// ## Correctness of concurrent access +/// +/// `update()` may write stats while `partition_filters()` reads them for +/// promotion/demotion. Both hold a shared `filter_stats` read lock; the +/// per-filter `Mutex` ensures they do not interleave on the same filter's +/// stats. One proceeds first; the other sees a consistent (slightly newer +/// or older) snapshot. This is benign — the single-lock design that +/// preceded this split already allowed stats to change between consecutive +/// reads within `partition_filters()`. +/// +/// On promote/demote, `partition_filters()` zeros a filter's stats via the +/// per-filter `Mutex`. An `update()` running concurrently may write one +/// stale batch's worth of data to the freshly-zeroed stats; this is quickly +/// diluted by hundreds of correct-context batches and is functionally +/// identical to the old design where `update()` queued behind the write +/// lock and ran immediately after. +/// +/// # Filter state machine +/// +/// ```text +/// ┌─────────┐ +/// │ New │ +/// └─────────┘ +/// │ +/// ▼ +/// ┌────────────────────────┐ +/// │ Estimated Cost │ +/// │Bytes needed for filter │ +/// └────────────────────────┘ +/// │ +/// ┌──────────────────┴──────────────────┐ +/// ┌────────▼────────┐ ┌────────▼────────┐ +/// │ Post-scan │ │ Row filter │ +/// │ │ │ │ +/// └─────────────────┘ └─────────────────┘ +/// │ │ +/// ▼ ▼ +/// ┌─────────────────┐ ┌─────────────────┐ +/// │ Effectiveness │ │ Effectiveness │ +/// │ Bytes pruned │ │ Bytes pruned │ +/// │ per │ │ per │ +/// │Second of compute│ │Second of compute│ +/// └─────────────────┘ └─────────────────┘ +/// │ │ +/// └──────────────────┬──────────────────┘ +/// ▼ +/// ┌───────────────────────────────────────────────┐ +/// │ New Scan │ +/// │ Move filters based on effectiveness. │ +/// │ Promote (move post-scan -> row filter). │ +/// │ Demote (move row-filter -> post-scan). │ +/// │ Disable (for optional filters; either row │ +/// │ filter or disabled). │ +/// └───────────────────────────────────────────────┘ +/// │ +/// ┌──────────────────┴──────────────────┐ +/// ┌────────▼────────┐ ┌────────▼────────┐ +/// │ Post-scan │ │ Row filter │ +/// │ │ │ │ +/// └─────────────────┘ └─────────────────┘ +/// ``` +/// +/// See `TrackerConfig` for configuration knobs. +pub struct SelectivityTracker { + config: TrackerConfig, + /// Per-filter selectivity statistics, each individually `Mutex`-protected. + /// + /// The outer `RwLock` is almost always read-locked: both `update()` (hot, + /// per-batch) and `partition_filters()` (cold, per-file-open) only need + /// shared access to look up existing entries. The write lock is taken + /// only when `partition_filters()` inserts entries for newly-seen filter + /// IDs — a brief, infrequent operation. + /// + /// Each inner `Mutex` protects a single filter's + /// counters, so concurrent `update()` calls on *different* filters + /// proceed in parallel with zero contention. + /// Cumulative wall time spent inside `AsyncFileReader::get_byte_ranges` + /// across all openers using this tracker. + total_fetch_ns: AtomicU64, + /// Number of byte-range fetches recorded. + total_fetches: AtomicU64, + filter_stats: RwLock>>, + /// Per-filter "skip" flags — when set, the corresponding filter is + /// treated as a no-op by both the row-filter + /// (`DatafusionArrowPredicate::evaluate`) and the post-scan path + /// (`apply_post_scan_filters_with_stats`). This is the mid-stream + /// equivalent of dropping an optional filter: once the per-batch + /// `update()` path proves an `OptionalFilterPhysicalExpr` is + /// CPU-dominated and ineffective, it flips the flag and subsequent + /// batches stop paying the evaluation cost. The decoder still decodes + /// the filter columns (we cannot rebuild it mid-scan), so I/O is not + /// reclaimed; only the predicate evaluation is skipped. + /// + /// Only ever set for filters whose `is_optional` flag (cached on the + /// per-filter [`SelectivityStats`]) is `true` — mandatory filters + /// must always execute or queries return wrong rows. + skip_flags: RwLock>>, + /// Filter lifecycle state machine and dynamic-filter generation tracking. + /// + /// Only `partition_filters()` acquires this lock (once per file open). + /// `update()` never touches it, so the hot per-batch path is completely + /// decoupled from the cold state-machine path. + inner: Mutex, +} + +impl std::fmt::Debug for SelectivityTracker { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("SelectivityTracker") + .field("config.min_bytes_per_sec", &self.config.min_bytes_per_sec) + .finish() + } +} + +impl Default for SelectivityTracker { + fn default() -> Self { + Self::new() + } +} + +impl SelectivityTracker { + /// Create a new tracker with default settings (feature disabled). + pub fn new() -> Self { + TrackerConfig::new().build() + } + + /// Record one batch of `get_byte_ranges` activity (latency-aware z input). + pub fn record_fetch(&self, ranges: usize, elapsed_ns: u64) { + if ranges == 0 || elapsed_ns == 0 { + return; + } + self.total_fetch_ns.fetch_add(elapsed_ns, Ordering::Relaxed); + self.total_fetches + .fetch_add(ranges as u64, Ordering::Relaxed); + } + + fn avg_fetch_ms(&self) -> f64 { + let fetches = self.total_fetches.load(Ordering::Relaxed); + if fetches == 0 { + return 0.0; + } + let ns = self.total_fetch_ns.load(Ordering::Relaxed) as f64; + ns / fetches as f64 / 1_000_000.0 + } + + fn effective_z(&self) -> f64 { + let z = self.config.confidence_z; + if self.config.latency_z_baseline_ms <= 0.0 { + return z; + } + let avg = self.avg_fetch_ms(); + if avg <= self.config.latency_z_baseline_ms { + return z; + } + let factor = (avg / self.config.latency_z_baseline_ms) + .clamp(1.0, self.config.latency_z_max_scale); + z / factor + } + + /// Update stats for a filter after processing a batch. + /// + /// **Locking:** acquires `filter_stats.read()` (shared) then a per-filter + /// `Mutex`. Never touches `inner`, so this hot per-batch path cannot + /// contend with the cold per-file-open `partition_filters()` path. + /// + /// Silently skips unknown filter IDs (can occur if `update()` is called + /// before `partition_filters()` has registered the filter — in practice + /// this cannot happen because `partition_filters()` runs during file open + /// before any batches are processed). + /// + /// **Mid-stream drop:** after every `SKIP_FLAG_CHECK_INTERVAL`'th batch + /// we evaluate the CI upper bound; if it falls below + /// `min_bytes_per_sec` and the filter is wrapped in + /// `OptionalFilterPhysicalExpr`, we set the per-filter skip flag. + /// Subsequent calls to `DatafusionArrowPredicate::evaluate` (row-level) + /// and `apply_post_scan_filters_with_stats` (post-scan) observe the + /// flag and short-circuit their work for that filter. Mandatory + /// filters are never flagged because doing so would change the result + /// set. + #[doc(hidden)] + pub fn update( + &self, + id: FilterId, + matched: u64, + total: u64, + eval_nanos: u64, + batch_bytes: u64, + ) { + let stats_map = self.filter_stats.read(); + let Some(entry) = stats_map.get(&id) else { + return; + }; + let mut stats = entry.lock(); + stats.update(matched, total, eval_nanos, batch_bytes); + + // Fast path for non-optional filters: nothing else to do. The + // SKIP_FLAG mid-stream drop only applies to + // `OptionalFilterPhysicalExpr`-wrapped filters (hash-join / + // TopK dynamic), and `is_optional` is cached inline on + // `SelectivityStats` at filter registration so this is a single + // field load on the already-held lock. + if !stats.is_optional { + return; + } + + // Optional filter: do the SKIP_FLAG check every batch — there's + // no SKIP_FLAG_CHECK_INTERVAL gate here on purpose. We want + // join/TopK skip flags to fire as soon as stats prove the + // filter's selectivity has collapsed, even mid-row-group. The + // CI-bound calc is cheap arithmetic on already-locked stats. + if !self.config.min_bytes_per_sec.is_finite() { + return; + } + let z = self.effective_z(); + let Some(ub) = stats.confidence_upper_bound(z) else { + return; + }; + if ub >= self.config.min_bytes_per_sec { + return; + } + drop(stats); + drop(stats_map); + + if let Some(flag) = self.skip_flags.read().get(&id) + && !flag.swap(true, Ordering::Release) + { + debug!( + "FilterId {id}: mid-stream skip — CI upper bound {ub} < {} bytes/sec", + self.config.min_bytes_per_sec + ); + } + } + + /// Returns the shared skip flag for `id`, creating one if absent. + /// + /// Cloned into [`crate::row_filter::DatafusionArrowPredicate`] so the + /// row-filter path can short-circuit when the per-batch update path + /// decides the filter has stopped pulling its weight. The post-scan + /// path uses [`Self::is_filter_skipped`] instead — it does not need a + /// long-lived handle. + pub(crate) fn skip_flag(&self, id: FilterId) -> Arc { + if let Some(existing) = self.skip_flags.read().get(&id) { + return Arc::clone(existing); + } + let mut write = self.skip_flags.write(); + Arc::clone( + write + .entry(id) + .or_insert_with(|| Arc::new(AtomicBool::new(false))), + ) + } + + /// Returns `true` when `id` has been mid-stream-dropped by the tracker. + /// + /// Cheap: a single `RwLock::read` plus an atomic load. Called from the + /// post-scan filter loop in `apply_post_scan_filters_with_stats`. + pub(crate) fn is_filter_skipped(&self, id: FilterId) -> bool { + self.skip_flags + .read() + .get(&id) + .is_some_and(|f| f.load(Ordering::Acquire)) + } + + /// Partition filters into row-level predicates vs post-scan filters. + /// + /// Called once per file open (cold path). + /// + /// **Locking — two phases:** + /// 1. Acquires `inner` (exclusive) and `filter_stats` (shared read) for + /// all decision logic — promotion, demotion, initial placement, and + /// sorting by effectiveness. Because `filter_stats` is only + /// read-locked, concurrent `update()` calls proceed unblocked. + /// 2. If new filter IDs were seen, briefly acquires `filter_stats` (write) + /// to insert per-filter `Mutex` entries so that future `update()` calls + /// can find them. + #[doc(hidden)] + #[expect(clippy::too_many_arguments)] + pub fn partition_filters( + &self, + filters: Vec<(FilterId, Arc)>, + projection_columns: &std::collections::HashSet, + projection_scan_size: usize, + metadata: &ParquetMetaData, + arrow_schema: &SchemaRef, + parquet_schema: &SchemaDescriptor, + page_pruning_rates: &HashMap, + ) -> PartitionedFilters { + // Phase 1: inner.lock() + filter_stats.read() → all decision logic + let z_eff = self.effective_z(); + let mut guard = self.inner.lock(); + let stats_map = self.filter_stats.read(); + let result = guard.partition_filters( + filters, + projection_columns, + projection_scan_size, + metadata, + arrow_schema, + parquet_schema, + &self.config, + z_eff, + page_pruning_rates, + &stats_map, + ); + drop(stats_map); + drop(guard); + + // Phase 2: if new filters were seen, briefly acquire write locks + // to insert per-filter `Mutex` (with + // `is_optional` cached inline so the per-batch `update()` hot + // path can fast-return for mandatory filters) and an + // `AtomicBool` skip-flag (only consulted for optional filters). + if !result.new_optional_flags.is_empty() { + let mut stats_write = self.filter_stats.write(); + let mut skip_write = self.skip_flags.write(); + for (id, is_optional) in result.new_optional_flags { + stats_write.entry(id).or_insert_with(|| { + Mutex::new(SelectivityStats { + is_optional, + ..Default::default() + }) + }); + skip_write + .entry(id) + .or_insert_with(|| Arc::new(AtomicBool::new(false))); + } + } + + result.partitioned + } + + /// Test-only convenience that derives `arrow_schema` / `parquet_schema` + /// from the parquet metadata and forwards to the public + /// [`Self::partition_filters`]. Lets test code keep its existing call + /// sites without threading two more arguments through every test. + #[doc(hidden)] + pub fn partition_filters_for_test( + &self, + filters: Vec<(FilterId, Arc)>, + projection_columns: &std::collections::HashSet, + projection_scan_size: usize, + metadata: &ParquetMetaData, + ) -> PartitionedFilters { + let parquet_schema = metadata.file_metadata().schema_descr_ptr(); + let arrow_schema: SchemaRef = match parquet::arrow::parquet_to_arrow_schema( + parquet_schema.as_ref(), + None, + ) { + Ok(s) => Arc::new(s), + Err(_) => Arc::new(arrow::datatypes::Schema::empty()), + }; + self.partition_filters( + filters, + projection_columns, + projection_scan_size, + metadata, + &arrow_schema, + parquet_schema.as_ref(), + &HashMap::new(), + ) + } + + /// Test helper: ensure a stats entry exists for the given filter ID. + /// In production, `partition_filters()` inserts entries for new filters. + /// Tests that call `update()` without prior `partition_filters()` need this. + #[cfg(test)] + fn ensure_stats_entry(&self, id: FilterId) { + let map = self.filter_stats.read(); + if map.get(&id).is_none() { + drop(map); + self.filter_stats + .write() + .entry(id) + .or_insert_with(|| Mutex::new(SelectivityStats::default())); + } + } +} + +/// Internal result from [`SelectivityTrackerInner::partition_filters`]. +/// +/// Carries both the partitioned filters and the `(FilterId, is_optional)` +/// entries seen for the first time, so the outer +/// [`SelectivityTracker::partition_filters`] can insert per-filter +/// `Mutex` entries (with `is_optional` cached inline) +/// in a brief Phase 2 write lock. +struct PartitionResult { + partitioned: PartitionedFilters, + /// `(FilterId, is_optional)` entries observed for the first time in + /// this `partition_filters` call. + new_optional_flags: Vec<(FilterId, bool)>, +} + +/// Filter state-machine and generation tracking, guarded by the `Mutex` +/// inside [`SelectivityTracker`]. +/// +/// This struct intentionally does **not** contain per-filter stats — those +/// live in the separate `filter_stats` lock so that the hot `update()` path +/// can modify stats without acquiring this lock. Only the cold +/// `partition_filters()` path (once per file open) needs this lock. +#[derive(Debug)] +struct SelectivityTrackerInner { + /// Per-filter lifecycle state (RowFilter / PostScan / Dropped). + filter_states: HashMap, + /// Last-seen snapshot generation per filter, for detecting when a dynamic + /// filter's selectivity has changed (e.g. hash-join build side grew). + snapshot_generations: HashMap, +} + +impl SelectivityTrackerInner { + fn new() -> Self { + Self { + filter_states: HashMap::new(), + snapshot_generations: HashMap::new(), + } + } + + /// Check and update the snapshot generation for a filter. + fn note_generation( + &mut self, + id: FilterId, + generation: u64, + stats_map: &HashMap>, + ) { + if generation == 0 { + return; + } + match self.snapshot_generations.get(&id) { + Some(&prev_generation) if prev_generation == generation => {} + Some(_) => { + let current_state = self.filter_states.get(&id).copied(); + // Always reset stats since selectivity changed with new generation. + if let Some(entry) = stats_map.get(&id) { + *entry.lock() = SelectivityStats::default(); + } + self.snapshot_generations.insert(id, generation); + + // Optional/dynamic filters only get more selective over time + // (hash join build side accumulates more values). So if the + // filter was already working (RowFilter or PostScan), preserve + // its state. Only un-drop Dropped filters back to PostScan + // so they get another chance with the new selectivity. + if current_state == Some(FilterState::Dropped) { + debug!("FilterId {id} generation changed, un-dropping to PostScan"); + self.filter_states.insert(id, FilterState::PostScan); + } else { + debug!( + "FilterId {id} generation changed, resetting stats but preserving state {current_state:?}" + ); + } + } + None => { + self.snapshot_generations.insert(id, generation); + } + } + } + + /// Get the effectiveness for a filter by ID. + fn get_effectiveness_by_id( + &self, + id: FilterId, + stats_map: &HashMap>, + ) -> Option { + stats_map + .get(&id) + .and_then(|entry| entry.lock().effectiveness()) + } + + /// Demote a filter to post-scan or drop it entirely if optional. + fn demote_or_drop( + &mut self, + id: FilterId, + expr: &Arc, + post_scan: &mut Vec<(FilterId, Arc)>, + stats_map: &HashMap>, + ) { + if expr.downcast_ref::().is_none() { + self.filter_states.insert(id, FilterState::PostScan); + post_scan.push((id, Arc::clone(expr))); + // Reset stats for this filter so it can be re-evaluated as a post-scan filter. + if let Some(entry) = stats_map.get(&id) { + *entry.lock() = SelectivityStats::default(); + } + } else { + self.filter_states.insert(id, FilterState::Dropped); + } + } + + /// Promote a filter to row-level. + fn promote( + &mut self, + id: FilterId, + expr: Arc, + row_filters: &mut Vec<(FilterId, Arc)>, + stats_map: &HashMap>, + ) { + row_filters.push((id, expr)); + self.filter_states.insert(id, FilterState::RowFilter); + // Reset stats for this filter since it will be evaluated at row-level now. + if let Some(entry) = stats_map.get(&id) { + *entry.lock() = SelectivityStats::default(); + } + } + + /// Partition filters into collecting / promoted / post-scan buckets. + #[expect(clippy::too_many_arguments)] + fn partition_filters( + &mut self, + filters: Vec<(FilterId, Arc)>, + projection_columns: &std::collections::HashSet, + projection_scan_size: usize, + metadata: &ParquetMetaData, + arrow_schema: &SchemaRef, + parquet_schema: &SchemaDescriptor, + config: &TrackerConfig, + z_eff: f64, + page_pruning_rates: &HashMap, + stats_map: &HashMap>, + ) -> PartitionResult { + let mut new_optional_flags: Vec<(FilterId, bool)> = Vec::new(); + + // If min_bytes_per_sec is INFINITY -> all filters are post-scan. + if config.min_bytes_per_sec.is_infinite() { + debug!( + "Filter promotion disabled via min_bytes_per_sec=INFINITY; all {} filters post-scan", + filters.len() + ); + // Register all filter IDs so update() can find them + for (id, expr) in &filters { + if !stats_map.contains_key(id) { + new_optional_flags.push((*id, is_optional_filter(expr))); + } + } + return PartitionResult { + partitioned: PartitionedFilters { + row_filters: Vec::new(), + post_scan: filters, + }, + new_optional_flags, + }; + } + // If min_bytes_per_sec is 0 -> all filters are promoted. + if config.min_bytes_per_sec == 0.0 { + debug!( + "All filters promoted via min_bytes_per_sec=0; all {} filters row-level", + filters.len() + ); + // Register all filter IDs so update() can find them + for (id, expr) in &filters { + if !stats_map.contains_key(id) { + new_optional_flags.push((*id, is_optional_filter(expr))); + } + } + return PartitionResult { + partitioned: PartitionedFilters { + row_filters: filters, + post_scan: Vec::new(), + }, + new_optional_flags, + }; + } + + // Note snapshot generations for dynamic filter detection. + // This clears stats for any filter whose generation has changed since the last scan. + // This must be done before any other logic since it can change filter states and stats. + for &(id, ref expr) in &filters { + let generation = snapshot_generation(expr); + self.note_generation(id, generation, stats_map); + } + + // Separate into row filters and post-scan filters based on effectiveness and state. + let mut row_filters: Vec<(FilterId, Arc)> = Vec::new(); + let mut post_scan_filters: Vec<(FilterId, Arc)> = Vec::new(); + + // Use the latency-aware effective z (clamped to <= config.confidence_z). + let confidence_z = z_eff; + for (id, expr) in filters { + let state = self.filter_states.get(&id).copied(); + + let Some(state) = state else { + // New filter: decide initial placement. + // + // We start at row-level only when the filter pulls in a + // small amount of *extra* I/O — bytes for filter columns + // **not already in the user projection** — relative to the + // projection. These are the cases where the row-level + // I/O cost is bounded and late materialization on a + // selective filter is a clear win (think a small int + // column predicate against a heavy string projection). + // + // Two cases default to post-scan instead, with the + // tracker free to promote later if measured + // bytes-saved-per-sec exceeds `min_bytes_per_sec`: + // + // - `extra_bytes == 0`: filter cols are entirely in the + // projection (e.g. `WHERE col <> '' GROUP BY col`). + // There's no I/O to save; the only payoff is late + // materialization on the *non*-filter projection + // columns, which depends on selectivity we don't know + // yet. Empirically (ClickBench Q10/11/13/14/26) + // defaulting these to row-level loses to post-scan + // because predicate-cache eviction on heavy string + // columns means the filter column is decoded twice. + // + // - `byte_ratio > byte_ratio_threshold`: extra I/O is + // too high to justify before we have evidence the + // filter is selective. + // + // Pre-existing snapshot-generation handling + // ([`SelectivityTrackerInner::note_generation`]) keeps + // dynamic filters (hash-join, TopK) at post-scan when + // they re-arm with new values — those rely on row-group + // statistics pruning rather than row-level I/O savings, + // so post-scan is correct for them too. + let filter_columns: Vec = collect_columns(&expr) + .iter() + .map(|col| col.index()) + .collect(); + let extra_columns: Vec = filter_columns + .iter() + .copied() + .filter(|c| !projection_columns.contains(c)) + .collect(); + let extra_bytes = + crate::row_filter::total_compressed_bytes(&extra_columns, metadata); + let byte_ratio = if projection_scan_size > 0 { + extra_bytes as f64 / projection_scan_size as f64 + } else { + 1.0 + }; + + if !stats_map.contains_key(&id) { + new_optional_flags.push((id, is_optional_filter(&expr))); + } + + // Selectivity prior from page-index pruning that the + // opener already ran on this file (see + // `PagePruningAccessPlanFilter::prune_plan_with_per_conjunct_stats`). + // No extra pruning work is done here — we just look up + // this filter's per-conjunct rate. When no rate is + // available (page index disabled, predicate not + // single-column, or schema mismatch), we fall back to + // the existing byte-ratio heuristic. + // + // **Dynamic-filter refresh**: when this conjunct is a + // populated DynamicFilter (snapshot_generation > 0) + // we evaluate a per-conjunct `PruningPredicate` against + // the file's row-group stats *now*, because the + // side-effect rates captured at file open were taken + // when the filter was still a placeholder. This is + // targeted re-evaluation — only for dynamic conjuncts + // that have updated since file open — so it doesn't + // count as an "extra pruning run" on the static path. + let dynamic_rate = if snapshot_generation(&expr) > 0 { + fresh_rate_for_dynamic_conjunct( + &expr, + arrow_schema, + parquet_schema, + metadata, + ) + } else { + None + }; + let prior = dynamic_rate.or_else(|| page_pruning_rates.get(&id).copied()); + + let row_level = match prior { + Some(p) if p >= config.prior_promote_threshold => { + debug!( + "FilterId {id}: New filter → Row filter via page-prior (pruned_rate={p:.3} >= {}) — {expr}", + config.prior_promote_threshold + ); + true + } + Some(p) if p <= config.prior_demote_threshold => { + debug!( + "FilterId {id}: New filter → Post-scan via page-prior (pruned_rate={p:.3} <= {}) — {expr}", + config.prior_demote_threshold + ); + false + } + _ => { + let r = + extra_bytes > 0 && byte_ratio <= config.byte_ratio_threshold; + debug!( + "FilterId {id}: New filter → {} via byte_ratio (byte_ratio={byte_ratio:.4}, extra_bytes={extra_bytes}, prior={prior:?}) — {expr}", + if r { "Row filter" } else { "Post-scan" } + ); + r + } + }; + + if row_level { + self.filter_states.insert(id, FilterState::RowFilter); + row_filters.push((id, expr)); + } else { + self.filter_states.insert(id, FilterState::PostScan); + post_scan_filters.push((id, expr)); + } + continue; + }; + + match state { + FilterState::RowFilter => { + // Should we demote this filter based on CI upper bound? + if let Some(entry) = stats_map.get(&id) { + let stats = entry.lock(); + if let Some(ub) = stats.confidence_upper_bound(confidence_z) + && ub < config.min_bytes_per_sec + { + drop(stats); + debug!( + "FilterId {id}: Row filter → Post-scan via CI upper bound {ub} < {} bytes/sec — {expr}", + config.min_bytes_per_sec + ); + self.demote_or_drop( + id, + &expr, + &mut post_scan_filters, + stats_map, + ); + continue; + } + } + // If not demoted, keep as row filter. + row_filters.push((id, expr)); + } + FilterState::PostScan => { + // Single gate: scatter-aware CI lower bound on + // bytes-saved-per-sec ≥ `min_bytes_per_sec`. + // + // The metric (see [`SelectivityStats::update`]) + // counts only sub-batch windows the filter empties + // out, so a 50% uniform filter scores ~0 and stays + // at post-scan; a TopK / hash-join / `Title LIKE` + // style filter where most batches drop entirely + // blows past the threshold. + // + // Earlier revisions also required `prune_rate ≥ 99%` + // on the theory that arrow-rs's row-level path + // double-decoded heavy string columns when the + // filter and projection overlapped. EXPLAIN ANALYZE + // on the ClickBench Q23 workload (URL LIKE + // `%google%`) showed the predicate cache is in fact + // active (`predicate_cache_inner_records=8.76M`) + // and the filter column is decoded once. The gate + // was removed; the residual ClickBench regressions + // we attributed to it (Q26 / Q31) trace to a + // different cause: post-scan filtering inside the + // opener changes batch-arrival order at downstream + // TopK, shifting the convergence point of TopK's + // dynamic filter and slightly weakening file-stats + // pruning. That has nothing to do with the + // promotion decision. + if let Some(entry) = stats_map.get(&id) { + let stats = entry.lock(); + if let Some(lb) = stats.confidence_lower_bound(confidence_z) + && lb >= config.min_bytes_per_sec + { + drop(stats); + debug!( + "FilterId {id}: Post-scan → Row filter via CI lower bound {lb} >= {} bytes/sec — {expr}", + config.min_bytes_per_sec + ); + self.promote(id, expr, &mut row_filters, stats_map); + continue; + } + } + // Should we drop this filter if it's optional and ineffective? + // Non-optional filters must stay as post-scan regardless. + if let Some(entry) = stats_map.get(&id) { + let stats = entry.lock(); + if let Some(ub) = stats.confidence_upper_bound(confidence_z) + && ub < config.min_bytes_per_sec + && expr.downcast_ref::().is_some() + { + drop(stats); + debug!( + "FilterId {id}: Post-scan → Dropped via CI upper bound {ub} < {} bytes/sec — {expr}", + config.min_bytes_per_sec + ); + self.filter_states.insert(id, FilterState::Dropped); + continue; + } + } + // Keep as post-scan filter (don't reset stats for mandatory filters). + post_scan_filters.push((id, expr)); + } + FilterState::Dropped => continue, + } + } + + // Sort row filters by: + // - Effectiveness (descending, higher = better) if available for both filters. + // - Scan size (ascending, cheapest first) as fallback — cheap filters prune + // rows before expensive ones, reducing downstream evaluation cost. + let cmp_row_filters = + |(id_a, expr_a): &(FilterId, Arc), + (id_b, expr_b): &(FilterId, Arc)| { + let eff_a = self.get_effectiveness_by_id(*id_a, stats_map); + let eff_b = self.get_effectiveness_by_id(*id_b, stats_map); + if let (Some(eff_a), Some(eff_b)) = (eff_a, eff_b) { + eff_b + .partial_cmp(&eff_a) + .unwrap_or(std::cmp::Ordering::Equal) + } else { + let size_a = filter_scan_size(expr_a, metadata); + let size_b = filter_scan_size(expr_b, metadata); + size_a.cmp(&size_b) + } + }; + row_filters.sort_by(cmp_row_filters); + // Post-scan filters: same logic (cheaper post-scan filters first to reduce + // the batch size for subsequent filters). + post_scan_filters.sort_by(cmp_row_filters); + + debug!( + "Partitioned filters: {} row-level, {} post-scan", + row_filters.len(), + post_scan_filters.len() + ); + PartitionResult { + partitioned: PartitionedFilters { + row_filters, + post_scan: post_scan_filters, + }, + new_optional_flags, + } + } +} + +/// Returns `true` if `expr` is wrapped in [`OptionalFilterPhysicalExpr`]. +fn is_optional_filter(expr: &Arc) -> bool { + expr.downcast_ref::().is_some() +} + +/// Calculate the estimated number of bytes needed to evaluate a filter based on the columns +/// it references as if it were applied to the entire file. +/// This is used for initial placement of new filters before any stats are available, and as a fallback for filters without stats. +fn filter_scan_size(expr: &Arc, metadata: &ParquetMetaData) -> usize { + let columns: Vec = collect_columns(expr) + .iter() + .map(|col| col.index()) + .collect(); + + crate::row_filter::total_compressed_bytes(&columns, metadata) +} + +// (Per-conjunct page-pruning rates are now extracted as a side-effect +// of the opener's existing page-index pruning pass — see +// `PagePruningAccessPlanFilter::prune_plan_with_per_conjunct_stats`. +// `partition_filters` reads them through its `page_pruning_rates` +// parameter; no extra pruning runs happen on the static path.) + +/// Compute a fresh row-group pruning rate for a single dynamic +/// conjunct, evaluated against the file's row-group statistics +/// *now*. Used by `partition_filters` to refresh the prior for +/// dynamic filters that were placeholders when the side-effect +/// rates were captured at file open and have since been populated +/// by the join build side. +/// +/// Returns `None` when the conjunct doesn't translate into a +/// usable pruning predicate (e.g. always-true after rewriting, +/// references columns missing from the schema, contains +/// hash_lookup-style nodes the rewriter can't handle). +fn fresh_rate_for_dynamic_conjunct( + expr: &Arc, + arrow_schema: &SchemaRef, + parquet_schema: &SchemaDescriptor, + metadata: &ParquetMetaData, +) -> Option { + use datafusion_pruning::PruningPredicate; + // Unwrap OptionalFilterPhysicalExpr — pruning should evaluate + // the underlying predicate, not the marker. + let inner = if let Some(opt) = expr.downcast_ref::() { + opt.inner() + } else { + Arc::clone(expr) + }; + let groups = metadata.row_groups(); + if groups.is_empty() { + return None; + } + let stats = crate::row_group_filter::RowGroupPruningStatistics { + parquet_schema, + row_group_metadatas: groups.iter().collect(), + arrow_schema: arrow_schema.as_ref(), + missing_null_counts_as_zero: false, + }; + + // First try: build a PruningPredicate from the whole conjunct. + if let Ok(pp) = + PruningPredicate::try_new(Arc::clone(&inner), Arc::clone(arrow_schema)) + && !pp.always_true() + && let Ok(kept) = pp.prune(&stats) + && !kept.is_empty() + { + let total = kept.len(); + let pruned = total - kept.iter().filter(|b| **b).count(); + return Some(pruned as f64 / total as f64); + } + + // Second try (the AND-with-hash-lookup case): snapshot the + // dynamic filter to materialize its current inner expression, + // then split the AND inside. `split_conjunction` doesn't descend + // into DynamicFilterPhysicalExpr wrappers, so without this step + // the split would return `[dynamic_filter]` and miss the + // prunable parts inside. We take the *max* pruning rate across + // sub-parts as a *promote* signal — if any sub-conjunct prunes + // a high fraction, the whole AND prunes at least that much. We + // deliberately do NOT use this as a demote signal. + let snapshot_result = + datafusion_physical_expr_common::physical_expr::snapshot_physical_expr_opt( + Arc::clone(&inner), + ) + .ok()?; + let snapshotted = snapshot_result.data; + let parts = datafusion_physical_expr::split_conjunction(&snapshotted); + if parts.len() < 2 { + return None; + } + let mut max_rate: Option = None; + for part in parts { + let Ok(pp) = + PruningPredicate::try_new(Arc::clone(part), Arc::clone(arrow_schema)) + else { + continue; + }; + if pp.always_true() { + continue; + } + let Ok(kept) = pp.prune(&stats) else { continue }; + if kept.is_empty() { + continue; + } + let total = kept.len(); + let pruned = total - kept.iter().filter(|b| **b).count(); + let rate = pruned as f64 / total as f64; + max_rate = Some(max_rate.map_or(rate, |m| m.max(rate))); + } + // Promote-only semantics: only return when the partial-AND rate + // is high enough to be a confident promote signal. Below that we + // return None and let the standard prior / byte-ratio fallback + // run, which won't be misled by an undercounted rate. + max_rate.filter(|&r| r >= 0.5) +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion_physical_expr::expressions::Column; + use parquet::basic::Type as PhysicalType; + use parquet::file::metadata::{ColumnChunkMetaData, FileMetaData, RowGroupMetaData}; + use parquet::schema::types::SchemaDescPtr; + use parquet::schema::types::Type as SchemaType; + use std::sync::Arc; + + mod helper_functions { + use super::*; + + /// Creates test ParquetMetaData with specified row groups and column sizes. + /// + /// # Arguments + /// * `specs` - Vec of (num_rows, vec![compressed_size]) tuples for each row group + pub fn create_test_metadata(specs: Vec<(i64, Vec)>) -> ParquetMetaData { + // Get the maximum number of columns from all specs + let num_columns = specs + .iter() + .map(|(_, sizes)| sizes.len()) + .max() + .unwrap_or(1); + let schema_descr = get_test_schema_descr_with_columns(num_columns); + + let row_group_metadata: Vec<_> = specs + .into_iter() + .map(|(num_rows, column_sizes)| { + let columns = column_sizes + .into_iter() + .enumerate() + .map(|(col_idx, size)| { + ColumnChunkMetaData::builder(schema_descr.column(col_idx)) + .set_num_values(num_rows) + .set_total_compressed_size(size as i64) + .build() + .unwrap() + }) + .collect(); + + RowGroupMetaData::builder(schema_descr.clone()) + .set_num_rows(num_rows) + .set_column_metadata(columns) + .build() + .unwrap() + }) + .collect(); + + let total_rows: i64 = row_group_metadata.iter().map(|rg| rg.num_rows()).sum(); + let file_metadata = + FileMetaData::new(1, total_rows, None, None, schema_descr.clone(), None); + + ParquetMetaData::new(file_metadata, row_group_metadata) + } + + /// Creates a simple column expression with given name and index. + pub fn col_expr(name: &str, index: usize) -> Arc { + Arc::new(Column::new(name, index)) + } + + /// Create schema with specified number of columns, each named "a", "b", etc. + pub fn get_test_schema_descr_with_columns(num_columns: usize) -> SchemaDescPtr { + use parquet::basic::LogicalType; + + let fields: Vec<_> = (0..num_columns) + .map(|i| { + let col_name = format!("{}", (b'a' + i as u8) as char); + SchemaType::primitive_type_builder( + &col_name, + PhysicalType::BYTE_ARRAY, + ) + .with_logical_type(Some(LogicalType::String)) + .build() + .unwrap() + }) + .map(Arc::new) + .collect(); + + let schema = SchemaType::group_type_builder("schema") + .with_fields(fields) + .build() + .unwrap(); + Arc::new(SchemaDescriptor::new(Arc::new(schema))) + } + } + + mod selectivity_stats_tests { + use super::*; + + #[test] + fn test_effectiveness_basic_calculation() { + let mut stats = SelectivityStats::default(); + + // skippable_bytes is now caller-computed (= rows_pruned * + // bytes_per_row in the simple case), so passing 5000 directly + // models the same scenario the old test described: + // "100 rows total, 50 pruned, 100 bytes/row → 5000 saved". + stats.update(50, 100, 1_000_000_000, 5_000); + + let eff = stats.effectiveness().unwrap(); + assert!((eff - 5000.0).abs() < 0.1); + } + + #[test] + fn test_effectiveness_zero_rows_total() { + let mut stats = SelectivityStats::default(); + stats.update(0, 0, 1_000_000_000, 10_000); + + assert_eq!(stats.effectiveness(), None); + } + + #[test] + fn test_effectiveness_zero_eval_nanos() { + let mut stats = SelectivityStats::default(); + stats.update(50, 100, 0, 10_000); + + assert_eq!(stats.effectiveness(), None); + } + + #[test] + fn test_effectiveness_zero_bytes_seen() { + // A batch with zero skippable_bytes is a legitimate sample + // ("filter ran, late-mat had nothing to save") — Welford + // records it as eff=0 rather than discarding it, so the + // demotion path can see "CPU spent, no payoff." + let mut stats = SelectivityStats::default(); + stats.update(50, 100, 1_000_000_000, 0); + + assert_eq!(stats.effectiveness(), Some(0.0)); + } + + #[test] + fn test_effectiveness_all_rows_matched() { + let mut stats = SelectivityStats::default(); + // All rows matched (no pruning) — caller computes + // skippable_bytes = rows_pruned * bytes_per_row = 0. + stats.update(100, 100, 1_000_000_000, 0); + + let eff = stats.effectiveness().unwrap(); + assert_eq!(eff, 0.0); + } + + #[test] + fn test_confidence_bounds_single_sample() { + let mut stats = SelectivityStats::default(); + stats.update(50, 100, 1_000_000_000, 10_000); + + // Single sample returns None for confidence bounds + assert_eq!(stats.confidence_lower_bound(2.0), None); + assert_eq!(stats.confidence_upper_bound(2.0), None); + } + + #[test] + fn test_welford_identical_samples() { + let mut stats = SelectivityStats::default(); + + // Add two identical samples + stats.update(50, 100, 1_000_000_000, 10_000); + stats.update(50, 100, 1_000_000_000, 10_000); + + // Variance should be 0 + assert_eq!(stats.sample_count, 2); + let lb = stats.confidence_lower_bound(2.0).unwrap(); + let ub = stats.confidence_upper_bound(2.0).unwrap(); + + // Both should be equal to the mean since variance is 0 + assert!((lb - ub).abs() < 0.01); + } + + #[test] + fn test_welford_variance_calculation() { + let mut stats = SelectivityStats::default(); + + // Add samples that produce effectiveness values 5000, 6000, 7000 + // (caller-computed skippable_bytes is the lever now). + stats.update(50, 100, 1_000_000_000, 5_000); // eff = 5000 + stats.update(40, 100, 1_000_000_000, 6_000); // eff = 6000 + stats.update(30, 100, 1_000_000_000, 7_000); // eff = 7000 + + // We should have 3 samples + assert_eq!(stats.sample_count, 3); + + // Mean should be 6000 + assert!((stats.eff_mean - 6000.0).abs() < 1.0); + + // Both bounds should be defined + let lb = stats.confidence_lower_bound(1.0).unwrap(); + let ub = stats.confidence_upper_bound(1.0).unwrap(); + + assert!(lb < stats.eff_mean); + assert!(ub > stats.eff_mean); + } + + #[test] + fn test_confidence_bounds_asymmetry() { + let mut stats = SelectivityStats::default(); + + stats.update(50, 100, 1_000_000_000, 10_000); + stats.update(40, 100, 1_000_000_000, 10_000); + + let lb = stats.confidence_lower_bound(2.0).unwrap(); + let ub = stats.confidence_upper_bound(2.0).unwrap(); + + // Bounds should be symmetric around the mean + let lower_dist = stats.eff_mean - lb; + let upper_dist = ub - stats.eff_mean; + + assert!((lower_dist - upper_dist).abs() < 0.01); + } + + #[test] + fn test_welford_incremental_vs_batch() { + // Create two identical stats objects + let mut stats_incremental = SelectivityStats::default(); + let mut stats_batch = SelectivityStats::default(); + + // Incremental: add one at a time + stats_incremental.update(50, 100, 1_000_000_000, 10_000); + stats_incremental.update(40, 100, 1_000_000_000, 10_000); + stats_incremental.update(30, 100, 1_000_000_000, 10_000); + + // Batch: simulate batch update (all at once) + stats_batch.update(120, 300, 3_000_000_000, 30_000); + + // Both should produce the same overall statistics + assert_eq!(stats_incremental.rows_total, stats_batch.rows_total); + assert_eq!(stats_incremental.rows_matched, stats_batch.rows_matched); + + // Means should be close + assert!((stats_incremental.eff_mean - stats_batch.eff_mean).abs() < 100.0); + } + + #[test] + fn test_effectiveness_numerical_stability() { + let mut stats = SelectivityStats::default(); + + // Test with large values to ensure numerical stability + stats.update( + 500_000_000, + 1_000_000_000, + 10_000_000_000_000, + 1_000_000_000_000, + ); + + let eff = stats.effectiveness(); + assert!(eff.is_some()); + assert!(eff.unwrap() > 0.0); + assert!(!eff.unwrap().is_nan()); + assert!(!eff.unwrap().is_infinite()); + } + } + + mod tracker_config_tests { + use super::*; + + #[test] + fn test_default_config() { + let config = TrackerConfig::default(); + + assert!(config.min_bytes_per_sec.is_infinite()); + assert_eq!(config.byte_ratio_threshold, 0.20); + assert_eq!(config.confidence_z, 2.0); + } + + #[test] + fn test_with_min_bytes_per_sec() { + let config = TrackerConfig::new().with_min_bytes_per_sec(1000.0); + + assert_eq!(config.min_bytes_per_sec, 1000.0); + } + + #[test] + fn test_with_byte_ratio_threshold() { + let config = TrackerConfig::new().with_byte_ratio_threshold(0.5); + + assert_eq!(config.byte_ratio_threshold, 0.5); + } + + #[test] + fn test_with_confidence_z() { + let config = TrackerConfig::new().with_confidence_z(3.0); + + assert_eq!(config.confidence_z, 3.0); + } + + #[test] + fn test_builder_chain() { + let config = TrackerConfig::new() + .with_min_bytes_per_sec(500.0) + .with_byte_ratio_threshold(0.3) + .with_confidence_z(1.5); + + assert_eq!(config.min_bytes_per_sec, 500.0); + assert_eq!(config.byte_ratio_threshold, 0.3); + assert_eq!(config.confidence_z, 1.5); + } + + #[test] + fn test_build_creates_tracker() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1000.0).build(); + + // Tracker should be created and functional + assert_eq!(tracker.config.min_bytes_per_sec, 1000.0); + } + } + + mod state_machine_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_initial_placement_low_byte_ratio() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.2) + .build(); + + // Create metadata: 1 row group, 100 rows, 1000 bytes for column + let metadata = create_test_metadata(vec![(100, vec![1000])]); + + // Filter using column 0 (1000 bytes out of 1000 projection = 100% ratio > 0.2) + // So this should be placed in post-scan initially + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // With 100% byte ratio, should go to post-scan + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_initial_placement_filter_in_projection_low_ratio() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + // Create metadata: 1 row group, 100 rows, 100 bytes for column + let metadata = create_test_metadata(vec![(100, vec![100])]); + + // Filter using column 0 which IS in the projection. + // filter_bytes=100, projection=1000, ratio=0.10 <= 0.5 → RowFilter + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_initial_placement_high_byte_ratio() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + // Create metadata: 1 row group, 100 rows, 100 bytes for column + let metadata = create_test_metadata(vec![(100, vec![100])]); + + // Filter using column 0 (100 bytes / 1000 projection = 10% ratio <= 0.5) + // So this should be placed in row-filter immediately + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // With 10% byte ratio, should go to row-filter + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_min_bytes_per_sec_infinity_disables_promotion() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(f64::INFINITY) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // All filters should go to post_scan when min_bytes_per_sec is INFINITY + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_min_bytes_per_sec_zero_promotes_all() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(0.0).build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // All filters should be promoted to row_filters when min_bytes_per_sec is 0 + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_promotion_via_confidence_lower_bound() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) // Force to PostScan initially + .with_confidence_z(0.5) // Lower z for easier promotion + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // First partition: goes to PostScan (high byte ratio) + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters.len(), 0); + + // Feed high effectiveness stats + for _ in 0..5 { + tracker.update(1, 1, 100, 100_000, 1000); // high effectiveness + } + + // Second partition: should be promoted to RowFilter + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_demotion_via_confidence_upper_bound() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.1) // Force to RowFilter initially + .with_confidence_z(0.5) // Lower z for easier demotion + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // First partition: goes to RowFilter (low byte ratio) + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + + // Feed low effectiveness stats — all rows matched, no rows + // pruned, so caller-computed skippable_bytes is 0. + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 0); + } + + // Second partition: should be demoted to PostScan + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_demotion_resets_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.1) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as RowFilter + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Add stats — no pruning, so skippable_bytes = 0 + tracker.update(1, 100, 100, 100_000, 0); + tracker.update(1, 100, 100, 100_000, 0); + + // Demote + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Stats should be zeroed after demotion + let stats_map = tracker.filter_stats.read(); + assert_eq!( + *stats_map.get(&1).unwrap().lock(), + SelectivityStats::default() + ); + } + + #[test] + fn test_promotion_resets_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(100.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as PostScan + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Add stats with high prune_rate so the selectivity gate + // (>= 0.99) lets the promotion fire. + for _ in 0..3 { + tracker.update(1, 1, 100, 100_000, 1000); + } + + // Promote + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Stats should be zeroed after promotion + let stats_map = tracker.filter_stats.read(); + assert_eq!( + *stats_map.get(&1).unwrap().lock(), + SelectivityStats::default() + ); + } + + #[test] + fn test_optional_filter_dropping() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as PostScan + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Feed poor effectiveness stats — no pruning, no skippable_bytes + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 0); + } + + // Next partition: should stay as PostScan (not dropped because not optional) + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters.len(), 0); + } + + #[test] + fn test_persistent_dropped_state() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Mark filter as dropped by manually setting state + tracker + .inner + .lock() + .filter_states + .insert(1, FilterState::Dropped); + + // On next partition, dropped filters should not reappear + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 0); + } + } + + mod filter_ordering_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_filters_get_partitioned() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1.0) // Very low threshold + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100, 100, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + // Partition should process all filters + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // With min_bytes_per_sec=1.0, filters should be partitioned + assert!(result.row_filters.len() + result.post_scan.len() > 0); + + // Add stats and partition again + tracker.update(1, 60, 100, 1_000_000, 100); + tracker.update(2, 1, 100, 1_000_000, 100); + tracker.update(3, 40, 100, 1_000_000, 100); + + let result2 = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Filters should still be partitioned + assert!(result2.row_filters.len() + result2.post_scan.len() > 0); + } + + #[test] + fn test_filters_processed_without_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1.0) // Very low threshold + .build(); + + // Different column sizes: 300, 200, 100 bytes + let metadata = create_test_metadata(vec![(100, vec![300, 200, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + // First partition - no stats yet + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // All filters should be processed (partitioned into row/post-scan) + assert!(result.row_filters.len() + result.post_scan.len() > 0); + + // Filters should be consistent on repeated calls + let result2 = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!( + result.row_filters.len() + result.post_scan.len(), + result2.row_filters.len() + result2.post_scan.len() + ); + } + + #[test] + fn test_filters_with_partial_stats() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1.0).build(); + + // Give filter 2 larger bytes so it's prioritized when falling back to byte ratio + let metadata = create_test_metadata(vec![(100, vec![100, 300, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + // First partition + let result1 = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert!(result1.row_filters.len() + result1.post_scan.len() > 0); + + // Only add stats for filters 1 and 3, not 2 + tracker.update(1, 60, 100, 1_000_000, 100); + tracker.update(3, 60, 100, 1_000_000, 100); + + // Second partition with partial stats + let result2 = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert!(result2.row_filters.len() + result2.post_scan.len() > 0); + } + + #[test] + fn test_ordering_stability_with_identical_values() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(0.0).build(); + + let metadata = create_test_metadata(vec![(100, vec![100, 100, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + let result1 = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + let result2 = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // Without stats and with identical byte sizes, order should be stable + assert_eq!(result1.row_filters[0].0, result2.row_filters[0].0); + assert_eq!(result1.row_filters[1].0, result2.row_filters[1].0); + assert_eq!(result1.row_filters[2].0, result2.row_filters[2].0); + } + } + + mod dynamic_filter_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_generation_zero_ignored() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + + // Create two filters with same ID but generation 0 and 1 + // Generation 0 should be ignored + let expr1 = col_expr("a", 0); + let filters1 = vec![(1, expr1)]; + + tracker.partition_filters_for_test( + filters1, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + tracker.update(1, 50, 100, 100_000, 1000); + + // Generation 0 doesn't trigger state reset + let snapshot_gen = tracker.inner.lock().snapshot_generations.get(&1).copied(); + assert_eq!(snapshot_gen, None); + } + + #[test] + fn test_generation_change_clears_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + // Pre-populate stats entry so update() can find it + tracker.ensure_stats_entry(1); + + // Initialize generation to 100 + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Add stats + tracker.update(1, 50, 100, 100_000, 1000); + tracker.update(1, 50, 100, 100_000, 1000); + + let stats_before = { + let stats_map = tracker.filter_stats.read(); + *stats_map.get(&1).unwrap().lock() != SelectivityStats::default() + }; + assert!(stats_before); + + // Simulate generation change to a different value + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 101, &stats); + } + + // Stats should be zeroed on generation change + let stats_after = { + let stats_map = tracker.filter_stats.read(); + *stats_map.get(&1).unwrap().lock() == SelectivityStats::default() + }; + assert!(stats_after); + } + + #[test] + fn test_generation_unchanged_preserves_stats() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1000.0).build(); + + // Pre-populate stats entry so update() can find it + tracker.ensure_stats_entry(1); + + // Manually set generation + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Add stats + tracker.update(1, 50, 100, 100_000, 1000); + tracker.update(1, 50, 100, 100_000, 1000); + + let sample_count_before = { + let stats_map = tracker.filter_stats.read(); + stats_map.get(&1).map(|s| s.lock().sample_count) + }; + assert_eq!(sample_count_before, Some(2)); + + // Call note_generation with same generation + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Stats should be preserved + let sample_count_after = { + let stats_map = tracker.filter_stats.read(); + stats_map.get(&1).map(|s| s.lock().sample_count) + }; + assert_eq!(sample_count_after, Some(2)); + } + + #[test] + fn test_generation_change_preserves_state() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.1) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + + // First partition: goes to RowFilter + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + let state_before = tracker.inner.lock().filter_states.get(&1).copied(); + assert_eq!(state_before, Some(FilterState::RowFilter)); + + // Simulate generation change + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // State should be preserved despite stats being cleared + let state_after = tracker.inner.lock().filter_states.get(&1).copied(); + assert_eq!(state_after, Some(FilterState::RowFilter)); + } + + #[test] + fn test_generation_change_undrops_dropped_filter() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.1) + .build(); + + // Manually set filter state to Dropped + tracker + .inner + .lock() + .filter_states + .insert(1, FilterState::Dropped); + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Simulate generation change + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 101, &stats); + } + + // Dropped filter should be un-dropped to PostScan + let state_after = tracker.inner.lock().filter_states.get(&1).copied(); + assert_eq!(state_after, Some(FilterState::PostScan)); + } + + #[test] + fn test_multiple_filters_independent_generation_tracking() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1000.0).build(); + + // Pre-populate stats entries so update() can find them + tracker.ensure_stats_entry(1); + tracker.ensure_stats_entry(2); + + // Set generations for multiple filters + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + inner.note_generation(2, 200, &stats); + } + + // Add stats to both + tracker.update(1, 50, 100, 100_000, 1000); + tracker.update(2, 50, 100, 100_000, 1000); + + // Change generation of filter 1 only + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 101, &stats); + } + + // Filter 1 stats should be zeroed, filter 2 preserved + let stats_map = tracker.filter_stats.read(); + assert_eq!( + *stats_map.get(&1).unwrap().lock(), + SelectivityStats::default() + ); + assert_ne!( + *stats_map.get(&2).unwrap().lock(), + SelectivityStats::default() + ); + } + } + + mod integration_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_full_promotion_lifecycle() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(500.0) + .with_byte_ratio_threshold(0.5) // Force initial PostScan + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Step 1: Initial placement (PostScan) + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters.len(), 0); + + // Step 2: Accumulate high effectiveness stats + for _ in 0..5 { + tracker.update(1, 1, 100, 100_000, 1000); // high effectiveness + } + + // Step 3: Promotion should occur + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + + // Step 4: Continue to partition without additional updates + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_full_demotion_lifecycle() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.1) // Force initial RowFilter + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Step 1: Initial placement (RowFilter) + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + + // Step 2: Accumulate low effectiveness stats — no pruning, + // so skippable_bytes = 0 + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 0); + } + + // Step 3: Demotion should occur + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + + // Step 4: Continue to partition without additional updates + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_multiple_filters_mixed_states() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.4) // Force PostScan initially (500/1000=0.5 > 0.4) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![500, 500])]); + let filters = vec![(1, col_expr("a", 0)), (2, col_expr("a", 1))]; + + // Initial partition: both go to PostScan (500/1000 = 0.5 > 0.4) + let result = tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.post_scan.len(), 2); + + // Filter 1: high effectiveness — 99/100 rows pruned out of + // 500 batch bytes ≈ 495 skippable bytes + for _ in 0..3 { + tracker.update(1, 1, 100, 100_000, 495); + } + + // Filter 2: low effectiveness — no rows pruned, so 0 skippable + for _ in 0..3 { + tracker.update(2, 100, 100, 100_000, 0); + } + + // Next partition: Filter 1 promoted, Filter 2 stays PostScan + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters[0].0, 1); + assert_eq!(result.post_scan[0].0, 2); + } + + #[test] + fn test_empty_filter_list() { + let tracker = TrackerConfig::new().build(); + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let filters = vec![]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_single_filter() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(0.0).build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_zero_effectiveness_stays_at_boundary() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(100.0) + .with_byte_ratio_threshold(0.1) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as RowFilter + tracker.partition_filters_for_test( + filters.clone(), + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + + // All rows match (zero effectiveness) — no rows pruned, so + // skippable_bytes = 0 + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 0); + } + + // Should demote due to CI upper bound being 0 + let result = tracker.partition_filters_for_test( + filters, + &std::collections::HashSet::new(), + 1000, + &metadata, + ); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_confidence_z_parameter_stored() { + // Test that different confidence_z values are properly stored in config + let tracker_conservative = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(3.0) // Harder to promote + .build(); + + let tracker_aggressive = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(0.5) // Easier to promote + .build(); + + // Verify configs are stored correctly + assert_eq!(tracker_conservative.config.confidence_z, 3.0); + assert_eq!(tracker_aggressive.config.confidence_z, 0.5); + + // The z-score affects confidence intervals during promotion/demotion decisions. + // With identical stats, higher z requires narrower confidence intervals, + // making promotion harder. With lower z, confidence intervals are wider, + // making promotion easier. This is tested in other integration tests + // that verify actual promotion/demotion behavior. + } + } +} diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 91e3d2274932d..be2eafbf94f5c 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -39,8 +39,9 @@ use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_datasource::TableSchema; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::conjunction; use datafusion_physical_expr::projection::ProjectionExprs; -use datafusion_physical_expr::{EquivalenceProperties, conjunction}; use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::physical_expr::fmt_sql; @@ -277,8 +278,10 @@ pub struct ParquetSource { /// In particular, this is the schema of the table without partition columns, /// *not* the physical schema of the file. pub(crate) table_schema: TableSchema, - /// Optional predicate for row filtering during parquet scan - pub(crate) predicate: Option>, + /// Optional predicate conjuncts for row filtering during parquet scan. + /// Each conjunct is tagged with a stable FilterId for selectivity tracking. + pub(crate) predicate_conjuncts: + Option)>>, /// Optional user defined parquet file reader factory pub(crate) parquet_file_reader_factory: Option>, /// Batch size configuration @@ -294,6 +297,10 @@ pub struct ParquetSource { /// so we still need to sort them after reading, so the reverse scan is inexact. /// Used to optimize ORDER BY ... DESC on sorted data. reverse_row_groups: bool, + /// Tracks filter selectivity across files for adaptive filter reordering. + /// Shared across all openers - each opener reads stats and makes its own + /// decision about which filters to push down vs. apply post-scan. + pub(crate) selectivity_tracker: Arc, } impl ParquetSource { @@ -312,13 +319,16 @@ impl ParquetSource { table_schema, table_parquet_options: TableParquetOptions::default(), metrics: ExecutionPlanMetricsSet::new(), - predicate: None, + predicate_conjuncts: None, parquet_file_reader_factory: None, batch_size: None, metadata_size_hint: None, #[cfg(feature = "parquet_encryption")] encryption_factory: None, reverse_row_groups: false, + selectivity_tracker: Arc::new( + crate::selectivity::SelectivityTracker::default(), + ), } } @@ -327,6 +337,15 @@ impl ParquetSource { mut self, table_parquet_options: TableParquetOptions, ) -> Self { + // Update the selectivity tracker from the config + let opts = &table_parquet_options.global; + self.selectivity_tracker = Arc::new( + crate::selectivity::TrackerConfig::new() + .with_min_bytes_per_sec(opts.filter_pushdown_min_bytes_per_sec) + .with_byte_ratio_threshold(opts.filter_collecting_byte_ratio_threshold) + .with_confidence_z(opts.filter_confidence_z) + .build(), + ); self.table_parquet_options = table_parquet_options; self } @@ -342,11 +361,23 @@ impl ParquetSource { self } - /// Set predicate information + /// Set predicate information. + /// + /// The predicate is split into conjuncts and each is assigned a stable + /// `FilterId` (its index in the conjunct list). These IDs are used for + /// selectivity tracking across files, avoiding ExprKey mismatch issues + /// when expressions are rebased or simplified per-file. #[expect(clippy::needless_pass_by_value)] pub fn with_predicate(&self, predicate: Arc) -> Self { + use datafusion_physical_expr::split_conjunction; let mut conf = self.clone(); - conf.predicate = Some(Arc::clone(&predicate)); + let conjuncts: Vec<(crate::selectivity::FilterId, Arc)> = + split_conjunction(&predicate) + .into_iter() + .enumerate() + .map(|(id, expr)| (id, Arc::clone(expr))) + .collect(); + conf.predicate_conjuncts = Some(conjuncts); conf } @@ -367,8 +398,15 @@ impl ParquetSource { /// Optional predicate. #[deprecated(since = "50.2.0", note = "use `filter` instead")] - pub fn predicate(&self) -> Option<&Arc> { - self.predicate.as_ref() + pub fn predicate(&self) -> Option> { + self.combined_predicate() + } + + /// Build a combined predicate from the conjuncts, if any. + fn combined_predicate(&self) -> Option> { + self.predicate_conjuncts + .as_ref() + .map(|conjuncts| conjunction(conjuncts.iter().map(|(_, e)| Arc::clone(e)))) } /// return the optional file reader factory @@ -399,19 +437,30 @@ impl ParquetSource { self.table_parquet_options.global.pushdown_filters } - /// If true, the `RowFilter` made by `pushdown_filters` may try to - /// minimize the cost of filter evaluation by reordering the - /// predicate [`Expr`]s. If false, the predicates are applied in - /// the same order as specified in the query. Defaults to false. + /// Set the legacy `reorder_filters` config flag. /// - /// [`Expr`]: datafusion_expr::Expr + /// The adaptive selectivity tracker subsumes static filter reordering + /// (filters are now ranked online by measured bytes-saved/sec), so the + /// flag is preserved for backwards compatibility but no longer has any + /// effect on filter placement. + #[deprecated( + since = "53.2.0", + note = "the adaptive selectivity tracker reorders filters by measured \ + effectiveness; this flag is now a no-op" + )] pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self { self.table_parquet_options.global.reorder_filters = reorder_filters; self } - /// Return the value described in [`Self::with_reorder_filters`] - fn reorder_filters(&self) -> bool { + /// Return the value of the legacy `reorder_filters` config flag. Now a + /// no-op — see [`Self::with_reorder_filters`]. + #[deprecated( + since = "53.2.0", + note = "the adaptive selectivity tracker reorders filters by measured \ + effectiveness; this flag is now a no-op" + )] + pub fn reorder_filters(&self) -> bool { self.table_parquet_options.global.reorder_filters } @@ -562,13 +611,13 @@ impl FileSource for ParquetSource { .expect("Batch size must set before creating ParquetMorselizer"), limit: base_config.limit, preserve_order: base_config.preserve_order, - predicate: self.predicate.clone(), + predicate_conjuncts: self.predicate_conjuncts.clone(), + selectivity_tracker: Arc::clone(&self.selectivity_tracker), table_schema: self.table_schema.clone(), metadata_size_hint: self.metadata_size_hint, metrics: self.metrics().clone(), parquet_file_reader_factory, pushdown_filters: self.pushdown_filters(), - reorder_filters: self.reorder_filters(), force_filter_selections: self.force_filter_selections(), enable_page_index: self.enable_page_index(), enable_bloom_filter: self.bloom_filter_on_read(), @@ -589,7 +638,7 @@ impl FileSource for ParquetSource { } fn filter(&self) -> Option> { - self.predicate.clone() + self.combined_predicate() } fn with_batch_size(&self, batch_size: usize) -> Arc { @@ -642,7 +691,7 @@ impl FileSource for ParquetSource { // the actual predicates are built in reference to the physical schema of // each file, which we do not have at this point and hence cannot use. // Instead, we use the logical schema of the file (the table schema without partition columns). - if let Some(predicate) = &self.predicate { + if let Some(predicate) = &self.combined_predicate() { let predicate_creation_errors = Count::new(); if let Some(pruning_predicate) = build_pruning_predicates( Some(predicate), @@ -719,13 +768,16 @@ impl FileSource for ParquetSource { PushedDown::No => None, }) .collect_vec(); - let predicate = match source.predicate { - Some(predicate) => { - conjunction(std::iter::once(predicate).chain(allowed_filters)) - } - None => conjunction(allowed_filters), - }; - source.predicate = Some(predicate); + // Merge existing conjuncts with new allowed filters + let mut all_conjuncts: Vec> = source + .predicate_conjuncts + .as_ref() + .map(|c| c.iter().map(|(_, e)| Arc::clone(e)).collect()) + .unwrap_or_default(); + all_conjuncts.extend(allowed_filters); + // Re-assign FilterIds by index + source.predicate_conjuncts = + Some(all_conjuncts.into_iter().enumerate().collect()); source = source.with_pushdown_filters(pushdown_filters); let source = Arc::new(source); // If pushdown_filters is false we tell our parents that they still have to handle the filters, @@ -836,8 +888,10 @@ impl FileSource for ParquetSource { ) -> datafusion_common::Result { // Visit predicate (filter) expression if present let mut tnr = TreeNodeRecursion::Continue; - if let Some(predicate) = &self.predicate { - tnr = tnr.visit_sibling(|| f(predicate.as_ref()))?; + if let Some(ref conjuncts) = self.predicate_conjuncts { + for (_, expr) in conjuncts { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } } // Visit projection expressions @@ -862,8 +916,9 @@ mod tests { let parquet_source = ParquetSource::new(Arc::new(Schema::empty())).with_predicate(predicate); - // same value. but filter() call Arc::clone internally - assert_eq!(parquet_source.predicate(), parquet_source.filter().as_ref()); + // Both should return equivalent predicates + assert!(parquet_source.predicate().is_some()); + assert!(parquet_source.filter().is_some()); } #[test] diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 9f4f8aa0f3635..7c9281dcc2f26 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -790,7 +790,7 @@ impl DataSink for MemSink { } // write the outputs into the batches - for (target, mut batches) in self.batches.iter().zip(new_batches.into_iter()) { + for (target, mut batches) in self.batches.iter().zip(new_batches) { // Append all the new batches in one go to minimize locking overhead target.write().await.append(&mut batches); } diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 032f4cec0b03f..6abfafe9d39d4 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -261,7 +261,7 @@ impl MinMaxStatistics { /// Return a sorted list of the min statistics together with the original indices pub fn min_values_sorted(&self) -> Vec<(usize, Row<'_>)> { let mut sort: Vec<_> = self.min_by_sort_order.iter().enumerate().collect(); - sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + sort.sort_unstable_by_key(|(_, row)| *row); sort } diff --git a/datafusion/datasource/src/url.rs b/datafusion/datasource/src/url.rs index 4d7f5bf14c697..14f9b2af0021d 100644 --- a/datafusion/datasource/src/url.rs +++ b/datafusion/datasource/src/url.rs @@ -258,7 +258,7 @@ impl ListingTableUrl { let full_prefix = if let Some(ref p) = prefix { let mut parts = self.prefix.parts().collect::>(); parts.extend(p.parts()); - Path::from_iter(parts.into_iter()) + Path::from_iter(parts) } else { self.prefix.clone() }; diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index aac95b9d6a81f..52b601d5cd78b 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -500,7 +500,7 @@ impl TrackConsumersPool { ) }) .collect::>(); - consumers.sort_by(|a, b| b.1.cmp(&a.1)); // inverse ordering + consumers.sort_by_key(|consumer| std::cmp::Reverse(consumer.1)); consumers[0..std::cmp::min(top, consumers.len())] .iter() diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 36ef6cf1f5ba9..6f420a7153057 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -26,7 +26,7 @@ use std::sync::Arc; use crate::expr_fn::binary_expr; use crate::function::WindowFunctionSimplification; -use crate::higher_order_function::HigherOrderUDF; +use crate::higher_order_function::{HigherOrderUDF, resolve_lambda_variables}; use crate::logical_plan::Subquery; use crate::type_coercion::functions::value_fields_with_higher_order_udf; use crate::{AggregateUDF, LambdaParametersProgress, ValueOrLambda, Volatility}; @@ -453,7 +453,12 @@ impl HigherOrderFunction { } /// Invokes the inner function [`HigherOrderUDF::lambda_parameters`] - /// using the arguments of this invocation + /// using the arguments of this invocation. This expression lambda + /// variables must be already resolved either by coming from the + /// default sql planner or by calling [Expr::resolve_lambda_variables] + /// or [LogicalPlan::resolve_lambda_variables] + /// + /// [LogicalPlan::resolve_lambda_variables]: crate::LogicalPlan::resolve_lambda_variables pub fn lambda_parameters( &self, schema: &dyn ExprSchema, @@ -496,9 +501,20 @@ impl PartialEq for HigherOrderFunction { } /// A named reference to a lambda parameter which includes it's own [`FieldRef`], -/// which is used to implement [`ExprSchemable`], for example. Note the field must -/// be set in order to create a physical lambda variable. A helper to automatically -/// set them will be added in the future +/// which is used to implement [`ExprSchemable`], for example. It is an option only to make +/// easier for `expr_api` users to construct lambda variables, but any expression +/// tree or [`LogicalPlan`] containing unresolved variables must be resolved before +/// usage with either [`Expr::resolve_lambda_variables`] or +/// [`LogicalPlan::resolve_lambda_variables`]. The default SQL planner produces +/// already resolved variables and no further resolving is required. +/// +/// After resolving, if any argument from the lambda function which this +/// variables originates from have it's field changed (type, nullability, +/// metadata, etc), the resolved variable may became outdated and must be +/// resolved again. +/// +/// [`LogicalPlan`]: crate::LogicalPlan +/// [`LogicalPlan::resolve_lambda_variables`]: crate::LogicalPlan::resolve_lambda_variables #[derive(Clone, PartialEq, PartialOrd, Eq, Debug, Hash)] pub struct LambdaVariable { pub name: String, @@ -507,7 +523,12 @@ pub struct LambdaVariable { } impl LambdaVariable { - /// Create a lambda variable from a name and a Field. + /// Create a lambda variable from a name and an optional field. + /// If the field is none, the expression tree or LogicalPlan which + /// owns this variable must be resolved before usage with either + /// [`Expr::resolve_lambda_variables`] or [`LogicalPlan::resolve_lambda_variables`]. + /// + /// [`LogicalPlan::resolve_lambda_variables`]: crate::LogicalPlan::resolve_lambda_variables pub fn new(name: String, field: Option) -> Self { Self { name, @@ -2229,6 +2250,16 @@ impl Expr { None } } + + /// Return a `Expr` with all [`LambdaVariable`] resolved only if all of them + /// are contained in the subtree of the [`HigherOrderFunction`] it originates from, + /// otherwise returns an error + pub fn resolve_lambda_variables( + self, + schema: &DFSchema, + ) -> Result> { + resolve_lambda_variables(self, schema, &mut HashMap::new()) + } } impl Normalizeable for Expr { diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index cec6b7ec0565c..9d711113e4f74 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -728,10 +728,14 @@ pub fn lambda(params: impl IntoIterator>, body: Expr) - )) } -/// Create a lambda variable expression -// todo: make this pub when support for optional field lands -#[expect(unused)] -fn lambda_var(name: impl Into) -> Expr { +/// Create an unresolved lambda variable expression +/// +/// The expression tree or [`LogicalPlan`] which +/// owns this variable must be resolved before usage with either +/// [`Expr::resolve_lambda_variables`] or [`LogicalPlan::resolve_lambda_variables`]. +/// +/// [LogicalPlan::resolve_lambda_variables]: crate::LogicalPlan::resolve_lambda_variables +pub fn lambda_var(name: impl Into) -> Expr { Expr::LambdaVariable(LambdaVariable::new(name.into(), None)) } diff --git a/datafusion/expr/src/higher_order_function.rs b/datafusion/expr/src/higher_order_function.rs index 5fac02846a6bb..3dc143b8e5211 100644 --- a/datafusion/expr/src/higher_order_function.rs +++ b/datafusion/expr/src/higher_order_function.rs @@ -17,13 +17,25 @@ //! [`HigherOrderUDF`]: User Defined Higher Order Functions -use crate::expr::schema_name_from_exprs_comma_separated_without_space; -use crate::{ColumnarValue, Documentation, Expr}; +use crate::expr::{ + HigherOrderFunction, display_comma_separated, + schema_name_from_exprs_comma_separated_without_space, +}; +use crate::type_coercion::functions::value_fields_with_higher_order_udf; +use crate::{ColumnarValue, Documentation, Expr, ExprSchemable}; use arrow::array::{ArrayRef, RecordBatch}; use arrow::datatypes::{DataType, FieldRef, Schema}; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{Result, ScalarValue, exec_err, not_impl_err}; +use datafusion_common::datatype::FieldExt; +use datafusion_common::hash_map::EntryRef; +use datafusion_common::tree_node::{ + Transformed, TreeNode, TreeNodeContainer, TreeNodeRecursion, +}; +use datafusion_common::{ + DFSchema, HashMap, HashSet, Result, ScalarValue, exec_err, internal_datafusion_err, + internal_err, not_impl_err, plan_datafusion_err, plan_err, +}; use datafusion_expr_common::dyn_eq::{DynEq, DynHash}; use datafusion_expr_common::signature::Volatility; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; @@ -31,6 +43,7 @@ use std::any::Any; use std::cmp::Ordering; use std::fmt::Debug; use std::hash::{Hash, Hasher}; +use std::mem; use std::sync::Arc; /// The types of arguments for which a function has implementations. @@ -794,12 +807,279 @@ pub trait HigherOrderUDF: Debug + DynEq + DynHash + Send + Sync + Any { } } +pub(crate) fn resolve_lambda_variables( + expr: Expr, + schema: &DFSchema, + // a map of lambda variable name => a never empty stack of fields [ [..shadowed], in_scope ] + vars: &mut HashMap>, +) -> Result> { + expr.transform_down(|expr| match expr { + Expr::HigherOrderFunction(HigherOrderFunction { func, args }) => { + // not inlined to reduce nesting + resolve_higher_order_function(func, args, schema, vars) + } + Expr::LambdaVariable(mut var) => { + let field_stack = vars.get(&var.name).ok_or_else(|| { + plan_datafusion_err!( + "missing field of lambda variable {} while resolving", + var.name + ) + })?; + + let field = field_stack.last().ok_or_else(|| { + internal_datafusion_err!("every entry should have at least one field") + })?; + + let field = Arc::clone(field).renamed(&var.name); + + let transformed = var.field.as_ref().is_none_or(|old| old != &field); + + var.field = Some(field); + + Ok(Transformed::new_transformed( + Expr::LambdaVariable(var), + transformed, + )) + } + _ => Ok(Transformed::no(expr)), + }) +} + +fn resolve_higher_order_function( + func: Arc, + args: Vec, + schema: &DFSchema, + // a map of lambda variable name => a never empty stack of fields [ [..shadowed], in_scope ] + vars: &mut HashMap>, +) -> Result> { + let args = if !vars.is_empty() { + /* if this is a nested lambda, we must resolve non-lambda args before invoking + lambda_parameters because it will invoke ExprSchemable::to_field for every + non-lambda parameter, and if one them contains a lambda variable, it will fail + due to it being unresolved. Example query: + + array_transform([[1, 2]], a -> array_transform(a, b -> b+1)) + + the nested array_transform's lambda_parameters will call Lambdavariable::to_field + on it's first argument, the variable `a`, which must be resolved + */ + args.map_elements(|arg| match arg { + Expr::Lambda(_) => Ok(Transformed::no(arg)), + _ => resolve_lambda_variables(arg, schema, vars), + })? + } else { + Transformed::no(args) + }; + + let transformed = args.transformed; + let mut args = args.data; + + let current_fields = args + .iter() + .map(|e| match e { + Expr::Lambda(_lambda_function) => Ok(ValueOrLambda::Lambda(None)), + _ => Ok(ValueOrLambda::Value(e.to_field(schema)?.1)), + }) + .collect::>>()?; + + // coerce fields because coercion may alter the lambda parameters + let mut fields = value_fields_with_higher_order_udf(¤t_fields, func.as_ref())?; + + let num_lambdas = args.iter().filter(|a| matches!(a, Expr::Lambda(_))).count(); + + let mut step = 0; + + let lambda_params = loop { + match func.lambda_parameters(step, &fields)? { + LambdaParametersProgress::Partial(params) => { + let mut params = params.into_iter(); + + if params.len() != num_lambdas { + return plan_err!( + "{} lambda_parameters returned {} lambdas but {num_lambdas} expected", + func.name(), + params.len() + ); + } + + for (arg, field) in std::iter::zip(&mut args, &mut fields) { + match (arg, field) { + (Expr::Lambda(lambda), ValueOrLambda::Lambda(field)) => { + let params = params.next().ok_or_else(|| { + internal_datafusion_err!( + "params len should have been checked above" + ) + })?; + + if let Some(params) = params { + for (name, field) in + std::iter::zip(&lambda.params, params) + { + vars.entry_ref(name) + .or_default() + .push(field.renamed(name.as_str())); + } + + let body_with_vars = resolve_lambda_variables( + mem::take(lambda.body.as_mut()), + schema, + vars, + )?; + + remove_scope(vars, &lambda.params)?; + + *field = Some(body_with_vars.data.to_field(schema)?.1); + *lambda.body = body_with_vars.data; + } + } + (_, ValueOrLambda::Lambda(_)) => { + return internal_err!( + "value_fields_with_higher_order_udf returned a value for a lambda argument" + ); + } + (Expr::Lambda(_), ValueOrLambda::Value(_)) => { + return internal_err!( + "value_fields_with_higher_order_udf returned a lambda for a value argument" + ); + } + (_, ValueOrLambda::Value(_)) => {} // nothing to do + } + } + } + LambdaParametersProgress::Complete(params) => break params, + } + + let limit = func.signature().lambda_parameters_max_iterations; + + step += 1; + + if step > limit { + return plan_err!( + "{} lambda_parameters called {limit} times without completion", + func.name() + ); + } + }; + + let mut lambda_params = lambda_params.into_iter(); + + if num_lambdas != lambda_params.len() { + return plan_err!( + "{} lambda_parameters returned {} values for {num_lambdas} lambdas", + func.name(), + lambda_params.len() + ); + } + + let args = args.map_elements(|arg| match arg { + Expr::Lambda(mut lambda) => { + let lambda_params = lambda_params.next().ok_or_else(|| { + internal_datafusion_err!( + "lambda_params len should have been checked above" + ) + })?; + + if lambda.params.len() > lambda_params.len() { + return plan_err!( + "{} lambda defined {} params ({}), but only {} supported", + func.name(), + lambda.params.len(), + display_comma_separated(&lambda.params), + lambda_params.len() + ); + } + + if !all_unique(&lambda.params) { + return plan_err!( + "lambda params must be unique, got ({})", + lambda.params.join(", ") + ); + } + + for (param, field) in std::iter::zip(&lambda.params, lambda_params) { + vars.entry_ref(param) + .or_default() + .push(field.renamed(param.as_str())); + } + + let transformed = + resolve_lambda_variables(mem::take(lambda.body.as_mut()), schema, vars)?; + + *lambda.body = transformed.data; + + remove_scope(vars, &lambda.params)?; + + Ok(Transformed::new( + Expr::Lambda(lambda), + transformed.transformed, + TreeNodeRecursion::Jump, + )) + } + arg => Ok(Transformed::no(arg)), // resolved at the start of the function + })?; + + Ok(Transformed::new( + Expr::HigherOrderFunction(HigherOrderFunction::new(func, args.data)), + transformed || args.transformed, + TreeNodeRecursion::Jump, + )) +} + +fn remove_scope( + vars: &mut HashMap>, + scope: &[String], +) -> Result<()> { + for param in scope { + match vars.entry_ref(param) { + EntryRef::Occupied(mut v) => { + if v.get().len() == 1 { + v.remove(); + } else { + v.get_mut().pop().ok_or_else(|| { + internal_datafusion_err!( + "every entry should have at least one field" + ) + })?; + } + } + EntryRef::Vacant(_v) => { + return internal_err!("no empty value should be in the map"); + } + } + } + + Ok(()) +} + +fn all_unique(params: &[String]) -> bool { + match params.len() { + 0 | 1 => true, + 2 => params[0] != params[1], + _ => { + let mut set = HashSet::with_capacity(params.len()); + + params.iter().all(|p| set.insert(p.as_str())) + } + } +} + #[cfg(test)] mod tests { - use datafusion_expr_common::signature::Volatility; - use super::*; use std::hash::DefaultHasher; + use std::sync::Arc; + + use arrow_schema::{DataType, Field, FieldRef, Schema}; + use datafusion_common::{DFSchema, Result}; + use datafusion_expr_common::columnar_value::ColumnarValue; + use datafusion_expr_common::signature::Volatility; + + use crate::{ + Expr, HigherOrderSignature, HigherOrderUDF, LambdaParametersProgress, + ValueOrLambda, col, + expr::{HigherOrderFunction, LambdaVariable}, + lambda, lambda_var, lit, + }; #[derive(Debug, PartialEq, Eq, Hash)] struct TestHigherOrderUDF { @@ -883,4 +1163,220 @@ mod tests { value.hash(hasher); hasher.finish() } + + #[derive(Debug, PartialEq, Eq, Hash)] + struct MockArrayReduce { + signature: HigherOrderSignature, + } + + impl HigherOrderUDF for MockArrayReduce { + fn name(&self) -> &str { + "array_reduce" + } + + fn aliases(&self) -> &[String] { + &[] + } + + fn signature(&self) -> &HigherOrderSignature { + &self.signature + } + + fn lambda_parameters( + &self, + step: usize, + fields: &[ValueOrLambda>], + ) -> Result { + // optional finish not supported for simplicity + let [ + ValueOrLambda::Value(list), + ValueOrLambda::Value(initial_value), + ValueOrLambda::Lambda(merge), + ValueOrLambda::Lambda(_finish), + ] = fields + else { + unreachable!() + }; + + let list_field = match list.data_type() { + DataType::List(field) => field, + _ => unreachable!(), + }; + + Ok(match (step, merge) { + (0, None) => { + // at the first step, we use the initial_value as merge accumulator, + // and return None for finish since we don't know the output of merge + LambdaParametersProgress::Partial(vec![ + // merge + Some(vec![Arc::clone(initial_value), Arc::clone(list_field)]), + // finish + None, + ]) + } + (1, Some(accumulator)) | (0, Some(accumulator)) => { + // now we can use the merge output as it's accumulator and + // as the finish parameter + LambdaParametersProgress::Complete(vec![ + // merge + vec![Arc::clone(accumulator), Arc::clone(list_field)], + // finish + vec![Arc::clone(accumulator)], + ]) + } + (1, None) => { + unreachable!() + } + _ => unreachable!(), + }) + } + + fn return_field_from_args( + &self, + args: HigherOrderReturnFieldArgs, + ) -> Result { + // optional finish not supported for simplicity + let [ + ValueOrLambda::Value(_list), + ValueOrLambda::Value(_initial_value), + ValueOrLambda::Lambda(_merge), + ValueOrLambda::Lambda(finish), + ] = args.arg_fields + else { + unreachable!() + }; + + Ok(Arc::clone(finish)) + } + + fn invoke_with_args( + &self, + _args: HigherOrderFunctionArgs, + ) -> Result { + unreachable!() + } + } + + #[test] + fn test_resolve_lambda_variables() { + let schema = DFSchema::try_from(Schema::new(vec![Field::new( + "c", + DataType::new_list(DataType::new_list(DataType::Int32, true), true), + true, + )])) + .unwrap(); + + let func = Arc::new(MockArrayReduce { + signature: HigherOrderSignature::variadic_any(Volatility::Immutable), + }) as _; + + /* + array_reduce( + c, + 0, + (acc1, v) -> acc + array_reduce( + v, + 0, + (acc2, v) -> acc2 + acc1 + v, + reduced -> reduced * 2.0 + ), + reduced -> reduced * 2 + ) + */ + let expr = Expr::HigherOrderFunction(HigherOrderFunction::new( + Arc::clone(&func), + vec![ + col("c"), + lit(0), + lambda( + ["acc1", "v"], + lambda_var("acc1") + + Expr::HigherOrderFunction(HigherOrderFunction::new( + Arc::clone(&func), + vec![ + lambda_var("v"), + lit(0), + lambda( + ["acc2", "v"], + lambda_var("acc2") + + lambda_var("acc1") + + lambda_var("v"), + ), + lambda(["reduced"], lambda_var("reduced") * lit(2.0)), + ], + )), + ), + lambda(["reduced"], lambda_var("reduced") * lit(2)), + ], + )); + + let resolved_expr = expr.resolve_lambda_variables(&schema).unwrap().data; + + /* + array_reduce( + c@[[Int32]], + 0@Int64, + (acc1@Float64, v@[Int32]) -> acc@Float64 + array_reduce( + v@[Int32], + 0@Int64, + (acc2@Float64, v@Int32) -> acc2@Float64 + acc1@Float64 + v@Int32, + reducedFloat64 -> reduced@Float64 * 2.0@Float64 + ), + reduced@Float64 -> reduced@Float64 * 2@Int64 + ) + */ + let expected = Expr::HigherOrderFunction(HigherOrderFunction::new( + Arc::clone(&func), + vec![ + col("c"), + lit(0), + lambda( + ["acc1", "v"], + resolved_lambda_var("acc1", DataType::Float64, true) + + Expr::HigherOrderFunction(HigherOrderFunction::new( + Arc::clone(&func), + vec![ + resolved_lambda_var( + "v", + DataType::new_list(DataType::Int32, true), + true, + ), + lit(0), + lambda( + ["acc2", "v"], + resolved_lambda_var("acc2", DataType::Float64, true) + + resolved_lambda_var( + "acc1", + DataType::Float64, + true, + ) + + resolved_lambda_var("v", DataType::Int32, true), + ), + lambda( + ["reduced"], + resolved_lambda_var( + "reduced", + DataType::Float64, + true, + ) * lit(2.0), + ), + ], + )), + ), + lambda( + ["reduced"], + resolved_lambda_var("reduced", DataType::Float64, true) * lit(2), + ), + ], + )); + + assert_eq!(resolved_expr, expected); + } + + fn resolved_lambda_var(name: &str, dt: DataType, nullable: bool) -> Expr { + Expr::LambdaVariable(LambdaVariable::new( + name.into(), + Some(Arc::new(Field::new(name, dt, nullable))), + )) + } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index db8b82fe87a14..c572b202f03ce 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -42,7 +42,7 @@ use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; use crate::utils::{ enumerate_grouping_sets, exprlist_to_fields, find_out_reference_exprs, - grouping_set_expr_count, grouping_set_to_exprlist, split_conjunction, + grouping_set_expr_count, grouping_set_to_exprlist, merge_schema, split_conjunction, }; use crate::{ BinaryExpr, CreateMemoryTable, CreateView, Execute, Expr, ExprSchemable, GroupingSet, @@ -2189,6 +2189,17 @@ impl LogicalPlan { } Wrapper(self) } + + /// Return a `LogicalPLan` with all [`LambdaVariable`]'s resolved + /// + /// [`LambdaVariable`]: crate::expr::LambdaVariable + pub fn resolve_lambda_variables(self) -> Result> { + self.transform_with_subqueries(|plan| { + let schema = merge_schema(&plan.inputs()); + + plan.map_expressions(|expr| expr.resolve_lambda_variables(&schema)) + }) + } } impl Display for LogicalPlan { diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 334c1fa2a090b..a61d9d689ae7a 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -252,15 +252,14 @@ impl WindowFrame { // one column. However, an ORDER BY clause may be absent or have // more than one column when the start/end bounds are UNBOUNDED or // CURRENT ROW. - WindowFrameUnits::Range if self.free_range() => { + WindowFrameUnits::Range if self.free_range() && order_by.is_empty() => { // If an ORDER BY clause is absent, it is equivalent to an // ORDER BY clause with constant value as sort key. If an // ORDER BY clause is present but has more than one column, // it is unchanged. Note that this follows PostgreSQL behavior. - if order_by.is_empty() { - order_by.push(lit(1u64).sort(true, false)); - } + order_by.push(lit(1u64).sort(true, false)); } + WindowFrameUnits::Range if self.free_range() => {} WindowFrameUnits::Range if order_by.len() != 1 => { return plan_err!("RANGE requires exactly one ORDER BY column"); } diff --git a/datafusion/expr/src/window_state.rs b/datafusion/expr/src/window_state.rs index d7da7a778b011..f8d4609d3690c 100644 --- a/datafusion/expr/src/window_state.rs +++ b/datafusion/expr/src/window_state.rs @@ -396,6 +396,11 @@ impl WindowFrameStateRange { length: usize, ) -> Result { let current_row_values = get_row_at_idx(range_columns, idx)?; + let search_start = if SIDE { + last_range.start + } else { + last_range.end + }; let end_range = if let Some(delta) = delta { let is_descending: bool = self .sort_options @@ -407,34 +412,40 @@ impl WindowFrameStateRange { })? .descending; - current_row_values - .iter() - .map(|value| { - if value.is_null() { - return Ok(value.clone()); + // On overflow the boundary exceeds the type's range and is + // effectively unbounded within the partition. Collapse to the + // partition edge rather than feeding `search_in_slice` a + // wrapped-around target: PRECEDING searches reach `search_start`, + // FOLLOWING searches reach `length`. + let unbounded_edge = if SEARCH_SIDE { search_start } else { length }; + let mut targets = Vec::with_capacity(current_row_values.len()); + for value in ¤t_row_values { + if value.is_null() { + targets.push(value.clone()); + continue; + } + let target = if SEARCH_SIDE == is_descending { + match value.add_checked(delta) { + Ok(v) => v, + Err(_) => return Ok(unbounded_edge), } - if SEARCH_SIDE == is_descending { - // TODO: Handle positive overflows. - value.add(delta) - } else if value.is_unsigned() && value < delta { - // NOTE: This gets a polymorphic zero without having long coercion code for ScalarValue. - // If we decide to implement a "default" construction mechanism for ScalarValue, - // change the following statement to use that. - value.sub(value) - } else { - // TODO: Handle negative overflows. - value.sub(delta) + } else if value.is_unsigned() && value < delta { + // NOTE: This gets a polymorphic zero without having long coercion code for ScalarValue. + // If we decide to implement a "default" construction mechanism for ScalarValue, + // change the following statement to use that. + value.sub(value)? + } else { + match value.sub_checked(delta) { + Ok(v) => v, + Err(_) => return Ok(unbounded_edge), } - }) - .collect::>>()? + }; + targets.push(target); + } + targets } else { current_row_values }; - let search_start = if SIDE { - last_range.start - } else { - last_range.end - }; let compare_fn = |current: &[ScalarValue], target: &[ScalarValue]| { let cmp = compare_rows(current, target, &self.sort_options)?; Ok(if SIDE { cmp.is_lt() } else { cmp.is_le() }) diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index ea9a12665ad4c..7eed11c0c69e8 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -48,6 +48,7 @@ arrow = { workspace = true, features = ["ffi"] } arrow-schema = { workspace = true } async-ffi = { version = "0.5.0" } async-trait = { workspace = true } +chrono = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } datafusion-datasource = { workspace = true } diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index 7541880a233c8..1a8c9767fbed3 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -23,6 +23,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr_common::metrics::MetricsSet; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; @@ -32,6 +33,7 @@ use tokio::runtime::Handle; use crate::config::FFI_ConfigOptions; use crate::execution::FFI_TaskContext; +use crate::physical_expr::metrics::FFI_MetricsSet; use crate::plan_properties::FFI_PlanProperties; use crate::record_batch_stream::FFI_RecordBatchStream; use crate::util::{FFI_Option, FFI_Result}; @@ -68,6 +70,10 @@ pub struct FFI_ExecutionPlan { ) -> FFI_Result>, + /// Snapshot the plan's execution metrics. Returns `None` when the + /// underlying [`ExecutionPlan::metrics`] returned `None`. + pub metrics: unsafe extern "C" fn(plan: &Self) -> FFI_Option, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -179,6 +185,16 @@ unsafe extern "C" fn name_fn_wrapper(plan: &FFI_ExecutionPlan) -> SString { plan.inner().name().into() } +unsafe extern "C" fn metrics_fn_wrapper( + plan: &FFI_ExecutionPlan, +) -> FFI_Option { + plan.inner() + .metrics() + .as_ref() + .map(FFI_MetricsSet::from) + .into() +} + unsafe extern "C" fn release_fn_wrapper(plan: &mut FFI_ExecutionPlan) { unsafe { debug_assert!(!plan.private_data.is_null()); @@ -270,6 +286,7 @@ impl FFI_ExecutionPlan { name: name_fn_wrapper, execute: execute_fn_wrapper, repartitioned: repartitioned_fn_wrapper, + metrics: metrics_fn_wrapper, clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -431,6 +448,12 @@ impl ExecutionPlan for ForeignExecutionPlan { .map(|plan| >::try_from(&plan)) .transpose() } + + fn metrics(&self) -> Option { + let ffi: Option = + unsafe { (self.plan.metrics)(&self.plan) }.into(); + ffi.map(MetricsSet::from) + } } #[cfg(any(test, feature = "integration-tests"))] @@ -444,6 +467,7 @@ pub mod tests { pub struct EmptyExec { props: Arc, children: Vec>, + metrics: Option, } impl EmptyExec { @@ -456,8 +480,14 @@ pub mod tests { Boundedness::Bounded, )), children: Vec::default(), + metrics: None, } } + + pub fn with_metrics(mut self, metrics: MetricsSet) -> Self { + self.metrics = Some(metrics); + self + } } impl DisplayAs for EmptyExec { @@ -490,6 +520,7 @@ pub mod tests { Ok(Arc::new(EmptyExec { props: Arc::clone(&self.props), children, + metrics: self.metrics.clone(), })) } @@ -501,6 +532,10 @@ pub mod tests { unimplemented!() } + fn metrics(&self) -> Option { + self.metrics.clone() + } + fn apply_expressions( &self, f: &mut dyn FnMut( @@ -587,6 +622,43 @@ pub mod tests { Ok(()) } + #[test] + fn test_ffi_execution_plan_metrics_round_trip() -> Result<()> { + use datafusion_physical_expr_common::metrics::{Count, Metric, MetricValue}; + + let schema = Arc::new(arrow::datatypes::Schema::new(vec![ + arrow::datatypes::Field::new("a", arrow::datatypes::DataType::Float32, false), + ])); + + // Plans without metrics still return None across the boundary. + let bare_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); + let mut bare_local = FFI_ExecutionPlan::new(bare_plan, None); + bare_local.library_marker_id = crate::mock_foreign_marker_id; + let bare_foreign: Arc = (&bare_local).try_into()?; + assert!(bare_foreign.metrics().is_none()); + + // Plans with metrics produce equivalent MetricsSets after a round trip. + let mut original_metrics = MetricsSet::new(); + let c0 = Count::new(); + c0.add(11); + original_metrics + .push(Arc::new(Metric::new(MetricValue::OutputRows(c0), Some(0)))); + let c1 = Count::new(); + c1.add(31); + original_metrics + .push(Arc::new(Metric::new(MetricValue::OutputRows(c1), Some(1)))); + + let metric_plan = Arc::new(EmptyExec::new(schema).with_metrics(original_metrics)); + let mut metric_local = FFI_ExecutionPlan::new(metric_plan, None); + metric_local.library_marker_id = crate::mock_foreign_marker_id; + let metric_foreign: Arc = (&metric_local).try_into()?; + + let observed = metric_foreign.metrics().expect("metrics should be present"); + assert_eq!(observed.output_rows(), Some(42)); + + Ok(()) + } + #[test] fn test_ffi_execution_plan_local_bypass() { let schema = Arc::new(arrow::datatypes::Schema::new(vec![ diff --git a/datafusion/ffi/src/physical_expr/metrics.rs b/datafusion/ffi/src/physical_expr/metrics.rs new file mode 100644 index 0000000000000..6c29bd0ea6095 --- /dev/null +++ b/datafusion/ffi/src/physical_expr/metrics.rs @@ -0,0 +1,803 @@ +// 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. + +//! FFI-stable mirrors of [`MetricsSet`] and related metric types. +//! +//! Metrics are passed across the FFI boundary as a **snapshot**: all +//! atomic-backed counters/gauges/timers are read into plain integer fields +//! at conversion time. Callers re-invoke [`ExecutionPlan::metrics()`] across +//! the boundary to observe newer values. This matches the documented contract +//! ("Once `self.execute()` has returned... metrics should be complete") and +//! all in-tree consumers (`AnalyzeExec`, `DisplayableExecutionPlan`). +//! +//! The variant *order* of [`FFI_MetricValue`] is part of the stable ABI and +//! must not be reordered. New variants must be appended at the end. +//! +//! [`ExecutionPlan::metrics()`]: datafusion_physical_plan::ExecutionPlan::metrics + +use std::any::Any; +use std::borrow::Cow; +use std::fmt::{self, Debug, Display}; +use std::sync::Arc; + +use chrono::{DateTime, Utc}; +use datafusion_common::format::{MetricCategory, MetricType}; +use datafusion_physical_expr_common::metrics::{ + Count, CustomMetricValue, Gauge, MetricValue, MetricsSet, PruningMetrics, + RatioMergeStrategy, RatioMetrics, Time, Timestamp, +}; +use datafusion_physical_expr_common::metrics::{Label, Metric}; +use stabby::string::String as SString; +use stabby::vec::Vec as SVec; + +use crate::ffi_option::FFI_Option; + +/// FFI-stable mirror of [`MetricsSet`]. +#[repr(C)] +#[derive(Debug, Clone)] +pub struct FFI_MetricsSet { + pub metrics: SVec, +} + +/// FFI-stable mirror of [`Metric`]. +#[repr(C)] +#[derive(Debug, Clone)] +pub struct FFI_Metric { + pub value: FFI_MetricValue, + pub labels: SVec, + pub partition: FFI_Option, + pub metric_type: FFI_MetricType, + pub metric_category: FFI_Option, +} + +/// FFI-stable mirror of [`Label`]. +#[repr(C)] +#[derive(Debug, Clone)] +pub struct FFI_Label { + pub name: SString, + pub value: SString, +} + +/// FFI-stable mirror of [`MetricType`]. +#[expect(non_camel_case_types)] +#[repr(u8)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum FFI_MetricType { + Summary, + Dev, +} + +/// FFI-stable mirror of [`MetricCategory`]. +#[expect(non_camel_case_types)] +#[repr(u8)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum FFI_MetricCategory { + Rows, + Bytes, + Timing, + Uncategorized, +} + +/// FFI-stable mirror of [`PruningMetrics`]. All counts are snapshotted at +/// conversion time. +#[repr(C)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct FFI_PruningMetrics { + pub pruned: u64, + pub matched: u64, + pub fully_matched: u64, +} + +/// FFI-stable mirror of [`RatioMergeStrategy`]. +#[expect(non_camel_case_types)] +#[expect( + clippy::enum_variant_names, + reason = "match RatioMergeStrategy variants" +)] +#[repr(u8)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum FFI_RatioMergeStrategy { + AddPartAddTotal, + AddPartSetTotal, + SetPartAddTotal, +} + +/// FFI-stable mirror of [`RatioMetrics`]. Numerator/denominator are +/// snapshotted at conversion time. +#[repr(C)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct FFI_RatioMetrics { + pub part: u64, + pub total: u64, + pub merge_strategy: FFI_RatioMergeStrategy, + pub display_raw_values: bool, +} + +/// FFI-stable mirror of [`MetricValue`]. +#[repr(C, u8)] +#[derive(Debug, Clone)] +pub enum FFI_MetricValue { + OutputRows(u64), + ElapsedComputeNs(u64), + SpillCount(u64), + SpilledBytes(u64), + OutputBytes(u64), + OutputBatches(u64), + SpilledRows(u64), + CurrentMemoryUsage(u64), + Count { + name: SString, + count: u64, + }, + Gauge { + name: SString, + gauge: u64, + }, + Time { + name: SString, + time_ns: u64, + }, + StartTimestampNsUTC(FFI_Option), + EndTimestampNsUTC(FFI_Option), + PruningMetrics { + name: SString, + pruning_metrics: FFI_PruningMetrics, + }, + Ratio { + name: SString, + ratio_metrics: FFI_RatioMetrics, + }, + /// Custom metrics are marshalled as their `Display` output plus the + /// `as_usize()` fallback. The underlying `dyn CustomMetricValue` type is + /// not preserved across the boundary, so `aggregate`/`as_any` downcasting + /// are lost; the reconstructed value uses [`FfiCustomMetricValue`]. + Custom { + name: SString, + display: SString, + as_usize_value: u64, + }, +} + +// ----------------------------------------------------------------------------- +// MetricsSet <-> FFI_MetricsSet +// ----------------------------------------------------------------------------- + +impl From<&MetricsSet> for FFI_MetricsSet { + fn from(set: &MetricsSet) -> Self { + Self { + metrics: set.iter().map(|m| FFI_Metric::from(m.as_ref())).collect(), + } + } +} + +impl From for MetricsSet { + fn from(set: FFI_MetricsSet) -> Self { + let mut out = MetricsSet::new(); + for ffi_metric in set.metrics { + out.push(Arc::new(Metric::from(ffi_metric))); + } + out + } +} + +// ----------------------------------------------------------------------------- +// Metric <-> FFI_Metric +// ----------------------------------------------------------------------------- + +impl From<&Metric> for FFI_Metric { + fn from(m: &Metric) -> Self { + Self { + value: FFI_MetricValue::from(m.value()), + labels: m.labels().iter().map(FFI_Label::from).collect(), + partition: m.partition().map(|p| p as u64).into(), + metric_type: m.metric_type().into(), + metric_category: m.metric_category().map(FFI_MetricCategory::from).into(), + } + } +} + +impl From for Metric { + fn from(m: FFI_Metric) -> Self { + let labels: Vec