From 37ab45a06f63a5de2c56e7c7853cb1dae0bab49a Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Wed, 5 Nov 2025 17:00:07 +0800 Subject: [PATCH 01/27] base code --- parquet/Cargo.toml | 5 + parquet/benches/row_selection_state.rs | 467 ++++++++++++++++++ parquet/src/arrow/arrow_reader/mod.rs | 191 ++++++- parquet/src/arrow/arrow_reader/read_plan.rs | 228 ++++++++- parquet/src/arrow/async_reader/mod.rs | 85 +++- parquet/src/arrow/push_decoder/mod.rs | 1 + .../arrow/push_decoder/reader_builder/mod.rs | 2 +- 7 files changed, 963 insertions(+), 16 deletions(-) create mode 100644 parquet/benches/row_selection_state.rs diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index aa0071ca38e5..79569691b214 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -260,5 +260,10 @@ name = "row_selector" harness = false required-features = ["arrow"] +[[bench]] +name = "row_selection_state" +harness = false +required-features = ["arrow"] + [lib] bench = false diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs new file mode 100644 index 000000000000..057752eec1f1 --- /dev/null +++ b/parquet/benches/row_selection_state.rs @@ -0,0 +1,467 @@ +// 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 std::hint; +use std::sync::Arc; + +use arrow_array::builder::StringViewBuilder; +use arrow_array::{ArrayRef, Float64Array, Int32Array, RecordBatch, StringViewArray}; +use arrow_schema::{DataType, Field, Schema}; +use bytes::Bytes; +use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; +use parquet::arrow::ArrowWriter; +use parquet::arrow::arrow_reader::{ + ParquetRecordBatchReaderBuilder, RowSelection, RowSelectionStrategy, RowSelector, +}; +use rand::rngs::StdRng; +use rand::{Rng, SeedableRng}; + +const TOTAL_ROWS: usize = 1 << 20; +const BATCH_SIZE: usize = 1 << 10; +const BASE_SEED: u64 = 0xA55AA55A; +const READ_STRATEGIES: &[(&str, RowSelectionStrategy)] = &[ + ("read_mask", RowSelectionStrategy::Mask), + ("read_selectors", RowSelectionStrategy::Selectors), +]; +const AVG_SELECTOR_LENGTHS: &[usize] = &[4, 8, 12, 16, 20, 24, 28, 32, 36, 40]; +const COLUMN_WIDTHS: &[usize] = &[2, 4, 8, 16, 32]; + +struct DataProfile { + name: &'static str, + build_batch: fn(usize) -> RecordBatch, +} + +const DATA_PROFILES: &[DataProfile] = &[ + DataProfile { + name: "int32", + build_batch: build_int32_batch, + }, + DataProfile { + name: "float64", + build_batch: build_float64_batch, + }, + DataProfile { + name: "utf8view", + build_batch: build_utf8view_batch, + }, +]; + +fn criterion_benchmark(c: &mut Criterion) { + /* uniform50 (50% selected, constant run lengths, starts with skip) + ```text + ┌───────────────┐ + │ │ skip + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ │ skip + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ ... │ + └───────────────┘ + ``` */ + let default_scenario = Scenario { + name: "uniform50", + select_ratio: 0.5, + start_with_select: false, + distribution: RunDistribution::Constant, + }; + + let default_parquet = build_parquet_data(TOTAL_ROWS, build_int32_batch); + bench_over_lengths( + c, + "len", + "default", + &default_parquet, + &default_scenario, + BASE_SEED, + ); + + // Additional scenarios reuse the same data/type defaults but change row selection shapes. + let scenarios = [ + /* spread50 (50% selected, large jitter in run lengths, starts with skip) + ```text + ┌───────────────┐ + │ │ skip (long) + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (short) + │ │ skip (short) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (long) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ │ skip (medium) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (medium) + │ ... │ + └───────────────┘ + ``` */ + Scenario { + name: "spread50", + select_ratio: 0.5, + start_with_select: false, + distribution: RunDistribution::Uniform { spread: 0.9 }, + }, + /* sparse20 (20% selected, bimodal: occasional long runs, starts with skip) + ```text + ┌───────────────┐ + │ │ skip (long) + │ │ + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (short) + │ │ skip (long) + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (occasional long) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ ... │ + └───────────────┘ + ``` */ + Scenario { + name: "sparse20", + select_ratio: 0.2, + start_with_select: false, + distribution: RunDistribution::Bimodal { + long_factor: 6.0, + long_prob: 0.1, + }, + }, + /* dense80 (80% selected, bimodal: occasional long runs, starts with select) + ```text + ┌───────────────┐ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (long) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ │ skip (short) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (long) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ │ skip (very short) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select (long) + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ ... │ + └───────────────┘ + ``` */ + Scenario { + name: "dense80", + select_ratio: 0.8, + start_with_select: true, + distribution: RunDistribution::Bimodal { + long_factor: 4.0, + long_prob: 0.05, + }, + }, + ]; + + for (idx, scenario) in scenarios.iter().enumerate() { + bench_over_lengths( + c, + "scenario", + scenario.name, + &default_parquet, + scenario, + BASE_SEED ^ ((idx as u64 + 1) << 16), + ); + } + + for (profile_idx, profile) in DATA_PROFILES.iter().enumerate() { + let parquet_data = build_parquet_data(TOTAL_ROWS, profile.build_batch); + bench_over_lengths( + c, + "dtype", + profile.name, + &parquet_data, + &default_scenario, + BASE_SEED ^ ((profile_idx as u64) << 24), + ); + } + + for (offset, &column_count) in COLUMN_WIDTHS.iter().enumerate() { + let parquet_data = write_parquet_batch(build_int32_columns_batch(TOTAL_ROWS, column_count)); + let variant_label = format!("C{:02}", column_count); + bench_over_lengths( + c, + "columns", + &variant_label, + &parquet_data, + &default_scenario, + BASE_SEED ^ ((offset as u64) << 32), + ); + } +} + +fn bench_over_lengths( + c: &mut Criterion, + suite: &str, + variant: &str, + parquet_data: &Bytes, + scenario: &Scenario, + seed_base: u64, +) { + for (offset, &avg_len) in AVG_SELECTOR_LENGTHS.iter().enumerate() { + let selectors = + generate_selectors(avg_len, TOTAL_ROWS, scenario, seed_base + offset as u64); + let stats = SelectorStats::new(&selectors); + let selection = RowSelection::from(selectors); + let suffix = format!( + "{}-{}-{}-L{:02}-avg{:.1}-sel{:02}", + suite, + scenario.name, + variant, + avg_len, + stats.average_selector_len, + (stats.select_ratio * 100.0).round() as u32 + ); + + let bench_input = BenchInput { + parquet_data: parquet_data.clone(), + selection, + }; + + for (label, strategy) in READ_STRATEGIES.iter().copied() { + c.bench_with_input( + BenchmarkId::new(label, &suffix), + &bench_input, + |b, input| { + b.iter(|| { + let total = run_read(&input.parquet_data, &input.selection, strategy); + hint::black_box(total); + }); + }, + ); + } + } +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); + +struct BenchInput { + parquet_data: Bytes, + selection: RowSelection, +} + +fn run_read( + parquet_data: &Bytes, + selection: &RowSelection, + strategy: RowSelectionStrategy, +) -> usize { + let reader = ParquetRecordBatchReaderBuilder::try_new(parquet_data.clone()) + .unwrap() + .with_batch_size(BATCH_SIZE) + .with_row_selection(selection.clone()) + .with_row_selection_strategy(strategy) + .build() + .unwrap(); + + let mut total_rows = 0usize; + for batch in reader { + let batch = batch.unwrap(); + total_rows += batch.num_rows(); + } + total_rows +} + +fn build_parquet_data(total_rows: usize, build_batch: fn(usize) -> RecordBatch) -> Bytes { + let batch = build_batch(total_rows); + write_parquet_batch(batch) +} + +fn build_single_column_batch(data_type: DataType, array: ArrayRef) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("value", data_type, false)])); + RecordBatch::try_new(schema, vec![array]).unwrap() +} + +fn build_int32_batch(total_rows: usize) -> RecordBatch { + let values = Int32Array::from_iter_values((0..total_rows).map(|v| v as i32)); + build_single_column_batch(DataType::Int32, Arc::new(values) as ArrayRef) +} + +fn build_float64_batch(total_rows: usize) -> RecordBatch { + let values = Float64Array::from_iter_values((0..total_rows).map(|v| v as f64)); + build_single_column_batch(DataType::Float64, Arc::new(values) as ArrayRef) +} + +fn build_utf8view_batch(total_rows: usize) -> RecordBatch { + let mut builder = StringViewBuilder::new(); + // Mix short and long values. + for i in 0..total_rows { + match i % 5 { + 0 => builder.append_value("alpha"), + 1 => builder.append_value("beta"), + 2 => builder.append_value("gamma"), + 3 => builder.append_value("delta"), + _ => builder.append_value("a longer utf8 string payload to test view storage"), + } + } + let values: StringViewArray = builder.finish(); + build_single_column_batch(DataType::Utf8View, Arc::new(values) as ArrayRef) +} + +fn build_int32_columns_batch(total_rows: usize, num_columns: usize) -> RecordBatch { + let base_values: ArrayRef = Arc::new(Int32Array::from_iter_values( + (0..total_rows).map(|v| v as i32), + )); + let mut fields = Vec::with_capacity(num_columns); + let mut columns = Vec::with_capacity(num_columns); + for idx in 0..num_columns { + fields.push(Field::new(format!("value{}", idx), DataType::Int32, false)); + columns.push(base_values.clone()); + } + let schema = Arc::new(Schema::new(fields)); + RecordBatch::try_new(schema, columns).unwrap() +} + +fn write_parquet_batch(batch: RecordBatch) -> Bytes { + let schema = batch.schema(); + let mut writer = ArrowWriter::try_new(Vec::new(), schema.clone(), None).unwrap(); + writer.write(&batch).unwrap(); + let buffer = writer.into_inner().unwrap(); + Bytes::from(buffer) +} + +#[derive(Clone)] +struct Scenario { + name: &'static str, + select_ratio: f64, + start_with_select: bool, + distribution: RunDistribution, +} + +#[derive(Clone)] +enum RunDistribution { + Constant, + Uniform { spread: f64 }, + Bimodal { long_factor: f64, long_prob: f64 }, +} + +fn generate_selectors( + avg_selector_len: usize, + total_rows: usize, + scenario: &Scenario, + seed: u64, +) -> Vec { + assert!( + (0.0..=1.0).contains(&scenario.select_ratio), + "select_ratio must be in [0, 1]" + ); + + let mut select_mean = scenario.select_ratio * 2.0 * avg_selector_len as f64; + let mut skip_mean = (1.0 - scenario.select_ratio) * 2.0 * avg_selector_len as f64; + + select_mean = select_mean.max(1.0); + skip_mean = skip_mean.max(1.0); + + let sum = select_mean + skip_mean; + // Rebalance the sampled select/skip run lengths so their sum matches the requested + // average selector length while respecting the configured selectivity ratio. + let scale = if sum == 0.0 { + 1.0 + } else { + (2.0 * avg_selector_len as f64) / sum + }; + select_mean *= scale; + skip_mean *= scale; + + let mut rng = StdRng::seed_from_u64(seed ^ (avg_selector_len as u64).wrapping_mul(0x9E3779B1)); + let mut selectors = Vec::with_capacity(total_rows / avg_selector_len.max(1)); + let mut remaining = total_rows; + let mut is_select = scenario.start_with_select; + + while remaining > 0 { + let mean = if is_select { select_mean } else { skip_mean }; + let len = sample_length(mean, &scenario.distribution, &mut rng).max(1); + let len = len.min(remaining); + selectors.push(if is_select { + RowSelector::select(len) + } else { + RowSelector::skip(len) + }); + remaining -= len; + if remaining == 0 { + break; + } + is_select = !is_select; + } + + let selection: RowSelection = selectors.into(); + selection.into() +} + +fn sample_length(mean: f64, distribution: &RunDistribution, rng: &mut StdRng) -> usize { + match distribution { + RunDistribution::Constant => mean.round().max(1.0) as usize, + RunDistribution::Uniform { spread } => { + let spread = spread.clamp(0.0, 0.99); + let lower = (mean * (1.0 - spread)).max(1.0); + let upper = (mean * (1.0 + spread)).max(lower + f64::EPSILON); + if (upper - lower) < 1.0 { + lower.round().max(1.0) as usize + } else { + let low = lower.floor() as usize; + let high = upper.ceil() as usize; + rng.random_range(low..=high).max(1) + } + } + RunDistribution::Bimodal { + long_factor, + long_prob, + } => { + let long_prob = long_prob.clamp(0.0, 0.5); + let short_prob = 1.0 - long_prob; + let short_factor = if short_prob == 0.0 { + 1.0 / long_factor.max(f64::EPSILON) + } else { + (1.0 - long_prob * long_factor).max(0.0) / short_prob + }; + let use_long = rng.random_bool(long_prob); + let factor = if use_long { + *long_factor + } else { + short_factor.max(0.1) + }; + (mean * factor).round().max(1.0) as usize + } + } +} + +struct SelectorStats { + average_selector_len: f64, + select_ratio: f64, +} + +impl SelectorStats { + fn new(selectors: &[RowSelector]) -> Self { + if selectors.is_empty() { + return Self { + average_selector_len: 0.0, + select_ratio: 0.0, + }; + } + + let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); + let selected_rows: usize = selectors + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + Self { + average_selector_len: total_rows as f64 / selectors.len() as f64, + select_ratio: if total_rows == 0 { + 0.0 + } else { + selected_rows as f64 / total_rows as f64 + }, + } + } +} diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 1cc7673a5776..191708d2e5ae 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -17,10 +17,10 @@ //! Contains reader which reads parquet data into arrow [`RecordBatch`] -use arrow_array::Array; use arrow_array::cast::AsArray; -use arrow_array::{RecordBatch, RecordBatchReader}; +use arrow_array::{Array, RecordBatch, RecordBatchReader}; use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; +use arrow_select::filter::filter_record_batch; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; @@ -43,7 +43,7 @@ use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; -pub use read_plan::{ReadPlan, ReadPlanBuilder}; +pub use read_plan::{ReadPlan, ReadPlanBuilder, RowSelectionStrategy}; mod filter; pub mod metrics; @@ -117,6 +117,8 @@ pub struct ArrowReaderBuilder { pub(crate) selection: Option, + pub(crate) selection_strategy: RowSelectionStrategy, + pub(crate) limit: Option, pub(crate) offset: Option, @@ -138,6 +140,7 @@ impl Debug for ArrowReaderBuilder { .field("projection", &self.projection) .field("filter", &self.filter) .field("selection", &self.selection) + .field("selection_strategy", &self.selection_strategy) .field("limit", &self.limit) .field("offset", &self.offset) .field("metrics", &self.metrics) @@ -157,6 +160,7 @@ impl ArrowReaderBuilder { projection: ProjectionMask::all(), filter: None, selection: None, + selection_strategy: RowSelectionStrategy::Auto, limit: None, offset: None, metrics: ArrowReaderMetrics::Disabled, @@ -271,6 +275,14 @@ impl ArrowReaderBuilder { } } + /// Override the strategy used to execute the configured [`RowSelection`] + pub fn with_row_selection_strategy(self, strategy: RowSelectionStrategy) -> Self { + Self { + selection_strategy: strategy, + ..self + } + } + /// Provide a [`RowFilter`] to skip decoding rows /// /// Row filters are applied after row group selection and row selection @@ -866,6 +878,7 @@ impl ParquetRecordBatchReaderBuilder { projection, mut filter, selection, + selection_strategy, limit, offset, metrics, @@ -886,7 +899,9 @@ impl ParquetRecordBatchReaderBuilder { row_groups, }; - let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); + let mut plan_builder = ReadPlanBuilder::new(batch_size) + .with_selection(selection) + .with_selection_strategy(selection_strategy); // Update selection based on any filters if let Some(filter) = filter.as_mut() { @@ -1034,9 +1049,82 @@ impl ParquetRecordBatchReader { let mut read_records = 0; let batch_size = self.batch_size(); match self.read_plan.selection_mut() { - Some(selection) => { - while read_records < batch_size && !selection.is_empty() { - let front = selection.pop_front().unwrap(); + Some(selection_cursor) => { + if selection_cursor.is_mask_backed() { + // Stream the record batch reader using contiguous segments of the selection + // mask, avoiding the need to materialize intermediate `RowSelector` ranges. + while !selection_cursor.is_empty() { + let Some(mask_chunk) = selection_cursor.next_mask_chunk(batch_size) else { + return Ok(None); + }; + + if mask_chunk.initial_skip > 0 { + let skipped = + self.array_reader.skip_records(mask_chunk.initial_skip)?; + if skipped != mask_chunk.initial_skip { + return Err(general_err!( + "failed to skip rows, expected {}, got {}", + mask_chunk.initial_skip, + skipped + )); + } + } + + if mask_chunk.chunk_rows == 0 { + if selection_cursor.is_empty() && mask_chunk.selected_rows == 0 { + return Ok(None); + } + continue; + } + + let mask = selection_cursor + .mask_values_for(&mask_chunk) + .ok_or_else(|| general_err!("row selection mask out of bounds"))?; + + let read = self.array_reader.read_records(mask_chunk.chunk_rows)?; + if read == 0 { + return Err(general_err!( + "reached end of column while expecting {} rows", + mask_chunk.chunk_rows + )); + } + if read != mask_chunk.chunk_rows { + return Err(general_err!( + "insufficient rows read from array reader - expected {}, got {}", + mask_chunk.chunk_rows, + read + )); + } + + let array = self.array_reader.consume_batch()?; + // The column reader exposes the projection as a struct array; convert this + // into a record batch before applying the boolean filter mask. + let struct_array = array.as_struct_opt().ok_or_else(|| { + ArrowError::ParquetError( + "Struct array reader should return struct array".to_string(), + ) + })?; + + let filtered_batch = + filter_record_batch(&RecordBatch::from(struct_array), &mask)?; + + if filtered_batch.num_rows() != mask_chunk.selected_rows { + return Err(general_err!( + "filtered rows mismatch selection - expected {}, got {}", + mask_chunk.selected_rows, + filtered_batch.num_rows() + )); + } + + if filtered_batch.num_rows() == 0 { + continue; + } + + return Ok(Some(filtered_batch)); + } + } + while read_records < batch_size && !selection_cursor.is_empty() { + let front = selection_cursor.next_selector().unwrap(); if front.skip { let skipped = self.array_reader.skip_records(front.row_count)?; @@ -1062,7 +1150,7 @@ impl ParquetRecordBatchReader { Some(remaining) if remaining != 0 => { // if page row count less than batch_size we must set batch size to page row count. // add check avoid dead loop - selection.push_front(RowSelector::select(remaining)); + selection_cursor.return_selector(RowSelector::select(remaining)); need_read } _ => front.row_count, @@ -4588,6 +4676,93 @@ mod tests { assert_eq!(out, batch.slice(2, 1)); } + #[test] + fn test_row_selection_interleaved_skip() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "v", + ArrowDataType::Int32, + false, + )])); + + let values = Int32Array::from(vec![0, 1, 2, 3, 4]); + let batch = RecordBatch::try_from_iter([("v", Arc::new(values) as ArrayRef)]).unwrap(); + + let mut buffer = Vec::with_capacity(1024); + let mut writer = ArrowWriter::try_new(&mut buffer, schema.clone(), None).unwrap(); + writer.write(&batch)?; + writer.close()?; + + let selection = RowSelection::from(vec![ + RowSelector::select(1), + RowSelector::skip(2), + RowSelector::select(2), + ]); + + let mut reader = ParquetRecordBatchReaderBuilder::try_new(Bytes::from(buffer))? + .with_batch_size(4) + .with_row_selection(selection) + .build()?; + + let out = reader.next().unwrap()?; + assert_eq!(out.num_rows(), 3); + let values = out + .column(0) + .as_primitive::() + .values(); + assert_eq!(values, &[0, 3, 4]); + assert!(reader.next().is_none()); + Ok(()) + } + + #[test] + fn test_row_selection_mask_sparse_rows() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "v", + ArrowDataType::Int32, + false, + )])); + + let values = Int32Array::from((0..30).collect::>()); + let batch = RecordBatch::try_from_iter([("v", Arc::new(values) as ArrayRef)])?; + + let mut buffer = Vec::with_capacity(1024); + let mut writer = ArrowWriter::try_new(&mut buffer, schema.clone(), None)?; + writer.write(&batch)?; + writer.close()?; + + let total_rows = batch.num_rows(); + let ranges = (1..total_rows) + .step_by(2) + .map(|i| i..i + 1) + .collect::>(); + let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), total_rows); + + let selectors: Vec = selection.clone().into(); + assert!(total_rows < selectors.len() * 8); + + let bytes = Bytes::from(buffer); + + let reader = ParquetRecordBatchReaderBuilder::try_new(bytes.clone())? + .with_batch_size(7) + .with_row_selection(selection) + .build()?; + + let mut collected = Vec::new(); + for batch in reader { + let batch = batch?; + collected.extend_from_slice( + batch + .column(0) + .as_primitive::() + .values(), + ); + } + + let expected: Vec = (1..total_rows).step_by(2).map(|i| i as i32).collect(); + assert_eq!(collected, expected); + Ok(()) + } + fn test_decimal32_roundtrip() { let d = |values: Vec, p: u8| { let iter = values.into_iter(); diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 2210f47df2c1..8cc1927357e1 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -23,16 +23,32 @@ use crate::arrow::arrow_reader::{ ArrowPredicate, ParquetRecordBatchReader, RowSelection, RowSelector, }; use crate::errors::{ParquetError, Result}; -use arrow_array::Array; +use arrow_array::{Array, BooleanArray}; +use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; use arrow_select::filter::prep_null_mask_filter; use std::collections::VecDeque; +/// Strategy for materialising [`RowSelection`] during execution. +#[derive(Clone, Copy, Debug, Default, Eq, PartialEq)] +pub enum RowSelectionStrategy { + /// Automatically choose between mask- and selector-backed execution + /// based on heuristics + #[default] + Auto, + /// Always use a boolean mask to materialise the selection + Mask, + /// Always use a queue of [`RowSelector`] values + Selectors, +} + /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] pub struct ReadPlanBuilder { batch_size: usize, /// Current to apply, includes all filters selection: Option, + /// Strategy to use when materialising the row selection + selection_strategy: RowSelectionStrategy, } impl ReadPlanBuilder { @@ -41,6 +57,7 @@ impl ReadPlanBuilder { Self { batch_size, selection: None, + selection_strategy: RowSelectionStrategy::Auto, } } @@ -50,6 +67,12 @@ impl ReadPlanBuilder { self } + /// Force a specific strategy when materialising the [`RowSelection`] + pub fn with_selection_strategy(mut self, strategy: RowSelectionStrategy) -> Self { + self.selection_strategy = strategy; + self + } + /// Returns the current selection, if any pub fn selection(&self) -> Option<&RowSelection> { self.selection.as_ref() @@ -129,9 +152,14 @@ impl ReadPlanBuilder { let Self { batch_size, selection, + selection_strategy, } = self; - let selection = selection.map(|s| s.trim().into()); + let selection = selection.map(|s| { + let trimmed = s.trim(); + let selectors: Vec = trimmed.into(); + RowSelectionCursor::new(selectors, selection_strategy) + }); ReadPlan { batch_size, @@ -233,12 +261,12 @@ pub struct ReadPlan { /// The number of rows to read in each batch batch_size: usize, /// Row ranges to be selected from the data source - selection: Option>, + selection: Option, } impl ReadPlan { /// Returns a mutable reference to the selection, if any - pub fn selection_mut(&mut self) -> Option<&mut VecDeque> { + pub fn selection_mut(&mut self) -> Option<&mut RowSelectionCursor> { self.selection.as_mut() } @@ -248,3 +276,195 @@ impl ReadPlan { self.batch_size } } + +/// Cursor for iterating a [`RowSelection`] during execution within a [`ReadPlan`]. +/// +/// This keeps per-reader state such as the current position and delegates the +/// actual storage strategy to [`RowSelectionBacking`]. +#[derive(Debug)] +pub struct RowSelectionCursor { + /// Backing storage describing how the selection is materialised + storage: RowSelectionBacking, + /// Current absolute offset into the selection + position: usize, +} + +/// Backing storage that powers [`RowSelectionCursor`]. +/// +/// The cursor either walks a boolean mask (dense representation) or a queue +/// of [`RowSelector`] ranges (sparse representation). +#[derive(Debug)] +enum RowSelectionBacking { + Mask(BooleanBuffer), + Selectors(VecDeque), +} + +/// Result of computing the next chunk to read when using a bitmap mask +pub struct MaskChunk { + /// Number of leading rows to skip before reaching selected rows + pub initial_skip: usize, + /// Total rows covered by this chunk (selected + skipped) + pub chunk_rows: usize, + /// Rows actually selected within the chunk + pub selected_rows: usize, + /// Starting offset within the mask where the chunk begins + pub mask_start: usize, +} + +impl RowSelectionCursor { + /// Create a cursor, choosing an efficient backing representation + fn new(selectors: Vec, strategy: RowSelectionStrategy) -> Self { + if matches!(strategy, RowSelectionStrategy::Selectors) { + return Self { + storage: RowSelectionBacking::Selectors(selectors.into()), + position: 0, + }; + } + + let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); + let selector_count = selectors.len(); + const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 16; + // Prefer a bitmap mask when the selectors are short on average, as the mask + // (re)construction cost is amortized by a simpler execution path during reads. + let use_mask = match strategy { + RowSelectionStrategy::Mask => true, + RowSelectionStrategy::Auto => { + selector_count == 0 + || total_rows < selector_count.saturating_mul(AVG_SELECTOR_LEN_MASK_THRESHOLD) + } + RowSelectionStrategy::Selectors => unreachable!(), + }; + + let storage = if use_mask { + RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) + } else { + RowSelectionBacking::Selectors(selectors.into()) + }; + + Self { + storage, + position: 0, + } + } + + /// Returns `true` when no further rows remain + pub fn is_empty(&self) -> bool { + match &self.storage { + RowSelectionBacking::Mask(mask) => self.position >= mask.len(), + RowSelectionBacking::Selectors(selectors) => selectors.is_empty(), + } + } + + /// Current position within the overall selection + pub fn position(&self) -> usize { + self.position + } + + /// Return the next [`RowSelector`] when using the sparse representation + pub fn next_selector(&mut self) -> Option { + match &mut self.storage { + RowSelectionBacking::Selectors(selectors) => { + let selector = selectors.pop_front()?; + self.position += selector.row_count; + Some(selector) + } + RowSelectionBacking::Mask(_) => { + unreachable!("next_selector called for mask-based RowSelectionCursor") + } + } + } + + /// Return a selector to the front, rewinding the position (sparse-only) + pub fn return_selector(&mut self, selector: RowSelector) { + match &mut self.storage { + RowSelectionBacking::Selectors(selectors) => { + self.position = self.position.saturating_sub(selector.row_count); + selectors.push_front(selector); + } + RowSelectionBacking::Mask(_) => { + unreachable!("return_selector called for mask-based RowSelectionCursor") + } + } + } + + /// Returns `true` if the cursor is backed by a boolean mask + pub fn is_mask_backed(&self) -> bool { + matches!(self.storage, RowSelectionBacking::Mask(_)) + } + + /// Advance through the mask representation, producing the next chunk summary + pub fn next_mask_chunk(&mut self, batch_size: usize) -> Option { + if !self.is_mask_backed() { + unreachable!("next_mask_chunk called for selector-based RowSelectionCursor") + } + let (initial_skip, chunk_rows, selected_rows, mask_start, end_position) = { + let mask = match &self.storage { + RowSelectionBacking::Mask(mask) => mask, + RowSelectionBacking::Selectors(_) => return None, + }; + + if self.position >= mask.len() { + return None; + } + + let start_position = self.position; + let mut cursor = start_position; + let mut initial_skip = 0; + + while cursor < mask.len() && !mask.value(cursor) { + initial_skip += 1; + cursor += 1; + } + + let mask_start = cursor; + let mut chunk_rows = 0; + let mut selected_rows = 0; + + // Advance until enough rows have been selected to satisfy the batch size, + // or until the mask is exhausted. This mirrors the behaviour of the legacy + // `RowSelector` queue-based iteration. + while cursor < mask.len() && selected_rows < batch_size { + chunk_rows += 1; + if mask.value(cursor) { + selected_rows += 1; + } + cursor += 1; + } + + (initial_skip, chunk_rows, selected_rows, mask_start, cursor) + }; + + self.position = end_position; + + Some(MaskChunk { + initial_skip, + chunk_rows, + selected_rows, + mask_start, + }) + } + + /// Materialise the boolean values for a mask-backed chunk + pub fn mask_values_for(&self, chunk: &MaskChunk) -> Option { + match &self.storage { + RowSelectionBacking::Mask(mask) => { + if chunk.mask_start.saturating_add(chunk.chunk_rows) > mask.len() { + return None; + } + Some(BooleanArray::from( + mask.slice(chunk.mask_start, chunk.chunk_rows), + )) + } + RowSelectionBacking::Selectors(_) => None, + } + } +} + +fn boolean_mask_from_selectors(selectors: &[RowSelector]) -> BooleanBuffer { + let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); + let mut builder = BooleanBufferBuilder::new(total_rows); + for selector in selectors { + builder.append_n(selector.row_count, !selector.skip); + } + builder.finish() +} diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index c5badea7f32c..c8cd924fb802 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -40,7 +40,7 @@ use arrow_schema::{DataType, Fields, Schema, SchemaRef}; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, - RowFilter, RowSelection, + RowFilter, RowSelection, RowSelectionStrategy, }; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; @@ -508,6 +508,7 @@ impl ParquetRecordBatchStreamBuilder { filter: self.filter, metadata: self.metadata.clone(), fields: self.fields, + selection_strategy: self.selection_strategy, limit: self.limit, offset: self.offset, metrics: self.metrics, @@ -557,7 +558,10 @@ struct ReaderFactory { /// Optional filter filter: Option, - /// Limit to apply to remaining row groups. + /// Strategy used to materialise row selections for this reader + selection_strategy: RowSelectionStrategy, + + /// Limit to apply to remaining row groups. limit: Option, /// Offset to apply to the next @@ -620,7 +624,9 @@ where let cache_options_builder = CacheOptionsBuilder::new(&cache_projection, &row_group_cache); let filter = self.filter.as_mut(); - let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); + let mut plan_builder = ReadPlanBuilder::new(batch_size) + .with_selection(selection) + .with_selection_strategy(self.selection_strategy); // Update selection based on any filters if let Some(filter) = filter { @@ -1774,6 +1780,7 @@ mod tests { fields: fields.map(Arc::new), input: async_reader, filter: None, + selection_strategy: RowSelectionStrategy::Auto, limit: None, offset: None, metrics: ArrowReaderMetrics::disabled(), @@ -2181,6 +2188,78 @@ mod tests { assert_eq!(requests.lock().unwrap().len(), 3); } + #[tokio::test] + async fn test_cache_projection_excludes_nested_columns() { + use arrow_array::{ArrayRef, StringArray}; + + // Build a simple RecordBatch with a primitive column `a` and a nested struct column `b { aa, bb }` + let a = StringArray::from_iter_values(["r1", "r2"]); + let b = StructArray::from(vec![ + ( + Arc::new(Field::new("aa", DataType::Utf8, true)), + Arc::new(StringArray::from_iter_values(["v1", "v2"])) as ArrayRef, + ), + ( + Arc::new(Field::new("bb", DataType::Utf8, true)), + Arc::new(StringArray::from_iter_values(["w1", "w2"])) as ArrayRef, + ), + ]); + + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, true), + Field::new("b", b.data_type().clone(), true), + ])); + + let mut buf = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, None).unwrap(); + let batch = RecordBatch::try_from_iter([ + ("a", Arc::new(a) as ArrayRef), + ("b", Arc::new(b) as ArrayRef), + ]) + .unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + // Load Parquet metadata + let data: Bytes = buf.into(); + let metadata = ParquetMetaDataReader::new() + .parse_and_finish(&data) + .unwrap(); + let metadata = Arc::new(metadata); + + // Build a RowFilter whose predicate projects a leaf under the nested root `b` + // Leaf indices are depth-first; with schema [a, b.aa, b.bb] we pick index 1 (b.aa) + let parquet_schema = metadata.file_metadata().schema_descr(); + let nested_leaf_mask = ProjectionMask::leaves(parquet_schema, vec![1]); + + let always_true = ArrowPredicateFn::new(nested_leaf_mask.clone(), |batch: RecordBatch| { + Ok(arrow_array::BooleanArray::from(vec![ + true; + batch.num_rows() + ])) + }); + let filter = RowFilter::new(vec![Box::new(always_true)]); + + // Construct a ReaderFactory and compute cache projection + let reader_factory = ReaderFactory { + metadata: Arc::clone(&metadata), + fields: None, + input: TestReader::new(data), + filter: Some(filter), + selection_strategy: RowSelectionStrategy::Auto, + limit: None, + offset: None, + metrics: ArrowReaderMetrics::disabled(), + max_predicate_cache_size: 0, + }; + + // Provide an output projection that also selects the same nested leaf + let cache_projection = reader_factory.compute_cache_projection(&nested_leaf_mask); + + // Expect None since nested columns should be excluded from cache projection + assert!(cache_projection.is_none()); + } + #[tokio::test] #[allow(deprecated)] async fn empty_offset_index_doesnt_panic_in_read_row_group() { diff --git a/parquet/src/arrow/push_decoder/mod.rs b/parquet/src/arrow/push_decoder/mod.rs index 4b932fb08034..3da7c4bd2fbc 100644 --- a/parquet/src/arrow/push_decoder/mod.rs +++ b/parquet/src/arrow/push_decoder/mod.rs @@ -168,6 +168,7 @@ impl ParquetPushDecoderBuilder { projection, filter, selection, + selection_strategy: _, limit, offset, metrics, diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index be9070ae8b49..69b32d181d3c 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -654,6 +654,6 @@ mod tests { #[test] // Verify that the size of RowGroupDecoderState does not grow too large fn test_structure_size() { - assert_eq!(std::mem::size_of::(), 184); + assert_eq!(std::mem::size_of::(), 192); } } From 5e81ee41c0f4fc85e2cba4ffc55f4b2e5676d4aa Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Thu, 6 Nov 2025 02:15:02 +0800 Subject: [PATCH 02/27] Try to fix the page skip issue --- parquet/src/arrow/arrow_reader/mod.rs | 205 +++++++++++++++++++++++--- parquet/src/arrow/async_reader/mod.rs | 39 ++++- 2 files changed, 219 insertions(+), 25 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 191708d2e5ae..aec87199511c 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -39,6 +39,7 @@ use crate::column::page::{PageIterator, PageReader}; use crate::encryption::decrypt::FileDecryptionProperties; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; +use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; @@ -51,6 +52,61 @@ mod read_plan; mod selection; pub mod statistics; +/// Returns true when `selection` keeps some rows for `projection` but prunes whole +/// data pages (determined via `OffsetIndex`). Masks can't handle this because the +/// page data is never fetched, so the caller should fall back to RowSelectors. +pub(crate) fn selection_skips_any_page( + selection: &RowSelection, + projection: &ProjectionMask, + columns: &[OffsetIndexMetaData], +) -> bool { + columns.iter().enumerate().any(|(leaf_idx, column)| { + if !projection.leaf_included(leaf_idx) { + return false; + } + + let locations = column.page_locations(); + if locations.is_empty() { + return false; + } + + let ranges = selection.scan_ranges(locations); + !ranges.is_empty() && ranges.len() < locations.len() + }) +} + +fn selection_requires_selectors_for_row_groups( + selection: Option<&RowSelection>, + projection: &ProjectionMask, + metadata: &ParquetMetaData, + row_groups: &[usize], +) -> bool { + let mut remaining = match selection { + Some(selection) => selection.clone(), + None => return false, + }; + + let offset_index = match metadata.offset_index() { + Some(index) => index, + None => return false, + }; + + for &rg_idx in row_groups { + let columns = match offset_index.get(rg_idx) { + Some(columns) if !columns.is_empty() => columns, + _ => continue, + }; + + let row_count = metadata.row_group(rg_idx).num_rows() as usize; + let rg_selection = remaining.split_off(row_count); + if selection_skips_any_page(&rg_selection, projection, columns) { + return true; + } + } + + false +} + /// Builder for constructing Parquet readers that decode into [Apache Arrow] /// arrays. /// @@ -918,18 +974,43 @@ impl ParquetRecordBatchReaderBuilder { .build_array_reader(fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; + + // Once a predicate has refined the selection, re-check if any requested + // column now skips entire pages. In that case the boolean-mask strategy + // would try to filter data we never fetched; selectors are safe. + if selection_requires_selectors_for_row_groups( + plan_builder.selection(), + predicate.projection(), + reader.metadata.as_ref(), + &reader.row_groups, + ) { + plan_builder = + plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); + } } } let array_reader = ArrayReaderBuilder::new(&reader, &metrics) .build_array_reader(fields.as_deref(), &projection)?; - let read_plan = plan_builder + let mut plan_builder = plan_builder .limited(reader.num_rows()) .with_offset(offset) .with_limit(limit) - .build_limited() - .build(); + .build_limited(); + + // Offset/limit can also trim per-row-group selections. Ensure the final + // plan doesn't leave mask-backed cursors pointing at pruned pages. + if selection_requires_selectors_for_row_groups( + plan_builder.selection(), + &projection, + reader.metadata.as_ref(), + &reader.row_groups, + ) { + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); + } + + let read_plan = plan_builder.build(); Ok(ParquetRecordBatchReader::new(array_reader, read_plan)) } @@ -1257,6 +1338,28 @@ mod tests { use std::path::PathBuf; use std::sync::Arc; + use crate::arrow::arrow_reader::{ + ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader, + ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelectionStrategy, + RowSelector, + }; + use crate::arrow::schema::add_encoded_arrow_schema_to_metadata; + use crate::arrow::{ArrowWriter, ProjectionMask}; + use crate::basic::{Compression, ConvertedType, Encoding, Repetition, Type as PhysicalType}; + use crate::column::reader::decoder::REPETITION_LEVELS_BATCH_SIZE; + use crate::data_type::{ + BoolType, ByteArray, ByteArrayType, DataType, FixedLenByteArray, FixedLenByteArrayType, + FloatType, Int32Type, Int64Type, Int96, Int96Type, + }; + use crate::errors::Result; + use crate::file::metadata::ParquetMetaData; + use crate::file::properties::{EnabledStatistics, WriterProperties, WriterVersion}; + use crate::file::writer::SerializedFileWriter; + use crate::schema::parser::parse_message_type; + use crate::schema::types::{Type, TypePtr}; + use crate::util::test_common::rand_gen::RandGen; + use arrow::compute::kernels::cmp::eq; + use arrow::compute::or; use arrow_array::builder::*; use arrow_array::cast::AsArray; use arrow_array::types::{ @@ -1277,26 +1380,6 @@ mod tests { use rand::{Rng, RngCore, rng}; use tempfile::tempfile; - use crate::arrow::arrow_reader::{ - ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader, - ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, - }; - use crate::arrow::schema::add_encoded_arrow_schema_to_metadata; - use crate::arrow::{ArrowWriter, ProjectionMask}; - use crate::basic::{ConvertedType, Encoding, Repetition, Type as PhysicalType}; - use crate::column::reader::decoder::REPETITION_LEVELS_BATCH_SIZE; - use crate::data_type::{ - BoolType, ByteArray, ByteArrayType, DataType, FixedLenByteArray, FixedLenByteArrayType, - FloatType, Int32Type, Int64Type, Int96, Int96Type, - }; - use crate::errors::Result; - use crate::file::metadata::ParquetMetaData; - use crate::file::properties::{EnabledStatistics, WriterProperties, WriterVersion}; - use crate::file::writer::SerializedFileWriter; - use crate::schema::parser::parse_message_type; - use crate::schema::types::{Type, TypePtr}; - use crate::util::test_common::rand_gen::RandGen; - #[test] fn test_arrow_reader_all_columns() { let file = get_test_file("parquet/generated_simple_numerics/blogs.parquet"); @@ -5145,6 +5228,82 @@ mod tests { c0.iter().zip(c1.iter()).for_each(|(l, r)| assert_eq!(l, r)); } + const SECOND_MATCH_INDEX: usize = 4096; + const SECOND_MATCH_VALUE: i64 = 12345; + + fn build_mask_pruning_parquet() -> Bytes { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", ArrowDataType::Int64, false), + Field::new("value", ArrowDataType::Float64, false), + ])); + + let num_rows = 8192usize; + let mut int_values: Vec = (0..num_rows as i64).collect(); + int_values[0] = 9999; + int_values[SECOND_MATCH_INDEX] = SECOND_MATCH_VALUE; + let keys = Int64Array::from(int_values); + let values = Float64Array::from_iter_values((0..num_rows).map(|v| v as f64 * 1.5)); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], + ) + .unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_data_page_size_limit(1024) + .set_data_page_row_count_limit(32) + .build(); + + let mut buffer = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + Bytes::from(buffer) + } + + #[test] + fn test_mask_strategy_full_page_skip_triggers_error() { + let data = build_mask_pruning_parquet(); + + let filter_mask; + let output_mask; + { + let options = ArrowReaderOptions::new().with_page_index(true); + let builder = + ParquetRecordBatchReaderBuilder::try_new_with_options(data.clone(), options) + .unwrap(); + let schema = builder.parquet_schema().clone(); + filter_mask = ProjectionMask::leaves(&schema, [0]); + output_mask = ProjectionMask::leaves(&schema, [1]); + } + + let options = ArrowReaderOptions::new().with_page_index(true); + let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { + let column = batch.column(0); + let match_first = eq(column, &Int64Array::new_scalar(9999))?; + let match_second = eq(column, &Int64Array::new_scalar(SECOND_MATCH_VALUE))?; + or(&match_first, &match_second) + }); + + let mut reader = ParquetRecordBatchReaderBuilder::try_new_with_options(data, options) + .unwrap() + .with_projection(output_mask) + .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) + .with_batch_size(256) + .with_row_selection_strategy(RowSelectionStrategy::Mask) + .build() + .unwrap(); + + // The mask strategy used to panic once predicate pruning removed whole pages. + // Collecting into batches validates the plan now downgrades to selectors instead. + let schema = reader.schema().clone(); + let batches = reader.collect::, _>>().unwrap(); + let result = concat_batches(&schema, &batches).unwrap(); + assert_eq!(result.num_rows(), 2); + } + #[test] fn test_get_row_group_column_bloom_filter_with_length() { // convert to new parquet file with bloom_filter_length diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index c8cd924fb802..2ad419285307 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -40,7 +40,7 @@ use arrow_schema::{DataType, Fields, Schema, SchemaRef}; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, - RowFilter, RowSelection, RowSelectionStrategy, + RowFilter, RowSelection, RowSelectionStrategy, selection_skips_any_page, }; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; @@ -49,6 +49,7 @@ use crate::bloom_filter::{ }; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; +use crate::file::page_index::offset_index::OffsetIndexMetaData; mod metadata; pub use metadata::*; @@ -637,6 +638,17 @@ where return Ok((self, None)); // ruled out entire row group } + // Predicate evaluation can zero out some pages; switch to selectors + // before the mask-backed cursor tries to touch data we won't fetch. + if selection_requires_selector( + plan_builder.selection(), + predicate.projection(), + offset_index, + ) { + plan_builder = + plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); + } + // (pre) Fetch only the columns that are selected by the predicate let selection = plan_builder.selection(); // Fetch predicate columns; expand selection only for cached predicate columns @@ -669,12 +681,17 @@ where } // Apply any limit and offset - let plan_builder = plan_builder + let mut plan_builder = plan_builder .limited(row_group.row_count) .with_offset(self.offset) .with_limit(self.limit) .build_limited(); + // Offset/limit may further trim pages, so re-evaluate the strategy here too. + if selection_requires_selector(plan_builder.selection(), &projection, offset_index) { + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); + } + let rows_after = plan_builder .num_rows_selected() .unwrap_or(row_group.row_count); @@ -765,6 +782,24 @@ where } } +fn selection_requires_selector( + selection: Option<&RowSelection>, + projection: &ProjectionMask, + offset_index: Option<&[OffsetIndexMetaData]>, +) -> bool { + let selection = match selection { + Some(selection) => selection, + None => return false, + }; + + let offset_index = match offset_index { + Some(index) => index, + None => return false, + }; + + selection_skips_any_page(selection, projection, offset_index) +} + enum StreamState { /// At the start of a new row group, or the end of the parquet stream Init, From 3abea7a81904998928a233d492101425ebc47436 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Thu, 6 Nov 2025 18:51:54 +0800 Subject: [PATCH 03/27] Add strategy selection in builder --- parquet/src/arrow/arrow_reader/mod.rs | 86 +-------------------- parquet/src/arrow/arrow_reader/read_plan.rs | 19 ++++- parquet/src/arrow/async_reader/mod.rs | 70 ++++++++++------- 3 files changed, 64 insertions(+), 111 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index aec87199511c..bea8b3c8e1da 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -52,61 +52,6 @@ mod read_plan; mod selection; pub mod statistics; -/// Returns true when `selection` keeps some rows for `projection` but prunes whole -/// data pages (determined via `OffsetIndex`). Masks can't handle this because the -/// page data is never fetched, so the caller should fall back to RowSelectors. -pub(crate) fn selection_skips_any_page( - selection: &RowSelection, - projection: &ProjectionMask, - columns: &[OffsetIndexMetaData], -) -> bool { - columns.iter().enumerate().any(|(leaf_idx, column)| { - if !projection.leaf_included(leaf_idx) { - return false; - } - - let locations = column.page_locations(); - if locations.is_empty() { - return false; - } - - let ranges = selection.scan_ranges(locations); - !ranges.is_empty() && ranges.len() < locations.len() - }) -} - -fn selection_requires_selectors_for_row_groups( - selection: Option<&RowSelection>, - projection: &ProjectionMask, - metadata: &ParquetMetaData, - row_groups: &[usize], -) -> bool { - let mut remaining = match selection { - Some(selection) => selection.clone(), - None => return false, - }; - - let offset_index = match metadata.offset_index() { - Some(index) => index, - None => return false, - }; - - for &rg_idx in row_groups { - let columns = match offset_index.get(rg_idx) { - Some(columns) if !columns.is_empty() => columns, - _ => continue, - }; - - let row_count = metadata.row_group(rg_idx).num_rows() as usize; - let rg_selection = remaining.split_off(row_count); - if selection_skips_any_page(&rg_selection, projection, columns) { - return true; - } - } - - false -} - /// Builder for constructing Parquet readers that decode into [Apache Arrow] /// arrays. /// @@ -974,43 +919,18 @@ impl ParquetRecordBatchReaderBuilder { .build_array_reader(fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; - - // Once a predicate has refined the selection, re-check if any requested - // column now skips entire pages. In that case the boolean-mask strategy - // would try to filter data we never fetched; selectors are safe. - if selection_requires_selectors_for_row_groups( - plan_builder.selection(), - predicate.projection(), - reader.metadata.as_ref(), - &reader.row_groups, - ) { - plan_builder = - plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); - } } } let array_reader = ArrayReaderBuilder::new(&reader, &metrics) .build_array_reader(fields.as_deref(), &projection)?; - let mut plan_builder = plan_builder + let read_plan = plan_builder .limited(reader.num_rows()) .with_offset(offset) .with_limit(limit) - .build_limited(); - - // Offset/limit can also trim per-row-group selections. Ensure the final - // plan doesn't leave mask-backed cursors pointing at pruned pages. - if selection_requires_selectors_for_row_groups( - plan_builder.selection(), - &projection, - reader.metadata.as_ref(), - &reader.row_groups, - ) { - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); - } - - let read_plan = plan_builder.build(); + .build_limited() + .build(); Ok(ParquetRecordBatchReader::new(array_reader, read_plan)) } diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 8cc1927357e1..1211b7ea12ca 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -49,6 +49,8 @@ pub struct ReadPlanBuilder { selection: Option, /// Strategy to use when materialising the row selection selection_strategy: RowSelectionStrategy, + /// Tracks whether selectors were required due to skipped pages + force_selectors: bool, } impl ReadPlanBuilder { @@ -58,6 +60,7 @@ impl ReadPlanBuilder { batch_size, selection: None, selection_strategy: RowSelectionStrategy::Auto, + force_selectors: false, } } @@ -73,6 +76,12 @@ impl ReadPlanBuilder { self } + /// Remember that selectors are required whenever we detect skipped pages. + pub fn force_selectors_on_skip(mut self, has_skipped_page: bool) -> Self { + self.force_selectors |= has_skipped_page; + self + } + /// Returns the current selection, if any pub fn selection(&self) -> Option<&RowSelection> { self.selection.as_ref() @@ -149,10 +158,18 @@ impl ReadPlanBuilder { if !self.selects_any() { self.selection = Some(RowSelection::from(vec![])); } + let selection_strategy = if self.force_selectors + || matches!(self.selection_strategy, RowSelectionStrategy::Selectors) + { + RowSelectionStrategy::Selectors + } else { + self.selection_strategy + }; let Self { batch_size, selection, - selection_strategy, + selection_strategy: _, + force_selectors: _, } = self; let selection = selection.map(|s| { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 2ad419285307..75f783b9c49d 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -40,7 +40,7 @@ use arrow_schema::{DataType, Fields, Schema, SchemaRef}; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, - RowFilter, RowSelection, RowSelectionStrategy, selection_skips_any_page, + RowFilter, RowSelection, RowSelectionStrategy, }; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; @@ -66,6 +66,42 @@ use crate::arrow::schema::ParquetField; #[cfg(feature = "object_store")] pub use store::*; +/// Returns true when `selection` keeps some rows for `projection` but prunes whole +/// data pages (determined via `OffsetIndex`). Masks can't handle this because the +/// page data is never fetched, so the caller should fall back to RowSelectors. +pub(crate) fn selection_skips_any_page( + selection: &RowSelection, + projection: &ProjectionMask, + columns: &[OffsetIndexMetaData], +) -> bool { + columns.iter().enumerate().any(|(leaf_idx, column)| { + if !projection.leaf_included(leaf_idx) { + return false; + } + + let locations = column.page_locations(); + if locations.is_empty() { + return false; + } + + let ranges = selection.scan_ranges(locations); + !ranges.is_empty() && ranges.len() < locations.len() + }) +} + +fn selection_has_skipped_page( + selection: Option<&RowSelection>, + projection: &ProjectionMask, + offset_index: Option<&[OffsetIndexMetaData]>, +) -> bool { + match (selection, offset_index) { + (Some(selection), Some(offset_index)) => { + selection_skips_any_page(selection, projection, offset_index) + } + _ => false, + } +} + /// The asynchronous interface used by [`ParquetRecordBatchStream`] to read parquet files /// /// Notes: @@ -640,14 +676,12 @@ where // Predicate evaluation can zero out some pages; switch to selectors // before the mask-backed cursor tries to touch data we won't fetch. - if selection_requires_selector( + let has_skipped_page = selection_has_skipped_page( plan_builder.selection(), predicate.projection(), offset_index, - ) { - plan_builder = - plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); - } + ); + plan_builder = plan_builder.force_selectors_on_skip(has_skipped_page); // (pre) Fetch only the columns that are selected by the predicate let selection = plan_builder.selection(); @@ -688,9 +722,9 @@ where .build_limited(); // Offset/limit may further trim pages, so re-evaluate the strategy here too. - if selection_requires_selector(plan_builder.selection(), &projection, offset_index) { - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); - } + let has_skipped_page = + selection_has_skipped_page(plan_builder.selection(), &projection, offset_index); + plan_builder = plan_builder.force_selectors_on_skip(has_skipped_page); let rows_after = plan_builder .num_rows_selected() @@ -782,24 +816,6 @@ where } } -fn selection_requires_selector( - selection: Option<&RowSelection>, - projection: &ProjectionMask, - offset_index: Option<&[OffsetIndexMetaData]>, -) -> bool { - let selection = match selection { - Some(selection) => selection, - None => return false, - }; - - let offset_index = match offset_index { - Some(index) => index, - None => return false, - }; - - selection_skips_any_page(selection, projection, offset_index) -} - enum StreamState { /// At the start of a new row group, or the end of the parquet stream Init, From 35a2a59aade5c8ad16f70dcf96b469aae6e9844b Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Thu, 6 Nov 2025 19:13:25 +0800 Subject: [PATCH 04/27] Update ReadPlanBuilder --- parquet/src/arrow/arrow_reader/mod.rs | 1 - parquet/src/arrow/arrow_reader/read_plan.rs | 18 +----------------- parquet/src/arrow/async_reader/mod.rs | 14 ++++++++------ 3 files changed, 9 insertions(+), 24 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index bea8b3c8e1da..bdbacba628a3 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -39,7 +39,6 @@ use crate::column::page::{PageIterator, PageReader}; use crate::encryption::decrypt::FileDecryptionProperties; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; -use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 1211b7ea12ca..58603843df64 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -49,8 +49,6 @@ pub struct ReadPlanBuilder { selection: Option, /// Strategy to use when materialising the row selection selection_strategy: RowSelectionStrategy, - /// Tracks whether selectors were required due to skipped pages - force_selectors: bool, } impl ReadPlanBuilder { @@ -60,7 +58,6 @@ impl ReadPlanBuilder { batch_size, selection: None, selection_strategy: RowSelectionStrategy::Auto, - force_selectors: false, } } @@ -76,12 +73,6 @@ impl ReadPlanBuilder { self } - /// Remember that selectors are required whenever we detect skipped pages. - pub fn force_selectors_on_skip(mut self, has_skipped_page: bool) -> Self { - self.force_selectors |= has_skipped_page; - self - } - /// Returns the current selection, if any pub fn selection(&self) -> Option<&RowSelection> { self.selection.as_ref() @@ -158,18 +149,11 @@ impl ReadPlanBuilder { if !self.selects_any() { self.selection = Some(RowSelection::from(vec![])); } - let selection_strategy = if self.force_selectors - || matches!(self.selection_strategy, RowSelectionStrategy::Selectors) - { - RowSelectionStrategy::Selectors - } else { - self.selection_strategy - }; + let selection_strategy = self.selection_strategy; let Self { batch_size, selection, selection_strategy: _, - force_selectors: _, } = self; let selection = selection.map(|s| { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 75f783b9c49d..f3f3e4dea3c9 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -676,12 +676,14 @@ where // Predicate evaluation can zero out some pages; switch to selectors // before the mask-backed cursor tries to touch data we won't fetch. - let has_skipped_page = selection_has_skipped_page( + if selection_has_skipped_page( plan_builder.selection(), predicate.projection(), offset_index, - ); - plan_builder = plan_builder.force_selectors_on_skip(has_skipped_page); + ) { + plan_builder = + plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); + } // (pre) Fetch only the columns that are selected by the predicate let selection = plan_builder.selection(); @@ -722,9 +724,9 @@ where .build_limited(); // Offset/limit may further trim pages, so re-evaluate the strategy here too. - let has_skipped_page = - selection_has_skipped_page(plan_builder.selection(), &projection, offset_index); - plan_builder = plan_builder.force_selectors_on_skip(has_skipped_page); + if selection_has_skipped_page(plan_builder.selection(), &projection, offset_index) { + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); + } let rows_after = plan_builder .num_rows_selected() From eeccdc6f8ebcac30b86ff3078d515fc997f8ee88 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Thu, 6 Nov 2025 21:09:49 +0800 Subject: [PATCH 05/27] Update the skip page structure --- parquet/benches/row_selection_state.rs | 38 +++------ parquet/src/arrow/arrow_reader/mod.rs | 93 ++++++++++++++++----- parquet/src/arrow/arrow_reader/read_plan.rs | 59 +++++++------ parquet/src/arrow/async_reader/mod.rs | 70 ++-------------- parquet/src/arrow/push_decoder/mod.rs | 1 - 5 files changed, 119 insertions(+), 142 deletions(-) diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs index 057752eec1f1..ec809cbadc3a 100644 --- a/parquet/benches/row_selection_state.rs +++ b/parquet/benches/row_selection_state.rs @@ -24,19 +24,14 @@ use arrow_schema::{DataType, Field, Schema}; use bytes::Bytes; use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use parquet::arrow::ArrowWriter; -use parquet::arrow::arrow_reader::{ - ParquetRecordBatchReaderBuilder, RowSelection, RowSelectionStrategy, RowSelector, -}; +use parquet::arrow::arrow_reader::{ParquetRecordBatchReaderBuilder, RowSelection, RowSelector}; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; const TOTAL_ROWS: usize = 1 << 20; const BATCH_SIZE: usize = 1 << 10; const BASE_SEED: u64 = 0xA55AA55A; -const READ_STRATEGIES: &[(&str, RowSelectionStrategy)] = &[ - ("read_mask", RowSelectionStrategy::Mask), - ("read_selectors", RowSelectionStrategy::Selectors), -]; +const BENCH_LABEL: &str = "read_auto"; const AVG_SELECTOR_LENGTHS: &[usize] = &[4, 8, 12, 16, 20, 24, 28, 32, 36, 40]; const COLUMN_WIDTHS: &[usize] = &[2, 4, 8, 16, 32]; @@ -227,18 +222,16 @@ fn bench_over_lengths( selection, }; - for (label, strategy) in READ_STRATEGIES.iter().copied() { - c.bench_with_input( - BenchmarkId::new(label, &suffix), - &bench_input, - |b, input| { - b.iter(|| { - let total = run_read(&input.parquet_data, &input.selection, strategy); - hint::black_box(total); - }); - }, - ); - } + c.bench_with_input( + BenchmarkId::new(BENCH_LABEL, &suffix), + &bench_input, + |b, input| { + b.iter(|| { + let total = run_read(&input.parquet_data, &input.selection); + hint::black_box(total); + }); + }, + ); } } @@ -250,16 +243,11 @@ struct BenchInput { selection: RowSelection, } -fn run_read( - parquet_data: &Bytes, - selection: &RowSelection, - strategy: RowSelectionStrategy, -) -> usize { +fn run_read(parquet_data: &Bytes, selection: &RowSelection) -> usize { let reader = ParquetRecordBatchReaderBuilder::try_new(parquet_data.clone()) .unwrap() .with_batch_size(BATCH_SIZE) .with_row_selection(selection.clone()) - .with_row_selection_strategy(strategy) .build() .unwrap(); diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index bdbacba628a3..27665db8b819 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -39,6 +39,7 @@ use crate::column::page::{PageIterator, PageReader}; use crate::encryption::decrypt::FileDecryptionProperties; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; +use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; @@ -117,8 +118,6 @@ pub struct ArrowReaderBuilder { pub(crate) selection: Option, - pub(crate) selection_strategy: RowSelectionStrategy, - pub(crate) limit: Option, pub(crate) offset: Option, @@ -140,7 +139,6 @@ impl Debug for ArrowReaderBuilder { .field("projection", &self.projection) .field("filter", &self.filter) .field("selection", &self.selection) - .field("selection_strategy", &self.selection_strategy) .field("limit", &self.limit) .field("offset", &self.offset) .field("metrics", &self.metrics) @@ -148,6 +146,39 @@ impl Debug for ArrowReaderBuilder { } } +fn selection_skips_any_page( + selection: &RowSelection, + projection: &ProjectionMask, + columns: &[OffsetIndexMetaData], +) -> bool { + columns.iter().enumerate().any(|(leaf_idx, column)| { + if !projection.leaf_included(leaf_idx) { + return false; + } + + let locations = column.page_locations(); + if locations.is_empty() { + return false; + } + + let ranges = selection.scan_ranges(locations); + !ranges.is_empty() && ranges.len() < locations.len() + }) +} + +pub(crate) fn should_force_selectors( + selection: Option<&RowSelection>, + projection: &ProjectionMask, + offset_index: Option<&[OffsetIndexMetaData]>, +) -> bool { + match (selection, offset_index) { + (Some(selection), Some(columns)) => { + selection_skips_any_page(selection, projection, columns) + } + _ => false, + } +} + impl ArrowReaderBuilder { pub(crate) fn new_builder(input: T, metadata: ArrowReaderMetadata) -> Self { Self { @@ -160,7 +191,6 @@ impl ArrowReaderBuilder { projection: ProjectionMask::all(), filter: None, selection: None, - selection_strategy: RowSelectionStrategy::Auto, limit: None, offset: None, metrics: ArrowReaderMetrics::Disabled, @@ -275,14 +305,6 @@ impl ArrowReaderBuilder { } } - /// Override the strategy used to execute the configured [`RowSelection`] - pub fn with_row_selection_strategy(self, strategy: RowSelectionStrategy) -> Self { - Self { - selection_strategy: strategy, - ..self - } - } - /// Provide a [`RowFilter`] to skip decoding rows /// /// Row filters are applied after row group selection and row selection @@ -878,7 +900,6 @@ impl ParquetRecordBatchReaderBuilder { projection, mut filter, selection, - selection_strategy, limit, offset, metrics, @@ -899,9 +920,7 @@ impl ParquetRecordBatchReaderBuilder { row_groups, }; - let mut plan_builder = ReadPlanBuilder::new(batch_size) - .with_selection(selection) - .with_selection_strategy(selection_strategy); + let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); // Update selection based on any filters if let Some(filter) = filter.as_mut() { @@ -924,12 +943,41 @@ impl ParquetRecordBatchReaderBuilder { let array_reader = ArrayReaderBuilder::new(&reader, &metrics) .build_array_reader(fields.as_deref(), &projection)?; - let read_plan = plan_builder + let mut plan_builder = plan_builder .limited(reader.num_rows()) .with_offset(offset) .with_limit(limit) - .build_limited() - .build(); + .build_limited(); + + let mask_preferred = plan_builder.mask_preferred(); + if mask_preferred { + let mut force_selectors = false; + // if let (Some(offset_index), Some(mut selection)) = ( + // reader + // .metadata + // .offset_index() + // .filter(|index| !index.is_empty()), + // plan_builder.selection().cloned(), + // ) { + // for &row_group_idx in &reader.row_groups { + // let row_count = reader.metadata.row_group(row_group_idx).num_rows() as usize; + // let row_group_selection = selection.split_off(row_count); + // let columns = offset_index + // .get(row_group_idx) + // .map(|columns| columns.as_slice()); + // if should_force_selectors(Some(&row_group_selection), &projection, columns) { + // force_selectors = true; + // break; + // } + // } + // } + + if !force_selectors { + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); + } + } + + let read_plan = plan_builder.build(); Ok(ParquetRecordBatchReader::new(array_reader, read_plan)) } @@ -5183,7 +5231,7 @@ mod tests { } #[test] - fn test_mask_strategy_full_page_skip_triggers_error() { + fn test_row_filter_full_page_skip_is_handled() { let data = build_mask_pruning_parquet(); let filter_mask; @@ -5206,16 +5254,15 @@ mod tests { or(&match_first, &match_second) }); - let mut reader = ParquetRecordBatchReaderBuilder::try_new_with_options(data, options) + let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(data, options) .unwrap() .with_projection(output_mask) .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) .with_batch_size(256) - .with_row_selection_strategy(RowSelectionStrategy::Mask) .build() .unwrap(); - // The mask strategy used to panic once predicate pruning removed whole pages. + // Predicate pruning used to panic once mask-backed plans removed whole pages. // Collecting into batches validates the plan now downgrades to selectors instead. let schema = reader.schema().clone(); let batches = reader.collect::, _>>().unwrap(); diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 58603843df64..b97d4bf521a8 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -31,16 +31,15 @@ use std::collections::VecDeque; /// Strategy for materialising [`RowSelection`] during execution. #[derive(Clone, Copy, Debug, Default, Eq, PartialEq)] pub enum RowSelectionStrategy { - /// Automatically choose between mask- and selector-backed execution - /// based on heuristics + /// Use a queue of [`RowSelector`] values #[default] - Auto, - /// Always use a boolean mask to materialise the selection - Mask, - /// Always use a queue of [`RowSelector`] values Selectors, + /// Use a boolean mask to materialise the selection + Mask, } +const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 160000; + /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] pub struct ReadPlanBuilder { @@ -57,7 +56,7 @@ impl ReadPlanBuilder { Self { batch_size, selection: None, - selection_strategy: RowSelectionStrategy::Auto, + selection_strategy: RowSelectionStrategy::default(), } } @@ -102,6 +101,24 @@ impl ReadPlanBuilder { self.selection.as_ref().map(|s| s.row_count()) } + /// Returns `true` if materialising the current selection as a mask is expected to be cheaper. + pub fn mask_preferred(&self) -> bool { + let selection = match self.selection.as_ref() { + Some(selection) => selection, + None => return true, + }; + + let trimmed = selection.clone().trim(); + let selectors: Vec = trimmed.into(); + if selectors.is_empty() { + return true; + } + + let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); + let selector_count = selectors.len(); + total_rows < selector_count.saturating_mul(AVG_SELECTOR_LEN_MASK_THRESHOLD) + } + /// Evaluates an [`ArrowPredicate`], updating this plan's `selection` /// /// If the current `selection` is `Some`, the resulting [`RowSelection`] @@ -315,31 +332,11 @@ pub struct MaskChunk { impl RowSelectionCursor { /// Create a cursor, choosing an efficient backing representation fn new(selectors: Vec, strategy: RowSelectionStrategy) -> Self { - if matches!(strategy, RowSelectionStrategy::Selectors) { - return Self { - storage: RowSelectionBacking::Selectors(selectors.into()), - position: 0, - }; - } - - let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); - let selector_count = selectors.len(); - const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 16; - // Prefer a bitmap mask when the selectors are short on average, as the mask - // (re)construction cost is amortized by a simpler execution path during reads. - let use_mask = match strategy { - RowSelectionStrategy::Mask => true, - RowSelectionStrategy::Auto => { - selector_count == 0 - || total_rows < selector_count.saturating_mul(AVG_SELECTOR_LEN_MASK_THRESHOLD) + let storage = match strategy { + RowSelectionStrategy::Mask => { + RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) } - RowSelectionStrategy::Selectors => unreachable!(), - }; - - let storage = if use_mask { - RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) - } else { - RowSelectionBacking::Selectors(selectors.into()) + RowSelectionStrategy::Selectors => RowSelectionBacking::Selectors(selectors.into()), }; Self { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index f3f3e4dea3c9..70de7f899ce3 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -38,6 +38,7 @@ use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt}; use arrow_array::RecordBatch; use arrow_schema::{DataType, Fields, Schema, SchemaRef}; +use crate::arrow::arrow_reader::should_force_selectors; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection, RowSelectionStrategy, @@ -49,7 +50,6 @@ use crate::bloom_filter::{ }; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; -use crate::file::page_index::offset_index::OffsetIndexMetaData; mod metadata; pub use metadata::*; @@ -66,42 +66,6 @@ use crate::arrow::schema::ParquetField; #[cfg(feature = "object_store")] pub use store::*; -/// Returns true when `selection` keeps some rows for `projection` but prunes whole -/// data pages (determined via `OffsetIndex`). Masks can't handle this because the -/// page data is never fetched, so the caller should fall back to RowSelectors. -pub(crate) fn selection_skips_any_page( - selection: &RowSelection, - projection: &ProjectionMask, - columns: &[OffsetIndexMetaData], -) -> bool { - columns.iter().enumerate().any(|(leaf_idx, column)| { - if !projection.leaf_included(leaf_idx) { - return false; - } - - let locations = column.page_locations(); - if locations.is_empty() { - return false; - } - - let ranges = selection.scan_ranges(locations); - !ranges.is_empty() && ranges.len() < locations.len() - }) -} - -fn selection_has_skipped_page( - selection: Option<&RowSelection>, - projection: &ProjectionMask, - offset_index: Option<&[OffsetIndexMetaData]>, -) -> bool { - match (selection, offset_index) { - (Some(selection), Some(offset_index)) => { - selection_skips_any_page(selection, projection, offset_index) - } - _ => false, - } -} - /// The asynchronous interface used by [`ParquetRecordBatchStream`] to read parquet files /// /// Notes: @@ -545,7 +509,6 @@ impl ParquetRecordBatchStreamBuilder { filter: self.filter, metadata: self.metadata.clone(), fields: self.fields, - selection_strategy: self.selection_strategy, limit: self.limit, offset: self.offset, metrics: self.metrics, @@ -595,9 +558,6 @@ struct ReaderFactory { /// Optional filter filter: Option, - /// Strategy used to materialise row selections for this reader - selection_strategy: RowSelectionStrategy, - /// Limit to apply to remaining row groups. limit: Option, @@ -661,9 +621,7 @@ where let cache_options_builder = CacheOptionsBuilder::new(&cache_projection, &row_group_cache); let filter = self.filter.as_mut(); - let mut plan_builder = ReadPlanBuilder::new(batch_size) - .with_selection(selection) - .with_selection_strategy(self.selection_strategy); + let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); // Update selection based on any filters if let Some(filter) = filter { @@ -674,17 +632,6 @@ where return Ok((self, None)); // ruled out entire row group } - // Predicate evaluation can zero out some pages; switch to selectors - // before the mask-backed cursor tries to touch data we won't fetch. - if selection_has_skipped_page( - plan_builder.selection(), - predicate.projection(), - offset_index, - ) { - plan_builder = - plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); - } - // (pre) Fetch only the columns that are selected by the predicate let selection = plan_builder.selection(); // Fetch predicate columns; expand selection only for cached predicate columns @@ -723,11 +670,6 @@ where .with_limit(self.limit) .build_limited(); - // Offset/limit may further trim pages, so re-evaluate the strategy here too. - if selection_has_skipped_page(plan_builder.selection(), &projection, offset_index) { - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Selectors); - } - let rows_after = plan_builder .num_rows_selected() .unwrap_or(row_group.row_count); @@ -758,6 +700,12 @@ where ) .await?; + if plan_builder.mask_preferred() + && !should_force_selectors(plan_builder.selection(), &projection, offset_index) + { + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); + } + let plan = plan_builder.build(); let cache_options = cache_options_builder.consumer(); @@ -1833,7 +1781,6 @@ mod tests { fields: fields.map(Arc::new), input: async_reader, filter: None, - selection_strategy: RowSelectionStrategy::Auto, limit: None, offset: None, metrics: ArrowReaderMetrics::disabled(), @@ -2299,7 +2246,6 @@ mod tests { fields: None, input: TestReader::new(data), filter: Some(filter), - selection_strategy: RowSelectionStrategy::Auto, limit: None, offset: None, metrics: ArrowReaderMetrics::disabled(), diff --git a/parquet/src/arrow/push_decoder/mod.rs b/parquet/src/arrow/push_decoder/mod.rs index 3da7c4bd2fbc..4b932fb08034 100644 --- a/parquet/src/arrow/push_decoder/mod.rs +++ b/parquet/src/arrow/push_decoder/mod.rs @@ -168,7 +168,6 @@ impl ParquetPushDecoderBuilder { projection, filter, selection, - selection_strategy: _, limit, offset, metrics, From 4239db11a001f16e07ed60ef0111d68bb8311e46 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 02:03:21 +0800 Subject: [PATCH 06/27] Add unit test to readers. --- parquet/src/arrow/arrow_reader/mod.rs | 42 +++-------- parquet/src/arrow/arrow_reader/read_plan.rs | 2 +- parquet/src/arrow/async_reader/mod.rs | 84 ++++++++++++++++++++- 3 files changed, 90 insertions(+), 38 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 27665db8b819..a469b256ad0d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -951,30 +951,8 @@ impl ParquetRecordBatchReaderBuilder { let mask_preferred = plan_builder.mask_preferred(); if mask_preferred { - let mut force_selectors = false; - // if let (Some(offset_index), Some(mut selection)) = ( - // reader - // .metadata - // .offset_index() - // .filter(|index| !index.is_empty()), - // plan_builder.selection().cloned(), - // ) { - // for &row_group_idx in &reader.row_groups { - // let row_count = reader.metadata.row_group(row_group_idx).num_rows() as usize; - // let row_group_selection = selection.split_off(row_count); - // let columns = offset_index - // .get(row_group_idx) - // .map(|columns| columns.as_slice()); - // if should_force_selectors(Some(&row_group_selection), &projection, columns) { - // force_selectors = true; - // break; - // } - // } - // } - - if !force_selectors { - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); - } + // There's no page skipping in the sync reader, so always use Mask selection strategy when it's preferred + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); } let read_plan = plan_builder.build(); @@ -1307,12 +1285,11 @@ mod tests { use crate::arrow::arrow_reader::{ ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader, - ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelectionStrategy, - RowSelector, + ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, }; use crate::arrow::schema::add_encoded_arrow_schema_to_metadata; use crate::arrow::{ArrowWriter, ProjectionMask}; - use crate::basic::{Compression, ConvertedType, Encoding, Repetition, Type as PhysicalType}; + use crate::basic::{ConvertedType, Encoding, Repetition, Type as PhysicalType}; use crate::column::reader::decoder::REPETITION_LEVELS_BATCH_SIZE; use crate::data_type::{ BoolType, ByteArray, ByteArrayType, DataType, FixedLenByteArray, FixedLenByteArrayType, @@ -5195,8 +5172,8 @@ mod tests { c0.iter().zip(c1.iter()).for_each(|(l, r)| assert_eq!(l, r)); } - const SECOND_MATCH_INDEX: usize = 4096; - const SECOND_MATCH_VALUE: i64 = 12345; + const SECOND_MATCH_INDEX: usize = 31; + const SECOND_MATCH_VALUE: i64 = 9998; fn build_mask_pruning_parquet() -> Bytes { let schema = Arc::new(Schema::new(vec![ @@ -5204,7 +5181,7 @@ mod tests { Field::new("value", ArrowDataType::Float64, false), ])); - let num_rows = 8192usize; + let num_rows = 32usize; let mut int_values: Vec = (0..num_rows as i64).collect(); int_values[0] = 9999; int_values[SECOND_MATCH_INDEX] = SECOND_MATCH_VALUE; @@ -5217,9 +5194,8 @@ mod tests { .unwrap(); let props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .set_data_page_size_limit(1024) - .set_data_page_row_count_limit(32) + .set_write_batch_size(2) + .set_data_page_row_count_limit(2) .build(); let mut buffer = Vec::new(); diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index b97d4bf521a8..10366f21423f 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -38,7 +38,7 @@ pub enum RowSelectionStrategy { Mask, } -const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 160000; +const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 32; /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 70de7f899ce3..f78232eebbd1 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -38,7 +38,6 @@ use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt}; use arrow_array::RecordBatch; use arrow_schema::{DataType, Fields, Schema, SchemaRef}; -use crate::arrow::arrow_reader::should_force_selectors; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection, RowSelectionStrategy, @@ -701,7 +700,7 @@ where .await?; if plan_builder.mask_preferred() - && !should_force_selectors(plan_builder.selection(), &projection, offset_index) + // && !should_force_selectors(plan_builder.selection(), &projection, offset_index) { plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); } @@ -1010,18 +1009,21 @@ mod tests { }; use crate::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; + use crate::basic::Compression; use crate::file::metadata::ParquetMetaDataReader; use crate::file::properties::WriterProperties; use arrow::compute::kernels::cmp::eq; + use arrow::compute::or; use arrow::error::Result as ArrowResult; use arrow_array::builder::{ListBuilder, StringBuilder}; use arrow_array::cast::AsArray; use arrow_array::types::Int32Type; use arrow_array::{ - Array, ArrayRef, Int8Array, Int32Array, RecordBatchReader, Scalar, StringArray, - StructArray, UInt64Array, + Array, ArrayRef, Float64Array, Int8Array, Int32Array, Int64Array, RecordBatchReader, + Scalar, StringArray, StructArray, UInt64Array, }; use arrow_schema::{DataType, Field, Schema}; + use arrow_select::concat::concat_batches; use futures::{StreamExt, TryStreamExt}; use rand::{Rng, rng}; use std::collections::HashMap; @@ -1072,6 +1074,40 @@ mod tests { } } + const SECOND_MATCH_INDEX: usize = 31; + const SECOND_MATCH_VALUE: i64 = 9998; + + fn build_mask_pruning_parquet() -> Bytes { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int64, false), + Field::new("value", DataType::Float64, false), + ])); + + let num_rows = 32usize; + let mut int_values: Vec = (0..num_rows as i64).collect(); + int_values[0] = 9999; + int_values[SECOND_MATCH_INDEX] = SECOND_MATCH_VALUE; + let keys = Int64Array::from(int_values); + let values = Float64Array::from_iter_values((0..num_rows).map(|v| v as f64 * 1.5)); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], + ) + .unwrap(); + + let props = WriterProperties::builder() + .set_write_batch_size(2) + .set_data_page_row_count_limit(2) + .build(); + + let mut buffer = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + Bytes::from(buffer) + } + #[tokio::test] async fn test_async_reader() { let testdata = arrow::util::test_util::parquet_test_data(); @@ -1447,6 +1483,46 @@ mod tests { assert_eq!(actual_rows, expected_rows); } + #[tokio::test] + async fn test_row_filter_full_page_skip_is_handled_async() { + let data = build_mask_pruning_parquet(); + + let builder = ParquetRecordBatchStreamBuilder::new_with_options( + TestReader::new(data.clone()), + ArrowReaderOptions::new().with_page_index(true), + ) + .await + .unwrap(); + let schema = builder.parquet_schema().clone(); + let filter_mask = ProjectionMask::leaves(&schema, [0]); + let output_mask = ProjectionMask::leaves(&schema, [1]); + + let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { + let column = batch.column(0); + let match_first = eq(column, &Int64Array::new_scalar(9999))?; + let match_second = eq(column, &Int64Array::new_scalar(SECOND_MATCH_VALUE))?; + or(&match_first, &match_second) + }); + + let stream = ParquetRecordBatchStreamBuilder::new_with_options( + TestReader::new(data), + ArrowReaderOptions::new().with_page_index(true), + ) + .await + .unwrap() + .with_projection(output_mask) + .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) + .with_batch_size(32) + .build() + .unwrap(); + + // Collecting into batches validates the plan now downgrades to selectors instead of panicking. + let schema = stream.schema().clone(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + let result = concat_batches(&schema, &batches).unwrap(); + assert_eq!(result.num_rows(), 2); + } + #[tokio::test] async fn test_row_filter() { let a = StringArray::from_iter_values(["a", "b", "b", "b", "c", "c"]); From 3b3304e29e42cb19f4e3ac392008dfe40f7fd40f Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 02:16:36 +0800 Subject: [PATCH 07/27] Fix Clippy --- parquet/src/arrow/async_reader/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index f78232eebbd1..ccb4d18f5ac0 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -21,6 +21,7 @@ //! //! See example on [`ParquetRecordBatchStreamBuilder::new`] +use crate::arrow::arrow_reader::should_force_selectors; use std::collections::VecDeque; use std::fmt::Formatter; use std::io::SeekFrom; @@ -700,7 +701,7 @@ where .await?; if plan_builder.mask_preferred() - // && !should_force_selectors(plan_builder.selection(), &projection, offset_index) + && !should_force_selectors(plan_builder.selection(), &projection, offset_index) { plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); } @@ -1009,7 +1010,6 @@ mod tests { }; use crate::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; - use crate::basic::Compression; use crate::file::metadata::ParquetMetaDataReader; use crate::file::properties::WriterProperties; use arrow::compute::kernels::cmp::eq; From 0d8ea307908528df8cd7a93e7e0bfc8859715bf9 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 14:35:27 +0800 Subject: [PATCH 08/27] Add Comments to Unit tests --- parquet/src/arrow/arrow_reader/mod.rs | 66 ++++++++++------------- parquet/src/arrow/async_reader/mod.rs | 75 +++++++++++++-------------- 2 files changed, 63 insertions(+), 78 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 15714933f80a..b32e14e828ad 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -5172,26 +5172,27 @@ mod tests { c0.iter().zip(c1.iter()).for_each(|(l, r)| assert_eq!(l, r)); } - const SECOND_MATCH_INDEX: usize = 31; - const SECOND_MATCH_VALUE: i64 = 9998; + #[test] + fn test_row_filter_full_page_skip_is_handled() { + let first_value: i64 = 1111; + let last_value: i64 = 9999; + let num_rows: usize = 12; - fn build_mask_pruning_parquet() -> Bytes { - let schema = Arc::new(Schema::new(vec![ - Field::new("key", ArrowDataType::Int64, false), - Field::new("value", ArrowDataType::Float64, false), - ])); + // build data with row selection average length 4 + // The result would be (1111 XXXX) ... (4 page in the middle)... (XXXX 9999) + // The Row Selection would be [1111, (skip 10), 9999] + let schema = Arc::new(Schema::new(vec![Field::new( + "key", + ArrowDataType::Int64, + false, + )])); - let num_rows = 32usize; let mut int_values: Vec = (0..num_rows as i64).collect(); - int_values[0] = 9999; - int_values[SECOND_MATCH_INDEX] = SECOND_MATCH_VALUE; + int_values[0] = first_value; + int_values[num_rows - 1] = last_value; let keys = Int64Array::from(int_values); - let values = Float64Array::from_iter_values((0..num_rows).map(|v| v as f64 * 1.5)); - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], - ) - .unwrap(); + let batch = + RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(keys) as ArrayRef]).unwrap(); let props = WriterProperties::builder() .set_write_batch_size(2) @@ -5202,39 +5203,28 @@ mod tests { let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); writer.write(&batch).unwrap(); writer.close().unwrap(); + let data = Bytes::from(buffer); - Bytes::from(buffer) - } - - #[test] - fn test_row_filter_full_page_skip_is_handled() { - let data = build_mask_pruning_parquet(); - - let filter_mask; - let output_mask; - { - let options = ArrowReaderOptions::new().with_page_index(true); - let builder = - ParquetRecordBatchReaderBuilder::try_new_with_options(data.clone(), options) - .unwrap(); - let schema = builder.parquet_schema().clone(); - filter_mask = ProjectionMask::leaves(&schema, [0]); - output_mask = ProjectionMask::leaves(&schema, [1]); - } + let options = ArrowReaderOptions::new().with_page_index(true); + let builder = + ParquetRecordBatchReaderBuilder::try_new_with_options(data.clone(), options).unwrap(); + let schema = builder.parquet_schema().clone(); + let filter_mask = ProjectionMask::leaves(&schema, [0]); let options = ArrowReaderOptions::new().with_page_index(true); let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { let column = batch.column(0); - let match_first = eq(column, &Int64Array::new_scalar(9999))?; - let match_second = eq(column, &Int64Array::new_scalar(SECOND_MATCH_VALUE))?; + let match_first = eq(column, &Int64Array::new_scalar(first_value))?; + let match_second = eq(column, &Int64Array::new_scalar(last_value))?; or(&match_first, &match_second) }); + // The batch size is set to 12 to read all rows in one go after filtering + // If the Reader chooses mask to handle filter, it might cause panic because the mid 4 pages may not be decoded. let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(data, options) .unwrap() - .with_projection(output_mask) .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) - .with_batch_size(256) + .with_batch_size(12) .build() .unwrap(); diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index ccb4d18f5ac0..76e13557eb40 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -1074,40 +1074,6 @@ mod tests { } } - const SECOND_MATCH_INDEX: usize = 31; - const SECOND_MATCH_VALUE: i64 = 9998; - - fn build_mask_pruning_parquet() -> Bytes { - let schema = Arc::new(Schema::new(vec![ - Field::new("key", DataType::Int64, false), - Field::new("value", DataType::Float64, false), - ])); - - let num_rows = 32usize; - let mut int_values: Vec = (0..num_rows as i64).collect(); - int_values[0] = 9999; - int_values[SECOND_MATCH_INDEX] = SECOND_MATCH_VALUE; - let keys = Int64Array::from(int_values); - let values = Float64Array::from_iter_values((0..num_rows).map(|v| v as f64 * 1.5)); - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], - ) - .unwrap(); - - let props = WriterProperties::builder() - .set_write_batch_size(2) - .set_data_page_row_count_limit(2) - .build(); - - let mut buffer = Vec::new(); - let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); - writer.write(&batch).unwrap(); - writer.close().unwrap(); - - Bytes::from(buffer) - } - #[tokio::test] async fn test_async_reader() { let testdata = arrow::util::test_util::parquet_test_data(); @@ -1485,7 +1451,36 @@ mod tests { #[tokio::test] async fn test_row_filter_full_page_skip_is_handled_async() { - let data = build_mask_pruning_parquet(); + let first_value: i64 = 1111; + let last_value: i64 = 9999; + let num_rows: usize = 12; + + // build data with row selection average length 4 + // The result would be (1111 XXXX) ... (4 page in the middle)... (XXXX 9999) + // The Row Selection would be [1111, (skip 10), 9999] + let schema = Arc::new(Schema::new(vec![Field::new( + "key", + DataType::Int64, + false, + )])); + + let mut int_values: Vec = (0..num_rows as i64).collect(); + int_values[0] = first_value; + int_values[num_rows - 1] = last_value; + let keys = Int64Array::from(int_values); + let batch = + RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(keys) as ArrayRef]).unwrap(); + + let props = WriterProperties::builder() + .set_write_batch_size(2) + .set_data_page_row_count_limit(2) + .build(); + + let mut buffer = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + let data = Bytes::from(buffer); let builder = ParquetRecordBatchStreamBuilder::new_with_options( TestReader::new(data.clone()), @@ -1495,24 +1490,24 @@ mod tests { .unwrap(); let schema = builder.parquet_schema().clone(); let filter_mask = ProjectionMask::leaves(&schema, [0]); - let output_mask = ProjectionMask::leaves(&schema, [1]); let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { let column = batch.column(0); - let match_first = eq(column, &Int64Array::new_scalar(9999))?; - let match_second = eq(column, &Int64Array::new_scalar(SECOND_MATCH_VALUE))?; + let match_first = eq(column, &Int64Array::new_scalar(first_value))?; + let match_second = eq(column, &Int64Array::new_scalar(last_value))?; or(&match_first, &match_second) }); + // The batch size is set to 12 to read all rows in one go after filtering + // If the Reader chooses mask to handle filter, it might cause panic because the mid 4 pages may not be decoded. let stream = ParquetRecordBatchStreamBuilder::new_with_options( TestReader::new(data), ArrowReaderOptions::new().with_page_index(true), ) .await .unwrap() - .with_projection(output_mask) .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) - .with_batch_size(32) + .with_batch_size(12) .build() .unwrap(); From efc80d33a31dc7a1568aa86c8f8fd62908321b4e Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 14:51:27 +0800 Subject: [PATCH 09/27] Move should_force_selectors to row selection --- parquet/src/arrow/arrow_reader/mod.rs | 33 ------------------- parquet/src/arrow/arrow_reader/selection.rs | 36 ++++++++++++++++++++- parquet/src/arrow/async_reader/mod.rs | 9 +++--- 3 files changed, 40 insertions(+), 38 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index b32e14e828ad..c9a45c8d87e2 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -146,39 +146,6 @@ impl Debug for ArrowReaderBuilder { } } -fn selection_skips_any_page( - selection: &RowSelection, - projection: &ProjectionMask, - columns: &[OffsetIndexMetaData], -) -> bool { - columns.iter().enumerate().any(|(leaf_idx, column)| { - if !projection.leaf_included(leaf_idx) { - return false; - } - - let locations = column.page_locations(); - if locations.is_empty() { - return false; - } - - let ranges = selection.scan_ranges(locations); - !ranges.is_empty() && ranges.len() < locations.len() - }) -} - -pub(crate) fn should_force_selectors( - selection: Option<&RowSelection>, - projection: &ProjectionMask, - offset_index: Option<&[OffsetIndexMetaData]>, -) -> bool { - match (selection, offset_index) { - (Some(selection), Some(columns)) => { - selection_skips_any_page(selection, projection, columns) - } - _ => false, - } -} - impl ArrowReaderBuilder { pub(crate) fn new_builder(input: T, metadata: ArrowReaderMetadata) -> Self { Self { diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 1eb7c85d1d88..582f4210388c 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -21,7 +21,8 @@ use std::cmp::Ordering; use std::collections::VecDeque; use std::ops::Range; -use crate::file::page_index::offset_index::PageLocation; +use crate::arrow::ProjectionMask; +use crate::file::page_index::offset_index::{OffsetIndexMetaData, PageLocation}; /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when /// scanning a parquet file @@ -213,6 +214,39 @@ impl RowSelection { ranges } + /// Returns true if this selection would skip any data pages within the provided columns + fn selection_skips_any_page( + &self, + projection: &ProjectionMask, + columns: &[OffsetIndexMetaData], + ) -> bool { + columns.iter().enumerate().any(|(leaf_idx, column)| { + if !projection.leaf_included(leaf_idx) { + return false; + } + + let locations = column.page_locations(); + if locations.is_empty() { + return false; + } + + let ranges = self.scan_ranges(locations); + !ranges.is_empty() && ranges.len() < locations.len() + }) + } + + /// Returns true if selectors should be forced, preventing mask materialisation + pub(crate) fn should_force_selectors( + &self, + projection: &ProjectionMask, + offset_index: Option<&[OffsetIndexMetaData]>, + ) -> bool { + match offset_index { + Some(columns) => self.selection_skips_any_page(projection, columns), + None => false, + } + } + /// Splits off the first `row_count` from this [`RowSelection`] pub fn split_off(&mut self, row_count: usize) -> Self { let mut total_count = 0; diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 76e13557eb40..f351a414cef4 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -21,7 +21,6 @@ //! //! See example on [`ParquetRecordBatchStreamBuilder::new`] -use crate::arrow::arrow_reader::should_force_selectors; use std::collections::VecDeque; use std::fmt::Formatter; use std::io::SeekFrom; @@ -700,9 +699,11 @@ where ) .await?; - if plan_builder.mask_preferred() - && !should_force_selectors(plan_builder.selection(), &projection, offset_index) - { + let force_selectors = plan_builder.selection().map_or(false, |selection| { + selection.should_force_selectors(&projection, offset_index) + }); + + if plan_builder.mask_preferred() && !force_selectors { plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); } From 9db01073acdb76bb0577c50790c94967a32a19f0 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 15:18:26 +0800 Subject: [PATCH 10/27] Update the code style and comments --- parquet/src/arrow/arrow_reader/mod.rs | 11 +- parquet/src/arrow/arrow_reader/read_plan.rs | 203 ++------------------ parquet/src/arrow/arrow_reader/selection.rs | 185 ++++++++++++++++++ parquet/src/arrow/async_reader/mod.rs | 100 ++-------- 4 files changed, 216 insertions(+), 283 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index c9a45c8d87e2..6860044c0bbc 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -22,7 +22,7 @@ use arrow_array::{Array, RecordBatch, RecordBatchReader}; use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; use arrow_select::filter::filter_record_batch; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; -pub use selection::{RowSelection, RowSelector}; +pub use selection::{RowSelection, RowSelectionCursor, RowSelectionStrategy, RowSelector}; use std::fmt::{Debug, Formatter}; use std::sync::Arc; @@ -44,7 +44,7 @@ use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; -pub use read_plan::{ReadPlan, ReadPlanBuilder, RowSelectionStrategy}; +pub use read_plan::{ReadPlan, ReadPlanBuilder}; mod filter; pub mod metrics; @@ -916,11 +916,8 @@ impl ParquetRecordBatchReaderBuilder { .with_limit(limit) .build_limited(); - let mask_preferred = plan_builder.mask_preferred(); - if mask_preferred { - // There's no page skipping in the sync reader, so always use Mask selection strategy when it's preferred - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); - } + let preferred_strategy = plan_builder.preferred_selection_strategy(); + plan_builder = plan_builder.with_selection_strategy(preferred_strategy); let read_plan = plan_builder.build(); diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 10366f21423f..058f2cb7069e 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -20,23 +20,12 @@ use crate::arrow::array_reader::ArrayReader; use crate::arrow::arrow_reader::{ - ArrowPredicate, ParquetRecordBatchReader, RowSelection, RowSelector, + ArrowPredicate, ParquetRecordBatchReader, RowSelection, RowSelectionCursor, + RowSelectionStrategy, RowSelector, }; use crate::errors::{ParquetError, Result}; -use arrow_array::{Array, BooleanArray}; -use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; +use arrow_array::Array; use arrow_select::filter::prep_null_mask_filter; -use std::collections::VecDeque; - -/// Strategy for materialising [`RowSelection`] during execution. -#[derive(Clone, Copy, Debug, Default, Eq, PartialEq)] -pub enum RowSelectionStrategy { - /// Use a queue of [`RowSelector`] values - #[default] - Selectors, - /// Use a boolean mask to materialise the selection - Mask, -} const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 32; @@ -101,22 +90,26 @@ impl ReadPlanBuilder { self.selection.as_ref().map(|s| s.row_count()) } - /// Returns `true` if materialising the current selection as a mask is expected to be cheaper. - pub fn mask_preferred(&self) -> bool { + /// Returns the preferred [`RowSelectionStrategy`] for materialising the current selection. + pub fn preferred_selection_strategy(&self) -> RowSelectionStrategy { let selection = match self.selection.as_ref() { Some(selection) => selection, - None => return true, + None => return RowSelectionStrategy::Mask, }; let trimmed = selection.clone().trim(); let selectors: Vec = trimmed.into(); if selectors.is_empty() { - return true; + return RowSelectionStrategy::Mask; } let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); let selector_count = selectors.len(); - total_rows < selector_count.saturating_mul(AVG_SELECTOR_LEN_MASK_THRESHOLD) + if total_rows < selector_count.saturating_mul(AVG_SELECTOR_LEN_MASK_THRESHOLD) { + RowSelectionStrategy::Mask + } else { + RowSelectionStrategy::Selectors + } } /// Evaluates an [`ArrowPredicate`], updating this plan's `selection` @@ -294,175 +287,3 @@ impl ReadPlan { self.batch_size } } - -/// Cursor for iterating a [`RowSelection`] during execution within a [`ReadPlan`]. -/// -/// This keeps per-reader state such as the current position and delegates the -/// actual storage strategy to [`RowSelectionBacking`]. -#[derive(Debug)] -pub struct RowSelectionCursor { - /// Backing storage describing how the selection is materialised - storage: RowSelectionBacking, - /// Current absolute offset into the selection - position: usize, -} - -/// Backing storage that powers [`RowSelectionCursor`]. -/// -/// The cursor either walks a boolean mask (dense representation) or a queue -/// of [`RowSelector`] ranges (sparse representation). -#[derive(Debug)] -enum RowSelectionBacking { - Mask(BooleanBuffer), - Selectors(VecDeque), -} - -/// Result of computing the next chunk to read when using a bitmap mask -pub struct MaskChunk { - /// Number of leading rows to skip before reaching selected rows - pub initial_skip: usize, - /// Total rows covered by this chunk (selected + skipped) - pub chunk_rows: usize, - /// Rows actually selected within the chunk - pub selected_rows: usize, - /// Starting offset within the mask where the chunk begins - pub mask_start: usize, -} - -impl RowSelectionCursor { - /// Create a cursor, choosing an efficient backing representation - fn new(selectors: Vec, strategy: RowSelectionStrategy) -> Self { - let storage = match strategy { - RowSelectionStrategy::Mask => { - RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) - } - RowSelectionStrategy::Selectors => RowSelectionBacking::Selectors(selectors.into()), - }; - - Self { - storage, - position: 0, - } - } - - /// Returns `true` when no further rows remain - pub fn is_empty(&self) -> bool { - match &self.storage { - RowSelectionBacking::Mask(mask) => self.position >= mask.len(), - RowSelectionBacking::Selectors(selectors) => selectors.is_empty(), - } - } - - /// Current position within the overall selection - pub fn position(&self) -> usize { - self.position - } - - /// Return the next [`RowSelector`] when using the sparse representation - pub fn next_selector(&mut self) -> Option { - match &mut self.storage { - RowSelectionBacking::Selectors(selectors) => { - let selector = selectors.pop_front()?; - self.position += selector.row_count; - Some(selector) - } - RowSelectionBacking::Mask(_) => { - unreachable!("next_selector called for mask-based RowSelectionCursor") - } - } - } - - /// Return a selector to the front, rewinding the position (sparse-only) - pub fn return_selector(&mut self, selector: RowSelector) { - match &mut self.storage { - RowSelectionBacking::Selectors(selectors) => { - self.position = self.position.saturating_sub(selector.row_count); - selectors.push_front(selector); - } - RowSelectionBacking::Mask(_) => { - unreachable!("return_selector called for mask-based RowSelectionCursor") - } - } - } - - /// Returns `true` if the cursor is backed by a boolean mask - pub fn is_mask_backed(&self) -> bool { - matches!(self.storage, RowSelectionBacking::Mask(_)) - } - - /// Advance through the mask representation, producing the next chunk summary - pub fn next_mask_chunk(&mut self, batch_size: usize) -> Option { - if !self.is_mask_backed() { - unreachable!("next_mask_chunk called for selector-based RowSelectionCursor") - } - let (initial_skip, chunk_rows, selected_rows, mask_start, end_position) = { - let mask = match &self.storage { - RowSelectionBacking::Mask(mask) => mask, - RowSelectionBacking::Selectors(_) => return None, - }; - - if self.position >= mask.len() { - return None; - } - - let start_position = self.position; - let mut cursor = start_position; - let mut initial_skip = 0; - - while cursor < mask.len() && !mask.value(cursor) { - initial_skip += 1; - cursor += 1; - } - - let mask_start = cursor; - let mut chunk_rows = 0; - let mut selected_rows = 0; - - // Advance until enough rows have been selected to satisfy the batch size, - // or until the mask is exhausted. This mirrors the behaviour of the legacy - // `RowSelector` queue-based iteration. - while cursor < mask.len() && selected_rows < batch_size { - chunk_rows += 1; - if mask.value(cursor) { - selected_rows += 1; - } - cursor += 1; - } - - (initial_skip, chunk_rows, selected_rows, mask_start, cursor) - }; - - self.position = end_position; - - Some(MaskChunk { - initial_skip, - chunk_rows, - selected_rows, - mask_start, - }) - } - - /// Materialise the boolean values for a mask-backed chunk - pub fn mask_values_for(&self, chunk: &MaskChunk) -> Option { - match &self.storage { - RowSelectionBacking::Mask(mask) => { - if chunk.mask_start.saturating_add(chunk.chunk_rows) > mask.len() { - return None; - } - Some(BooleanArray::from( - mask.slice(chunk.mask_start, chunk.chunk_rows), - )) - } - RowSelectionBacking::Selectors(_) => None, - } - } -} - -fn boolean_mask_from_selectors(selectors: &[RowSelector]) -> BooleanBuffer { - let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); - let mut builder = BooleanBufferBuilder::new(total_rows); - for selector in selectors { - builder.append_n(selector.row_count, !selector.skip); - } - builder.finish() -} diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 582f4210388c..3d0e9ac7e549 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -16,6 +16,7 @@ // under the License. use arrow_array::{Array, BooleanArray}; +use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; use arrow_select::filter::SlicesIterator; use std::cmp::Ordering; use std::collections::VecDeque; @@ -24,6 +25,16 @@ use std::ops::Range; use crate::arrow::ProjectionMask; use crate::file::page_index::offset_index::{OffsetIndexMetaData, PageLocation}; +/// Strategy for materialising [`RowSelection`] during execution. +#[derive(Clone, Copy, Debug, Default, Eq, PartialEq)] +pub enum RowSelectionStrategy { + /// Use a queue of [`RowSelector`] values + #[default] + Selectors, + /// Use a boolean mask to materialise the selection + Mask, +} + /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when /// scanning a parquet file #[derive(Debug, Clone, Copy, Eq, PartialEq)] @@ -725,6 +736,180 @@ fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelec iter.collect() } +/// Cursor for iterating a [`RowSelection`] during execution within a +/// [`ReadPlan`](crate::arrow::arrow_reader::ReadPlan). +/// +/// This keeps per-reader state such as the current position and delegates the +/// actual storage strategy to [`RowSelectionBacking`]. +#[derive(Debug)] +pub struct RowSelectionCursor { + /// Backing storage describing how the selection is materialised + storage: RowSelectionBacking, + /// Current absolute offset into the selection + position: usize, +} + +/// Backing storage that powers [`RowSelectionCursor`]. +/// +/// The cursor either walks a boolean mask (dense representation) or a queue +/// of [`RowSelector`] ranges (sparse representation). +#[derive(Debug)] +enum RowSelectionBacking { + Mask(BooleanBuffer), + Selectors(VecDeque), +} + +/// Result of computing the next chunk to read when using a bitmap mask +#[derive(Debug)] +pub struct MaskChunk { + /// Number of leading rows to skip before reaching selected rows + pub initial_skip: usize, + /// Total rows covered by this chunk (selected + skipped) + pub chunk_rows: usize, + /// Rows actually selected within the chunk + pub selected_rows: usize, + /// Starting offset within the mask where the chunk begins + pub mask_start: usize, +} + +impl RowSelectionCursor { + /// Create a cursor, choosing an efficient backing representation + pub(crate) fn new(selectors: Vec, strategy: RowSelectionStrategy) -> Self { + let storage = match strategy { + RowSelectionStrategy::Mask => { + RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) + } + RowSelectionStrategy::Selectors => RowSelectionBacking::Selectors(selectors.into()), + }; + + Self { + storage, + position: 0, + } + } + + /// Returns `true` when no further rows remain + pub fn is_empty(&self) -> bool { + match &self.storage { + RowSelectionBacking::Mask(mask) => self.position >= mask.len(), + RowSelectionBacking::Selectors(selectors) => selectors.is_empty(), + } + } + + /// Current position within the overall selection + pub fn position(&self) -> usize { + self.position + } + + /// Return the next [`RowSelector`] when using the sparse representation + pub fn next_selector(&mut self) -> Option { + match &mut self.storage { + RowSelectionBacking::Selectors(selectors) => { + let selector = selectors.pop_front()?; + self.position += selector.row_count; + Some(selector) + } + RowSelectionBacking::Mask(_) => { + unreachable!("next_selector called for mask-based RowSelectionCursor") + } + } + } + + /// Return a selector to the front, rewinding the position (sparse-only) + pub fn return_selector(&mut self, selector: RowSelector) { + match &mut self.storage { + RowSelectionBacking::Selectors(selectors) => { + self.position = self.position.saturating_sub(selector.row_count); + selectors.push_front(selector); + } + RowSelectionBacking::Mask(_) => { + unreachable!("return_selector called for mask-based RowSelectionCursor") + } + } + } + + /// Returns `true` if the cursor is backed by a boolean mask + pub fn is_mask_backed(&self) -> bool { + matches!(self.storage, RowSelectionBacking::Mask(_)) + } + + /// Advance through the mask representation, producing the next chunk summary + pub fn next_mask_chunk(&mut self, batch_size: usize) -> Option { + if !self.is_mask_backed() { + unreachable!("next_mask_chunk called for selector-based RowSelectionCursor") + } + let (initial_skip, chunk_rows, selected_rows, mask_start, end_position) = { + let mask = match &self.storage { + RowSelectionBacking::Mask(mask) => mask, + RowSelectionBacking::Selectors(_) => return None, + }; + + if self.position >= mask.len() { + return None; + } + + let start_position = self.position; + let mut cursor = start_position; + let mut initial_skip = 0; + + while cursor < mask.len() && !mask.value(cursor) { + initial_skip += 1; + cursor += 1; + } + + let mask_start = cursor; + let mut chunk_rows = 0; + let mut selected_rows = 0; + + // Advance until enough rows have been selected to satisfy the batch size, + // or until the mask is exhausted. This mirrors the behaviour of the legacy + // `RowSelector` queue-based iteration. + while cursor < mask.len() && selected_rows < batch_size { + chunk_rows += 1; + if mask.value(cursor) { + selected_rows += 1; + } + cursor += 1; + } + + (initial_skip, chunk_rows, selected_rows, mask_start, cursor) + }; + + self.position = end_position; + + Some(MaskChunk { + initial_skip, + chunk_rows, + selected_rows, + mask_start, + }) + } + + /// Materialise the boolean values for a mask-backed chunk + pub fn mask_values_for(&self, chunk: &MaskChunk) -> Option { + match &self.storage { + RowSelectionBacking::Mask(mask) => { + if chunk.mask_start.saturating_add(chunk.chunk_rows) > mask.len() { + return None; + } + Some(BooleanArray::from( + mask.slice(chunk.mask_start, chunk.chunk_rows), + )) + } + RowSelectionBacking::Selectors(_) => None, + } + } +} + +fn boolean_mask_from_selectors(selectors: &[RowSelector]) -> BooleanBuffer { + let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); + let mut builder = BooleanBufferBuilder::new(total_rows); + for selector in selectors { + builder.append_n(selector.row_count, !selector.skip); + } + builder.finish() +} + #[cfg(test)] mod tests { use super::*; diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index f351a414cef4..1ec58583f48f 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -557,7 +557,7 @@ struct ReaderFactory { /// Optional filter filter: Option, - /// Limit to apply to remaining row groups. + /// Limit to apply to remaining row groups. limit: Option, /// Offset to apply to the next @@ -699,12 +699,17 @@ where ) .await?; - let force_selectors = plan_builder.selection().map_or(false, |selection| { - selection.should_force_selectors(&projection, offset_index) - }); - - if plan_builder.mask_preferred() && !force_selectors { - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); + // Determine the row selection strategy to use + let preferred_strategy = plan_builder.preferred_selection_strategy(); + if preferred_strategy == RowSelectionStrategy::Mask { + // If the plan builder prefers mask for better performance, but the there's some limits(e.g. the skipped pages) + // that prevent it from using mask, we need to check again here. + let force_selectors = plan_builder.selection().map_or(false, |selection| { + selection.should_force_selectors(&projection, offset_index) + }); + if force_selectors { + plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); + } } let plan = plan_builder.build(); @@ -1020,8 +1025,8 @@ mod tests { use arrow_array::cast::AsArray; use arrow_array::types::Int32Type; use arrow_array::{ - Array, ArrayRef, Float64Array, Int8Array, Int32Array, Int64Array, RecordBatchReader, - Scalar, StringArray, StructArray, UInt64Array, + Array, ArrayRef, Int8Array, Int32Array, Int64Array, RecordBatchReader, Scalar, StringArray, + StructArray, UInt64Array, }; use arrow_schema::{DataType, Field, Schema}; use arrow_select::concat::concat_batches; @@ -1459,11 +1464,7 @@ mod tests { // build data with row selection average length 4 // The result would be (1111 XXXX) ... (4 page in the middle)... (XXXX 9999) // The Row Selection would be [1111, (skip 10), 9999] - let schema = Arc::new(Schema::new(vec![Field::new( - "key", - DataType::Int64, - false, - )])); + let schema = Arc::new(Schema::new(vec![Field::new("key", DataType::Int64, false)])); let mut int_values: Vec = (0..num_rows as i64).collect(); int_values[0] = first_value; @@ -2260,77 +2261,6 @@ mod tests { assert_eq!(requests.lock().unwrap().len(), 3); } - #[tokio::test] - async fn test_cache_projection_excludes_nested_columns() { - use arrow_array::{ArrayRef, StringArray}; - - // Build a simple RecordBatch with a primitive column `a` and a nested struct column `b { aa, bb }` - let a = StringArray::from_iter_values(["r1", "r2"]); - let b = StructArray::from(vec![ - ( - Arc::new(Field::new("aa", DataType::Utf8, true)), - Arc::new(StringArray::from_iter_values(["v1", "v2"])) as ArrayRef, - ), - ( - Arc::new(Field::new("bb", DataType::Utf8, true)), - Arc::new(StringArray::from_iter_values(["w1", "w2"])) as ArrayRef, - ), - ]); - - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", b.data_type().clone(), true), - ])); - - let mut buf = Vec::new(); - let mut writer = ArrowWriter::try_new(&mut buf, schema, None).unwrap(); - let batch = RecordBatch::try_from_iter([ - ("a", Arc::new(a) as ArrayRef), - ("b", Arc::new(b) as ArrayRef), - ]) - .unwrap(); - writer.write(&batch).unwrap(); - writer.close().unwrap(); - - // Load Parquet metadata - let data: Bytes = buf.into(); - let metadata = ParquetMetaDataReader::new() - .parse_and_finish(&data) - .unwrap(); - let metadata = Arc::new(metadata); - - // Build a RowFilter whose predicate projects a leaf under the nested root `b` - // Leaf indices are depth-first; with schema [a, b.aa, b.bb] we pick index 1 (b.aa) - let parquet_schema = metadata.file_metadata().schema_descr(); - let nested_leaf_mask = ProjectionMask::leaves(parquet_schema, vec![1]); - - let always_true = ArrowPredicateFn::new(nested_leaf_mask.clone(), |batch: RecordBatch| { - Ok(arrow_array::BooleanArray::from(vec![ - true; - batch.num_rows() - ])) - }); - let filter = RowFilter::new(vec![Box::new(always_true)]); - - // Construct a ReaderFactory and compute cache projection - let reader_factory = ReaderFactory { - metadata: Arc::clone(&metadata), - fields: None, - input: TestReader::new(data), - filter: Some(filter), - limit: None, - offset: None, - metrics: ArrowReaderMetrics::disabled(), - max_predicate_cache_size: 0, - }; - - // Provide an output projection that also selects the same nested leaf - let cache_projection = reader_factory.compute_cache_projection(&nested_leaf_mask); - - // Expect None since nested columns should be excluded from cache projection - assert!(cache_projection.is_none()); - } - #[tokio::test] #[allow(deprecated)] async fn empty_offset_index_doesnt_panic_in_read_row_group() { From c960d7bf80d836f43ce79e4620c485b9280bf48b Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 15:28:43 +0800 Subject: [PATCH 11/27] Fix building --- parquet/src/arrow/arrow_reader/mod.rs | 1 - parquet/src/arrow/arrow_reader/selection.rs | 2 +- parquet/src/arrow/async_reader/mod.rs | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 6860044c0bbc..0d2b0b4fad72 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -39,7 +39,6 @@ use crate::column::page::{PageIterator, PageReader}; use crate::encryption::decrypt::FileDecryptionProperties; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; -use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 3d0e9ac7e549..61ffa956d0f5 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -754,7 +754,7 @@ pub struct RowSelectionCursor { /// The cursor either walks a boolean mask (dense representation) or a queue /// of [`RowSelector`] ranges (sparse representation). #[derive(Debug)] -enum RowSelectionBacking { +pub enum RowSelectionBacking { Mask(BooleanBuffer), Selectors(VecDeque), } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 1ec58583f48f..7e156a68671f 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -704,7 +704,7 @@ where if preferred_strategy == RowSelectionStrategy::Mask { // If the plan builder prefers mask for better performance, but the there's some limits(e.g. the skipped pages) // that prevent it from using mask, we need to check again here. - let force_selectors = plan_builder.selection().map_or(false, |selection| { + let force_selectors = plan_builder.selection().is_some_and(|selection| { selection.should_force_selectors(&projection, offset_index) }); if force_selectors { From 576aa0ff9bf20bb5e7ae65ce52d814fd3e9895e5 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 15:46:56 +0800 Subject: [PATCH 12/27] Fix building --- parquet/src/arrow/arrow_reader/selection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 61ffa956d0f5..0501267c2522 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -740,7 +740,7 @@ fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelec /// [`ReadPlan`](crate::arrow::arrow_reader::ReadPlan). /// /// This keeps per-reader state such as the current position and delegates the -/// actual storage strategy to [`RowSelectionBacking`]. +/// actual storage strategy to the internal `RowSelectionBacking`. #[derive(Debug)] pub struct RowSelectionCursor { /// Backing storage describing how the selection is materialised From 365465a3d4c9183723265bdcb6487b0f2e12304d Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 20:00:30 +0800 Subject: [PATCH 13/27] Try add utf-8 bench --- parquet/benches/row_selection_state.rs | 146 +++++++++++++------- parquet/src/arrow/arrow_reader/read_plan.rs | 80 ++++++++++- 2 files changed, 176 insertions(+), 50 deletions(-) diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs index ec809cbadc3a..a6d353eecc77 100644 --- a/parquet/benches/row_selection_state.rs +++ b/parquet/benches/row_selection_state.rs @@ -24,16 +24,20 @@ use arrow_schema::{DataType, Field, Schema}; use bytes::Bytes; use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use parquet::arrow::ArrowWriter; -use parquet::arrow::arrow_reader::{ParquetRecordBatchReaderBuilder, RowSelection, RowSelector}; +use parquet::arrow::arrow_reader::{ + AvgSelectorLenMaskThresholdGuard, ParquetRecordBatchReaderBuilder, RowSelection, RowSelector, + set_avg_selector_len_mask_threshold_for_test, +}; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; const TOTAL_ROWS: usize = 1 << 20; const BATCH_SIZE: usize = 1 << 10; const BASE_SEED: u64 = 0xA55AA55A; -const BENCH_LABEL: &str = "read_auto"; const AVG_SELECTOR_LENGTHS: &[usize] = &[4, 8, 12, 16, 20, 24, 28, 32, 36, 40]; const COLUMN_WIDTHS: &[usize] = &[2, 4, 8, 16, 32]; +const UTF8VIEW_LENS: &[usize] = &[4, 8, 16, 32, 64, 128, 256]; +const BENCH_MODES: &[BenchMode] = &[BenchMode::ReadSelector, BenchMode::ReadMask]; struct DataProfile { name: &'static str, @@ -56,39 +60,27 @@ const DATA_PROFILES: &[DataProfile] = &[ ]; fn criterion_benchmark(c: &mut Criterion) { - /* uniform50 (50% selected, constant run lengths, starts with skip) - ```text - ┌───────────────┐ - │ │ skip - │ │ - │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select - │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - │ │ skip - │ │ - │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select - │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - │ ... │ - └───────────────┘ - ``` */ - let default_scenario = Scenario { - name: "uniform50", - select_ratio: 0.5, - start_with_select: false, - distribution: RunDistribution::Constant, - }; - - let default_parquet = build_parquet_data(TOTAL_ROWS, build_int32_batch); - bench_over_lengths( - c, - "len", - "default", - &default_parquet, - &default_scenario, - BASE_SEED, - ); - - // Additional scenarios reuse the same data/type defaults but change row selection shapes. let scenarios = [ + /* uniform50 (50% selected, constant run lengths, starts with skip) + ```text + ┌───────────────┐ + │ │ skip + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ │ skip + │ │ + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ select + │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + │ ... │ + └───────────────┘ + ``` */ + Scenario { + name: "uniform50", + select_ratio: 0.5, + start_with_select: false, + distribution: RunDistribution::Constant, + }, /* spread50 (50% selected, large jitter in run lengths, starts with skip) ```text ┌───────────────┐ @@ -157,14 +149,20 @@ fn criterion_benchmark(c: &mut Criterion) { }, ]; + let base_parquet = build_parquet_data(TOTAL_ROWS, build_int32_batch); + let base_scenario = &scenarios[0]; + for (idx, scenario) in scenarios.iter().enumerate() { + // The first scenario is a special case for backwards compatibility with + // existing benchmark result formats. + let suite = if idx == 0 { "len" } else { "scenario" }; bench_over_lengths( c, - "scenario", + suite, scenario.name, - &default_parquet, + &base_parquet, scenario, - BASE_SEED ^ ((idx as u64 + 1) << 16), + BASE_SEED ^ ((idx as u64) << 16), ); } @@ -175,7 +173,7 @@ fn criterion_benchmark(c: &mut Criterion) { "dtype", profile.name, &parquet_data, - &default_scenario, + base_scenario, BASE_SEED ^ ((profile_idx as u64) << 24), ); } @@ -188,10 +186,24 @@ fn criterion_benchmark(c: &mut Criterion) { "columns", &variant_label, &parquet_data, - &default_scenario, + base_scenario, BASE_SEED ^ ((offset as u64) << 32), ); } + + for (offset, &len) in UTF8VIEW_LENS.iter().enumerate() { + let batch = build_utf8view_batch_with_len(TOTAL_ROWS, len); + let parquet_data = write_parquet_batch(batch); + let variant_label = format!("utf8view-L{:03}", len); + bench_over_lengths( + c, + "utf8view-len", + &variant_label, + &parquet_data, + base_scenario, + BASE_SEED ^ ((offset as u64) << 40), + ); + } } fn bench_over_lengths( @@ -222,16 +234,20 @@ fn bench_over_lengths( selection, }; - c.bench_with_input( - BenchmarkId::new(BENCH_LABEL, &suffix), - &bench_input, - |b, input| { - b.iter(|| { - let total = run_read(&input.parquet_data, &input.selection); - hint::black_box(total); - }); - }, - ); + for &mode in BENCH_MODES { + c.bench_with_input( + BenchmarkId::new(mode.label(), &suffix), + &bench_input, + |b, input| { + let _guard = mode.override_threshold(); + b.iter(|| { + let total = run_read(&input.parquet_data, &input.selection); + hint::black_box(total); + }); + drop(_guard); + }, + ); + } } } @@ -295,6 +311,16 @@ fn build_utf8view_batch(total_rows: usize) -> RecordBatch { build_single_column_batch(DataType::Utf8View, Arc::new(values) as ArrayRef) } +fn build_utf8view_batch_with_len(total_rows: usize, len: usize) -> RecordBatch { + let mut builder = StringViewBuilder::new(); + let value: String = std::iter::repeat('a').take(len).collect(); + for _ in 0..total_rows { + builder.append_value(&value); + } + let values: StringViewArray = builder.finish(); + build_single_column_batch(DataType::Utf8View, Arc::new(values) as ArrayRef) +} + fn build_int32_columns_batch(total_rows: usize, num_columns: usize) -> RecordBatch { let base_values: ArrayRef = Arc::new(Int32Array::from_iter_values( (0..total_rows).map(|v| v as i32), @@ -422,6 +448,28 @@ fn sample_length(mean: f64, distribution: &RunDistribution, rng: &mut StdRng) -> } } +#[derive(Clone, Copy)] +enum BenchMode { + ReadSelector, + ReadMask, +} + +impl BenchMode { + fn label(self) -> &'static str { + match self { + BenchMode::ReadSelector => "read_selector", + BenchMode::ReadMask => "read_mask", + } + } + + fn override_threshold(self) -> Option { + match self { + BenchMode::ReadSelector => Some(set_avg_selector_len_mask_threshold_for_test(0)), + BenchMode::ReadMask => Some(set_avg_selector_len_mask_threshold_for_test(usize::MAX)), + } + } +} + struct SelectorStats { average_selector_len: f64, select_ratio: f64, diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 058f2cb7069e..947536dda196 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -26,9 +26,31 @@ use crate::arrow::arrow_reader::{ use crate::errors::{ParquetError, Result}; use arrow_array::Array; use arrow_select::filter::prep_null_mask_filter; +use std::sync::atomic::{AtomicUsize, Ordering}; const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 32; +// The logic in `preferred_selection_strategy` depends on the constant +// `AVG_SELECTOR_LEN_MASK_THRESHOLD`. To allow unit testing of this logic, +// we use a mutable global variable that can be temporarily changed during tests. +// +// An `AtomicUsize` is used because the Rust test runner (`cargo test`) runs tests +// in parallel by default. The atomic operations prevent data races between +// different test threads that might try to modify this value simultaneously. +// +// For the production code path, `load(Ordering::Relaxed)` is used. This is the +// weakest memory ordering and for a simple load on most modern architectures, +// it compiles down to a regular memory read with negligible performance overhead. +// The more expensive atomic operations with stronger ordering are only used in the +// test-only functions below. +static AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE: AtomicUsize = + AtomicUsize::new(AVG_SELECTOR_LEN_MASK_THRESHOLD); + +#[inline(always)] +fn avg_selector_len_mask_threshold() -> usize { + AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.load(Ordering::Relaxed) +} + /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] pub struct ReadPlanBuilder { @@ -105,7 +127,7 @@ impl ReadPlanBuilder { let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); let selector_count = selectors.len(); - if total_rows < selector_count.saturating_mul(AVG_SELECTOR_LEN_MASK_THRESHOLD) { + if total_rows < selector_count.saturating_mul(avg_selector_len_mask_threshold()) { RowSelectionStrategy::Mask } else { RowSelectionStrategy::Selectors @@ -287,3 +309,59 @@ impl ReadPlan { self.batch_size } } + +/// An RAII guard that restores the previous value of the override when it is dropped. +/// This ensures that any change to the global threshold is temporary and scoped to +/// the test or benchmark where it's used, even in the case of a panic. +#[cfg(test)] +pub struct AvgSelectorLenMaskThresholdGuard { + previous: usize, +} + +#[cfg(test)] +impl Drop for AvgSelectorLenMaskThresholdGuard { + fn drop(&mut self) { + AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.store(self.previous, Ordering::SeqCst); + } +} + +/// Override [`AVG_SELECTOR_LEN_MASK_THRESHOLD`] for tests and benchmarks. +/// +/// Returns an [`AvgSelectorLenMaskThresholdGuard`] that restores the previous value on drop. +#[cfg(test)] +pub fn set_avg_selector_len_mask_threshold_for_test( + value: usize, +) -> AvgSelectorLenMaskThresholdGuard { + let previous = AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.swap(value, Ordering::SeqCst); + AvgSelectorLenMaskThresholdGuard { previous } +} + +#[cfg(test)] +mod tests { + use super::*; + + fn builder_with_selection(selection: RowSelection) -> ReadPlanBuilder { + ReadPlanBuilder::new(1024).with_selection(Some(selection)) + } + + #[test] + fn preferred_selection_strategy_prefers_mask_by_default() { + let selection = RowSelection::from(vec![RowSelector::select(8)]); + let builder = builder_with_selection(selection); + assert_eq!( + builder.preferred_selection_strategy(), + RowSelectionStrategy::Mask + ); + } + + #[test] + fn preferred_selection_strategy_prefers_selectors_when_threshold_small() { + let _guard = set_avg_selector_len_mask_threshold_for_test(1); + let selection = RowSelection::from(vec![RowSelector::select(8)]); + let builder = builder_with_selection(selection); + assert_eq!( + builder.preferred_selection_strategy(), + RowSelectionStrategy::Selectors + ); + } +} From f317ce06f0ee9dbe0d4f5e516f8368b81cb5dbaa Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 21:26:49 +0800 Subject: [PATCH 14/27] The row selection bench mark --- parquet/benches/row_selection_state.rs | 6 +-- parquet/src/arrow/arrow_reader/mod.rs | 6 ++- parquet/src/arrow/arrow_reader/read_plan.rs | 49 +++++++++------------ 3 files changed, 30 insertions(+), 31 deletions(-) diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs index a6d353eecc77..2879686202b3 100644 --- a/parquet/benches/row_selection_state.rs +++ b/parquet/benches/row_selection_state.rs @@ -26,7 +26,7 @@ use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use parquet::arrow::ArrowWriter; use parquet::arrow::arrow_reader::{ AvgSelectorLenMaskThresholdGuard, ParquetRecordBatchReaderBuilder, RowSelection, RowSelector, - set_avg_selector_len_mask_threshold_for_test, + set_avg_selector_len_mask_threshold, }; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -464,8 +464,8 @@ impl BenchMode { fn override_threshold(self) -> Option { match self { - BenchMode::ReadSelector => Some(set_avg_selector_len_mask_threshold_for_test(0)), - BenchMode::ReadMask => Some(set_avg_selector_len_mask_threshold_for_test(usize::MAX)), + BenchMode::ReadSelector => Some(set_avg_selector_len_mask_threshold(0)), + BenchMode::ReadMask => Some(set_avg_selector_len_mask_threshold(usize::MAX)), } } } diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 0d2b0b4fad72..70fb9e0ea018 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -43,7 +43,11 @@ use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; -pub use read_plan::{ReadPlan, ReadPlanBuilder}; +// Exposed so integration tests and benchmarks can temporarily override the threshold. +pub use read_plan::{ + AvgSelectorLenMaskThresholdGuard, ReadPlan, ReadPlanBuilder, + set_avg_selector_len_mask_threshold, +}; mod filter; pub mod metrics; diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 947536dda196..72a99f9d3b5f 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -51,6 +51,27 @@ fn avg_selector_len_mask_threshold() -> usize { AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.load(Ordering::Relaxed) } +/// An RAII guard that restores the previous value of the override when it is dropped. +/// This ensures that any change to the global threshold is temporary and scoped to +/// the test or benchmark where it's used, even in the case of a panic. +pub struct AvgSelectorLenMaskThresholdGuard { + previous: usize, +} + +impl Drop for AvgSelectorLenMaskThresholdGuard { + fn drop(&mut self) { + AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.store(self.previous, Ordering::SeqCst); + } +} + +/// Override [`AVG_SELECTOR_LEN_MASK_THRESHOLD`] (primarily for tests / benchmarks). +/// +/// Returns an [`AvgSelectorLenMaskThresholdGuard`] that restores the previous value on drop. +pub fn set_avg_selector_len_mask_threshold(value: usize) -> AvgSelectorLenMaskThresholdGuard { + let previous = AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.swap(value, Ordering::SeqCst); + AvgSelectorLenMaskThresholdGuard { previous } +} + /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] pub struct ReadPlanBuilder { @@ -310,32 +331,6 @@ impl ReadPlan { } } -/// An RAII guard that restores the previous value of the override when it is dropped. -/// This ensures that any change to the global threshold is temporary and scoped to -/// the test or benchmark where it's used, even in the case of a panic. -#[cfg(test)] -pub struct AvgSelectorLenMaskThresholdGuard { - previous: usize, -} - -#[cfg(test)] -impl Drop for AvgSelectorLenMaskThresholdGuard { - fn drop(&mut self) { - AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.store(self.previous, Ordering::SeqCst); - } -} - -/// Override [`AVG_SELECTOR_LEN_MASK_THRESHOLD`] for tests and benchmarks. -/// -/// Returns an [`AvgSelectorLenMaskThresholdGuard`] that restores the previous value on drop. -#[cfg(test)] -pub fn set_avg_selector_len_mask_threshold_for_test( - value: usize, -) -> AvgSelectorLenMaskThresholdGuard { - let previous = AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.swap(value, Ordering::SeqCst); - AvgSelectorLenMaskThresholdGuard { previous } -} - #[cfg(test)] mod tests { use super::*; @@ -356,7 +351,7 @@ mod tests { #[test] fn preferred_selection_strategy_prefers_selectors_when_threshold_small() { - let _guard = set_avg_selector_len_mask_threshold_for_test(1); + let _guard = set_avg_selector_len_mask_threshold(1); let selection = RowSelection::from(vec![RowSelector::select(8)]); let builder = builder_with_selection(selection); assert_eq!( From b177e09ec60538fd6e9e013a0240bad0bd4a8ca1 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 21:34:43 +0800 Subject: [PATCH 15/27] Add comments --- parquet/src/arrow/arrow_reader/read_plan.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 72a99f9d3b5f..9facfa811c2f 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -28,6 +28,10 @@ use arrow_array::Array; use arrow_select::filter::prep_null_mask_filter; use std::sync::atomic::{AtomicUsize, Ordering}; +// The average selector length threshold for choosing between +// `RowSelectionStrategy::Mask` and `RowSelectionStrategy::Selectors`. +// If the average selector length is less than this value, +// `RowSelectionStrategy::Mask` is preferred. const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 32; // The logic in `preferred_selection_strategy` depends on the constant From 4005626dc6a3b9655da0a681e63cbb57e86326bf Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Fri, 7 Nov 2025 21:38:46 +0800 Subject: [PATCH 16/27] Fix build --- parquet/benches/row_selection_state.rs | 2 +- parquet/src/arrow/arrow_reader/read_plan.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs index 2879686202b3..4bb8f4b79c4d 100644 --- a/parquet/benches/row_selection_state.rs +++ b/parquet/benches/row_selection_state.rs @@ -313,7 +313,7 @@ fn build_utf8view_batch(total_rows: usize) -> RecordBatch { fn build_utf8view_batch_with_len(total_rows: usize, len: usize) -> RecordBatch { let mut builder = StringViewBuilder::new(); - let value: String = std::iter::repeat('a').take(len).collect(); + let value: String = "a".repeat(len); for _ in 0..total_rows { builder.append_value(&value); } diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 9facfa811c2f..6db4237716d0 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -68,7 +68,7 @@ impl Drop for AvgSelectorLenMaskThresholdGuard { } } -/// Override [`AVG_SELECTOR_LEN_MASK_THRESHOLD`] (primarily for tests / benchmarks). +/// Override AVG_SELECTOR_LEN_MASK_THRESHOLD (primarily for tests / benchmarks). /// /// Returns an [`AvgSelectorLenMaskThresholdGuard`] that restores the previous value on drop. pub fn set_avg_selector_len_mask_threshold(value: usize) -> AvgSelectorLenMaskThresholdGuard { From fe238774fd3c2e0202e7328c6d1e3174afbd79a1 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Sat, 8 Nov 2025 00:46:07 +0800 Subject: [PATCH 17/27] Update the tests --- parquet/src/arrow/arrow_reader/mod.rs | 19 +++++++++++-------- parquet/src/arrow/async_reader/mod.rs | 17 ++++++++++++----- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 70fb9e0ea018..0eb16267b033 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -5148,18 +5148,21 @@ mod tests { // build data with row selection average length 4 // The result would be (1111 XXXX) ... (4 page in the middle)... (XXXX 9999) // The Row Selection would be [1111, (skip 10), 9999] - let schema = Arc::new(Schema::new(vec![Field::new( - "key", - ArrowDataType::Int64, - false, - )])); + let schema = Arc::new(Schema::new(vec![ + Field::new("key", arrow_schema::DataType::Int64, false), + Field::new("value", arrow_schema::DataType::Int64, false), + ])); let mut int_values: Vec = (0..num_rows as i64).collect(); int_values[0] = first_value; int_values[num_rows - 1] = last_value; - let keys = Int64Array::from(int_values); - let batch = - RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(keys) as ArrayRef]).unwrap(); + let keys = Int64Array::from(int_values.clone()); + let values = Int64Array::from(int_values.clone()); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], + ) + .unwrap(); let props = WriterProperties::builder() .set_write_batch_size(2) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 7e156a68671f..acd4b56aabe2 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -707,7 +707,7 @@ where let force_selectors = plan_builder.selection().is_some_and(|selection| { selection.should_force_selectors(&projection, offset_index) }); - if force_selectors { + if !force_selectors { plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); } } @@ -1464,14 +1464,21 @@ mod tests { // build data with row selection average length 4 // The result would be (1111 XXXX) ... (4 page in the middle)... (XXXX 9999) // The Row Selection would be [1111, (skip 10), 9999] - let schema = Arc::new(Schema::new(vec![Field::new("key", DataType::Int64, false)])); + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int64, false), + Field::new("value", DataType::Int64, false), + ])); let mut int_values: Vec = (0..num_rows as i64).collect(); int_values[0] = first_value; int_values[num_rows - 1] = last_value; - let keys = Int64Array::from(int_values); - let batch = - RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(keys) as ArrayRef]).unwrap(); + let keys = Int64Array::from(int_values.clone()); + let values = Int64Array::from(int_values.clone()); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], + ) + .unwrap(); let props = WriterProperties::builder() .set_write_batch_size(2) From e17e9072ade1b63610294206f5e95176ba8afdc3 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Sat, 8 Nov 2025 20:55:53 +0800 Subject: [PATCH 18/27] Update the return value of next_selector --- parquet/src/arrow/arrow_reader/mod.rs | 2 +- parquet/src/arrow/arrow_reader/selection.rs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 0eb16267b033..3a4e8e5b9d5c 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1117,7 +1117,7 @@ impl ParquetRecordBatchReader { } } while read_records < batch_size && !selection_cursor.is_empty() { - let front = selection_cursor.next_selector().unwrap(); + let front = selection_cursor.next_selector(); if front.skip { let skipped = self.array_reader.skip_records(front.row_count)?; diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 0501267c2522..bd6cfd5e29a8 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -802,12 +802,12 @@ impl RowSelectionCursor { } /// Return the next [`RowSelector`] when using the sparse representation - pub fn next_selector(&mut self) -> Option { + pub fn next_selector(&mut self) -> RowSelector { match &mut self.storage { RowSelectionBacking::Selectors(selectors) => { - let selector = selectors.pop_front()?; + let selector = selectors.pop_front().unwrap(); self.position += selector.row_count; - Some(selector) + selector } RowSelectionBacking::Mask(_) => { unreachable!("next_selector called for mask-based RowSelectionCursor") From 25bcdcad29befa607c9b0796122f57c5fe9761e9 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Sun, 9 Nov 2025 21:42:34 +0800 Subject: [PATCH 19/27] Update the strategy API --- parquet/benches/row_selection_state.rs | 21 +-- parquet/src/arrow/arrow_reader/mod.rs | 30 ++-- parquet/src/arrow/arrow_reader/read_plan.rs | 145 +++++++++--------- parquet/src/arrow/arrow_reader/selection.rs | 22 ++- parquet/src/arrow/async_reader/mod.rs | 32 ++-- parquet/src/arrow/push_decoder/mod.rs | 3 + .../arrow/push_decoder/reader_builder/mod.rs | 13 +- 7 files changed, 165 insertions(+), 101 deletions(-) diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs index 4bb8f4b79c4d..893f1c382cc8 100644 --- a/parquet/benches/row_selection_state.rs +++ b/parquet/benches/row_selection_state.rs @@ -25,8 +25,7 @@ use bytes::Bytes; use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use parquet::arrow::ArrowWriter; use parquet::arrow::arrow_reader::{ - AvgSelectorLenMaskThresholdGuard, ParquetRecordBatchReaderBuilder, RowSelection, RowSelector, - set_avg_selector_len_mask_threshold, + ParquetRecordBatchReaderBuilder, RowSelection, RowSelectionStrategy, RowSelector, }; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -239,12 +238,11 @@ fn bench_over_lengths( BenchmarkId::new(mode.label(), &suffix), &bench_input, |b, input| { - let _guard = mode.override_threshold(); b.iter(|| { - let total = run_read(&input.parquet_data, &input.selection); + let total = + run_read(&input.parquet_data, &input.selection, mode.strategy()); hint::black_box(total); }); - drop(_guard); }, ); } @@ -259,11 +257,16 @@ struct BenchInput { selection: RowSelection, } -fn run_read(parquet_data: &Bytes, selection: &RowSelection) -> usize { +fn run_read( + parquet_data: &Bytes, + selection: &RowSelection, + strategy: RowSelectionStrategy, +) -> usize { let reader = ParquetRecordBatchReaderBuilder::try_new(parquet_data.clone()) .unwrap() .with_batch_size(BATCH_SIZE) .with_row_selection(selection.clone()) + .with_row_selection_strategy(strategy) .build() .unwrap(); @@ -462,10 +465,10 @@ impl BenchMode { } } - fn override_threshold(self) -> Option { + fn strategy(self) -> RowSelectionStrategy { match self { - BenchMode::ReadSelector => Some(set_avg_selector_len_mask_threshold(0)), - BenchMode::ReadMask => Some(set_avg_selector_len_mask_threshold(usize::MAX)), + BenchMode::ReadSelector => RowSelectionStrategy::Selectors, + BenchMode::ReadMask => RowSelectionStrategy::Mask, } } } diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 3a4e8e5b9d5c..0f5fb5aa1ed0 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -44,10 +44,7 @@ use crate::schema::types::SchemaDescriptor; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; // Exposed so integration tests and benchmarks can temporarily override the threshold. -pub use read_plan::{ - AvgSelectorLenMaskThresholdGuard, ReadPlan, ReadPlanBuilder, - set_avg_selector_len_mask_threshold, -}; +pub use read_plan::{ReadPlan, ReadPlanBuilder}; mod filter; pub mod metrics; @@ -121,6 +118,8 @@ pub struct ArrowReaderBuilder { pub(crate) selection: Option, + pub(crate) selection_strategy: RowSelectionStrategy, + pub(crate) limit: Option, pub(crate) offset: Option, @@ -142,6 +141,7 @@ impl Debug for ArrowReaderBuilder { .field("projection", &self.projection) .field("filter", &self.filter) .field("selection", &self.selection) + .field("selection_strategy", &self.selection_strategy) .field("limit", &self.limit) .field("offset", &self.offset) .field("metrics", &self.metrics) @@ -161,6 +161,7 @@ impl ArrowReaderBuilder { projection: ProjectionMask::all(), filter: None, selection: None, + selection_strategy: RowSelectionStrategy::default(), limit: None, offset: None, metrics: ArrowReaderMetrics::Disabled, @@ -209,6 +210,14 @@ impl ArrowReaderBuilder { } } + /// Configure how row selections should be materialised during execution + pub fn with_row_selection_strategy(self, strategy: RowSelectionStrategy) -> Self { + Self { + selection_strategy: strategy, + ..self + } + } + /// Provide a [`RowSelection`] to filter out rows, and avoid fetching their /// data into memory. /// @@ -860,16 +869,19 @@ impl ParquetRecordBatchReaderBuilder { /// /// Note: this will eagerly evaluate any `RowFilter` before returning pub fn build(self) -> Result { + let selection_strategy = self.selection_strategy; + let Self { input, metadata, schema: _, fields, - batch_size: _, + batch_size, row_groups, projection, mut filter, selection, + selection_strategy: _, limit, offset, metrics, @@ -878,9 +890,7 @@ impl ParquetRecordBatchReaderBuilder { } = self; // Try to avoid allocate large buffer - let batch_size = self - .batch_size - .min(metadata.file_metadata().num_rows() as usize); + let batch_size = batch_size.min(metadata.file_metadata().num_rows() as usize); let row_groups = row_groups.unwrap_or_else(|| (0..metadata.num_row_groups()).collect()); @@ -890,7 +900,9 @@ impl ParquetRecordBatchReaderBuilder { row_groups, }; - let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); + let mut plan_builder = ReadPlanBuilder::new(batch_size) + .with_selection(selection) + .with_selection_strategy(selection_strategy); // Update selection based on any filters if let Some(filter) = filter.as_mut() { diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 6db4237716d0..1d389590bfd9 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -26,55 +26,6 @@ use crate::arrow::arrow_reader::{ use crate::errors::{ParquetError, Result}; use arrow_array::Array; use arrow_select::filter::prep_null_mask_filter; -use std::sync::atomic::{AtomicUsize, Ordering}; - -// The average selector length threshold for choosing between -// `RowSelectionStrategy::Mask` and `RowSelectionStrategy::Selectors`. -// If the average selector length is less than this value, -// `RowSelectionStrategy::Mask` is preferred. -const AVG_SELECTOR_LEN_MASK_THRESHOLD: usize = 32; - -// The logic in `preferred_selection_strategy` depends on the constant -// `AVG_SELECTOR_LEN_MASK_THRESHOLD`. To allow unit testing of this logic, -// we use a mutable global variable that can be temporarily changed during tests. -// -// An `AtomicUsize` is used because the Rust test runner (`cargo test`) runs tests -// in parallel by default. The atomic operations prevent data races between -// different test threads that might try to modify this value simultaneously. -// -// For the production code path, `load(Ordering::Relaxed)` is used. This is the -// weakest memory ordering and for a simple load on most modern architectures, -// it compiles down to a regular memory read with negligible performance overhead. -// The more expensive atomic operations with stronger ordering are only used in the -// test-only functions below. -static AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE: AtomicUsize = - AtomicUsize::new(AVG_SELECTOR_LEN_MASK_THRESHOLD); - -#[inline(always)] -fn avg_selector_len_mask_threshold() -> usize { - AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.load(Ordering::Relaxed) -} - -/// An RAII guard that restores the previous value of the override when it is dropped. -/// This ensures that any change to the global threshold is temporary and scoped to -/// the test or benchmark where it's used, even in the case of a panic. -pub struct AvgSelectorLenMaskThresholdGuard { - previous: usize, -} - -impl Drop for AvgSelectorLenMaskThresholdGuard { - fn drop(&mut self) { - AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.store(self.previous, Ordering::SeqCst); - } -} - -/// Override AVG_SELECTOR_LEN_MASK_THRESHOLD (primarily for tests / benchmarks). -/// -/// Returns an [`AvgSelectorLenMaskThresholdGuard`] that restores the previous value on drop. -pub fn set_avg_selector_len_mask_threshold(value: usize) -> AvgSelectorLenMaskThresholdGuard { - let previous = AVG_SELECTOR_LEN_MASK_THRESHOLD_OVERRIDE.swap(value, Ordering::SeqCst); - AvgSelectorLenMaskThresholdGuard { previous } -} /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] @@ -102,7 +53,7 @@ impl ReadPlanBuilder { self } - /// Force a specific strategy when materialising the [`RowSelection`] + /// Configure the strategy to use when materialising the [`RowSelection`] pub fn with_selection_strategy(mut self, strategy: RowSelectionStrategy) -> Self { self.selection_strategy = strategy; self @@ -139,24 +90,45 @@ impl ReadPlanBuilder { /// Returns the preferred [`RowSelectionStrategy`] for materialising the current selection. pub fn preferred_selection_strategy(&self) -> RowSelectionStrategy { - let selection = match self.selection.as_ref() { - Some(selection) => selection, - None => return RowSelectionStrategy::Mask, - }; - - let trimmed = selection.clone().trim(); - let selectors: Vec = trimmed.into(); - if selectors.is_empty() { - return RowSelectionStrategy::Mask; + match self.selection_strategy { + RowSelectionStrategy::Selectors => RowSelectionStrategy::Selectors, + RowSelectionStrategy::Mask => RowSelectionStrategy::Mask, + RowSelectionStrategy::Auto { threshold, .. } => { + let selection = match self.selection.as_ref() { + Some(selection) => selection, + None => return RowSelectionStrategy::Mask, + }; + + let trimmed = selection.clone().trim(); + let selectors: Vec = trimmed.into(); + if selectors.is_empty() { + return RowSelectionStrategy::Mask; + } + + let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); + let selector_count = selectors.len(); + if selector_count == 0 { + return RowSelectionStrategy::Mask; + } + + if total_rows < selector_count.saturating_mul(threshold) { + RowSelectionStrategy::Mask + } else { + RowSelectionStrategy::Selectors + } + } } + } - let total_rows: usize = selectors.iter().map(|s| s.row_count).sum(); - let selector_count = selectors.len(); - if total_rows < selector_count.saturating_mul(avg_selector_len_mask_threshold()) { - RowSelectionStrategy::Mask - } else { - RowSelectionStrategy::Selectors - } + /// Returns `true` if the configured strategy allows falling back to selectors for safety. + pub(crate) fn selection_strategy_allows_safe_fallback(&self) -> bool { + matches!( + self.selection_strategy, + RowSelectionStrategy::Auto { + safe_strategy: true, + .. + } + ) } /// Evaluates an [`ArrowPredicate`], updating this plan's `selection` @@ -206,7 +178,10 @@ impl ReadPlanBuilder { if !self.selects_any() { self.selection = Some(RowSelection::from(vec![])); } - let selection_strategy = self.selection_strategy; + let selection_strategy = match self.selection_strategy { + RowSelectionStrategy::Auto { .. } => self.preferred_selection_strategy(), + strategy => strategy, + }; let Self { batch_size, selection, @@ -355,12 +330,44 @@ mod tests { #[test] fn preferred_selection_strategy_prefers_selectors_when_threshold_small() { - let _guard = set_avg_selector_len_mask_threshold(1); let selection = RowSelection::from(vec![RowSelector::select(8)]); - let builder = builder_with_selection(selection); + let builder = + builder_with_selection(selection).with_selection_strategy(RowSelectionStrategy::Auto { + threshold: 1, + safe_strategy: true, + }); assert_eq!( builder.preferred_selection_strategy(), RowSelectionStrategy::Selectors ); } + + #[test] + fn selection_strategy_safe_fallback_detection() { + let selection = RowSelection::from(vec![RowSelector::select(8)]); + + let builder_safe = builder_with_selection(selection.clone()).with_selection_strategy( + RowSelectionStrategy::Auto { + threshold: 32, + safe_strategy: true, + }, + ); + assert!(builder_safe.selection_strategy_allows_safe_fallback()); + + let builder_unsafe = builder_with_selection(selection.clone()).with_selection_strategy( + RowSelectionStrategy::Auto { + threshold: 32, + safe_strategy: false, + }, + ); + assert!(!builder_unsafe.selection_strategy_allows_safe_fallback()); + + let builder_mask = builder_with_selection(selection.clone()) + .with_selection_strategy(RowSelectionStrategy::Mask); + assert!(!builder_mask.selection_strategy_allows_safe_fallback()); + + let builder_selectors = builder_with_selection(selection) + .with_selection_strategy(RowSelectionStrategy::Selectors); + assert!(!builder_selectors.selection_strategy_allows_safe_fallback()); + } } diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index bd6cfd5e29a8..ff975e2bc42d 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -26,13 +26,28 @@ use crate::arrow::ProjectionMask; use crate::file::page_index::offset_index::{OffsetIndexMetaData, PageLocation}; /// Strategy for materialising [`RowSelection`] during execution. -#[derive(Clone, Copy, Debug, Default, Eq, PartialEq)] +#[derive(Clone, Copy, Debug, Eq, PartialEq)] pub enum RowSelectionStrategy { /// Use a queue of [`RowSelector`] values - #[default] Selectors, /// Use a boolean mask to materialise the selection Mask, + /// Choose between [`Self::Mask`] and [`Self::Selectors`] based on selector density + Auto { + /// Average selector length below which masks are preferred + threshold: usize, + /// Fallback to selectors when mask would be unsafe (e.g. page skipping) + safe_strategy: bool, + }, +} + +impl Default for RowSelectionStrategy { + fn default() -> Self { + Self::Auto { + threshold: 32, + safe_strategy: true, + } + } } /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when @@ -780,6 +795,9 @@ impl RowSelectionCursor { RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) } RowSelectionStrategy::Selectors => RowSelectionBacking::Selectors(selectors.into()), + RowSelectionStrategy::Auto { .. } => { + panic!("RowSelectionStrategy::Auto must be resolved before creating cursor") + } }; Self { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index acd4b56aabe2..5cffefc7ca90 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -508,6 +508,7 @@ impl ParquetRecordBatchStreamBuilder { filter: self.filter, metadata: self.metadata.clone(), fields: self.fields, + selection_strategy: self.selection_strategy, limit: self.limit, offset: self.offset, metrics: self.metrics, @@ -557,6 +558,9 @@ struct ReaderFactory { /// Optional filter filter: Option, + /// Strategy used to materialise row selections + selection_strategy: RowSelectionStrategy, + /// Limit to apply to remaining row groups. limit: Option, @@ -620,7 +624,9 @@ where let cache_options_builder = CacheOptionsBuilder::new(&cache_projection, &row_group_cache); let filter = self.filter.as_mut(); - let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); + let mut plan_builder = ReadPlanBuilder::new(batch_size) + .with_selection(selection) + .with_selection_strategy(self.selection_strategy); // Update selection based on any filters if let Some(filter) = filter { @@ -700,17 +706,24 @@ where .await?; // Determine the row selection strategy to use + let allow_safe_fallback = plan_builder.selection_strategy_allows_safe_fallback(); let preferred_strategy = plan_builder.preferred_selection_strategy(); - if preferred_strategy == RowSelectionStrategy::Mask { - // If the plan builder prefers mask for better performance, but the there's some limits(e.g. the skipped pages) + let resolved_strategy = if preferred_strategy == RowSelectionStrategy::Mask { + // If the plan builder prefers mask for better performance, but there's some limits (e.g. skipped pages) // that prevent it from using mask, we need to check again here. - let force_selectors = plan_builder.selection().is_some_and(|selection| { - selection.should_force_selectors(&projection, offset_index) - }); - if !force_selectors { - plan_builder = plan_builder.with_selection_strategy(RowSelectionStrategy::Mask); + let force_selectors = allow_safe_fallback + && plan_builder.selection().is_some_and(|selection| { + selection.should_force_selectors(&projection, offset_index) + }); + if force_selectors { + RowSelectionStrategy::Selectors + } else { + RowSelectionStrategy::Mask } - } + } else { + preferred_strategy + }; + plan_builder = plan_builder.with_selection_strategy(resolved_strategy); let plan = plan_builder.build(); @@ -1861,6 +1874,7 @@ mod tests { fields: fields.map(Arc::new), input: async_reader, filter: None, + selection_strategy: RowSelectionStrategy::default(), limit: None, offset: None, metrics: ArrowReaderMetrics::disabled(), diff --git a/parquet/src/arrow/push_decoder/mod.rs b/parquet/src/arrow/push_decoder/mod.rs index 4b932fb08034..fb5cf6928f9e 100644 --- a/parquet/src/arrow/push_decoder/mod.rs +++ b/parquet/src/arrow/push_decoder/mod.rs @@ -158,6 +158,7 @@ impl ParquetPushDecoderBuilder { /// Create a [`ParquetPushDecoder`] with the configured options pub fn build(self) -> Result { + let selection_strategy = self.selection_strategy; let Self { input: file_len, metadata: parquet_metadata, @@ -171,6 +172,7 @@ impl ParquetPushDecoderBuilder { limit, offset, metrics, + selection_strategy: _, max_predicate_cache_size, } = self; @@ -191,6 +193,7 @@ impl ParquetPushDecoderBuilder { metrics, max_predicate_cache_size, buffers, + selection_strategy, ); // Initialize the decoder with the configured options diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index 69b32d181d3c..313d509b55f9 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -23,7 +23,7 @@ use crate::arrow::ProjectionMask; use crate::arrow::array_reader::{ArrayReaderBuilder, RowGroupCache}; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; use crate::arrow::arrow_reader::{ - ParquetRecordBatchReader, ReadPlanBuilder, RowFilter, RowSelection, + ParquetRecordBatchReader, ReadPlanBuilder, RowFilter, RowSelection, RowSelectionStrategy, }; use crate::arrow::in_memory_row_group::ColumnChunkData; use crate::arrow::push_decoder::reader_builder::data::DataRequestBuilder; @@ -155,6 +155,9 @@ pub(crate) struct RowGroupReaderBuilder { /// The metrics collector metrics: ArrowReaderMetrics, + /// Strategy for materialising row selections + selection_strategy: RowSelectionStrategy, + /// Current state of the decoder. /// /// It is taken when processing, and must be put back before returning @@ -179,6 +182,7 @@ impl RowGroupReaderBuilder { metrics: ArrowReaderMetrics, max_predicate_cache_size: usize, buffers: PushBuffers, + selection_strategy: RowSelectionStrategy, ) -> Self { Self { batch_size, @@ -190,6 +194,7 @@ impl RowGroupReaderBuilder { offset, metrics, max_predicate_cache_size, + selection_strategy, state: Some(RowGroupDecoderState::Finished), buffers, } @@ -233,7 +238,9 @@ impl RowGroupReaderBuilder { "Internal Error: next_row_group called while still reading a row group. Expected Finished state, got {state:?}" ))); } - let plan_builder = ReadPlanBuilder::new(self.batch_size).with_selection(selection); + let plan_builder = ReadPlanBuilder::new(self.batch_size) + .with_selection(selection) + .with_selection_strategy(self.selection_strategy); let row_group_info = RowGroupInfo { row_group_idx, @@ -654,6 +661,6 @@ mod tests { #[test] // Verify that the size of RowGroupDecoderState does not grow too large fn test_structure_size() { - assert_eq!(std::mem::size_of::(), 192); + assert_eq!(std::mem::size_of::(), 200); } } From 9cb9e82ff35a4f25851c6a1db19dca4bf2a356ad Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Mon, 10 Nov 2025 22:34:16 +0800 Subject: [PATCH 20/27] Support the latest push_decoder --- parquet/src/arrow/arrow_reader/read_plan.rs | 42 +--------- parquet/src/arrow/async_reader/mod.rs | 78 ++++++++++++++++++- .../arrow/push_decoder/reader_builder/mod.rs | 41 +++++++++- 3 files changed, 118 insertions(+), 43 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 1d389590bfd9..5b5191115f80 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -96,7 +96,7 @@ impl ReadPlanBuilder { RowSelectionStrategy::Auto { threshold, .. } => { let selection = match self.selection.as_ref() { Some(selection) => selection, - None => return RowSelectionStrategy::Mask, + None => return RowSelectionStrategy::Selectors, }; let trimmed = selection.clone().trim(); @@ -120,17 +120,6 @@ impl ReadPlanBuilder { } } - /// Returns `true` if the configured strategy allows falling back to selectors for safety. - pub(crate) fn selection_strategy_allows_safe_fallback(&self) -> bool { - matches!( - self.selection_strategy, - RowSelectionStrategy::Auto { - safe_strategy: true, - .. - } - ) - } - /// Evaluates an [`ArrowPredicate`], updating this plan's `selection` /// /// If the current `selection` is `Some`, the resulting [`RowSelection`] @@ -341,33 +330,4 @@ mod tests { RowSelectionStrategy::Selectors ); } - - #[test] - fn selection_strategy_safe_fallback_detection() { - let selection = RowSelection::from(vec![RowSelector::select(8)]); - - let builder_safe = builder_with_selection(selection.clone()).with_selection_strategy( - RowSelectionStrategy::Auto { - threshold: 32, - safe_strategy: true, - }, - ); - assert!(builder_safe.selection_strategy_allows_safe_fallback()); - - let builder_unsafe = builder_with_selection(selection.clone()).with_selection_strategy( - RowSelectionStrategy::Auto { - threshold: 32, - safe_strategy: false, - }, - ); - assert!(!builder_unsafe.selection_strategy_allows_safe_fallback()); - - let builder_mask = builder_with_selection(selection.clone()) - .with_selection_strategy(RowSelectionStrategy::Mask); - assert!(!builder_mask.selection_strategy_allows_safe_fallback()); - - let builder_selectors = builder_with_selection(selection) - .with_selection_strategy(RowSelectionStrategy::Selectors); - assert!(!builder_selectors.selection_strategy_allows_safe_fallback()); - } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 44c5465202e7..9f2b22ed29d7 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -486,6 +486,7 @@ impl ParquetRecordBatchStreamBuilder { projection, filter, selection, + selection_strategy, limit, offset, metrics, @@ -507,6 +508,7 @@ impl ParquetRecordBatchStreamBuilder { projection, filter, selection, + selection_strategy, batch_size, row_groups, limit, @@ -769,15 +771,17 @@ mod tests { use crate::file::metadata::ParquetMetaDataReader; use crate::file::properties::WriterProperties; use arrow::compute::kernels::cmp::eq; + use arrow::compute::or; use arrow::error::Result as ArrowResult; use arrow_array::builder::{ListBuilder, StringBuilder}; use arrow_array::cast::AsArray; use arrow_array::types::Int32Type; use arrow_array::{ - Array, ArrayRef, Int8Array, Int32Array, RecordBatchReader, Scalar, StringArray, + Array, ArrayRef, Int8Array, Int32Array, Int64Array, RecordBatchReader, Scalar, StringArray, StructArray, UInt64Array, }; use arrow_schema::{DataType, Field, Schema}; + use arrow_select::concat::concat_batches; use futures::{StreamExt, TryStreamExt}; use rand::{Rng, rng}; use std::collections::HashMap; @@ -1203,6 +1207,78 @@ mod tests { assert_eq!(actual_rows, expected_rows); } + #[tokio::test] + async fn test_row_filter_full_page_skip_is_handled_async() { + let first_value: i64 = 1111; + let last_value: i64 = 9999; + let num_rows: usize = 12; + + // build data with row selection average length 4 + // The result would be (1111 XXXX) ... (4 page in the middle)... (XXXX 9999) + // The Row Selection would be [1111, (skip 10), 9999] + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int64, false), + Field::new("value", DataType::Int64, false), + ])); + + let mut int_values: Vec = (0..num_rows as i64).collect(); + int_values[0] = first_value; + int_values[num_rows - 1] = last_value; + let keys = Int64Array::from(int_values.clone()); + let values = Int64Array::from(int_values.clone()); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(keys) as ArrayRef, Arc::new(values) as ArrayRef], + ) + .unwrap(); + + let props = WriterProperties::builder() + .set_write_batch_size(2) + .set_data_page_row_count_limit(2) + .build(); + + let mut buffer = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + let data = Bytes::from(buffer); + + let builder = ParquetRecordBatchStreamBuilder::new_with_options( + TestReader::new(data.clone()), + ArrowReaderOptions::new().with_page_index(true), + ) + .await + .unwrap(); + let schema = builder.parquet_schema().clone(); + let filter_mask = ProjectionMask::leaves(&schema, [0]); + + let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { + let column = batch.column(0); + let match_first = eq(column, &Int64Array::new_scalar(first_value))?; + let match_second = eq(column, &Int64Array::new_scalar(last_value))?; + or(&match_first, &match_second) + }); + + // The batch size is set to 12 to read all rows in one go after filtering + // If the Reader chooses mask to handle filter, it might cause panic because the mid 4 pages may not be decoded. + let stream = ParquetRecordBatchStreamBuilder::new_with_options( + TestReader::new(data), + ArrowReaderOptions::new().with_page_index(true), + ) + .await + .unwrap() + .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) + .with_batch_size(12) + .build() + .unwrap(); + + // Collecting into batches validates the plan now downgrades to selectors instead of panicking. + let schema = stream.schema().clone(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + let result = concat_batches(&schema, &batches).unwrap(); + assert_eq!(result.num_rows(), 2); + } + #[tokio::test] async fn test_row_filter() { let a = StringArray::from_iter_values(["a", "b", "b", "b", "c", "c"]); diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index 4b959196d415..4545c8d60e6e 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -31,6 +31,7 @@ use crate::arrow::push_decoder::reader_builder::filter::CacheInfo; use crate::arrow::schema::ParquetField; use crate::errors::ParquetError; use crate::file::metadata::ParquetMetaData; +use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::util::push_buffers::PushBuffers; use bytes::Bytes; use data::DataRequest; @@ -491,7 +492,7 @@ impl RowGroupReaderBuilder { } // Apply any limit and offset - let plan_builder = plan_builder + let mut plan_builder = plan_builder .limited(row_count) .with_offset(self.offset) .with_limit(self.limit) @@ -531,6 +532,34 @@ impl RowGroupReaderBuilder { // so don't call with_cache_projection here .build(); + match self.selection_strategy { + RowSelectionStrategy::Auto { + threshold: _threshold, + safe_strategy, + } => { + let preferred_strategy = plan_builder.preferred_selection_strategy(); + let offset_index = self.row_group_offset_index(row_group_idx); + let force_selectors = safe_strategy + && matches!(preferred_strategy, RowSelectionStrategy::Mask) + && plan_builder.selection().is_some_and(|selection| { + selection.should_force_selectors(&self.projection, offset_index) + }); + + let resolved_strategy = if force_selectors { + RowSelectionStrategy::Selectors + } else { + preferred_strategy + }; + + plan_builder = plan_builder.with_selection_strategy(resolved_strategy); + } + _ => { + // If a non-auto strategy is specified, override any plan builder strategy + plan_builder = + plan_builder.with_selection_strategy(self.selection_strategy); + } + } + let row_group_info = RowGroupInfo { row_group_idx, row_count, @@ -657,11 +686,21 @@ impl RowGroupReaderBuilder { Some(ProjectionMask::leaves(schema, included_leaves)) } } + + /// Get the offset index for the specified row group, if any + fn row_group_offset_index(&self, row_group_idx: usize) -> Option<&[OffsetIndexMetaData]> { + self.metadata + .offset_index() + .filter(|index| !index.is_empty()) + .and_then(|index| index.get(row_group_idx)) + .map(|columns| columns.as_slice()) + } } #[cfg(test)] mod tests { use super::*; + #[test] // Verify that the size of RowGroupDecoderState does not grow too large fn test_structure_size() { From 8edc8b15f28a3db9939a1b940155001d3faedcd2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 10 Nov 2025 10:34:11 -0500 Subject: [PATCH 21/27] Minor: clean up of selection strategy code --- parquet/src/arrow/arrow_reader/mod.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 1cce3d072e39..bbd994c29c01 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -894,8 +894,6 @@ impl ParquetRecordBatchReaderBuilder { /// /// Note: this will eagerly evaluate any `RowFilter` before returning pub fn build(self) -> Result { - let selection_strategy = self.selection_strategy; - let Self { input, metadata, @@ -906,7 +904,7 @@ impl ParquetRecordBatchReaderBuilder { projection, mut filter, selection, - selection_strategy: _, + selection_strategy, limit, offset, metrics, From d27d7d7062761b3693c9b134517e5d3b51309108 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 10 Nov 2025 12:54:11 -0500 Subject: [PATCH 22/27] Another minor cleanup for the read plan builder --- parquet/src/arrow/arrow_reader/mod.rs | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 1cce3d072e39..790225b858ec 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -950,16 +950,12 @@ impl ParquetRecordBatchReaderBuilder { let array_reader = ArrayReaderBuilder::new(&reader, &metrics) .build_array_reader(fields.as_deref(), &projection)?; - let mut plan_builder = plan_builder + let read_plan = plan_builder .limited(reader.num_rows()) .with_offset(offset) .with_limit(limit) - .build_limited(); - - let preferred_strategy = plan_builder.preferred_selection_strategy(); - plan_builder = plan_builder.with_selection_strategy(preferred_strategy); - - let read_plan = plan_builder.build(); + .build_limited() + .build(); Ok(ParquetRecordBatchReader::new(array_reader, read_plan)) } From 93fc72b861ca9ff6dfd61ce4d9355c1ac830c3d8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 10 Nov 2025 14:26:51 -0500 Subject: [PATCH 23/27] Rework RowSelectionCursor to use enums --- parquet/src/arrow/arrow_reader/mod.rs | 131 ++++++------ parquet/src/arrow/arrow_reader/read_plan.rs | 51 +++-- parquet/src/arrow/arrow_reader/selection.rs | 216 ++++++++++---------- 3 files changed, 206 insertions(+), 192 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index bae44f9acacd..af5ee7db4cca 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1072,83 +1072,80 @@ impl ParquetRecordBatchReader { fn next_inner(&mut self) -> Result> { let mut read_records = 0; let batch_size = self.batch_size(); - match self.read_plan.selection_mut() { - Some(selection_cursor) => { - if selection_cursor.is_mask_backed() { - // Stream the record batch reader using contiguous segments of the selection - // mask, avoiding the need to materialize intermediate `RowSelector` ranges. - while !selection_cursor.is_empty() { - let Some(mask_chunk) = selection_cursor.next_mask_chunk(batch_size) else { - return Ok(None); - }; - - if mask_chunk.initial_skip > 0 { - let skipped = - self.array_reader.skip_records(mask_chunk.initial_skip)?; - if skipped != mask_chunk.initial_skip { - return Err(general_err!( - "failed to skip rows, expected {}, got {}", - mask_chunk.initial_skip, - skipped - )); - } - } - - if mask_chunk.chunk_rows == 0 { - if selection_cursor.is_empty() && mask_chunk.selected_rows == 0 { - return Ok(None); - } - continue; - } - - let mask = selection_cursor - .mask_values_for(&mask_chunk) - .ok_or_else(|| general_err!("row selection mask out of bounds"))?; + match self.read_plan.row_selection_cursor_mut() { + RowSelectionCursor::Mask(mask_cursor) => { + // Stream the record batch reader using contiguous segments of the selection + // mask, avoiding the need to materialize intermediate `RowSelector` ranges. + while !mask_cursor.is_empty() { + let Some(mask_chunk) = mask_cursor.next_mask_chunk(batch_size) else { + return Ok(None); + }; - let read = self.array_reader.read_records(mask_chunk.chunk_rows)?; - if read == 0 { + if mask_chunk.initial_skip > 0 { + let skipped = self.array_reader.skip_records(mask_chunk.initial_skip)?; + if skipped != mask_chunk.initial_skip { return Err(general_err!( - "reached end of column while expecting {} rows", - mask_chunk.chunk_rows + "failed to skip rows, expected {}, got {}", + mask_chunk.initial_skip, + skipped )); } - if read != mask_chunk.chunk_rows { - return Err(general_err!( - "insufficient rows read from array reader - expected {}, got {}", - mask_chunk.chunk_rows, - read - )); + } + + if mask_chunk.chunk_rows == 0 { + if mask_cursor.is_empty() && mask_chunk.selected_rows == 0 { + return Ok(None); } + continue; + } - let array = self.array_reader.consume_batch()?; - // The column reader exposes the projection as a struct array; convert this - // into a record batch before applying the boolean filter mask. - let struct_array = array.as_struct_opt().ok_or_else(|| { - ArrowError::ParquetError( - "Struct array reader should return struct array".to_string(), - ) - })?; + let mask = mask_cursor.mask_values_for(&mask_chunk)?; - let filtered_batch = - filter_record_batch(&RecordBatch::from(struct_array), &mask)?; + let read = self.array_reader.read_records(mask_chunk.chunk_rows)?; + if read == 0 { + return Err(general_err!( + "reached end of column while expecting {} rows", + mask_chunk.chunk_rows + )); + } + if read != mask_chunk.chunk_rows { + return Err(general_err!( + "insufficient rows read from array reader - expected {}, got {}", + mask_chunk.chunk_rows, + read + )); + } - if filtered_batch.num_rows() != mask_chunk.selected_rows { - return Err(general_err!( - "filtered rows mismatch selection - expected {}, got {}", - mask_chunk.selected_rows, - filtered_batch.num_rows() - )); - } + let array = self.array_reader.consume_batch()?; + // The column reader exposes the projection as a struct array; convert this + // into a record batch before applying the boolean filter mask. + let struct_array = array.as_struct_opt().ok_or_else(|| { + ArrowError::ParquetError( + "Struct array reader should return struct array".to_string(), + ) + })?; - if filtered_batch.num_rows() == 0 { - continue; - } + let filtered_batch = + filter_record_batch(&RecordBatch::from(struct_array), &mask)?; + + if filtered_batch.num_rows() != mask_chunk.selected_rows { + return Err(general_err!( + "filtered rows mismatch selection - expected {}, got {}", + mask_chunk.selected_rows, + filtered_batch.num_rows() + )); + } - return Ok(Some(filtered_batch)); + if filtered_batch.num_rows() == 0 { + continue; } + + return Ok(Some(filtered_batch)); } - while read_records < batch_size && !selection_cursor.is_empty() { - let front = selection_cursor.next_selector(); + } + RowSelectionCursor::Selectors(selectors_cursor) => { + while read_records < batch_size && !selectors_cursor.is_empty() { + let front = selectors_cursor.next_selector(); if front.skip { let skipped = self.array_reader.skip_records(front.row_count)?; @@ -1174,7 +1171,7 @@ impl ParquetRecordBatchReader { Some(remaining) if remaining != 0 => { // if page row count less than batch_size we must set batch size to page row count. // add check avoid dead loop - selection_cursor.return_selector(RowSelector::select(remaining)); + selectors_cursor.return_selector(RowSelector::select(remaining)); need_read } _ => front.row_count, @@ -1185,7 +1182,7 @@ impl ParquetRecordBatchReader { }; } } - None => { + RowSelectionCursor::All => { self.array_reader.read_records(batch_size)?; } }; diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 5b5191115f80..d0cbe434c07d 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -26,6 +26,7 @@ use crate::arrow::arrow_reader::{ use crate::errors::{ParquetError, Result}; use arrow_array::Array; use arrow_select::filter::prep_null_mask_filter; +use std::collections::VecDeque; /// A builder for [`ReadPlan`] #[derive(Clone, Debug)] @@ -89,6 +90,8 @@ impl ReadPlanBuilder { } /// Returns the preferred [`RowSelectionStrategy`] for materialising the current selection. + /// + /// Guarantees to return either `Selectors` or `Mask`, never `Auto`. pub fn preferred_selection_strategy(&self) -> RowSelectionStrategy { match self.selection_strategy { RowSelectionStrategy::Selectors => RowSelectionStrategy::Selectors, @@ -167,25 +170,35 @@ impl ReadPlanBuilder { if !self.selects_any() { self.selection = Some(RowSelection::from(vec![])); } - let selection_strategy = match self.selection_strategy { - RowSelectionStrategy::Auto { .. } => self.preferred_selection_strategy(), - strategy => strategy, - }; + + // Preferred strategy must not be Auto + let selection_strategy = self.preferred_selection_strategy(); + let Self { batch_size, selection, selection_strategy: _, } = self; - let selection = selection.map(|s| { - let trimmed = s.trim(); - let selectors: Vec = trimmed.into(); - RowSelectionCursor::new(selectors, selection_strategy) - }); + let selection = selection.map(|s| s.trim()); + + let row_selection_cursor = selection + .map(|s| { + let trimmed = s.trim(); + let selectors: Vec = trimmed.into(); + match selection_strategy { + RowSelectionStrategy::Mask => { + RowSelectionCursor::new_mask_from_selectors(selectors) + } + RowSelectionStrategy::Selectors => RowSelectionCursor::new_selectors(selectors), + RowSelectionStrategy::Auto { .. } => unreachable!(), + } + }) + .unwrap_or(RowSelectionCursor::new_all()); ReadPlan { batch_size, - selection, + row_selection_cursor, } } } @@ -283,13 +296,23 @@ pub struct ReadPlan { /// The number of rows to read in each batch batch_size: usize, /// Row ranges to be selected from the data source - selection: Option, + row_selection_cursor: RowSelectionCursor, } impl ReadPlan { - /// Returns a mutable reference to the selection, if any - pub fn selection_mut(&mut self) -> Option<&mut RowSelectionCursor> { - self.selection.as_mut() + /// Returns a mutable reference to the selection selectors, if any + #[deprecated(since = "57.1.0", note = "Use `row_selection_cursor_mut` instead")] + pub fn selection_mut(&mut self) -> Option<&mut VecDeque> { + if let RowSelectionCursor::Selectors(selectors_cursor) = &mut self.row_selection_cursor { + Some(selectors_cursor.selectors_mut()) + } else { + None + } + } + + /// Returns a mutable reference to the row selection cursor + pub fn row_selection_cursor_mut(&mut self) -> &mut RowSelectionCursor { + &mut self.row_selection_cursor } /// Return the number of rows to read in each output batch diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index ff975e2bc42d..5fcf494454fd 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use crate::arrow::ProjectionMask; +use crate::errors::ParquetError; +use crate::file::page_index::offset_index::{OffsetIndexMetaData, PageLocation}; use arrow_array::{Array, BooleanArray}; use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; use arrow_select::filter::SlicesIterator; @@ -22,9 +25,6 @@ use std::cmp::Ordering; use std::collections::VecDeque; use std::ops::Range; -use crate::arrow::ProjectionMask; -use crate::file::page_index::offset_index::{OffsetIndexMetaData, PageLocation}; - /// Strategy for materialising [`RowSelection`] during execution. #[derive(Clone, Copy, Debug, Eq, PartialEq)] pub enum RowSelectionStrategy { @@ -751,116 +751,27 @@ fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelec iter.collect() } -/// Cursor for iterating a [`RowSelection`] during execution within a -/// [`ReadPlan`](crate::arrow::arrow_reader::ReadPlan). +/// Cursor for iterating a mask-backed [`RowSelection`] /// -/// This keeps per-reader state such as the current position and delegates the -/// actual storage strategy to the internal `RowSelectionBacking`. +/// This is best for dense selections where there are many small skips +/// or selections. For example, selecting every other row. #[derive(Debug)] -pub struct RowSelectionCursor { - /// Backing storage describing how the selection is materialised - storage: RowSelectionBacking, +pub struct MaskCursor { + mask: BooleanBuffer, /// Current absolute offset into the selection position: usize, } -/// Backing storage that powers [`RowSelectionCursor`]. -/// -/// The cursor either walks a boolean mask (dense representation) or a queue -/// of [`RowSelector`] ranges (sparse representation). -#[derive(Debug)] -pub enum RowSelectionBacking { - Mask(BooleanBuffer), - Selectors(VecDeque), -} - -/// Result of computing the next chunk to read when using a bitmap mask -#[derive(Debug)] -pub struct MaskChunk { - /// Number of leading rows to skip before reaching selected rows - pub initial_skip: usize, - /// Total rows covered by this chunk (selected + skipped) - pub chunk_rows: usize, - /// Rows actually selected within the chunk - pub selected_rows: usize, - /// Starting offset within the mask where the chunk begins - pub mask_start: usize, -} - -impl RowSelectionCursor { - /// Create a cursor, choosing an efficient backing representation - pub(crate) fn new(selectors: Vec, strategy: RowSelectionStrategy) -> Self { - let storage = match strategy { - RowSelectionStrategy::Mask => { - RowSelectionBacking::Mask(boolean_mask_from_selectors(&selectors)) - } - RowSelectionStrategy::Selectors => RowSelectionBacking::Selectors(selectors.into()), - RowSelectionStrategy::Auto { .. } => { - panic!("RowSelectionStrategy::Auto must be resolved before creating cursor") - } - }; - - Self { - storage, - position: 0, - } - } - +impl MaskCursor { /// Returns `true` when no further rows remain pub fn is_empty(&self) -> bool { - match &self.storage { - RowSelectionBacking::Mask(mask) => self.position >= mask.len(), - RowSelectionBacking::Selectors(selectors) => selectors.is_empty(), - } - } - - /// Current position within the overall selection - pub fn position(&self) -> usize { - self.position - } - - /// Return the next [`RowSelector`] when using the sparse representation - pub fn next_selector(&mut self) -> RowSelector { - match &mut self.storage { - RowSelectionBacking::Selectors(selectors) => { - let selector = selectors.pop_front().unwrap(); - self.position += selector.row_count; - selector - } - RowSelectionBacking::Mask(_) => { - unreachable!("next_selector called for mask-based RowSelectionCursor") - } - } - } - - /// Return a selector to the front, rewinding the position (sparse-only) - pub fn return_selector(&mut self, selector: RowSelector) { - match &mut self.storage { - RowSelectionBacking::Selectors(selectors) => { - self.position = self.position.saturating_sub(selector.row_count); - selectors.push_front(selector); - } - RowSelectionBacking::Mask(_) => { - unreachable!("return_selector called for mask-based RowSelectionCursor") - } - } - } - - /// Returns `true` if the cursor is backed by a boolean mask - pub fn is_mask_backed(&self) -> bool { - matches!(self.storage, RowSelectionBacking::Mask(_)) + self.position >= self.mask.len() } /// Advance through the mask representation, producing the next chunk summary pub fn next_mask_chunk(&mut self, batch_size: usize) -> Option { - if !self.is_mask_backed() { - unreachable!("next_mask_chunk called for selector-based RowSelectionCursor") - } let (initial_skip, chunk_rows, selected_rows, mask_start, end_position) = { - let mask = match &self.storage { - RowSelectionBacking::Mask(mask) => mask, - RowSelectionBacking::Selectors(_) => return None, - }; + let mask = &self.mask; if self.position >= mask.len() { return None; @@ -904,18 +815,101 @@ impl RowSelectionCursor { } /// Materialise the boolean values for a mask-backed chunk - pub fn mask_values_for(&self, chunk: &MaskChunk) -> Option { - match &self.storage { - RowSelectionBacking::Mask(mask) => { - if chunk.mask_start.saturating_add(chunk.chunk_rows) > mask.len() { - return None; - } - Some(BooleanArray::from( - mask.slice(chunk.mask_start, chunk.chunk_rows), - )) - } - RowSelectionBacking::Selectors(_) => None, + pub fn mask_values_for(&self, chunk: &MaskChunk) -> Result { + if chunk.mask_start.saturating_add(chunk.chunk_rows) > self.mask.len() { + return Err(ParquetError::General( + "Internal Error: MaskChunk exceeds mask length".to_string(), + )); } + Ok(BooleanArray::from( + self.mask.slice(chunk.mask_start, chunk.chunk_rows), + )) + } +} + +/// Cursor for iterating a selector-backed [`RowSelection`] +/// +/// This is best for sparse selections where large contiguous +/// blocks of rows are selected or skipped. +#[derive(Debug)] +pub struct SelectorsCursor { + selectors: VecDeque, + /// Current absolute offset into the selection + position: usize, +} + +impl SelectorsCursor { + /// Returns `true` when no further rows remain + pub fn is_empty(&self) -> bool { + self.selectors.is_empty() + } + + pub(crate) fn selectors_mut(&mut self) -> &mut VecDeque { + &mut self.selectors + } + + /// Return the next [`RowSelector`] + pub(crate) fn next_selector(&mut self) -> RowSelector { + let selector = self.selectors.pop_front().unwrap(); + self.position += selector.row_count; + selector + } + + /// Return a selector to the front, rewinding the position + pub(crate) fn return_selector(&mut self, selector: RowSelector) { + self.position = self.position.saturating_sub(selector.row_count); + self.selectors.push_front(selector); + } +} + +/// Result of computing the next chunk to read when using a [`MaskCursor`] +#[derive(Debug)] +pub struct MaskChunk { + /// Number of leading rows to skip before reaching selected rows + pub initial_skip: usize, + /// Total rows covered by this chunk (selected + skipped) + pub chunk_rows: usize, + /// Rows actually selected within the chunk + pub selected_rows: usize, + /// Starting offset within the mask where the chunk begins + pub mask_start: usize, +} + +/// Cursor for iterating a [`RowSelection`] during execution within a +/// [`ReadPlan`](crate::arrow::arrow_reader::ReadPlan). +/// +/// This keeps per-reader state such as the current position and delegates the +/// actual storage strategy to the internal `RowSelectionBacking`. +#[derive(Debug)] +pub enum RowSelectionCursor { + /// Reading all rows + All, + /// Use a bitmask to back the selection (dense selections) + Mask(MaskCursor), + /// Use a queue of selectors to back the selection (sparse selections) + Selectors(SelectorsCursor), +} + +impl RowSelectionCursor { + /// Create a [`MaskCursor`] cursor backed by a bitmask, from an existing set of selectors + pub(crate) fn new_mask_from_selectors(selectors: Vec) -> Self { + Self::Mask(MaskCursor { + mask: boolean_mask_from_selectors(&selectors), + position: 0, + }) + } + + /// Create a [`RowSelectionCursor::Selectors`] from the provided selectors + pub(crate) fn new_selectors(selectors: Vec) -> Self { + Self::Selectors(SelectorsCursor { + selectors: selectors.into(), + position: 0, + }) + } + + /// Create a cursor that selects all rows + pub(crate) fn new_all() -> Self { + Self::All } } From 631cb1780677dc45a372a45781ffbafc9b028810 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 10 Nov 2025 15:30:18 -0500 Subject: [PATCH 24/27] Split RowSelectionPolicy from RowSelectionStrategy --- parquet/benches/row_selection_state.rs | 19 ++-- parquet/src/arrow/arrow_reader/mod.rs | 20 ++-- parquet/src/arrow/arrow_reader/read_plan.rs | 49 ++++++---- parquet/src/arrow/arrow_reader/selection.rs | 21 +++- parquet/src/arrow/async_reader/mod.rs | 4 +- parquet/src/arrow/push_decoder/mod.rs | 5 +- .../arrow/push_decoder/reader_builder/mod.rs | 98 +++++++++++++------ 7 files changed, 134 insertions(+), 82 deletions(-) diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_state.rs index 893f1c382cc8..49c9e6d68acf 100644 --- a/parquet/benches/row_selection_state.rs +++ b/parquet/benches/row_selection_state.rs @@ -25,7 +25,7 @@ use bytes::Bytes; use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use parquet::arrow::ArrowWriter; use parquet::arrow::arrow_reader::{ - ParquetRecordBatchReaderBuilder, RowSelection, RowSelectionStrategy, RowSelector, + ParquetRecordBatchReaderBuilder, RowSelection, RowSelectionPolicy, RowSelector, }; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -239,8 +239,7 @@ fn bench_over_lengths( &bench_input, |b, input| { b.iter(|| { - let total = - run_read(&input.parquet_data, &input.selection, mode.strategy()); + let total = run_read(&input.parquet_data, &input.selection, mode.policy()); hint::black_box(total); }); }, @@ -257,16 +256,12 @@ struct BenchInput { selection: RowSelection, } -fn run_read( - parquet_data: &Bytes, - selection: &RowSelection, - strategy: RowSelectionStrategy, -) -> usize { +fn run_read(parquet_data: &Bytes, selection: &RowSelection, policy: RowSelectionPolicy) -> usize { let reader = ParquetRecordBatchReaderBuilder::try_new(parquet_data.clone()) .unwrap() .with_batch_size(BATCH_SIZE) .with_row_selection(selection.clone()) - .with_row_selection_strategy(strategy) + .with_row_selection_policy(policy) .build() .unwrap(); @@ -465,10 +460,10 @@ impl BenchMode { } } - fn strategy(self) -> RowSelectionStrategy { + fn policy(self) -> RowSelectionPolicy { match self { - BenchMode::ReadSelector => RowSelectionStrategy::Selectors, - BenchMode::ReadMask => RowSelectionStrategy::Mask, + BenchMode::ReadSelector => RowSelectionPolicy::Selectors, + BenchMode::ReadMask => RowSelectionPolicy::Mask, } } } diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index af5ee7db4cca..6667afa0f5b9 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -22,7 +22,7 @@ use arrow_array::{Array, RecordBatch, RecordBatchReader}; use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; use arrow_select::filter::filter_record_batch; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; -pub use selection::{RowSelection, RowSelectionCursor, RowSelectionStrategy, RowSelector}; +pub use selection::{RowSelection, RowSelectionCursor, RowSelectionPolicy, RowSelector}; use std::fmt::{Debug, Formatter}; use std::sync::Arc; @@ -49,7 +49,7 @@ pub use read_plan::{ReadPlan, ReadPlanBuilder}; mod filter; pub mod metrics; mod read_plan; -mod selection; +pub(crate) mod selection; pub mod statistics; /// Builder for constructing Parquet readers that decode into [Apache Arrow] @@ -125,7 +125,7 @@ pub struct ArrowReaderBuilder { pub(crate) selection: Option, - pub(crate) selection_strategy: RowSelectionStrategy, + pub(crate) row_selection_policy: RowSelectionPolicy, pub(crate) limit: Option, @@ -148,7 +148,7 @@ impl Debug for ArrowReaderBuilder { .field("projection", &self.projection) .field("filter", &self.filter) .field("selection", &self.selection) - .field("selection_strategy", &self.selection_strategy) + .field("row_selection_policy", &self.row_selection_policy) .field("limit", &self.limit) .field("offset", &self.offset) .field("metrics", &self.metrics) @@ -168,7 +168,7 @@ impl ArrowReaderBuilder { projection: ProjectionMask::all(), filter: None, selection: None, - selection_strategy: RowSelectionStrategy::default(), + row_selection_policy: RowSelectionPolicy::default(), limit: None, offset: None, metrics: ArrowReaderMetrics::Disabled, @@ -218,9 +218,11 @@ impl ArrowReaderBuilder { } /// Configure how row selections should be materialised during execution - pub fn with_row_selection_strategy(self, strategy: RowSelectionStrategy) -> Self { + /// + /// See [`RowSelectionPolicy`] for more details + pub fn with_row_selection_policy(self, policy: RowSelectionPolicy) -> Self { Self { - selection_strategy: strategy, + row_selection_policy: policy, ..self } } @@ -904,7 +906,7 @@ impl ParquetRecordBatchReaderBuilder { projection, mut filter, selection, - selection_strategy, + row_selection_policy, limit, offset, metrics, @@ -925,7 +927,7 @@ impl ParquetRecordBatchReaderBuilder { let mut plan_builder = ReadPlanBuilder::new(batch_size) .with_selection(selection) - .with_selection_strategy(selection_strategy); + .with_row_selection_policy(row_selection_policy); // Update selection based on any filters if let Some(filter) = filter.as_mut() { diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index d0cbe434c07d..23f2e1ce6913 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -19,9 +19,10 @@ //! from a Parquet file use crate::arrow::array_reader::ArrayReader; +use crate::arrow::arrow_reader::selection::RowSelectionPolicy; +use crate::arrow::arrow_reader::selection::RowSelectionStrategy; use crate::arrow::arrow_reader::{ - ArrowPredicate, ParquetRecordBatchReader, RowSelection, RowSelectionCursor, - RowSelectionStrategy, RowSelector, + ArrowPredicate, ParquetRecordBatchReader, RowSelection, RowSelectionCursor, RowSelector, }; use crate::errors::{ParquetError, Result}; use arrow_array::Array; @@ -32,10 +33,10 @@ use std::collections::VecDeque; #[derive(Clone, Debug)] pub struct ReadPlanBuilder { batch_size: usize, - /// Current to apply, includes all filters + /// Which rows to select. Includes the result of all filters applied so far selection: Option, - /// Strategy to use when materialising the row selection - selection_strategy: RowSelectionStrategy, + /// Policy to use when materializing the row selection + row_selection_policy: RowSelectionPolicy, } impl ReadPlanBuilder { @@ -44,7 +45,7 @@ impl ReadPlanBuilder { Self { batch_size, selection: None, - selection_strategy: RowSelectionStrategy::default(), + row_selection_policy: RowSelectionPolicy::default(), } } @@ -54,12 +55,19 @@ impl ReadPlanBuilder { self } - /// Configure the strategy to use when materialising the [`RowSelection`] - pub fn with_selection_strategy(mut self, strategy: RowSelectionStrategy) -> Self { - self.selection_strategy = strategy; + /// Configure the policy to use when materialising the [`RowSelection`] + /// + /// Defaults to [`RowSelectionPolicy::Auto`] + pub fn with_row_selection_policy(mut self, policy: RowSelectionPolicy) -> Self { + self.row_selection_policy = policy; self } + /// Returns the current row selection policy + pub fn row_selection_policy(&self) -> &RowSelectionPolicy { + &self.row_selection_policy + } + /// Returns the current selection, if any pub fn selection(&self) -> Option<&RowSelection> { self.selection.as_ref() @@ -89,14 +97,14 @@ impl ReadPlanBuilder { self.selection.as_ref().map(|s| s.row_count()) } - /// Returns the preferred [`RowSelectionStrategy`] for materialising the current selection. + /// Returns the [`RowSelectionStrategy`] for this plan. /// /// Guarantees to return either `Selectors` or `Mask`, never `Auto`. - pub fn preferred_selection_strategy(&self) -> RowSelectionStrategy { - match self.selection_strategy { - RowSelectionStrategy::Selectors => RowSelectionStrategy::Selectors, - RowSelectionStrategy::Mask => RowSelectionStrategy::Mask, - RowSelectionStrategy::Auto { threshold, .. } => { + pub(crate) fn resolve_selection_strategy(&self) -> RowSelectionStrategy { + match self.row_selection_policy { + RowSelectionPolicy::Selectors => RowSelectionStrategy::Selectors, + RowSelectionPolicy::Mask => RowSelectionStrategy::Mask, + RowSelectionPolicy::Auto { threshold, .. } => { let selection = match self.selection.as_ref() { Some(selection) => selection, None => return RowSelectionStrategy::Selectors, @@ -172,12 +180,12 @@ impl ReadPlanBuilder { } // Preferred strategy must not be Auto - let selection_strategy = self.preferred_selection_strategy(); + let selection_strategy = self.resolve_selection_strategy(); let Self { batch_size, selection, - selection_strategy: _, + row_selection_policy: _, } = self; let selection = selection.map(|s| s.trim()); @@ -191,7 +199,6 @@ impl ReadPlanBuilder { RowSelectionCursor::new_mask_from_selectors(selectors) } RowSelectionStrategy::Selectors => RowSelectionCursor::new_selectors(selectors), - RowSelectionStrategy::Auto { .. } => unreachable!(), } }) .unwrap_or(RowSelectionCursor::new_all()); @@ -335,7 +342,7 @@ mod tests { let selection = RowSelection::from(vec![RowSelector::select(8)]); let builder = builder_with_selection(selection); assert_eq!( - builder.preferred_selection_strategy(), + builder.resolve_selection_strategy(), RowSelectionStrategy::Mask ); } @@ -344,12 +351,12 @@ mod tests { fn preferred_selection_strategy_prefers_selectors_when_threshold_small() { let selection = RowSelection::from(vec![RowSelector::select(8)]); let builder = - builder_with_selection(selection).with_selection_strategy(RowSelectionStrategy::Auto { + builder_with_selection(selection).with_row_selection_policy(RowSelectionPolicy::Auto { threshold: 1, safe_strategy: true, }); assert_eq!( - builder.preferred_selection_strategy(), + builder.resolve_selection_strategy(), RowSelectionStrategy::Selectors ); } diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 5fcf494454fd..475f49343df5 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -25,9 +25,12 @@ use std::cmp::Ordering; use std::collections::VecDeque; use std::ops::Range; -/// Strategy for materialising [`RowSelection`] during execution. +/// Policy for picking a strategy to materialise [`RowSelection`] during execution. +/// +/// Note that this is a user-provided preference, and the actual strategy used +/// may differ based on safety considerations (e.g. page skipping). #[derive(Clone, Copy, Debug, Eq, PartialEq)] -pub enum RowSelectionStrategy { +pub enum RowSelectionPolicy { /// Use a queue of [`RowSelector`] values Selectors, /// Use a boolean mask to materialise the selection @@ -41,7 +44,7 @@ pub enum RowSelectionStrategy { }, } -impl Default for RowSelectionStrategy { +impl Default for RowSelectionPolicy { fn default() -> Self { Self::Auto { threshold: 32, @@ -50,6 +53,18 @@ impl Default for RowSelectionStrategy { } } +/// Fully resolved strategy for materializing [`RowSelection`] during execution. +/// +/// This is determined from a combination of user preference (via [`RowSelectionPolicy`]) +/// and safety considerations (e.g. page skipping). +#[derive(Clone, Copy, Debug, Eq, PartialEq)] +pub(crate) enum RowSelectionStrategy { + /// Use a queue of [`RowSelector`] values + Selectors, + /// Use a boolean mask to materialise the selection + Mask, +} + /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when /// scanning a parquet file #[derive(Debug, Clone, Copy, Eq, PartialEq)] diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 9f2b22ed29d7..d990361d21bc 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -486,7 +486,7 @@ impl ParquetRecordBatchStreamBuilder { projection, filter, selection, - selection_strategy, + row_selection_policy: selection_strategy, limit, offset, metrics, @@ -508,7 +508,7 @@ impl ParquetRecordBatchStreamBuilder { projection, filter, selection, - selection_strategy, + row_selection_policy: selection_strategy, batch_size, row_groups, limit, diff --git a/parquet/src/arrow/push_decoder/mod.rs b/parquet/src/arrow/push_decoder/mod.rs index c5d09b5e7e83..9b3893258ba5 100644 --- a/parquet/src/arrow/push_decoder/mod.rs +++ b/parquet/src/arrow/push_decoder/mod.rs @@ -162,7 +162,6 @@ impl ParquetPushDecoderBuilder { /// Create a [`ParquetPushDecoder`] with the configured options pub fn build(self) -> Result { - let selection_strategy = self.selection_strategy; let Self { input: NoInput, metadata: parquet_metadata, @@ -176,7 +175,7 @@ impl ParquetPushDecoderBuilder { limit, offset, metrics, - selection_strategy: _, + row_selection_policy, max_predicate_cache_size, } = self; @@ -198,7 +197,7 @@ impl ParquetPushDecoderBuilder { metrics, max_predicate_cache_size, buffers, - selection_strategy, + row_selection_policy, ); // Initialize the decoder with the configured options diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index 4545c8d60e6e..5fe473708a73 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -22,8 +22,9 @@ use crate::DecodeResult; use crate::arrow::ProjectionMask; use crate::arrow::array_reader::{ArrayReaderBuilder, RowGroupCache}; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; +use crate::arrow::arrow_reader::selection::RowSelectionStrategy; use crate::arrow::arrow_reader::{ - ParquetRecordBatchReader, ReadPlanBuilder, RowFilter, RowSelection, RowSelectionStrategy, + ParquetRecordBatchReader, ReadPlanBuilder, RowFilter, RowSelection, RowSelectionPolicy, }; use crate::arrow::in_memory_row_group::ColumnChunkData; use crate::arrow::push_decoder::reader_builder::data::DataRequestBuilder; @@ -157,7 +158,7 @@ pub(crate) struct RowGroupReaderBuilder { metrics: ArrowReaderMetrics, /// Strategy for materialising row selections - selection_strategy: RowSelectionStrategy, + row_selection_policy: RowSelectionPolicy, /// Current state of the decoder. /// @@ -183,7 +184,7 @@ impl RowGroupReaderBuilder { metrics: ArrowReaderMetrics, max_predicate_cache_size: usize, buffers: PushBuffers, - selection_strategy: RowSelectionStrategy, + row_selection_policy: RowSelectionPolicy, ) -> Self { Self { batch_size, @@ -195,7 +196,7 @@ impl RowGroupReaderBuilder { offset, metrics, max_predicate_cache_size, - selection_strategy, + row_selection_policy, state: Some(RowGroupDecoderState::Finished), buffers, } @@ -241,7 +242,7 @@ impl RowGroupReaderBuilder { } let plan_builder = ReadPlanBuilder::new(self.batch_size) .with_selection(selection) - .with_selection_strategy(self.selection_strategy); + .with_row_selection_policy(self.row_selection_policy); let row_group_info = RowGroupInfo { row_group_idx, @@ -532,33 +533,13 @@ impl RowGroupReaderBuilder { // so don't call with_cache_projection here .build(); - match self.selection_strategy { - RowSelectionStrategy::Auto { - threshold: _threshold, - safe_strategy, - } => { - let preferred_strategy = plan_builder.preferred_selection_strategy(); - let offset_index = self.row_group_offset_index(row_group_idx); - let force_selectors = safe_strategy - && matches!(preferred_strategy, RowSelectionStrategy::Mask) - && plan_builder.selection().is_some_and(|selection| { - selection.should_force_selectors(&self.projection, offset_index) - }); - - let resolved_strategy = if force_selectors { - RowSelectionStrategy::Selectors - } else { - preferred_strategy - }; - - plan_builder = plan_builder.with_selection_strategy(resolved_strategy); - } - _ => { - // If a non-auto strategy is specified, override any plan builder strategy - plan_builder = - plan_builder.with_selection_strategy(self.selection_strategy); - } - } + plan_builder = plan_builder.with_row_selection_policy(self.row_selection_policy); + + plan_builder = overide_selector_strategy_if_needed( + plan_builder, + &self.projection, + self.row_group_offset_index(row_group_idx), + ); let row_group_info = RowGroupInfo { row_group_idx, @@ -697,6 +678,59 @@ impl RowGroupReaderBuilder { } } +/// Overrider the selection strategy if needed. +/// +/// Some pages can be skipped during row-group construction of they are not read +/// by the selections. This means that the data pages for those rows are never +/// loaded and definition/repetition levels are never read. When using +/// `RowSelections` selection works because skip_records() handles this +/// case and skips the page accordingly. +/// +/// However, with the current Mask design, all values must be read and decoded +/// and then a Mask filter is applied filtering. Thus if any pages are skipped +/// during row-group construction, the data pages are missing and cannot be +/// decoded. +/// +/// A simple example: +/// * the page size is 2, the mask is 100001, row selection should be read(1) skip(4) read(1) +/// * the ColumnChunkData would be page1(10), page2(skipped), page3(01) +/// +/// Using the rowselection to skip(4), page2 won't be read at all, so in this +/// case we can't decode all the rows and apply a mask. To correctly apply the +/// bit mask, we need all 6 value be read, but the page2 is not in memory. +fn overide_selector_strategy_if_needed( + plan_builder: ReadPlanBuilder, + projection_mask: &ProjectionMask, + offset_index: Option<&[OffsetIndexMetaData]>, +) -> ReadPlanBuilder { + // override only applies to Auto policy + let RowSelectionPolicy::Auto { safe_strategy, .. } = plan_builder.row_selection_policy() else { + return plan_builder; + }; + + let preferred_strategy = plan_builder.resolve_selection_strategy(); + + let force_selectors = *safe_strategy + && matches!(preferred_strategy, RowSelectionStrategy::Mask) + && plan_builder.selection().is_some_and(|selection| { + selection.should_force_selectors(projection_mask, offset_index) + }); + + let resolved_strategy = if force_selectors { + RowSelectionStrategy::Selectors + } else { + preferred_strategy + }; + + // override the plan builder strategy with the resolved one + let new_policy = match resolved_strategy { + RowSelectionStrategy::Mask => RowSelectionPolicy::Mask, + RowSelectionStrategy::Selectors => RowSelectionPolicy::Selectors, + }; + + plan_builder.with_row_selection_policy(new_policy) +} + #[cfg(test)] mod tests { use super::*; From c30dca7b4f3986178202789b2237f0edb59cab5d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 10 Nov 2025 15:47:50 -0500 Subject: [PATCH 25/27] Remove safe flag --- parquet/src/arrow/arrow_reader/read_plan.rs | 7 ++----- parquet/src/arrow/arrow_reader/selection.rs | 7 +------ parquet/src/arrow/push_decoder/reader_builder/mod.rs | 5 ++--- 3 files changed, 5 insertions(+), 14 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/read_plan.rs b/parquet/src/arrow/arrow_reader/read_plan.rs index 23f2e1ce6913..3c17a358f084 100644 --- a/parquet/src/arrow/arrow_reader/read_plan.rs +++ b/parquet/src/arrow/arrow_reader/read_plan.rs @@ -350,11 +350,8 @@ mod tests { #[test] fn preferred_selection_strategy_prefers_selectors_when_threshold_small() { let selection = RowSelection::from(vec![RowSelector::select(8)]); - let builder = - builder_with_selection(selection).with_row_selection_policy(RowSelectionPolicy::Auto { - threshold: 1, - safe_strategy: true, - }); + let builder = builder_with_selection(selection) + .with_row_selection_policy(RowSelectionPolicy::Auto { threshold: 1 }); assert_eq!( builder.resolve_selection_strategy(), RowSelectionStrategy::Selectors diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 475f49343df5..2ddf812f9c39 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -39,17 +39,12 @@ pub enum RowSelectionPolicy { Auto { /// Average selector length below which masks are preferred threshold: usize, - /// Fallback to selectors when mask would be unsafe (e.g. page skipping) - safe_strategy: bool, }, } impl Default for RowSelectionPolicy { fn default() -> Self { - Self::Auto { - threshold: 32, - safe_strategy: true, - } + Self::Auto { threshold: 32 } } } diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index 5fe473708a73..cd4a6fa38892 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -704,14 +704,13 @@ fn overide_selector_strategy_if_needed( offset_index: Option<&[OffsetIndexMetaData]>, ) -> ReadPlanBuilder { // override only applies to Auto policy - let RowSelectionPolicy::Auto { safe_strategy, .. } = plan_builder.row_selection_policy() else { + let RowSelectionPolicy::Auto { .. } = plan_builder.row_selection_policy() else { return plan_builder; }; let preferred_strategy = plan_builder.resolve_selection_strategy(); - let force_selectors = *safe_strategy - && matches!(preferred_strategy, RowSelectionStrategy::Mask) + let force_selectors = matches!(preferred_strategy, RowSelectionStrategy::Mask) && plan_builder.selection().is_some_and(|selection| { selection.should_force_selectors(projection_mask, offset_index) }); From e2da9fe6a03a010ec235832f66bef3653bceb2aa Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Tue, 11 Nov 2025 09:00:28 +0800 Subject: [PATCH 26/27] Rename the benchmark name to cursor --- parquet/Cargo.toml | 2 +- .../benches/{row_selection_state.rs => row_selection_cursor.rs} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename parquet/benches/{row_selection_state.rs => row_selection_cursor.rs} (100%) diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index 5ce43382fe43..c447c4f2b145 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -261,7 +261,7 @@ harness = false required-features = ["arrow"] [[bench]] -name = "row_selection_state" +name = "row_selection_cursor" harness = false required-features = ["arrow"] diff --git a/parquet/benches/row_selection_state.rs b/parquet/benches/row_selection_cursor.rs similarity index 100% rename from parquet/benches/row_selection_state.rs rename to parquet/benches/row_selection_cursor.rs From 2566c26bdc24d77dd7839c0d91d4620f21adfb71 Mon Sep 17 00:00:00 2001 From: Qiwei Huang Date: Tue, 11 Nov 2025 19:33:12 +0800 Subject: [PATCH 27/27] Some miner code clean up. Refine error message when user hit error with Mask --- parquet/src/arrow/arrow_reader/mod.rs | 24 ++++++++++++------ parquet/src/arrow/async_reader/mod.rs | 21 ++++++++++------ parquet/src/arrow/in_memory_row_group.rs | 4 ++- .../arrow/push_decoder/reader_builder/mod.rs | 25 +++++++++---------- 4 files changed, 44 insertions(+), 30 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 6667afa0f5b9..b15e402fa7aa 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1074,6 +1074,9 @@ impl ParquetRecordBatchReader { fn next_inner(&mut self) -> Result> { let mut read_records = 0; let batch_size = self.batch_size(); + if batch_size == 0 { + return Ok(None); + } match self.read_plan.row_selection_cursor_mut() { RowSelectionCursor::Mask(mask_cursor) => { // Stream the record batch reader using contiguous segments of the selection @@ -1282,7 +1285,7 @@ mod tests { use crate::arrow::arrow_reader::{ ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader, - ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, + ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelectionPolicy, RowSelector, }; use crate::arrow::schema::add_encoded_arrow_schema_to_metadata; use crate::arrow::{ArrowWriter, ProjectionMask}; @@ -5211,19 +5214,24 @@ mod tests { let schema = builder.parquet_schema().clone(); let filter_mask = ProjectionMask::leaves(&schema, [0]); + let make_predicate = |mask: ProjectionMask| { + ArrowPredicateFn::new(mask, move |batch: RecordBatch| { + let column = batch.column(0); + let match_first = eq(column, &Int64Array::new_scalar(first_value))?; + let match_second = eq(column, &Int64Array::new_scalar(last_value))?; + or(&match_first, &match_second) + }) + }; + let options = ArrowReaderOptions::new().with_page_index(true); - let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { - let column = batch.column(0); - let match_first = eq(column, &Int64Array::new_scalar(first_value))?; - let match_second = eq(column, &Int64Array::new_scalar(last_value))?; - or(&match_first, &match_second) - }); + let predicate = make_predicate(filter_mask.clone()); // The batch size is set to 12 to read all rows in one go after filtering // If the Reader chooses mask to handle filter, it might cause panic because the mid 4 pages may not be decoded. - let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(data, options) + let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(data.clone(), options) .unwrap() .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) + .with_row_selection_policy(RowSelectionPolicy::Auto { threshold: 32 }) .with_batch_size(12) .build() .unwrap(); diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index d990361d21bc..61a84ea1e4bf 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -763,6 +763,7 @@ where #[cfg(test)] mod tests { use super::*; + use crate::arrow::arrow_reader::RowSelectionPolicy; use crate::arrow::arrow_reader::{ ArrowPredicateFn, ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, }; @@ -1252,27 +1253,31 @@ mod tests { let schema = builder.parquet_schema().clone(); let filter_mask = ProjectionMask::leaves(&schema, [0]); - let predicate = ArrowPredicateFn::new(filter_mask, move |batch: RecordBatch| { - let column = batch.column(0); - let match_first = eq(column, &Int64Array::new_scalar(first_value))?; - let match_second = eq(column, &Int64Array::new_scalar(last_value))?; - or(&match_first, &match_second) - }); + let make_predicate = |mask: ProjectionMask| { + ArrowPredicateFn::new(mask, move |batch: RecordBatch| { + let column = batch.column(0); + let match_first = eq(column, &Int64Array::new_scalar(first_value))?; + let match_second = eq(column, &Int64Array::new_scalar(last_value))?; + or(&match_first, &match_second) + }) + }; + + let predicate = make_predicate(filter_mask.clone()); // The batch size is set to 12 to read all rows in one go after filtering // If the Reader chooses mask to handle filter, it might cause panic because the mid 4 pages may not be decoded. let stream = ParquetRecordBatchStreamBuilder::new_with_options( - TestReader::new(data), + TestReader::new(data.clone()), ArrowReaderOptions::new().with_page_index(true), ) .await .unwrap() .with_row_filter(RowFilter::new(vec![Box::new(predicate)])) .with_batch_size(12) + .with_row_selection_policy(RowSelectionPolicy::Auto { threshold: 32 }) .build() .unwrap(); - // Collecting into batches validates the plan now downgrades to selectors instead of panicking. let schema = stream.schema().clone(); let batches: Vec<_> = stream.try_collect().await.unwrap(); let result = concat_batches(&schema, &batches).unwrap(); diff --git a/parquet/src/arrow/in_memory_row_group.rs b/parquet/src/arrow/in_memory_row_group.rs index 34e46cd34e91..7969f6f32321 100644 --- a/parquet/src/arrow/in_memory_row_group.rs +++ b/parquet/src/arrow/in_memory_row_group.rs @@ -258,7 +258,9 @@ impl ColumnChunkData { .map(|idx| data[idx].1.clone()) .map_err(|_| { ParquetError::General(format!( - "Invalid offset in sparse column chunk data: {start}" + "Invalid offset in sparse column chunk data: {start}, no matching page found.\ + If you are using a `SelectionStrategyPolicy::Mask`, ensure that the OffsetIndex is provided when \ + creating the InMemoryRowGroup." )) }), ColumnChunkData::Dense { offset, data } => { diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index cd4a6fa38892..312b8af84535 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -535,7 +535,7 @@ impl RowGroupReaderBuilder { plan_builder = plan_builder.with_row_selection_policy(self.row_selection_policy); - plan_builder = overide_selector_strategy_if_needed( + plan_builder = override_selector_strategy_if_needed( plan_builder, &self.projection, self.row_group_offset_index(row_group_idx), @@ -678,32 +678,31 @@ impl RowGroupReaderBuilder { } } -/// Overrider the selection strategy if needed. +/// Override the selection strategy if needed. /// -/// Some pages can be skipped during row-group construction of they are not read +/// Some pages can be skipped during row-group construction if they are not read /// by the selections. This means that the data pages for those rows are never /// loaded and definition/repetition levels are never read. When using -/// `RowSelections` selection works because skip_records() handles this +/// `RowSelections` selection works because `skip_records()` handles this /// case and skips the page accordingly. /// -/// However, with the current Mask design, all values must be read and decoded -/// and then a Mask filter is applied filtering. Thus if any pages are skipped -/// during row-group construction, the data pages are missing and cannot be -/// decoded. +/// However, with the current mask design, all values must be read and decoded +/// and then a mask filter is applied. Thus if any pages are skipped during +/// row-group construction, the data pages are missing and cannot be decoded. /// /// A simple example: /// * the page size is 2, the mask is 100001, row selection should be read(1) skip(4) read(1) -/// * the ColumnChunkData would be page1(10), page2(skipped), page3(01) +/// * the `ColumnChunkData` would be page1(10), page2(skipped), page3(01) /// -/// Using the rowselection to skip(4), page2 won't be read at all, so in this +/// Using the row selection to skip(4), page2 won't be read at all, so in this /// case we can't decode all the rows and apply a mask. To correctly apply the -/// bit mask, we need all 6 value be read, but the page2 is not in memory. -fn overide_selector_strategy_if_needed( +/// bit mask, we need all 6 values be read, but page2 is not in memory. +fn override_selector_strategy_if_needed( plan_builder: ReadPlanBuilder, projection_mask: &ProjectionMask, offset_index: Option<&[OffsetIndexMetaData]>, ) -> ReadPlanBuilder { - // override only applies to Auto policy + // override only applies to Auto policy, If the policy is already Mask or Selectors, respect that let RowSelectionPolicy::Auto { .. } = plan_builder.row_selection_policy() else { return plan_builder; };