Dandandan opened a new pull request, #9574: URL: https://github.com/apache/arrow-rs/pull/9574
## Summary - Adds dictionary page pruning for row filter predicates in the parquet reader - When evaluating predicates on dictionary-encoded columns, the predicate is first evaluated against dictionary values before decoding data pages - If **no** dictionary values match (`AllFalse`): skip the entire column chunk - If **all** dictionary values match (`AllTrue`): skip per-row predicate evaluation - Adds `evaluate_dictionary` method to `ArrowPredicate` trait with a default implementation that delegates to `evaluate` ## Details - Supports BYTE_ARRAY (strings), INT32, and INT64 physical types - Only applies when all data pages use dictionary encoding (no plain fallback) - Uses column encoding metadata and page encoding stats to verify safety - Currently implemented for the async push decoder path ## Benchmark Results (ClickBench async_object_store) | Query | Before | After | Change | Notes | |-------|--------|-------|--------|-------| | Q19 | 2.57ms | 1.66ms | **-35%** | `CounterID=62` — prunes 1 of 3 row groups | | Q42 | 3.63ms | 3.35ms | **-8%** | Same CounterID filter | | Q36 | 17.3ms | 16.7ms | **-3%** | CounterID + string predicates | | Others | — | — | ~0% | No regressions | The optimization is most effective for selective equality filters on dictionary-encoded columns where the target value doesn't appear in some row groups' dictionaries. ## Test plan - [ ] Existing parquet tests pass - [ ] ClickBench benchmark verifies correctness (row counts match expected) - [ ] Tested with columns that have plain encoding fallback (correctly skips pruning) - [ ] Structure size test unchanged (no new state machine variants) 🤖 Generated with [Claude Code](https://claude.com/claude-code) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
