This is an automated email from the ASF dual-hosted git repository.
alamb pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git
The following commit(s) were added to refs/heads/main by this push:
new 4ddaa8c1cf doc: add example of RowFilter usage (#9115)
4ddaa8c1cf is described below
commit 4ddaa8c1cf8e79ca778da998b7933285dc67d3ee
Author: Sonny <[email protected]>
AuthorDate: Tue Jan 13 20:33:24 2026 +0700
doc: add example of RowFilter usage (#9115)
# Which issue does this PR close?
- Closes #9096.
# Rationale for this change
The RowFilter API does exist and can evaluate predicates during
evaluation, but it has no examples.
# What changes are included in this PR?
- Added a rustdoc example and blog link to
`ParquetRecordBatchReaderBuilder::with_row_filter`.
- Added a running example in `parquet/examples/read_with_row_filter.rs`
# Are these changes tested?
Yes
```
cargo run -p parquet --example read_with_row_filter
cargo test -p parquet --doc
```
# Are there any user-facing changes?
Yes, doc only. No API changes.
---
parquet/Cargo.toml | 5 ++++
parquet/examples/read_with_row_filter.rs | 49 ++++++++++++++++++++++++++++++++
parquet/src/arrow/arrow_reader/mod.rs | 30 +++++++++++++++++++
3 files changed, 84 insertions(+)
diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml
index 454f8455ff..519fbc903d 100644
--- a/parquet/Cargo.toml
+++ b/parquet/Cargo.toml
@@ -156,6 +156,11 @@ name = "async_read_parquet"
required-features = ["arrow", "async"]
path = "./examples/async_read_parquet.rs"
+[[example]]
+name = "read_with_row_filter"
+required-features = ["arrow"]
+path = "./examples/read_with_row_filter.rs"
+
[[example]]
name = "read_with_rowgroup"
required-features = ["arrow", "async"]
diff --git a/parquet/examples/read_with_row_filter.rs
b/parquet/examples/read_with_row_filter.rs
new file mode 100644
index 0000000000..e0efb536b4
--- /dev/null
+++ b/parquet/examples/read_with_row_filter.rs
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use arrow_array::Int32Array;
+use arrow_cast::pretty::print_batches;
+use parquet::arrow::ProjectionMask;
+use parquet::arrow::arrow_reader::{ArrowPredicateFn,
ParquetRecordBatchReaderBuilder, RowFilter};
+use parquet::errors::Result;
+use std::fs::File;
+
+// RowFilter / with_row_filter usage. For background and more
+// context, see
<https://arrow.apache.org/blog/2025/12/11/parquet-late-materialization-deep-dive/>
+fn main() -> Result<()> {
+ let testdata = arrow::util::test_util::parquet_test_data();
+ let path = format!("{testdata}/alltypes_plain.parquet");
+ let file = File::open(&path)?;
+ let builder = ParquetRecordBatchReaderBuilder::try_new(file)?;
+ let schema_desc = builder.metadata().file_metadata().schema_descr_ptr();
+
+ // Create predicate: column id > 4. This col has index 0.
+ // Projection mask ensures only predicate columns are read to evaluate the
filter.
+ let projection_mask = ProjectionMask::leaves(&schema_desc, [0]);
+ let predicate = ArrowPredicateFn::new(projection_mask, |batch| {
+ let id_col = batch.column(0);
+ arrow::compute::kernels::cmp::gt(id_col, &Int32Array::new_scalar(4))
+ });
+
+ let row_filter = RowFilter::new(vec![Box::new(predicate)]);
+ let reader = builder.with_row_filter(row_filter).build()?;
+
+ let filtered_batches: Vec<_> = reader.map(|b| b.unwrap()).collect();
+ print_batches(&filtered_batches)?;
+
+ Ok(())
+}
diff --git a/parquet/src/arrow/arrow_reader/mod.rs
b/parquet/src/arrow/arrow_reader/mod.rs
index 3e7714a3b6..b0563d0d69 100644
--- a/parquet/src/arrow/arrow_reader/mod.rs
+++ b/parquet/src/arrow/arrow_reader/mod.rs
@@ -304,6 +304,36 @@ impl<T> ArrowReaderBuilder<T> {
///
/// It is recommended to enable reading the page index if using this
functionality, to allow
/// more efficient skipping over data pages. See
[`ArrowReaderOptions::with_page_index`].
+ ///
+ /// For a running example see `parquet/examples/read_with_row_filter.rs`.
+ /// See
<https://arrow.apache.org/blog/2025/12/11/parquet-late-materialization-deep-dive/>
+ /// for a technical explanation of late materialization.
+ ///
+ /// # Example
+ /// ```rust
+ /// # use std::fs::File;
+ /// # use arrow_array::Int32Array;
+ /// # use parquet::arrow::ProjectionMask;
+ /// # use parquet::arrow::arrow_reader::{ArrowPredicateFn,
ParquetRecordBatchReaderBuilder, RowFilter};
+ /// # fn main() -> Result<(), parquet::errors::ParquetError> {
+ /// # let testdata = arrow::util::test_util::parquet_test_data();
+ /// # let path = format!("{testdata}/alltypes_plain.parquet");
+ /// # let file = File::open(&path)?;
+ /// let builder = ParquetRecordBatchReaderBuilder::try_new(file)?;
+ /// let schema_desc =
builder.metadata().file_metadata().schema_descr_ptr();
+ ///
+ /// // Create predicate: column id > 4. This col has index 0.
+ /// let projection = ProjectionMask::leaves(&schema_desc, [0]);
+ /// let predicate = ArrowPredicateFn::new(projection, |batch| {
+ /// let id_col = batch.column(0);
+ /// arrow::compute::kernels::cmp::gt(id_col,
&Int32Array::new_scalar(4))
+ /// });
+ ///
+ /// let row_filter = RowFilter::new(vec![Box::new(predicate)]);
+ /// let _reader = builder.with_row_filter(row_filter).build()?;
+ /// # Ok(())
+ /// # }
+ /// ```
pub fn with_row_filter(self, filter: RowFilter) -> Self {
Self {
filter: Some(filter),