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 156c941ff9 Improve `ArrowReaderBuilder::with_row_filter` documentation
(#9153)
156c941ff9 is described below
commit 156c941ff9905ed620cdd97d0601d3defc23b1b4
Author: Andrew Lamb <[email protected]>
AuthorDate: Wed Jan 14 17:23:31 2026 -0500
Improve `ArrowReaderBuilder::with_row_filter` documentation (#9153)
# Which issue does this PR close?
- part of https://github.com/apache/arrow-rs/issues/9096
- Follow on to https://github.com/apache/arrow-rs/pull/9115
# Rationale for this change
@sonhmai started us off with
https://github.com/apache/arrow-rs/pull/9115
@Jefffrey and I had some suggestions on the PR and I found some more
while going through it again, so I figured I would make anew PR
# What changes are included in this PR?
1. Improve the documentation
2. Improve the doc comment example
3. Remove redundant example in parquet/examples/read_with_row_filter.rs
# Are these changes tested?
By CI
# Are there any user-facing changes?
<!--
If there are user-facing changes then we may require documentation to be
updated before approving the PR.
If there are any breaking changes to public APIs, please call them out.
-->
---------
Co-authored-by: Ed Seidl <[email protected]>
---
parquet/Cargo.toml | 5 ----
parquet/examples/read_with_row_filter.rs | 49 --------------------------------
parquet/src/arrow/arrow_reader/filter.rs | 6 +++-
parquet/src/arrow/arrow_reader/mod.rs | 23 ++++++++-------
4 files changed, 18 insertions(+), 65 deletions(-)
diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml
index 519fbc903d..454f8455ff 100644
--- a/parquet/Cargo.toml
+++ b/parquet/Cargo.toml
@@ -156,11 +156,6 @@ 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
deleted file mode 100644
index e0efb536b4..0000000000
--- a/parquet/examples/read_with_row_filter.rs
+++ /dev/null
@@ -1,49 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements. See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership. The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License. You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied. See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-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/filter.rs
b/parquet/src/arrow/arrow_reader/filter.rs
index 4fbe45748b..3fd5e1d650 100644
--- a/parquet/src/arrow/arrow_reader/filter.rs
+++ b/parquet/src/arrow/arrow_reader/filter.rs
@@ -134,9 +134,12 @@ where
/// Filter applied *during* the parquet read process
///
+/// See example on [`ArrowReaderBuilder::with_row_filter`]
+///
/// [`RowFilter`] applies predicates in order, after decoding only the columns
/// required. As predicates eliminate rows, fewer rows from subsequent columns
-/// may be required, thus potentially reducing IO and decode.
+/// may be required, thus potentially reducing IO and decode. This process is
+/// also known as *push down* filtering and *late materialization*.
///
/// A `RowFilter` consists of a list of [`ArrowPredicate`]s. Only the rows for
which
/// all the predicates evaluate to `true` will be returned.
@@ -170,6 +173,7 @@ where
/// not contiguous.
///
/// [`RowSelection`]: crate::arrow::arrow_reader::RowSelection
+/// [`ArrowReaderBuilder::with_row_filter`]:
crate::arrow::arrow_reader::ArrowReaderBuilder::with_row_filter
pub struct RowFilter {
/// A list of [`ArrowPredicate`]
pub(crate) predicates: Vec<Box<dyn ArrowPredicate>>,
diff --git a/parquet/src/arrow/arrow_reader/mod.rs
b/parquet/src/arrow/arrow_reader/mod.rs
index 24530c75b0..5641b4e554 100644
--- a/parquet/src/arrow/arrow_reader/mod.rs
+++ b/parquet/src/arrow/arrow_reader/mod.rs
@@ -305,9 +305,9 @@ 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.
+ /// See the [blog post on late materialization] for a more technical
explanation.
+ ///
+ /// [blog post on late materialization]:
https://arrow.apache.org/blog/2025/12/11/parquet-late-materialization-deep-dive
///
/// # Example
/// ```rust
@@ -321,16 +321,19 @@ impl<T> ArrowReaderBuilder<T> {
/// # 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]);
+ /// // Create predicate that evaluates `int_col != 1`.
+ /// // `int_col` column has index 4 (zero based) in the schema
+ /// let projection = ProjectionMask::leaves(&schema_desc, [4]);
+ /// // Only the projection columns are passed to the predicate so
+ /// // int_col is column 0 in the predicate
/// let predicate = ArrowPredicateFn::new(projection, |batch| {
- /// let id_col = batch.column(0);
- /// arrow::compute::kernels::cmp::gt(id_col,
&Int32Array::new_scalar(4))
+ /// let int_col = batch.column(0);
+ /// arrow::compute::kernels::cmp::neq(int_col,
&Int32Array::new_scalar(1))
/// });
- ///
/// let row_filter = RowFilter::new(vec![Box::new(predicate)]);
- /// let _reader = builder.with_row_filter(row_filter).build()?;
+ /// // The filter will be invoked during the reading process
+ /// let reader = builder.with_row_filter(row_filter).build()?;
+ /// # for b in reader { let _ = b?; }
/// # Ok(())
/// # }
/// ```