From cc6dd14b5f191b3f8d8d72b6c958f6c0965c8b5a Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 4 Sep 2024 09:53:31 -0400 Subject: [PATCH 01/73] update --- testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing b/testing index 735ae7128d57..e270341fb5f3 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 735ae7128d571398dd798d7ff004adebeb342883 +Subproject commit e270341fb5f3ff785410e6286cc42898e9d6a99c From 5837fc7e62c221451f6245b2739757e48df43495 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Sun, 22 Dec 2024 07:51:44 -0600 Subject: [PATCH 02/73] update --- parquet-testing | 2 +- testing | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet-testing b/parquet-testing index 4439a223a315..1ba34478f535 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 4439a223a315cf874746d3b5da25e6a6b2a2b16e +Subproject commit 1ba34478f535c89382263c42c675a9af4f57f2dd diff --git a/testing b/testing index e270341fb5f3..735ae7128d57 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit e270341fb5f3ff785410e6286cc42898e9d6a99c +Subproject commit 735ae7128d571398dd798d7ff004adebeb342883 From fec63139ba7e3e6a1b9f30d312a189122a7b674b Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Sat, 28 Dec 2024 12:57:00 -0600 Subject: [PATCH 03/73] update --- arrow-buffer/src/buffer/boolean.rs | 45 ++++ parquet/src/arrow/array_reader/builder.rs | 6 + .../src/arrow/array_reader/primitive_array.rs | 13 +- .../src/arrow/array_reader/struct_array.rs | 2 +- parquet/src/arrow/arrow_reader/filter.rs | 2 +- parquet/src/arrow/arrow_reader/selection.rs | 6 + .../src/arrow/async_reader/arrow_reader.rs | 244 ++++++++++++++++++ parquet/src/arrow/async_reader/mod.rs | 1 + 8 files changed, 313 insertions(+), 6 deletions(-) create mode 100644 parquet/src/arrow/async_reader/arrow_reader.rs diff --git a/arrow-buffer/src/buffer/boolean.rs b/arrow-buffer/src/buffer/boolean.rs index aaa86832f692..4e42d3c27e1a 100644 --- a/arrow-buffer/src/buffer/boolean.rs +++ b/arrow-buffer/src/buffer/boolean.rs @@ -204,6 +204,51 @@ impl BooleanBuffer { pub fn set_slices(&self) -> BitSliceIterator<'_> { BitSliceIterator::new(self.values(), self.offset, self.len) } + + /// Combines this [`BooleanBuffer`] with another using logical AND on the selected bits. + /// + /// Unlike intersection, the `other` [`BooleanBuffer`] must have exactly as many **set bits** as `self`, + /// i.e., self.count_set_bits() == other.len(). + /// + /// This method will keep only the bits in `self` that are also set in `other` + /// at the positions corresponding to `self`'s set bits. + /// For example: + /// self: NNYYYNNYYNYN + /// other: YNY NY N + /// result: NNYNYNNNYNNN + pub fn and_then(&self, other: &Self) -> Self { + // Ensure that 'other' has exactly as many set bits as 'self' + debug_assert_eq!( + self.count_set_bits(), + other.len(), + "The 'other' selection must have exactly as many set bits as 'self'." + ); + + if self.len() == other.len() { + // fast path if the two bool masks are the same length + // this happens when self selects all rows + debug_assert_eq!(self.count_set_bits(), self.len()); + return other.clone(); + } + + let mut buffer = MutableBuffer::from_len_zeroed(self.values().len()); + buffer.copy_from_slice(self.values()); + let mut builder = BooleanBufferBuilder::new_from_buffer(buffer, self.len()); + + // Create iterators for 'self' and 'other' bits + let mut other_bits = other.iter(); + + for bit_idx in self.set_indices() { + let predicate = other_bits + .next() + .expect("Mismatch in set bits between self and other"); + if !predicate { + builder.set_bit(bit_idx, false); + } + } + + builder.finish() + } } impl Not for &BooleanBuffer { diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 945f62526a7e..23f77a9ab96f 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -245,6 +245,7 @@ fn build_primitive_reader( page_iterator, column_desc, arrow_type, + col_idx, )?) as _, PhysicalType::INT32 => { if let Some(DataType::Null) = arrow_type { @@ -257,6 +258,7 @@ fn build_primitive_reader( page_iterator, column_desc, arrow_type, + col_idx, )?) as _ } } @@ -264,21 +266,25 @@ fn build_primitive_reader( page_iterator, column_desc, arrow_type, + col_idx, )?) as _, PhysicalType::INT96 => Box::new(PrimitiveArrayReader::::new( page_iterator, column_desc, arrow_type, + col_idx, )?) as _, PhysicalType::FLOAT => Box::new(PrimitiveArrayReader::::new( page_iterator, column_desc, arrow_type, + col_idx, )?) as _, PhysicalType::DOUBLE => Box::new(PrimitiveArrayReader::::new( page_iterator, column_desc, arrow_type, + col_idx, )?) as _, PhysicalType::BYTE_ARRAY => match arrow_type { Some(DataType::Dictionary(_, _)) => { diff --git a/parquet/src/arrow/array_reader/primitive_array.rs b/parquet/src/arrow/array_reader/primitive_array.rs index a952e00e12ef..529513ce200f 100644 --- a/parquet/src/arrow/array_reader/primitive_array.rs +++ b/parquet/src/arrow/array_reader/primitive_array.rs @@ -80,6 +80,7 @@ where def_levels_buffer: Option>, rep_levels_buffer: Option>, record_reader: RecordReader, + column_idx: usize, } impl PrimitiveArrayReader @@ -93,6 +94,7 @@ where pages: Box, column_desc: ColumnDescPtr, arrow_type: Option, + column_idx: usize, ) -> Result { // Check if Arrow type is specified, else create it from Parquet type let data_type = match arrow_type { @@ -110,6 +112,7 @@ where def_levels_buffer: None, rep_levels_buffer: None, record_reader, + column_idx, }) } } @@ -371,6 +374,7 @@ mod tests { Box::::default(), schema.column(0), None, + 0, ) .unwrap(); @@ -414,7 +418,7 @@ mod tests { let page_iterator = InMemoryPageIterator::new(page_lists); let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) .unwrap(); // Read first 50 values, which are all from the first column chunk @@ -484,6 +488,7 @@ mod tests { Box::new(page_iterator), column_desc.clone(), None, + 0, ) .expect("Unable to get array reader"); @@ -620,7 +625,7 @@ mod tests { let page_iterator = InMemoryPageIterator::new(page_lists); let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) .unwrap(); let mut accu_len: usize = 0; @@ -696,7 +701,7 @@ mod tests { let page_iterator = InMemoryPageIterator::new(page_lists); let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) .unwrap(); // read data from the reader @@ -755,7 +760,7 @@ mod tests { let page_iterator = InMemoryPageIterator::new(page_lists); let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) .unwrap(); // read data from the reader diff --git a/parquet/src/arrow/array_reader/struct_array.rs b/parquet/src/arrow/array_reader/struct_array.rs index fb2f2f8928b9..e048fbae66fa 100644 --- a/parquet/src/arrow/array_reader/struct_array.rs +++ b/parquet/src/arrow/array_reader/struct_array.rs @@ -25,7 +25,7 @@ use std::sync::Arc; /// Implementation of struct array reader. pub struct StructArrayReader { - children: Vec>, + pub children: Vec>, data_type: ArrowType, struct_def_level: i16, struct_rep_level: i16, diff --git a/parquet/src/arrow/arrow_reader/filter.rs b/parquet/src/arrow/arrow_reader/filter.rs index 2e22f7e01cf0..931e13e252f5 100644 --- a/parquet/src/arrow/arrow_reader/filter.rs +++ b/parquet/src/arrow/arrow_reader/filter.rs @@ -110,7 +110,7 @@ where /// [`RowSelection`]: crate::arrow::arrow_reader::RowSelection pub struct RowFilter { /// A list of [`ArrowPredicate`] - pub(crate) predicates: Vec>, + pub predicates: Vec>, } impl RowFilter { diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 378d2253f19a..92d7eab1e58c 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -366,6 +366,12 @@ impl RowSelection { self } + /// Returns the internal selectors of this [`RowSelection`], testing only + #[cfg(test)] + pub(crate) fn selectors(&self) -> &[RowSelector] { + &self.selectors + } + /// Applies an offset to this [`RowSelection`], skipping the first `offset` selected rows pub(crate) fn offset(mut self, offset: usize) -> Self { if offset == 0 { diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs new file mode 100644 index 000000000000..a2497fc9adcc --- /dev/null +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -0,0 +1,244 @@ +// 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::{collections::VecDeque, sync::Arc}; + +use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader, StructArray}; +use arrow_schema::{ArrowError, SchemaRef}; +use arrow_select::filter::prep_null_mask_filter; + +use crate::arrow::{ + array_reader::{build_array_reader, ArrayReader, RowGroups, StructArrayReader}, + arrow_reader::{ArrowPredicate, RowFilter, RowSelection, RowSelector}, +}; +use crate::errors::ParquetError; + +use super::ParquetField; + +pub struct FilteredParquetRecordBatchReader { + batch_size: usize, + array_reader: StructArrayReader, + predicate_readers: Vec>, + schema: SchemaRef, + selection: VecDeque, + row_filter: Option, +} + +fn read_selection( + reader: &mut dyn ArrayReader, + selection: &RowSelection, +) -> Result { + for selector in selection.iter() { + if selector.skip { + let skipped = reader.skip_records(selector.row_count)?; + debug_assert_eq!(skipped, selector.row_count, "failed to skip rows"); + } else { + let read_records = reader.read_records(selector.row_count)?; + debug_assert_eq!(read_records, selector.row_count, "failed to read rows"); + } + } + let array = reader.consume_batch()?; + let struct_array = array + .as_struct_opt() + .ok_or_else(|| general_err!("Struct array reader should return struct array"))?; + Ok(struct_array.clone()) +} + +/// Take the next selection from the selection queue, and return the selection +/// whose selected row count is to_select or less (if input selection is exhausted). +fn take_next_selection( + selection: &mut VecDeque, + to_select: usize, +) -> Option { + let mut current_selected = 0; + let mut rt = Vec::new(); + while let Some(front) = selection.pop_front() { + if front.skip { + rt.push(front); + continue; + } + + if current_selected + front.row_count <= to_select { + rt.push(front); + current_selected += front.row_count; + } else { + let select = to_select - current_selected; + let remaining = front.row_count - select; + rt.push(RowSelector::select(select)); + selection.push_front(RowSelector::select(remaining)); + + return Some(rt.into()); + } + } + if !rt.is_empty() { + return Some(rt.into()); + } + None +} + +fn build_array_reader_for_filters( + filters: &RowFilter, + fields: &Option>, + row_group: &dyn RowGroups, +) -> Result>, ArrowError> { + let mut array_readers = Vec::new(); + for predicate in filters.predicates.iter() { + let predicate_projection = predicate.projection(); + let array_reader = build_array_reader(fields.as_deref(), predicate_projection, row_group)?; + array_readers.push(array_reader); + } + Ok(array_readers) +} + +impl FilteredParquetRecordBatchReader { + fn new(batch_size: usize, array_reader: StructArrayReader, selection: RowSelection) -> Self { + todo!() + } + + fn build_predicate_filter( + &mut self, + selection: &RowSelection, + ) -> Result { + match &mut self.row_filter { + None => Ok(selection.clone()), + Some(filter) => { + debug_assert_eq!( + self.predicate_readers.len(), + filter.predicates.len(), + "predicate readers and predicates should have the same length" + ); + let mut selection = selection.clone(); + + for (predicate, reader) in filter + .predicates + .iter_mut() + .zip(self.predicate_readers.iter_mut()) + { + let array = read_selection(reader.as_mut(), &selection)?; + let batch = RecordBatch::from(array); + let input_rows = batch.num_rows(); + let predicate_filter = predicate.evaluate(batch)?; + if predicate_filter.len() != input_rows { + return Err(ArrowError::ParquetError(format!( + "ArrowPredicate predicate returned {} rows, expected {input_rows}", + predicate_filter.len() + ))); + } + let predicate_filter = match predicate_filter.null_count() { + 0 => predicate_filter, + _ => prep_null_mask_filter(&predicate_filter), + }; + let raw = RowSelection::from_filters(&[predicate_filter]); + selection = selection.and_then(&raw); + } + Ok(selection) + } + } + } +} + +impl Iterator for FilteredParquetRecordBatchReader { + type Item = Result; + + fn next(&mut self) -> Option { + let selection = take_next_selection(&mut self.selection, self.batch_size)?; + let filtered_selection = match self.build_predicate_filter(&selection) { + Ok(selection) => selection, + Err(e) => return Some(Err(e)), + }; + + let rt = read_selection(&mut self.array_reader, &filtered_selection); + match rt { + Ok(array) => Some(Ok(RecordBatch::from(array))), + Err(e) => Some(Err(e.into())), + } + } +} + +impl RecordBatchReader for FilteredParquetRecordBatchReader { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_take_next_selection_exact_match() { + let mut queue = VecDeque::from(vec![ + RowSelector::skip(5), + RowSelector::select(3), + RowSelector::skip(2), + RowSelector::select(7), + ]); + + // Request exactly 10 rows (5 skip + 3 select + 2 skip) + let selection = take_next_selection(&mut queue, 3).unwrap(); + assert_eq!( + selection, + vec![ + RowSelector::skip(5), + RowSelector::select(3), + RowSelector::skip(2) + ] + .into() + ); + + // Check remaining queue + assert_eq!(queue.len(), 1); + assert_eq!(queue[0].row_count, 7); + assert_eq!(queue[0].skip, false); + } + + #[test] + fn test_take_next_selection_split_required() { + let mut queue = VecDeque::from(vec![RowSelector::select(10), RowSelector::select(10)]); + + // Request 15 rows, which should split the first selector + let selection = take_next_selection(&mut queue, 15).unwrap(); + + assert_eq!( + selection, + vec![RowSelector::select(10), RowSelector::select(5)].into() + ); + + // Check remaining queue - should have 5 rows from split and original 10 + assert_eq!(queue.len(), 1); + assert_eq!(queue[0].skip, false); + assert_eq!(queue[0].row_count, 5); + } + + #[test] + fn test_take_next_selection_empty_queue() { + let mut queue = VecDeque::new(); + + // Should return None for empty queue + let selection = take_next_selection(&mut queue, 10); + assert!(selection.is_none()); + + // Test with queue that becomes empty + queue.push_back(RowSelector::select(5)); + let selection = take_next_selection(&mut queue, 10).unwrap(); + assert_eq!(selection, vec![RowSelector::select(5)].into()); + + // Queue should now be empty + let selection = take_next_selection(&mut queue, 10); + assert!(selection.is_none()); + } +} diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 4f3befe42662..2b31dcd91635 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -110,6 +110,7 @@ use crate::file::reader::{ChunkReader, Length, SerializedPageReader}; use crate::file::FOOTER_SIZE; use crate::format::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; +mod arrow_reader; mod metadata; pub use metadata::*; From 948db872bf622f623e11ef26b1d185614ef32886 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Sun, 29 Dec 2024 15:46:03 -0600 Subject: [PATCH 04/73] update --- .../src/arrow/array_reader/primitive_array.rs | 41 ++-- parquet/src/arrow/arrow_reader/selection.rs | 10 +- .../src/arrow/async_reader/arrow_reader.rs | 195 +++++++++++++++--- parquet/src/arrow/async_reader/mod.rs | 91 ++++---- parquet/src/file/serialized_reader.rs | 64 +++++- testing | 2 +- 6 files changed, 309 insertions(+), 94 deletions(-) diff --git a/parquet/src/arrow/array_reader/primitive_array.rs b/parquet/src/arrow/array_reader/primitive_array.rs index 529513ce200f..c665cee5bf26 100644 --- a/parquet/src/arrow/array_reader/primitive_array.rs +++ b/parquet/src/arrow/array_reader/primitive_array.rs @@ -80,6 +80,7 @@ where def_levels_buffer: Option>, rep_levels_buffer: Option>, record_reader: RecordReader, + #[allow(unused)] column_idx: usize, } @@ -417,9 +418,13 @@ mod tests { ); let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) - .unwrap(); + let mut array_reader = PrimitiveArrayReader::::new( + Box::new(page_iterator), + column_desc, + None, + 0, + ) + .unwrap(); // Read first 50 values, which are all from the first column chunk let array = array_reader.next_batch(50).unwrap(); @@ -624,9 +629,13 @@ mod tests { let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) - .unwrap(); + let mut array_reader = PrimitiveArrayReader::::new( + Box::new(page_iterator), + column_desc, + None, + 0, + ) + .unwrap(); let mut accu_len: usize = 0; @@ -700,9 +709,13 @@ mod tests { ); let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) - .unwrap(); + let mut array_reader = PrimitiveArrayReader::::new( + Box::new(page_iterator), + column_desc, + None, + 0, + ) + .unwrap(); // read data from the reader // the data type is decimal(8,2) @@ -759,9 +772,13 @@ mod tests { ); let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = - PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None, 0) - .unwrap(); + let mut array_reader = PrimitiveArrayReader::::new( + Box::new(page_iterator), + column_desc, + None, + 0, + ) + .unwrap(); // read data from the reader // the data type is decimal(18,4) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 92d7eab1e58c..f83724a3841b 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -366,12 +366,6 @@ impl RowSelection { self } - /// Returns the internal selectors of this [`RowSelection`], testing only - #[cfg(test)] - pub(crate) fn selectors(&self) -> &[RowSelector] { - &self.selectors - } - /// Applies an offset to this [`RowSelection`], skipping the first `offset` selected rows pub(crate) fn offset(mut self, offset: usize) -> Self { if offset == 0 { @@ -447,6 +441,10 @@ impl RowSelection { pub fn skipped_row_count(&self) -> usize { self.iter().filter(|s| s.skip).map(|s| s.row_count).sum() } + + pub(crate) fn extend(&mut self, other: Self) { + self.selectors.extend(other.selectors); + } } impl From> for RowSelection { diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index a2497fc9adcc..04cc115ce39c 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -15,23 +15,27 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; +use std::sync::RwLock; use std::{collections::VecDeque, sync::Arc}; use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader, StructArray}; -use arrow_schema::{ArrowError, SchemaRef}; +use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; use arrow_select::filter::prep_null_mask_filter; -use crate::arrow::{ - array_reader::{build_array_reader, ArrayReader, RowGroups, StructArrayReader}, - arrow_reader::{ArrowPredicate, RowFilter, RowSelection, RowSelector}, -}; +use crate::column::page::{Page, PageMetadata, PageReader}; use crate::errors::ParquetError; - -use super::ParquetField; +use crate::{ + arrow::{ + array_reader::ArrayReader, + arrow_reader::{RowFilter, RowSelection, RowSelector}, + }, + file::reader::{ChunkReader, SerializedPageReader}, +}; pub struct FilteredParquetRecordBatchReader { batch_size: usize, - array_reader: StructArrayReader, + array_reader: Box, predicate_readers: Vec>, schema: SchemaRef, selection: VecDeque, @@ -90,38 +94,47 @@ fn take_next_selection( None } -fn build_array_reader_for_filters( - filters: &RowFilter, - fields: &Option>, - row_group: &dyn RowGroups, -) -> Result>, ArrowError> { - let mut array_readers = Vec::new(); - for predicate in filters.predicates.iter() { - let predicate_projection = predicate.projection(); - let array_reader = build_array_reader(fields.as_deref(), predicate_projection, row_group)?; - array_readers.push(array_reader); - } - Ok(array_readers) -} - impl FilteredParquetRecordBatchReader { - fn new(batch_size: usize, array_reader: StructArrayReader, selection: RowSelection) -> Self { - todo!() + pub(crate) fn new( + batch_size: usize, + array_reader: Box, + selection: RowSelection, + filter_readers: Vec>, + row_filter: Option, + ) -> Self { + let schema = match array_reader.get_data_type() { + DataType::Struct(ref fields) => Schema::new(fields.clone()), + _ => unreachable!("Struct array reader's data type is not struct!"), + }; + + Self { + batch_size, + array_reader, + predicate_readers: filter_readers, + schema: Arc::new(schema), + selection: selection.into(), + row_filter, + } + } + + pub(crate) fn take_filter(&mut self) -> Option { + self.row_filter.take() } + #[inline(never)] + /// Take a selection, and return the new selection where the rows are filtered by the predicate. fn build_predicate_filter( &mut self, - selection: &RowSelection, + mut selection: RowSelection, ) -> Result { match &mut self.row_filter { - None => Ok(selection.clone()), + None => Ok(selection), Some(filter) => { debug_assert_eq!( self.predicate_readers.len(), filter.predicates.len(), "predicate readers and predicates should have the same length" ); - let mut selection = selection.clone(); for (predicate, reader) in filter .predicates @@ -155,13 +168,36 @@ impl Iterator for FilteredParquetRecordBatchReader { type Item = Result; fn next(&mut self) -> Option { - let selection = take_next_selection(&mut self.selection, self.batch_size)?; - let filtered_selection = match self.build_predicate_filter(&selection) { - Ok(selection) => selection, - Err(e) => return Some(Err(e)), - }; + // With filter pushdown, it's very hard to predict the number of rows to return -- depends on the selectivity of the filter. + // We can do one of the following: + // 1. Add a coalescing step to coalesce the resulting batches. + // 2. Ask parquet reader to collect more rows before returning. - let rt = read_selection(&mut self.array_reader, &filtered_selection); + // Approach 1 has the drawback of extra overhead of coalesce batch, which can be painful to be efficient. + // Code below implements approach 2, where we keep consuming the selection until we select at least 3/4 of the batch size. + // It boils down to leveraging array_reader's ability to collect large batches natively, + // rather than concatenating multiple small batches. + + let mut selection = RowSelection::default(); + let mut selected = 0; + while let Some(cur_selection) = + take_next_selection(&mut self.selection, self.batch_size - selected) + { + let filtered_selection = match self.build_predicate_filter(cur_selection) { + Ok(selection) => selection, + Err(e) => return Some(Err(e)), + }; + selected += filtered_selection.row_count(); + selection.extend(filtered_selection); + if selected >= (self.batch_size / 4 * 3) { + break; + } + } + if !selection.selects_any() { + return None; + } + + let rt = read_selection(&mut *self.array_reader, &selection); match rt { Ok(array) => Some(Ok(RecordBatch::from(array))), Err(e) => Some(Err(e.into())), @@ -175,6 +211,99 @@ impl RecordBatchReader for FilteredParquetRecordBatchReader { } } +struct PageCacheInner { + queue: VecDeque, + pages: HashMap, +} + +/// A simple FIFO cache for pages. +pub(crate) struct PageCache { + inner: RwLock, +} + +impl PageCache { + const CAPACITY: usize = 16; + + pub(crate) fn new() -> Self { + Self { + inner: RwLock::new(PageCacheInner { + queue: VecDeque::with_capacity(Self::CAPACITY), + pages: HashMap::with_capacity(Self::CAPACITY), + }), + } + } + + pub(crate) fn get_page(&self, offset: usize) -> Option { + let read_lock = self.inner.read().unwrap(); + read_lock.pages.get(&offset).cloned() + } + + pub(crate) fn insert_page(&self, offset: usize, page: Page) { + let mut write_lock = self.inner.write().unwrap(); + if write_lock.pages.len() >= Self::CAPACITY { + let oldest_offset = write_lock.queue.pop_front().unwrap(); + write_lock.pages.remove(&oldest_offset).unwrap(); + } + write_lock.pages.insert(offset, page); + write_lock.queue.push_back(offset); + } +} + +pub(crate) struct CachedPageReader { + inner: SerializedPageReader, + cache: Arc, +} + +impl CachedPageReader { + pub(crate) fn new(inner: SerializedPageReader, cache: Arc) -> Self { + Self { inner, cache } + } +} + +impl Iterator for CachedPageReader { + type Item = Result; + + fn next(&mut self) -> Option { + self.get_next_page().transpose() + } +} + +impl PageReader for CachedPageReader { + fn get_next_page(&mut self) -> Result, ParquetError> { + // self.inner.get_next_page() + let next_page_offset = self.inner.peek_next_page_offset()?; + + let Some(offset) = next_page_offset else { + return Ok(None); + }; + + let page = self.cache.get_page(offset); + if let Some(page) = page { + self.inner.skip_next_page()?; + Ok(Some(page)) + } else { + let inner_page = self.inner.get_next_page()?; + let Some(inner_page) = inner_page else { + return Ok(None); + }; + self.cache.insert_page(offset, inner_page.clone()); + Ok(Some(inner_page)) + } + } + + fn peek_next_page(&mut self) -> Result, ParquetError> { + self.inner.peek_next_page() + } + + fn skip_next_page(&mut self) -> Result<(), ParquetError> { + self.inner.skip_next_page() + } + + fn at_record_boundary(&mut self) -> Result { + self.inner.at_record_boundary() + } +} + #[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 2b31dcd91635..3ee85a26fff2 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -83,6 +83,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use arrow_reader::{CachedPageReader, FilteredParquetRecordBatchReader, PageCache}; use bytes::{Buf, Bytes}; use futures::future::{BoxFuture, FutureExt}; use futures::ready; @@ -94,8 +95,7 @@ use arrow_schema::{DataType, Fields, Schema, SchemaRef}; use crate::arrow::array_reader::{build_array_reader, RowGroups}; use crate::arrow::arrow_reader::{ - apply_range, evaluate_predicate, selects_any, ArrowReaderBuilder, ArrowReaderMetadata, - ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection, + ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, RowFilter, RowSelection, }; use crate::arrow::ProjectionMask; @@ -121,6 +121,8 @@ use crate::arrow::schema::ParquetField; #[cfg(feature = "object_store")] pub use store::*; +use super::arrow_reader::RowSelector; + /// The asynchronous interface used by [`ParquetRecordBatchStream`] to read parquet files /// /// Notes: @@ -470,7 +472,7 @@ impl ParquetRecordBatchStreamBuilder { } } -type ReadResult = Result<(ReaderFactory, Option)>; +type ReadResult = Result<(ReaderFactory, Option)>; /// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create /// [`ParquetRecordBatchReader`] @@ -498,18 +500,16 @@ where async fn read_row_group( mut self, row_group_idx: usize, - mut selection: Option, + selection: Option, projection: ProjectionMask, batch_size: usize, ) -> ReadResult { - // TODO: calling build_array multiple times is wasteful - let meta = self.metadata.row_group(row_group_idx); let offset_index = self .metadata .offset_index() // filter out empty offset indexes (old versions specified Some(vec![]) when no present) - .filter(|index| !index.is_empty()) + .filter(|index| index.first().map(|v| !v.is_empty()).unwrap_or(false)) .map(|x| x[row_group_idx].as_slice()); let mut row_group = InMemoryRowGroup { @@ -518,48 +518,47 @@ where row_count: meta.num_rows() as usize, column_chunks: vec![None; meta.columns().len()], offset_index, + cache: Arc::new(PageCache::new()), }; + let mut selection = + selection.unwrap_or_else(|| vec![RowSelector::select(row_group.row_count)].into()); + + let mut filter_readers = Vec::new(); if let Some(filter) = self.filter.as_mut() { for predicate in filter.predicates.iter_mut() { - if !selects_any(selection.as_ref()) { + if !selection.selects_any() { return Ok((self, None)); } let predicate_projection = predicate.projection(); row_group - .fetch(&mut self.input, predicate_projection, selection.as_ref()) + .fetch(&mut self.input, predicate_projection, Some(&selection)) .await?; let array_reader = build_array_reader(self.fields.as_deref(), predicate_projection, &row_group)?; - - selection = Some(evaluate_predicate( - batch_size, - array_reader, - selection, - predicate.as_mut(), - )?); + filter_readers.push(array_reader); } } // Compute the number of rows in the selection before applying limit and offset - let rows_before = selection - .as_ref() - .map(|s| s.row_count()) - .unwrap_or(row_group.row_count); + let rows_before = selection.row_count(); if rows_before == 0 { return Ok((self, None)); } - selection = apply_range(selection, row_group.row_count, self.offset, self.limit); + if let Some(offset) = self.offset { + selection = selection.offset(offset); + } + + if let Some(limit) = self.limit { + selection = selection.limit(limit); + } // Compute the number of rows in the selection after applying limit and offset - let rows_after = selection - .as_ref() - .map(|s| s.row_count()) - .unwrap_or(row_group.row_count); + let rows_after = selection.row_count(); // Update offset if necessary if let Some(offset) = &mut self.offset { @@ -577,13 +576,16 @@ where } row_group - .fetch(&mut self.input, &projection, selection.as_ref()) + .fetch(&mut self.input, &projection, Some(&selection)) .await?; - let reader = ParquetRecordBatchReader::new( + let array_reader = build_array_reader(self.fields.as_deref(), &projection, &row_group)?; + let reader = FilteredParquetRecordBatchReader::new( batch_size, - build_array_reader(self.fields.as_deref(), &projection, &row_group)?, + array_reader, selection, + filter_readers, + self.filter.take(), ); Ok((self, Some(reader))) @@ -594,7 +596,7 @@ enum StreamState { /// At the start of a new row group, or the end of the parquet stream Init, /// Decoding a batch - Decoding(ParquetRecordBatchReader), + Decoding(FilteredParquetRecordBatchReader), /// Reading data from input Reading(BoxFuture<'static, ReadResult>), /// Error @@ -739,7 +741,12 @@ where self.state = StreamState::Error; return Poll::Ready(Some(Err(ParquetError::ArrowError(e.to_string())))); } - None => self.state = StreamState::Init, + None => { + // this is ugly, but works for now. + let filter = batch_reader.take_filter(); + self.reader.as_mut().unwrap().filter = filter; + self.state = StreamState::Init + } }, StreamState::Init => { let row_group_idx = match self.row_groups.pop_front() { @@ -791,6 +798,7 @@ struct InMemoryRowGroup<'a> { offset_index: Option<&'a [OffsetIndexMetaData]>, column_chunks: Vec>>, row_count: usize, + cache: Arc, } impl InMemoryRowGroup<'_> { @@ -902,12 +910,23 @@ impl RowGroups for InMemoryRowGroup<'_> { // filter out empty offset indexes (old versions specified Some(vec![]) when no present) .filter(|index| !index.is_empty()) .map(|index| index[i].page_locations.clone()); - let page_reader: Box = Box::new(SerializedPageReader::new( - data.clone(), - self.metadata.column(i), - self.row_count, - page_locations, - )?); + + // let page_reader: Box = Box::new(SerializedPageReader::new( + // data.clone(), + // self.metadata.column(i), + // self.row_count, + // page_locations, + // )?); + + let page_reader: Box = Box::new(CachedPageReader::new( + SerializedPageReader::new( + data.clone(), + self.metadata.column(i), + self.row_count, + page_locations, + )?, + self.cache.clone(), + )); Ok(Box::new(ColumnChunkIterator { reader: Some(Ok(page_reader)), diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index a942481f7e4d..146b16f38422 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -371,7 +371,7 @@ fn read_page_header_len(input: &mut T) -> Result<(usize, PageHeader)> { /// Decodes a [`Page`] from the provided `buffer` pub(crate) fn decode_page( page_header: PageHeader, - buffer: Bytes, + buffer: Vec, physical_type: Type, decompressor: Option<&mut Box>, ) -> Result { @@ -406,8 +406,8 @@ pub(crate) fn decode_page( Some(decompressor) if can_decompress => { let uncompressed_size = page_header.uncompressed_page_size as usize; let mut decompressed = Vec::with_capacity(uncompressed_size); - let compressed = &buffer.as_ref()[offset..]; - decompressed.extend_from_slice(&buffer.as_ref()[..offset]); + let compressed = &buffer[offset..]; + decompressed.extend_from_slice(&buffer[..offset]); decompressor.decompress( compressed, &mut decompressed, @@ -422,10 +422,11 @@ pub(crate) fn decode_page( )); } - Bytes::from(decompressed) + decompressed } _ => buffer, }; + let buffer = Bytes::from(buffer); let result = match page_header.type_ { PageType::DICTIONARY_PAGE => { @@ -568,6 +569,57 @@ impl SerializedPageReader { physical_type: meta.column_type(), }) } + + pub(crate) fn peek_next_page_offset(&mut self) -> Result> { + match &mut self.state { + SerializedPageReaderState::Values { + offset, + remaining_bytes, + next_page_header, + } => { + loop { + if *remaining_bytes == 0 { + return Ok(None); + } + return if let Some(header) = next_page_header.as_ref() { + if let Ok(_page_meta) = PageMetadata::try_from(&**header) { + Ok(Some(*offset)) + } else { + // For unknown page type (e.g., INDEX_PAGE), skip and read next. + *next_page_header = None; + continue; + } + } else { + let mut read = self.reader.get_read(*offset as u64)?; + let (header_len, header) = read_page_header_len(&mut read)?; + *offset += header_len; + *remaining_bytes -= header_len; + let page_meta = if let Ok(_page_meta) = PageMetadata::try_from(&header) { + Ok(Some(*offset)) + } else { + // For unknown page type (e.g., INDEX_PAGE), skip and read next. + continue; + }; + *next_page_header = Some(Box::new(header)); + page_meta + }; + } + } + SerializedPageReaderState::Pages { + page_locations, + dictionary_page, + .. + } => { + if let Some(page) = dictionary_page { + Ok(Some(page.offset as usize)) + } else if let Some(page) = page_locations.front() { + Ok(Some(page.offset as usize)) + } else { + Ok(None) + } + } + } + } } impl Iterator for SerializedPageReader { @@ -648,7 +700,7 @@ impl PageReader for SerializedPageReader { decode_page( header, - Bytes::from(buffer), + buffer, self.physical_type, self.decompressor.as_mut(), )? @@ -677,7 +729,7 @@ impl PageReader for SerializedPageReader { let bytes = buffer.slice(offset..); decode_page( header, - bytes, + bytes.to_vec(), self.physical_type, self.decompressor.as_mut(), )? diff --git a/testing b/testing index 735ae7128d57..4d209492d514 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 735ae7128d571398dd798d7ff004adebeb342883 +Subproject commit 4d209492d514c2d3cb2d392681b9aa00e6d8da1c From 8c50d90b98a10419c57b362621f68a8db147e70c Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Mon, 30 Dec 2024 09:58:04 -0600 Subject: [PATCH 05/73] poc reader --- parquet/src/arrow/arrow_reader/selection.rs | 4 - .../src/arrow/async_reader/arrow_reader.rs | 86 ++++++++++++------- parquet/src/arrow/async_reader/mod.rs | 30 ++++--- 3 files changed, 71 insertions(+), 49 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index f83724a3841b..378d2253f19a 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -441,10 +441,6 @@ impl RowSelection { pub fn skipped_row_count(&self) -> usize { self.iter().filter(|s| s.skip).map(|s| s.row_count).sum() } - - pub(crate) fn extend(&mut self, other: Self) { - self.selectors.extend(other.selectors); - } } impl From> for RowSelection { diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 04cc115ce39c..f2f681cc7d37 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -19,7 +19,8 @@ use std::collections::HashMap; use std::sync::RwLock; use std::{collections::VecDeque, sync::Arc}; -use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader, StructArray}; +use arrow_array::ArrayRef; +use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; use arrow_select::filter::prep_null_mask_filter; @@ -45,7 +46,7 @@ pub struct FilteredParquetRecordBatchReader { fn read_selection( reader: &mut dyn ArrayReader, selection: &RowSelection, -) -> Result { +) -> Result { for selector in selection.iter() { if selector.skip { let skipped = reader.skip_records(selector.row_count)?; @@ -55,11 +56,7 @@ fn read_selection( debug_assert_eq!(read_records, selector.row_count, "failed to read rows"); } } - let array = reader.consume_batch()?; - let struct_array = array - .as_struct_opt() - .ok_or_else(|| general_err!("Struct array reader should return struct array"))?; - Ok(struct_array.clone()) + reader.consume_batch() } /// Take the next selection from the selection queue, and return the selection @@ -142,7 +139,9 @@ impl FilteredParquetRecordBatchReader { .zip(self.predicate_readers.iter_mut()) { let array = read_selection(reader.as_mut(), &selection)?; - let batch = RecordBatch::from(array); + let batch = RecordBatch::from(array.as_struct_opt().ok_or_else(|| { + general_err!("Struct array reader should return struct array") + })?); let input_rows = batch.num_rows(); let predicate_filter = predicate.evaluate(batch)?; if predicate_filter.len() != input_rows { @@ -178,7 +177,6 @@ impl Iterator for FilteredParquetRecordBatchReader { // It boils down to leveraging array_reader's ability to collect large batches natively, // rather than concatenating multiple small batches. - let mut selection = RowSelection::default(); let mut selected = 0; while let Some(cur_selection) = take_next_selection(&mut self.selection, self.batch_size - selected) @@ -187,21 +185,29 @@ impl Iterator for FilteredParquetRecordBatchReader { Ok(selection) => selection, Err(e) => return Some(Err(e)), }; + + for selector in filtered_selection.iter() { + if selector.skip { + self.array_reader.skip_records(selector.row_count).ok()?; + } else { + self.array_reader.read_records(selector.row_count).ok()?; + } + } selected += filtered_selection.row_count(); - selection.extend(filtered_selection); if selected >= (self.batch_size / 4 * 3) { break; } } - if !selection.selects_any() { + if selected == 0 { return None; } - let rt = read_selection(&mut *self.array_reader, &selection); - match rt { - Ok(array) => Some(Ok(RecordBatch::from(array))), - Err(e) => Some(Err(e.into())), - } + let array = self.array_reader.consume_batch().ok()?; + let struct_array = array + .as_struct_opt() + .ok_or_else(|| general_err!("Struct array reader should return struct array")) + .ok()?; + Some(Ok(RecordBatch::from(struct_array.clone()))) } } @@ -212,11 +218,11 @@ impl RecordBatchReader for FilteredParquetRecordBatchReader { } struct PageCacheInner { - queue: VecDeque, - pages: HashMap, + pages: HashMap, // col_id -> (offset, page) } -/// A simple FIFO cache for pages. +/// A simple cache for decompressed pages. +/// We cache only one page per column pub(crate) struct PageCache { inner: RwLock, } @@ -227,36 +233,49 @@ impl PageCache { pub(crate) fn new() -> Self { Self { inner: RwLock::new(PageCacheInner { - queue: VecDeque::with_capacity(Self::CAPACITY), pages: HashMap::with_capacity(Self::CAPACITY), }), } } - pub(crate) fn get_page(&self, offset: usize) -> Option { + pub(crate) fn get_page(&self, col_id: usize, offset: usize) -> Option { let read_lock = self.inner.read().unwrap(); - read_lock.pages.get(&offset).cloned() + read_lock + .pages + .get(&col_id) + .and_then(|(cached_offset, page)| { + if *cached_offset == offset { + Some(page) + } else { + None + } + }) + .cloned() } - pub(crate) fn insert_page(&self, offset: usize, page: Page) { + pub(crate) fn insert_page(&self, col_id: usize, offset: usize, page: Page) { let mut write_lock = self.inner.write().unwrap(); - if write_lock.pages.len() >= Self::CAPACITY { - let oldest_offset = write_lock.queue.pop_front().unwrap(); - write_lock.pages.remove(&oldest_offset).unwrap(); - } - write_lock.pages.insert(offset, page); - write_lock.queue.push_back(offset); + write_lock.pages.insert(col_id, (offset, page)); } } pub(crate) struct CachedPageReader { inner: SerializedPageReader, cache: Arc, + col_id: usize, } impl CachedPageReader { - pub(crate) fn new(inner: SerializedPageReader, cache: Arc) -> Self { - Self { inner, cache } + pub(crate) fn new( + inner: SerializedPageReader, + cache: Arc, + col_id: usize, + ) -> Self { + Self { + inner, + cache, + col_id, + } } } @@ -277,7 +296,7 @@ impl PageReader for CachedPageReader { return Ok(None); }; - let page = self.cache.get_page(offset); + let page = self.cache.get_page(self.col_id, offset); if let Some(page) = page { self.inner.skip_next_page()?; Ok(Some(page)) @@ -286,7 +305,8 @@ impl PageReader for CachedPageReader { let Some(inner_page) = inner_page else { return Ok(None); }; - self.cache.insert_page(offset, inner_page.clone()); + self.cache + .insert_page(self.col_id, offset, inner_page.clone()); Ok(Some(inner_page)) } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 3ee85a26fff2..cf48b391ed68 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -911,22 +911,28 @@ impl RowGroups for InMemoryRowGroup<'_> { .filter(|index| !index.is_empty()) .map(|index| index[i].page_locations.clone()); - // let page_reader: Box = Box::new(SerializedPageReader::new( - // data.clone(), - // self.metadata.column(i), - // self.row_count, - // page_locations, - // )?); - - let page_reader: Box = Box::new(CachedPageReader::new( - SerializedPageReader::new( + let page_reader: Box = if std::env::var("CACHE_PAGES") + .map(|v| v == "1") + .unwrap_or(false) + { + Box::new(CachedPageReader::new( + SerializedPageReader::new( + data.clone(), + self.metadata.column(i), + self.row_count, + page_locations, + )?, + self.cache.clone(), + i, + )) + } else { + Box::new(SerializedPageReader::new( data.clone(), self.metadata.column(i), self.row_count, page_locations, - )?, - self.cache.clone(), - )); + )?) + }; Ok(Box::new(ColumnChunkIterator { reader: Some(Ok(page_reader)), From f5422ced6c46c99f43978c96ad5b18fe13b58ceb Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 31 Dec 2024 09:37:59 -0600 Subject: [PATCH 06/73] update --- parquet/Cargo.toml | 1 + .../src/arrow/array_reader/byte_view_array.rs | 6 +- .../src/arrow/async_reader/arrow_reader.rs | 58 ++++++++++++++----- parquet/src/arrow/buffer/view_buffer.rs | 7 +++ 4 files changed, 56 insertions(+), 16 deletions(-) diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index e4085472ea20..ba5d8c3d6949 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -69,6 +69,7 @@ paste = { version = "1.0" } half = { version = "2.1", default-features = false, features = ["num-traits"] } sysinfo = { version = "0.33.0", optional = true, default-features = false, features = ["system"] } crc32fast = { version = "1.4.2", optional = true, default-features = false } +simdutf8 = "0.1.5" [dev-dependencies] base64 = { version = "0.22", default-features = false, features = ["std"] } diff --git a/parquet/src/arrow/array_reader/byte_view_array.rs b/parquet/src/arrow/array_reader/byte_view_array.rs index 92a8b0592d0d..cff55d192454 100644 --- a/parquet/src/arrow/array_reader/byte_view_array.rs +++ b/parquet/src/arrow/array_reader/byte_view_array.rs @@ -161,7 +161,7 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { )); } - let mut buffer = ViewBuffer::default(); + let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); let mut decoder = ByteViewArrayDecoderPlain::new( buf, num_values as usize, @@ -457,6 +457,8 @@ impl ByteViewArrayDecoderDictionary { } } + output.views.reserve(len); + // Calculate the offset of the dictionary buffers in the output buffers // For example if the 2nd buffer in the dictionary is the 5th buffer in the output buffers, // then the base_buffer_idx is 5 - 2 = 3 @@ -677,7 +679,7 @@ impl ByteViewArrayDecoderDelta { /// Check that `val` is a valid UTF-8 sequence pub fn check_valid_utf8(val: &[u8]) -> Result<()> { - match std::str::from_utf8(val) { + match simdutf8::basic::from_utf8(val) { Ok(_) => Ok(()), Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)), } diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index f2f681cc7d37..91e651b77e29 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::hash_map::Entry; use std::collections::HashMap; use std::sync::RwLock; use std::{collections::VecDeque, sync::Arc}; @@ -24,6 +25,7 @@ use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; use arrow_select::filter::prep_null_mask_filter; +use crate::basic::PageType; use crate::column::page::{Page, PageMetadata, PageReader}; use crate::errors::ParquetError; use crate::{ @@ -217,12 +219,17 @@ impl RecordBatchReader for FilteredParquetRecordBatchReader { } } +struct CachedPage { + dict: Option<(usize, Page)>, + data: Option<(usize, Page)>, +} + struct PageCacheInner { - pages: HashMap, // col_id -> (offset, page) + pages: HashMap, // col_id -> CachedPage } /// A simple cache for decompressed pages. -/// We cache only one page per column +/// We cache only one dictionary page and one data page per column pub(crate) struct PageCache { inner: RwLock, } @@ -240,22 +247,45 @@ impl PageCache { pub(crate) fn get_page(&self, col_id: usize, offset: usize) -> Option { let read_lock = self.inner.read().unwrap(); - read_lock - .pages - .get(&col_id) - .and_then(|(cached_offset, page)| { - if *cached_offset == offset { - Some(page) - } else { - None - } - }) - .cloned() + read_lock.pages.get(&col_id).and_then(|pages| { + pages + .dict + .iter() + .chain(pages.data.iter()) + .find(|(page_offset, _)| *page_offset == offset) + .map(|(_, page)| page.clone()) + }) } pub(crate) fn insert_page(&self, col_id: usize, offset: usize, page: Page) { let mut write_lock = self.inner.write().unwrap(); - write_lock.pages.insert(col_id, (offset, page)); + + let is_dict = page.page_type() == PageType::DICTIONARY_PAGE; + + let cached_pages = write_lock.pages.entry(col_id); + match cached_pages { + Entry::Occupied(mut entry) => { + if is_dict { + entry.get_mut().dict = Some((offset, page)); + } else { + entry.get_mut().data = Some((offset, page)); + } + } + Entry::Vacant(entry) => { + let cached_page = if is_dict { + CachedPage { + dict: Some((offset, page)), + data: None, + } + } else { + CachedPage { + dict: None, + data: Some((offset, page)), + } + }; + entry.insert(cached_page); + } + } } } diff --git a/parquet/src/arrow/buffer/view_buffer.rs b/parquet/src/arrow/buffer/view_buffer.rs index fd7d6c213f04..a21d153df76f 100644 --- a/parquet/src/arrow/buffer/view_buffer.rs +++ b/parquet/src/arrow/buffer/view_buffer.rs @@ -33,6 +33,13 @@ pub struct ViewBuffer { } impl ViewBuffer { + pub fn with_capacity(view_capacity: usize, buffer_capacity: usize) -> Self { + Self { + views: Vec::with_capacity(view_capacity), + buffers: Vec::with_capacity(buffer_capacity), + } + } + pub fn is_empty(&self) -> bool { self.views.is_empty() } From dfdc1b644bcce4e4e74b05f8cc123b8c601b4518 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 31 Dec 2024 13:04:35 -0600 Subject: [PATCH 07/73] avoid recreating new buffers --- .../src/arrow/array_reader/byte_view_array.rs | 74 +++++++++++++++---- 1 file changed, 58 insertions(+), 16 deletions(-) diff --git a/parquet/src/arrow/array_reader/byte_view_array.rs b/parquet/src/arrow/array_reader/byte_view_array.rs index cff55d192454..45cfc5715a4d 100644 --- a/parquet/src/arrow/array_reader/byte_view_array.rs +++ b/parquet/src/arrow/array_reader/byte_view_array.rs @@ -33,6 +33,9 @@ use arrow_data::ByteView; use arrow_schema::DataType as ArrowType; use bytes::Bytes; use std::any::Any; +use std::collections::hash_map::Entry; +use std::collections::HashMap; +use std::sync::{Arc, LazyLock, Mutex}; /// Returns an [`ArrayReader`] that decodes the provided byte array column to view types. pub fn make_byte_view_array_reader( @@ -127,11 +130,14 @@ impl ArrayReader for ByteViewArrayReader { /// A [`ColumnValueDecoder`] for variable length byte arrays struct ByteViewArrayColumnValueDecoder { - dict: Option, + dict: Option>, decoder: Option, validate_utf8: bool, } +pub(crate) static DICT_CACHE: LazyLock>>> = + LazyLock::new(|| Mutex::new(HashMap::new())); + impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { type Buffer = ViewBuffer; @@ -144,6 +150,7 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { } } + #[inline(never)] fn set_dict( &mut self, buf: Bytes, @@ -161,18 +168,35 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { )); } - let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); - let mut decoder = ByteViewArrayDecoderPlain::new( - buf, - num_values as usize, - Some(num_values as usize), - self.validate_utf8, - ); - decoder.read(&mut buffer, usize::MAX)?; - self.dict = Some(buffer); + let buf_id = buf.as_ptr() as usize; + + let mut cache = DICT_CACHE.lock().unwrap(); + + match cache.entry(buf_id) { + Entry::Vacant(v) => { + let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); + let mut decoder = ByteViewArrayDecoderPlain::new( + buf, + num_values as usize, + Some(num_values as usize), + self.validate_utf8, + ); + decoder.read(&mut buffer, usize::MAX)?; + + let dict = Arc::new(buffer); + v.insert(dict.clone()); + self.dict = Some(dict); + } + Entry::Occupied(e) => { + // Remove and take ownership of the existing dictionary + self.dict = Some(e.remove()); + // self.dict = Some(e.get().clone()); + } + } Ok(()) } + #[inline(never)] fn set_data( &mut self, encoding: Encoding, @@ -190,22 +214,24 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { Ok(()) } + #[inline(never)] fn read(&mut self, out: &mut Self::Buffer, num_values: usize) -> Result { let decoder = self .decoder .as_mut() .ok_or_else(|| general_err!("no decoder set"))?; - decoder.read(out, num_values, self.dict.as_ref()) + decoder.read(out, num_values, self.dict.as_ref().map(|b| b.as_ref())) } + #[inline(never)] fn skip_values(&mut self, num_values: usize) -> Result { let decoder = self .decoder .as_mut() .ok_or_else(|| general_err!("no decoder set"))?; - decoder.skip(num_values, self.dict.as_ref()) + decoder.skip(num_values, self.dict.as_ref().map(|b| b.as_ref())) } } @@ -255,6 +281,7 @@ impl ByteViewArrayDecoder { } /// Read up to `len` values to `out` with the optional dictionary + #[inline(never)] pub fn read( &mut self, out: &mut ViewBuffer, @@ -290,7 +317,7 @@ impl ByteViewArrayDecoder { /// Decoder from [`Encoding::PLAIN`] data to [`ViewBuffer`] pub struct ByteViewArrayDecoderPlain { - buf: Bytes, + buf: Buffer, offset: usize, validate_utf8: bool, @@ -307,6 +334,9 @@ impl ByteViewArrayDecoderPlain { num_values: Option, validate_utf8: bool, ) -> Self { + // Here we convert `bytes::Bytes` into `arrow_buffer::Bytes`, which is zero copy + // Then we convert `arrow_buffer::Bytes` into `arrow_buffer:Buffer`, which is also zero copy + let buf = arrow_buffer::Buffer::from_bytes(buf.clone().into()); Self { buf, offset: 0, @@ -315,10 +345,21 @@ impl ByteViewArrayDecoderPlain { } } + #[inline(never)] pub fn read(&mut self, output: &mut ViewBuffer, len: usize) -> Result { - // Zero copy convert `bytes::Bytes` into `arrow_buffer::Buffer` - let buf = arrow_buffer::Buffer::from(self.buf.clone()); - let block_id = output.append_block(buf); + let need_to_create_new_buffer = { + if let Some(last_buffer) = output.buffers.last() { + last_buffer.ptr_eq(&self.buf) + } else { + true + } + }; + + let block_id = if need_to_create_new_buffer { + output.append_block(self.buf.clone()) + } else { + output.buffers.len() as u32 - 1 + }; let to_read = len.min(self.max_remaining_values); @@ -432,6 +473,7 @@ impl ByteViewArrayDecoderDictionary { /// Assumptions / Optimization /// This function checks if dict.buffers() are the last buffers in `output`, and if so /// reuses the dictionary page buffers directly without copying data + #[inline(never)] fn read(&mut self, output: &mut ViewBuffer, dict: &ViewBuffer, len: usize) -> Result { if dict.is_empty() || len == 0 { return Ok(0); From 3c526f8ff3a36182914a23d060436a1acd6c5656 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 31 Dec 2024 14:02:34 -0600 Subject: [PATCH 08/73] update --- .../src/arrow/array_reader/byte_view_array.rs | 72 +++++++++++-------- 1 file changed, 42 insertions(+), 30 deletions(-) diff --git a/parquet/src/arrow/array_reader/byte_view_array.rs b/parquet/src/arrow/array_reader/byte_view_array.rs index 45cfc5715a4d..3b6d9f8a22fd 100644 --- a/parquet/src/arrow/array_reader/byte_view_array.rs +++ b/parquet/src/arrow/array_reader/byte_view_array.rs @@ -33,9 +33,9 @@ use arrow_data::ByteView; use arrow_schema::DataType as ArrowType; use bytes::Bytes; use std::any::Any; -use std::collections::hash_map::Entry; -use std::collections::HashMap; -use std::sync::{Arc, LazyLock, Mutex}; +// use std::collections::hash_map::Entry; +// use std::collections::HashMap; +use std::sync::Arc; /// Returns an [`ArrayReader`] that decodes the provided byte array column to view types. pub fn make_byte_view_array_reader( @@ -135,8 +135,8 @@ struct ByteViewArrayColumnValueDecoder { validate_utf8: bool, } -pub(crate) static DICT_CACHE: LazyLock>>> = - LazyLock::new(|| Mutex::new(HashMap::new())); +// pub(crate) static DICT_CACHE: LazyLock>>> = +// LazyLock::new(|| Mutex::new(HashMap::new())); impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { type Buffer = ViewBuffer; @@ -168,31 +168,43 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { )); } - let buf_id = buf.as_ptr() as usize; - - let mut cache = DICT_CACHE.lock().unwrap(); - - match cache.entry(buf_id) { - Entry::Vacant(v) => { - let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); - let mut decoder = ByteViewArrayDecoderPlain::new( - buf, - num_values as usize, - Some(num_values as usize), - self.validate_utf8, - ); - decoder.read(&mut buffer, usize::MAX)?; - - let dict = Arc::new(buffer); - v.insert(dict.clone()); - self.dict = Some(dict); - } - Entry::Occupied(e) => { - // Remove and take ownership of the existing dictionary - self.dict = Some(e.remove()); - // self.dict = Some(e.get().clone()); - } - } + let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); + let mut decoder = ByteViewArrayDecoderPlain::new( + buf, + num_values as usize, + Some(num_values as usize), + self.validate_utf8, + ); + decoder.read(&mut buffer, usize::MAX)?; + + let dict = Arc::new(buffer); + self.dict = Some(dict); + + // let buf_id = buf.as_ptr() as usize; + + // let mut cache = DICT_CACHE.lock().unwrap(); + + // match cache.entry(buf_id) { + // Entry::Vacant(v) => { + // let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); + // let mut decoder = ByteViewArrayDecoderPlain::new( + // buf, + // num_values as usize, + // Some(num_values as usize), + // self.validate_utf8, + // ); + // decoder.read(&mut buffer, usize::MAX)?; + + // let dict = Arc::new(buffer); + // v.insert(dict.clone()); + // self.dict = Some(dict); + // } + // Entry::Occupied(e) => { + // // Remove and take ownership of the existing dictionary + // self.dict = Some(e.remove()); + // // self.dict = Some(e.get().clone()); + // } + // } Ok(()) } From 53f5fad772578f74dd7e9a4b05945da0cf9205c5 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 31 Dec 2024 14:26:56 -0600 Subject: [PATCH 09/73] bug fix --- .../src/arrow/array_reader/byte_view_array.rs | 52 +++---------------- 1 file changed, 7 insertions(+), 45 deletions(-) diff --git a/parquet/src/arrow/array_reader/byte_view_array.rs b/parquet/src/arrow/array_reader/byte_view_array.rs index 3b6d9f8a22fd..a4366877dcb7 100644 --- a/parquet/src/arrow/array_reader/byte_view_array.rs +++ b/parquet/src/arrow/array_reader/byte_view_array.rs @@ -33,9 +33,6 @@ use arrow_data::ByteView; use arrow_schema::DataType as ArrowType; use bytes::Bytes; use std::any::Any; -// use std::collections::hash_map::Entry; -// use std::collections::HashMap; -use std::sync::Arc; /// Returns an [`ArrayReader`] that decodes the provided byte array column to view types. pub fn make_byte_view_array_reader( @@ -130,14 +127,11 @@ impl ArrayReader for ByteViewArrayReader { /// A [`ColumnValueDecoder`] for variable length byte arrays struct ByteViewArrayColumnValueDecoder { - dict: Option>, + dict: Option, decoder: Option, validate_utf8: bool, } -// pub(crate) static DICT_CACHE: LazyLock>>> = -// LazyLock::new(|| Mutex::new(HashMap::new())); - impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { type Buffer = ViewBuffer; @@ -150,7 +144,6 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { } } - #[inline(never)] fn set_dict( &mut self, buf: Bytes, @@ -177,38 +170,11 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { ); decoder.read(&mut buffer, usize::MAX)?; - let dict = Arc::new(buffer); - self.dict = Some(dict); - - // let buf_id = buf.as_ptr() as usize; - - // let mut cache = DICT_CACHE.lock().unwrap(); - - // match cache.entry(buf_id) { - // Entry::Vacant(v) => { - // let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); - // let mut decoder = ByteViewArrayDecoderPlain::new( - // buf, - // num_values as usize, - // Some(num_values as usize), - // self.validate_utf8, - // ); - // decoder.read(&mut buffer, usize::MAX)?; - - // let dict = Arc::new(buffer); - // v.insert(dict.clone()); - // self.dict = Some(dict); - // } - // Entry::Occupied(e) => { - // // Remove and take ownership of the existing dictionary - // self.dict = Some(e.remove()); - // // self.dict = Some(e.get().clone()); - // } - // } + self.dict = Some(buffer); + Ok(()) } - #[inline(never)] fn set_data( &mut self, encoding: Encoding, @@ -226,24 +192,22 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { Ok(()) } - #[inline(never)] fn read(&mut self, out: &mut Self::Buffer, num_values: usize) -> Result { let decoder = self .decoder .as_mut() .ok_or_else(|| general_err!("no decoder set"))?; - decoder.read(out, num_values, self.dict.as_ref().map(|b| b.as_ref())) + decoder.read(out, num_values, self.dict.as_ref()) } - #[inline(never)] fn skip_values(&mut self, num_values: usize) -> Result { let decoder = self .decoder .as_mut() .ok_or_else(|| general_err!("no decoder set"))?; - decoder.skip(num_values, self.dict.as_ref().map(|b| b.as_ref())) + decoder.skip(num_values, self.dict.as_ref()) } } @@ -293,7 +257,6 @@ impl ByteViewArrayDecoder { } /// Read up to `len` values to `out` with the optional dictionary - #[inline(never)] pub fn read( &mut self, out: &mut ViewBuffer, @@ -357,11 +320,11 @@ impl ByteViewArrayDecoderPlain { } } - #[inline(never)] pub fn read(&mut self, output: &mut ViewBuffer, len: usize) -> Result { + // let block_id = output.append_block(self.buf.clone()); let need_to_create_new_buffer = { if let Some(last_buffer) = output.buffers.last() { - last_buffer.ptr_eq(&self.buf) + !last_buffer.ptr_eq(&self.buf) } else { true } @@ -485,7 +448,6 @@ impl ByteViewArrayDecoderDictionary { /// Assumptions / Optimization /// This function checks if dict.buffers() are the last buffers in `output`, and if so /// reuses the dictionary page buffers directly without copying data - #[inline(never)] fn read(&mut self, output: &mut ViewBuffer, dict: &ViewBuffer, len: usize) -> Result { if dict.is_empty() || len == 0 { return Ok(0); From 56980defc5f52fd315d452d7f1f1c1df1a9e1d84 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 31 Dec 2024 22:04:18 -0600 Subject: [PATCH 10/73] selective cache --- parquet/src/arrow/async_reader/mod.rs | 35 +++++++++++++++++++++------ 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index cf48b391ed68..f490bb6d5bc5 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -512,6 +512,22 @@ where .filter(|index| index.first().map(|v| !v.is_empty()).unwrap_or(false)) .map(|x| x[row_group_idx].as_slice()); + let mut predicate_projection: Option = None; + if let Some(filter) = self.filter.as_mut() { + for predicate in filter.predicates.iter_mut() { + let p_projection = predicate.projection(); + if let Some(ref mut p) = predicate_projection { + p.union(&p_projection); + } else { + predicate_projection = Some(p_projection.clone()); + } + } + } + let projection_to_cache = predicate_projection.map(|mut p| { + p.intersect(&projection); + p + }); + let mut row_group = InMemoryRowGroup { metadata: meta, // schema: meta.schema_descr_ptr(), @@ -519,6 +535,7 @@ where column_chunks: vec![None; meta.columns().len()], offset_index, cache: Arc::new(PageCache::new()), + projection_to_cache, }; let mut selection = @@ -531,13 +548,13 @@ where return Ok((self, None)); } - let predicate_projection = predicate.projection(); + let p_projection = predicate.projection(); row_group - .fetch(&mut self.input, predicate_projection, Some(&selection)) + .fetch(&mut self.input, p_projection, Some(&selection)) .await?; let array_reader = - build_array_reader(self.fields.as_deref(), predicate_projection, &row_group)?; + build_array_reader(self.fields.as_deref(), p_projection, &row_group)?; filter_readers.push(array_reader); } } @@ -799,6 +816,7 @@ struct InMemoryRowGroup<'a> { column_chunks: Vec>>, row_count: usize, cache: Arc, + projection_to_cache: Option, } impl InMemoryRowGroup<'_> { @@ -911,10 +929,13 @@ impl RowGroups for InMemoryRowGroup<'_> { .filter(|index| !index.is_empty()) .map(|index| index[i].page_locations.clone()); - let page_reader: Box = if std::env::var("CACHE_PAGES") - .map(|v| v == "1") - .unwrap_or(false) - { + let cached_reader = if let Some(projection_to_cache) = &self.projection_to_cache { + projection_to_cache.leaf_included(i) + } else { + false + }; + + let page_reader: Box = if cached_reader { Box::new(CachedPageReader::new( SerializedPageReader::new( data.clone(), From 4dd1b6b8cb6b7f988cb0c6d06ff2e90f62bbc68d Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 31 Dec 2024 22:27:14 -0600 Subject: [PATCH 11/73] clean up changes --- parquet/src/arrow/array_reader/builder.rs | 6 ------ parquet/src/arrow/array_reader/primitive_array.rs | 10 ---------- 2 files changed, 16 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 23f77a9ab96f..945f62526a7e 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -245,7 +245,6 @@ fn build_primitive_reader( page_iterator, column_desc, arrow_type, - col_idx, )?) as _, PhysicalType::INT32 => { if let Some(DataType::Null) = arrow_type { @@ -258,7 +257,6 @@ fn build_primitive_reader( page_iterator, column_desc, arrow_type, - col_idx, )?) as _ } } @@ -266,25 +264,21 @@ fn build_primitive_reader( page_iterator, column_desc, arrow_type, - col_idx, )?) as _, PhysicalType::INT96 => Box::new(PrimitiveArrayReader::::new( page_iterator, column_desc, arrow_type, - col_idx, )?) as _, PhysicalType::FLOAT => Box::new(PrimitiveArrayReader::::new( page_iterator, column_desc, arrow_type, - col_idx, )?) as _, PhysicalType::DOUBLE => Box::new(PrimitiveArrayReader::::new( page_iterator, column_desc, arrow_type, - col_idx, )?) as _, PhysicalType::BYTE_ARRAY => match arrow_type { Some(DataType::Dictionary(_, _)) => { diff --git a/parquet/src/arrow/array_reader/primitive_array.rs b/parquet/src/arrow/array_reader/primitive_array.rs index c665cee5bf26..6fe49945cffd 100644 --- a/parquet/src/arrow/array_reader/primitive_array.rs +++ b/parquet/src/arrow/array_reader/primitive_array.rs @@ -80,8 +80,6 @@ where def_levels_buffer: Option>, rep_levels_buffer: Option>, record_reader: RecordReader, - #[allow(unused)] - column_idx: usize, } impl PrimitiveArrayReader @@ -95,7 +93,6 @@ where pages: Box, column_desc: ColumnDescPtr, arrow_type: Option, - column_idx: usize, ) -> Result { // Check if Arrow type is specified, else create it from Parquet type let data_type = match arrow_type { @@ -113,7 +110,6 @@ where def_levels_buffer: None, rep_levels_buffer: None, record_reader, - column_idx, }) } } @@ -375,7 +371,6 @@ mod tests { Box::::default(), schema.column(0), None, - 0, ) .unwrap(); @@ -422,7 +417,6 @@ mod tests { Box::new(page_iterator), column_desc, None, - 0, ) .unwrap(); @@ -493,7 +487,6 @@ mod tests { Box::new(page_iterator), column_desc.clone(), None, - 0, ) .expect("Unable to get array reader"); @@ -633,7 +626,6 @@ mod tests { Box::new(page_iterator), column_desc, None, - 0, ) .unwrap(); @@ -713,7 +705,6 @@ mod tests { Box::new(page_iterator), column_desc, None, - 0, ) .unwrap(); @@ -776,7 +767,6 @@ mod tests { Box::new(page_iterator), column_desc, None, - 0, ) .unwrap(); From f8f983e5f9ea66b18656a0588e58e9dc4b7c4658 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 1 Jan 2025 10:37:02 -0600 Subject: [PATCH 12/73] clean up more and format --- .../src/arrow/array_reader/primitive_array.rs | 36 +++++++------------ .../src/arrow/array_reader/struct_array.rs | 2 +- parquet/src/file/serialized_reader.rs | 13 ++++--- 3 files changed, 19 insertions(+), 32 deletions(-) diff --git a/parquet/src/arrow/array_reader/primitive_array.rs b/parquet/src/arrow/array_reader/primitive_array.rs index 6fe49945cffd..a952e00e12ef 100644 --- a/parquet/src/arrow/array_reader/primitive_array.rs +++ b/parquet/src/arrow/array_reader/primitive_array.rs @@ -413,12 +413,9 @@ mod tests { ); let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = PrimitiveArrayReader::::new( - Box::new(page_iterator), - column_desc, - None, - ) - .unwrap(); + let mut array_reader = + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + .unwrap(); // Read first 50 values, which are all from the first column chunk let array = array_reader.next_batch(50).unwrap(); @@ -622,12 +619,9 @@ mod tests { let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = PrimitiveArrayReader::::new( - Box::new(page_iterator), - column_desc, - None, - ) - .unwrap(); + let mut array_reader = + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + .unwrap(); let mut accu_len: usize = 0; @@ -701,12 +695,9 @@ mod tests { ); let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = PrimitiveArrayReader::::new( - Box::new(page_iterator), - column_desc, - None, - ) - .unwrap(); + let mut array_reader = + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + .unwrap(); // read data from the reader // the data type is decimal(8,2) @@ -763,12 +754,9 @@ mod tests { ); let page_iterator = InMemoryPageIterator::new(page_lists); - let mut array_reader = PrimitiveArrayReader::::new( - Box::new(page_iterator), - column_desc, - None, - ) - .unwrap(); + let mut array_reader = + PrimitiveArrayReader::::new(Box::new(page_iterator), column_desc, None) + .unwrap(); // read data from the reader // the data type is decimal(18,4) diff --git a/parquet/src/arrow/array_reader/struct_array.rs b/parquet/src/arrow/array_reader/struct_array.rs index e048fbae66fa..fb2f2f8928b9 100644 --- a/parquet/src/arrow/array_reader/struct_array.rs +++ b/parquet/src/arrow/array_reader/struct_array.rs @@ -25,7 +25,7 @@ use std::sync::Arc; /// Implementation of struct array reader. pub struct StructArrayReader { - pub children: Vec>, + children: Vec>, data_type: ArrowType, struct_def_level: i16, struct_rep_level: i16, diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 146b16f38422..7d173bc0c16a 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -371,7 +371,7 @@ fn read_page_header_len(input: &mut T) -> Result<(usize, PageHeader)> { /// Decodes a [`Page`] from the provided `buffer` pub(crate) fn decode_page( page_header: PageHeader, - buffer: Vec, + buffer: Bytes, physical_type: Type, decompressor: Option<&mut Box>, ) -> Result { @@ -406,8 +406,8 @@ pub(crate) fn decode_page( Some(decompressor) if can_decompress => { let uncompressed_size = page_header.uncompressed_page_size as usize; let mut decompressed = Vec::with_capacity(uncompressed_size); - let compressed = &buffer[offset..]; - decompressed.extend_from_slice(&buffer[..offset]); + let compressed = &buffer.as_ref()[offset..]; + decompressed.extend_from_slice(&buffer.as_ref()[..offset]); decompressor.decompress( compressed, &mut decompressed, @@ -421,8 +421,7 @@ pub(crate) fn decode_page( uncompressed_size )); } - - decompressed + Bytes::from(decompressed) } _ => buffer, }; @@ -700,7 +699,7 @@ impl PageReader for SerializedPageReader { decode_page( header, - buffer, + Bytes::from(buffer), self.physical_type, self.decompressor.as_mut(), )? @@ -729,7 +728,7 @@ impl PageReader for SerializedPageReader { let bytes = buffer.slice(offset..); decode_page( header, - bytes.to_vec(), + bytes, self.physical_type, self.decompressor.as_mut(), )? From 882aaf14eba48d02846697f7739f6b106021c581 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 1 Jan 2025 11:32:11 -0600 Subject: [PATCH 13/73] cleanup and add docs --- .../src/arrow/async_reader/arrow_reader.rs | 181 ++++++++++++++++-- parquet/src/arrow/async_reader/mod.rs | 14 +- parquet/src/file/serialized_reader.rs | 2 +- 3 files changed, 169 insertions(+), 28 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 91e651b77e29..44f327ee8538 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -220,27 +220,74 @@ impl RecordBatchReader for FilteredParquetRecordBatchReader { } struct CachedPage { - dict: Option<(usize, Page)>, - data: Option<(usize, Page)>, + dict: Option<(usize, Page)>, // page offset -> page + data: Option<(usize, Page)>, // page offset -> page } -struct PageCacheInner { - pages: HashMap, // col_id -> CachedPage +struct PredicatePageCacheInner { + pages: HashMap, // col_id (Parquet's leaf column index) -> CachedPage } -/// A simple cache for decompressed pages. -/// We cache only one dictionary page and one data page per column -pub(crate) struct PageCache { - inner: RwLock, +/// A simple cache to avoid double-decompressing pages with filter pushdown. +/// In filter pushdown, we first decompress a page, apply the filter, and then decompress the page again. +/// This double decompression is expensive, so we cache the decompressed page. +/// +/// This implementation contains subtle dynamics that can be hard to understand. +/// +/// ## Which columns to cache +/// +/// Let's consider this example: SELECT B, C FROM table WHERE A = 42 and B = 37; +/// We have 3 columns, and the predicate is applied to column A and B, and projection is on B and C. +/// +/// For column A, we need to decompress it, apply the filter (A=42), and never have to decompress it again, as it's not in the projection. +/// For column B, we need to decompress it, apply the filter (B=37), and then decompress it again, as it's in the projection. +/// For column C, we don't have predicate, so we only decompress it once. +/// +/// A, C is only decompressed once, and B is decompressed twice (as it appears in both the predicate and the projection). +/// The PredicatePageCache will only cache B. +/// We use B's col_id (Parquet's leaf column index) to identify the cache entry. +/// +/// ## How many pages to cache +/// +/// Now we identified the columns to cache, next question is to determine the **minimal** number of pages to cache. +/// +/// Let's revisit our decoding pipeline: +/// Load batch 1 -> evaluate predicates -> filter 1 -> load & emit batch 1 +/// Load batch 2 -> evaluate predicates -> filter 2 -> load & emit batch 2 +/// ... +/// Load batch N -> evaluate predicates -> filter N -> load & emit batch N +/// +/// Assumption & observation: each page consists multiple batches. +/// Then our pipeline looks like this: +/// Load Page 1 +/// Load batch 1 -> evaluate predicates -> filter 1 -> load & emit batch 1 +/// Load batch 2 -> evaluate predicates -> filter 2 -> load & emit batch 2 +/// Load batch 3 -> evaluate predicates -> filter 3 -> load & emit batch 3 +/// Load Page 2 +/// Load batch 4 -> evaluate predicates -> filter 4 -> load & emit batch 4 +/// Load batch 5 -> evaluate predicates -> filter 5 -> load & emit batch 5 +/// ... +/// +/// This means that we only need to cache one page per column, +/// because the page that is used by the predicate is the same page, and is immediately used in loading the batch. +/// +/// The only exception is the dictionary page -- the first page of each column. +/// If we encountered a dict page, we will need to immediately read next page, and cache it. +/// +/// To summarize, the cache only contains 2 pages per column: one dict page and one data page. +/// This is a nice property as it means the caching memory consumption is negligible and constant to the number of columns. +/// +/// ## How to identify a page +/// We use the page offset (the offset to the Parquet file) to uniquely identify a page. +pub(crate) struct PredicatePageCache { + inner: RwLock, } -impl PageCache { - const CAPACITY: usize = 16; - - pub(crate) fn new() -> Self { +impl PredicatePageCache { + pub(crate) fn new(capacity: usize) -> Self { Self { - inner: RwLock::new(PageCacheInner { - pages: HashMap::with_capacity(Self::CAPACITY), + inner: RwLock::new(PredicatePageCacheInner { + pages: HashMap::with_capacity(capacity), }), } } @@ -257,6 +304,9 @@ impl PageCache { }) } + /// Insert a page into the cache. + /// Inserting a page will override the existing page, if any. + /// This is because we only need to cache 2 pages per column, see above. pub(crate) fn insert_page(&self, col_id: usize, offset: usize, page: Page) { let mut write_lock = self.inner.write().unwrap(); @@ -291,14 +341,14 @@ impl PageCache { pub(crate) struct CachedPageReader { inner: SerializedPageReader, - cache: Arc, + cache: Arc, col_id: usize, } impl CachedPageReader { pub(crate) fn new( inner: SerializedPageReader, - cache: Arc, + cache: Arc, col_id: usize, ) -> Self { Self { @@ -420,4 +470,103 @@ mod tests { let selection = take_next_selection(&mut queue, 10); assert!(selection.is_none()); } + + #[test] + fn test_predicate_page_cache_basic_operations() { + use super::*; + + let cache = PredicatePageCache::new(2); + let page1 = Page::dummy_page(PageType::DATA_PAGE, 100); + let page2 = Page::dummy_page(PageType::DICTIONARY_PAGE, 200); + + // Insert and retrieve a data page + cache.insert_page(0, 1000, page1.clone()); + let retrieved = cache.get_page(0, 1000); + assert!(retrieved.is_some()); + assert_eq!(retrieved.unwrap().page_type(), PageType::DATA_PAGE); + + // Insert and retrieve a dictionary page for same column + cache.insert_page(0, 2000, page2.clone()); + let retrieved = cache.get_page(0, 2000); + assert!(retrieved.is_some()); + assert_eq!(retrieved.unwrap().page_type(), PageType::DICTIONARY_PAGE); + + // Both pages should still be accessible + assert!(cache.get_page(0, 1000).is_some()); + assert!(cache.get_page(0, 2000).is_some()); + } + + #[test] + fn test_predicate_page_cache_replacement() { + use super::*; + + let cache = PredicatePageCache::new(2); + let data_page1 = Page::dummy_page(PageType::DATA_PAGE, 100); + let data_page2 = Page::dummy_page(PageType::DATA_PAGE_V2, 200); + + // Insert first data page + cache.insert_page(0, 1000, data_page1.clone()); + assert!(cache.get_page(0, 1000).is_some()); + + // Insert second data page - should replace first data page + cache.insert_page(0, 2000, data_page2.clone()); + assert!(cache.get_page(0, 2000).is_some()); + assert!(cache.get_page(0, 1000).is_none()); // First page should be gone + } + + #[test] + fn test_predicate_page_cache_multiple_columns() { + use super::*; + + let cache = PredicatePageCache::new(2); + let page1 = Page::dummy_page(PageType::DATA_PAGE, 100); + let page2 = Page::dummy_page(PageType::DATA_PAGE_V2, 200); + + // Insert pages for different columns + cache.insert_page(0, 1000, page1.clone()); + cache.insert_page(1, 1000, page2.clone()); + + // Both pages should be accessible + assert!(cache.get_page(0, 1000).is_some()); + assert!(cache.get_page(1, 1000).is_some()); + + // Non-existent column should return None + assert!(cache.get_page(2, 1000).is_none()); + } +} + +// Helper implementation for testing +#[cfg(test)] +impl Page { + fn dummy_page(page_type: PageType, size: usize) -> Self { + use crate::basic::Encoding; + match page_type { + PageType::DATA_PAGE => Page::DataPage { + buf: vec![0; size].into(), + num_values: size as u32, + encoding: Encoding::PLAIN, + def_level_encoding: Encoding::PLAIN, + rep_level_encoding: Encoding::PLAIN, + statistics: None, + }, + PageType::DICTIONARY_PAGE => Page::DictionaryPage { + buf: vec![0; size].into(), + num_values: size as u32, + encoding: Encoding::PLAIN, + is_sorted: false, + }, + PageType::DATA_PAGE_V2 => Page::DataPageV2 { + buf: vec![0; size].into(), + num_values: size as u32, + encoding: Encoding::PLAIN, + def_levels_byte_len: 0, + rep_levels_byte_len: 0, + is_compressed: false, + statistics: None, + num_nulls: 0, + num_rows: 0, + }, + _ => unreachable!(), + } + } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index f490bb6d5bc5..b466e0b318e1 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -83,7 +83,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow_reader::{CachedPageReader, FilteredParquetRecordBatchReader, PageCache}; +use arrow_reader::{CachedPageReader, FilteredParquetRecordBatchReader, PredicatePageCache}; use bytes::{Buf, Bytes}; use futures::future::{BoxFuture, FutureExt}; use futures::ready; @@ -528,15 +528,7 @@ where p }); - let mut row_group = InMemoryRowGroup { - metadata: meta, - // schema: meta.schema_descr_ptr(), - row_count: meta.num_rows() as usize, - column_chunks: vec![None; meta.columns().len()], - offset_index, - cache: Arc::new(PageCache::new()), - projection_to_cache, - }; + let mut row_group = InMemoryRowGroup::new(meta, offset_index, projection_to_cache); let mut selection = selection.unwrap_or_else(|| vec![RowSelector::select(row_group.row_count)].into()); @@ -815,7 +807,7 @@ struct InMemoryRowGroup<'a> { offset_index: Option<&'a [OffsetIndexMetaData]>, column_chunks: Vec>>, row_count: usize, - cache: Arc, + cache: Arc, projection_to_cache: Option, } diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 7d173bc0c16a..0356455f9017 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -425,7 +425,6 @@ pub(crate) fn decode_page( } _ => buffer, }; - let buffer = Bytes::from(buffer); let result = match page_header.type_ { PageType::DICTIONARY_PAGE => { @@ -569,6 +568,7 @@ impl SerializedPageReader { }) } + #[cfg(feature = "async")] pub(crate) fn peek_next_page_offset(&mut self) -> Result> { match &mut self.state { SerializedPageReaderState::Values { From c8bdbcf13e67fdcbb7938d22b127d2343692d036 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 2 Jan 2025 10:40:41 -0600 Subject: [PATCH 14/73] switch to mutex instead of rwlock --- .../src/arrow/async_reader/arrow_reader.rs | 136 +++++++++--------- 1 file changed, 70 insertions(+), 66 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 44f327ee8538..12a8b839ab75 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -17,7 +17,7 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; -use std::sync::RwLock; +use std::sync::{Mutex, MutexGuard, RwLock}; use std::{collections::VecDeque, sync::Arc}; use arrow_array::ArrayRef; @@ -228,6 +228,51 @@ struct PredicatePageCacheInner { pages: HashMap, // col_id (Parquet's leaf column index) -> CachedPage } +impl PredicatePageCacheInner { + pub(crate) fn get_page(&self, col_id: usize, offset: usize) -> Option { + self.pages.get(&col_id).and_then(|pages| { + pages + .dict + .iter() + .chain(pages.data.iter()) + .find(|(page_offset, _)| *page_offset == offset) + .map(|(_, page)| page.clone()) + }) + } + + /// Insert a page into the cache. + /// Inserting a page will override the existing page, if any. + /// This is because we only need to cache 2 pages per column, see below. + pub(crate) fn insert_page(&mut self, col_id: usize, offset: usize, page: Page) { + let is_dict = page.page_type() == PageType::DICTIONARY_PAGE; + + let cached_pages = self.pages.entry(col_id); + match cached_pages { + Entry::Occupied(mut entry) => { + if is_dict { + entry.get_mut().dict = Some((offset, page)); + } else { + entry.get_mut().data = Some((offset, page)); + } + } + Entry::Vacant(entry) => { + let cached_page = if is_dict { + CachedPage { + dict: Some((offset, page)), + data: None, + } + } else { + CachedPage { + dict: None, + data: Some((offset, page)), + } + }; + entry.insert(cached_page); + } + } + } +} + /// A simple cache to avoid double-decompressing pages with filter pushdown. /// In filter pushdown, we first decompress a page, apply the filter, and then decompress the page again. /// This double decompression is expensive, so we cache the decompressed page. @@ -280,62 +325,20 @@ struct PredicatePageCacheInner { /// ## How to identify a page /// We use the page offset (the offset to the Parquet file) to uniquely identify a page. pub(crate) struct PredicatePageCache { - inner: RwLock, + inner: Mutex, } impl PredicatePageCache { pub(crate) fn new(capacity: usize) -> Self { Self { - inner: RwLock::new(PredicatePageCacheInner { + inner: Mutex::new(PredicatePageCacheInner { pages: HashMap::with_capacity(capacity), }), } } - pub(crate) fn get_page(&self, col_id: usize, offset: usize) -> Option { - let read_lock = self.inner.read().unwrap(); - read_lock.pages.get(&col_id).and_then(|pages| { - pages - .dict - .iter() - .chain(pages.data.iter()) - .find(|(page_offset, _)| *page_offset == offset) - .map(|(_, page)| page.clone()) - }) - } - - /// Insert a page into the cache. - /// Inserting a page will override the existing page, if any. - /// This is because we only need to cache 2 pages per column, see above. - pub(crate) fn insert_page(&self, col_id: usize, offset: usize, page: Page) { - let mut write_lock = self.inner.write().unwrap(); - - let is_dict = page.page_type() == PageType::DICTIONARY_PAGE; - - let cached_pages = write_lock.pages.entry(col_id); - match cached_pages { - Entry::Occupied(mut entry) => { - if is_dict { - entry.get_mut().dict = Some((offset, page)); - } else { - entry.get_mut().data = Some((offset, page)); - } - } - Entry::Vacant(entry) => { - let cached_page = if is_dict { - CachedPage { - dict: Some((offset, page)), - data: None, - } - } else { - CachedPage { - dict: None, - data: Some((offset, page)), - } - }; - entry.insert(cached_page); - } - } + fn get(&self) -> MutexGuard { + self.inner.lock().unwrap() } } @@ -376,7 +379,9 @@ impl PageReader for CachedPageReader { return Ok(None); }; - let page = self.cache.get_page(self.col_id, offset); + let mut cache = self.cache.get(); + + let page = cache.get_page(self.col_id, offset); if let Some(page) = page { self.inner.skip_next_page()?; Ok(Some(page)) @@ -385,8 +390,7 @@ impl PageReader for CachedPageReader { let Some(inner_page) = inner_page else { return Ok(None); }; - self.cache - .insert_page(self.col_id, offset, inner_page.clone()); + cache.insert_page(self.col_id, offset, inner_page.clone()); Ok(Some(inner_page)) } } @@ -480,20 +484,20 @@ mod tests { let page2 = Page::dummy_page(PageType::DICTIONARY_PAGE, 200); // Insert and retrieve a data page - cache.insert_page(0, 1000, page1.clone()); - let retrieved = cache.get_page(0, 1000); + cache.get().insert_page(0, 1000, page1.clone()); + let retrieved = cache.get().get_page(0, 1000); assert!(retrieved.is_some()); assert_eq!(retrieved.unwrap().page_type(), PageType::DATA_PAGE); // Insert and retrieve a dictionary page for same column - cache.insert_page(0, 2000, page2.clone()); - let retrieved = cache.get_page(0, 2000); + cache.get().insert_page(0, 2000, page2.clone()); + let retrieved = cache.get().get_page(0, 2000); assert!(retrieved.is_some()); assert_eq!(retrieved.unwrap().page_type(), PageType::DICTIONARY_PAGE); // Both pages should still be accessible - assert!(cache.get_page(0, 1000).is_some()); - assert!(cache.get_page(0, 2000).is_some()); + assert!(cache.get().get_page(0, 1000).is_some()); + assert!(cache.get().get_page(0, 2000).is_some()); } #[test] @@ -505,13 +509,13 @@ mod tests { let data_page2 = Page::dummy_page(PageType::DATA_PAGE_V2, 200); // Insert first data page - cache.insert_page(0, 1000, data_page1.clone()); - assert!(cache.get_page(0, 1000).is_some()); + cache.get().insert_page(0, 1000, data_page1.clone()); + assert!(cache.get().get_page(0, 1000).is_some()); // Insert second data page - should replace first data page - cache.insert_page(0, 2000, data_page2.clone()); - assert!(cache.get_page(0, 2000).is_some()); - assert!(cache.get_page(0, 1000).is_none()); // First page should be gone + cache.get().insert_page(0, 2000, data_page2.clone()); + assert!(cache.get().get_page(0, 2000).is_some()); + assert!(cache.get().get_page(0, 1000).is_none()); // First page should be gone } #[test] @@ -523,15 +527,15 @@ mod tests { let page2 = Page::dummy_page(PageType::DATA_PAGE_V2, 200); // Insert pages for different columns - cache.insert_page(0, 1000, page1.clone()); - cache.insert_page(1, 1000, page2.clone()); + cache.get().insert_page(0, 1000, page1.clone()); + cache.get().insert_page(1, 1000, page2.clone()); // Both pages should be accessible - assert!(cache.get_page(0, 1000).is_some()); - assert!(cache.get_page(1, 1000).is_some()); + assert!(cache.get().get_page(0, 1000).is_some()); + assert!(cache.get().get_page(1, 1000).is_some()); // Non-existent column should return None - assert!(cache.get_page(2, 1000).is_none()); + assert!(cache.get().get_page(2, 1000).is_none()); } } From cdb1d859d0b300c74b61d6ce3bc2354977fc96dd Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 2 Jan 2025 10:46:01 -0600 Subject: [PATCH 15/73] revert irrelevant changes --- arrow-buffer/src/buffer/boolean.rs | 45 ------------------- .../src/arrow/array_reader/byte_view_array.rs | 2 +- .../src/arrow/async_reader/arrow_reader.rs | 2 +- parquet/src/arrow/buffer/view_buffer.rs | 7 --- 4 files changed, 2 insertions(+), 54 deletions(-) diff --git a/arrow-buffer/src/buffer/boolean.rs b/arrow-buffer/src/buffer/boolean.rs index 4e42d3c27e1a..aaa86832f692 100644 --- a/arrow-buffer/src/buffer/boolean.rs +++ b/arrow-buffer/src/buffer/boolean.rs @@ -204,51 +204,6 @@ impl BooleanBuffer { pub fn set_slices(&self) -> BitSliceIterator<'_> { BitSliceIterator::new(self.values(), self.offset, self.len) } - - /// Combines this [`BooleanBuffer`] with another using logical AND on the selected bits. - /// - /// Unlike intersection, the `other` [`BooleanBuffer`] must have exactly as many **set bits** as `self`, - /// i.e., self.count_set_bits() == other.len(). - /// - /// This method will keep only the bits in `self` that are also set in `other` - /// at the positions corresponding to `self`'s set bits. - /// For example: - /// self: NNYYYNNYYNYN - /// other: YNY NY N - /// result: NNYNYNNNYNNN - pub fn and_then(&self, other: &Self) -> Self { - // Ensure that 'other' has exactly as many set bits as 'self' - debug_assert_eq!( - self.count_set_bits(), - other.len(), - "The 'other' selection must have exactly as many set bits as 'self'." - ); - - if self.len() == other.len() { - // fast path if the two bool masks are the same length - // this happens when self selects all rows - debug_assert_eq!(self.count_set_bits(), self.len()); - return other.clone(); - } - - let mut buffer = MutableBuffer::from_len_zeroed(self.values().len()); - buffer.copy_from_slice(self.values()); - let mut builder = BooleanBufferBuilder::new_from_buffer(buffer, self.len()); - - // Create iterators for 'self' and 'other' bits - let mut other_bits = other.iter(); - - for bit_idx in self.set_indices() { - let predicate = other_bits - .next() - .expect("Mismatch in set bits between self and other"); - if !predicate { - builder.set_bit(bit_idx, false); - } - } - - builder.finish() - } } impl Not for &BooleanBuffer { diff --git a/parquet/src/arrow/array_reader/byte_view_array.rs b/parquet/src/arrow/array_reader/byte_view_array.rs index a4366877dcb7..5573e49a42cc 100644 --- a/parquet/src/arrow/array_reader/byte_view_array.rs +++ b/parquet/src/arrow/array_reader/byte_view_array.rs @@ -161,7 +161,7 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { )); } - let mut buffer = ViewBuffer::with_capacity(num_values as usize, 1); + let mut buffer = ViewBuffer::default(); let mut decoder = ByteViewArrayDecoderPlain::new( buf, num_values as usize, diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 12a8b839ab75..a654928f0b2c 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -17,7 +17,7 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; -use std::sync::{Mutex, MutexGuard, RwLock}; +use std::sync::{Mutex, MutexGuard}; use std::{collections::VecDeque, sync::Arc}; use arrow_array::ArrayRef; diff --git a/parquet/src/arrow/buffer/view_buffer.rs b/parquet/src/arrow/buffer/view_buffer.rs index a21d153df76f..fd7d6c213f04 100644 --- a/parquet/src/arrow/buffer/view_buffer.rs +++ b/parquet/src/arrow/buffer/view_buffer.rs @@ -33,13 +33,6 @@ pub struct ViewBuffer { } impl ViewBuffer { - pub fn with_capacity(view_capacity: usize, buffer_capacity: usize) -> Self { - Self { - views: Vec::with_capacity(view_capacity), - buffers: Vec::with_capacity(buffer_capacity), - } - } - pub fn is_empty(&self) -> bool { self.views.is_empty() } From 69720e507533dc5ce47090510405b3a99023c50c Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Fri, 3 Jan 2025 10:23:15 -0600 Subject: [PATCH 16/73] submodule --- parquet-testing | 2 +- testing | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet-testing b/parquet-testing index 1ba34478f535..4439a223a315 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 1ba34478f535c89382263c42c675a9af4f57f2dd +Subproject commit 4439a223a315cf874746d3b5da25e6a6b2a2b16e diff --git a/testing b/testing index 4d209492d514..735ae7128d57 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 4d209492d514c2d3cb2d392681b9aa00e6d8da1c +Subproject commit 735ae7128d571398dd798d7ff004adebeb342883 From a9550ab730db807460f14d4de424c747a97ca9af Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Fri, 3 Jan 2025 11:17:44 -0600 Subject: [PATCH 17/73] update --- parquet/src/arrow/async_reader/arrow_reader.rs | 2 -- parquet/src/arrow/async_reader/mod.rs | 2 +- parquet/src/file/serialized_reader.rs | 3 +++ 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index a654928f0b2c..b8a1466c8389 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -120,7 +120,6 @@ impl FilteredParquetRecordBatchReader { self.row_filter.take() } - #[inline(never)] /// Take a selection, and return the new selection where the rows are filtered by the predicate. fn build_predicate_filter( &mut self, @@ -372,7 +371,6 @@ impl Iterator for CachedPageReader { impl PageReader for CachedPageReader { fn get_next_page(&mut self) -> Result, ParquetError> { - // self.inner.get_next_page() let next_page_offset = self.inner.peek_next_page_offset()?; let Some(offset) = next_page_offset else { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index b466e0b318e1..48d12d6434fa 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -686,7 +686,7 @@ where /// - `Ok(None)` if the stream has ended. /// - `Err(error)` if the stream has errored. All subsequent calls will return `Ok(None)`. /// - `Ok(Some(reader))` which holds all the data for the row group. - pub async fn next_row_group(&mut self) -> Result> { + pub async fn next_row_group(&mut self) -> Result> { loop { match &mut self.state { StreamState::Decoding(_) | StreamState::Reading(_) => { diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 0356455f9017..4bd69b41029b 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -568,6 +568,9 @@ impl SerializedPageReader { }) } + /// Similar to `peek_next_page`, but returns the offset of the next page instead of the page metadata. + /// Unlike page metadata, an offset can uniquely identify a page. + /// Useful when we want to if the next page is being cached or read previously. #[cfg(feature = "async")] pub(crate) fn peek_next_page_offset(&mut self) -> Result> { match &mut self.state { From be1435f8ab10c1e0bd089e1be88f087d4d17c9e7 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Mon, 6 Jan 2025 11:29:35 -0600 Subject: [PATCH 18/73] rebase --- parquet/src/arrow/async_reader/mod.rs | 28 ++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 48d12d6434fa..8f66c5ddd741 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -811,7 +811,33 @@ struct InMemoryRowGroup<'a> { projection_to_cache: Option, } -impl InMemoryRowGroup<'_> { +impl<'a> InMemoryRowGroup<'a> { + fn new( + metadata: &'a RowGroupMetaData, + offset_index: Option<&'a [OffsetIndexMetaData]>, + projection_to_cache: Option, + ) -> Self { + let to_cache_column_cnt = projection_to_cache + .as_ref() + .map(|p| { + if let Some(mask) = &p.mask { + mask.iter().filter(|&&b| b).count() + } else { + metadata.columns().len() + } + }) + .unwrap_or(0); + Self { + metadata, + offset_index, + column_chunks: vec![None; metadata.columns().len()], + row_count: metadata.num_rows() as usize, + cache: Arc::new(PredicatePageCache::new(to_cache_column_cnt)), + projection_to_cache, + } + } +} +impl<'a> InMemoryRowGroup<'a> { /// Fetches the necessary column data into memory async fn fetch( &mut self, From 21e015b13dae40442bec0a918188131dc8f2906f Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 8 Jan 2025 10:04:25 -0600 Subject: [PATCH 19/73] remove unrelated changes --- parquet/src/arrow/array_reader/byte_view_array.rs | 5 ----- parquet/src/arrow/arrow_reader/filter.rs | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/parquet/src/arrow/array_reader/byte_view_array.rs b/parquet/src/arrow/array_reader/byte_view_array.rs index f3b7bf46d5d1..64d839fa5874 100644 --- a/parquet/src/arrow/array_reader/byte_view_array.rs +++ b/parquet/src/arrow/array_reader/byte_view_array.rs @@ -169,9 +169,7 @@ impl ColumnValueDecoder for ByteViewArrayColumnValueDecoder { self.validate_utf8, ); decoder.read(&mut buffer, usize::MAX)?; - self.dict = Some(buffer); - Ok(()) } @@ -309,9 +307,6 @@ impl ByteViewArrayDecoderPlain { num_values: Option, validate_utf8: bool, ) -> Self { - // Here we convert `bytes::Bytes` into `arrow_buffer::Bytes`, which is zero copy - // Then we convert `arrow_buffer::Bytes` into `arrow_buffer:Buffer`, which is also zero copy - let buf = arrow_buffer::Buffer::from_bytes(buf.clone().into()); Self { buf: Buffer::from(buf), offset: 0, diff --git a/parquet/src/arrow/arrow_reader/filter.rs b/parquet/src/arrow/arrow_reader/filter.rs index 931e13e252f5..2e22f7e01cf0 100644 --- a/parquet/src/arrow/arrow_reader/filter.rs +++ b/parquet/src/arrow/arrow_reader/filter.rs @@ -110,7 +110,7 @@ where /// [`RowSelection`]: crate::arrow::arrow_reader::RowSelection pub struct RowFilter { /// A list of [`ArrowPredicate`] - pub predicates: Vec>, + pub(crate) predicates: Vec>, } impl RowFilter { From 547fb4611ef86849d5d5a66119ab1066c21b6aaa Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Fri, 10 Jan 2025 09:59:13 -0600 Subject: [PATCH 20/73] fix clippy --- parquet/src/file/serialized_reader.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index fd6fad7326ab..e5796a0ba7d4 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -573,8 +573,8 @@ impl SerializedPageReader { /// /// This is used when we need to read parquet with row-filter, and we don't want to decompress the page twice. /// This function allows us to check if the next page is being cached or read previously. - #[cfg(test)] - fn peek_next_page_offset(&mut self) -> Result> { + #[cfg(feature = "async")] + pub(crate) fn peek_next_page_offset(&mut self) -> Result> { match &mut self.state { SerializedPageReaderState::Values { offset, From 05c8c8f05b9b4a136dbdfdb6feb524274cbd9027 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Fri, 10 Jan 2025 10:44:30 -0600 Subject: [PATCH 21/73] make various ci improvements --- .../src/arrow/async_reader/arrow_reader.rs | 76 +++++++++---------- parquet/src/arrow/async_reader/mod.rs | 6 +- parquet/src/file/serialized_reader.rs | 5 +- 3 files changed, 44 insertions(+), 43 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index b8a1466c8389..92e585756d49 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -406,6 +406,42 @@ impl PageReader for CachedPageReader { } } +// Helper implementation for testing +#[cfg(test)] +impl Page { + fn dummy_page(page_type: PageType, size: usize) -> Self { + use crate::basic::Encoding; + match page_type { + PageType::DATA_PAGE => Page::DataPage { + buf: vec![0; size].into(), + num_values: size as u32, + encoding: Encoding::PLAIN, + def_level_encoding: Encoding::PLAIN, + rep_level_encoding: Encoding::PLAIN, + statistics: None, + }, + PageType::DICTIONARY_PAGE => Page::DictionaryPage { + buf: vec![0; size].into(), + num_values: size as u32, + encoding: Encoding::PLAIN, + is_sorted: false, + }, + PageType::DATA_PAGE_V2 => Page::DataPageV2 { + buf: vec![0; size].into(), + num_values: size as u32, + encoding: Encoding::PLAIN, + def_levels_byte_len: 0, + rep_levels_byte_len: 0, + is_compressed: false, + statistics: None, + num_nulls: 0, + num_rows: 0, + }, + _ => unreachable!(), + } + } +} + #[cfg(test)] mod tests { use super::*; @@ -434,7 +470,7 @@ mod tests { // Check remaining queue assert_eq!(queue.len(), 1); assert_eq!(queue[0].row_count, 7); - assert_eq!(queue[0].skip, false); + assert!(!queue[0].skip); } #[test] @@ -451,7 +487,7 @@ mod tests { // Check remaining queue - should have 5 rows from split and original 10 assert_eq!(queue.len(), 1); - assert_eq!(queue[0].skip, false); + assert!(!queue[0].skip); assert_eq!(queue[0].row_count, 5); } @@ -536,39 +572,3 @@ mod tests { assert!(cache.get().get_page(2, 1000).is_none()); } } - -// Helper implementation for testing -#[cfg(test)] -impl Page { - fn dummy_page(page_type: PageType, size: usize) -> Self { - use crate::basic::Encoding; - match page_type { - PageType::DATA_PAGE => Page::DataPage { - buf: vec![0; size].into(), - num_values: size as u32, - encoding: Encoding::PLAIN, - def_level_encoding: Encoding::PLAIN, - rep_level_encoding: Encoding::PLAIN, - statistics: None, - }, - PageType::DICTIONARY_PAGE => Page::DictionaryPage { - buf: vec![0; size].into(), - num_values: size as u32, - encoding: Encoding::PLAIN, - is_sorted: false, - }, - PageType::DATA_PAGE_V2 => Page::DataPageV2 { - buf: vec![0; size].into(), - num_values: size as u32, - encoding: Encoding::PLAIN, - def_levels_byte_len: 0, - rep_levels_byte_len: 0, - is_compressed: false, - statistics: None, - num_nulls: 0, - num_rows: 0, - }, - _ => unreachable!(), - } - } -} diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 40e68f01d4b0..f93846292ba0 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -475,7 +475,7 @@ impl ParquetRecordBatchStreamBuilder { type ReadResult = Result<(ReaderFactory, Option)>; /// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create -/// [`ParquetRecordBatchReader`] +/// [`FilteredParquetRecordBatchReader`] struct ReaderFactory { metadata: Arc, @@ -517,7 +517,7 @@ where for predicate in filter.predicates.iter_mut() { let p_projection = predicate.projection(); if let Some(ref mut p) = predicate_projection { - p.union(&p_projection); + p.union(p_projection); } else { predicate_projection = Some(p_projection.clone()); } @@ -849,7 +849,7 @@ impl<'a> InMemoryRowGroup<'a> { } } } -impl<'a> InMemoryRowGroup<'a> { +impl InMemoryRowGroup<'_> { /// Fetches the necessary column data into memory async fn fetch( &mut self, diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index e5796a0ba7d4..3f2f1e0539d6 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -858,7 +858,6 @@ impl PageReader for SerializedPageReader { #[cfg(test)] mod tests { - use std::collections::HashSet; use bytes::Buf; @@ -1165,6 +1164,7 @@ mod tests { assert_eq!(page_count, 2); } + #[cfg(feature = "async")] fn get_serialized_page_reader( file_reader: &SerializedFileReader, row_group: usize, @@ -1201,12 +1201,13 @@ mod tests { ) } + #[cfg(feature = "async")] #[test] fn test_peek_next_page_offset_matches_actual() -> Result<()> { let test_file = get_test_file("alltypes_plain.parquet"); let reader = SerializedFileReader::new(test_file)?; - let mut offset_set = HashSet::new(); + let mut offset_set = std::collections::HashSet::new(); let num_row_groups = reader.metadata.num_row_groups(); for row_group in 0..num_row_groups { let num_columns = reader.metadata.row_group(row_group).num_columns(); From c895dd21735f458cbfaeae10a206c662880c8c8e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 21 Mar 2025 16:07:19 -0400 Subject: [PATCH 22/73] whitespace --- parquet/src/file/serialized_reader.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 2e9dde8e8a54..87cad95ca649 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -464,6 +464,7 @@ pub(crate) fn decode_page( uncompressed_size )); } + Bytes::from(decompressed) } _ => buffer, From 3cf0a986810c5087af34bfb52c36d252d8f9e3ef Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 21 Mar 2025 16:16:34 -0400 Subject: [PATCH 23/73] Reduce some ugliness, avoid unwrap --- parquet/src/arrow/async_reader/mod.rs | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 4579ccd0713f..9195b77cc0cf 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -587,7 +587,7 @@ impl ParquetRecordBatchStreamBuilder { projection: self.projection, selection: self.selection, schema, - reader: Some(reader), + reader_factory: Some(reader), state: StreamState::Init, }) } @@ -780,7 +780,7 @@ pub struct ParquetRecordBatchStream { selection: Option, /// This is an option so it can be moved into a future - reader: Option>, + reader_factory: Option>, state: StreamState, } @@ -842,7 +842,7 @@ where let selection = self.selection.as_mut().map(|s| s.split_off(row_count)); - let reader_factory = self.reader.take().expect("lost reader"); + let reader_factory = self.reader_factory.take().expect("lost reader"); let (reader_factory, maybe_reader) = reader_factory .read_row_group( @@ -856,7 +856,7 @@ where self.state = StreamState::Error; err })?; - self.reader = Some(reader_factory); + self.reader_factory = Some(reader_factory); if let Some(reader) = maybe_reader { return Ok(Some(reader)); @@ -891,7 +891,10 @@ where None => { // this is ugly, but works for now. let filter = batch_reader.take_filter(); - self.reader.as_mut().unwrap().filter = filter; + let Some(reader_factory) = self.reader_factory.as_mut() else { + return Poll::Ready(Some(Err(ParquetError::General("Internal: Unexpected state".into())))) + }; + reader_factory.filter = filter; self.state = StreamState::Init } }, @@ -901,7 +904,7 @@ where None => return Poll::Ready(None), }; - let reader = self.reader.take().expect("lost reader"); + let reader = self.reader_factory.take().expect("lost reader"); let row_count = self.metadata.row_group(row_group_idx).num_rows() as usize; @@ -920,7 +923,7 @@ where } StreamState::Reading(f) => match ready!(f.poll_unpin(cx)) { Ok((reader_factory, maybe_reader)) => { - self.reader = Some(reader_factory); + self.reader_factory = Some(reader_factory); match maybe_reader { // Read records from [`ParquetRecordBatchReader`] Some(reader) => self.state = StreamState::Decoding(reader), From 7b72f9d6c65b68a23b2037b26815af5315a53f31 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 21 Mar 2025 16:23:19 -0400 Subject: [PATCH 24/73] more factory --- parquet/src/arrow/async_reader/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 9195b77cc0cf..53b15cd639bf 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -560,7 +560,7 @@ impl ParquetRecordBatchStreamBuilder { let batch_size = self .batch_size .min(self.metadata.file_metadata().num_rows() as usize); - let reader = ReaderFactory { + let reader_factory = ReaderFactory { input: self.input.0, filter: self.filter, metadata: self.metadata.clone(), @@ -571,7 +571,7 @@ impl ParquetRecordBatchStreamBuilder { // Ensure schema of ParquetRecordBatchStream respects projection, and does // not store metadata (same as for ParquetRecordBatchReader and emitted RecordBatches) - let projected_fields = match reader.fields.as_deref().map(|pf| &pf.arrow_type) { + let projected_fields = match reader_factory.fields.as_deref().map(|pf| &pf.arrow_type) { Some(DataType::Struct(fields)) => { fields.filter_leaves(|idx, _| self.projection.leaf_included(idx)) } @@ -587,7 +587,7 @@ impl ParquetRecordBatchStreamBuilder { projection: self.projection, selection: self.selection, schema, - reader_factory: Some(reader), + reader_factory: Some(reader_factory), state: StreamState::Init, }) } From 5bdf51a240c7d96b350830a5996116a5980ea07c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 21 Mar 2025 20:16:47 -0400 Subject: [PATCH 25/73] lint --- parquet/src/arrow/async_reader/mod.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 53b15cd639bf..10a8ab00cb82 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -891,9 +891,8 @@ where None => { // this is ugly, but works for now. let filter = batch_reader.take_filter(); - let Some(reader_factory) = self.reader_factory.as_mut() else { - return Poll::Ready(Some(Err(ParquetError::General("Internal: Unexpected state".into())))) - }; + let reader_factory = + self.reader_factory.as_mut().expect("have reader factory"); reader_factory.filter = filter; self.state = StreamState::Init } From 90a55d5d701c2b6501f2d788469ef8c6b8b4b1f2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 26 Mar 2025 16:05:04 -0400 Subject: [PATCH 26/73] Isolate reader cache more --- parquet/src/arrow/arrow_reader/mod.rs | 1 + parquet/src/arrow/async_reader/mod.rs | 31 +++++++++++++++++---------- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index d8c116f0135b..c8d995210f5d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -718,6 +718,7 @@ impl Iterator for ReaderPageIterator { None }; + // todo: add cache??? let ret = SerializedPageReader::new(reader, meta, total_rows, page_locations); #[cfg(feature = "encryption")] diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 729ca38bbc40..7bc8f0d9b50a 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -1074,6 +1074,25 @@ impl InMemoryRowGroup<'_> { Ok(()) } + + /// Returns a CachedPageReader if the specified column is included in the projection to cache + fn maybe_cached_reader( + &self, + column_idx: usize, + page_reader: SerializedPageReader, + ) -> Box { + let Some(projection_to_cache) = &self.projection_to_cache else { + return Box::new(page_reader); + }; + if !projection_to_cache.leaf_included(column_idx) { + return Box::new(page_reader); + } + Box::new(CachedPageReader::new( + page_reader, + self.cache.clone(), + column_idx, + )) + } } impl RowGroups for InMemoryRowGroup<'_> { @@ -1093,12 +1112,6 @@ impl RowGroups for InMemoryRowGroup<'_> { .filter(|index| !index.is_empty()) .map(|index| index[i].page_locations.clone()); - let cached_reader = if let Some(projection_to_cache) = &self.projection_to_cache { - projection_to_cache.leaf_included(i) - } else { - false - }; - let column_metadata = self.metadata.row_group(self.row_group_idx).column(i); let page_reader = SerializedPageReader::new( data.clone(), @@ -1125,11 +1138,7 @@ impl RowGroups for InMemoryRowGroup<'_> { #[cfg(feature = "encryption")] let page_reader = page_reader.with_crypto_context(crypto_context); - let page_reader: Box = if cached_reader { - Box::new(CachedPageReader::new(page_reader, self.cache.clone(), i)) - } else { - Box::new(page_reader) - }; + let page_reader = self.maybe_cached_reader(i, page_reader); Ok(Box::new(ColumnChunkIterator { reader: Some(Ok(page_reader)), From 822760c0df669118a0d4239acf84209c8d1f2cd4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 10 Apr 2025 22:49:38 +0800 Subject: [PATCH 27/73] Add benchmark for parquet reader with row_filter and project settings --- parquet/Cargo.toml | 4 + parquet/benches/arrow_reader_row_filter.rs | 316 +++++++++++++++++++++ 2 files changed, 320 insertions(+) create mode 100644 parquet/benches/arrow_reader_row_filter.rs diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index 2f31a290e398..80b398df21de 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -211,6 +211,10 @@ name = "arrow_statistics" required-features = ["arrow"] harness = false +[[bench]] +name = "arrow_reader_row_filter" +required-features = ["arrow"] +harness = false [[bench]] name = "compression" diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs new file mode 100644 index 000000000000..788ca3e533b7 --- /dev/null +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -0,0 +1,316 @@ +// 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. + +//! Benchmark for evaluating row filters and projections on a Parquet file. +//! +//! This benchmark creates a Parquet file in memory with 100K rows and four columns: +//! - int64: sequential integers +//! - float64: floating-point values (derived from the integers) +//! - utf8View: string values where about half are non-empty, +//! and a few rows (every 10Kth row) are the constant "const" +//! - ts: timestamp values (using, e.g., a millisecond epoch) +//! +//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! +//! Filters tested: +//! - A string filter: `utf8View <> ''` (non-empty) +//! - A string filter: `utf8View = 'const'` (selective) +//! - An integer non-selective filter (e.g. even numbers) +//! - An integer selective filter (e.g. `int64 = 0`) +//! - A timestamp filter (e.g. `ts > threshold`) +//! +//! Projections tested: +//! - All 4 columns. +//! - All columns except the one used for the filter. +//! +//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. + +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; +use tempfile::NamedTempFile; + +use arrow::array::{ + ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, +}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::record_batch::RecordBatch; +use arrow_array::builder::StringViewBuilder; +use arrow_array::{Array, StringViewArray}; +use parquet::arrow::arrow_reader::{ + ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, RowFilter, +}; +use parquet::arrow::{ArrowWriter, ProjectionMask}; +use parquet::file::properties::WriterProperties; + +/// Create a RecordBatch with 100K rows and four columns. +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + + // int64 column: sequential numbers 0..num_rows + let int_values: Vec = (0..num_rows as i64).collect(); + let int_array = Arc::new(Int64Array::from(int_values)) as ArrayRef; + + // float64 column: derived from int64 (e.g., multiplied by 0.1) + let float_values: Vec = (0..num_rows).map(|i| i as f64 * 0.1).collect(); + let float_array = Arc::new(Float64Array::from(float_values)) as ArrayRef; + + // utf8View column: even rows get non-empty strings; odd rows get an empty string; + // every 10Kth even row is "const" to be selective. + let mut string_view_builder = StringViewBuilder::with_capacity(100_000); + for i in 0..num_rows { + if i % 2 == 0 { + if i % 10_000 == 0 { + string_view_builder.append_value("const"); + } else { + string_view_builder.append_value("nonempty"); + } + } else { + string_view_builder.append_value(""); + } + } + let utf8_view_array = Arc::new(string_view_builder.finish()) as ArrayRef; + + // Timestamp column: using milliseconds from an epoch (simply using the row index) + let ts_values: Vec = (0..num_rows as i64).collect(); + let ts_array = Arc::new(TimestampMillisecondArray::from(ts_values)) as ArrayRef; + + let schema = Arc::new(Schema::new(vec![ + Field::new("int64", DataType::Int64, false), + Field::new("float64", DataType::Float64, false), + Field::new("utf8View", DataType::Utf8View, false), + Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + ])); + + RecordBatch::try_new( + schema, + vec![int_array, float_array, utf8_view_array, ts_array], + ) + .unwrap() +} + +/// Writes the record batch to a temporary Parquet file. +fn write_parquet_file() -> NamedTempFile { + let batch = make_record_batch(); + let schema = batch.schema(); + let props = WriterProperties::builder().build(); + + let file = tempfile::Builder::new() + .suffix(".parquet") + .tempfile() + .unwrap(); + { + let file_reopen = file.reopen().unwrap(); + let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); + // Write the entire batch as a single row group. + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + file +} + +/// Filter function: returns a BooleanArray with true when utf8View <> "". +fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("utf8View").unwrap()) + .as_any() + .downcast_ref::() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) != ""; + builder.append_value(keep); + } + builder.finish() +} + +/// Filter function: returns a BooleanArray with true when utf8View == "const". +fn filter_utf8_view_const(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("utf8View").unwrap()) + .as_any() + .downcast_ref::() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) == "const"; + builder.append_value(keep); + } + builder.finish() +} + +/// Integer non-selective filter: returns true for even numbers. +fn filter_int64_even(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("int64").unwrap()) + .as_any() + .downcast_ref::() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) % 2 == 0; + builder.append_value(keep); + } + builder.finish() +} + +/// Integer selective filter: returns true only when int64 equals 0. +fn filter_int64_eq_zero(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("int64").unwrap()) + .as_any() + .downcast_ref::() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) == 0; + builder.append_value(keep); + } + builder.finish() +} + +/// Timestamp filter: returns true when ts > threshold (using 50_000 as example threshold). +fn filter_timestamp_gt(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("ts").unwrap()) + .as_any() + .downcast_ref::() + .unwrap(); + let threshold = 50_000; + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) > threshold; + builder.append_value(keep); + } + builder.finish() +} + +#[derive(Clone)] +enum FilterType { + Utf8ViewNonEmpty, + Utf8ViewConst, + Int64Even, + Int64EqZero, + TimestampGt, +} + +impl std::fmt::Display for FilterType { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + FilterType::Utf8ViewNonEmpty => write!(f, "utf8View <> ''"), + FilterType::Utf8ViewConst => write!(f, "utf8View = 'const'"), + FilterType::Int64Even => write!(f, "int64 even"), + FilterType::Int64EqZero => write!(f, "int64 = 0"), + FilterType::TimestampGt => write!(f, "ts > 50_000"), + } + } +} + +fn benchmark_filters_and_projections(c: &mut Criterion) { + let parquet_file = write_parquet_file(); + + // Define filter functions associated with each FilterType. + let filter_funcs: Vec<(FilterType, fn(&RecordBatch) -> BooleanArray)> = vec![ + (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), + (FilterType::Utf8ViewConst, filter_utf8_view_const), + (FilterType::Int64Even, filter_int64_even), + (FilterType::Int64EqZero, filter_int64_eq_zero), + (FilterType::TimestampGt, filter_timestamp_gt), + ]; + + let mut group = c.benchmark_group("arrow_reader_row_filter"); + + // Iterate by value (Copy is available for FilterType and fn pointers) + for (filter_type, filter_fn) in filter_funcs.into_iter() { + for proj_case in ["all_columns", "exclude_filter_column"].iter() { + // Define indices for all columns: [0: "int64", 1: "float64", 2: "utf8View", 3: "ts"] + let all_indices = vec![0, 1, 2, 3]; + + // For the output projection, conditionally exclude the filter column. + let output_projection: Vec = if *proj_case == "all_columns" { + all_indices.clone() + } else { + all_indices + .into_iter() + .filter(|i| match filter_type { + FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => *i != 2, // Exclude "utf8" (index 2) + FilterType::Int64Even | FilterType::Int64EqZero => *i != 0, // Exclude "int64" (index 0) + FilterType::TimestampGt => *i != 3, // Exclude "ts" (index 3) + }) + .collect() + }; + + // For predicate pushdown, define a projection that includes the column required for the predicate. + let predicate_projection: Vec = match filter_type { + FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => vec![2], + FilterType::Int64Even | FilterType::Int64EqZero => vec![0], + FilterType::TimestampGt => vec![3], + }; + + // Create a benchmark id combining filter type and projection case. + let bench_id = BenchmarkId::new( + format!("filter_case: {} project_case: {}", filter_type, proj_case), + "", + ); + group.bench_function(bench_id, |b| { + b.iter(|| { + // Reopen the Parquet file for each iteration. + let file = parquet_file.reopen().unwrap(); + let options = ArrowReaderOptions::new().with_page_index(true); + let builder = ArrowReaderBuilder::try_new_with_options(file, options).unwrap(); + let file_metadata = builder.metadata().file_metadata().clone(); + // Build the projection mask from the output projection (clone to avoid move) + let mask = ProjectionMask::roots( + file_metadata.schema_descr(), + output_projection.clone(), + ); + + // Build the predicate mask from the predicate projection (clone to avoid move) + let pred_mask = ProjectionMask::roots( + file_metadata.schema_descr(), + predicate_projection.clone(), + ); + + // Copy the filter function pointer. + let f = filter_fn; + // Wrap the filter function in a closure to satisfy the expected signature. + let filter = + ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| Ok(f(&batch))); + let row_filter = RowFilter::new(vec![Box::new(filter)]); + + // Build the reader with row filter and output projection. + let reader = builder + .with_row_filter(row_filter) + .with_projection(mask) + .build() + .unwrap(); + + // Collect result batches, unwrapping errors. + let _result: Vec = reader.map(|r| r.unwrap()).collect(); + }); + }); + } + } + + group.finish(); +} + +criterion_group!(benches, benchmark_filters_and_projections); +criterion_main!(benches); From 31a544fd9bc5506d8c52014a4d7c922680ec1b08 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 10 Apr 2025 23:03:51 +0800 Subject: [PATCH 28/73] fix clippy --- parquet/benches/arrow_reader_row_filter.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 788ca3e533b7..5284275c97bf 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -21,7 +21,7 @@ //! - int64: sequential integers //! - float64: floating-point values (derived from the integers) //! - utf8View: string values where about half are non-empty, -//! and a few rows (every 10Kth row) are the constant "const" +//! and a few rows (every 10Kth row) are the constant "const" //! - ts: timestamp values (using, e.g., a millisecond epoch) //! //! It then applies several filter functions and projections, benchmarking the read-back speed. @@ -135,7 +135,7 @@ fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { .unwrap(); let mut builder = BooleanBuilder::with_capacity(array.len()); for i in 0..array.len() { - let keep = array.value(i) != ""; + let keep = !array.value(i).is_empty(); builder.append_value(keep); } builder.finish() @@ -227,7 +227,8 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); // Define filter functions associated with each FilterType. - let filter_funcs: Vec<(FilterType, fn(&RecordBatch) -> BooleanArray)> = vec![ + type FilterFn = fn(&RecordBatch) -> BooleanArray; + let filter_funcs: Vec<(FilterType, FilterFn)> = vec![ (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), (FilterType::Utf8ViewConst, filter_utf8_view_const), (FilterType::Int64Even, filter_int64_even), From b16428db3fdeebf5085dde4486e987dbcfaa5ed4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 17:15:32 +0800 Subject: [PATCH 29/73] change bench mark to use asyn read to trigger the page cache --- 7401.patch | 391 +++++++++++++++++++++ parquet/Cargo.toml | 5 +- parquet/benches/arrow_reader_row_filter.rs | 88 ++--- 3 files changed, 441 insertions(+), 43 deletions(-) create mode 100644 7401.patch diff --git a/7401.patch b/7401.patch new file mode 100644 index 000000000000..0bc9bce05063 --- /dev/null +++ b/7401.patch @@ -0,0 +1,391 @@ +From 822760c0df669118a0d4239acf84209c8d1f2cd4 Mon Sep 17 00:00:00 2001 +From: zhuqi-lucas <821684824@qq.com> +Date: Thu, 10 Apr 2025 22:49:38 +0800 +Subject: [PATCH 1/2] Add benchmark for parquet reader with row_filter and + project settings + +--- + parquet/Cargo.toml | 4 + + parquet/benches/arrow_reader_row_filter.rs | 316 +++++++++++++++++++++ + 2 files changed, 320 insertions(+) + create mode 100644 parquet/benches/arrow_reader_row_filter.rs + +diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml +index 2f31a290e398..80b398df21de 100644 +--- a/parquet/Cargo.toml ++++ b/parquet/Cargo.toml +@@ -211,6 +211,10 @@ name = "arrow_statistics" + required-features = ["arrow"] + harness = false + ++[[bench]] ++name = "arrow_reader_row_filter" ++required-features = ["arrow"] ++harness = false + + [[bench]] + name = "compression" +diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs +new file mode 100644 +index 000000000000..788ca3e533b7 +--- /dev/null ++++ b/parquet/benches/arrow_reader_row_filter.rs +@@ -0,0 +1,316 @@ ++// 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. ++ ++//! Benchmark for evaluating row filters and projections on a Parquet file. ++//! ++//! This benchmark creates a Parquet file in memory with 100K rows and four columns: ++//! - int64: sequential integers ++//! - float64: floating-point values (derived from the integers) ++//! - utf8View: string values where about half are non-empty, ++//! and a few rows (every 10Kth row) are the constant "const" ++//! - ts: timestamp values (using, e.g., a millisecond epoch) ++//! ++//! It then applies several filter functions and projections, benchmarking the read-back speed. ++//! ++//! Filters tested: ++//! - A string filter: `utf8View <> ''` (non-empty) ++//! - A string filter: `utf8View = 'const'` (selective) ++//! - An integer non-selective filter (e.g. even numbers) ++//! - An integer selective filter (e.g. `int64 = 0`) ++//! - A timestamp filter (e.g. `ts > threshold`) ++//! ++//! Projections tested: ++//! - All 4 columns. ++//! - All columns except the one used for the filter. ++//! ++//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. ++ ++use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; ++use std::sync::Arc; ++use tempfile::NamedTempFile; ++ ++use arrow::array::{ ++ ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, ++}; ++use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; ++use arrow::record_batch::RecordBatch; ++use arrow_array::builder::StringViewBuilder; ++use arrow_array::{Array, StringViewArray}; ++use parquet::arrow::arrow_reader::{ ++ ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, RowFilter, ++}; ++use parquet::arrow::{ArrowWriter, ProjectionMask}; ++use parquet::file::properties::WriterProperties; ++ ++/// Create a RecordBatch with 100K rows and four columns. ++fn make_record_batch() -> RecordBatch { ++ let num_rows = 100_000; ++ ++ // int64 column: sequential numbers 0..num_rows ++ let int_values: Vec = (0..num_rows as i64).collect(); ++ let int_array = Arc::new(Int64Array::from(int_values)) as ArrayRef; ++ ++ // float64 column: derived from int64 (e.g., multiplied by 0.1) ++ let float_values: Vec = (0..num_rows).map(|i| i as f64 * 0.1).collect(); ++ let float_array = Arc::new(Float64Array::from(float_values)) as ArrayRef; ++ ++ // utf8View column: even rows get non-empty strings; odd rows get an empty string; ++ // every 10Kth even row is "const" to be selective. ++ let mut string_view_builder = StringViewBuilder::with_capacity(100_000); ++ for i in 0..num_rows { ++ if i % 2 == 0 { ++ if i % 10_000 == 0 { ++ string_view_builder.append_value("const"); ++ } else { ++ string_view_builder.append_value("nonempty"); ++ } ++ } else { ++ string_view_builder.append_value(""); ++ } ++ } ++ let utf8_view_array = Arc::new(string_view_builder.finish()) as ArrayRef; ++ ++ // Timestamp column: using milliseconds from an epoch (simply using the row index) ++ let ts_values: Vec = (0..num_rows as i64).collect(); ++ let ts_array = Arc::new(TimestampMillisecondArray::from(ts_values)) as ArrayRef; ++ ++ let schema = Arc::new(Schema::new(vec![ ++ Field::new("int64", DataType::Int64, false), ++ Field::new("float64", DataType::Float64, false), ++ Field::new("utf8View", DataType::Utf8View, false), ++ Field::new( ++ "ts", ++ DataType::Timestamp(TimeUnit::Millisecond, None), ++ false, ++ ), ++ ])); ++ ++ RecordBatch::try_new( ++ schema, ++ vec![int_array, float_array, utf8_view_array, ts_array], ++ ) ++ .unwrap() ++} ++ ++/// Writes the record batch to a temporary Parquet file. ++fn write_parquet_file() -> NamedTempFile { ++ let batch = make_record_batch(); ++ let schema = batch.schema(); ++ let props = WriterProperties::builder().build(); ++ ++ let file = tempfile::Builder::new() ++ .suffix(".parquet") ++ .tempfile() ++ .unwrap(); ++ { ++ let file_reopen = file.reopen().unwrap(); ++ let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); ++ // Write the entire batch as a single row group. ++ writer.write(&batch).unwrap(); ++ writer.close().unwrap(); ++ } ++ file ++} ++ ++/// Filter function: returns a BooleanArray with true when utf8View <> "". ++fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { ++ let array = batch ++ .column(batch.schema().index_of("utf8View").unwrap()) ++ .as_any() ++ .downcast_ref::() ++ .unwrap(); ++ let mut builder = BooleanBuilder::with_capacity(array.len()); ++ for i in 0..array.len() { ++ let keep = array.value(i) != ""; ++ builder.append_value(keep); ++ } ++ builder.finish() ++} ++ ++/// Filter function: returns a BooleanArray with true when utf8View == "const". ++fn filter_utf8_view_const(batch: &RecordBatch) -> BooleanArray { ++ let array = batch ++ .column(batch.schema().index_of("utf8View").unwrap()) ++ .as_any() ++ .downcast_ref::() ++ .unwrap(); ++ let mut builder = BooleanBuilder::with_capacity(array.len()); ++ for i in 0..array.len() { ++ let keep = array.value(i) == "const"; ++ builder.append_value(keep); ++ } ++ builder.finish() ++} ++ ++/// Integer non-selective filter: returns true for even numbers. ++fn filter_int64_even(batch: &RecordBatch) -> BooleanArray { ++ let array = batch ++ .column(batch.schema().index_of("int64").unwrap()) ++ .as_any() ++ .downcast_ref::() ++ .unwrap(); ++ let mut builder = BooleanBuilder::with_capacity(array.len()); ++ for i in 0..array.len() { ++ let keep = array.value(i) % 2 == 0; ++ builder.append_value(keep); ++ } ++ builder.finish() ++} ++ ++/// Integer selective filter: returns true only when int64 equals 0. ++fn filter_int64_eq_zero(batch: &RecordBatch) -> BooleanArray { ++ let array = batch ++ .column(batch.schema().index_of("int64").unwrap()) ++ .as_any() ++ .downcast_ref::() ++ .unwrap(); ++ let mut builder = BooleanBuilder::with_capacity(array.len()); ++ for i in 0..array.len() { ++ let keep = array.value(i) == 0; ++ builder.append_value(keep); ++ } ++ builder.finish() ++} ++ ++/// Timestamp filter: returns true when ts > threshold (using 50_000 as example threshold). ++fn filter_timestamp_gt(batch: &RecordBatch) -> BooleanArray { ++ let array = batch ++ .column(batch.schema().index_of("ts").unwrap()) ++ .as_any() ++ .downcast_ref::() ++ .unwrap(); ++ let threshold = 50_000; ++ let mut builder = BooleanBuilder::with_capacity(array.len()); ++ for i in 0..array.len() { ++ let keep = array.value(i) > threshold; ++ builder.append_value(keep); ++ } ++ builder.finish() ++} ++ ++#[derive(Clone)] ++enum FilterType { ++ Utf8ViewNonEmpty, ++ Utf8ViewConst, ++ Int64Even, ++ Int64EqZero, ++ TimestampGt, ++} ++ ++impl std::fmt::Display for FilterType { ++ fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { ++ match self { ++ FilterType::Utf8ViewNonEmpty => write!(f, "utf8View <> ''"), ++ FilterType::Utf8ViewConst => write!(f, "utf8View = 'const'"), ++ FilterType::Int64Even => write!(f, "int64 even"), ++ FilterType::Int64EqZero => write!(f, "int64 = 0"), ++ FilterType::TimestampGt => write!(f, "ts > 50_000"), ++ } ++ } ++} ++ ++fn benchmark_filters_and_projections(c: &mut Criterion) { ++ let parquet_file = write_parquet_file(); ++ ++ // Define filter functions associated with each FilterType. ++ let filter_funcs: Vec<(FilterType, fn(&RecordBatch) -> BooleanArray)> = vec![ ++ (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), ++ (FilterType::Utf8ViewConst, filter_utf8_view_const), ++ (FilterType::Int64Even, filter_int64_even), ++ (FilterType::Int64EqZero, filter_int64_eq_zero), ++ (FilterType::TimestampGt, filter_timestamp_gt), ++ ]; ++ ++ let mut group = c.benchmark_group("arrow_reader_row_filter"); ++ ++ // Iterate by value (Copy is available for FilterType and fn pointers) ++ for (filter_type, filter_fn) in filter_funcs.into_iter() { ++ for proj_case in ["all_columns", "exclude_filter_column"].iter() { ++ // Define indices for all columns: [0: "int64", 1: "float64", 2: "utf8View", 3: "ts"] ++ let all_indices = vec![0, 1, 2, 3]; ++ ++ // For the output projection, conditionally exclude the filter column. ++ let output_projection: Vec = if *proj_case == "all_columns" { ++ all_indices.clone() ++ } else { ++ all_indices ++ .into_iter() ++ .filter(|i| match filter_type { ++ FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => *i != 2, // Exclude "utf8" (index 2) ++ FilterType::Int64Even | FilterType::Int64EqZero => *i != 0, // Exclude "int64" (index 0) ++ FilterType::TimestampGt => *i != 3, // Exclude "ts" (index 3) ++ }) ++ .collect() ++ }; ++ ++ // For predicate pushdown, define a projection that includes the column required for the predicate. ++ let predicate_projection: Vec = match filter_type { ++ FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => vec![2], ++ FilterType::Int64Even | FilterType::Int64EqZero => vec![0], ++ FilterType::TimestampGt => vec![3], ++ }; ++ ++ // Create a benchmark id combining filter type and projection case. ++ let bench_id = BenchmarkId::new( ++ format!("filter_case: {} project_case: {}", filter_type, proj_case), ++ "", ++ ); ++ group.bench_function(bench_id, |b| { ++ b.iter(|| { ++ // Reopen the Parquet file for each iteration. ++ let file = parquet_file.reopen().unwrap(); ++ let options = ArrowReaderOptions::new().with_page_index(true); ++ let builder = ArrowReaderBuilder::try_new_with_options(file, options).unwrap(); ++ let file_metadata = builder.metadata().file_metadata().clone(); ++ // Build the projection mask from the output projection (clone to avoid move) ++ let mask = ProjectionMask::roots( ++ file_metadata.schema_descr(), ++ output_projection.clone(), ++ ); ++ ++ // Build the predicate mask from the predicate projection (clone to avoid move) ++ let pred_mask = ProjectionMask::roots( ++ file_metadata.schema_descr(), ++ predicate_projection.clone(), ++ ); ++ ++ // Copy the filter function pointer. ++ let f = filter_fn; ++ // Wrap the filter function in a closure to satisfy the expected signature. ++ let filter = ++ ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| Ok(f(&batch))); ++ let row_filter = RowFilter::new(vec![Box::new(filter)]); ++ ++ // Build the reader with row filter and output projection. ++ let reader = builder ++ .with_row_filter(row_filter) ++ .with_projection(mask) ++ .build() ++ .unwrap(); ++ ++ // Collect result batches, unwrapping errors. ++ let _result: Vec = reader.map(|r| r.unwrap()).collect(); ++ }); ++ }); ++ } ++ } ++ ++ group.finish(); ++} ++ ++criterion_group!(benches, benchmark_filters_and_projections); ++criterion_main!(benches); + +From 31a544fd9bc5506d8c52014a4d7c922680ec1b08 Mon Sep 17 00:00:00 2001 +From: zhuqi-lucas <821684824@qq.com> +Date: Thu, 10 Apr 2025 23:03:51 +0800 +Subject: [PATCH 2/2] fix clippy + +--- + parquet/benches/arrow_reader_row_filter.rs | 7 ++++--- + 1 file changed, 4 insertions(+), 3 deletions(-) + +diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs +index 788ca3e533b7..5284275c97bf 100644 +--- a/parquet/benches/arrow_reader_row_filter.rs ++++ b/parquet/benches/arrow_reader_row_filter.rs +@@ -21,7 +21,7 @@ + //! - int64: sequential integers + //! - float64: floating-point values (derived from the integers) + //! - utf8View: string values where about half are non-empty, +-//! and a few rows (every 10Kth row) are the constant "const" ++//! and a few rows (every 10Kth row) are the constant "const" + //! - ts: timestamp values (using, e.g., a millisecond epoch) + //! + //! It then applies several filter functions and projections, benchmarking the read-back speed. +@@ -135,7 +135,7 @@ fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { +- let keep = array.value(i) != ""; ++ let keep = !array.value(i).is_empty(); + builder.append_value(keep); + } + builder.finish() +@@ -227,7 +227,8 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { + let parquet_file = write_parquet_file(); + + // Define filter functions associated with each FilterType. +- let filter_funcs: Vec<(FilterType, fn(&RecordBatch) -> BooleanArray)> = vec![ ++ type FilterFn = fn(&RecordBatch) -> BooleanArray; ++ let filter_funcs: Vec<(FilterType, FilterFn)> = vec![ + (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), + (FilterType::Utf8ViewConst, filter_utf8_view_const), + (FilterType::Int64Even, filter_int64_even), diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index 80b398df21de..eda58e751a62 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -74,8 +74,7 @@ ring = { version = "0.17", default-features = false, features = ["std"], optiona [dev-dependencies] base64 = { version = "0.22", default-features = false, features = ["std"] } -criterion = { version = "0.5", default-features = false } -snap = { version = "1.0", default-features = false } +criterion = { version = "0.5", default-features = false, features = ["async_futures"] } tempfile = { version = "3.0", default-features = false } brotli = { version = "7.0", default-features = false, features = ["std"] } flate2 = { version = "1.0", default-features = false, features = ["rust_backend"] } @@ -213,7 +212,7 @@ harness = false [[bench]] name = "arrow_reader_row_filter" -required-features = ["arrow"] +required-features = ["arrow", "async"] harness = false [[bench]] diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 5284275c97bf..af07636e49de 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -50,11 +50,13 @@ use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arrow::record_batch::RecordBatch; use arrow_array::builder::StringViewBuilder; use arrow_array::{Array, StringViewArray}; -use parquet::arrow::arrow_reader::{ - ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, RowFilter, -}; -use parquet::arrow::{ArrowWriter, ProjectionMask}; +use criterion::async_executor::FuturesExecutor; +use futures::TryStreamExt; +use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::properties::WriterProperties; +use tokio::fs::File; +use tokio::runtime::Runtime; /// Create a RecordBatch with 100K rows and four columns. fn make_record_batch() -> RecordBatch { @@ -226,6 +228,8 @@ impl std::fmt::Display for FilterType { fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); + let runtime = Runtime::new().unwrap(); // Create a new Tokio runtime + // Define filter functions associated with each FilterType. type FilterFn = fn(&RecordBatch) -> BooleanArray; let filter_funcs: Vec<(FilterType, FilterFn)> = vec![ @@ -270,47 +274,51 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { format!("filter_case: {} project_case: {}", filter_type, proj_case), "", ); + group.bench_function(bench_id, |b| { - b.iter(|| { - // Reopen the Parquet file for each iteration. - let file = parquet_file.reopen().unwrap(); - let options = ArrowReaderOptions::new().with_page_index(true); - let builder = ArrowReaderBuilder::try_new_with_options(file, options).unwrap(); - let file_metadata = builder.metadata().file_metadata().clone(); - // Build the projection mask from the output projection (clone to avoid move) - let mask = ProjectionMask::roots( - file_metadata.schema_descr(), - output_projection.clone(), - ); - - // Build the predicate mask from the predicate projection (clone to avoid move) - let pred_mask = ProjectionMask::roots( - file_metadata.schema_descr(), - predicate_projection.clone(), - ); - - // Copy the filter function pointer. - let f = filter_fn; - // Wrap the filter function in a closure to satisfy the expected signature. - let filter = - ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| Ok(f(&batch))); - let row_filter = RowFilter::new(vec![Box::new(filter)]); - - // Build the reader with row filter and output projection. - let reader = builder - .with_row_filter(row_filter) - .with_projection(mask) - .build() - .unwrap(); - - // Collect result batches, unwrapping errors. - let _result: Vec = reader.map(|r| r.unwrap()).collect(); + b.to_async(FuturesExecutor).iter(|| async { + runtime.block_on(async { + // Reopen the Parquet file for each iteration. + let file = File::open(parquet_file.path()).await.unwrap(); + + // Create a async parquet reader builder with batch_size. + let options = ArrowReaderOptions::new().with_page_index(true); + + let builder = + ParquetRecordBatchStreamBuilder::new_with_options(file, options) + .await + .unwrap() + .with_batch_size(8192); + + let file_metadata = builder.metadata().file_metadata().clone(); + + let mask = ProjectionMask::roots( + file_metadata.schema_descr(), + output_projection.clone(), + ); + + let pred_mask = ProjectionMask::roots( + file_metadata.schema_descr(), + predicate_projection.clone(), + ); + + let f = filter_fn; + let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { + Ok(f(&batch)) + }); + let stream = builder + .with_projection(mask) + .with_row_filter(RowFilter::new(vec![Box::new(filter)])) + .build() + .unwrap(); + + // Collect the results into a vector of RecordBatches. + stream.try_collect::>().await.unwrap(); + }) }); }); } } - - group.finish(); } criterion_group!(benches, benchmark_filters_and_projections); From 1aacd01ee29cbdff5e7ab7ce9edf4093b9116978 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 17:16:40 +0800 Subject: [PATCH 30/73] fix --- 7401.patch | 391 ----------------------------------------------------- 1 file changed, 391 deletions(-) delete mode 100644 7401.patch diff --git a/7401.patch b/7401.patch deleted file mode 100644 index 0bc9bce05063..000000000000 --- a/7401.patch +++ /dev/null @@ -1,391 +0,0 @@ -From 822760c0df669118a0d4239acf84209c8d1f2cd4 Mon Sep 17 00:00:00 2001 -From: zhuqi-lucas <821684824@qq.com> -Date: Thu, 10 Apr 2025 22:49:38 +0800 -Subject: [PATCH 1/2] Add benchmark for parquet reader with row_filter and - project settings - ---- - parquet/Cargo.toml | 4 + - parquet/benches/arrow_reader_row_filter.rs | 316 +++++++++++++++++++++ - 2 files changed, 320 insertions(+) - create mode 100644 parquet/benches/arrow_reader_row_filter.rs - -diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml -index 2f31a290e398..80b398df21de 100644 ---- a/parquet/Cargo.toml -+++ b/parquet/Cargo.toml -@@ -211,6 +211,10 @@ name = "arrow_statistics" - required-features = ["arrow"] - harness = false - -+[[bench]] -+name = "arrow_reader_row_filter" -+required-features = ["arrow"] -+harness = false - - [[bench]] - name = "compression" -diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs -new file mode 100644 -index 000000000000..788ca3e533b7 ---- /dev/null -+++ b/parquet/benches/arrow_reader_row_filter.rs -@@ -0,0 +1,316 @@ -+// 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. -+ -+//! Benchmark for evaluating row filters and projections on a Parquet file. -+//! -+//! This benchmark creates a Parquet file in memory with 100K rows and four columns: -+//! - int64: sequential integers -+//! - float64: floating-point values (derived from the integers) -+//! - utf8View: string values where about half are non-empty, -+//! and a few rows (every 10Kth row) are the constant "const" -+//! - ts: timestamp values (using, e.g., a millisecond epoch) -+//! -+//! It then applies several filter functions and projections, benchmarking the read-back speed. -+//! -+//! Filters tested: -+//! - A string filter: `utf8View <> ''` (non-empty) -+//! - A string filter: `utf8View = 'const'` (selective) -+//! - An integer non-selective filter (e.g. even numbers) -+//! - An integer selective filter (e.g. `int64 = 0`) -+//! - A timestamp filter (e.g. `ts > threshold`) -+//! -+//! Projections tested: -+//! - All 4 columns. -+//! - All columns except the one used for the filter. -+//! -+//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. -+ -+use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -+use std::sync::Arc; -+use tempfile::NamedTempFile; -+ -+use arrow::array::{ -+ ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, -+}; -+use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; -+use arrow::record_batch::RecordBatch; -+use arrow_array::builder::StringViewBuilder; -+use arrow_array::{Array, StringViewArray}; -+use parquet::arrow::arrow_reader::{ -+ ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, RowFilter, -+}; -+use parquet::arrow::{ArrowWriter, ProjectionMask}; -+use parquet::file::properties::WriterProperties; -+ -+/// Create a RecordBatch with 100K rows and four columns. -+fn make_record_batch() -> RecordBatch { -+ let num_rows = 100_000; -+ -+ // int64 column: sequential numbers 0..num_rows -+ let int_values: Vec = (0..num_rows as i64).collect(); -+ let int_array = Arc::new(Int64Array::from(int_values)) as ArrayRef; -+ -+ // float64 column: derived from int64 (e.g., multiplied by 0.1) -+ let float_values: Vec = (0..num_rows).map(|i| i as f64 * 0.1).collect(); -+ let float_array = Arc::new(Float64Array::from(float_values)) as ArrayRef; -+ -+ // utf8View column: even rows get non-empty strings; odd rows get an empty string; -+ // every 10Kth even row is "const" to be selective. -+ let mut string_view_builder = StringViewBuilder::with_capacity(100_000); -+ for i in 0..num_rows { -+ if i % 2 == 0 { -+ if i % 10_000 == 0 { -+ string_view_builder.append_value("const"); -+ } else { -+ string_view_builder.append_value("nonempty"); -+ } -+ } else { -+ string_view_builder.append_value(""); -+ } -+ } -+ let utf8_view_array = Arc::new(string_view_builder.finish()) as ArrayRef; -+ -+ // Timestamp column: using milliseconds from an epoch (simply using the row index) -+ let ts_values: Vec = (0..num_rows as i64).collect(); -+ let ts_array = Arc::new(TimestampMillisecondArray::from(ts_values)) as ArrayRef; -+ -+ let schema = Arc::new(Schema::new(vec![ -+ Field::new("int64", DataType::Int64, false), -+ Field::new("float64", DataType::Float64, false), -+ Field::new("utf8View", DataType::Utf8View, false), -+ Field::new( -+ "ts", -+ DataType::Timestamp(TimeUnit::Millisecond, None), -+ false, -+ ), -+ ])); -+ -+ RecordBatch::try_new( -+ schema, -+ vec![int_array, float_array, utf8_view_array, ts_array], -+ ) -+ .unwrap() -+} -+ -+/// Writes the record batch to a temporary Parquet file. -+fn write_parquet_file() -> NamedTempFile { -+ let batch = make_record_batch(); -+ let schema = batch.schema(); -+ let props = WriterProperties::builder().build(); -+ -+ let file = tempfile::Builder::new() -+ .suffix(".parquet") -+ .tempfile() -+ .unwrap(); -+ { -+ let file_reopen = file.reopen().unwrap(); -+ let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); -+ // Write the entire batch as a single row group. -+ writer.write(&batch).unwrap(); -+ writer.close().unwrap(); -+ } -+ file -+} -+ -+/// Filter function: returns a BooleanArray with true when utf8View <> "". -+fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { -+ let array = batch -+ .column(batch.schema().index_of("utf8View").unwrap()) -+ .as_any() -+ .downcast_ref::() -+ .unwrap(); -+ let mut builder = BooleanBuilder::with_capacity(array.len()); -+ for i in 0..array.len() { -+ let keep = array.value(i) != ""; -+ builder.append_value(keep); -+ } -+ builder.finish() -+} -+ -+/// Filter function: returns a BooleanArray with true when utf8View == "const". -+fn filter_utf8_view_const(batch: &RecordBatch) -> BooleanArray { -+ let array = batch -+ .column(batch.schema().index_of("utf8View").unwrap()) -+ .as_any() -+ .downcast_ref::() -+ .unwrap(); -+ let mut builder = BooleanBuilder::with_capacity(array.len()); -+ for i in 0..array.len() { -+ let keep = array.value(i) == "const"; -+ builder.append_value(keep); -+ } -+ builder.finish() -+} -+ -+/// Integer non-selective filter: returns true for even numbers. -+fn filter_int64_even(batch: &RecordBatch) -> BooleanArray { -+ let array = batch -+ .column(batch.schema().index_of("int64").unwrap()) -+ .as_any() -+ .downcast_ref::() -+ .unwrap(); -+ let mut builder = BooleanBuilder::with_capacity(array.len()); -+ for i in 0..array.len() { -+ let keep = array.value(i) % 2 == 0; -+ builder.append_value(keep); -+ } -+ builder.finish() -+} -+ -+/// Integer selective filter: returns true only when int64 equals 0. -+fn filter_int64_eq_zero(batch: &RecordBatch) -> BooleanArray { -+ let array = batch -+ .column(batch.schema().index_of("int64").unwrap()) -+ .as_any() -+ .downcast_ref::() -+ .unwrap(); -+ let mut builder = BooleanBuilder::with_capacity(array.len()); -+ for i in 0..array.len() { -+ let keep = array.value(i) == 0; -+ builder.append_value(keep); -+ } -+ builder.finish() -+} -+ -+/// Timestamp filter: returns true when ts > threshold (using 50_000 as example threshold). -+fn filter_timestamp_gt(batch: &RecordBatch) -> BooleanArray { -+ let array = batch -+ .column(batch.schema().index_of("ts").unwrap()) -+ .as_any() -+ .downcast_ref::() -+ .unwrap(); -+ let threshold = 50_000; -+ let mut builder = BooleanBuilder::with_capacity(array.len()); -+ for i in 0..array.len() { -+ let keep = array.value(i) > threshold; -+ builder.append_value(keep); -+ } -+ builder.finish() -+} -+ -+#[derive(Clone)] -+enum FilterType { -+ Utf8ViewNonEmpty, -+ Utf8ViewConst, -+ Int64Even, -+ Int64EqZero, -+ TimestampGt, -+} -+ -+impl std::fmt::Display for FilterType { -+ fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { -+ match self { -+ FilterType::Utf8ViewNonEmpty => write!(f, "utf8View <> ''"), -+ FilterType::Utf8ViewConst => write!(f, "utf8View = 'const'"), -+ FilterType::Int64Even => write!(f, "int64 even"), -+ FilterType::Int64EqZero => write!(f, "int64 = 0"), -+ FilterType::TimestampGt => write!(f, "ts > 50_000"), -+ } -+ } -+} -+ -+fn benchmark_filters_and_projections(c: &mut Criterion) { -+ let parquet_file = write_parquet_file(); -+ -+ // Define filter functions associated with each FilterType. -+ let filter_funcs: Vec<(FilterType, fn(&RecordBatch) -> BooleanArray)> = vec![ -+ (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), -+ (FilterType::Utf8ViewConst, filter_utf8_view_const), -+ (FilterType::Int64Even, filter_int64_even), -+ (FilterType::Int64EqZero, filter_int64_eq_zero), -+ (FilterType::TimestampGt, filter_timestamp_gt), -+ ]; -+ -+ let mut group = c.benchmark_group("arrow_reader_row_filter"); -+ -+ // Iterate by value (Copy is available for FilterType and fn pointers) -+ for (filter_type, filter_fn) in filter_funcs.into_iter() { -+ for proj_case in ["all_columns", "exclude_filter_column"].iter() { -+ // Define indices for all columns: [0: "int64", 1: "float64", 2: "utf8View", 3: "ts"] -+ let all_indices = vec![0, 1, 2, 3]; -+ -+ // For the output projection, conditionally exclude the filter column. -+ let output_projection: Vec = if *proj_case == "all_columns" { -+ all_indices.clone() -+ } else { -+ all_indices -+ .into_iter() -+ .filter(|i| match filter_type { -+ FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => *i != 2, // Exclude "utf8" (index 2) -+ FilterType::Int64Even | FilterType::Int64EqZero => *i != 0, // Exclude "int64" (index 0) -+ FilterType::TimestampGt => *i != 3, // Exclude "ts" (index 3) -+ }) -+ .collect() -+ }; -+ -+ // For predicate pushdown, define a projection that includes the column required for the predicate. -+ let predicate_projection: Vec = match filter_type { -+ FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => vec![2], -+ FilterType::Int64Even | FilterType::Int64EqZero => vec![0], -+ FilterType::TimestampGt => vec![3], -+ }; -+ -+ // Create a benchmark id combining filter type and projection case. -+ let bench_id = BenchmarkId::new( -+ format!("filter_case: {} project_case: {}", filter_type, proj_case), -+ "", -+ ); -+ group.bench_function(bench_id, |b| { -+ b.iter(|| { -+ // Reopen the Parquet file for each iteration. -+ let file = parquet_file.reopen().unwrap(); -+ let options = ArrowReaderOptions::new().with_page_index(true); -+ let builder = ArrowReaderBuilder::try_new_with_options(file, options).unwrap(); -+ let file_metadata = builder.metadata().file_metadata().clone(); -+ // Build the projection mask from the output projection (clone to avoid move) -+ let mask = ProjectionMask::roots( -+ file_metadata.schema_descr(), -+ output_projection.clone(), -+ ); -+ -+ // Build the predicate mask from the predicate projection (clone to avoid move) -+ let pred_mask = ProjectionMask::roots( -+ file_metadata.schema_descr(), -+ predicate_projection.clone(), -+ ); -+ -+ // Copy the filter function pointer. -+ let f = filter_fn; -+ // Wrap the filter function in a closure to satisfy the expected signature. -+ let filter = -+ ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| Ok(f(&batch))); -+ let row_filter = RowFilter::new(vec![Box::new(filter)]); -+ -+ // Build the reader with row filter and output projection. -+ let reader = builder -+ .with_row_filter(row_filter) -+ .with_projection(mask) -+ .build() -+ .unwrap(); -+ -+ // Collect result batches, unwrapping errors. -+ let _result: Vec = reader.map(|r| r.unwrap()).collect(); -+ }); -+ }); -+ } -+ } -+ -+ group.finish(); -+} -+ -+criterion_group!(benches, benchmark_filters_and_projections); -+criterion_main!(benches); - -From 31a544fd9bc5506d8c52014a4d7c922680ec1b08 Mon Sep 17 00:00:00 2001 -From: zhuqi-lucas <821684824@qq.com> -Date: Thu, 10 Apr 2025 23:03:51 +0800 -Subject: [PATCH 2/2] fix clippy - ---- - parquet/benches/arrow_reader_row_filter.rs | 7 ++++--- - 1 file changed, 4 insertions(+), 3 deletions(-) - -diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs -index 788ca3e533b7..5284275c97bf 100644 ---- a/parquet/benches/arrow_reader_row_filter.rs -+++ b/parquet/benches/arrow_reader_row_filter.rs -@@ -21,7 +21,7 @@ - //! - int64: sequential integers - //! - float64: floating-point values (derived from the integers) - //! - utf8View: string values where about half are non-empty, --//! and a few rows (every 10Kth row) are the constant "const" -+//! and a few rows (every 10Kth row) are the constant "const" - //! - ts: timestamp values (using, e.g., a millisecond epoch) - //! - //! It then applies several filter functions and projections, benchmarking the read-back speed. -@@ -135,7 +135,7 @@ fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { - .unwrap(); - let mut builder = BooleanBuilder::with_capacity(array.len()); - for i in 0..array.len() { -- let keep = array.value(i) != ""; -+ let keep = !array.value(i).is_empty(); - builder.append_value(keep); - } - builder.finish() -@@ -227,7 +227,8 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { - let parquet_file = write_parquet_file(); - - // Define filter functions associated with each FilterType. -- let filter_funcs: Vec<(FilterType, fn(&RecordBatch) -> BooleanArray)> = vec![ -+ type FilterFn = fn(&RecordBatch) -> BooleanArray; -+ let filter_funcs: Vec<(FilterType, FilterFn)> = vec![ - (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), - (FilterType::Utf8ViewConst, filter_utf8_view_const), - (FilterType::Int64Even, filter_int64_even), From 768826e2e7776108e6fdb99ca3c7c27939526b17 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 17:48:43 +0800 Subject: [PATCH 31/73] fix --- parquet/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index eda58e751a62..1d2737a0c629 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -75,6 +75,7 @@ ring = { version = "0.17", default-features = false, features = ["std"], optiona [dev-dependencies] base64 = { version = "0.22", default-features = false, features = ["std"] } criterion = { version = "0.5", default-features = false, features = ["async_futures"] } +snap = { version = "1.0", default-features = false } tempfile = { version = "3.0", default-features = false } brotli = { version = "7.0", default-features = false, features = ["std"] } flate2 = { version = "1.0", default-features = false, features = ["rust_backend"] } From f624b91c8ad18ffa486c3110b8ef29ed4cc4cd27 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 11 Apr 2025 09:47:21 -0400 Subject: [PATCH 32/73] Update comments, add background --- parquet/benches/arrow_reader_row_filter.rs | 53 +++++++++++++++------- 1 file changed, 37 insertions(+), 16 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index af07636e49de..da89279065e5 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -17,26 +17,28 @@ //! Benchmark for evaluating row filters and projections on a Parquet file. //! -//! This benchmark creates a Parquet file in memory with 100K rows and four columns: -//! - int64: sequential integers -//! - float64: floating-point values (derived from the integers) -//! - utf8View: string values where about half are non-empty, -//! and a few rows (every 10Kth row) are the constant "const" -//! - ts: timestamp values (using, e.g., a millisecond epoch) +//! # Background: //! -//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! As described in [Efficient Filter Pushdown in Parquet], evaluating +//! pushdown filters is a two step process: //! -//! Filters tested: -//! - A string filter: `utf8View <> ''` (non-empty) -//! - A string filter: `utf8View = 'const'` (selective) -//! - An integer non-selective filter (e.g. even numbers) -//! - An integer selective filter (e.g. `int64 = 0`) -//! - A timestamp filter (e.g. `ts > threshold`) +//! 1. Build a filter mask by decoding and evaluating filter functions on +//! the filter column(s). //! -//! Projections tested: -//! - All 4 columns. -//! - All columns except the one used for the filter. +//! 2. Decode the rows that match the filter mask from the projected columns. //! +//! The performance of this process depending on several factors, including: +//! +//! 1. How many rows are selected as well and how well clustered the results +//! are, where the representation of the filter mask is important. +//! 2. If the same column is used for both filtering and projection, as the +//! columns that appear in both filtering and projection are decoded twice. +//! +//! This benchmark helps measure the performance of these operations. +//! +//! [Efficient Filter Pushdown in Parquet]: https://datafusion.apache.org/blog/2025/03/21/parquet-pushdown/ +//! +//! # To run: //! To run the benchmark, use `cargo bench --bench bench_filter_projection`. use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; @@ -59,6 +61,12 @@ use tokio::fs::File; use tokio::runtime::Runtime; /// Create a RecordBatch with 100K rows and four columns. +/// +/// - int64: sequential integers +/// - float64: floating-point values (derived from the integers) +/// - utf8View: string values where about half are non-empty, +/// and a few rows (every 10Kth row) are the constant "const" +/// - ts: timestamp values (using, e.g., a millisecond epoch) fn make_record_batch() -> RecordBatch { let num_rows = 100_000; @@ -204,6 +212,12 @@ fn filter_timestamp_gt(batch: &RecordBatch) -> BooleanArray { builder.finish() } +/// Filters tested: +/// - A string filter: `utf8View <> ''` (non-empty) +/// - A string filter: `utf8View = 'const'` (selective) +/// - An integer non-selective filter (e.g. even numbers) +/// - An integer selective filter (e.g. `int64 = 0`) +/// - A timestamp filter (e.g. `ts > threshold`) #[derive(Clone)] enum FilterType { Utf8ViewNonEmpty, @@ -225,6 +239,13 @@ impl std::fmt::Display for FilterType { } } +/// This benchmark tests the performance of row filters and projections +/// +/// Tests combinations of FilterType and ProjectionType +/// +/// Projections tested: +/// - All 4 columns. +/// - All columns except the one used for the filter. fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); From 6c28e447fa5d6d9919273be3f68771537bd1a5b3 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 22:28:58 +0800 Subject: [PATCH 33/73] incremently addressing the comments --- parquet/benches/arrow_reader_row_filter.rs | 289 ++++++++++----------- 1 file changed, 134 insertions(+), 155 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index af07636e49de..69718d0d7160 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -18,97 +18,136 @@ //! Benchmark for evaluating row filters and projections on a Parquet file. //! //! This benchmark creates a Parquet file in memory with 100K rows and four columns: -//! - int64: sequential integers -//! - float64: floating-point values (derived from the integers) -//! - utf8View: string values where about half are non-empty, -//! and a few rows (every 10Kth row) are the constant "const" -//! - ts: timestamp values (using, e.g., a millisecond epoch) -//! -//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! - int64: random integers generated using a fixed seed (range: 0..100) +//! - float64: random floating-point values generated using a fixed seed (range: 0.0..100.0) +//! - utf8View: random strings (with some empty values and the constant "const"). +//! Randomly produces short strings (3-12 bytes) and long strings (13-20 bytes). +//! - ts: sequential timestamps in milliseconds //! //! Filters tested: -//! - A string filter: `utf8View <> ''` (non-empty) -//! - A string filter: `utf8View = 'const'` (selective) -//! - An integer non-selective filter (e.g. even numbers) -//! - An integer selective filter (e.g. `int64 = 0`) -//! - A timestamp filter (e.g. `ts > threshold`) +//! - utf8View <> '' (no selective) %80 +//! - utf8View = 'const' (selective) %5 +//! - int64 = 0 (selective) +//! - ts > 50_000 (no selective) %50 //! //! Projections tested: -//! - All 4 columns. -//! - All columns except the one used for the filter. +//! - All columns. +//! - All columns except the one used for filtering. //! //! To run the benchmark, use `cargo bench --bench bench_filter_projection`. use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use rand::{rngs::StdRng, Rng, SeedableRng}; use std::sync::Arc; use tempfile::NamedTempFile; -use arrow::array::{ - ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, -}; -use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; -use arrow::record_batch::RecordBatch; +use arrow::array::{ArrayRef, Float64Array, Int64Array, TimestampMillisecondArray}; +use arrow::compute::kernels::cmp::{eq, gt, neq}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; +use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use arrow_array::builder::StringViewBuilder; -use arrow_array::{Array, StringViewArray}; -use criterion::async_executor::FuturesExecutor; +use arrow_array::StringViewArray; +use arrow_cast::pretty::pretty_format_batches; use futures::TryStreamExt; use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::properties::WriterProperties; use tokio::fs::File; -use tokio::runtime::Runtime; - -/// Create a RecordBatch with 100K rows and four columns. -fn make_record_batch() -> RecordBatch { - let num_rows = 100_000; - - // int64 column: sequential numbers 0..num_rows - let int_values: Vec = (0..num_rows as i64).collect(); - let int_array = Arc::new(Int64Array::from(int_values)) as ArrayRef; - // float64 column: derived from int64 (e.g., multiplied by 0.1) - let float_values: Vec = (0..num_rows).map(|i| i as f64 * 0.1).collect(); - let float_array = Arc::new(Float64Array::from(float_values)) as ArrayRef; - - // utf8View column: even rows get non-empty strings; odd rows get an empty string; - // every 10Kth even row is "const" to be selective. - let mut string_view_builder = StringViewBuilder::with_capacity(100_000); - for i in 0..num_rows { - if i % 2 == 0 { - if i % 10_000 == 0 { - string_view_builder.append_value("const"); - } else { - string_view_builder.append_value("nonempty"); +fn create_random_array( + field: &Field, + size: usize, + null_density: f32, + _true_density: f32, +) -> arrow::error::Result { + match field.data_type() { + DataType::Int64 => { + let mut rng = StdRng::seed_from_u64(42); + let values: Vec = (0..size).map(|_| rng.random_range(0..100)).collect(); + Ok(Arc::new(Int64Array::from(values)) as ArrayRef) + } + DataType::Float64 => { + let mut rng = StdRng::seed_from_u64(43); + let values: Vec = (0..size).map(|_| rng.random_range(0.0..100.0)).collect(); + Ok(Arc::new(Float64Array::from(values)) as ArrayRef) + } + DataType::Utf8View => { + let mut builder = StringViewBuilder::with_capacity(size); + let mut rng = StdRng::seed_from_u64(44); + for _ in 0..size { + let choice = rng.random_range(0..100); + if choice < (null_density * 100.0) as u32 { + builder.append_value(""); + } else if choice < 25 { + builder.append_value("const"); + } else { + let is_long = rng.random_range(0.5); + let len = if is_long { + rng.random_range(13..21) + } else { + rng.random_range(3..12) + }; + let charset = b"abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + let s: String = (0..len) + .map(|_| { + let idx = rng.random_range(0..charset.len()); + charset[idx] as char + }) + .collect(); + builder.append_value(&s); + } } - } else { - string_view_builder.append_value(""); + Ok(Arc::new(builder.finish()) as ArrayRef) + } + DataType::Timestamp(TimeUnit::Millisecond, _) => { + let values: Vec = (0..size as i64).collect(); + Ok(Arc::new(TimestampMillisecondArray::from(values)) as ArrayRef) } + _ => unimplemented!("Field type not supported in create_random_array"), } - let utf8_view_array = Arc::new(string_view_builder.finish()) as ArrayRef; +} - // Timestamp column: using milliseconds from an epoch (simply using the row index) - let ts_values: Vec = (0..num_rows as i64).collect(); - let ts_array = Arc::new(TimestampMillisecondArray::from(ts_values)) as ArrayRef; +pub fn create_random_batch( + schema: SchemaRef, + size: usize, + null_density: f32, + true_density: f32, +) -> arrow::error::Result { + let columns = schema + .fields() + .iter() + .map(|field| create_random_array(field, size, null_density, true_density)) + .collect::>>()?; + RecordBatch::try_new_with_options( + schema, + columns, + &RecordBatchOptions::new().with_match_field_names(false), + ) +} - let schema = Arc::new(Schema::new(vec![ +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + let fields = vec![ Field::new("int64", DataType::Int64, false), Field::new("float64", DataType::Float64, false), - Field::new("utf8View", DataType::Utf8View, false), + Field::new("utf8View", DataType::Utf8View, true), Field::new( "ts", DataType::Timestamp(TimeUnit::Millisecond, None), false, ), - ])); - - RecordBatch::try_new( - schema, - vec![int_array, float_array, utf8_view_array, ts_array], - ) - .unwrap() + ]; + let schema = Arc::new(Schema::new(fields)); + let batch = create_random_batch(schema, num_rows, 0.2, 0.5).unwrap(); + + println!("Batch created with {} rows", num_rows); + println!( + "First 100 rows:\n{}", + pretty_format_batches(&[batch.clone().slice(0, 100)]).unwrap() + ); + batch } -/// Writes the record batch to a temporary Parquet file. fn write_parquet_file() -> NamedTempFile { let batch = make_record_batch(); let schema = batch.schema(); @@ -121,94 +160,47 @@ fn write_parquet_file() -> NamedTempFile { { let file_reopen = file.reopen().unwrap(); let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); - // Write the entire batch as a single row group. writer.write(&batch).unwrap(); writer.close().unwrap(); } file } -/// Filter function: returns a BooleanArray with true when utf8View <> "". -fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { - let array = batch - .column(batch.schema().index_of("utf8View").unwrap()) - .as_any() - .downcast_ref::() - .unwrap(); - let mut builder = BooleanBuilder::with_capacity(array.len()); - for i in 0..array.len() { - let keep = !array.value(i).is_empty(); - builder.append_value(keep); - } - builder.finish() +// Use Arrow compute kernels for filtering. +// Returns a BooleanArray where true indicates the row satisfies the condition. +fn filter_utf8_view_nonempty( + batch: &RecordBatch, +) -> arrow::error::Result { + let array = batch.column(batch.schema().index_of("utf8View").unwrap()); + let string_view_scalar = StringViewArray::new_scalar(""); + // Compare with empty string + let not_equals_empty = neq(array, &string_view_scalar)?; + Ok(not_equals_empty) } -/// Filter function: returns a BooleanArray with true when utf8View == "const". -fn filter_utf8_view_const(batch: &RecordBatch) -> BooleanArray { - let array = batch - .column(batch.schema().index_of("utf8View").unwrap()) - .as_any() - .downcast_ref::() - .unwrap(); - let mut builder = BooleanBuilder::with_capacity(array.len()); - for i in 0..array.len() { - let keep = array.value(i) == "const"; - builder.append_value(keep); - } - builder.finish() +fn filter_utf8_view_const(batch: &RecordBatch) -> arrow::error::Result { + let array = batch.column(batch.schema().index_of("utf8View").unwrap()); + let string_view_scalar = StringViewArray::new_scalar("const"); + let eq_const = eq(array, &string_view_scalar)?; + Ok(eq_const) } - -/// Integer non-selective filter: returns true for even numbers. -fn filter_int64_even(batch: &RecordBatch) -> BooleanArray { - let array = batch - .column(batch.schema().index_of("int64").unwrap()) - .as_any() - .downcast_ref::() - .unwrap(); - let mut builder = BooleanBuilder::with_capacity(array.len()); - for i in 0..array.len() { - let keep = array.value(i) % 2 == 0; - builder.append_value(keep); - } - builder.finish() +fn filter_int64_eq_zero(batch: &RecordBatch) -> arrow::error::Result { + let array = batch.column(batch.schema().index_of("int64").unwrap()); + let eq_zero = eq(array, &Int64Array::new_scalar(0))?; + Ok(eq_zero) } -/// Integer selective filter: returns true only when int64 equals 0. -fn filter_int64_eq_zero(batch: &RecordBatch) -> BooleanArray { - let array = batch - .column(batch.schema().index_of("int64").unwrap()) - .as_any() - .downcast_ref::() - .unwrap(); - let mut builder = BooleanBuilder::with_capacity(array.len()); - for i in 0..array.len() { - let keep = array.value(i) == 0; - builder.append_value(keep); - } - builder.finish() -} - -/// Timestamp filter: returns true when ts > threshold (using 50_000 as example threshold). -fn filter_timestamp_gt(batch: &RecordBatch) -> BooleanArray { - let array = batch - .column(batch.schema().index_of("ts").unwrap()) - .as_any() - .downcast_ref::() - .unwrap(); - let threshold = 50_000; - let mut builder = BooleanBuilder::with_capacity(array.len()); - for i in 0..array.len() { - let keep = array.value(i) > threshold; - builder.append_value(keep); - } - builder.finish() +fn filter_timestamp_gt(batch: &RecordBatch) -> arrow::error::Result { + let array = batch.column(batch.schema().index_of("ts").unwrap()); + // For Timestamp arrays, use ScalarValue::TimestampMillisecond. + let gt_thresh = gt(array, &TimestampMillisecondArray::new_scalar(50_000))?; + Ok(gt_thresh) } #[derive(Clone)] enum FilterType { Utf8ViewNonEmpty, Utf8ViewConst, - Int64Even, Int64EqZero, TimestampGt, } @@ -218,7 +210,6 @@ impl std::fmt::Display for FilterType { match self { FilterType::Utf8ViewNonEmpty => write!(f, "utf8View <> ''"), FilterType::Utf8ViewConst => write!(f, "utf8View = 'const'"), - FilterType::Int64Even => write!(f, "int64 even"), FilterType::Int64EqZero => write!(f, "int64 = 0"), FilterType::TimestampGt => write!(f, "ts > 50_000"), } @@ -228,62 +219,53 @@ impl std::fmt::Display for FilterType { fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); - let runtime = Runtime::new().unwrap(); // Create a new Tokio runtime - - // Define filter functions associated with each FilterType. - type FilterFn = fn(&RecordBatch) -> BooleanArray; + type FilterFn = fn(&RecordBatch) -> arrow::error::Result; let filter_funcs: Vec<(FilterType, FilterFn)> = vec![ (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), (FilterType::Utf8ViewConst, filter_utf8_view_const), - (FilterType::Int64Even, filter_int64_even), (FilterType::Int64EqZero, filter_int64_eq_zero), (FilterType::TimestampGt, filter_timestamp_gt), ]; let mut group = c.benchmark_group("arrow_reader_row_filter"); - // Iterate by value (Copy is available for FilterType and fn pointers) for (filter_type, filter_fn) in filter_funcs.into_iter() { for proj_case in ["all_columns", "exclude_filter_column"].iter() { - // Define indices for all columns: [0: "int64", 1: "float64", 2: "utf8View", 3: "ts"] let all_indices = vec![0, 1, 2, 3]; - // For the output projection, conditionally exclude the filter column. let output_projection: Vec = if *proj_case == "all_columns" { all_indices.clone() } else { all_indices .into_iter() .filter(|i| match filter_type { - FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => *i != 2, // Exclude "utf8" (index 2) - FilterType::Int64Even | FilterType::Int64EqZero => *i != 0, // Exclude "int64" (index 0) - FilterType::TimestampGt => *i != 3, // Exclude "ts" (index 3) + FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => *i != 2, + FilterType::Int64EqZero => *i != 0, + FilterType::TimestampGt => *i != 3, }) .collect() }; - // For predicate pushdown, define a projection that includes the column required for the predicate. let predicate_projection: Vec = match filter_type { FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => vec![2], - FilterType::Int64Even | FilterType::Int64EqZero => vec![0], + FilterType::Int64EqZero => vec![0], FilterType::TimestampGt => vec![3], }; - // Create a benchmark id combining filter type and projection case. let bench_id = BenchmarkId::new( format!("filter_case: {} project_case: {}", filter_type, proj_case), "", ); group.bench_function(bench_id, |b| { - b.to_async(FuturesExecutor).iter(|| async { - runtime.block_on(async { - // Reopen the Parquet file for each iteration. + let rt = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build() + .unwrap(); + b.iter(|| { + rt.block_on(async { let file = File::open(parquet_file.path()).await.unwrap(); - - // Create a async parquet reader builder with batch_size. let options = ArrowReaderOptions::new().with_page_index(true); - let builder = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await @@ -291,12 +273,10 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .with_batch_size(8192); let file_metadata = builder.metadata().file_metadata().clone(); - let mask = ProjectionMask::roots( file_metadata.schema_descr(), output_projection.clone(), ); - let pred_mask = ProjectionMask::roots( file_metadata.schema_descr(), predicate_projection.clone(), @@ -304,7 +284,7 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { let f = filter_fn; let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { - Ok(f(&batch)) + Ok(f(&batch).unwrap()) }); let stream = builder .with_projection(mask) @@ -312,7 +292,6 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .build() .unwrap(); - // Collect the results into a vector of RecordBatches. stream.try_collect::>().await.unwrap(); }) }); From 69a2617e97fca04b47f41221437a5b1d7a79554a Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 22:33:57 +0800 Subject: [PATCH 34/73] Fix bool random --- parquet/benches/arrow_reader_row_filter.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 69718d0d7160..a3ac9537fa9f 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -81,7 +81,7 @@ fn create_random_array( } else if choice < 25 { builder.append_value("const"); } else { - let is_long = rng.random_range(0.5); + let is_long = rng.random_bool(0.5); let len = if is_long { rng.random_range(13..21) } else { From 6a378182e60cd6c6a61a34cc6dd86147c17ebd29 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 11 Apr 2025 10:38:51 -0400 Subject: [PATCH 35/73] fixup --- parquet/benches/arrow_reader_row_filter.rs | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 547a2e33dbf5..a5f1b6c30a8f 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -147,7 +147,6 @@ pub fn create_random_batch( ) } ->>>>>>> 69a2617 fn make_record_batch() -> RecordBatch { let num_rows = 100_000; let fields = vec![ @@ -220,12 +219,6 @@ fn filter_timestamp_gt(batch: &RecordBatch) -> arrow::error::Result ''` (non-empty) -/// - A string filter: `utf8View = 'const'` (selective) -/// - An integer non-selective filter (e.g. even numbers) -/// - An integer selective filter (e.g. `int64 = 0`) -/// - A timestamp filter (e.g. `ts > threshold`) #[derive(Clone)] enum FilterType { Utf8ViewNonEmpty, @@ -245,13 +238,6 @@ impl std::fmt::Display for FilterType { } } -/// This benchmark tests the performance of row filters and projections -/// -/// Tests combinations of FilterType and ProjectionType -/// -/// Projections tested: -/// - All 4 columns. -/// - All columns except the one used for the filter. fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); From 2f6ccbb2609ce76f561c468e0b45c6af4f0cdca1 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 23:01:18 +0800 Subject: [PATCH 36/73] Add fn switch and project enum --- parquet/benches/arrow_reader_row_filter.rs | 151 ++++++++++++--------- 1 file changed, 90 insertions(+), 61 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index a3ac9537fa9f..ab9e25390315 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -25,10 +25,10 @@ //! - ts: sequential timestamps in milliseconds //! //! Filters tested: -//! - utf8View <> '' (no selective) %80 -//! - utf8View = 'const' (selective) %5 +//! - utf8View <> '' (non-selective) +//! - utf8View = 'const' (selective) //! - int64 = 0 (selective) -//! - ts > 50_000 (no selective) %50 +//! - ts > 50_000 (non-selective) //! //! Projections tested: //! - All columns. @@ -41,7 +41,7 @@ use rand::{rngs::StdRng, Rng, SeedableRng}; use std::sync::Arc; use tempfile::NamedTempFile; -use arrow::array::{ArrayRef, Float64Array, Int64Array, TimestampMillisecondArray}; +use arrow::array::{ArrayRef, BooleanArray, Float64Array, Int64Array, TimestampMillisecondArray}; use arrow::compute::kernels::cmp::{eq, gt, neq}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; @@ -54,6 +54,7 @@ use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMas use parquet::file::properties::WriterProperties; use tokio::fs::File; +/// Create a random array for a given field. fn create_random_array( field: &Field, size: usize, @@ -77,6 +78,7 @@ fn create_random_array( for _ in 0..size { let choice = rng.random_range(0..100); if choice < (null_density * 100.0) as u32 { + // Use empty string to represent a null value. builder.append_value(""); } else if choice < 25 { builder.append_value("const"); @@ -107,6 +109,7 @@ fn create_random_array( } } +/// Create a random RecordBatch from the given schema. pub fn create_random_batch( schema: SchemaRef, size: usize, @@ -125,6 +128,7 @@ pub fn create_random_batch( ) } +/// Create a RecordBatch with 100K rows and four columns. fn make_record_batch() -> RecordBatch { let num_rows = 100_000; let fields = vec![ @@ -148,6 +152,7 @@ fn make_record_batch() -> RecordBatch { batch } +/// Write the RecordBatch to a temporary Parquet file. fn write_parquet_file() -> NamedTempFile { let batch = make_record_batch(); let schema = batch.schema(); @@ -166,37 +171,7 @@ fn write_parquet_file() -> NamedTempFile { file } -// Use Arrow compute kernels for filtering. -// Returns a BooleanArray where true indicates the row satisfies the condition. -fn filter_utf8_view_nonempty( - batch: &RecordBatch, -) -> arrow::error::Result { - let array = batch.column(batch.schema().index_of("utf8View").unwrap()); - let string_view_scalar = StringViewArray::new_scalar(""); - // Compare with empty string - let not_equals_empty = neq(array, &string_view_scalar)?; - Ok(not_equals_empty) -} - -fn filter_utf8_view_const(batch: &RecordBatch) -> arrow::error::Result { - let array = batch.column(batch.schema().index_of("utf8View").unwrap()); - let string_view_scalar = StringViewArray::new_scalar("const"); - let eq_const = eq(array, &string_view_scalar)?; - Ok(eq_const) -} -fn filter_int64_eq_zero(batch: &RecordBatch) -> arrow::error::Result { - let array = batch.column(batch.schema().index_of("int64").unwrap()); - let eq_zero = eq(array, &Int64Array::new_scalar(0))?; - Ok(eq_zero) -} - -fn filter_timestamp_gt(batch: &RecordBatch) -> arrow::error::Result { - let array = batch.column(batch.schema().index_of("ts").unwrap()); - // For Timestamp arrays, use ScalarValue::TimestampMillisecond. - let gt_thresh = gt(array, &TimestampMillisecondArray::new_scalar(50_000))?; - Ok(gt_thresh) -} - +/// FilterType encapsulates the different filter comparisons. #[derive(Clone)] enum FilterType { Utf8ViewNonEmpty, @@ -216,46 +191,95 @@ impl std::fmt::Display for FilterType { } } +impl FilterType { + /// Filters the given batch according to self using Arrow compute kernels. + /// Returns a BooleanArray where true indicates that the row satisfies the condition. + fn filter_batch(&self, batch: &RecordBatch) -> arrow::error::Result { + match self { + FilterType::Utf8ViewNonEmpty => { + let array = batch.column(batch.schema().index_of("utf8View").unwrap()); + let string_view_scalar = StringViewArray::new_scalar(""); + let not_equals_empty = neq(array, &string_view_scalar)?; + Ok(not_equals_empty) + } + FilterType::Utf8ViewConst => { + let array = batch.column(batch.schema().index_of("utf8View").unwrap()); + let string_view_scalar = StringViewArray::new_scalar("const"); + let eq_const = eq(array, &string_view_scalar)?; + Ok(eq_const) + } + FilterType::Int64EqZero => { + let array = batch.column(batch.schema().index_of("int64").unwrap()); + let eq_zero = eq(array, &Int64Array::new_scalar(0))?; + Ok(eq_zero) + } + FilterType::TimestampGt => { + let array = batch.column(batch.schema().index_of("ts").unwrap()); + let gt_thresh = gt(array, &TimestampMillisecondArray::new_scalar(50_000))?; + Ok(gt_thresh) + } + } + } +} + +/// ProjectionCase defines the projection mode. +#[derive(Clone)] +enum ProjectionCase { + AllColumns, + ExcludeFilterColumn, +} + +impl std::fmt::Display for ProjectionCase { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + ProjectionCase::AllColumns => write!(f, "all_columns"), + ProjectionCase::ExcludeFilterColumn => write!(f, "exclude_filter_column"), + } + } +} + fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); - type FilterFn = fn(&RecordBatch) -> arrow::error::Result; - let filter_funcs: Vec<(FilterType, FilterFn)> = vec![ - (FilterType::Utf8ViewNonEmpty, filter_utf8_view_nonempty), - (FilterType::Utf8ViewConst, filter_utf8_view_const), - (FilterType::Int64EqZero, filter_int64_eq_zero), - (FilterType::TimestampGt, filter_timestamp_gt), + let filter_types: Vec = vec![ + FilterType::Utf8ViewNonEmpty, + FilterType::Utf8ViewConst, + FilterType::Int64EqZero, + FilterType::TimestampGt, + ]; + + let projection_cases = vec![ + ProjectionCase::AllColumns, + ProjectionCase::ExcludeFilterColumn, ]; let mut group = c.benchmark_group("arrow_reader_row_filter"); - for (filter_type, filter_fn) in filter_funcs.into_iter() { - for proj_case in ["all_columns", "exclude_filter_column"].iter() { + for filter_type in filter_types.iter().cloned() { + for proj_case in &projection_cases { + // All column indices: [0: int64, 1: float64, 2: utf8View, 3: ts] let all_indices = vec![0, 1, 2, 3]; - - let output_projection: Vec = if *proj_case == "all_columns" { - all_indices.clone() - } else { - all_indices + let filter_col = match filter_type { + FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => 2, + FilterType::Int64EqZero => 0, + FilterType::TimestampGt => 3, + }; + let output_projection: Vec = match proj_case { + ProjectionCase::AllColumns => all_indices.clone(), + ProjectionCase::ExcludeFilterColumn => all_indices .into_iter() - .filter(|i| match filter_type { - FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => *i != 2, - FilterType::Int64EqZero => *i != 0, - FilterType::TimestampGt => *i != 3, - }) - .collect() + .filter(|i| *i != filter_col) + .collect(), }; - + // For predicate pushdown, include the filter column. let predicate_projection: Vec = match filter_type { FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => vec![2], FilterType::Int64EqZero => vec![0], FilterType::TimestampGt => vec![3], }; - let bench_id = BenchmarkId::new( - format!("filter_case: {} project_case: {}", filter_type, proj_case), - "", - ); + let bench_id = + BenchmarkId::new(format!("filter: {} proj: {}", filter_type, proj_case), ""); group.bench_function(bench_id, |b| { let rt = tokio::runtime::Builder::new_multi_thread() @@ -263,6 +287,9 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .build() .unwrap(); b.iter(|| { + // Clone filter_type inside the closure to avoid moving it + let filter_type_inner = filter_type.clone(); + rt.block_on(async { let file = File::open(parquet_file.path()).await.unwrap(); let options = ArrowReaderOptions::new().with_page_index(true); @@ -282,10 +309,12 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { predicate_projection.clone(), ); - let f = filter_fn; let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { - Ok(f(&batch).unwrap()) + // Clone filter_type within the closure + let filter_type_inner = filter_type_inner.clone(); + Ok(filter_type_inner.filter_batch(&batch).unwrap()) }); + let stream = builder .with_projection(mask) .with_row_filter(RowFilter::new(vec![Box::new(filter)])) From d0a656b8d2a16b594a7df52ba6f21ea0e5f3b14c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 11 Apr 2025 23:13:26 +0800 Subject: [PATCH 37/73] Fix clippy --- parquet/benches/arrow_reader_row_filter.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 83c323ff69ff..5d7e50e0912b 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -45,7 +45,7 @@ //! - int64: random integers generated using a fixed seed (range: 0..100) //! - float64: random floating-point values generated using a fixed seed (range: 0.0..100.0) //! - utf8View: random strings (with some empty values and the constant "const"). -//! Randomly produces short strings (3-12 bytes) and long strings (13-20 bytes). +//! Randomly produces short strings (3-12 bytes) and long strings (13-20 bytes). //! - ts: sequential timestamps in milliseconds //! //! Filters tested: From 67480b9a4428effcaf8b2f6f97650563669a1e02 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 12 Apr 2025 14:38:54 +0800 Subject: [PATCH 38/73] Address comment --- parquet/benches/arrow_reader_row_filter.rs | 513 +++++++++++++++------ 1 file changed, 378 insertions(+), 135 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 5d7e50e0912b..12fd02e23c93 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -20,63 +20,77 @@ //! # Background: //! //! As described in [Efficient Filter Pushdown in Parquet], evaluating -//! pushdown filters is a two step process: +//! pushdown filters is a two-step process: //! //! 1. Build a filter mask by decoding and evaluating filter functions on //! the filter column(s). //! //! 2. Decode the rows that match the filter mask from the projected columns. //! -//! The performance of this process depending on several factors, including: -//! -//! 1. How many rows are selected as well and how well clustered the results -//! are, where the representation of the filter mask is important. -//! 2. If the same column is used for both filtering and projection, as the -//! columns that appear in both filtering and projection are decoded twice. +//! The performance depends on factors such as the number of rows selected, +//! the clustering of results (which affects the efficiency of the filter mask), +//! and whether the same column is used for both filtering and projection. //! //! This benchmark helps measure the performance of these operations. //! //! [Efficient Filter Pushdown in Parquet]: https://datafusion.apache.org/blog/2025/03/21/parquet-pushdown/ //! -//! # To run: -//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. -//! -//! This benchmark creates a Parquet file in memory with 100K rows and four columns: -//! - int64: random integers generated using a fixed seed (range: 0..100) -//! - float64: random floating-point values generated using a fixed seed (range: 0.0..100.0) -//! - utf8View: random strings (with some empty values and the constant "const"). -//! Randomly produces short strings (3-12 bytes) and long strings (13-20 bytes). -//! - ts: sequential timestamps in milliseconds +//! The benchmark creates an in-memory Parquet file with 100K rows and ten columns. +//! The first four columns are: +//! - int64: random integers (range: 0..100) generated with a fixed seed. +//! - float64: random floating-point values (range: 0.0..100.0) generated with a fixed seed. +//! - utf8View: random strings with some empty values and occasional constant "const" values. +//! - ts: sequential timestamps in milliseconds. //! -//! Filters tested: -//! - utf8View <> '' (non-selective) -//! - utf8View = 'const' (selective) -//! - int64 = 0 (selective) -//! - ts > 50_000 (non-selective) +//! The following six columns (for filtering) are generated to mimic different +//! filter selectivity and clustering patterns: +//! - pt: for Point Lookup – exactly one row is set to "unique_point", all others are random strings. +//! - sel: for Selective Unclustered – exactly 1% of rows (those with i % 100 == 0) are "selected". +//! - mod_clustered: for Moderately Selective Clustered – in each 10K-row block, the first 10 rows are "mod_clustered". +//! - mod_unclustered: for Moderately Selective Unclustered – exactly 10% of rows (those with i % 10 == 1) are "mod_unclustered". +//! - unsel_unclustered: for Unselective Unclustered – exactly 99% of rows (those with i % 100 != 0) are "unsel_unclustered". +//! - unsel_clustered: for Unselective Clustered – in each 10K-row block, rows with an offset >= 1000 are "unsel_clustered". //! -//! Projections tested: -//! - All columns. -//! - All columns except the one used for filtering. - -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use rand::{rngs::StdRng, Rng, SeedableRng}; -use std::sync::Arc; -use tempfile::NamedTempFile; +//! As a side note, an additional composite benchmark is provided which demonstrates +//! the performance when applying two filters simultaneously (i.e. chaining row selectors). use arrow::array::{ArrayRef, BooleanArray, Float64Array, Int64Array, TimestampMillisecondArray}; use arrow::compute::kernels::cmp::{eq, gt, neq}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; -use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::record_batch::RecordBatch; use arrow_array::builder::StringViewBuilder; use arrow_array::StringViewArray; use arrow_cast::pretty::pretty_format_batches; +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use futures::TryStreamExt; use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::properties::WriterProperties; +use rand::{rngs::StdRng, Rng, SeedableRng}; +use std::sync::Arc; +use tempfile::NamedTempFile; use tokio::fs::File; -/// Create a random array for a given field. +/// Generates a random string (either short: 3–11 bytes or long: 13–20 bytes) with 50% probability. +/// This is used to fill non-selected rows in the filter columns. +fn random_string(rng: &mut StdRng) -> String { + let charset = b"abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + let is_long = rng.random_bool(0.5); + let len = if is_long { + rng.random_range(13..21) + } else { + rng.random_range(3..12) + }; + (0..len) + .map(|_| charset[rng.random_range(0..charset.len())] as char) + .collect() +} + +/// Create a random array for a given field, generating data with fixed seed reproducibility. +/// - For Int64, random integers in [0, 100). +/// - For Float64, random floats in [0.0, 100.0). +/// - For Utf8View, a mix of empty strings, the constant "const", and random strings. +/// - For Timestamp, sequential timestamps in milliseconds. fn create_random_array( field: &Field, size: usize, @@ -100,25 +114,11 @@ fn create_random_array( for _ in 0..size { let choice = rng.random_range(0..100); if choice < (null_density * 100.0) as u32 { - // Use empty string to represent a null value. builder.append_value(""); } else if choice < 25 { builder.append_value("const"); } else { - let is_long = rng.random_bool(0.5); - let len = if is_long { - rng.random_range(13..21) - } else { - rng.random_range(3..12) - }; - let charset = b"abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; - let s: String = (0..len) - .map(|_| { - let idx = rng.random_range(0..charset.len()); - charset[idx] as char - }) - .collect(); - builder.append_value(&s); + builder.append_value(random_string(&mut rng)); } } Ok(Arc::new(builder.finish()) as ArrayRef) @@ -131,28 +131,103 @@ fn create_random_array( } } -/// Create a random RecordBatch from the given schema. -pub fn create_random_batch( - schema: SchemaRef, - size: usize, - null_density: f32, - true_density: f32, -) -> arrow::error::Result { - let columns = schema - .fields() - .iter() - .map(|field| create_random_array(field, size, null_density, true_density)) - .collect::>>()?; - RecordBatch::try_new_with_options( - schema, - columns, - &RecordBatchOptions::new().with_match_field_names(false), - ) +/// Create the "pt" column: one random index is set to "unique_point", the remaining rows are filled with random strings. +fn create_filter_array_pt(size: usize) -> ArrayRef { + let mut builder = StringViewBuilder::with_capacity(size); + let mut rng = StdRng::seed_from_u64(100); + let unique_index = rng.random_range(0..size); + for i in 0..size { + if i == unique_index { + builder.append_value("unique_point"); + } else { + builder.append_value(random_string(&mut rng)); + } + } + Arc::new(builder.finish()) as ArrayRef } -/// Create a RecordBatch with 100K rows and four columns. -fn make_record_batch() -> RecordBatch { - let num_rows = 100_000; +/// Create the "sel" column: exactly 1% of rows (those with index % 100 == 0) are set to "selected", +/// while the other 99% of rows are filled with random strings. +fn create_filter_array_sel(size: usize) -> ArrayRef { + let mut builder = StringViewBuilder::with_capacity(size); + let mut rng = StdRng::seed_from_u64(101); + for i in 0..size { + if i % 100 == 0 { + builder.append_value("selected"); + } else { + builder.append_value(random_string(&mut rng)); + } + } + Arc::new(builder.finish()) as ArrayRef +} + +/// Create the "mod_clustered" column: in each 10,000-row block, the first 10 rows are set to "mod_clustered" +/// (simulating a clustered filter with 10 rows per block), and the rest are filled with random strings. +fn create_filter_array_mod_clustered(size: usize) -> ArrayRef { + let mut builder = StringViewBuilder::with_capacity(size); + let block_size = 10_000; + let mut rng = StdRng::seed_from_u64(102); + for i in 0..size { + if (i % block_size) < 10 { + builder.append_value("mod_clustered"); + } else { + builder.append_value(random_string(&mut rng)); + } + } + Arc::new(builder.finish()) as ArrayRef +} + +/// Create the "mod_unclustered" column: exactly 10% of rows (those with index % 10 == 1) +/// are set to "mod_unclustered", while the remaining rows receive random strings. +fn create_filter_array_mod_unclustered(size: usize) -> ArrayRef { + let mut builder = StringViewBuilder::with_capacity(size); + let mut rng = StdRng::seed_from_u64(103); + for i in 0..size { + if i % 10 == 1 { + builder.append_value("mod_unclustered"); + } else { + builder.append_value(random_string(&mut rng)); + } + } + Arc::new(builder.finish()) as ArrayRef +} + +/// Create the "unsel_unclustered" column: exactly 99% of rows (those with index % 100 != 0) +/// are set to "unsel_unclustered", and the remaining 1% get random strings. +fn create_filter_array_unsel_unclustered(size: usize) -> ArrayRef { + let mut builder = StringViewBuilder::with_capacity(size); + let mut rng = StdRng::seed_from_u64(104); + for i in 0..size { + if i % 100 != 0 { + builder.append_value("unsel_unclustered"); + } else { + builder.append_value(random_string(&mut rng)); + } + } + Arc::new(builder.finish()) as ArrayRef +} + +/// Create the "unsel_clustered" column: in each 10,000-row block, rows with an offset >= 1000 +/// are set to "unsel_clustered" (representing a clustered filter selecting 90% of the rows), +/// while rows with offset < 1000 are filled with random strings. +fn create_filter_array_unsel_clustered(size: usize) -> ArrayRef { + let mut builder = StringViewBuilder::with_capacity(size); + let block_size = 10_000; + let mut rng = StdRng::seed_from_u64(105); + for i in 0..size { + if (i % block_size) >= 1000 { + builder.append_value("unsel_clustered"); + } else { + builder.append_value(random_string(&mut rng)); + } + } + Arc::new(builder.finish()) as ArrayRef +} + +/// Create an extended RecordBatch with 100K rows and ten columns. +/// The schema includes the original four columns and the six additional filter columns, +/// whose names have been updated to use "clustered" and "unclustered" as appropriate. +fn create_extended_batch(size: usize) -> RecordBatch { let fields = vec![ Field::new("int64", DataType::Int64, false), Field::new("float64", DataType::Float64, false), @@ -162,10 +237,69 @@ fn make_record_batch() -> RecordBatch { DataType::Timestamp(TimeUnit::Millisecond, None), false, ), + Field::new("pt", DataType::Utf8View, true), + Field::new("sel", DataType::Utf8View, true), + Field::new("mod_clustered", DataType::Utf8View, true), + Field::new("mod_unclustered", DataType::Utf8View, true), + Field::new("unsel_unclustered", DataType::Utf8View, true), + Field::new("unsel_clustered", DataType::Utf8View, true), ]; let schema = Arc::new(Schema::new(fields)); - let batch = create_random_batch(schema, num_rows, 0.2, 0.5).unwrap(); + let int64_array = + create_random_array(&Field::new("int64", DataType::Int64, false), size, 0.0, 0.0).unwrap(); + let float64_array = create_random_array( + &Field::new("float64", DataType::Float64, false), + size, + 0.0, + 0.0, + ) + .unwrap(); + let utf8_array = create_random_array( + &Field::new("utf8View", DataType::Utf8View, true), + size, + 0.2, + 0.5, + ) + .unwrap(); + let ts_array = create_random_array( + &Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + size, + 0.0, + 0.0, + ) + .unwrap(); + + let pt_array = create_filter_array_pt(size); + let sel_array = create_filter_array_sel(size); + let mod_clustered_array = create_filter_array_mod_clustered(size); + let mod_unclustered_array = create_filter_array_mod_unclustered(size); + let unsel_unclustered_array = create_filter_array_unsel_unclustered(size); + let unsel_clustered_array = create_filter_array_unsel_clustered(size); + + let arrays: Vec = vec![ + int64_array, + float64_array, + utf8_array, + ts_array, + pt_array, + sel_array, + mod_clustered_array, + mod_unclustered_array, + unsel_unclustered_array, + unsel_clustered_array, + ]; + RecordBatch::try_new(schema, arrays).unwrap() +} + +/// Create a RecordBatch with 100K rows and print a summary (first 100 rows) to the console. +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + let batch = create_extended_batch(num_rows); println!("Batch created with {} rows", num_rows); println!( "First 100 rows:\n{}", @@ -174,12 +308,11 @@ fn make_record_batch() -> RecordBatch { batch } -/// Write the RecordBatch to a temporary Parquet file. +/// Write the RecordBatch to a temporary Parquet file and return the file handle. fn write_parquet_file() -> NamedTempFile { let batch = make_record_batch(); let schema = batch.schema(); let props = WriterProperties::builder().build(); - let file = tempfile::Builder::new() .suffix(".parquet") .tempfile() @@ -193,99 +326,156 @@ fn write_parquet_file() -> NamedTempFile { file } +/// ProjectionCase defines the projection mode for the benchmark: +/// either projecting all columns or excluding the column that is used for filtering. +#[derive(Clone)] +enum ProjectionCase { + AllColumns, + ExcludeFilterColumn, +} + +impl std::fmt::Display for ProjectionCase { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + ProjectionCase::AllColumns => write!(f, "all_columns"), + ProjectionCase::ExcludeFilterColumn => write!(f, "exclude_filter_column"), + } + } +} + /// FilterType encapsulates the different filter comparisons. +/// The variants correspond to the different filter patterns. #[derive(Clone)] enum FilterType { Utf8ViewNonEmpty, Utf8ViewConst, Int64EqZero, TimestampGt, + PointLookup, + SelectiveUnclustered, + ModeratelySelectiveClustered, + ModeratelySelectiveUnclustered, + UnselectiveUnclustered, + UnselectiveClustered, } impl std::fmt::Display for FilterType { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - FilterType::Utf8ViewNonEmpty => write!(f, "utf8View <> ''"), - FilterType::Utf8ViewConst => write!(f, "utf8View = 'const'"), - FilterType::Int64EqZero => write!(f, "int64 = 0"), - FilterType::TimestampGt => write!(f, "ts > 50_000"), - } + use FilterType::*; + let s = match self { + Utf8ViewNonEmpty => "utf8View <> ''", + Utf8ViewConst => "utf8View = 'const'", + Int64EqZero => "int64 = 0", + TimestampGt => "ts > 50_000", + PointLookup => "Point Lookup", + SelectiveUnclustered => "1% Unclustered Filter", + ModeratelySelectiveClustered => "10% Clustered Filter", + ModeratelySelectiveUnclustered => "10% Unclustered Filter", + UnselectiveUnclustered => "99% Unclustered Filter", + UnselectiveClustered => "90% Clustered Filter", + }; + write!(f, "{}", s) } } impl FilterType { - /// Filters the given batch according to self using Arrow compute kernels. - /// Returns a BooleanArray where true indicates that the row satisfies the condition. + /// Applies the specified filter on the given record batch, returning a BooleanArray mask. + /// Each filter uses its dedicated column and checks equality against a fixed string. fn filter_batch(&self, batch: &RecordBatch) -> arrow::error::Result { + use FilterType::*; match self { - FilterType::Utf8ViewNonEmpty => { + Utf8ViewNonEmpty => { let array = batch.column(batch.schema().index_of("utf8View").unwrap()); - let string_view_scalar = StringViewArray::new_scalar(""); - let not_equals_empty = neq(array, &string_view_scalar)?; - Ok(not_equals_empty) + let scalar = StringViewArray::new_scalar(""); + neq(array, &scalar) } - FilterType::Utf8ViewConst => { + Utf8ViewConst => { let array = batch.column(batch.schema().index_of("utf8View").unwrap()); - let string_view_scalar = StringViewArray::new_scalar("const"); - let eq_const = eq(array, &string_view_scalar)?; - Ok(eq_const) + let scalar = StringViewArray::new_scalar("const"); + eq(array, &scalar) } - FilterType::Int64EqZero => { + Int64EqZero => { let array = batch.column(batch.schema().index_of("int64").unwrap()); - let eq_zero = eq(array, &Int64Array::new_scalar(0))?; - Ok(eq_zero) + eq(array, &Int64Array::new_scalar(0)) } - FilterType::TimestampGt => { + TimestampGt => { let array = batch.column(batch.schema().index_of("ts").unwrap()); - let gt_thresh = gt(array, &TimestampMillisecondArray::new_scalar(50_000))?; - Ok(gt_thresh) + gt(array, &TimestampMillisecondArray::new_scalar(50_000)) + } + PointLookup => { + let array = batch.column(batch.schema().index_of("pt").unwrap()); + let scalar = StringViewArray::new_scalar("unique_point"); + eq(array, &scalar) + } + SelectiveUnclustered => { + let array = batch.column(batch.schema().index_of("sel").unwrap()); + let scalar = StringViewArray::new_scalar("selected"); + eq(array, &scalar) + } + ModeratelySelectiveClustered => { + let array = batch.column(batch.schema().index_of("mod_clustered").unwrap()); + let scalar = StringViewArray::new_scalar("mod_clustered"); + eq(array, &scalar) + } + ModeratelySelectiveUnclustered => { + let array = batch.column(batch.schema().index_of("mod_unclustered").unwrap()); + let scalar = StringViewArray::new_scalar("mod_unclustered"); + eq(array, &scalar) + } + UnselectiveUnclustered => { + let array = batch.column(batch.schema().index_of("unsel_unclustered").unwrap()); + let scalar = StringViewArray::new_scalar("unsel_unclustered"); + eq(array, &scalar) + } + UnselectiveClustered => { + let array = batch.column(batch.schema().index_of("unsel_clustered").unwrap()); + let scalar = StringViewArray::new_scalar("unsel_clustered"); + eq(array, &scalar) } } } } -/// ProjectionCase defines the projection mode. -#[derive(Clone)] -enum ProjectionCase { - AllColumns, - ExcludeFilterColumn, -} - -impl std::fmt::Display for ProjectionCase { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - ProjectionCase::AllColumns => write!(f, "all_columns"), - ProjectionCase::ExcludeFilterColumn => write!(f, "exclude_filter_column"), - } - } -} - +/// Benchmark filters and projections by reading the Parquet file. +/// This benchmark iterates over all individual filter types and two projection cases. +/// It measures the time to read and filter the Parquet file according to each scenario. fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); - - let filter_types: Vec = vec![ + let filter_types = vec![ FilterType::Utf8ViewNonEmpty, FilterType::Utf8ViewConst, FilterType::Int64EqZero, FilterType::TimestampGt, + FilterType::PointLookup, + FilterType::SelectiveUnclustered, + FilterType::ModeratelySelectiveClustered, + FilterType::ModeratelySelectiveUnclustered, + FilterType::UnselectiveUnclustered, + FilterType::UnselectiveClustered, ]; - let projection_cases = vec![ ProjectionCase::AllColumns, ProjectionCase::ExcludeFilterColumn, ]; - let mut group = c.benchmark_group("arrow_reader_row_filter"); - for filter_type in filter_types.iter().cloned() { + for filter_type in filter_types.clone() { for proj_case in &projection_cases { - // All column indices: [0: int64, 1: float64, 2: utf8View, 3: ts] - let all_indices = vec![0, 1, 2, 3]; + // All indices corresponding to the 10 columns. + let all_indices = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]; + // Determine the filter column index based on the filter type. let filter_col = match filter_type { FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => 2, FilterType::Int64EqZero => 0, FilterType::TimestampGt => 3, + FilterType::PointLookup => 4, + FilterType::SelectiveUnclustered => 5, + FilterType::ModeratelySelectiveClustered => 6, + FilterType::ModeratelySelectiveUnclustered => 7, + FilterType::UnselectiveUnclustered => 8, + FilterType::UnselectiveClustered => 9, }; + // For the projection, either select all columns or exclude the filter column. let output_projection: Vec = match proj_case { ProjectionCase::AllColumns => all_indices.clone(), ProjectionCase::ExcludeFilterColumn => all_indices @@ -293,25 +483,16 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .filter(|i| *i != filter_col) .collect(), }; - // For predicate pushdown, include the filter column. - let predicate_projection: Vec = match filter_type { - FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => vec![2], - FilterType::Int64EqZero => vec![0], - FilterType::TimestampGt => vec![3], - }; let bench_id = BenchmarkId::new(format!("filter: {} proj: {}", filter_type, proj_case), ""); - group.bench_function(bench_id, |b| { let rt = tokio::runtime::Builder::new_multi_thread() .enable_all() .build() .unwrap(); b.iter(|| { - // Clone filter_type inside the closure to avoid moving it let filter_type_inner = filter_type.clone(); - rt.block_on(async { let file = File::open(parquet_file.path()).await.unwrap(); let options = ArrowReaderOptions::new().with_page_index(true); @@ -320,29 +501,21 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .await .unwrap() .with_batch_size(8192); - let file_metadata = builder.metadata().file_metadata().clone(); let mask = ProjectionMask::roots( file_metadata.schema_descr(), output_projection.clone(), ); - let pred_mask = ProjectionMask::roots( - file_metadata.schema_descr(), - predicate_projection.clone(), - ); - + let pred_mask = + ProjectionMask::roots(file_metadata.schema_descr(), vec![filter_col]); let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { - // Clone filter_type within the closure - let filter_type_inner = filter_type_inner.clone(); Ok(filter_type_inner.filter_batch(&batch).unwrap()) }); - let stream = builder .with_projection(mask) .with_row_filter(RowFilter::new(vec![Box::new(filter)])) .build() .unwrap(); - stream.try_collect::>().await.unwrap(); }) }); @@ -351,5 +524,75 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { } } -criterion_group!(benches, benchmark_filters_and_projections); +/// Benchmark composite filters by applying two filters simultaneously. +/// This benchmark creates a composite row filter that ANDs two predicates: +/// one on the "sel" column (exactly 1% selected) and one on the "mod_clustered" column +/// (first 10 rows in each 10K block), then measures the performance of the combined filtering. +fn benchmark_composite_filters(c: &mut Criterion) { + let parquet_file = write_parquet_file(); + let mut group = c.benchmark_group("composite_filter"); + + // For composite filtering, we choose: + // - Filter1: SelectiveUnclustered on column "sel" (index 5) + // - Filter2: ModeratelySelectiveClustered on column "mod_clustered" (index 6) + // These filters are applied sequentially (logical AND). + let filter1_col = 5; + let filter2_col = 6; + let bench_id = BenchmarkId::new("Composite Filter: sel AND mod_clustered", ""); + group.bench_function(bench_id, |b| { + let rt = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build() + .unwrap(); + b.iter(|| { + rt.block_on(async { + let file = File::open(parquet_file.path()).await.unwrap(); + let options = ArrowReaderOptions::new().with_page_index(true); + let builder = ParquetRecordBatchStreamBuilder::new_with_options(file, options) + .await + .unwrap() + .with_batch_size(8192); + let file_metadata = builder.metadata().file_metadata().clone(); + // For projection, we select all columns. + let all_indices = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]; + let mask = ProjectionMask::roots(file_metadata.schema_descr(), all_indices.clone()); + let pred_mask1 = + ProjectionMask::roots(file_metadata.schema_descr(), vec![filter1_col]); + let pred_mask2 = + ProjectionMask::roots(file_metadata.schema_descr(), vec![filter2_col]); + + // Create first filter: applies the "sel" filter. + let filter1 = ArrowPredicateFn::new(pred_mask1, move |batch: RecordBatch| { + let scalar = StringViewArray::new_scalar("selected"); + eq( + batch.column(batch.schema().index_of("sel").unwrap()), + &scalar, + ) + }); + // Create second filter: applies the "mod_clustered" filter. + let filter2 = ArrowPredicateFn::new(pred_mask2, move |batch: RecordBatch| { + let scalar = StringViewArray::new_scalar("mod_clustered"); + eq( + batch.column(batch.schema().index_of("mod_clustered").unwrap()), + &scalar, + ) + }); + let composite_filter = RowFilter::new(vec![Box::new(filter1), Box::new(filter2)]); + + let stream = builder + .with_projection(mask) + .with_row_filter(composite_filter) + .build() + .unwrap(); + stream.try_collect::>().await.unwrap(); + }) + }); + }); +} + +criterion_group!( + benches, + benchmark_filters_and_projections, + benchmark_composite_filters +); criterion_main!(benches); From 16bc1bfaff7092b05667d820a29d9e8b56cd317b Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 12 Apr 2025 14:49:28 +0800 Subject: [PATCH 39/73] Add float(half set) and int(full set) change --- parquet/benches/arrow_reader_row_filter.rs | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 12fd02e23c93..2f06e07fb897 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -349,7 +349,8 @@ impl std::fmt::Display for ProjectionCase { enum FilterType { Utf8ViewNonEmpty, Utf8ViewConst, - Int64EqZero, + Int64GTZero, + Float64GTHalf, TimestampGt, PointLookup, SelectiveUnclustered, @@ -365,7 +366,8 @@ impl std::fmt::Display for FilterType { let s = match self { Utf8ViewNonEmpty => "utf8View <> ''", Utf8ViewConst => "utf8View = 'const'", - Int64EqZero => "int64 = 0", + Int64GTZero => "int64 > 0", + Float64GTHalf => "float64 > 50.0", TimestampGt => "ts > 50_000", PointLookup => "Point Lookup", SelectiveUnclustered => "1% Unclustered Filter", @@ -394,9 +396,13 @@ impl FilterType { let scalar = StringViewArray::new_scalar("const"); eq(array, &scalar) } - Int64EqZero => { + Int64GTZero => { let array = batch.column(batch.schema().index_of("int64").unwrap()); - eq(array, &Int64Array::new_scalar(0)) + gt(array, &Int64Array::new_scalar(0)) + } + Float64GTHalf => { + let array = batch.column(batch.schema().index_of("float64").unwrap()); + gt(array, &Float64Array::new_scalar(50.0)) } TimestampGt => { let array = batch.column(batch.schema().index_of("ts").unwrap()); @@ -444,7 +450,8 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { let filter_types = vec![ FilterType::Utf8ViewNonEmpty, FilterType::Utf8ViewConst, - FilterType::Int64EqZero, + FilterType::Int64GTZero, + FilterType::Float64GTHalf, FilterType::TimestampGt, FilterType::PointLookup, FilterType::SelectiveUnclustered, @@ -466,7 +473,8 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { // Determine the filter column index based on the filter type. let filter_col = match filter_type { FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => 2, - FilterType::Int64EqZero => 0, + FilterType::Int64GTZero => 0, + FilterType::Float64GTHalf => 1, FilterType::TimestampGt => 3, FilterType::PointLookup => 4, FilterType::SelectiveUnclustered => 5, From d0ab2fe851babe158452104e823f8b57f8b3df01 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 12 Apr 2025 19:30:06 +0800 Subject: [PATCH 40/73] Fix corner case: skipping page should also make dic page to none --- parquet/src/file/serialized_reader.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 178adc1c520b..1d27d5216d20 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -981,8 +981,16 @@ impl PageReader for SerializedPageReader { } Ok(()) } - SerializedPageReaderState::Pages { page_locations, .. } => { - page_locations.pop_front(); + SerializedPageReaderState::Pages { + page_locations, + dictionary_page, .. } => { + if dictionary_page.is_some() { + // If a dictionary page exists, consume it by taking it (sets to None) + dictionary_page.take(); + } else { + // If no dictionary page exists, simply pop the data page from page_locations + page_locations.pop_front(); + } Ok(()) } From 7638c41032a1a7310f912571b2eb171a546db5ce Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 13 Apr 2025 23:28:32 +0800 Subject: [PATCH 41/73] Address comments --- parquet/benches/arrow_reader_row_filter.rs | 501 +++++++-------------- 1 file changed, 158 insertions(+), 343 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 2f06e07fb897..e2b67b7c8904 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -51,11 +51,10 @@ //! - unsel_unclustered: for Unselective Unclustered – exactly 99% of rows (those with i % 100 != 0) are "unsel_unclustered". //! - unsel_clustered: for Unselective Clustered – in each 10K-row block, rows with an offset >= 1000 are "unsel_clustered". //! -//! As a side note, an additional composite benchmark is provided which demonstrates -//! the performance when applying two filters simultaneously (i.e. chaining row selectors). use arrow::array::{ArrayRef, BooleanArray, Float64Array, Int64Array, TimestampMillisecondArray}; -use arrow::compute::kernels::cmp::{eq, gt, neq}; +use arrow::compute::and; +use arrow::compute::kernels::cmp::{eq, gt, lt, neq}; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arrow::record_batch::RecordBatch; use arrow_array::builder::StringViewBuilder; @@ -71,8 +70,8 @@ use std::sync::Arc; use tempfile::NamedTempFile; use tokio::fs::File; -/// Generates a random string (either short: 3–11 bytes or long: 13–20 bytes) with 50% probability. -/// This is used to fill non-selected rows in the filter columns. +/// Generates a random string. Has a 50% chance to generate a short string (3–11 characters) +/// or a long string (13–20 characters). fn random_string(rng: &mut StdRng) -> String { let charset = b"abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; let is_long = rng.random_bool(0.5); @@ -86,120 +85,34 @@ fn random_string(rng: &mut StdRng) -> String { .collect() } -/// Create a random array for a given field, generating data with fixed seed reproducibility. -/// - For Int64, random integers in [0, 100). -/// - For Float64, random floats in [0.0, 100.0). -/// - For Utf8View, a mix of empty strings, the constant "const", and random strings. -/// - For Timestamp, sequential timestamps in milliseconds. -fn create_random_array( - field: &Field, - size: usize, - null_density: f32, - _true_density: f32, -) -> arrow::error::Result { - match field.data_type() { - DataType::Int64 => { - let mut rng = StdRng::seed_from_u64(42); - let values: Vec = (0..size).map(|_| rng.random_range(0..100)).collect(); - Ok(Arc::new(Int64Array::from(values)) as ArrayRef) - } - DataType::Float64 => { - let mut rng = StdRng::seed_from_u64(43); - let values: Vec = (0..size).map(|_| rng.random_range(0.0..100.0)).collect(); - Ok(Arc::new(Float64Array::from(values)) as ArrayRef) - } - DataType::Utf8View => { - let mut builder = StringViewBuilder::with_capacity(size); - let mut rng = StdRng::seed_from_u64(44); - for _ in 0..size { - let choice = rng.random_range(0..100); - if choice < (null_density * 100.0) as u32 { - builder.append_value(""); - } else if choice < 25 { - builder.append_value("const"); - } else { - builder.append_value(random_string(&mut rng)); - } - } - Ok(Arc::new(builder.finish()) as ArrayRef) - } - DataType::Timestamp(TimeUnit::Millisecond, _) => { - let values: Vec = (0..size as i64).collect(); - Ok(Arc::new(TimestampMillisecondArray::from(values)) as ArrayRef) - } - _ => unimplemented!("Field type not supported in create_random_array"), - } -} - -/// Create the "pt" column: one random index is set to "unique_point", the remaining rows are filled with random strings. -fn create_filter_array_pt(size: usize) -> ArrayRef { - let mut builder = StringViewBuilder::with_capacity(size); - let mut rng = StdRng::seed_from_u64(100); +/// Creates an int64 array of a given size with random integers in [0, 100). +/// Then, it overwrites a single random index with 9999 to serve as the unique value for point lookup. +fn create_int64_array(size: usize) -> ArrayRef { + let mut rng = StdRng::seed_from_u64(42); + let mut values: Vec = (0..size).map(|_| rng.random_range(0..100)).collect(); let unique_index = rng.random_range(0..size); - for i in 0..size { - if i == unique_index { - builder.append_value("unique_point"); - } else { - builder.append_value(random_string(&mut rng)); - } - } - Arc::new(builder.finish()) as ArrayRef -} - -/// Create the "sel" column: exactly 1% of rows (those with index % 100 == 0) are set to "selected", -/// while the other 99% of rows are filled with random strings. -fn create_filter_array_sel(size: usize) -> ArrayRef { - let mut builder = StringViewBuilder::with_capacity(size); - let mut rng = StdRng::seed_from_u64(101); - for i in 0..size { - if i % 100 == 0 { - builder.append_value("selected"); - } else { - builder.append_value(random_string(&mut rng)); - } - } - Arc::new(builder.finish()) as ArrayRef + values[unique_index] = 9999; // Unique value for point lookup + Arc::new(Int64Array::from(values)) as ArrayRef } -/// Create the "mod_clustered" column: in each 10,000-row block, the first 10 rows are set to "mod_clustered" -/// (simulating a clustered filter with 10 rows per block), and the rest are filled with random strings. -fn create_filter_array_mod_clustered(size: usize) -> ArrayRef { - let mut builder = StringViewBuilder::with_capacity(size); - let block_size = 10_000; - let mut rng = StdRng::seed_from_u64(102); - for i in 0..size { - if (i % block_size) < 10 { - builder.append_value("mod_clustered"); - } else { - builder.append_value(random_string(&mut rng)); - } - } - Arc::new(builder.finish()) as ArrayRef -} - -/// Create the "mod_unclustered" column: exactly 10% of rows (those with index % 10 == 1) -/// are set to "mod_unclustered", while the remaining rows receive random strings. -fn create_filter_array_mod_unclustered(size: usize) -> ArrayRef { - let mut builder = StringViewBuilder::with_capacity(size); - let mut rng = StdRng::seed_from_u64(103); - for i in 0..size { - if i % 10 == 1 { - builder.append_value("mod_unclustered"); - } else { - builder.append_value(random_string(&mut rng)); - } - } - Arc::new(builder.finish()) as ArrayRef +/// Creates a float64 array of a given size with random floats in [0.0, 100.0). +fn create_float64_array(size: usize) -> ArrayRef { + let mut rng = StdRng::seed_from_u64(43); + let values: Vec = (0..size).map(|_| rng.random_range(0.0..100.0)).collect(); + Arc::new(Float64Array::from(values)) as ArrayRef } -/// Create the "unsel_unclustered" column: exactly 99% of rows (those with index % 100 != 0) -/// are set to "unsel_unclustered", and the remaining 1% get random strings. -fn create_filter_array_unsel_unclustered(size: usize) -> ArrayRef { +/// Creates a utf8View array of a given size with random strings. +/// Now, this column is used in one filter case. +fn create_utf8_view_array(size: usize, null_density: f32) -> ArrayRef { let mut builder = StringViewBuilder::with_capacity(size); - let mut rng = StdRng::seed_from_u64(104); - for i in 0..size { - if i % 100 != 0 { - builder.append_value("unsel_unclustered"); + let mut rng = StdRng::seed_from_u64(44); + for _ in 0..size { + let choice = rng.random_range(0..100); + if choice < (null_density * 100.0) as u32 { + builder.append_value(""); + } else if choice < 25 { + builder.append_value("const"); } else { builder.append_value(random_string(&mut rng)); } @@ -207,27 +120,15 @@ fn create_filter_array_unsel_unclustered(size: usize) -> ArrayRef { Arc::new(builder.finish()) as ArrayRef } -/// Create the "unsel_clustered" column: in each 10,000-row block, rows with an offset >= 1000 -/// are set to "unsel_clustered" (representing a clustered filter selecting 90% of the rows), -/// while rows with offset < 1000 are filled with random strings. -fn create_filter_array_unsel_clustered(size: usize) -> ArrayRef { - let mut builder = StringViewBuilder::with_capacity(size); - let block_size = 10_000; - let mut rng = StdRng::seed_from_u64(105); - for i in 0..size { - if (i % block_size) >= 1000 { - builder.append_value("unsel_clustered"); - } else { - builder.append_value(random_string(&mut rng)); - } - } - Arc::new(builder.finish()) as ArrayRef +/// Creates a ts (timestamp) array of a given size. Each value is computed as i % 10_000, +/// which simulates repeating blocks (each block of 10,000) to model clustered patterns. +fn create_ts_array(size: usize) -> ArrayRef { + let values: Vec = (0..size).map(|i| (i % 10_000) as i64).collect(); + Arc::new(TimestampMillisecondArray::from(values)) as ArrayRef } -/// Create an extended RecordBatch with 100K rows and ten columns. -/// The schema includes the original four columns and the six additional filter columns, -/// whose names have been updated to use "clustered" and "unclustered" as appropriate. -fn create_extended_batch(size: usize) -> RecordBatch { +/// Creates a RecordBatch with 100K rows and 4 columns: int64, float64, utf8View, and ts. +fn create_record_batch(size: usize) -> RecordBatch { let fields = vec![ Field::new("int64", DataType::Int64, false), Field::new("float64", DataType::Float64, false), @@ -237,80 +138,26 @@ fn create_extended_batch(size: usize) -> RecordBatch { DataType::Timestamp(TimeUnit::Millisecond, None), false, ), - Field::new("pt", DataType::Utf8View, true), - Field::new("sel", DataType::Utf8View, true), - Field::new("mod_clustered", DataType::Utf8View, true), - Field::new("mod_unclustered", DataType::Utf8View, true), - Field::new("unsel_unclustered", DataType::Utf8View, true), - Field::new("unsel_clustered", DataType::Utf8View, true), ]; let schema = Arc::new(Schema::new(fields)); - let int64_array = - create_random_array(&Field::new("int64", DataType::Int64, false), size, 0.0, 0.0).unwrap(); - let float64_array = create_random_array( - &Field::new("float64", DataType::Float64, false), - size, - 0.0, - 0.0, - ) - .unwrap(); - let utf8_array = create_random_array( - &Field::new("utf8View", DataType::Utf8View, true), - size, - 0.2, - 0.5, - ) - .unwrap(); - let ts_array = create_random_array( - &Field::new( - "ts", - DataType::Timestamp(TimeUnit::Millisecond, None), - false, - ), - size, - 0.0, - 0.0, - ) - .unwrap(); - - let pt_array = create_filter_array_pt(size); - let sel_array = create_filter_array_sel(size); - let mod_clustered_array = create_filter_array_mod_clustered(size); - let mod_unclustered_array = create_filter_array_mod_unclustered(size); - let unsel_unclustered_array = create_filter_array_unsel_unclustered(size); - let unsel_clustered_array = create_filter_array_unsel_clustered(size); + let int64_array = create_int64_array(size); + let float64_array = create_float64_array(size); + let utf8_array = create_utf8_view_array(size, 0.2); + let ts_array = create_ts_array(size); - let arrays: Vec = vec![ - int64_array, - float64_array, - utf8_array, - ts_array, - pt_array, - sel_array, - mod_clustered_array, - mod_unclustered_array, - unsel_unclustered_array, - unsel_clustered_array, - ]; + let arrays: Vec = vec![int64_array, float64_array, utf8_array, ts_array]; RecordBatch::try_new(schema, arrays).unwrap() } -/// Create a RecordBatch with 100K rows and print a summary (first 100 rows) to the console. -fn make_record_batch() -> RecordBatch { - let num_rows = 100_000; - let batch = create_extended_batch(num_rows); - println!("Batch created with {} rows", num_rows); +/// Writes the RecordBatch to a temporary Parquet file and returns the file handle. +fn write_parquet_file() -> NamedTempFile { + let batch = create_record_batch(100_000); + println!("Batch created with {} rows", 100_000); println!( "First 100 rows:\n{}", pretty_format_batches(&[batch.clone().slice(0, 100)]).unwrap() ); - batch -} - -/// Write the RecordBatch to a temporary Parquet file and return the file handle. -fn write_parquet_file() -> NamedTempFile { - let batch = make_record_batch(); let schema = batch.schema(); let props = WriterProperties::builder().build(); let file = tempfile::Builder::new() @@ -347,96 +194,139 @@ impl std::fmt::Display for ProjectionCase { /// The variants correspond to the different filter patterns. #[derive(Clone)] enum FilterType { - Utf8ViewNonEmpty, - Utf8ViewConst, - Int64GTZero, - Float64GTHalf, - TimestampGt, + /// Here is the 6 filter types: + /// ┌───────────────┐ ┌───────────────┐ + /// │ │ │ │ + /// │ │ │ ... │ + /// │ │ │ │ + /// │ │ │ │ + /// │ ... │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │ │ + /// │ │ │ ... │ + /// │ │ │ │ + /// │ │ │ │ + /// └───────────────┘ └───────────────┘ + /// + /// "Point Lookup": selects a single row + /// (1 RowSelection of 1 row) + /// + /// ┌───────────────┐ ┌───────────────┐ + /// │ ... │ │ │ + /// │ │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ + /// │ │ │ ... │ + /// │ │ │ │ + /// │ │ │ │ + /// │ ... │ │ │ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │ │ + /// └───────────────┘ └───────────────┘ + /// selective (1%) unclustered filter + /// (1000 RowSelection of 10 rows each) + /// + /// + /// ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ + /// │ ... │ │ │ │ │ │ │ + /// │ │ │ │ │ │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ ... │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ │ │ + /// │ │ │ │ │ ... │ │ │ + /// │ │ │ ... │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ ... │ │ │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │ │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// └───────────────┘ └───────────────┘ └───────────────┘ └───────────────┘ + /// moderately selective (10%) unclustered filter moderately selective (10%) clustered filter + /// (10000 RowSelection of 10 rows each) (10 RowSelections of 10,000 rows each) + /// ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ ... │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ ... │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ └───────────────┘ └───────────────┘ + /// └───────────────┘ └───────────────┘ + /// unselective (99%) unclustered filter unselective (90%) clustered filter + /// (99,000 RowSelections of 10 rows each) (99 RowSelection of 10,000 rows each) PointLookup, SelectiveUnclustered, ModeratelySelectiveClustered, ModeratelySelectiveUnclustered, UnselectiveUnclustered, UnselectiveClustered, + /// The following are Composite and Utf8ViewNonEmpty filters, which is the additional to above 6 filters. + Composite, + Utf8ViewNonEmpty, } impl std::fmt::Display for FilterType { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - use FilterType::*; let s = match self { - Utf8ViewNonEmpty => "utf8View <> ''", - Utf8ViewConst => "utf8View = 'const'", - Int64GTZero => "int64 > 0", - Float64GTHalf => "float64 > 50.0", - TimestampGt => "ts > 50_000", - PointLookup => "Point Lookup", - SelectiveUnclustered => "1% Unclustered Filter", - ModeratelySelectiveClustered => "10% Clustered Filter", - ModeratelySelectiveUnclustered => "10% Unclustered Filter", - UnselectiveUnclustered => "99% Unclustered Filter", - UnselectiveClustered => "90% Clustered Filter", + FilterType::PointLookup => "int64 == 9999", + FilterType::SelectiveUnclustered => "float64 > 99.0", + FilterType::ModeratelySelectiveClustered => "ts >= 9000", + FilterType::ModeratelySelectiveUnclustered => "int64 > 90", + FilterType::UnselectiveUnclustered => "float64 <= 99.0", + FilterType::UnselectiveClustered => "ts < 9000", + FilterType::Composite => "float64 > 99.0 AND ts >= 9000", + FilterType::Utf8ViewNonEmpty => "utf8View <> ''", }; write!(f, "{}", s) } } impl FilterType { - /// Applies the specified filter on the given record batch, returning a BooleanArray mask. - /// Each filter uses its dedicated column and checks equality against a fixed string. + /// Applies the specified filter on the given RecordBatch and returns a BooleanArray mask. fn filter_batch(&self, batch: &RecordBatch) -> arrow::error::Result { - use FilterType::*; match self { - Utf8ViewNonEmpty => { - let array = batch.column(batch.schema().index_of("utf8View").unwrap()); - let scalar = StringViewArray::new_scalar(""); - neq(array, &scalar) - } - Utf8ViewConst => { - let array = batch.column(batch.schema().index_of("utf8View").unwrap()); - let scalar = StringViewArray::new_scalar("const"); - eq(array, &scalar) - } - Int64GTZero => { + // Point Lookup on int64 column + FilterType::PointLookup => { let array = batch.column(batch.schema().index_of("int64").unwrap()); - gt(array, &Int64Array::new_scalar(0)) + let scalar = Int64Array::new_scalar(9999); + eq(array, &scalar) } - Float64GTHalf => { + // Selective Unclustered on float64 column: float64 > 99.0 + FilterType::SelectiveUnclustered => { let array = batch.column(batch.schema().index_of("float64").unwrap()); - gt(array, &Float64Array::new_scalar(50.0)) + let scalar = Float64Array::new_scalar(99.0); + gt(array, &scalar) } - TimestampGt => { + // Moderately Selective Clustered on ts column: ts >= 9000 (implemented as > 8999) + FilterType::ModeratelySelectiveClustered => { let array = batch.column(batch.schema().index_of("ts").unwrap()); - gt(array, &TimestampMillisecondArray::new_scalar(50_000)) - } - PointLookup => { - let array = batch.column(batch.schema().index_of("pt").unwrap()); - let scalar = StringViewArray::new_scalar("unique_point"); - eq(array, &scalar) + gt(array, &TimestampMillisecondArray::new_scalar(8999)) } - SelectiveUnclustered => { - let array = batch.column(batch.schema().index_of("sel").unwrap()); - let scalar = StringViewArray::new_scalar("selected"); - eq(array, &scalar) + // Moderately Selective Unclustered on int64 column: int64 > 90 + FilterType::ModeratelySelectiveUnclustered => { + let array = batch.column(batch.schema().index_of("int64").unwrap()); + let scalar = Int64Array::new_scalar(90); + gt(array, &scalar) } - ModeratelySelectiveClustered => { - let array = batch.column(batch.schema().index_of("mod_clustered").unwrap()); - let scalar = StringViewArray::new_scalar("mod_clustered"); - eq(array, &scalar) + // Unselective Unclustered on float64 column: NOT (float64 > 99.0) + FilterType::UnselectiveUnclustered => { + let array = batch.column(batch.schema().index_of("float64").unwrap()); + gt(array, &Float64Array::new_scalar(99.0)) } - ModeratelySelectiveUnclustered => { - let array = batch.column(batch.schema().index_of("mod_unclustered").unwrap()); - let scalar = StringViewArray::new_scalar("mod_unclustered"); - eq(array, &scalar) + // Unselective Clustered on ts column: ts < 9000 + FilterType::UnselectiveClustered => { + let array = batch.column(batch.schema().index_of("ts").unwrap()); + lt(array, &TimestampMillisecondArray::new_scalar(9000)) } - UnselectiveUnclustered => { - let array = batch.column(batch.schema().index_of("unsel_unclustered").unwrap()); - let scalar = StringViewArray::new_scalar("unsel_unclustered"); - eq(array, &scalar) + // Composite filter: logical AND of (float64 > 99.0) and (ts >= 9000) + FilterType::Composite => { + let mask1 = FilterType::SelectiveUnclustered.filter_batch(batch)?; + let mask2 = FilterType::ModeratelySelectiveClustered.filter_batch(batch)?; + and(&mask1, &mask2) } - UnselectiveClustered => { - let array = batch.column(batch.schema().index_of("unsel_clustered").unwrap()); - let scalar = StringViewArray::new_scalar("unsel_clustered"); - eq(array, &scalar) + // Utf8ViewNonEmpty: selects rows where the utf8View column is not an empty string. + FilterType::Utf8ViewNonEmpty => { + let array = batch.column(batch.schema().index_of("utf8View").unwrap()); + let scalar = StringViewArray::new_scalar(""); + neq(array, &scalar) } } } @@ -448,17 +338,14 @@ impl FilterType { fn benchmark_filters_and_projections(c: &mut Criterion) { let parquet_file = write_parquet_file(); let filter_types = vec![ - FilterType::Utf8ViewNonEmpty, - FilterType::Utf8ViewConst, - FilterType::Int64GTZero, - FilterType::Float64GTHalf, - FilterType::TimestampGt, FilterType::PointLookup, FilterType::SelectiveUnclustered, FilterType::ModeratelySelectiveClustered, FilterType::ModeratelySelectiveUnclustered, FilterType::UnselectiveUnclustered, FilterType::UnselectiveClustered, + FilterType::Utf8ViewNonEmpty, + FilterType::Composite, ]; let projection_cases = vec![ ProjectionCase::AllColumns, @@ -469,19 +356,17 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { for filter_type in filter_types.clone() { for proj_case in &projection_cases { // All indices corresponding to the 10 columns. - let all_indices = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]; + let all_indices = vec![0, 1, 2, 3]; // Determine the filter column index based on the filter type. let filter_col = match filter_type { - FilterType::Utf8ViewNonEmpty | FilterType::Utf8ViewConst => 2, - FilterType::Int64GTZero => 0, - FilterType::Float64GTHalf => 1, - FilterType::TimestampGt => 3, - FilterType::PointLookup => 4, - FilterType::SelectiveUnclustered => 5, - FilterType::ModeratelySelectiveClustered => 6, - FilterType::ModeratelySelectiveUnclustered => 7, - FilterType::UnselectiveUnclustered => 8, - FilterType::UnselectiveClustered => 9, + FilterType::PointLookup => 0, + FilterType::SelectiveUnclustered => 1, + FilterType::ModeratelySelectiveClustered => 3, + FilterType::ModeratelySelectiveUnclustered => 0, + FilterType::UnselectiveUnclustered => 1, + FilterType::UnselectiveClustered => 3, + FilterType::Composite => 1, // Use float64 column as representative for composite + FilterType::Utf8ViewNonEmpty => 2, }; // For the projection, either select all columns or exclude the filter column. let output_projection: Vec = match proj_case { @@ -532,75 +417,5 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { } } -/// Benchmark composite filters by applying two filters simultaneously. -/// This benchmark creates a composite row filter that ANDs two predicates: -/// one on the "sel" column (exactly 1% selected) and one on the "mod_clustered" column -/// (first 10 rows in each 10K block), then measures the performance of the combined filtering. -fn benchmark_composite_filters(c: &mut Criterion) { - let parquet_file = write_parquet_file(); - let mut group = c.benchmark_group("composite_filter"); - - // For composite filtering, we choose: - // - Filter1: SelectiveUnclustered on column "sel" (index 5) - // - Filter2: ModeratelySelectiveClustered on column "mod_clustered" (index 6) - // These filters are applied sequentially (logical AND). - let filter1_col = 5; - let filter2_col = 6; - let bench_id = BenchmarkId::new("Composite Filter: sel AND mod_clustered", ""); - group.bench_function(bench_id, |b| { - let rt = tokio::runtime::Builder::new_multi_thread() - .enable_all() - .build() - .unwrap(); - b.iter(|| { - rt.block_on(async { - let file = File::open(parquet_file.path()).await.unwrap(); - let options = ArrowReaderOptions::new().with_page_index(true); - let builder = ParquetRecordBatchStreamBuilder::new_with_options(file, options) - .await - .unwrap() - .with_batch_size(8192); - let file_metadata = builder.metadata().file_metadata().clone(); - // For projection, we select all columns. - let all_indices = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]; - let mask = ProjectionMask::roots(file_metadata.schema_descr(), all_indices.clone()); - let pred_mask1 = - ProjectionMask::roots(file_metadata.schema_descr(), vec![filter1_col]); - let pred_mask2 = - ProjectionMask::roots(file_metadata.schema_descr(), vec![filter2_col]); - - // Create first filter: applies the "sel" filter. - let filter1 = ArrowPredicateFn::new(pred_mask1, move |batch: RecordBatch| { - let scalar = StringViewArray::new_scalar("selected"); - eq( - batch.column(batch.schema().index_of("sel").unwrap()), - &scalar, - ) - }); - // Create second filter: applies the "mod_clustered" filter. - let filter2 = ArrowPredicateFn::new(pred_mask2, move |batch: RecordBatch| { - let scalar = StringViewArray::new_scalar("mod_clustered"); - eq( - batch.column(batch.schema().index_of("mod_clustered").unwrap()), - &scalar, - ) - }); - let composite_filter = RowFilter::new(vec![Box::new(filter1), Box::new(filter2)]); - - let stream = builder - .with_projection(mask) - .with_row_filter(composite_filter) - .build() - .unwrap(); - stream.try_collect::>().await.unwrap(); - }) - }); - }); -} - -criterion_group!( - benches, - benchmark_filters_and_projections, - benchmark_composite_filters -); +criterion_group!(benches, benchmark_filters_and_projections,); criterion_main!(benches); From 9271cc939ff97a88689af4a7114e3c84647d4a0c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 14 Apr 2025 13:24:48 +0800 Subject: [PATCH 42/73] Set compression --- parquet/benches/arrow_reader_row_filter.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index e2b67b7c8904..2d4b48bdc582 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -69,6 +69,7 @@ use rand::{rngs::StdRng, Rng, SeedableRng}; use std::sync::Arc; use tempfile::NamedTempFile; use tokio::fs::File; +use parquet::basic::Compression; /// Generates a random string. Has a 50% chance to generate a short string (3–11 characters) /// or a long string (13–20 characters). @@ -159,7 +160,7 @@ fn write_parquet_file() -> NamedTempFile { pretty_format_batches(&[batch.clone().slice(0, 100)]).unwrap() ); let schema = batch.schema(); - let props = WriterProperties::builder().build(); + let props = WriterProperties::builder().set_compression( Compression::SNAPPY).build(); let file = tempfile::Builder::new() .suffix(".parquet") .tempfile() From 8e00ac58a95c7a904dbe434dcb4ca0b688cf92ae Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 14 Apr 2025 13:52:10 +0800 Subject: [PATCH 43/73] fix --- parquet/benches/arrow_reader_row_filter.rs | 43 ++++++++++++---------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 2d4b48bdc582..d04855eaa54a 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -64,12 +64,12 @@ use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use futures::TryStreamExt; use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::basic::Compression; use parquet::file::properties::WriterProperties; use rand::{rngs::StdRng, Rng, SeedableRng}; use std::sync::Arc; use tempfile::NamedTempFile; use tokio::fs::File; -use parquet::basic::Compression; /// Generates a random string. Has a 50% chance to generate a short string (3–11 characters) /// or a long string (13–20 characters). @@ -160,7 +160,9 @@ fn write_parquet_file() -> NamedTempFile { pretty_format_batches(&[batch.clone().slice(0, 100)]).unwrap() ); let schema = batch.schema(); - let props = WriterProperties::builder().set_compression( Compression::SNAPPY).build(); + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); let file = tempfile::Builder::new() .suffix(".parquet") .tempfile() @@ -286,35 +288,35 @@ impl FilterType { match self { // Point Lookup on int64 column FilterType::PointLookup => { - let array = batch.column(batch.schema().index_of("int64").unwrap()); + let array = batch.column(batch.schema().index_of("int64")?); let scalar = Int64Array::new_scalar(9999); eq(array, &scalar) } // Selective Unclustered on float64 column: float64 > 99.0 FilterType::SelectiveUnclustered => { - let array = batch.column(batch.schema().index_of("float64").unwrap()); + let array = batch.column(batch.schema().index_of("float64")?); let scalar = Float64Array::new_scalar(99.0); gt(array, &scalar) } // Moderately Selective Clustered on ts column: ts >= 9000 (implemented as > 8999) FilterType::ModeratelySelectiveClustered => { - let array = batch.column(batch.schema().index_of("ts").unwrap()); + let array = batch.column(batch.schema().index_of("ts")?); gt(array, &TimestampMillisecondArray::new_scalar(8999)) } // Moderately Selective Unclustered on int64 column: int64 > 90 FilterType::ModeratelySelectiveUnclustered => { - let array = batch.column(batch.schema().index_of("int64").unwrap()); + let array = batch.column(batch.schema().index_of("int64")?); let scalar = Int64Array::new_scalar(90); gt(array, &scalar) } // Unselective Unclustered on float64 column: NOT (float64 > 99.0) FilterType::UnselectiveUnclustered => { - let array = batch.column(batch.schema().index_of("float64").unwrap()); + let array = batch.column(batch.schema().index_of("float64")?); gt(array, &Float64Array::new_scalar(99.0)) } // Unselective Clustered on ts column: ts < 9000 FilterType::UnselectiveClustered => { - let array = batch.column(batch.schema().index_of("ts").unwrap()); + let array = batch.column(batch.schema().index_of("ts")?); lt(array, &TimestampMillisecondArray::new_scalar(9000)) } // Composite filter: logical AND of (float64 > 99.0) and (ts >= 9000) @@ -325,7 +327,7 @@ impl FilterType { } // Utf8ViewNonEmpty: selects rows where the utf8View column is not an empty string. FilterType::Utf8ViewNonEmpty => { - let array = batch.column(batch.schema().index_of("utf8View").unwrap()); + let array = batch.column(batch.schema().index_of("utf8View")?); let scalar = StringViewArray::new_scalar(""); neq(array, &scalar) } @@ -360,21 +362,22 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { let all_indices = vec![0, 1, 2, 3]; // Determine the filter column index based on the filter type. let filter_col = match filter_type { - FilterType::PointLookup => 0, - FilterType::SelectiveUnclustered => 1, - FilterType::ModeratelySelectiveClustered => 3, - FilterType::ModeratelySelectiveUnclustered => 0, - FilterType::UnselectiveUnclustered => 1, - FilterType::UnselectiveClustered => 3, - FilterType::Composite => 1, // Use float64 column as representative for composite - FilterType::Utf8ViewNonEmpty => 2, + FilterType::PointLookup => vec![0], + FilterType::SelectiveUnclustered => vec![1], + FilterType::ModeratelySelectiveClustered => vec![3], + FilterType::ModeratelySelectiveUnclustered => vec![0], + FilterType::UnselectiveUnclustered => vec![1], + FilterType::UnselectiveClustered => vec![3], + FilterType::Composite => vec![1, 3], // Use float64 column and ts column as representative for composite + FilterType::Utf8ViewNonEmpty => vec![2], }; - // For the projection, either select all columns or exclude the filter column. + + // For the projection, either select all columns or exclude the filter column(s). let output_projection: Vec = match proj_case { ProjectionCase::AllColumns => all_indices.clone(), ProjectionCase::ExcludeFilterColumn => all_indices .into_iter() - .filter(|i| *i != filter_col) + .filter(|i| !filter_col.contains(i)) .collect(), }; @@ -401,7 +404,7 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { output_projection.clone(), ); let pred_mask = - ProjectionMask::roots(file_metadata.schema_descr(), vec![filter_col]); + ProjectionMask::roots(file_metadata.schema_descr(), filter_col.clone()); let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { Ok(filter_type_inner.filter_batch(&batch).unwrap()) }); From 890519e31d711aa09990d5198295580e00ee8ef8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Apr 2025 13:08:13 -0400 Subject: [PATCH 44/73] Update comments --- parquet/benches/arrow_reader_row_filter.rs | 112 ++++++++++++++------- 1 file changed, 75 insertions(+), 37 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index d04855eaa54a..cf41c16a00c9 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -58,7 +58,7 @@ use arrow::compute::kernels::cmp::{eq, gt, lt, neq}; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arrow::record_batch::RecordBatch; use arrow_array::builder::StringViewBuilder; -use arrow_array::StringViewArray; +use arrow_array::{Array, StringViewArray}; use arrow_cast::pretty::pretty_format_batches; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use futures::TryStreamExt; @@ -197,7 +197,8 @@ impl std::fmt::Display for ProjectionCase { /// The variants correspond to the different filter patterns. #[derive(Clone)] enum FilterType { - /// Here is the 6 filter types: + /// "Point Lookup": selects a single row + /// ```text /// ┌───────────────┐ ┌───────────────┐ /// │ │ │ │ /// │ │ │ ... │ @@ -209,10 +210,11 @@ enum FilterType { /// │ │ │ │ /// │ │ │ │ /// └───────────────┘ └───────────────┘ - /// - /// "Point Lookup": selects a single row + /// ``` /// (1 RowSelection of 1 row) - /// + PointLookup, + /// selective (1%) unclustered filter + /// ```text /// ┌───────────────┐ ┌───────────────┐ /// │ ... │ │ │ /// │ │ │ │ @@ -224,45 +226,81 @@ enum FilterType { /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ /// │ │ │ │ /// └───────────────┘ └───────────────┘ - /// selective (1%) unclustered filter + /// ``` /// (1000 RowSelection of 10 rows each) - /// - /// - /// ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ - /// │ ... │ │ │ │ │ │ │ - /// │ │ │ │ │ │ │ │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ ... │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ │ │ - /// │ │ │ │ │ ... │ │ │ - /// │ │ │ ... │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │ ... │ │ │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │ │ │ │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// └───────────────┘ └───────────────┘ └───────────────┘ └───────────────┘ - /// moderately selective (10%) unclustered filter moderately selective (10%) clustered filter - /// (10000 RowSelection of 10 rows each) (10 RowSelections of 10,000 rows each) - /// ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ ... │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ ... │ │ │ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ - /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ └───────────────┘ └───────────────┘ - /// └───────────────┘ └───────────────┘ - /// unselective (99%) unclustered filter unselective (90%) clustered filter - /// (99,000 RowSelections of 10 rows each) (99 RowSelection of 10,000 rows each) - PointLookup, SelectiveUnclustered, + /// moderately selective (10%) clustered filter + /// ```text + /// ┌───────────────┐ ┌───────────────┐ + /// │ │ │ │ + /// │ │ │ │ + /// │ │ │ ... │ + /// │ │ │ │ + /// │ ... │ │ │ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// └───────────────┘ └───────────────┘ + /// ``` + /// (10 RowSelections of 10,000 rows each) ModeratelySelectiveClustered, + /// moderately selective (10%) clustered filter + /// ```text + /// ┌───────────────┐ ┌───────────────┐ + /// │ ... │ │ │ + /// │ │ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ + /// │ │ │ │ + /// │ │ │ ... │ + /// │ ... │ │ │ + /// │ │ │ │ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// └───────────────┘ └───────────────┘ + /// ``` + /// (10 RowSelections of 10,000 rows each) ModeratelySelectiveUnclustered, + /// unselective (99%) unclustered filter + /// ```text + /// ┌───────────────┐ ┌───────────────┐ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │ │ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// └───────────────┘ └───────────────┘ + /// ``` + /// (99,000 RowSelections of 10 rows each) UnselectiveUnclustered, + /// unselective (90%) clustered filter + /// ```text + /// ┌───────────────┐ ┌───────────────┐ + /// │ │ │ │ + /// │ │ │ │ + /// │ │ │ ... │ + /// │ │ │ │ + /// │ ... │ │ │ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// │ │ │▒▒▒▒▒▒▒▒▒▒▒▒▒▒▒│ + /// └───────────────┘ └───────────────┘ + /// ``` + /// (99 RowSelection of 10,000 rows each) UnselectiveClustered, - /// The following are Composite and Utf8ViewNonEmpty filters, which is the additional to above 6 filters. + /// [`Self::SelectivelUnclusered`] `AND` + /// [`Self::ModeratelySelectiveClustered`] Composite, + /// `utf8View <> ''` modeling [ClickBench] [Q21-Q27] + /// + /// [ClickBench]: https://github.com/ClickHouse/ClickBench + /// [Q21-Q27]: https://github.com/apache/datafusion/blob/b7177234e65cbbb2dcc04c252f6acd80bb026362/benchmarks/queries/clickbench/queries.sql#L22-L28 Utf8ViewNonEmpty, } From 7eb0476af3f2221147262ac91e62f74912ce172e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Apr 2025 13:12:43 -0400 Subject: [PATCH 45/73] refactor filter column indexes --- parquet/benches/arrow_reader_row_filter.rs | 27 ++++++++++++---------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index cf41c16a00c9..636372406689 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -371,6 +371,20 @@ impl FilterType { } } } + + /// Return the indexes in the batch's schema that are used for filtering. + fn filter_columns(&self) -> &'static [usize] { + match self { + FilterType::PointLookup => &[0], + FilterType::SelectiveUnclustered => &[1], + FilterType::ModeratelySelectiveClustered => &[3], + FilterType::ModeratelySelectiveUnclustered => &[0], + FilterType::UnselectiveUnclustered => &[1], + FilterType::UnselectiveClustered => &[3], + FilterType::Composite => &[1, 3], // Use float64 column and ts column as representative for composite + FilterType::Utf8ViewNonEmpty => &[2], + } + } } /// Benchmark filters and projections by reading the Parquet file. @@ -398,18 +412,7 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { for proj_case in &projection_cases { // All indices corresponding to the 10 columns. let all_indices = vec![0, 1, 2, 3]; - // Determine the filter column index based on the filter type. - let filter_col = match filter_type { - FilterType::PointLookup => vec![0], - FilterType::SelectiveUnclustered => vec![1], - FilterType::ModeratelySelectiveClustered => vec![3], - FilterType::ModeratelySelectiveUnclustered => vec![0], - FilterType::UnselectiveUnclustered => vec![1], - FilterType::UnselectiveClustered => vec![3], - FilterType::Composite => vec![1, 3], // Use float64 column and ts column as representative for composite - FilterType::Utf8ViewNonEmpty => vec![2], - }; - + let filter_col = filter_type.filter_columns(); // For the projection, either select all columns or exclude the filter column(s). let output_projection: Vec = match proj_case { ProjectionCase::AllColumns => all_indices.clone(), From 22c7b39bf84513af2edcfffdbedb3b60366116d0 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Apr 2025 13:34:01 -0400 Subject: [PATCH 46/73] Read from in memory buffer --- parquet/benches/arrow_reader_row_filter.rs | 78 ++++++++++++++++------ 1 file changed, 56 insertions(+), 22 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 636372406689..41d41a3a6f9a 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -58,18 +58,21 @@ use arrow::compute::kernels::cmp::{eq, gt, lt, neq}; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arrow::record_batch::RecordBatch; use arrow_array::builder::StringViewBuilder; -use arrow_array::{Array, StringViewArray}; +use arrow_array::StringViewArray; use arrow_cast::pretty::pretty_format_batches; +use bytes::Bytes; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use futures::TryStreamExt; +use futures::future::BoxFuture; +use futures::{FutureExt, TryStreamExt}; use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::basic::Compression; +use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; use parquet::file::properties::WriterProperties; use rand::{rngs::StdRng, Rng, SeedableRng}; +use std::ops::Range; use std::sync::Arc; -use tempfile::NamedTempFile; -use tokio::fs::File; /// Generates a random string. Has a 50% chance to generate a short string (3–11 characters) /// or a long string (13–20 characters). @@ -151,8 +154,8 @@ fn create_record_batch(size: usize) -> RecordBatch { RecordBatch::try_new(schema, arrays).unwrap() } -/// Writes the RecordBatch to a temporary Parquet file and returns the file handle. -fn write_parquet_file() -> NamedTempFile { +/// Writes the RecordBatch to an in memory buffer, returning the buffer +fn write_parquet_file() -> Vec { let batch = create_record_batch(100_000); println!("Batch created with {} rows", 100_000); println!( @@ -163,17 +166,13 @@ fn write_parquet_file() -> NamedTempFile { let props = WriterProperties::builder() .set_compression(Compression::SNAPPY) .build(); - let file = tempfile::Builder::new() - .suffix(".parquet") - .tempfile() - .unwrap(); + let mut buffer = vec![]; { - let file_reopen = file.reopen().unwrap(); - let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); + let mut writer = ArrowWriter::try_new(&mut buffer, schema.clone(), Some(props)).unwrap(); writer.write(&batch).unwrap(); writer.close().unwrap(); } - file + buffer } /// ProjectionCase defines the projection mode for the benchmark: @@ -195,7 +194,7 @@ impl std::fmt::Display for ProjectionCase { /// FilterType encapsulates the different filter comparisons. /// The variants correspond to the different filter patterns. -#[derive(Clone)] +#[derive(Clone, Copy)] enum FilterType { /// "Point Lookup": selects a single row /// ```text @@ -373,7 +372,7 @@ impl FilterType { } /// Return the indexes in the batch's schema that are used for filtering. - fn filter_columns(&self) -> &'static [usize] { + fn filter_projection(&self) -> &'static [usize] { match self { FilterType::PointLookup => &[0], FilterType::SelectiveUnclustered => &[1], @@ -391,7 +390,8 @@ impl FilterType { /// This benchmark iterates over all individual filter types and two projection cases. /// It measures the time to read and filter the Parquet file according to each scenario. fn benchmark_filters_and_projections(c: &mut Criterion) { - let parquet_file = write_parquet_file(); + // make the parquet file in memory that can be shared + let parquet_file = Bytes::from(write_parquet_file()); let filter_types = vec![ FilterType::PointLookup, FilterType::SelectiveUnclustered, @@ -408,11 +408,11 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { ]; let mut group = c.benchmark_group("arrow_reader_row_filter"); - for filter_type in filter_types.clone() { + for filter_type in filter_types { for proj_case in &projection_cases { // All indices corresponding to the 10 columns. let all_indices = vec![0, 1, 2, 3]; - let filter_col = filter_type.filter_columns(); + let filter_col = filter_type.filter_projection().to_vec(); // For the projection, either select all columns or exclude the filter column(s). let output_projection: Vec = match proj_case { ProjectionCase::AllColumns => all_indices.clone(), @@ -430,12 +430,11 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .build() .unwrap(); b.iter(|| { - let filter_type_inner = filter_type.clone(); rt.block_on(async { - let file = File::open(parquet_file.path()).await.unwrap(); + let reader = MemoryAsyncReader::new(&parquet_file); let options = ArrowReaderOptions::new().with_page_index(true); let builder = - ParquetRecordBatchStreamBuilder::new_with_options(file, options) + ParquetRecordBatchStreamBuilder::new_with_options(reader, options) .await .unwrap() .with_batch_size(8192); @@ -447,7 +446,7 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { let pred_mask = ProjectionMask::roots(file_metadata.schema_descr(), filter_col.clone()); let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { - Ok(filter_type_inner.filter_batch(&batch).unwrap()) + Ok(filter_type.filter_batch(&batch).unwrap()) }); let stream = builder .with_projection(mask) @@ -462,5 +461,40 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { } } +/// Adapter to read asynchronously from in memory bytes +#[derive(Debug)] +struct MemoryAsyncReader { + inner: Bytes, +} + +impl MemoryAsyncReader { + fn new(inner: &Bytes) -> Self { + // clone of bytes is cheap -- increments a refcount + Self { + inner: inner.clone(), + } + } +} + +impl AsyncFileReader for MemoryAsyncReader { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + let data = self.inner.slice(range.start as usize..range.end as usize); + async move { Ok(data) }.boxed() + } + + fn get_metadata<'a>( + &'a mut self, + _options: Option<&'a ArrowReaderOptions>, + ) -> BoxFuture<'a, parquet::errors::Result>> { + let inner = self.inner.clone(); + async move { + let mut metadata_reader = ParquetMetaDataReader::new().with_page_indexes(true); + metadata_reader.try_parse(&inner)?; + metadata_reader.finish().map(Arc::new) + } + .boxed() + } +} + criterion_group!(benches, benchmark_filters_and_projections,); criterion_main!(benches); From 5ae9b5811f90816967a33f6135a3b0078542ce16 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Apr 2025 13:37:37 -0400 Subject: [PATCH 47/73] celanu --- parquet/benches/arrow_reader_row_filter.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 41d41a3a6f9a..82060905196f 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -484,11 +484,12 @@ impl AsyncFileReader for MemoryAsyncReader { fn get_metadata<'a>( &'a mut self, - _options: Option<&'a ArrowReaderOptions>, + options: Option<&'a ArrowReaderOptions>, ) -> BoxFuture<'a, parquet::errors::Result>> { let inner = self.inner.clone(); + let page_index = options.map(|o| o.page_index()).unwrap_or(true); async move { - let mut metadata_reader = ParquetMetaDataReader::new().with_page_indexes(true); + let mut metadata_reader = ParquetMetaDataReader::new().with_page_indexes(page_index); metadata_reader.try_parse(&inner)?; metadata_reader.finish().map(Arc::new) } From 1effe88d71d2f7c87d1f3021b6841c04b6856a67 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Apr 2025 14:17:16 -0400 Subject: [PATCH 48/73] Test both sync and async readers --- parquet/benches/arrow_reader_row_filter.rs | 166 ++++++++++++++------- 1 file changed, 115 insertions(+), 51 deletions(-) diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index 82060905196f..e59ba59cc69d 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -63,8 +63,10 @@ use arrow_cast::pretty::pretty_format_batches; use bytes::Bytes; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use futures::future::BoxFuture; -use futures::{FutureExt, TryStreamExt}; -use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use futures::{FutureExt, StreamExt}; +use parquet::arrow::arrow_reader::{ + ArrowPredicateFn, ArrowReaderOptions, ParquetRecordBatchReaderBuilder, RowFilter, +}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::basic::Compression; @@ -194,7 +196,7 @@ impl std::fmt::Display for ProjectionCase { /// FilterType encapsulates the different filter comparisons. /// The variants correspond to the different filter patterns. -#[derive(Clone, Copy)] +#[derive(Clone, Copy, Debug)] enum FilterType { /// "Point Lookup": selects a single row /// ```text @@ -406,6 +408,12 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { ProjectionCase::AllColumns, ProjectionCase::ExcludeFilterColumn, ]; + + let rt = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build() + .unwrap(); + let mut group = c.benchmark_group("arrow_reader_row_filter"); for filter_type in filter_types { @@ -422,61 +430,123 @@ fn benchmark_filters_and_projections(c: &mut Criterion) { .collect(), }; - let bench_id = - BenchmarkId::new(format!("filter: {} proj: {}", filter_type, proj_case), ""); + let reader = InMemoryReader::try_new(&parquet_file).unwrap(); + let metadata = Arc::clone(reader.metadata()); + + let schema_descr = metadata.file_metadata().schema_descr(); + let projection_mask = ProjectionMask::roots(schema_descr, output_projection.clone()); + let pred_mask = ProjectionMask::roots(schema_descr, filter_col.clone()); + + let benchmark_name = format!("{filter_type:?}/{proj_case}",); + + // run the benchmark for the async reader + let bench_id = BenchmarkId::new(benchmark_name.clone(), "async"); + let rt_captured = rt.handle().clone(); group.bench_function(bench_id, |b| { - let rt = tokio::runtime::Builder::new_multi_thread() - .enable_all() - .build() - .unwrap(); b.iter(|| { - rt.block_on(async { - let reader = MemoryAsyncReader::new(&parquet_file); - let options = ArrowReaderOptions::new().with_page_index(true); - let builder = - ParquetRecordBatchStreamBuilder::new_with_options(reader, options) - .await - .unwrap() - .with_batch_size(8192); - let file_metadata = builder.metadata().file_metadata().clone(); - let mask = ProjectionMask::roots( - file_metadata.schema_descr(), - output_projection.clone(), - ); - let pred_mask = - ProjectionMask::roots(file_metadata.schema_descr(), filter_col.clone()); - let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { - Ok(filter_type.filter_batch(&batch).unwrap()) - }); - let stream = builder - .with_projection(mask) - .with_row_filter(RowFilter::new(vec![Box::new(filter)])) - .build() - .unwrap(); - stream.try_collect::>().await.unwrap(); + let reader = reader.clone(); + let pred_mask = pred_mask.clone(); + let projection_mask = projection_mask.clone(); + // row filters are not clone, so must make it each iter + let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { + Ok(filter_type.filter_batch(&batch).unwrap()) + }); + let row_filter = RowFilter::new(vec![Box::new(filter)]); + + rt_captured.block_on(async { + benchmark_async_reader(reader, projection_mask, row_filter).await; }) }); }); + + // run the benchmark for the sync reader + let bench_id = BenchmarkId::new(benchmark_name, "sync"); + group.bench_function(bench_id, |b| { + b.iter(|| { + let reader = reader.clone(); + let pred_mask = pred_mask.clone(); + let projection_mask = projection_mask.clone(); + // row filters are not clone, so must make it each iter + let filter = ArrowPredicateFn::new(pred_mask, move |batch: RecordBatch| { + Ok(filter_type.filter_batch(&batch).unwrap()) + }); + let row_filter = RowFilter::new(vec![Box::new(filter)]); + + benchmark_sync_reader(reader, projection_mask, row_filter) + }); + }); } } } -/// Adapter to read asynchronously from in memory bytes -#[derive(Debug)] -struct MemoryAsyncReader { +/// Use async API +async fn benchmark_async_reader( + reader: InMemoryReader, + projection_mask: ProjectionMask, + row_filter: RowFilter, +) { + let mut stream = ParquetRecordBatchStreamBuilder::new(reader) + .await + .unwrap() + .with_batch_size(8192) + .with_projection(projection_mask) + .with_row_filter(row_filter) + .build() + .unwrap(); + while let Some(b) = stream.next().await { + b.unwrap(); // consume the batches, no buffering + } +} + +/// Use sync API +fn benchmark_sync_reader( + reader: InMemoryReader, + projection_mask: ProjectionMask, + row_filter: RowFilter, +) { + let stream = ParquetRecordBatchReaderBuilder::try_new(reader.into_inner()) + .unwrap() + .with_batch_size(8192) + .with_projection(projection_mask) + .with_row_filter(row_filter) + .build() + .unwrap(); + for b in stream { + b.unwrap(); // consume the batches, no buffering + } +} + +/// Adapter to read asynchronously from in memory bytes and always loads the +/// metadata with page indexes. +#[derive(Debug, Clone)] +struct InMemoryReader { inner: Bytes, + metadata: Arc, } -impl MemoryAsyncReader { - fn new(inner: &Bytes) -> Self { - // clone of bytes is cheap -- increments a refcount - Self { +impl InMemoryReader { + fn try_new(inner: &Bytes) -> parquet::errors::Result { + let mut metadata_reader = ParquetMetaDataReader::new().with_page_indexes(true); + metadata_reader.try_parse(inner)?; + let metadata = metadata_reader.finish().map(Arc::new)?; + + Ok(Self { + // clone of bytes is cheap -- increments a refcount inner: inner.clone(), - } + metadata, + }) + } + + fn metadata(&self) -> &Arc { + &self.metadata + } + + fn into_inner(self) -> Bytes { + self.inner } } -impl AsyncFileReader for MemoryAsyncReader { +impl AsyncFileReader for InMemoryReader { fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { let data = self.inner.slice(range.start as usize..range.end as usize); async move { Ok(data) }.boxed() @@ -484,16 +554,10 @@ impl AsyncFileReader for MemoryAsyncReader { fn get_metadata<'a>( &'a mut self, - options: Option<&'a ArrowReaderOptions>, + _options: Option<&'a ArrowReaderOptions>, ) -> BoxFuture<'a, parquet::errors::Result>> { - let inner = self.inner.clone(); - let page_index = options.map(|o| o.page_index()).unwrap_or(true); - async move { - let mut metadata_reader = ParquetMetaDataReader::new().with_page_indexes(page_index); - metadata_reader.try_parse(&inner)?; - metadata_reader.finish().map(Arc::new) - } - .boxed() + let metadata = Arc::clone(&self.metadata); + async move { Ok(metadata) }.boxed() } } From 0c3aa9b3d4fa913029742f36b650f7b1e1ea2339 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 19 Apr 2025 13:49:45 +0800 Subject: [PATCH 49/73] Improve the performance for skip record --- .../src/arrow/async_reader/arrow_reader.rs | 25 +++++++++++++++++-- parquet/src/file/serialized_reader.rs | 4 ++- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 92e585756d49..2c734d79ede1 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -179,26 +179,47 @@ impl Iterator for FilteredParquetRecordBatchReader { // rather than concatenating multiple small batches. let mut selected = 0; + + // Try to merge skip selectors, because from testing merge then to skip + // is faster than skipping each one. + let mut skip_accum = 0; + while let Some(cur_selection) = take_next_selection(&mut self.selection, self.batch_size - selected) { let filtered_selection = match self.build_predicate_filter(cur_selection) { Ok(selection) => selection, - Err(e) => return Some(Err(e)), + Err(e) => { + if skip_accum > 0 { + self.array_reader.skip_records(skip_accum).ok()?; + } + return Some(Err(e)); + } }; for selector in filtered_selection.iter() { if selector.skip { - self.array_reader.skip_records(selector.row_count).ok()?; + skip_accum += selector.row_count; } else { + if skip_accum > 0 { + self.array_reader.skip_records(skip_accum).ok()?; + skip_accum = 0; + } self.array_reader.read_records(selector.row_count).ok()?; } } + selected += filtered_selection.row_count(); + if selected >= (self.batch_size / 4 * 3) { break; } } + + if skip_accum > 0 { + self.array_reader.skip_records(skip_accum).ok()?; + } + if selected == 0 { return None; } diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index ce4b16ac1fa3..46a94f6b7416 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -984,7 +984,9 @@ impl PageReader for SerializedPageReader { SerializedPageReaderState::Pages { page_locations, - dictionary_page, .. } => { + dictionary_page, + .. + } => { if dictionary_page.is_some() { // If a dictionary page exists, consume it by taking it (sets to None) dictionary_page.take(); From a1d349663bb85dd44ed5848cb2223a2b75a29fb5 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 25 Apr 2025 18:01:40 +0800 Subject: [PATCH 50/73] Init the boolean_selector --- .../arrow/arrow_reader/boolean_selector.rs | 313 ++++++++++++++++++ parquet/src/arrow/arrow_reader/mod.rs | 1 + 2 files changed, 314 insertions(+) create mode 100644 parquet/src/arrow/arrow_reader/boolean_selector.rs diff --git a/parquet/src/arrow/arrow_reader/boolean_selector.rs b/parquet/src/arrow/arrow_reader/boolean_selector.rs new file mode 100644 index 000000000000..1621ed7b6177 --- /dev/null +++ b/parquet/src/arrow/arrow_reader/boolean_selector.rs @@ -0,0 +1,313 @@ +// 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::ops::Range; + +use arrow_array::{Array, BooleanArray}; +use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder, MutableBuffer}; +use arrow_data::bit_iterator::BitIndexIterator; + +use super::{RowSelection, RowSelector}; + +/// A selection of rows, similar to [`RowSelection`], but based on a boolean array +#[derive(Debug, Clone, PartialEq)] +pub struct BooleanRowSelection { + selector: BooleanArray, +} + +impl BooleanRowSelection { + /// Create a new [`BooleanRowSelection] from a list of [`BooleanArray`]. + pub fn from_filters(filters: &[BooleanArray]) -> Self { + let arrays: Vec<&dyn Array> = filters.iter().map(|x| x as &dyn Array).collect(); + let result = arrow_select::concat::concat(&arrays).unwrap().into_data(); + let boolean_array = BooleanArray::from(result); + BooleanRowSelection { + selector: boolean_array, + } + } + + /// Create a new [`BooleanRowSelection`] with all rows unselected + pub fn new_unselected(row_count: usize) -> Self { + let buffer = BooleanBuffer::new_unset(row_count); + let boolean_array = BooleanArray::from(buffer); + BooleanRowSelection { selector: boolean_array } + } + + /// Create a new [`BooleanRowSelection`] with all rows selected + pub fn new_selected(row_count: usize) -> Self { + let buffer = BooleanBuffer::new_set(row_count); + let boolean_array = BooleanArray::from(buffer); + BooleanRowSelection { selector: boolean_array } + } + + /// Returns a new [`BooleanRowSelection`] that selects the inverse of this [`BooleanRowSelection`]. + pub fn as_inverted(&self) -> Self { + let buffer = !self.selector.values(); + BooleanRowSelection { selector: BooleanArray::from(buffer) } + } + + /// Returns the number of rows selected by this [`BooleanRowSelection`]. + pub fn row_count(&self) -> usize { + self.selector.true_count() + } + + /// Create a new [`BooleanRowSelection`] from a list of consecutive ranges. + pub fn from_consecutive_ranges( + ranges: impl Iterator>, + total_rows: usize, + ) -> Self { + let mut buffer = BooleanBufferBuilder::new(total_rows); + let mut last_end = 0; + + for range in ranges { + let len = range.end - range.start; + if len == 0 { + continue; + } + + if range.start > last_end { + buffer.append_n(range.start - last_end, false); + } + buffer.append_n(len, true); + last_end = range.end; + } + + if last_end != total_rows { + buffer.append_n(total_rows - last_end, false); + } + + BooleanRowSelection { + selector: BooleanArray::from(buffer.finish()), + } + } + + /// Compute the union of two [`BooleanRowSelection`] + /// For example: + /// self: NNYYYYNNYYNYN + /// other: NYNNNNNNN + /// + /// returned: NYYYYYNNYYNYN + #[must_use] + pub fn union(&self, other: &Self) -> Self { + // use arrow::compute::kernels::boolean::or; + + let union_selectors = self.selector.values() | other.selector.values(); + + BooleanRowSelection { + selector: BooleanArray::from(union_selectors), + } + } + + /// Compute the intersection of two [`BooleanRowSelection`] + /// For example: + /// self: NNYYYYNNYYNYN + /// other: NYNNNNNNY + /// + /// returned: NNNNNNNNYYNYN + #[must_use] + pub fn intersection(&self, other: &Self) -> Self { + let intersection_selectors = self.selector.values() & other.selector.values(); + + BooleanRowSelection { + selector: BooleanArray::from(intersection_selectors), + } + } + + /// Combines this [`BooleanRowSelection`] with another using logical AND on the selected bits. + /// + /// Unlike intersection, the `other` [`BooleanRowSelection`] must have exactly as many set bits as `self`. + /// This method will keep only the bits in `self` that are also set in `other` + /// at the positions corresponding to `self`'s set bits. + pub fn and_then(&self, other: &Self) -> Self { + // Ensure that 'other' has exactly as many set bits as 'self' + debug_assert_eq!( + self.row_count(), + other.selector.len(), + "The 'other' selection must have exactly as many set bits as 'self'." + ); + + if self.selector.len() == other.selector.len() { + // fast path if the two selections are the same length + // common if this is the first predicate + debug_assert_eq!(self.row_count(), self.selector.len()); + return self.intersection(other); + } + + let mut buffer = MutableBuffer::from_len_zeroed(self.selector.values().inner().len()); + buffer.copy_from_slice(self.selector.values().inner().as_slice()); + let mut builder = BooleanBufferBuilder::new_from_buffer(buffer, self.selector.len()); + + // Create iterators for 'self' and 'other' bits + let mut other_bits = other.selector.iter(); + + for bit_idx in self.true_iter() { + let predicate = other_bits + .next() + .expect("Mismatch in set bits between self and other"); + if !predicate.unwrap() { + builder.set_bit(bit_idx, false); + } + } + + BooleanRowSelection { + selector: BooleanArray::from(builder.finish()), + } + } + + /// Returns an iterator over the indices of the set bits in this [`BooleanRowSelection`] + pub fn true_iter(&self) -> BitIndexIterator<'_> { + self.selector.values().set_indices() + } + + /// Returns `true` if this [`BooleanRowSelection`] selects any rows + pub fn selects_any(&self) -> bool { + self.true_iter().next().is_some() + } + + /// Returns a new [`BooleanRowSelection`] that selects the rows in this [`BooleanRowSelection`] from `offset` to `offset + len` + pub fn slice(&self, offset: usize, len: usize) -> BooleanArray { + self.selector.slice(offset, len) + } +} + +impl From> for BooleanRowSelection { + fn from(selection: Vec) -> Self { + let selection = RowSelection::from(selection); + RowSelection::into(selection) + } +} + +impl From for BooleanRowSelection { + fn from(selection: RowSelection) -> Self { + let total_rows = selection.row_count(); + let mut builder = BooleanBufferBuilder::new(total_rows); + + for selector in selection.iter() { + if selector.skip { + builder.append_n(selector.row_count, false); + } else { + builder.append_n(selector.row_count, true); + } + } + + BooleanRowSelection { + selector: BooleanArray::from(builder.finish()), + } + } +} + +impl From<&BooleanRowSelection> for RowSelection{ + fn from(selection: &BooleanRowSelection) -> Self { + RowSelection::from_filters(&[selection.selector.clone()]) + } +} + +#[cfg(test)] +mod tests { + use rand::Rng; + + use super::*; + + fn generate_random_row_selection(total_rows: usize, selection_ratio: f64) -> BooleanArray { + let mut rng = rand::thread_rng(); + let bools: Vec = (0..total_rows) + .map(|_| rng.gen_bool(selection_ratio)) + .collect(); + BooleanArray::from(bools) + } + + #[test] + fn test_boolean_row_selection_round_trip() { + let total_rows = 1_000; + for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { + let selection = generate_random_row_selection(total_rows, selection_ratio); + let boolean_selection = BooleanRowSelection::from_filters(&[selection]); + let row_selection = RowSelection::from(&boolean_selection); + let boolean_selection_again = row_selection.into(); + assert_eq!(boolean_selection, boolean_selection_again); + } + } + + #[test] + fn test_boolean_union_intersection() { + let total_rows = 1_000; + + let base_boolean_selection = + BooleanRowSelection::from_filters(&[generate_random_row_selection(total_rows, 0.1)]); + let base_row_selection = RowSelection::from(&base_boolean_selection); + for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { + let boolean_selection = + BooleanRowSelection::from_filters(&[generate_random_row_selection( + total_rows, + selection_ratio, + )]); + let row_selection = RowSelection::from(&boolean_selection); + + let boolean_union = boolean_selection.union(&base_boolean_selection); + let row_union = row_selection.union(&base_row_selection); + assert_eq!(boolean_union, BooleanRowSelection::from(row_union)); + + let boolean_intersection = boolean_selection.intersection(&base_boolean_selection); + let row_intersection = row_selection.intersection(&base_row_selection); + assert_eq!( + boolean_intersection, + BooleanRowSelection::from(row_intersection) + ); + } + } + + #[test] + fn test_boolean_selection_and_then() { + // Initial mask: 001011010101 + let self_filters = vec![BooleanArray::from(vec![ + false, false, true, false, true, true, false, true, false, true, false, true, + ])]; + let self_selection = BooleanRowSelection::from_filters(&self_filters); + + // Predicate mask (only for selected bits): 001101 + let other_filters = vec![BooleanArray::from(vec![ + false, false, true, true, false, true, + ])]; + let other_selection = BooleanRowSelection::from_filters(&other_filters); + + let result = self_selection.and_then(&other_selection); + + // Expected result: 000001010001 + let expected_filters = vec![BooleanArray::from(vec![ + false, false, false, false, false, true, false, true, false, false, false, true, + ])]; + let expected_selection = BooleanRowSelection::from_filters(&expected_filters); + + assert_eq!(result, expected_selection); + } + + #[test] + #[should_panic( + expected = "The 'other' selection must have exactly as many set bits as 'self'." + )] + fn test_and_then_mismatched_set_bits() { + let self_filters = vec![BooleanArray::from(vec![true, true, false])]; + let self_selection = BooleanRowSelection::from_filters(&self_filters); + + // 'other' has only one set bit, but 'self' has two + let other_filters = vec![BooleanArray::from(vec![true, false, false])]; + let other_selection = BooleanRowSelection::from_filters(&other_filters); + + // This should panic + let _ = self_selection.and_then(&other_selection); + } +} \ No newline at end of file diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 8bbe175dafb8..4ed82dc41326 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -42,6 +42,7 @@ use crate::schema::types::SchemaDescriptor; mod filter; mod selection; pub mod statistics; +mod boolean_selector; /// Builder for constructing Parquet readers that decode into [Apache Arrow] /// arrays. From 2d6c86668f09613e846f1ed8c5b03296526ff7f8 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 26 Apr 2025 15:14:35 +0800 Subject: [PATCH 51/73] Init version for unified select --- .../arrow/arrow_reader/boolean_selector.rs | 254 +-- parquet/src/arrow/arrow_reader/mod.rs | 4 +- parquet/src/arrow/arrow_reader/selection.rs | 1904 +++++++++-------- 3 files changed, 1180 insertions(+), 982 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/boolean_selector.rs b/parquet/src/arrow/arrow_reader/boolean_selector.rs index 1621ed7b6177..7e1dd2a98efc 100644 --- a/parquet/src/arrow/arrow_reader/boolean_selector.rs +++ b/parquet/src/arrow/arrow_reader/boolean_selector.rs @@ -184,130 +184,130 @@ impl BooleanRowSelection { } } -impl From> for BooleanRowSelection { - fn from(selection: Vec) -> Self { - let selection = RowSelection::from(selection); - RowSelection::into(selection) - } -} - -impl From for BooleanRowSelection { - fn from(selection: RowSelection) -> Self { - let total_rows = selection.row_count(); - let mut builder = BooleanBufferBuilder::new(total_rows); - - for selector in selection.iter() { - if selector.skip { - builder.append_n(selector.row_count, false); - } else { - builder.append_n(selector.row_count, true); - } - } - - BooleanRowSelection { - selector: BooleanArray::from(builder.finish()), - } - } -} - -impl From<&BooleanRowSelection> for RowSelection{ - fn from(selection: &BooleanRowSelection) -> Self { - RowSelection::from_filters(&[selection.selector.clone()]) - } -} - -#[cfg(test)] -mod tests { - use rand::Rng; - - use super::*; - - fn generate_random_row_selection(total_rows: usize, selection_ratio: f64) -> BooleanArray { - let mut rng = rand::thread_rng(); - let bools: Vec = (0..total_rows) - .map(|_| rng.gen_bool(selection_ratio)) - .collect(); - BooleanArray::from(bools) - } - - #[test] - fn test_boolean_row_selection_round_trip() { - let total_rows = 1_000; - for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { - let selection = generate_random_row_selection(total_rows, selection_ratio); - let boolean_selection = BooleanRowSelection::from_filters(&[selection]); - let row_selection = RowSelection::from(&boolean_selection); - let boolean_selection_again = row_selection.into(); - assert_eq!(boolean_selection, boolean_selection_again); - } - } - - #[test] - fn test_boolean_union_intersection() { - let total_rows = 1_000; - - let base_boolean_selection = - BooleanRowSelection::from_filters(&[generate_random_row_selection(total_rows, 0.1)]); - let base_row_selection = RowSelection::from(&base_boolean_selection); - for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { - let boolean_selection = - BooleanRowSelection::from_filters(&[generate_random_row_selection( - total_rows, - selection_ratio, - )]); - let row_selection = RowSelection::from(&boolean_selection); - - let boolean_union = boolean_selection.union(&base_boolean_selection); - let row_union = row_selection.union(&base_row_selection); - assert_eq!(boolean_union, BooleanRowSelection::from(row_union)); - - let boolean_intersection = boolean_selection.intersection(&base_boolean_selection); - let row_intersection = row_selection.intersection(&base_row_selection); - assert_eq!( - boolean_intersection, - BooleanRowSelection::from(row_intersection) - ); - } - } - - #[test] - fn test_boolean_selection_and_then() { - // Initial mask: 001011010101 - let self_filters = vec![BooleanArray::from(vec![ - false, false, true, false, true, true, false, true, false, true, false, true, - ])]; - let self_selection = BooleanRowSelection::from_filters(&self_filters); - - // Predicate mask (only for selected bits): 001101 - let other_filters = vec![BooleanArray::from(vec![ - false, false, true, true, false, true, - ])]; - let other_selection = BooleanRowSelection::from_filters(&other_filters); - - let result = self_selection.and_then(&other_selection); - - // Expected result: 000001010001 - let expected_filters = vec![BooleanArray::from(vec![ - false, false, false, false, false, true, false, true, false, false, false, true, - ])]; - let expected_selection = BooleanRowSelection::from_filters(&expected_filters); - - assert_eq!(result, expected_selection); - } - - #[test] - #[should_panic( - expected = "The 'other' selection must have exactly as many set bits as 'self'." - )] - fn test_and_then_mismatched_set_bits() { - let self_filters = vec![BooleanArray::from(vec![true, true, false])]; - let self_selection = BooleanRowSelection::from_filters(&self_filters); - - // 'other' has only one set bit, but 'self' has two - let other_filters = vec![BooleanArray::from(vec![true, false, false])]; - let other_selection = BooleanRowSelection::from_filters(&other_filters); - - // This should panic - let _ = self_selection.and_then(&other_selection); - } -} \ No newline at end of file +// impl From> for BooleanRowSelection { +// fn from(selection: Vec) -> Self { +// let selection = RowSelection::from(selection); +// RowSelection::into(selection) +// } +// } +// +// impl From for BooleanRowSelection { +// fn from(selection: RowSelection) -> Self { +// let total_rows = selection.row_count(); +// let mut builder = BooleanBufferBuilder::new(total_rows); +// +// for selector in selection.iter() { +// if selector.skip { +// builder.append_n(selector.row_count, false); +// } else { +// builder.append_n(selector.row_count, true); +// } +// } +// +// BooleanRowSelection { +// selector: BooleanArray::from(builder.finish()), +// } +// } +// } +// +// impl From<&BooleanRowSelection> for RowSelection{ +// fn from(selection: &BooleanRowSelection) -> Self { +// RowSelection::from_filters(&[selection.selector.clone()]) +// } +// } + +// #[cfg(test)] +// mod tests { +// use rand::Rng; +// +// use super::*; +// +// fn generate_random_row_selection(total_rows: usize, selection_ratio: f64) -> BooleanArray { +// let mut rng = rand::thread_rng(); +// let bools: Vec = (0..total_rows) +// .map(|_| rng.gen_bool(selection_ratio)) +// .collect(); +// BooleanArray::from(bools) +// } +// +// #[test] +// fn test_boolean_row_selection_round_trip() { +// let total_rows = 1_000; +// for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { +// let selection = generate_random_row_selection(total_rows, selection_ratio); +// let boolean_selection = BooleanRowSelection::from_filters(&[selection]); +// let row_selection = RowSelection::from(&boolean_selection); +// let boolean_selection_again = row_selection.into(); +// assert_eq!(boolean_selection, boolean_selection_again); +// } +// } +// +// #[test] +// fn test_boolean_union_intersection() { +// let total_rows = 1_000; +// +// let base_boolean_selection = +// BooleanRowSelection::from_filters(&[generate_random_row_selection(total_rows, 0.1)]); +// let base_row_selection = RowSelection::from(&base_boolean_selection); +// for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { +// let boolean_selection = +// BooleanRowSelection::from_filters(&[generate_random_row_selection( +// total_rows, +// selection_ratio, +// )]); +// let row_selection = RowSelection::from(&boolean_selection); +// +// let boolean_union = boolean_selection.union(&base_boolean_selection); +// let row_union = row_selection.union(&base_row_selection); +// assert_eq!(boolean_union, BooleanRowSelection::from(row_union)); +// +// let boolean_intersection = boolean_selection.intersection(&base_boolean_selection); +// let row_intersection = row_selection.intersection(&base_row_selection); +// assert_eq!( +// boolean_intersection, +// BooleanRowSelection::from(row_intersection) +// ); +// } +// } +// +// #[test] +// fn test_boolean_selection_and_then() { +// // Initial mask: 001011010101 +// let self_filters = vec![BooleanArray::from(vec![ +// false, false, true, false, true, true, false, true, false, true, false, true, +// ])]; +// let self_selection = BooleanRowSelection::from_filters(&self_filters); +// +// // Predicate mask (only for selected bits): 001101 +// let other_filters = vec![BooleanArray::from(vec![ +// false, false, true, true, false, true, +// ])]; +// let other_selection = BooleanRowSelection::from_filters(&other_filters); +// +// let result = self_selection.and_then(&other_selection); +// +// // Expected result: 000001010001 +// let expected_filters = vec![BooleanArray::from(vec![ +// false, false, false, false, false, true, false, true, false, false, false, true, +// ])]; +// let expected_selection = BooleanRowSelection::from_filters(&expected_filters); +// +// assert_eq!(result, expected_selection); +// } +// +// #[test] +// #[should_panic( +// expected = "The 'other' selection must have exactly as many set bits as 'self'." +// )] +// fn test_and_then_mismatched_set_bits() { +// let self_filters = vec![BooleanArray::from(vec![true, true, false])]; +// let self_selection = BooleanRowSelection::from_filters(&self_filters); +// +// // 'other' has only one set bit, but 'self' has two +// let other_filters = vec![BooleanArray::from(vec![true, false, false])]; +// let other_selection = BooleanRowSelection::from_filters(&other_filters); +// +// // This should panic +// let _ = self_selection.and_then(&other_selection); +// } +// } \ No newline at end of file diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 4ed82dc41326..068d60678bf9 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -996,7 +996,9 @@ pub(crate) fn evaluate_predicate( }; } - let raw = RowSelection::from_filters(&filters); + // let raw = RowSelection::from_filters(&filters); + // Testing using + let raw = RowSelection::from_filters_as_bitmap(&filters); Ok(match input_selection { Some(selection) => selection.and_then(&raw), None => raw, diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index c53d47be2e56..0784c14040f4 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -20,6 +20,9 @@ use arrow_select::filter::SlicesIterator; use std::cmp::Ordering; use std::collections::VecDeque; use std::ops::Range; +use arrow_buffer::{BooleanBufferBuilder, MutableBuffer}; +use arrow_schema::ArrowError::NotYetImplemented; +use crate::arrow::arrow_reader::boolean_selector::BooleanRowSelection; /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when /// scanning a parquet file @@ -96,9 +99,21 @@ impl RowSelector { /// * Consecutive [`RowSelector`]s alternate skipping or selecting rows /// /// [`PageIndex`]: crate::file::page_index::index::PageIndex -#[derive(Debug, Clone, Default, Eq, PartialEq)] -pub struct RowSelection { - selectors: Vec, +// #[derive(Debug, Clone, Default, Eq, PartialEq)] +// pub struct RowSelection { +// selectors: Vec, +// } + +#[derive(Debug, Clone, PartialEq)] +pub enum RowSelection { + Ranges(Vec), + BitMap(BooleanArray), +} + +impl Default for RowSelection { + fn default() -> Self { + RowSelection::Ranges(Vec::new()) + } } impl RowSelection { @@ -120,6 +135,15 @@ impl RowSelection { Self::from_consecutive_ranges(iter, total_rows) } + + /// Creates a [`RowSelection`] from a slice of [`BooleanArray`] as a bitmap + /// + pub fn from_filters_as_bitmap(filters: &[BooleanArray]) -> Self { + let arrays: Vec<&dyn Array> = filters.iter().map(|x| x as &dyn Array).collect(); + let result = arrow_select::concat::concat(&arrays).unwrap().into_data(); + let boolean_array = BooleanArray::from(result); + Self::BitMap(boolean_array) + } /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep pub fn from_consecutive_ranges>>( @@ -152,7 +176,7 @@ impl RowSelection { selectors.push(RowSelector::skip(total_rows - last_end)) } - Self { selectors } + RowSelection::Ranges(selectors) } /// Given an offset index, return the byte ranges for all data pages selected by `self` @@ -163,91 +187,108 @@ impl RowSelection { /// ranges that are close together. This is instead delegated to the IO subsystem to optimise, /// e.g. [`ObjectStore::get_ranges`](object_store::ObjectStore::get_ranges) pub fn scan_ranges(&self, page_locations: &[crate::format::PageLocation]) -> Vec> { - let mut ranges: Vec> = vec![]; - let mut row_offset = 0; - - let mut pages = page_locations.iter().peekable(); - let mut selectors = self.selectors.iter().cloned(); - let mut current_selector = selectors.next(); - let mut current_page = pages.next(); - - let mut current_page_included = false; - - while let Some((selector, page)) = current_selector.as_mut().zip(current_page) { - if !(selector.skip || current_page_included) { - let start = page.offset as u64; - let end = start + page.compressed_page_size as u64; - ranges.push(start..end); - current_page_included = true; - } + match self { + RowSelection::Ranges(selectors) => { + let mut ranges: Vec> = vec![]; + let mut row_offset = 0; + + let mut pages = page_locations.iter().peekable(); + let mut selectors = selectors.iter().cloned(); + let mut current_selector = selectors.next(); + let mut current_page = pages.next(); + + let mut current_page_included = false; + + while let Some((selector, page)) = current_selector.as_mut().zip(current_page) { + if !(selector.skip || current_page_included) { + let start = page.offset as u64; + let end = start + page.compressed_page_size as u64; + ranges.push(start..end); + current_page_included = true; + } - if let Some(next_page) = pages.peek() { - if row_offset + selector.row_count > next_page.first_row_index as usize { - let remaining_in_page = next_page.first_row_index as usize - row_offset; - selector.row_count -= remaining_in_page; - row_offset += remaining_in_page; - current_page = pages.next(); - current_page_included = false; - - continue; - } else { - if row_offset + selector.row_count == next_page.first_row_index as usize { - current_page = pages.next(); - current_page_included = false; + if let Some(next_page) = pages.peek() { + if row_offset + selector.row_count > next_page.first_row_index as usize { + let remaining_in_page = next_page.first_row_index as usize - row_offset; + selector.row_count -= remaining_in_page; + row_offset += remaining_in_page; + current_page = pages.next(); + current_page_included = false; + + continue; + } else { + if row_offset + selector.row_count == next_page.first_row_index as usize { + current_page = pages.next(); + current_page_included = false; + } + row_offset += selector.row_count; + current_selector = selectors.next(); + } + } else { + if !(selector.skip || current_page_included) { + let start = page.offset as u64; + let end = start + page.compressed_page_size as u64; + ranges.push(start..end); + } + current_selector = selectors.next() } - row_offset += selector.row_count; - current_selector = selectors.next(); - } - } else { - if !(selector.skip || current_page_included) { - let start = page.offset as u64; - let end = start + page.compressed_page_size as u64; - ranges.push(start..end); } - current_selector = selectors.next() + + ranges + } + RowSelection::BitMap(bitmap) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") } } - - ranges + + } /// Splits off the first `row_count` from this [`RowSelection`] pub fn split_off(&mut self, row_count: usize) -> Self { - let mut total_count = 0; - - // Find the index where the selector exceeds the row count - let find = self.selectors.iter().position(|selector| { - total_count += selector.row_count; - total_count > row_count - }); - - let split_idx = match find { - Some(idx) => idx, - None => { - let selectors = std::mem::take(&mut self.selectors); - return Self { selectors }; + match self { + RowSelection::BitMap(_) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") } - }; + RowSelection::Ranges(selectors) => { + let mut total_count = 0; + + // Find the index where the selector exceeds the row count + let find = selectors.iter().position(|selector| { + total_count += selector.row_count; + total_count > row_count + }); + + let split_idx = match find { + Some(idx) => idx, + None => { + let selectors = std::mem::take(selectors); + return RowSelection::Ranges(selectors); + } + }; - let mut remaining = self.selectors.split_off(split_idx); + let mut remaining = selectors.split_off(split_idx); - // Always present as `split_idx < self.selectors.len` - let next = remaining.first_mut().unwrap(); - let overflow = total_count - row_count; + // Always present as `split_idx < self.selectors.len` + let next = remaining.first_mut().unwrap(); + let overflow = total_count - row_count; - if next.row_count != overflow { - self.selectors.push(RowSelector { - row_count: next.row_count - overflow, - skip: next.skip, - }) - } - next.row_count = overflow; + if next.row_count != overflow { + selectors.push(RowSelector { + row_count: next.row_count - overflow, + skip: next.skip, + }) + } + next.row_count = overflow; - std::mem::swap(&mut remaining, &mut self.selectors); - Self { - selectors: remaining, + std::mem::swap(&mut remaining, selectors); + RowSelection::Ranges(remaining) + } } } + /// returns a [`RowSelection`] representing rows that are selected in both /// input [`RowSelection`]s. /// @@ -271,66 +312,106 @@ impl RowSelection { /// by this RowSelection /// pub fn and_then(&self, other: &Self) -> Self { - let mut selectors = vec![]; - let mut first = self.selectors.iter().cloned().peekable(); - let mut second = other.selectors.iter().cloned().peekable(); - - let mut to_skip = 0; - while let Some(b) = second.peek_mut() { - let a = first - .peek_mut() - .expect("selection exceeds the number of selected rows"); - - if b.row_count == 0 { - second.next().unwrap(); - continue; - } + match (self, other) { + (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bitmap)) => { + // Ensure that 'other' has exactly as many set bits as 'self' + debug_assert_eq!( + self.row_count(), + other_bitmap.len(), + "The 'other' selection must have exactly as many set bits as 'self'." + ); - if a.row_count == 0 { - first.next().unwrap(); - continue; - } + if bit_map.len() == other_bitmap.len() { + // fast path if the two selections are the same length + // common if this is the first predicate + debug_assert_eq!(self.row_count(), other_bitmap.len()); + return self.intersection(other); + } - if a.skip { - // Records were skipped when producing second - to_skip += a.row_count; - first.next().unwrap(); - continue; + let mut buffer = MutableBuffer::from_len_zeroed(bit_map.values().inner().len()); + buffer.copy_from_slice(bit_map.values().inner().as_slice()); + let mut builder = BooleanBufferBuilder::new_from_buffer(buffer, bit_map.len()); + + // Create iterators for 'self' and 'other' bits + let mut other_bits = other_bitmap.iter(); + + for bit_idx in bit_map.values().set_indices() { + let predicate = other_bits + .next() + .expect("Mismatch in set bits between self and other"); + if !predicate.unwrap() { + builder.set_bit(bit_idx, false); + } + } + + RowSelection::BitMap(BooleanArray::from(builder.finish())) } + (RowSelection::Ranges(selectors), RowSelection::Ranges(other_selectors)) => { + let mut and_then_select = vec![]; + let mut first = selectors.iter().cloned().peekable(); + let mut second = selectors.iter().cloned().peekable(); + + let mut to_skip = 0; + while let Some(b) = second.peek_mut() { + let a = first + .peek_mut() + .expect("selection exceeds the number of selected rows"); + + if b.row_count == 0 { + second.next().unwrap(); + continue; + } + + if a.row_count == 0 { + first.next().unwrap(); + continue; + } - let skip = b.skip; - let to_process = a.row_count.min(b.row_count); + if a.skip { + // Records were skipped when producing second + to_skip += a.row_count; + first.next().unwrap(); + continue; + } + + let skip = b.skip; + let to_process = a.row_count.min(b.row_count); - a.row_count -= to_process; - b.row_count -= to_process; + a.row_count -= to_process; + b.row_count -= to_process; - match skip { - true => to_skip += to_process, - false => { - if to_skip != 0 { - selectors.push(RowSelector::skip(to_skip)); - to_skip = 0; + match skip { + true => to_skip += to_process, + false => { + if to_skip != 0 { + and_then_select.push(RowSelector::skip(to_skip)); + to_skip = 0; + } + and_then_select.push(RowSelector::select(to_process)) + } } - selectors.push(RowSelector::select(to_process)) } - } - } - for v in first { - if v.row_count != 0 { - assert!( - v.skip, - "selection contains less than the number of selected rows" - ); - to_skip += v.row_count - } - } + for v in first { + if v.row_count != 0 { + assert!( + v.skip, + "selection contains less than the number of selected rows" + ); + to_skip += v.row_count + } + } - if to_skip != 0 { - selectors.push(RowSelector::skip(to_skip)); + if to_skip != 0 { + and_then_select.push(RowSelector::skip(to_skip)); + } + RowSelection::Ranges(and_then_select) + } + (_, _) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } } - - Self { selectors } } /// Compute the intersection of two [`RowSelection`] @@ -340,7 +421,16 @@ impl RowSelection { /// /// returned: NNNNNNNNYYNYN pub fn intersection(&self, other: &Self) -> Self { - intersect_row_selections(&self.selectors, &other.selectors) + match (self, other) { + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => + RowSelection::Ranges(intersect_row_selections(a, b)), + (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bit_map)) =>{ + let intersection_selectors = bit_map.values() & other_bit_map.values(); + RowSelection::BitMap(BooleanArray::from(intersection_selectors)) + }, + (RowSelection::BitMap(_), RowSelection::Ranges(_)) => todo!(), + (RowSelection::Ranges(_), RowSelection::BitMap(_)) => todo!() + } } /// Compute the union of two [`RowSelection`] @@ -350,91 +440,138 @@ impl RowSelection { /// /// returned: NYYYYYNNYYNYN pub fn union(&self, other: &Self) -> Self { - union_row_selections(&self.selectors, &other.selectors) + match (self, other) { + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => + RowSelection::Ranges(union_row_selections(a, b)), + (RowSelection::BitMap(_), _) | (_, RowSelection::BitMap(_)) => + unimplemented!("BitMap variant is not yet supported"), + } } /// Returns `true` if this [`RowSelection`] selects any rows pub fn selects_any(&self) -> bool { - self.selectors.iter().any(|x| !x.skip) + match self { + RowSelection::Ranges(selectors) => { + selectors.iter().any(|x| !x.skip) + } + RowSelection::BitMap(bitmap) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } + } } /// Trims this [`RowSelection`] removing any trailing skips pub(crate) fn trim(mut self) -> Self { - while self.selectors.last().map(|x| x.skip).unwrap_or(false) { - self.selectors.pop(); + match self { + RowSelection::Ranges(mut selectors) => { + while selectors.last().map(|x| x.skip).unwrap_or(false) { + selectors.pop(); + } + RowSelection::Ranges(selectors) + } + RowSelection::BitMap(_) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } } - self } /// Applies an offset to this [`RowSelection`], skipping the first `offset` selected rows - pub(crate) fn offset(mut self, offset: usize) -> Self { - if offset == 0 { - return self; - } - - let mut selected_count = 0; - let mut skipped_count = 0; - - // Find the index where the selector exceeds the row count - let find = self - .selectors - .iter() - .position(|selector| match selector.skip { - true => { - skipped_count += selector.row_count; - false + pub(crate) fn offset(self, offset: usize) -> Self { + match self { + RowSelection::Ranges(mut selectors) => { + if offset == 0 { + return RowSelection::Ranges(selectors) } - false => { - selected_count += selector.row_count; - selected_count > offset - } - }); - let split_idx = match find { - Some(idx) => idx, - None => { - self.selectors.clear(); - return self; - } - }; + let mut selected_count = 0; + let mut skipped_count = 0; - let mut selectors = Vec::with_capacity(self.selectors.len() - split_idx + 1); - selectors.push(RowSelector::skip(skipped_count + offset)); - selectors.push(RowSelector::select(selected_count - offset)); - selectors.extend_from_slice(&self.selectors[split_idx + 1..]); + // Find the index where the selector exceeds the row count + let find = selectors + .iter() + .position(|selector| match selector.skip { + true => { + skipped_count += selector.row_count; + false + } + false => { + selected_count += selector.row_count; + selected_count > offset + } + }); - Self { selectors } + let split_idx = match find { + Some(idx) => idx, + None => { + selectors.clear(); + return RowSelection::Ranges(selectors); + } + }; + + let mut select = Vec::with_capacity(selectors.len() - split_idx + 1); + select.push(RowSelector::skip(skipped_count + offset)); + select.push(RowSelector::select(selected_count - offset)); + select.extend_from_slice(&selectors[split_idx + 1..]); + + RowSelection::Ranges(select) + } + RowSelection::BitMap(_) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } + } } /// Limit this [`RowSelection`] to only select `limit` rows pub(crate) fn limit(mut self, mut limit: usize) -> Self { - if limit == 0 { - self.selectors.clear(); - } + match self { + RowSelection::Ranges(mut selectors) => { + if limit == 0 { + selectors.clear(); + } - for (idx, selection) in self.selectors.iter_mut().enumerate() { - if !selection.skip { - if selection.row_count >= limit { - selection.row_count = limit; - self.selectors.truncate(idx + 1); - break; - } else { - limit -= selection.row_count; + for (idx, selection) in selectors.iter_mut().enumerate() { + if !selection.skip { + if selection.row_count >= limit { + selection.row_count = limit; + selectors.truncate(idx + 1); + break; + } else { + limit -= selection.row_count; + } + } } + RowSelection::Ranges(selectors) + } + RowSelection::BitMap(_) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") } } - self } /// Returns an iterator over the [`RowSelector`]s for this /// [`RowSelection`]. pub fn iter(&self) -> impl Iterator { - self.selectors.iter() + match self { + RowSelection::Ranges(selectors) => selectors.iter(), + RowSelection::BitMap(bitmap) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } + } } /// Returns the number of selected rows pub fn row_count(&self) -> usize { - self.iter().filter(|s| !s.skip).map(|s| s.row_count).sum() + match self { + RowSelection::Ranges(selectors) => selectors.iter().map(|s| s.row_count).sum(), + RowSelection::BitMap(bitmap) => { + bitmap.true_count() + } + } } /// Returns the number of de-selected rows @@ -449,6 +586,12 @@ impl From> for RowSelection { } } +impl From for RowSelection { + fn from(value: BooleanArray) -> Self { + RowSelection::BitMap(value) + } +} + impl FromIterator for RowSelection { fn from_iter>(iter: T) -> Self { let iter = iter.into_iter(); @@ -474,20 +617,32 @@ impl FromIterator for RowSelection { selectors.push(s) } } - - Self { selectors } + + RowSelection::Ranges(selectors) } } impl From for Vec { fn from(r: RowSelection) -> Self { - r.selectors + match r { + RowSelection::Ranges(selectors) => selectors, + RowSelection::BitMap(bitmap) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } + } } } impl From for VecDeque { fn from(r: RowSelection) -> Self { - r.selectors.into() + match r { + RowSelection::Ranges(selectors) => selectors.into(), + RowSelection::BitMap(bitmap) => { + // not implemented yet + unimplemented!("BitMap variant is not yet supported") + } + } } } @@ -497,7 +652,7 @@ impl From for VecDeque { /// other: NYNNNNNNY /// /// returned: NNNNNNNNYYNYN -fn intersect_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelection { +fn intersect_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec { let mut l_iter = left.iter().copied().peekable(); let mut r_iter = right.iter().copied().peekable(); @@ -559,7 +714,7 @@ fn intersect_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowS /// returned: NYYYYYNNYYNYN /// /// This can be removed from here once RowSelection::union is in parquet::arrow -fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelection { +fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec { let mut l_iter = left.iter().copied().peekable(); let mut r_iter = right.iter().copied().peekable(); @@ -654,15 +809,17 @@ mod tests { let selection = RowSelection::from_filters(&filters[..1]); assert!(selection.selects_any()); + let expected: Vec = selection.into(); assert_eq!( - selection.selectors, + expected, vec![RowSelector::skip(3), RowSelector::select(4)] ); let selection = RowSelection::from_filters(&filters[..2]); assert!(selection.selects_any()); + let expected: Vec = selection.into(); assert_eq!( - selection.selectors, + expected, vec![ RowSelector::skip(3), RowSelector::select(6), @@ -673,8 +830,9 @@ mod tests { let selection = RowSelection::from_filters(&filters); assert!(selection.selects_any()); + let expected: Vec = selection.into(); assert_eq!( - selection.selectors, + expected, vec![ RowSelector::skip(3), RowSelector::select(6), @@ -686,7 +844,8 @@ mod tests { let selection = RowSelection::from_filters(&filters[2..3]); assert!(!selection.selects_any()); - assert_eq!(selection.selectors, vec![RowSelector::skip(4)]); + let expected: Vec = selection.into(); + assert_eq!(expected, vec![RowSelector::skip(4)]); } #[test] @@ -699,683 +858,720 @@ mod tests { ]); let split = selection.split_off(34); - assert_eq!(split.selectors, vec![RowSelector::skip(34)]); - assert_eq!( - selection.selectors, - vec![ - RowSelector::select(12), - RowSelector::skip(3), - RowSelector::select(35) - ] - ); + match split { + RowSelection::Ranges(selectors) => { + assert_eq!(selectors, vec![RowSelector::skip(34)]); + } + _ => panic!("Expected Ranges variant"), + } + match selection { + RowSelection::Ranges(ref selectors) => { + assert_eq!( + selectors, + &vec![ + RowSelector::select(12), + RowSelector::skip(3), + RowSelector::select(35) + ] + ); + } + _ => panic!("Expected Ranges variant"), + } let split = selection.split_off(5); - assert_eq!(split.selectors, vec![RowSelector::select(5)]); - assert_eq!( - selection.selectors, - vec![ - RowSelector::select(7), - RowSelector::skip(3), - RowSelector::select(35) - ] - ); + match split { + RowSelection::Ranges(selectors) => { + assert_eq!(selectors, vec![RowSelector::select(5)]); + } + _ => panic!("Expected Ranges variant"), + } + match selection { + RowSelection::Ranges(ref selectors) => { + assert_eq!( + selectors, + &vec![ + RowSelector::select(7), + RowSelector::skip(3), + RowSelector::select(35) + ] + ); + } + _ => panic!("Expected Ranges variant"), + } let split = selection.split_off(8); - assert_eq!( - split.selectors, - vec![RowSelector::select(7), RowSelector::skip(1)] - ); - assert_eq!( - selection.selectors, - vec![RowSelector::skip(2), RowSelector::select(35)] - ); + match split { + RowSelection::Ranges(selectors) => { + assert_eq!(selectors, vec![RowSelector::select(7), RowSelector::skip(1)]); + } + _ => panic!("Expected Ranges variant"), + } + match selection { + RowSelection::Ranges(ref selectors) => { + assert_eq!( + selectors, + &vec![RowSelector::skip(2), RowSelector::select(35)] + ); + } + _ => panic!("Expected Ranges variant"), + } let split = selection.split_off(200); - assert_eq!( - split.selectors, - vec![RowSelector::skip(2), RowSelector::select(35)] - ); - assert!(selection.selectors.is_empty()); - } - - #[test] - fn test_offset() { - let selection = RowSelection::from(vec![ - RowSelector::select(5), - RowSelector::skip(23), - RowSelector::select(7), - RowSelector::skip(33), - RowSelector::select(6), - ]); - - let selection = selection.offset(2); - assert_eq!( - selection.selectors, - vec![ - RowSelector::skip(2), - RowSelector::select(3), - RowSelector::skip(23), - RowSelector::select(7), - RowSelector::skip(33), - RowSelector::select(6), - ] - ); - - let selection = selection.offset(5); - assert_eq!( - selection.selectors, - vec![ - RowSelector::skip(30), - RowSelector::select(5), - RowSelector::skip(33), - RowSelector::select(6), - ] - ); - - let selection = selection.offset(3); - assert_eq!( - selection.selectors, - vec![ - RowSelector::skip(33), - RowSelector::select(2), - RowSelector::skip(33), - RowSelector::select(6), - ] - ); - - let selection = selection.offset(2); - assert_eq!( - selection.selectors, - vec![RowSelector::skip(68), RowSelector::select(6),] - ); - - let selection = selection.offset(3); - assert_eq!( - selection.selectors, - vec![RowSelector::skip(71), RowSelector::select(3),] - ); - } - - #[test] - fn test_and() { - let mut a = RowSelection::from(vec![ - RowSelector::skip(12), - RowSelector::select(23), - RowSelector::skip(3), - RowSelector::select(5), - ]); - - let b = RowSelection::from(vec![ - RowSelector::select(5), - RowSelector::skip(4), - RowSelector::select(15), - RowSelector::skip(4), - ]); - - let mut expected = RowSelection::from(vec![ - RowSelector::skip(12), - RowSelector::select(5), - RowSelector::skip(4), - RowSelector::select(14), - RowSelector::skip(3), - RowSelector::select(1), - RowSelector::skip(4), - ]); - - assert_eq!(a.and_then(&b), expected); - - a.split_off(7); - expected.split_off(7); - assert_eq!(a.and_then(&b), expected); - - let a = RowSelection::from(vec![RowSelector::select(5), RowSelector::skip(3)]); - - let b = RowSelection::from(vec![ - RowSelector::select(2), - RowSelector::skip(1), - RowSelector::select(1), - RowSelector::skip(1), - ]); - - assert_eq!( - a.and_then(&b).selectors, - vec![ - RowSelector::select(2), - RowSelector::skip(1), - RowSelector::select(1), - RowSelector::skip(4) - ] - ); - } - - #[test] - fn test_combine() { - let a = vec![ - RowSelector::skip(3), - RowSelector::skip(3), - RowSelector::select(10), - RowSelector::skip(4), - ]; - - let b = vec![ - RowSelector::skip(3), - RowSelector::skip(3), - RowSelector::select(10), - RowSelector::skip(4), - RowSelector::skip(0), - ]; - - let c = vec![ - RowSelector::skip(2), - RowSelector::skip(4), - RowSelector::select(3), - RowSelector::select(3), - RowSelector::select(4), - RowSelector::skip(3), - RowSelector::skip(1), - RowSelector::skip(0), - ]; - - let expected = RowSelection::from(vec![ - RowSelector::skip(6), - RowSelector::select(10), - RowSelector::skip(4), - ]); - - assert_eq!(RowSelection::from_iter(a), expected); - assert_eq!(RowSelection::from_iter(b), expected); - assert_eq!(RowSelection::from_iter(c), expected); - } - - #[test] - fn test_combine_2elements() { - let a = vec![RowSelector::select(10), RowSelector::select(5)]; - let a_expect = vec![RowSelector::select(15)]; - assert_eq!(RowSelection::from_iter(a).selectors, a_expect); - - let b = vec![RowSelector::select(10), RowSelector::skip(5)]; - let b_expect = vec![RowSelector::select(10), RowSelector::skip(5)]; - assert_eq!(RowSelection::from_iter(b).selectors, b_expect); - - let c = vec![RowSelector::skip(10), RowSelector::select(5)]; - let c_expect = vec![RowSelector::skip(10), RowSelector::select(5)]; - assert_eq!(RowSelection::from_iter(c).selectors, c_expect); - - let d = vec![RowSelector::skip(10), RowSelector::skip(5)]; - let d_expect = vec![RowSelector::skip(15)]; - assert_eq!(RowSelection::from_iter(d).selectors, d_expect); - } - - #[test] - fn test_from_one_and_empty() { - let a = vec![RowSelector::select(10)]; - let selection1 = RowSelection::from(a.clone()); - assert_eq!(selection1.selectors, a); - - let b = vec![]; - let selection1 = RowSelection::from(b.clone()); - assert_eq!(selection1.selectors, b) - } - - #[test] - #[should_panic(expected = "selection exceeds the number of selected rows")] - fn test_and_longer() { - let a = RowSelection::from(vec![ - RowSelector::select(3), - RowSelector::skip(33), - RowSelector::select(3), - RowSelector::skip(33), - ]); - let b = RowSelection::from(vec![RowSelector::select(36)]); - a.and_then(&b); - } - - #[test] - #[should_panic(expected = "selection contains less than the number of selected rows")] - fn test_and_shorter() { - let a = RowSelection::from(vec![ - RowSelector::select(3), - RowSelector::skip(33), - RowSelector::select(3), - RowSelector::skip(33), - ]); - let b = RowSelection::from(vec![RowSelector::select(3)]); - a.and_then(&b); - } - - #[test] - fn test_intersect_row_selection_and_combine() { - // a size equal b size - let a = vec![ - RowSelector::select(5), - RowSelector::skip(4), - RowSelector::select(1), - ]; - let b = vec![ - RowSelector::select(8), - RowSelector::skip(1), - RowSelector::select(1), - ]; - - let res = intersect_row_selections(&a, &b); - assert_eq!( - res.selectors, - vec![ - RowSelector::select(5), - RowSelector::skip(4), - RowSelector::select(1), - ], - ); - - // a size larger than b size - let a = vec![ - RowSelector::select(3), - RowSelector::skip(33), - RowSelector::select(3), - RowSelector::skip(33), - ]; - let b = vec![RowSelector::select(36), RowSelector::skip(36)]; - let res = intersect_row_selections(&a, &b); - assert_eq!( - res.selectors, - vec![RowSelector::select(3), RowSelector::skip(69)] - ); - - // a size less than b size - let a = vec![RowSelector::select(3), RowSelector::skip(7)]; - let b = vec![ - RowSelector::select(2), - RowSelector::skip(2), - RowSelector::select(2), - RowSelector::skip(2), - RowSelector::select(2), - ]; - let res = intersect_row_selections(&a, &b); - assert_eq!( - res.selectors, - vec![RowSelector::select(2), RowSelector::skip(8)] - ); - - let a = vec![RowSelector::select(3), RowSelector::skip(7)]; - let b = vec![ - RowSelector::select(2), - RowSelector::skip(2), - RowSelector::select(2), - RowSelector::skip(2), - RowSelector::select(2), - ]; - let res = intersect_row_selections(&a, &b); - assert_eq!( - res.selectors, - vec![RowSelector::select(2), RowSelector::skip(8)] - ); - } - - #[test] - fn test_and_fuzz() { - let mut rand = rng(); - for _ in 0..100 { - let a_len = rand.random_range(10..100); - let a_bools: Vec<_> = (0..a_len).map(|_| rand.random_bool(0.2)).collect(); - let a = RowSelection::from_filters(&[BooleanArray::from(a_bools.clone())]); - - let b_len: usize = a_bools.iter().map(|x| *x as usize).sum(); - let b_bools: Vec<_> = (0..b_len).map(|_| rand.random_bool(0.8)).collect(); - let b = RowSelection::from_filters(&[BooleanArray::from(b_bools.clone())]); - - let mut expected_bools = vec![false; a_len]; - - let mut iter_b = b_bools.iter(); - for (idx, b) in a_bools.iter().enumerate() { - if *b && *iter_b.next().unwrap() { - expected_bools[idx] = true; - } + match split { + RowSelection::Ranges(selectors) => { + assert_eq!( + selectors, + vec![RowSelector::skip(2), RowSelector::select(35)] + ); } - - let expected = RowSelection::from_filters(&[BooleanArray::from(expected_bools)]); - - let total_rows: usize = expected.selectors.iter().map(|s| s.row_count).sum(); - assert_eq!(a_len, total_rows); - - assert_eq!(a.and_then(&b), expected); + _ => panic!("Expected Ranges variant"), + } + match selection { + RowSelection::Ranges(ref selectors) => { + assert!(selectors.is_empty()); + } + _ => panic!("Expected Ranges variant"), } } - #[test] - fn test_iter() { - // use the iter() API to show it does what is expected and - // avoid accidental deletion - let selectors = vec![ - RowSelector::select(3), - RowSelector::skip(33), - RowSelector::select(4), - ]; - - let round_tripped = RowSelection::from(selectors.clone()) - .iter() - .cloned() - .collect::>(); - assert_eq!(selectors, round_tripped); - } - - #[test] - fn test_limit() { - // Limit to existing limit should no-op - let selection = RowSelection::from(vec![RowSelector::select(10), RowSelector::skip(90)]); - let limited = selection.limit(10); - assert_eq!(RowSelection::from(vec![RowSelector::select(10)]), limited); - - let selection = RowSelection::from(vec![ - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(10), - ]); - - let limited = selection.clone().limit(5); - let expected = vec![RowSelector::select(5)]; - assert_eq!(limited.selectors, expected); - - let limited = selection.clone().limit(15); - let expected = vec![ - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(5), - ]; - assert_eq!(limited.selectors, expected); - - let limited = selection.clone().limit(0); - let expected = vec![]; - assert_eq!(limited.selectors, expected); - - let limited = selection.clone().limit(30); - let expected = vec![ - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(10), - ]; - assert_eq!(limited.selectors, expected); - - let limited = selection.limit(100); - let expected = vec![ - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(10), - ]; - assert_eq!(limited.selectors, expected); - } - - #[test] - fn test_scan_ranges() { - let index = vec![ - PageLocation { - offset: 0, - compressed_page_size: 10, - first_row_index: 0, - }, - PageLocation { - offset: 10, - compressed_page_size: 10, - first_row_index: 10, - }, - PageLocation { - offset: 20, - compressed_page_size: 10, - first_row_index: 20, - }, - PageLocation { - offset: 30, - compressed_page_size: 10, - first_row_index: 30, - }, - PageLocation { - offset: 40, - compressed_page_size: 10, - first_row_index: 40, - }, - PageLocation { - offset: 50, - compressed_page_size: 10, - first_row_index: 50, - }, - PageLocation { - offset: 60, - compressed_page_size: 10, - first_row_index: 60, - }, - ]; - - let selection = RowSelection::from(vec![ - // Skip first page - RowSelector::skip(10), - // Multiple selects in same page - RowSelector::select(3), - RowSelector::skip(3), - RowSelector::select(4), - // Select to page boundary - RowSelector::skip(5), - RowSelector::select(5), - // Skip full page past page boundary - RowSelector::skip(12), - // Select across page boundaries - RowSelector::select(12), - // Skip final page - RowSelector::skip(12), - ]); - - let ranges = selection.scan_ranges(&index); - - // assert_eq!(mask, vec![false, true, true, false, true, true, false]); - assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60]); - - let selection = RowSelection::from(vec![ - // Skip first page - RowSelector::skip(10), - // Multiple selects in same page - RowSelector::select(3), - RowSelector::skip(3), - RowSelector::select(4), - // Select to page boundary - RowSelector::skip(5), - RowSelector::select(5), - // Skip full page past page boundary - RowSelector::skip(12), - // Select across page boundaries - RowSelector::select(12), - RowSelector::skip(1), - // Select across page boundaries including final page - RowSelector::select(8), - ]); - - let ranges = selection.scan_ranges(&index); - - // assert_eq!(mask, vec![false, true, true, false, true, true, true]); - assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); - - let selection = RowSelection::from(vec![ - // Skip first page - RowSelector::skip(10), - // Multiple selects in same page - RowSelector::select(3), - RowSelector::skip(3), - RowSelector::select(4), - // Select to page boundary - RowSelector::skip(5), - RowSelector::select(5), - // Skip full page past page boundary - RowSelector::skip(12), - // Select to final page boundary - RowSelector::select(12), - RowSelector::skip(1), - // Skip across final page boundary - RowSelector::skip(8), - // Select from final page - RowSelector::select(4), - ]); - - let ranges = selection.scan_ranges(&index); - - // assert_eq!(mask, vec![false, true, true, false, true, true, true]); - assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); - - let selection = RowSelection::from(vec![ - // Skip first page - RowSelector::skip(10), - // Multiple selects in same page - RowSelector::select(3), - RowSelector::skip(3), - RowSelector::select(4), - // Select to remaining in page and first row of next page - RowSelector::skip(5), - RowSelector::select(6), - // Skip remaining - RowSelector::skip(50), - ]); - - let ranges = selection.scan_ranges(&index); - - // assert_eq!(mask, vec![false, true, true, false, true, true, true]); - assert_eq!(ranges, vec![10..20, 20..30, 30..40]); - } - - #[test] - fn test_from_ranges() { - let ranges = [1..3, 4..6, 6..6, 8..8, 9..10]; - let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), 10); - assert_eq!( - selection.selectors, - vec![ - RowSelector::skip(1), - RowSelector::select(2), - RowSelector::skip(1), - RowSelector::select(2), - RowSelector::skip(3), - RowSelector::select(1) - ] - ); - - let out_of_order_ranges = [1..3, 8..10, 4..7]; - let result = std::panic::catch_unwind(|| { - RowSelection::from_consecutive_ranges(out_of_order_ranges.into_iter(), 10) - }); - assert!(result.is_err()); - } - - #[test] - fn test_empty_selector() { - let selection = RowSelection::from(vec![ - RowSelector::skip(0), - RowSelector::select(2), - RowSelector::skip(0), - RowSelector::select(2), - ]); - assert_eq!(selection.selectors, vec![RowSelector::select(4)]); - - let selection = RowSelection::from(vec![ - RowSelector::select(0), - RowSelector::skip(2), - RowSelector::select(0), - RowSelector::skip(2), - ]); - assert_eq!(selection.selectors, vec![RowSelector::skip(4)]); - } - - #[test] - fn test_intersection() { - let selection = RowSelection::from(vec![RowSelector::select(1048576)]); - let result = selection.intersection(&selection); - assert_eq!(result, selection); - - let a = RowSelection::from(vec![ - RowSelector::skip(10), - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(20), - ]); - - let b = RowSelection::from(vec![ - RowSelector::skip(20), - RowSelector::select(20), - RowSelector::skip(10), - ]); - - let result = a.intersection(&b); - assert_eq!( - result.selectors, - vec![ - RowSelector::skip(30), - RowSelector::select(10), - RowSelector::skip(10) - ] - ); - } - - #[test] - fn test_union() { - let selection = RowSelection::from(vec![RowSelector::select(1048576)]); - let result = selection.union(&selection); - assert_eq!(result, selection); - - // NYNYY - let a = RowSelection::from(vec![ - RowSelector::skip(10), - RowSelector::select(10), - RowSelector::skip(10), - RowSelector::select(20), - ]); - - // NNYYNYN - let b = RowSelection::from(vec![ - RowSelector::skip(20), - RowSelector::select(20), - RowSelector::skip(10), - RowSelector::select(10), - RowSelector::skip(10), - ]); - - let result = a.union(&b); - - // NYYYYYN - assert_eq!( - result.iter().collect::>(), - vec![ - &RowSelector::skip(10), - &RowSelector::select(50), - &RowSelector::skip(10), - ] - ); - } - - #[test] - fn test_row_count() { - let selection = RowSelection::from(vec![ - RowSelector::skip(34), - RowSelector::select(12), - RowSelector::skip(3), - RowSelector::select(35), - ]); - - assert_eq!(selection.row_count(), 12 + 35); - assert_eq!(selection.skipped_row_count(), 34 + 3); - - let selection = RowSelection::from(vec![RowSelector::select(12), RowSelector::select(35)]); - - assert_eq!(selection.row_count(), 12 + 35); - assert_eq!(selection.skipped_row_count(), 0); - - let selection = RowSelection::from(vec![RowSelector::skip(34), RowSelector::skip(3)]); - - assert_eq!(selection.row_count(), 0); - assert_eq!(selection.skipped_row_count(), 34 + 3); - - let selection = RowSelection::from(vec![]); - - assert_eq!(selection.row_count(), 0); - assert_eq!(selection.skipped_row_count(), 0); - } +// #[test] +// fn test_offset() { +// let selection = RowSelection::from(vec![ +// RowSelector::select(5), +// RowSelector::skip(23), +// RowSelector::select(7), +// RowSelector::skip(33), +// RowSelector::select(6), +// ]); +// +// let selection = selection.offset(2); +// assert_eq!( +// selection.selectors, +// vec![ +// RowSelector::skip(2), +// RowSelector::select(3), +// RowSelector::skip(23), +// RowSelector::select(7), +// RowSelector::skip(33), +// RowSelector::select(6), +// ] +// ); +// +// let selection = selection.offset(5); +// assert_eq!( +// selection.selectors, +// vec![ +// RowSelector::skip(30), +// RowSelector::select(5), +// RowSelector::skip(33), +// RowSelector::select(6), +// ] +// ); +// +// let selection = selection.offset(3); +// assert_eq!( +// selection.selectors, +// vec![ +// RowSelector::skip(33), +// RowSelector::select(2), +// RowSelector::skip(33), +// RowSelector::select(6), +// ] +// ); +// +// let selection = selection.offset(2); +// assert_eq!( +// selection.selectors, +// vec![RowSelector::skip(68), RowSelector::select(6),] +// ); +// +// let selection = selection.offset(3); +// assert_eq!( +// selection.selectors, +// vec![RowSelector::skip(71), RowSelector::select(3),] +// ); +// } +// +// #[test] +// fn test_and() { +// let mut a = RowSelection::from(vec![ +// RowSelector::skip(12), +// RowSelector::select(23), +// RowSelector::skip(3), +// RowSelector::select(5), +// ]); +// +// let b = RowSelection::from(vec![ +// RowSelector::select(5), +// RowSelector::skip(4), +// RowSelector::select(15), +// RowSelector::skip(4), +// ]); +// +// let mut expected = RowSelection::from(vec![ +// RowSelector::skip(12), +// RowSelector::select(5), +// RowSelector::skip(4), +// RowSelector::select(14), +// RowSelector::skip(3), +// RowSelector::select(1), +// RowSelector::skip(4), +// ]); +// +// assert_eq!(a.and_then(&b), expected); +// +// a.split_off(7); +// expected.split_off(7); +// assert_eq!(a.and_then(&b), expected); +// +// let a = RowSelection::from(vec![RowSelector::select(5), RowSelector::skip(3)]); +// +// let b = RowSelection::from(vec![ +// RowSelector::select(2), +// RowSelector::skip(1), +// RowSelector::select(1), +// RowSelector::skip(1), +// ]); +// +// assert_eq!( +// a.and_then(&b).selectors, +// vec![ +// RowSelector::select(2), +// RowSelector::skip(1), +// RowSelector::select(1), +// RowSelector::skip(4) +// ] +// ); +// } +// +// #[test] +// fn test_combine() { +// let a = vec![ +// RowSelector::skip(3), +// RowSelector::skip(3), +// RowSelector::select(10), +// RowSelector::skip(4), +// ]; +// +// let b = vec![ +// RowSelector::skip(3), +// RowSelector::skip(3), +// RowSelector::select(10), +// RowSelector::skip(4), +// RowSelector::skip(0), +// ]; +// +// let c = vec![ +// RowSelector::skip(2), +// RowSelector::skip(4), +// RowSelector::select(3), +// RowSelector::select(3), +// RowSelector::select(4), +// RowSelector::skip(3), +// RowSelector::skip(1), +// RowSelector::skip(0), +// ]; +// +// let expected = RowSelection::from(vec![ +// RowSelector::skip(6), +// RowSelector::select(10), +// RowSelector::skip(4), +// ]); +// +// assert_eq!(RowSelection::from_iter(a), expected); +// assert_eq!(RowSelection::from_iter(b), expected); +// assert_eq!(RowSelection::from_iter(c), expected); +// } +// +// #[test] +// fn test_combine_2elements() { +// let a = vec![RowSelector::select(10), RowSelector::select(5)]; +// let a_expect = vec![RowSelector::select(15)]; +// assert_eq!(RowSelection::from_iter(a).selectors, a_expect); +// +// let b = vec![RowSelector::select(10), RowSelector::skip(5)]; +// let b_expect = vec![RowSelector::select(10), RowSelector::skip(5)]; +// assert_eq!(RowSelection::from_iter(b).selectors, b_expect); +// +// let c = vec![RowSelector::skip(10), RowSelector::select(5)]; +// let c_expect = vec![RowSelector::skip(10), RowSelector::select(5)]; +// assert_eq!(RowSelection::from_iter(c).selectors, c_expect); +// +// let d = vec![RowSelector::skip(10), RowSelector::skip(5)]; +// let d_expect = vec![RowSelector::skip(15)]; +// assert_eq!(RowSelection::from_iter(d).selectors, d_expect); +// } +// +// #[test] +// fn test_from_one_and_empty() { +// let a = vec![RowSelector::select(10)]; +// let selection1 = RowSelection::from(a.clone()); +// assert_eq!(selection1.selectors, a); +// +// let b = vec![]; +// let selection1 = RowSelection::from(b.clone()); +// assert_eq!(selection1.selectors, b) +// } +// +// #[test] +// #[should_panic(expected = "selection exceeds the number of selected rows")] +// fn test_and_longer() { +// let a = RowSelection::from(vec![ +// RowSelector::select(3), +// RowSelector::skip(33), +// RowSelector::select(3), +// RowSelector::skip(33), +// ]); +// let b = RowSelection::from(vec![RowSelector::select(36)]); +// a.and_then(&b); +// } +// +// #[test] +// #[should_panic(expected = "selection contains less than the number of selected rows")] +// fn test_and_shorter() { +// let a = RowSelection::from(vec![ +// RowSelector::select(3), +// RowSelector::skip(33), +// RowSelector::select(3), +// RowSelector::skip(33), +// ]); +// let b = RowSelection::from(vec![RowSelector::select(3)]); +// a.and_then(&b); +// } +// +// #[test] +// fn test_intersect_row_selection_and_combine() { +// // a size equal b size +// let a = vec![ +// RowSelector::select(5), +// RowSelector::skip(4), +// RowSelector::select(1), +// ]; +// let b = vec![ +// RowSelector::select(8), +// RowSelector::skip(1), +// RowSelector::select(1), +// ]; +// +// let res = intersect_row_selections(&a, &b); +// assert_eq!( +// res.selectors, +// vec![ +// RowSelector::select(5), +// RowSelector::skip(4), +// RowSelector::select(1), +// ], +// ); +// +// // a size larger than b size +// let a = vec![ +// RowSelector::select(3), +// RowSelector::skip(33), +// RowSelector::select(3), +// RowSelector::skip(33), +// ]; +// let b = vec![RowSelector::select(36), RowSelector::skip(36)]; +// let res = intersect_row_selections(&a, &b); +// assert_eq!( +// res.selectors, +// vec![RowSelector::select(3), RowSelector::skip(69)] +// ); +// +// // a size less than b size +// let a = vec![RowSelector::select(3), RowSelector::skip(7)]; +// let b = vec![ +// RowSelector::select(2), +// RowSelector::skip(2), +// RowSelector::select(2), +// RowSelector::skip(2), +// RowSelector::select(2), +// ]; +// let res = intersect_row_selections(&a, &b); +// assert_eq!( +// res.selectors, +// vec![RowSelector::select(2), RowSelector::skip(8)] +// ); +// +// let a = vec![RowSelector::select(3), RowSelector::skip(7)]; +// let b = vec![ +// RowSelector::select(2), +// RowSelector::skip(2), +// RowSelector::select(2), +// RowSelector::skip(2), +// RowSelector::select(2), +// ]; +// let res = intersect_row_selections(&a, &b); +// assert_eq!( +// res.selectors, +// vec![RowSelector::select(2), RowSelector::skip(8)] +// ); +// } +// +// #[test] +// fn test_and_fuzz() { +// let mut rand = rng(); +// for _ in 0..100 { +// let a_len = rand.random_range(10..100); +// let a_bools: Vec<_> = (0..a_len).map(|_| rand.random_bool(0.2)).collect(); +// let a = RowSelection::from_filters(&[BooleanArray::from(a_bools.clone())]); +// +// let b_len: usize = a_bools.iter().map(|x| *x as usize).sum(); +// let b_bools: Vec<_> = (0..b_len).map(|_| rand.random_bool(0.8)).collect(); +// let b = RowSelection::from_filters(&[BooleanArray::from(b_bools.clone())]); +// +// let mut expected_bools = vec![false; a_len]; +// +// let mut iter_b = b_bools.iter(); +// for (idx, b) in a_bools.iter().enumerate() { +// if *b && *iter_b.next().unwrap() { +// expected_bools[idx] = true; +// } +// } +// +// let expected = RowSelection::from_filters(&[BooleanArray::from(expected_bools)]); +// +// let total_rows: usize = expected.selectors.iter().map(|s| s.row_count).sum(); +// assert_eq!(a_len, total_rows); +// +// assert_eq!(a.and_then(&b), expected); +// } +// } +// +// #[test] +// fn test_iter() { +// // use the iter() API to show it does what is expected and +// // avoid accidental deletion +// let selectors = vec![ +// RowSelector::select(3), +// RowSelector::skip(33), +// RowSelector::select(4), +// ]; +// +// let round_tripped = RowSelection::from(selectors.clone()) +// .iter() +// .cloned() +// .collect::>(); +// assert_eq!(selectors, round_tripped); +// } +// +// #[test] +// fn test_limit() { +// // Limit to existing limit should no-op +// let selection = RowSelection::from(vec![RowSelector::select(10), RowSelector::skip(90)]); +// let limited = selection.limit(10); +// assert_eq!(RowSelection::from(vec![RowSelector::select(10)]), limited); +// +// let selection = RowSelection::from(vec![ +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(10), +// ]); +// +// let limited = selection.clone().limit(5); +// let expected = vec![RowSelector::select(5)]; +// assert_eq!(limited.selectors, expected); +// +// let limited = selection.clone().limit(15); +// let expected = vec![ +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(5), +// ]; +// assert_eq!(limited.selectors, expected); +// +// let limited = selection.clone().limit(0); +// let expected = vec![]; +// assert_eq!(limited.selectors, expected); +// +// let limited = selection.clone().limit(30); +// let expected = vec![ +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(10), +// ]; +// assert_eq!(limited.selectors, expected); +// +// let limited = selection.limit(100); +// let expected = vec![ +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(10), +// ]; +// assert_eq!(limited.selectors, expected); +// } +// +// #[test] +// fn test_scan_ranges() { +// let index = vec![ +// PageLocation { +// offset: 0, +// compressed_page_size: 10, +// first_row_index: 0, +// }, +// PageLocation { +// offset: 10, +// compressed_page_size: 10, +// first_row_index: 10, +// }, +// PageLocation { +// offset: 20, +// compressed_page_size: 10, +// first_row_index: 20, +// }, +// PageLocation { +// offset: 30, +// compressed_page_size: 10, +// first_row_index: 30, +// }, +// PageLocation { +// offset: 40, +// compressed_page_size: 10, +// first_row_index: 40, +// }, +// PageLocation { +// offset: 50, +// compressed_page_size: 10, +// first_row_index: 50, +// }, +// PageLocation { +// offset: 60, +// compressed_page_size: 10, +// first_row_index: 60, +// }, +// ]; +// +// let selection = RowSelection::from(vec![ +// // Skip first page +// RowSelector::skip(10), +// // Multiple selects in same page +// RowSelector::select(3), +// RowSelector::skip(3), +// RowSelector::select(4), +// // Select to page boundary +// RowSelector::skip(5), +// RowSelector::select(5), +// // Skip full page past page boundary +// RowSelector::skip(12), +// // Select across page boundaries +// RowSelector::select(12), +// // Skip final page +// RowSelector::skip(12), +// ]); +// +// let ranges = selection.scan_ranges(&index); +// +// // assert_eq!(mask, vec![false, true, true, false, true, true, false]); +// assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60]); +// +// let selection = RowSelection::from(vec![ +// // Skip first page +// RowSelector::skip(10), +// // Multiple selects in same page +// RowSelector::select(3), +// RowSelector::skip(3), +// RowSelector::select(4), +// // Select to page boundary +// RowSelector::skip(5), +// RowSelector::select(5), +// // Skip full page past page boundary +// RowSelector::skip(12), +// // Select across page boundaries +// RowSelector::select(12), +// RowSelector::skip(1), +// // Select across page boundaries including final page +// RowSelector::select(8), +// ]); +// +// let ranges = selection.scan_ranges(&index); +// +// // assert_eq!(mask, vec![false, true, true, false, true, true, true]); +// assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); +// +// let selection = RowSelection::from(vec![ +// // Skip first page +// RowSelector::skip(10), +// // Multiple selects in same page +// RowSelector::select(3), +// RowSelector::skip(3), +// RowSelector::select(4), +// // Select to page boundary +// RowSelector::skip(5), +// RowSelector::select(5), +// // Skip full page past page boundary +// RowSelector::skip(12), +// // Select to final page boundary +// RowSelector::select(12), +// RowSelector::skip(1), +// // Skip across final page boundary +// RowSelector::skip(8), +// // Select from final page +// RowSelector::select(4), +// ]); +// +// let ranges = selection.scan_ranges(&index); +// +// // assert_eq!(mask, vec![false, true, true, false, true, true, true]); +// assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); +// +// let selection = RowSelection::from(vec![ +// // Skip first page +// RowSelector::skip(10), +// // Multiple selects in same page +// RowSelector::select(3), +// RowSelector::skip(3), +// RowSelector::select(4), +// // Select to remaining in page and first row of next page +// RowSelector::skip(5), +// RowSelector::select(6), +// // Skip remaining +// RowSelector::skip(50), +// ]); +// +// let ranges = selection.scan_ranges(&index); +// +// // assert_eq!(mask, vec![false, true, true, false, true, true, true]); +// assert_eq!(ranges, vec![10..20, 20..30, 30..40]); +// } +// +// #[test] +// fn test_from_ranges() { +// let ranges = [1..3, 4..6, 6..6, 8..8, 9..10]; +// let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), 10); +// assert_eq!( +// selection.selectors, +// vec![ +// RowSelector::skip(1), +// RowSelector::select(2), +// RowSelector::skip(1), +// RowSelector::select(2), +// RowSelector::skip(3), +// RowSelector::select(1) +// ] +// ); +// +// let out_of_order_ranges = [1..3, 8..10, 4..7]; +// let result = std::panic::catch_unwind(|| { +// RowSelection::from_consecutive_ranges(out_of_order_ranges.into_iter(), 10) +// }); +// assert!(result.is_err()); +// } +// +// #[test] +// fn test_empty_selector() { +// let selection = RowSelection::from(vec![ +// RowSelector::skip(0), +// RowSelector::select(2), +// RowSelector::skip(0), +// RowSelector::select(2), +// ]); +// assert_eq!(selection.selectors, vec![RowSelector::select(4)]); +// +// let selection = RowSelection::from(vec![ +// RowSelector::select(0), +// RowSelector::skip(2), +// RowSelector::select(0), +// RowSelector::skip(2), +// ]); +// assert_eq!(selection.selectors, vec![RowSelector::skip(4)]); +// } +// +// #[test] +// fn test_intersection() { +// let selection = RowSelection::from(vec![RowSelector::select(1048576)]); +// let result = selection.intersection(&selection); +// assert_eq!(result, selection); +// +// let a = RowSelection::from(vec![ +// RowSelector::skip(10), +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(20), +// ]); +// +// let b = RowSelection::from(vec![ +// RowSelector::skip(20), +// RowSelector::select(20), +// RowSelector::skip(10), +// ]); +// +// let result = a.intersection(&b); +// assert_eq!( +// result.selectors, +// vec![ +// RowSelector::skip(30), +// RowSelector::select(10), +// RowSelector::skip(10) +// ] +// ); +// } +// +// #[test] +// fn test_union() { +// let selection = RowSelection::from(vec![RowSelector::select(1048576)]); +// let result = selection.union(&selection); +// assert_eq!(result, selection); +// +// // NYNYY +// let a = RowSelection::from(vec![ +// RowSelector::skip(10), +// RowSelector::select(10), +// RowSelector::skip(10), +// RowSelector::select(20), +// ]); +// +// // NNYYNYN +// let b = RowSelection::from(vec![ +// RowSelector::skip(20), +// RowSelector::select(20), +// RowSelector::skip(10), +// RowSelector::select(10), +// RowSelector::skip(10), +// ]); +// +// let result = a.union(&b); +// +// // NYYYYYN +// assert_eq!( +// result.iter().collect::>(), +// vec![ +// &RowSelector::skip(10), +// &RowSelector::select(50), +// &RowSelector::skip(10), +// ] +// ); +// } +// +// #[test] +// fn test_row_count() { +// let selection = RowSelection::from(vec![ +// RowSelector::skip(34), +// RowSelector::select(12), +// RowSelector::skip(3), +// RowSelector::select(35), +// ]); +// +// assert_eq!(selection.row_count(), 12 + 35); +// assert_eq!(selection.skipped_row_count(), 34 + 3); +// +// let selection = RowSelection::from(vec![RowSelector::select(12), RowSelector::select(35)]); +// +// assert_eq!(selection.row_count(), 12 + 35); +// assert_eq!(selection.skipped_row_count(), 0); +// +// let selection = RowSelection::from(vec![RowSelector::skip(34), RowSelector::skip(3)]); +// +// assert_eq!(selection.row_count(), 0); +// assert_eq!(selection.skipped_row_count(), 34 + 3); +// +// let selection = RowSelection::from(vec![]); +// +// assert_eq!(selection.row_count(), 0); +// assert_eq!(selection.skipped_row_count(), 0); +// } } From 1e9b6e5f081beaaaaf1ec1b5d67ee511d38ad0e4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 27 Apr 2025 14:18:43 +0800 Subject: [PATCH 52/73] Change to use filter --- parquet/src/arrow/arrow_reader/mod.rs | 61 +++++++++++++++++---- parquet/src/arrow/arrow_reader/selection.rs | 17 ++++-- 2 files changed, 61 insertions(+), 17 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 068d60678bf9..1aebf4865f45 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -18,15 +18,16 @@ //! Contains reader which reads parquet data into arrow [`RecordBatch`] use arrow_array::cast::AsArray; -use arrow_array::Array; +use arrow_array::{Array, BooleanArray}; use arrow_array::{RecordBatch, RecordBatchReader}; use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; use arrow_select::filter::prep_null_mask_filter; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::collections::VecDeque; +use std::mem::take; use std::sync::Arc; - +use arrow_select::take::take_record_batch; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{build_array_reader, ArrayReader}; use crate::arrow::schema::{parquet_to_arrow_schema_and_fields, ParquetField}; @@ -780,7 +781,7 @@ pub struct ParquetRecordBatchReader { batch_size: usize, array_reader: Box, schema: SchemaRef, - selection: Option>, + selection: Option, } impl Iterator for ParquetRecordBatchReader { @@ -788,10 +789,12 @@ impl Iterator for ParquetRecordBatchReader { fn next(&mut self) -> Option { let mut read_records = 0; + let mut filter: Option = None; match self.selection.as_mut() { - Some(selection) => { - while read_records < self.batch_size && !selection.is_empty() { - let front = selection.pop_front().unwrap(); + Some(RowSelection::Ranges(selectors)) => { + let mut selectors: VecDeque = VecDeque::from_iter(take(selectors)); + while read_records < self.batch_size && !selectors.is_empty() { + let front = selectors.pop_front().unwrap(); if front.skip { let skipped = match self.array_reader.skip_records(front.row_count) { Ok(skipped) => skipped, @@ -821,7 +824,7 @@ impl Iterator for 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)); + selectors.push_front(RowSelector::select(remaining)); need_read } _ => front.row_count, @@ -832,7 +835,14 @@ impl Iterator for ParquetRecordBatchReader { Err(error) => return Some(Err(error.into())), } } + self.selection = Some(RowSelection::Ranges(selectors.into())); + } + Some(RowSelection::BitMap(bitmap)) => { + self.array_reader.read_records(bitmap.len()).unwrap(); + filter = Some(bitmap.clone()); + self.selection = None; } + None => { if let Err(error) = self.array_reader.read_records(self.batch_size) { return Some(Err(error.into())); @@ -849,10 +859,37 @@ impl Iterator for ParquetRecordBatchReader { ) }); - match struct_array { - Err(err) => Some(Err(err)), - Ok(e) => (e.len() > 0).then(|| Ok(RecordBatch::from(e))), + + let batch:RecordBatch = match struct_array { + Err(err) => return Some(Err(err)), + Ok(e) => e.into(), + }; + + if let Some(filter) = filter.as_mut() { + if batch.num_rows() == 0 { + return None + } + if filter.len() != batch.num_rows() { + return Some(Err(ArrowError::ComputeError(format!( + "Filter length ({}) does not match batch rows ({})", + filter.len(), + batch.num_rows() + )))); + } + + match arrow_select::filter::filter_record_batch(&batch, filter) { + Ok(filtered_batch) => Some(Ok(filtered_batch)), + Err(e) => Some(Err(e)), + } + + } else { + if batch.num_rows() > 0 { + Some(Ok(batch)) + } else { + None + } } + } } } @@ -895,7 +932,7 @@ impl ParquetRecordBatchReader { batch_size, array_reader, schema: Arc::new(Schema::new(levels.fields.clone())), - selection: selection.map(|s| s.trim().into()), + selection: selection.map(|s| s.trim()), }) } @@ -916,7 +953,7 @@ impl ParquetRecordBatchReader { batch_size, array_reader, schema: Arc::new(schema), - selection: selection.map(|s| s.trim().into()), + selection: selection.map(|s| s.trim()), } } } diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 0784c14040f4..482a4783fdeb 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -455,8 +455,7 @@ impl RowSelection { selectors.iter().any(|x| !x.skip) } RowSelection::BitMap(bitmap) => { - // not implemented yet - unimplemented!("BitMap variant is not yet supported") + bitmap.true_count() > 0 } } } @@ -470,9 +469,17 @@ impl RowSelection { } RowSelection::Ranges(selectors) } - RowSelection::BitMap(_) => { - // not implemented yet - unimplemented!("BitMap variant is not yet supported") + RowSelection::BitMap(bitmap) => { + // find the last `true` in the mask + let new_len = bitmap + .iter() + .rposition(|opt| opt == Some(true)) + .map(|idx| idx + 1) + .unwrap_or(0); + + // slice off any trailing `false` (or null) values + let trimmed = bitmap.slice(0, new_len); + RowSelection::BitMap(trimmed) } } } From 21dadbef697039a9adc0e022b7d3b100ada75281 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 27 Apr 2025 16:26:36 +0800 Subject: [PATCH 53/73] Fix then --- 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 482a4783fdeb..9cdabdedfae3 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -349,7 +349,7 @@ impl RowSelection { (RowSelection::Ranges(selectors), RowSelection::Ranges(other_selectors)) => { let mut and_then_select = vec![]; let mut first = selectors.iter().cloned().peekable(); - let mut second = selectors.iter().cloned().peekable(); + let mut second = other_selectors.iter().cloned().peekable(); let mut to_skip = 0; while let Some(b) = second.peek_mut() { From 3fe4cefd89207cb9d470ecc60eab69df75f65a56 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 27 Apr 2025 20:48:24 +0800 Subject: [PATCH 54/73] Adaptive push down --- parquet/src/arrow/arrow_reader/mod.rs | 7 +++--- parquet/src/arrow/arrow_reader/selection.rs | 1 + parquet/src/arrow/async_reader/mod.rs | 25 ++++++++++++++++++++- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 1aebf4865f45..deed660a4cc7 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -678,7 +678,7 @@ impl ParquetRecordBatchReaderBuilder { let array_reader = build_array_reader(self.fields.as_deref(), predicate.projection(), &reader)?; - + selection = Some(evaluate_predicate( batch_size, array_reader, @@ -1027,15 +1027,14 @@ pub(crate) fn evaluate_predicate( filter.len() )); } + match filter.null_count() { 0 => filters.push(filter), _ => filters.push(prep_null_mask_filter(&filter)), }; } - // let raw = RowSelection::from_filters(&filters); - // Testing using - let raw = RowSelection::from_filters_as_bitmap(&filters); + let raw = RowSelection::from_filters(&filters); Ok(match input_selection { Some(selection) => selection.and_then(&raw), None => raw, diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 9cdabdedfae3..c088130f89dd 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -23,6 +23,7 @@ use std::ops::Range; use arrow_buffer::{BooleanBufferBuilder, MutableBuffer}; use arrow_schema::ArrowError::NotYetImplemented; use crate::arrow::arrow_reader::boolean_selector::BooleanRowSelection; +use crate::schema::printer; /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when /// scanning a parquet file diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 45df68821ca8..9244212d221f 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -645,6 +645,29 @@ where row_group .fetch(&mut self.input, &projection, selection.as_ref()) .await?; + + + + let selection = match selection { + Some(RowSelection::Ranges(selectors)) => { + if rows_after / selectors.len() > 100 { + Some(RowSelection::Ranges(selectors)) + } else { + let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); + + for selector in selectors.iter() { + if selector.skip { + builder.append_n(selector.row_count, false); + } else { + builder.append_n(selector.row_count, true); + } + } + Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) + } + } + _ => None, + }; + let reader = ParquetRecordBatchReader::new( batch_size, @@ -829,7 +852,7 @@ where let row_count = self.metadata.row_group(row_group_idx).num_rows() as usize; let selection = self.selection.as_mut().map(|s| s.split_off(row_count)); - + let fut = reader .read_row_group( row_group_idx, From e5aad7c3c50686dfcfb7d6cb6bbc676c4554c592 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 27 Apr 2025 21:16:29 +0800 Subject: [PATCH 55/73] Fix --- parquet/src/arrow/arrow_reader/selection.rs | 9 ++++++++- parquet/src/arrow/async_reader/mod.rs | 16 ++++++++++------ 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index c088130f89dd..4840a89f0ab2 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -571,11 +571,18 @@ impl RowSelection { } } } + + pub fn total_rows(&self) -> usize { + match self { + RowSelection::Ranges(selectors) => selectors.iter().map(|s| s.row_count).sum(), + RowSelection::BitMap(bitmap) => bitmap.len(), + } + } /// Returns the number of selected rows pub fn row_count(&self) -> usize { match self { - RowSelection::Ranges(selectors) => selectors.iter().map(|s| s.row_count).sum(), + RowSelection::Ranges(selectors) => selectors.iter().filter(|s| !s.skip).map(|s| s.row_count).sum(), RowSelection::BitMap(bitmap) => { bitmap.true_count() } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 9244212d221f..6b6e3e13d4b3 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -645,16 +645,20 @@ where row_group .fetch(&mut self.input, &projection, selection.as_ref()) .await?; - - + + + let total_rows = selection + .as_ref() + .map(|s| s.total_rows()) + .unwrap_or(row_group.row_count); let selection = match selection { Some(RowSelection::Ranges(selectors)) => { - if rows_after / selectors.len() > 100 { + if total_rows / selectors.len() > 200 { Some(RowSelection::Ranges(selectors)) } else { let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); - + for selector in selectors.iter() { if selector.skip { builder.append_n(selector.row_count, false); @@ -662,7 +666,7 @@ where builder.append_n(selector.row_count, true); } } - Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) + Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) } } _ => None, @@ -852,7 +856,7 @@ where let row_count = self.metadata.row_group(row_group_idx).num_rows() as usize; let selection = self.selection.as_mut().map(|s| s.split_off(row_count)); - + let fut = reader .read_row_group( row_group_idx, From 6432de2a2dcba8145e596c5f0ec638622b3798c0 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 1 May 2025 13:49:39 +0800 Subject: [PATCH 56/73] Init: combine page cache with unified select --- .../src/arrow/async_reader/arrow_reader.rs | 36 ++++++++++--- parquet/src/arrow/async_reader/mod.rs | 50 +++++++++---------- 2 files changed, 53 insertions(+), 33 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 92e585756d49..f07fe7785437 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -41,7 +41,7 @@ pub struct FilteredParquetRecordBatchReader { array_reader: Box, predicate_readers: Vec>, schema: SchemaRef, - selection: VecDeque, + selection: Option, row_filter: Option, } @@ -64,12 +64,21 @@ fn read_selection( /// Take the next selection from the selection queue, and return the selection /// whose selected row count is to_select or less (if input selection is exhausted). fn take_next_selection( - selection: &mut VecDeque, + selection: &mut Option, to_select: usize, ) -> Option { let mut current_selected = 0; let mut rt = Vec::new(); - while let Some(front) = selection.pop_front() { + + let mut queue: VecDeque = match selection.take() { + Some(RowSelection::Ranges(selectors)) => selectors.into(), + Some(RowSelection::BitMap(_)) => { + unimplemented!("BitMap variant is not yet supported") + } + None => return None, + }; + + while let Some(front) = queue.pop_front() { if front.skip { rt.push(front); continue; @@ -82,11 +91,22 @@ fn take_next_selection( let select = to_select - current_selected; let remaining = front.row_count - select; rt.push(RowSelector::select(select)); - selection.push_front(RowSelector::select(remaining)); - + queue.push_front(RowSelector::select(remaining)); + *selection = if queue.is_empty() { + None + } else { + Some(queue.into_iter().collect()) + }; return Some(rt.into()); } } + + *selection = if queue.is_empty() { + None + } else { + Some(queue.into_iter().collect()) + }; + if !rt.is_empty() { return Some(rt.into()); } @@ -97,7 +117,7 @@ impl FilteredParquetRecordBatchReader { pub(crate) fn new( batch_size: usize, array_reader: Box, - selection: RowSelection, + selection: Option, filter_readers: Vec>, row_filter: Option, ) -> Self { @@ -111,7 +131,7 @@ impl FilteredParquetRecordBatchReader { array_reader, predicate_readers: filter_readers, schema: Arc::new(schema), - selection: selection.into(), + selection, row_filter, } } @@ -186,6 +206,8 @@ impl Iterator for FilteredParquetRecordBatchReader { Ok(selection) => selection, Err(e) => return Some(Err(e)), }; + + // println!("Filtered selection: {:?}", filtered_selection); for selector in filtered_selection.iter() { if selector.skip { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index adf3fed2e73f..6fa46af7ca24 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -658,32 +658,30 @@ where row_group .fetch(&mut self.input, &projection, Some(&selection)) .await?; - - let total_rows = selection - .as_ref() - .map(|s| s.total_rows()) - .unwrap_or(row_group.row_count); - - let selection = match selection { - Some(RowSelection::Ranges(selectors)) => { - if total_rows / selectors.len() > 200 { - Some(RowSelection::Ranges(selectors)) - } else { - let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); - - for selector in selectors.iter() { - if selector.skip { - builder.append_n(selector.row_count, false); - } else { - builder.append_n(selector.row_count, true); - } - } - Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) - } - } - _ => None, - }; + + + // let total_rows = selection.total_rows(); + // + // let selection = match selection { + // RowSelection::Ranges(selectors) => { + // if total_rows / selectors.len() > 200 { + // Some(RowSelection::Ranges(selectors)) + // } else { + // let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); + // + // for selector in selectors.iter() { + // if selector.skip { + // builder.append_n(selector.row_count, false); + // } else { + // builder.append_n(selector.row_count, true); + // } + // } + // Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) + // } + // } + // _ => None, + // }; @@ -691,7 +689,7 @@ where let reader = FilteredParquetRecordBatchReader::new( batch_size, array_reader, - selection, + Some(selection), filter_readers, self.filter.take(), ); From d26de886685a8fc658b84d7f4e73b87243df5037 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 5 May 2025 23:03:19 +0800 Subject: [PATCH 57/73] Perf: make the cache not missing to avoid some clickbench regression --- .../src/arrow/async_reader/arrow_reader.rs | 66 ++++++++++++------- 1 file changed, 41 insertions(+), 25 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 2c734d79ede1..2c199acd349b 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -241,7 +241,7 @@ impl RecordBatchReader for FilteredParquetRecordBatchReader { struct CachedPage { dict: Option<(usize, Page)>, // page offset -> page - data: Option<(usize, Page)>, // page offset -> page + data: VecDeque<(usize, Page)>, // page offset -> page, use 2 pages, because the batch size will exceed the page size sometimes } struct PredicatePageCacheInner { @@ -251,43 +251,59 @@ struct PredicatePageCacheInner { impl PredicatePageCacheInner { pub(crate) fn get_page(&self, col_id: usize, offset: usize) -> Option { self.pages.get(&col_id).and_then(|pages| { - pages - .dict - .iter() - .chain(pages.data.iter()) - .find(|(page_offset, _)| *page_offset == offset) - .map(|(_, page)| page.clone()) + + if let Some((off, page)) = &pages.dict { + if *off == offset { + return Some(page.clone()); + } + } + + pages.data.iter().find(|(off, _)| *off == offset).map(|(_, page)| page.clone()) }) } /// Insert a page into the cache. /// Inserting a page will override the existing page, if any. - /// This is because we only need to cache 2 pages per column, see below. - pub(crate) fn insert_page(&mut self, col_id: usize, offset: usize, page: Page) { + /// This is because we only need to cache 4 pages per column, see below. + /// Note: 1 page is dic page, another 3 are data pages. + /// Why do we need 3 data pages? + /// Because of the performance testing result, the batch size will across multi pages. It causes original page reader + /// cache page reader doesn't always hit the cache page. So here we keep 3 pages, and from the testing result it + /// shows that 3 pages are enough to cover the batch size when we're setting batch size to 8192. And the 3 data page size + /// is not too large, it only uses 3MB in memory, so we can keep 3 pages in the cache. + /// TODO, in future we may use adaptive cache size according the dynamic batch size. + pub(crate) fn insert_page( + &mut self, + col_id: usize, + offset: usize, + page: Page, + ) { let is_dict = page.page_type() == PageType::DICTIONARY_PAGE; - let cached_pages = self.pages.entry(col_id); - match cached_pages { - Entry::Occupied(mut entry) => { + match self.pages.entry(col_id) { + Entry::Occupied(mut occ) => { + let cp: &mut CachedPage = occ.get_mut(); if is_dict { - entry.get_mut().dict = Some((offset, page)); + cp.dict = Some((offset, page)); } else { - entry.get_mut().data = Some((offset, page)); + cp.data.push_back((offset, page)); + // Keep only 3 data pages in the cache + if cp.data.len() > 3 { + cp.data.pop_front(); + } } } - Entry::Vacant(entry) => { - let cached_page = if is_dict { - CachedPage { - dict: Some((offset, page)), - data: None, - } + + Entry::Vacant(vac) => { + let mut data = VecDeque::new(); + let dict = if is_dict { + Some((offset, page)) } else { - CachedPage { - dict: None, - data: Some((offset, page)), - } + data.push_back((offset, page.clone())); + None }; - entry.insert(cached_page); + let cp = CachedPage { dict, data }; + vac.insert(cp); } } } From 04ca371409be4d8f021084a7c4fe7bf24e19cb9c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 7 May 2025 13:29:06 +0800 Subject: [PATCH 58/73] Revert "Improve the performance for skip record" This reverts commit 0c3aa9b3d4fa913029742f36b650f7b1e1ea2339. --- .../src/arrow/async_reader/arrow_reader.rs | 25 ++----------------- parquet/src/file/serialized_reader.rs | 4 +-- 2 files changed, 3 insertions(+), 26 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 2c199acd349b..53410a46f10b 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -179,47 +179,26 @@ impl Iterator for FilteredParquetRecordBatchReader { // rather than concatenating multiple small batches. let mut selected = 0; - - // Try to merge skip selectors, because from testing merge then to skip - // is faster than skipping each one. - let mut skip_accum = 0; - while let Some(cur_selection) = take_next_selection(&mut self.selection, self.batch_size - selected) { let filtered_selection = match self.build_predicate_filter(cur_selection) { Ok(selection) => selection, - Err(e) => { - if skip_accum > 0 { - self.array_reader.skip_records(skip_accum).ok()?; - } - return Some(Err(e)); - } + Err(e) => return Some(Err(e)), }; for selector in filtered_selection.iter() { if selector.skip { - skip_accum += selector.row_count; + self.array_reader.skip_records(selector.row_count).ok()?; } else { - if skip_accum > 0 { - self.array_reader.skip_records(skip_accum).ok()?; - skip_accum = 0; - } self.array_reader.read_records(selector.row_count).ok()?; } } - selected += filtered_selection.row_count(); - if selected >= (self.batch_size / 4 * 3) { break; } } - - if skip_accum > 0 { - self.array_reader.skip_records(skip_accum).ok()?; - } - if selected == 0 { return None; } diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 46a94f6b7416..ce4b16ac1fa3 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -984,9 +984,7 @@ impl PageReader for SerializedPageReader { SerializedPageReaderState::Pages { page_locations, - dictionary_page, - .. - } => { + dictionary_page, .. } => { if dictionary_page.is_some() { // If a dictionary page exists, consume it by taking it (sets to None) dictionary_page.take(); From c045a4a0cd0cfbbf2a9e60ad82f6ee607e38474b Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 7 May 2025 17:06:28 +0800 Subject: [PATCH 59/73] Combine with page cache --- parquet/src/arrow/arrow_reader/selection.rs | 44 +++- .../src/arrow/async_reader/arrow_reader.rs | 170 ++++++++---- parquet/src/arrow/async_reader/mod.rs | 46 ++-- patch.1 | 243 ++++++++++++++++++ 4 files changed, 422 insertions(+), 81 deletions(-) create mode 100644 patch.1 diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 4840a89f0ab2..60cb0870408d 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -595,6 +595,45 @@ impl RowSelection { } } +/// Convert a BooleanArray (no nulls) into a Vec, +/// alternating `select(run_len)` and `skip(run_len)`. +fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { + let mut selectors = Vec::new(); + let mut run_skip = !bitmap.value(0); // start by skipping if first bit is false + let mut run_len = 0; + + for i in 0..bitmap.len() { + let bit = bitmap.value(i); + if bit == !run_skip { + // same as current run type (select vs skip) + run_len += 1; + } else { + // flush previous run + selectors.push( + if run_skip { + RowSelector::skip(run_len) + } else { + RowSelector::select(run_len) + } + ); + // start new run + run_skip = !run_skip; + run_len = 1; + } + } + // flush final run + if run_len > 0 { + selectors.push( + if run_skip { + RowSelector::skip(run_len) + } else { + RowSelector::select(run_len) + } + ); + } + selectors +} + impl From> for RowSelection { fn from(selectors: Vec) -> Self { selectors.into_iter().collect() @@ -641,10 +680,7 @@ impl From for Vec { fn from(r: RowSelection) -> Self { match r { RowSelection::Ranges(selectors) => selectors, - RowSelection::BitMap(bitmap) => { - // not implemented yet - unimplemented!("BitMap variant is not yet supported") - } + RowSelection::BitMap(bitmap) => selectors_from_bitmap(&bitmap), } } } diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index d1eb79ed26f7..ab993525350c 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -20,10 +20,11 @@ use std::collections::HashMap; use std::sync::{Mutex, MutexGuard}; use std::{collections::VecDeque, sync::Arc}; -use arrow_array::ArrayRef; +use arrow_array::{ArrayRef, BooleanArray}; use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; +use arrow_buffer::BooleanBufferBuilder; use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; -use arrow_select::filter::prep_null_mask_filter; +use arrow_select::filter::{filter, filter_record_batch, prep_null_mask_filter}; use crate::basic::PageType; use crate::column::page::{Page, PageMetadata, PageReader}; @@ -49,70 +50,96 @@ fn read_selection( reader: &mut dyn ArrayReader, selection: &RowSelection, ) -> Result { - for selector in selection.iter() { - if selector.skip { - let skipped = reader.skip_records(selector.row_count)?; - debug_assert_eq!(skipped, selector.row_count, "failed to skip rows"); - } else { - let read_records = reader.read_records(selector.row_count)?; - debug_assert_eq!(read_records, selector.row_count, "failed to read rows"); + match selection { + RowSelection::Ranges(selectors) => { + // … your existing loop over selectors … + for selector in selectors { + if selector.skip { + reader.skip_records(selector.row_count)?; + } else { + reader.read_records(selector.row_count)?; + } + } + reader.consume_batch() + }, + + RowSelection::BitMap(bitmap) => { + // 1) 先一次性读入 bitmap.len() 条记录 + let to_read = bitmap.len(); + reader.read_records(to_read)?; + let array = reader.consume_batch()?; + + // 2) 直接对 StructArray 做 filter,得到新的 ArrayRef(也是 StructArray) + let filtered_array = filter(&array, bitmap) + .map_err(|e| ParquetError::General(e.to_string()))?; + + // 3) 返回 ArrayRef,后续你的 next() 里再拿它去 as_struct_opt() + Ok(filtered_array) } } - reader.consume_batch() } -/// Take the next selection from the selection queue, and return the selection -/// whose selected row count is to_select or less (if input selection is exhausted). fn take_next_selection( selection: &mut Option, to_select: usize, ) -> Option { - let mut current_selected = 0; - let mut rt = Vec::new(); - - let mut queue: VecDeque = match selection.take() { - Some(RowSelection::Ranges(selectors)) => selectors.into(), - Some(RowSelection::BitMap(_)) => { - unimplemented!("BitMap variant is not yet supported") - } - None => return None, + // 1) pull the current selection out exactly once + let current = match selection.take()? { + RowSelection::Ranges(r) => RowSelection::Ranges(r), + RowSelection::BitMap(bm) => RowSelection::BitMap(bm), }; - - while let Some(front) = queue.pop_front() { - if front.skip { - rt.push(front); + + // 2) fast-path if it's a bitmap + if let RowSelection::BitMap(mut bitmap) = current { + let take = bitmap.len().min(to_select); + let prefix = bitmap.slice(0, take); + let suffix_len = bitmap.len() - take; + let suffix = if suffix_len > 0 { + Some(bitmap.slice(take, suffix_len)) + } else { + None + }; + *selection = suffix.map(RowSelection::BitMap); + return Some(RowSelection::BitMap(prefix)); + } + + // 3) otherwise it must be ranges + let RowSelection::Ranges(runs) = current else { unreachable!() }; + let mut queue: VecDeque = runs.into(); + let mut taken = Vec::new(); + let mut count = 0; + + while let Some(run) = queue.pop_front() { + if run.skip { + taken.push(run); continue; } - - if current_selected + front.row_count <= to_select { - rt.push(front); - current_selected += front.row_count; + let room = to_select.saturating_sub(count); + if run.row_count <= room { + taken.push(run.clone()); + count += run.row_count; } else { - let select = to_select - current_selected; - let remaining = front.row_count - select; - rt.push(RowSelector::select(select)); - queue.push_front(RowSelector::select(remaining)); - *selection = if queue.is_empty() { - None - } else { - Some(queue.into_iter().collect()) - }; - return Some(rt.into()); + taken.push(RowSelector::select(room)); + queue.push_front(RowSelector::select(run.row_count - room)); + break; } } - + *selection = if queue.is_empty() { None } else { - Some(queue.into_iter().collect()) + Some(RowSelection::Ranges(queue.into_iter().collect())) }; - - if !rt.is_empty() { - return Some(rt.into()); + + // if we only got skips, then there's nothing to return + if taken.iter().all(|s| s.skip) { + None + } else { + Some(RowSelection::Ranges(taken.into())) } - None } + impl FilteredParquetRecordBatchReader { pub(crate) fn new( batch_size: usize, @@ -154,6 +181,8 @@ impl FilteredParquetRecordBatchReader { "predicate readers and predicates should have the same length" ); + let mut is_bitmap = matches!(selection, RowSelection::BitMap(_)); + for (predicate, reader) in filter .predicates .iter_mut() @@ -175,7 +204,14 @@ impl FilteredParquetRecordBatchReader { 0 => predicate_filter, _ => prep_null_mask_filter(&predicate_filter), }; - let raw = RowSelection::from_filters(&[predicate_filter]); + let raw = if is_bitmap { + // 一直保持 bitmap 逻辑 + RowSelection::from_filters_as_bitmap(&[predicate_filter]) + } else { + // ranges 逻辑:累积各段 select/skip + RowSelection::from_filters(&[predicate_filter]) + }; + selection = selection.and_then(&raw); } Ok(selection) @@ -199,6 +235,8 @@ impl Iterator for FilteredParquetRecordBatchReader { // rather than concatenating multiple small batches. let mut selected = 0; + let mut final_bitmap_builder: Option = None; + while let Some(cur_selection) = take_next_selection(&mut self.selection, self.batch_size - selected) { @@ -206,26 +244,50 @@ impl Iterator for FilteredParquetRecordBatchReader { Ok(selection) => selection, Err(e) => return Some(Err(e)), }; - - // println!("Filtered selection: {:?}", filtered_selection); - for selector in filtered_selection.iter() { - if selector.skip { - self.array_reader.skip_records(selector.row_count).ok()?; - } else { - self.array_reader.read_records(selector.row_count).ok()?; + match &filtered_selection { + RowSelection::Ranges(_) => { + for selector in filtered_selection.iter() { + if selector.skip { + self.array_reader.skip_records(selector.row_count).ok()?; + } else { + self.array_reader.read_records(selector.row_count).ok()?; + } + } + selected += filtered_selection.row_count(); + } + RowSelection::BitMap(bitmap) => { + let to_read = bitmap.len(); + self.array_reader.read_records(to_read).ok()?; + selected += bitmap.true_count(); + + let builder = final_bitmap_builder + .get_or_insert_with(|| BooleanBufferBuilder::new(to_read)); + for value in bitmap.iter() { + builder.append(value?); + } } } - selected += filtered_selection.row_count(); + if selected >= (self.batch_size / 4 * 3) { break; } } + if selected == 0 { return None; } let array = self.array_reader.consume_batch().ok()?; + let array = if let Some(mut bitmap_builder) = final_bitmap_builder { + let bitmap = BooleanArray::new(bitmap_builder.finish(), None); + filter(&array, &bitmap) + .map_err(|e| ParquetError::General(e.to_string())) + .ok()? + } else { + array + }; + let struct_array = array .as_struct_opt() .ok_or_else(|| general_err!("Struct array reader should return struct array")) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 6fa46af7ca24..7f0732f6adab 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -602,7 +602,7 @@ where ); let mut selection = - selection.unwrap_or_else(|| vec![RowSelector::select(row_group.row_count)].into()); + selection.unwrap_or_else(|| vec![RowSelector::select(row_group.row_count)].into()).trim(); let mut filter_readers = Vec::new(); if let Some(filter) = self.filter.as_mut() { @@ -661,27 +661,27 @@ where - // let total_rows = selection.total_rows(); - // - // let selection = match selection { - // RowSelection::Ranges(selectors) => { - // if total_rows / selectors.len() > 200 { - // Some(RowSelection::Ranges(selectors)) - // } else { - // let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); - // - // for selector in selectors.iter() { - // if selector.skip { - // builder.append_n(selector.row_count, false); - // } else { - // builder.append_n(selector.row_count, true); - // } - // } - // Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) - // } - // } - // _ => None, - // }; + let total_rows = selection.total_rows(); + + let selection = match selection { + RowSelection::Ranges(selectors) => { + if total_rows / selectors.len() > 200 { + Some(RowSelection::Ranges(selectors)) + } else { + let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); + + for selector in selectors.iter() { + if selector.skip { + builder.append_n(selector.row_count, false); + } else { + builder.append_n(selector.row_count, true); + } + } + Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) + } + } + _ => None, + }; @@ -689,7 +689,7 @@ where let reader = FilteredParquetRecordBatchReader::new( batch_size, array_reader, - Some(selection), + selection, filter_readers, self.filter.take(), ); diff --git a/patch.1 b/patch.1 new file mode 100644 index 000000000000..eedf9ee3f533 --- /dev/null +++ b/patch.1 @@ -0,0 +1,243 @@ +diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs +index 4840a89f0..60cb08704 100644 +--- a/parquet/src/arrow/arrow_reader/selection.rs ++++ b/parquet/src/arrow/arrow_reader/selection.rs +@@ -595,6 +595,45 @@ impl RowSelection { + } + } + ++/// Convert a BooleanArray (no nulls) into a Vec, ++/// alternating `select(run_len)` and `skip(run_len)`. ++fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { ++ let mut selectors = Vec::new(); ++ let mut run_skip = !bitmap.value(0); // start by skipping if first bit is false ++ let mut run_len = 0; ++ ++ for i in 0..bitmap.len() { ++ let bit = bitmap.value(i); ++ if bit == !run_skip { ++ // same as current run type (select vs skip) ++ run_len += 1; ++ } else { ++ // flush previous run ++ selectors.push( ++ if run_skip { ++ RowSelector::skip(run_len) ++ } else { ++ RowSelector::select(run_len) ++ } ++ ); ++ // start new run ++ run_skip = !run_skip; ++ run_len = 1; ++ } ++ } ++ // flush final run ++ if run_len > 0 { ++ selectors.push( ++ if run_skip { ++ RowSelector::skip(run_len) ++ } else { ++ RowSelector::select(run_len) ++ } ++ ); ++ } ++ selectors ++} ++ + impl From> for RowSelection { + fn from(selectors: Vec) -> Self { + selectors.into_iter().collect() +@@ -641,10 +680,7 @@ impl From for Vec { + fn from(r: RowSelection) -> Self { + match r { + RowSelection::Ranges(selectors) => selectors, +- RowSelection::BitMap(bitmap) => { +- // not implemented yet +- unimplemented!("BitMap variant is not yet supported") +- } ++ RowSelection::BitMap(bitmap) => selectors_from_bitmap(&bitmap), + } + } + } +diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs +index d1eb79ed2..8c0acbf05 100644 +--- a/parquet/src/arrow/async_reader/arrow_reader.rs ++++ b/parquet/src/arrow/async_reader/arrow_reader.rs +@@ -23,7 +23,7 @@ use std::{collections::VecDeque, sync::Arc}; + use arrow_array::ArrayRef; + use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; + use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; +-use arrow_select::filter::prep_null_mask_filter; ++use arrow_select::filter::{filter, filter_record_batch, prep_null_mask_filter}; + + use crate::basic::PageType; + use crate::column::page::{Page, PageMetadata, PageReader}; +@@ -49,16 +49,33 @@ fn read_selection( + reader: &mut dyn ArrayReader, + selection: &RowSelection, + ) -> Result { +- for selector in selection.iter() { +- if selector.skip { +- let skipped = reader.skip_records(selector.row_count)?; +- debug_assert_eq!(skipped, selector.row_count, "failed to skip rows"); +- } else { +- let read_records = reader.read_records(selector.row_count)?; +- debug_assert_eq!(read_records, selector.row_count, "failed to read rows"); ++ match selection { ++ RowSelection::Ranges(selectors) => { ++ // … your existing loop over selectors … ++ for selector in selectors { ++ if selector.skip { ++ reader.skip_records(selector.row_count)?; ++ } else { ++ reader.read_records(selector.row_count)?; ++ } ++ } ++ reader.consume_batch() ++ }, ++ ++ RowSelection::BitMap(bitmap) => { ++ // 1) 先一次性读入 bitmap.len() 条记录 ++ let to_read = bitmap.len(); ++ reader.read_records(to_read)?; ++ let array = reader.consume_batch()?; ++ ++ // 2) 直接对 StructArray 做 filter,得到新的 ArrayRef(也是 StructArray) ++ let filtered_array = filter(&array, bitmap) ++ .map_err(|e| ParquetError::General(e.to_string()))?; ++ ++ // 3) 返回 ArrayRef,后续你的 next() 里再拿它去 as_struct_opt() ++ Ok(filtered_array) + } + } +- reader.consume_batch() + } + + /// Take the next selection from the selection queue, and return the selection +@@ -67,52 +84,70 @@ fn take_next_selection( + selection: &mut Option, + to_select: usize, + ) -> Option { +- let mut current_selected = 0; +- let mut rt = Vec::new(); +- ++ // 1) If it’s a bitmap, handle it directly and return ++ if let Some(RowSelection::BitMap(bitmap)) = selection.take() { ++ // how many to take ++ let take = bitmap.len().min(to_select); ++ let prefix = bitmap.slice(0, take); ++ // compute any remainder ++ let rem = bitmap.len() - take; ++ let suffix = if rem > 0 { ++ Some(bitmap.slice(take, rem)) ++ } else { ++ None ++ }; ++ // put the suffix back (if any) ++ *selection = suffix.map(RowSelection::BitMap); ++ // return just the prefix ++ return Some(RowSelection::BitMap(prefix)); ++ } ++ ++ // 2) Otherwise it must be Ranges (or empty) + let mut queue: VecDeque = match selection.take() { +- Some(RowSelection::Ranges(selectors)) => selectors.into(), +- Some(RowSelection::BitMap(_)) => { +- unimplemented!("BitMap variant is not yet supported") +- } ++ Some(RowSelection::Ranges(r)) => r.into(), + None => return None, ++ // no other variants possible here ++ _ => unreachable!(), + }; +- +- while let Some(front) = queue.pop_front() { +- if front.skip { +- rt.push(front); ++ ++ let mut taken = Vec::new(); ++ let mut count = 0; ++ ++ while let Some(sel) = queue.pop_front() { ++ if sel.skip { ++ taken.push(sel); + continue; + } +- +- if current_selected + front.row_count <= to_select { +- rt.push(front); +- current_selected += front.row_count; ++ // how many we can still take ++ let room = to_select - count; ++ if sel.row_count <= room { ++ taken.push(sel); ++ count += sel.row_count; + } else { +- let select = to_select - current_selected; +- let remaining = front.row_count - select; +- rt.push(RowSelector::select(select)); +- queue.push_front(RowSelector::select(remaining)); +- *selection = if queue.is_empty() { +- None +- } else { +- Some(queue.into_iter().collect()) +- }; +- return Some(rt.into()); ++ // split the run ++ taken.push(RowSelector::select(room)); ++ queue.push_front(RowSelector::select(sel.row_count - room)); ++ count = to_select; ++ break; + } + } +- ++ ++ // 3) stash back any leftover selectors + *selection = if queue.is_empty() { + None + } else { + Some(queue.into_iter().collect()) + }; +- +- if !rt.is_empty() { +- return Some(rt.into()); ++ ++ // 4) return what we took (if any) ++ if taken.is_empty() { ++ None ++ } else { ++ Some(taken.into()) + } +- None + } + ++ + impl FilteredParquetRecordBatchReader { + pub(crate) fn new( + batch_size: usize, +@@ -154,6 +189,8 @@ impl FilteredParquetRecordBatchReader { + "predicate readers and predicates should have the same length" + ); + ++ let mut is_bitmap = matches!(selection, RowSelection::BitMap(_)); ++ + for (predicate, reader) in filter + .predicates + .iter_mut() +@@ -175,7 +212,14 @@ impl FilteredParquetRecordBatchReader { + 0 => predicate_filter, + _ => prep_null_mask_filter(&predicate_filter), + }; +- let raw = RowSelection::from_filters(&[predicate_filter]); ++ let raw = if is_bitmap { ++ // 一直保持 bitmap 逻辑 ++ RowSelection::from_filters_as_bitmap(&[predicate_filter]) ++ } else { ++ // ranges 逻辑:累积各段 select/skip ++ RowSelection::from_filters(&[predicate_filter]) ++ }; ++ + selection = selection.and_then(&raw); + } + Ok(selection) From 6e32d3bddb8123a2f0aa48176d1b8134635de6a4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 8 May 2025 18:18:36 +0800 Subject: [PATCH 60/73] Need to fix --- .../src/arrow/async_reader/arrow_reader.rs | 147 +++++++++++------- parquet/src/arrow/async_reader/mod.rs | 52 ++++--- 2 files changed, 119 insertions(+), 80 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index ab993525350c..d27f3e6279d0 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -83,13 +83,10 @@ fn take_next_selection( selection: &mut Option, to_select: usize, ) -> Option { - // 1) pull the current selection out exactly once - let current = match selection.take()? { - RowSelection::Ranges(r) => RowSelection::Ranges(r), - RowSelection::BitMap(bm) => RowSelection::BitMap(bm), - }; + // 1) clone 而不 take,让我们在任何情况下都能重建剩余 selection + let current = selection.as_ref()?.clone(); - // 2) fast-path if it's a bitmap + // 2) bitmap 分支保持不变 if let RowSelection::BitMap(mut bitmap) = current { let take = bitmap.len().min(to_select); let prefix = bitmap.slice(0, take); @@ -103,7 +100,7 @@ fn take_next_selection( return Some(RowSelection::BitMap(prefix)); } - // 3) otherwise it must be ranges + // 3) Ranges 分支 let RowSelection::Ranges(runs) = current else { unreachable!() }; let mut queue: VecDeque = runs.into(); let mut taken = Vec::new(); @@ -111,35 +108,40 @@ fn take_next_selection( while let Some(run) = queue.pop_front() { if run.skip { - taken.push(run); + // 只跳过也要加入 taken,保证真正调用 skip_records + taken.push(run.clone()); continue; } let room = to_select.saturating_sub(count); + if room == 0 { + // 如果已满,就把当前 run 和剩余所有放回 queue + queue.push_front(run); + break; + } if run.row_count <= room { taken.push(run.clone()); count += run.row_count; } else { + // 部分切分 taken.push(RowSelector::select(room)); queue.push_front(RowSelector::select(run.row_count - room)); break; } } + // 4) 根据剩余 queue 更新 selection *selection = if queue.is_empty() { None } else { Some(RowSelection::Ranges(queue.into_iter().collect())) }; - // if we only got skips, then there's nothing to return - if taken.iter().all(|s| s.skip) { - None - } else { - Some(RowSelection::Ranges(taken.into())) - } + // 5) 返回 taken,无论是否全 skip + Some(RowSelection::Ranges(taken.into())) } + impl FilteredParquetRecordBatchReader { pub(crate) fn new( batch_size: usize, @@ -167,6 +169,14 @@ impl FilteredParquetRecordBatchReader { self.row_filter.take() } + fn create_bitmap_from_ranges(&mut self, runs: &[RowSelector]) -> BooleanArray { + let mut bitmap_builder = BooleanBufferBuilder::new(runs.iter().map(|r| r.row_count).sum()); + for run in runs.iter() { + bitmap_builder.append_n(run.row_count, !run.skip); + } + BooleanArray::new(bitmap_builder.finish(), None) + } + /// Take a selection, and return the new selection where the rows are filtered by the predicate. fn build_predicate_filter( &mut self, @@ -181,8 +191,6 @@ impl FilteredParquetRecordBatchReader { "predicate readers and predicates should have the same length" ); - let mut is_bitmap = matches!(selection, RowSelection::BitMap(_)); - for (predicate, reader) in filter .predicates .iter_mut() @@ -204,14 +212,9 @@ impl FilteredParquetRecordBatchReader { 0 => predicate_filter, _ => prep_null_mask_filter(&predicate_filter), }; - let raw = if is_bitmap { - // 一直保持 bitmap 逻辑 - RowSelection::from_filters_as_bitmap(&[predicate_filter]) - } else { - // ranges 逻辑:累积各段 select/skip - RowSelection::from_filters(&[predicate_filter]) - }; + // ranges 逻辑:累积各段 select/skip + let raw = RowSelection::from_filters(&[predicate_filter]); selection = selection.and_then(&raw); } Ok(selection) @@ -234,68 +237,98 @@ impl Iterator for FilteredParquetRecordBatchReader { // It boils down to leveraging array_reader's ability to collect large batches natively, // rather than concatenating multiple small batches. - let mut selected = 0; - let mut final_bitmap_builder: Option = None; + let mut rows_accum = 0; + // Build one combined mask over everything we read + let mut mask_builder = BooleanBufferBuilder::new(self.batch_size); - while let Some(cur_selection) = - take_next_selection(&mut self.selection, self.batch_size - selected) - { - let filtered_selection = match self.build_predicate_filter(cur_selection) { - Ok(selection) => selection, + // 1) Read/skip in small chunks until ~¾ of batch_size + while let Some(raw_sel) = take_next_selection(&mut self.selection, self.batch_size - rows_accum) { + let sel = match self.build_predicate_filter(raw_sel) { + Ok(s) => s, Err(e) => return Some(Err(e)), }; - match &filtered_selection { - RowSelection::Ranges(_) => { - for selector in filtered_selection.iter() { - if selector.skip { - self.array_reader.skip_records(selector.row_count).ok()?; + match sel { + RowSelection::Ranges(runs) => { + // Count skip/read rows + let mut range_skip_count = 0; + let mut range_read_count = 0; + + for run in runs.iter() { + if run.skip { + range_skip_count += run.row_count; } else { - self.array_reader.read_records(selector.row_count).ok()?; + range_read_count += run.row_count; + } + } + + // If the number of "skip" is too high, switch to Bitmap + if range_skip_count as f32 / (range_skip_count + range_read_count) as f32 > 0.5 { + // Too many skips, switch to bitmap + let bitmap = self.create_bitmap_from_ranges(&runs); + self.array_reader.read_records(bitmap.len()).ok()?; + mask_builder.append_buffer(bitmap.values()); + rows_accum += bitmap.true_count(); + } else { + // Otherwise, use range + for run in runs.iter() { + if run.skip { + self.array_reader.skip_records(run.row_count).ok()?; + } else { + self.array_reader.read_records(run.row_count).ok()?; + mask_builder.append_n(run.row_count, true); + rows_accum += run.row_count; + } } } - selected += filtered_selection.row_count(); } + RowSelection::BitMap(bitmap) => { - let to_read = bitmap.len(); - self.array_reader.read_records(to_read).ok()?; - selected += bitmap.true_count(); - - let builder = final_bitmap_builder - .get_or_insert_with(|| BooleanBufferBuilder::new(to_read)); - for value in bitmap.iter() { - builder.append(value?); - } + // read exactly bitmap.len() rows + let n = bitmap.len(); + self.array_reader.read_records(n).ok()?; + let raw = bitmap.values(); + mask_builder.append_buffer(raw); + rows_accum += bitmap.true_count(); } } - if selected >= (self.batch_size / 4 * 3) { + if rows_accum >= (self.batch_size * 3 / 4) { break; } } - if selected == 0 { + // 2) If we read nothing, we’re done + if rows_accum == 0 { return None; } - let array = self.array_reader.consume_batch().ok()?; - let array = if let Some(mut bitmap_builder) = final_bitmap_builder { - let bitmap = BooleanArray::new(bitmap_builder.finish(), None); + // 3) Consume all buffered rows in the reader + let array = match self.array_reader.consume_batch() { + Ok(a) => a, + Err(_) => return None, + }; + + // 4) If we ever appended any mask bits, build and apply it + let final_array = if mask_builder.is_empty() { + array + } else { + let bitmap = BooleanArray::new(mask_builder.finish(), None); filter(&array, &bitmap) .map_err(|e| ParquetError::General(e.to_string())) .ok()? - } else { - array }; - let struct_array = array + // 5) Wrap into RecordBatch and return + let struct_arr = final_array .as_struct_opt() - .ok_or_else(|| general_err!("Struct array reader should return struct array")) - .ok()?; - Some(Ok(RecordBatch::from(struct_array.clone()))) + .expect("StructArray expected"); + Some(Ok(RecordBatch::from(struct_arr.clone()))) } } + + impl RecordBatchReader for FilteredParquetRecordBatchReader { fn schema(&self) -> SchemaRef { self.schema.clone() diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 7f0732f6adab..345022c77655 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -602,7 +602,7 @@ where ); let mut selection = - selection.unwrap_or_else(|| vec![RowSelector::select(row_group.row_count)].into()).trim(); + selection.unwrap_or_else(|| vec![RowSelector::select(row_group.row_count)].into()); let mut filter_readers = Vec::new(); if let Some(filter) = self.filter.as_mut() { @@ -661,27 +661,33 @@ where - let total_rows = selection.total_rows(); - - let selection = match selection { - RowSelection::Ranges(selectors) => { - if total_rows / selectors.len() > 200 { - Some(RowSelection::Ranges(selectors)) - } else { - let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); - - for selector in selectors.iter() { - if selector.skip { - builder.append_n(selector.row_count, false); - } else { - builder.append_n(selector.row_count, true); - } - } - Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) - } - } - _ => None, - }; + // let total_rows = selection.total_rows(); + // + // let selection = match selection { + // RowSelection::Ranges(selectors) => { + // // if total_rows / selectors.len() > 200 { + // // Some(RowSelection::Ranges(selectors)) + // // } else { + // + // println!("selectors len : {:?}", selectors.len()); + // println!("total select rows : {:?}", rows_after); + // println!("total rows : {:?}", row_group.row_count); + // println!("total rows for selectors : {:?}", rows_before); + // + // let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); + // + // for selector in selectors.iter() { + // if selector.skip { + // builder.append_n(selector.row_count, false); + // } else { + // builder.append_n(selector.row_count, true); + // } + // } + // Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) + // } + // // } + // _ => None, + // }; @@ -689,7 +695,7 @@ where let reader = FilteredParquetRecordBatchReader::new( batch_size, array_reader, - selection, + Some(selection), filter_readers, self.filter.take(), ); From b80f596f38d07a4ed65daf024b16860b09ac131f Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 8 May 2025 22:46:08 +0800 Subject: [PATCH 61/73] Add performance good result --- parquet/src/arrow/async_reader/arrow_reader.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index d27f3e6279d0..4422a83d0301 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -250,6 +250,8 @@ impl Iterator for FilteredParquetRecordBatchReader { match sel { RowSelection::Ranges(runs) => { + let select_count = runs.len(); + // Count skip/read rows let mut range_skip_count = 0; let mut range_read_count = 0; @@ -263,7 +265,7 @@ impl Iterator for FilteredParquetRecordBatchReader { } // If the number of "skip" is too high, switch to Bitmap - if range_skip_count as f32 / (range_skip_count + range_read_count) as f32 > 0.5 { + if ((range_skip_count + range_read_count) as f32 / select_count as f32) < 10.0 { // Too many skips, switch to bitmap let bitmap = self.create_bitmap_from_ranges(&runs); self.array_reader.read_records(bitmap.len()).ok()?; From 01d1dea0f58b901aa1e2434f20a71d7aba88c04c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 9 May 2025 13:48:08 +0800 Subject: [PATCH 62/73] Fix --- .../src/arrow/async_reader/arrow_reader.rs | 89 ++++++++++--------- 1 file changed, 47 insertions(+), 42 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 4422a83d0301..2c71822d1f82 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -52,7 +52,6 @@ fn read_selection( ) -> Result { match selection { RowSelection::Ranges(selectors) => { - // … your existing loop over selectors … for selector in selectors { if selector.skip { reader.skip_records(selector.row_count)?; @@ -64,16 +63,13 @@ fn read_selection( }, RowSelection::BitMap(bitmap) => { - // 1) 先一次性读入 bitmap.len() 条记录 let to_read = bitmap.len(); reader.read_records(to_read)?; let array = reader.consume_batch()?; - // 2) 直接对 StructArray 做 filter,得到新的 ArrayRef(也是 StructArray) let filtered_array = filter(&array, bitmap) .map_err(|e| ParquetError::General(e.to_string()))?; - // 3) 返回 ArrayRef,后续你的 next() 里再拿它去 as_struct_opt() Ok(filtered_array) } } @@ -83,10 +79,8 @@ fn take_next_selection( selection: &mut Option, to_select: usize, ) -> Option { - // 1) clone 而不 take,让我们在任何情况下都能重建剩余 selection let current = selection.as_ref()?.clone(); - // 2) bitmap 分支保持不变 if let RowSelection::BitMap(mut bitmap) = current { let take = bitmap.len().min(to_select); let prefix = bitmap.slice(0, take); @@ -100,7 +94,6 @@ fn take_next_selection( return Some(RowSelection::BitMap(prefix)); } - // 3) Ranges 分支 let RowSelection::Ranges(runs) = current else { unreachable!() }; let mut queue: VecDeque = runs.into(); let mut taken = Vec::new(); @@ -108,13 +101,11 @@ fn take_next_selection( while let Some(run) = queue.pop_front() { if run.skip { - // 只跳过也要加入 taken,保证真正调用 skip_records taken.push(run.clone()); continue; } let room = to_select.saturating_sub(count); if room == 0 { - // 如果已满,就把当前 run 和剩余所有放回 queue queue.push_front(run); break; } @@ -122,21 +113,18 @@ fn take_next_selection( taken.push(run.clone()); count += run.row_count; } else { - // 部分切分 taken.push(RowSelector::select(room)); queue.push_front(RowSelector::select(run.row_count - room)); break; } } - // 4) 根据剩余 queue 更新 selection *selection = if queue.is_empty() { None } else { Some(RowSelection::Ranges(queue.into_iter().collect())) }; - // 5) 返回 taken,无论是否全 skip Some(RowSelection::Ranges(taken.into())) } @@ -213,7 +201,6 @@ impl FilteredParquetRecordBatchReader { _ => prep_null_mask_filter(&predicate_filter), }; - // ranges 逻辑:累积各段 select/skip let raw = RowSelection::from_filters(&[predicate_filter]); selection = selection.and_then(&raw); } @@ -238,10 +225,10 @@ impl Iterator for FilteredParquetRecordBatchReader { // rather than concatenating multiple small batches. let mut rows_accum = 0; - // Build one combined mask over everything we read let mut mask_builder = BooleanBufferBuilder::new(self.batch_size); + // Move acc_skip here so it persists across loop iterations + let mut acc_skip = 0; - // 1) Read/skip in small chunks until ~¾ of batch_size while let Some(raw_sel) = take_next_selection(&mut self.selection, self.batch_size - rows_accum) { let sel = match self.build_predicate_filter(raw_sel) { Ok(s) => s, @@ -250,47 +237,63 @@ impl Iterator for FilteredParquetRecordBatchReader { match sel { RowSelection::Ranges(runs) => { - let select_count = runs.len(); + // First, compute total skip/read counts + let mut total_skip = 0; + let mut total_read = 0; + for r in &runs { + if r.skip { total_skip += r.row_count; } + else { total_read += r.row_count; } + } - // Count skip/read rows - let mut range_skip_count = 0; - let mut range_read_count = 0; + // If nothing to read, accumulate skip and continue + if total_read == 0 { + acc_skip += total_skip; + continue; + } - for run in runs.iter() { - if run.skip { - range_skip_count += run.row_count; - } else { - range_read_count += run.row_count; - } + //println!("select_count = {}, read_nums {}, total_skip{}, acc_skip {}", runs.len(), total_read, total_skip, acc_skip); + + // Before any read, flush accumulated skips + if acc_skip > 0 { + self.array_reader.skip_records(acc_skip).ok()?; + acc_skip = 0; } - // If the number of "skip" is too high, switch to Bitmap - if ((range_skip_count + range_read_count) as f32 / select_count as f32) < 10.0 { - // Too many skips, switch to bitmap + let select_count = runs.len(); + let total = total_skip + total_read; + if total < 10 * select_count { + // Bitmap branch let bitmap = self.create_bitmap_from_ranges(&runs); self.array_reader.read_records(bitmap.len()).ok()?; mask_builder.append_buffer(bitmap.values()); rows_accum += bitmap.true_count(); } else { - // Otherwise, use range - for run in runs.iter() { - if run.skip { - self.array_reader.skip_records(run.row_count).ok()?; + // Range branch with internal skip coalescing + for r in &runs { + if r.skip { + acc_skip += r.row_count; } else { - self.array_reader.read_records(run.row_count).ok()?; - mask_builder.append_n(run.row_count, true); - rows_accum += run.row_count; + if acc_skip > 0 { + self.array_reader.skip_records(acc_skip).ok()?; + acc_skip = 0; + } + self.array_reader.read_records(r.row_count).ok()?; + mask_builder.append_n(r.row_count, true); + rows_accum += r.row_count; } } } } RowSelection::BitMap(bitmap) => { - // read exactly bitmap.len() rows + // Flush any pending skips before bitmap + if acc_skip > 0 { + self.array_reader.skip_records(acc_skip).ok()?; + acc_skip = 0; + } let n = bitmap.len(); self.array_reader.read_records(n).ok()?; - let raw = bitmap.values(); - mask_builder.append_buffer(raw); + mask_builder.append_buffer(bitmap.values()); rows_accum += bitmap.true_count(); } } @@ -300,18 +303,21 @@ impl Iterator for FilteredParquetRecordBatchReader { } } - // 2) If we read nothing, we’re done + // At loop exit, flush any remaining skips before finishing batch + if acc_skip > 0 { + self.array_reader.skip_records(acc_skip).ok()?; + acc_skip = 0; + } + if rows_accum == 0 { return None; } - // 3) Consume all buffered rows in the reader let array = match self.array_reader.consume_batch() { Ok(a) => a, Err(_) => return None, }; - // 4) If we ever appended any mask bits, build and apply it let final_array = if mask_builder.is_empty() { array } else { @@ -321,7 +327,6 @@ impl Iterator for FilteredParquetRecordBatchReader { .ok()? }; - // 5) Wrap into RecordBatch and return let struct_arr = final_array .as_struct_opt() .expect("StructArray expected"); From 7bcd01178a4c6a4a1ecf788e7c573efb382a7802 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 20:21:06 +0800 Subject: [PATCH 63/73] Remove unused for the improvement --- .../arrow/arrow_reader/boolean_selector.rs | 313 ------------------ patch.1 | 243 -------------- 2 files changed, 556 deletions(-) delete mode 100644 parquet/src/arrow/arrow_reader/boolean_selector.rs delete mode 100644 patch.1 diff --git a/parquet/src/arrow/arrow_reader/boolean_selector.rs b/parquet/src/arrow/arrow_reader/boolean_selector.rs deleted file mode 100644 index 7e1dd2a98efc..000000000000 --- a/parquet/src/arrow/arrow_reader/boolean_selector.rs +++ /dev/null @@ -1,313 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::ops::Range; - -use arrow_array::{Array, BooleanArray}; -use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder, MutableBuffer}; -use arrow_data::bit_iterator::BitIndexIterator; - -use super::{RowSelection, RowSelector}; - -/// A selection of rows, similar to [`RowSelection`], but based on a boolean array -#[derive(Debug, Clone, PartialEq)] -pub struct BooleanRowSelection { - selector: BooleanArray, -} - -impl BooleanRowSelection { - /// Create a new [`BooleanRowSelection] from a list of [`BooleanArray`]. - pub fn from_filters(filters: &[BooleanArray]) -> Self { - let arrays: Vec<&dyn Array> = filters.iter().map(|x| x as &dyn Array).collect(); - let result = arrow_select::concat::concat(&arrays).unwrap().into_data(); - let boolean_array = BooleanArray::from(result); - BooleanRowSelection { - selector: boolean_array, - } - } - - /// Create a new [`BooleanRowSelection`] with all rows unselected - pub fn new_unselected(row_count: usize) -> Self { - let buffer = BooleanBuffer::new_unset(row_count); - let boolean_array = BooleanArray::from(buffer); - BooleanRowSelection { selector: boolean_array } - } - - /// Create a new [`BooleanRowSelection`] with all rows selected - pub fn new_selected(row_count: usize) -> Self { - let buffer = BooleanBuffer::new_set(row_count); - let boolean_array = BooleanArray::from(buffer); - BooleanRowSelection { selector: boolean_array } - } - - /// Returns a new [`BooleanRowSelection`] that selects the inverse of this [`BooleanRowSelection`]. - pub fn as_inverted(&self) -> Self { - let buffer = !self.selector.values(); - BooleanRowSelection { selector: BooleanArray::from(buffer) } - } - - /// Returns the number of rows selected by this [`BooleanRowSelection`]. - pub fn row_count(&self) -> usize { - self.selector.true_count() - } - - /// Create a new [`BooleanRowSelection`] from a list of consecutive ranges. - pub fn from_consecutive_ranges( - ranges: impl Iterator>, - total_rows: usize, - ) -> Self { - let mut buffer = BooleanBufferBuilder::new(total_rows); - let mut last_end = 0; - - for range in ranges { - let len = range.end - range.start; - if len == 0 { - continue; - } - - if range.start > last_end { - buffer.append_n(range.start - last_end, false); - } - buffer.append_n(len, true); - last_end = range.end; - } - - if last_end != total_rows { - buffer.append_n(total_rows - last_end, false); - } - - BooleanRowSelection { - selector: BooleanArray::from(buffer.finish()), - } - } - - /// Compute the union of two [`BooleanRowSelection`] - /// For example: - /// self: NNYYYYNNYYNYN - /// other: NYNNNNNNN - /// - /// returned: NYYYYYNNYYNYN - #[must_use] - pub fn union(&self, other: &Self) -> Self { - // use arrow::compute::kernels::boolean::or; - - let union_selectors = self.selector.values() | other.selector.values(); - - BooleanRowSelection { - selector: BooleanArray::from(union_selectors), - } - } - - /// Compute the intersection of two [`BooleanRowSelection`] - /// For example: - /// self: NNYYYYNNYYNYN - /// other: NYNNNNNNY - /// - /// returned: NNNNNNNNYYNYN - #[must_use] - pub fn intersection(&self, other: &Self) -> Self { - let intersection_selectors = self.selector.values() & other.selector.values(); - - BooleanRowSelection { - selector: BooleanArray::from(intersection_selectors), - } - } - - /// Combines this [`BooleanRowSelection`] with another using logical AND on the selected bits. - /// - /// Unlike intersection, the `other` [`BooleanRowSelection`] must have exactly as many set bits as `self`. - /// This method will keep only the bits in `self` that are also set in `other` - /// at the positions corresponding to `self`'s set bits. - pub fn and_then(&self, other: &Self) -> Self { - // Ensure that 'other' has exactly as many set bits as 'self' - debug_assert_eq!( - self.row_count(), - other.selector.len(), - "The 'other' selection must have exactly as many set bits as 'self'." - ); - - if self.selector.len() == other.selector.len() { - // fast path if the two selections are the same length - // common if this is the first predicate - debug_assert_eq!(self.row_count(), self.selector.len()); - return self.intersection(other); - } - - let mut buffer = MutableBuffer::from_len_zeroed(self.selector.values().inner().len()); - buffer.copy_from_slice(self.selector.values().inner().as_slice()); - let mut builder = BooleanBufferBuilder::new_from_buffer(buffer, self.selector.len()); - - // Create iterators for 'self' and 'other' bits - let mut other_bits = other.selector.iter(); - - for bit_idx in self.true_iter() { - let predicate = other_bits - .next() - .expect("Mismatch in set bits between self and other"); - if !predicate.unwrap() { - builder.set_bit(bit_idx, false); - } - } - - BooleanRowSelection { - selector: BooleanArray::from(builder.finish()), - } - } - - /// Returns an iterator over the indices of the set bits in this [`BooleanRowSelection`] - pub fn true_iter(&self) -> BitIndexIterator<'_> { - self.selector.values().set_indices() - } - - /// Returns `true` if this [`BooleanRowSelection`] selects any rows - pub fn selects_any(&self) -> bool { - self.true_iter().next().is_some() - } - - /// Returns a new [`BooleanRowSelection`] that selects the rows in this [`BooleanRowSelection`] from `offset` to `offset + len` - pub fn slice(&self, offset: usize, len: usize) -> BooleanArray { - self.selector.slice(offset, len) - } -} - -// impl From> for BooleanRowSelection { -// fn from(selection: Vec) -> Self { -// let selection = RowSelection::from(selection); -// RowSelection::into(selection) -// } -// } -// -// impl From for BooleanRowSelection { -// fn from(selection: RowSelection) -> Self { -// let total_rows = selection.row_count(); -// let mut builder = BooleanBufferBuilder::new(total_rows); -// -// for selector in selection.iter() { -// if selector.skip { -// builder.append_n(selector.row_count, false); -// } else { -// builder.append_n(selector.row_count, true); -// } -// } -// -// BooleanRowSelection { -// selector: BooleanArray::from(builder.finish()), -// } -// } -// } -// -// impl From<&BooleanRowSelection> for RowSelection{ -// fn from(selection: &BooleanRowSelection) -> Self { -// RowSelection::from_filters(&[selection.selector.clone()]) -// } -// } - -// #[cfg(test)] -// mod tests { -// use rand::Rng; -// -// use super::*; -// -// fn generate_random_row_selection(total_rows: usize, selection_ratio: f64) -> BooleanArray { -// let mut rng = rand::thread_rng(); -// let bools: Vec = (0..total_rows) -// .map(|_| rng.gen_bool(selection_ratio)) -// .collect(); -// BooleanArray::from(bools) -// } -// -// #[test] -// fn test_boolean_row_selection_round_trip() { -// let total_rows = 1_000; -// for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { -// let selection = generate_random_row_selection(total_rows, selection_ratio); -// let boolean_selection = BooleanRowSelection::from_filters(&[selection]); -// let row_selection = RowSelection::from(&boolean_selection); -// let boolean_selection_again = row_selection.into(); -// assert_eq!(boolean_selection, boolean_selection_again); -// } -// } -// -// #[test] -// fn test_boolean_union_intersection() { -// let total_rows = 1_000; -// -// let base_boolean_selection = -// BooleanRowSelection::from_filters(&[generate_random_row_selection(total_rows, 0.1)]); -// let base_row_selection = RowSelection::from(&base_boolean_selection); -// for &selection_ratio in &[0.0, 0.1, 0.5, 0.9, 1.0] { -// let boolean_selection = -// BooleanRowSelection::from_filters(&[generate_random_row_selection( -// total_rows, -// selection_ratio, -// )]); -// let row_selection = RowSelection::from(&boolean_selection); -// -// let boolean_union = boolean_selection.union(&base_boolean_selection); -// let row_union = row_selection.union(&base_row_selection); -// assert_eq!(boolean_union, BooleanRowSelection::from(row_union)); -// -// let boolean_intersection = boolean_selection.intersection(&base_boolean_selection); -// let row_intersection = row_selection.intersection(&base_row_selection); -// assert_eq!( -// boolean_intersection, -// BooleanRowSelection::from(row_intersection) -// ); -// } -// } -// -// #[test] -// fn test_boolean_selection_and_then() { -// // Initial mask: 001011010101 -// let self_filters = vec![BooleanArray::from(vec![ -// false, false, true, false, true, true, false, true, false, true, false, true, -// ])]; -// let self_selection = BooleanRowSelection::from_filters(&self_filters); -// -// // Predicate mask (only for selected bits): 001101 -// let other_filters = vec![BooleanArray::from(vec![ -// false, false, true, true, false, true, -// ])]; -// let other_selection = BooleanRowSelection::from_filters(&other_filters); -// -// let result = self_selection.and_then(&other_selection); -// -// // Expected result: 000001010001 -// let expected_filters = vec![BooleanArray::from(vec![ -// false, false, false, false, false, true, false, true, false, false, false, true, -// ])]; -// let expected_selection = BooleanRowSelection::from_filters(&expected_filters); -// -// assert_eq!(result, expected_selection); -// } -// -// #[test] -// #[should_panic( -// expected = "The 'other' selection must have exactly as many set bits as 'self'." -// )] -// fn test_and_then_mismatched_set_bits() { -// let self_filters = vec![BooleanArray::from(vec![true, true, false])]; -// let self_selection = BooleanRowSelection::from_filters(&self_filters); -// -// // 'other' has only one set bit, but 'self' has two -// let other_filters = vec![BooleanArray::from(vec![true, false, false])]; -// let other_selection = BooleanRowSelection::from_filters(&other_filters); -// -// // This should panic -// let _ = self_selection.and_then(&other_selection); -// } -// } \ No newline at end of file diff --git a/patch.1 b/patch.1 deleted file mode 100644 index eedf9ee3f533..000000000000 --- a/patch.1 +++ /dev/null @@ -1,243 +0,0 @@ -diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs -index 4840a89f0..60cb08704 100644 ---- a/parquet/src/arrow/arrow_reader/selection.rs -+++ b/parquet/src/arrow/arrow_reader/selection.rs -@@ -595,6 +595,45 @@ impl RowSelection { - } - } - -+/// Convert a BooleanArray (no nulls) into a Vec, -+/// alternating `select(run_len)` and `skip(run_len)`. -+fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { -+ let mut selectors = Vec::new(); -+ let mut run_skip = !bitmap.value(0); // start by skipping if first bit is false -+ let mut run_len = 0; -+ -+ for i in 0..bitmap.len() { -+ let bit = bitmap.value(i); -+ if bit == !run_skip { -+ // same as current run type (select vs skip) -+ run_len += 1; -+ } else { -+ // flush previous run -+ selectors.push( -+ if run_skip { -+ RowSelector::skip(run_len) -+ } else { -+ RowSelector::select(run_len) -+ } -+ ); -+ // start new run -+ run_skip = !run_skip; -+ run_len = 1; -+ } -+ } -+ // flush final run -+ if run_len > 0 { -+ selectors.push( -+ if run_skip { -+ RowSelector::skip(run_len) -+ } else { -+ RowSelector::select(run_len) -+ } -+ ); -+ } -+ selectors -+} -+ - impl From> for RowSelection { - fn from(selectors: Vec) -> Self { - selectors.into_iter().collect() -@@ -641,10 +680,7 @@ impl From for Vec { - fn from(r: RowSelection) -> Self { - match r { - RowSelection::Ranges(selectors) => selectors, -- RowSelection::BitMap(bitmap) => { -- // not implemented yet -- unimplemented!("BitMap variant is not yet supported") -- } -+ RowSelection::BitMap(bitmap) => selectors_from_bitmap(&bitmap), - } - } - } -diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs -index d1eb79ed2..8c0acbf05 100644 ---- a/parquet/src/arrow/async_reader/arrow_reader.rs -+++ b/parquet/src/arrow/async_reader/arrow_reader.rs -@@ -23,7 +23,7 @@ use std::{collections::VecDeque, sync::Arc}; - use arrow_array::ArrayRef; - use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; - use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; --use arrow_select::filter::prep_null_mask_filter; -+use arrow_select::filter::{filter, filter_record_batch, prep_null_mask_filter}; - - use crate::basic::PageType; - use crate::column::page::{Page, PageMetadata, PageReader}; -@@ -49,16 +49,33 @@ fn read_selection( - reader: &mut dyn ArrayReader, - selection: &RowSelection, - ) -> Result { -- for selector in selection.iter() { -- if selector.skip { -- let skipped = reader.skip_records(selector.row_count)?; -- debug_assert_eq!(skipped, selector.row_count, "failed to skip rows"); -- } else { -- let read_records = reader.read_records(selector.row_count)?; -- debug_assert_eq!(read_records, selector.row_count, "failed to read rows"); -+ match selection { -+ RowSelection::Ranges(selectors) => { -+ // … your existing loop over selectors … -+ for selector in selectors { -+ if selector.skip { -+ reader.skip_records(selector.row_count)?; -+ } else { -+ reader.read_records(selector.row_count)?; -+ } -+ } -+ reader.consume_batch() -+ }, -+ -+ RowSelection::BitMap(bitmap) => { -+ // 1) 先一次性读入 bitmap.len() 条记录 -+ let to_read = bitmap.len(); -+ reader.read_records(to_read)?; -+ let array = reader.consume_batch()?; -+ -+ // 2) 直接对 StructArray 做 filter,得到新的 ArrayRef(也是 StructArray) -+ let filtered_array = filter(&array, bitmap) -+ .map_err(|e| ParquetError::General(e.to_string()))?; -+ -+ // 3) 返回 ArrayRef,后续你的 next() 里再拿它去 as_struct_opt() -+ Ok(filtered_array) - } - } -- reader.consume_batch() - } - - /// Take the next selection from the selection queue, and return the selection -@@ -67,52 +84,70 @@ fn take_next_selection( - selection: &mut Option, - to_select: usize, - ) -> Option { -- let mut current_selected = 0; -- let mut rt = Vec::new(); -- -+ // 1) If it’s a bitmap, handle it directly and return -+ if let Some(RowSelection::BitMap(bitmap)) = selection.take() { -+ // how many to take -+ let take = bitmap.len().min(to_select); -+ let prefix = bitmap.slice(0, take); -+ // compute any remainder -+ let rem = bitmap.len() - take; -+ let suffix = if rem > 0 { -+ Some(bitmap.slice(take, rem)) -+ } else { -+ None -+ }; -+ // put the suffix back (if any) -+ *selection = suffix.map(RowSelection::BitMap); -+ // return just the prefix -+ return Some(RowSelection::BitMap(prefix)); -+ } -+ -+ // 2) Otherwise it must be Ranges (or empty) - let mut queue: VecDeque = match selection.take() { -- Some(RowSelection::Ranges(selectors)) => selectors.into(), -- Some(RowSelection::BitMap(_)) => { -- unimplemented!("BitMap variant is not yet supported") -- } -+ Some(RowSelection::Ranges(r)) => r.into(), - None => return None, -+ // no other variants possible here -+ _ => unreachable!(), - }; -- -- while let Some(front) = queue.pop_front() { -- if front.skip { -- rt.push(front); -+ -+ let mut taken = Vec::new(); -+ let mut count = 0; -+ -+ while let Some(sel) = queue.pop_front() { -+ if sel.skip { -+ taken.push(sel); - continue; - } -- -- if current_selected + front.row_count <= to_select { -- rt.push(front); -- current_selected += front.row_count; -+ // how many we can still take -+ let room = to_select - count; -+ if sel.row_count <= room { -+ taken.push(sel); -+ count += sel.row_count; - } else { -- let select = to_select - current_selected; -- let remaining = front.row_count - select; -- rt.push(RowSelector::select(select)); -- queue.push_front(RowSelector::select(remaining)); -- *selection = if queue.is_empty() { -- None -- } else { -- Some(queue.into_iter().collect()) -- }; -- return Some(rt.into()); -+ // split the run -+ taken.push(RowSelector::select(room)); -+ queue.push_front(RowSelector::select(sel.row_count - room)); -+ count = to_select; -+ break; - } - } -- -+ -+ // 3) stash back any leftover selectors - *selection = if queue.is_empty() { - None - } else { - Some(queue.into_iter().collect()) - }; -- -- if !rt.is_empty() { -- return Some(rt.into()); -+ -+ // 4) return what we took (if any) -+ if taken.is_empty() { -+ None -+ } else { -+ Some(taken.into()) - } -- None - } - -+ - impl FilteredParquetRecordBatchReader { - pub(crate) fn new( - batch_size: usize, -@@ -154,6 +189,8 @@ impl FilteredParquetRecordBatchReader { - "predicate readers and predicates should have the same length" - ); - -+ let mut is_bitmap = matches!(selection, RowSelection::BitMap(_)); -+ - for (predicate, reader) in filter - .predicates - .iter_mut() -@@ -175,7 +212,14 @@ impl FilteredParquetRecordBatchReader { - 0 => predicate_filter, - _ => prep_null_mask_filter(&predicate_filter), - }; -- let raw = RowSelection::from_filters(&[predicate_filter]); -+ let raw = if is_bitmap { -+ // 一直保持 bitmap 逻辑 -+ RowSelection::from_filters_as_bitmap(&[predicate_filter]) -+ } else { -+ // ranges 逻辑:累积各段 select/skip -+ RowSelection::from_filters(&[predicate_filter]) -+ }; -+ - selection = selection.and_then(&raw); - } - Ok(selection) From 123c3adab0fa5017075d4a2b8c8777a8bbaca0aa Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 20:47:32 +0800 Subject: [PATCH 64/73] Fix part of test --- parquet/src/arrow/arrow_reader/mod.rs | 33 +- parquet/src/arrow/arrow_reader/selection.rs | 1408 ++++++++--------- .../src/arrow/async_reader/arrow_reader.rs | 106 +- parquet/src/arrow/async_reader/mod.rs | 4 - parquet/src/file/serialized_reader.rs | 8 +- 5 files changed, 774 insertions(+), 785 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index e578d0003acd..2d3327045a2b 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -17,17 +17,6 @@ //! Contains reader which reads parquet data into arrow [`RecordBatch`] -use arrow_array::cast::AsArray; -use arrow_array::{Array, BooleanArray}; -use arrow_array::{RecordBatch, RecordBatchReader}; -use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; -use arrow_select::filter::prep_null_mask_filter; -pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; -pub use selection::{RowSelection, RowSelector}; -use std::collections::VecDeque; -use std::mem::take; -use std::sync::Arc; -use arrow_select::take::take_record_batch; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{build_array_reader, ArrayReader}; use crate::arrow::schema::{parquet_to_arrow_schema_and_fields, ParquetField}; @@ -39,11 +28,20 @@ use crate::errors::{ParquetError, Result}; use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; +use arrow_array::cast::AsArray; +use arrow_array::{Array, BooleanArray}; +use arrow_array::{RecordBatch, RecordBatchReader}; +use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; +use arrow_select::filter::prep_null_mask_filter; +pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; +pub use selection::{RowSelection, RowSelector}; +use std::collections::VecDeque; +use std::mem::take; +use std::sync::Arc; mod filter; mod selection; pub mod statistics; -mod boolean_selector; /// Builder for constructing Parquet readers that decode into [Apache Arrow] /// arrays. @@ -678,7 +676,7 @@ impl ParquetRecordBatchReaderBuilder { let array_reader = build_array_reader(self.fields.as_deref(), predicate.projection(), &reader)?; - + selection = Some(evaluate_predicate( batch_size, array_reader, @@ -860,15 +858,14 @@ impl Iterator for ParquetRecordBatchReader { ) }); - - let batch:RecordBatch = match struct_array { + let batch: RecordBatch = match struct_array { Err(err) => return Some(Err(err)), Ok(e) => e.into(), }; if let Some(filter) = filter.as_mut() { if batch.num_rows() == 0 { - return None + return None; } if filter.len() != batch.num_rows() { return Some(Err(ArrowError::ComputeError(format!( @@ -882,7 +879,6 @@ impl Iterator for ParquetRecordBatchReader { Ok(filtered_batch) => Some(Ok(filtered_batch)), Err(e) => Some(Err(e)), } - } else { if batch.num_rows() > 0 { Some(Ok(batch)) @@ -890,7 +886,6 @@ impl Iterator for ParquetRecordBatchReader { None } } - } } } @@ -1028,7 +1023,7 @@ pub(crate) fn evaluate_predicate( filter.len() )); } - + match filter.null_count() { 0 => filters.push(filter), _ => filters.push(prep_null_mask_filter(&filter)), diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 60cb0870408d..8fcc5595ab4e 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -16,14 +16,11 @@ // under the License. use arrow_array::{Array, BooleanArray}; +use arrow_buffer::{BooleanBufferBuilder, MutableBuffer}; use arrow_select::filter::SlicesIterator; use std::cmp::Ordering; use std::collections::VecDeque; use std::ops::Range; -use arrow_buffer::{BooleanBufferBuilder, MutableBuffer}; -use arrow_schema::ArrowError::NotYetImplemented; -use crate::arrow::arrow_reader::boolean_selector::BooleanRowSelection; -use crate::schema::printer; /// [`RowSelection`] is a collection of [`RowSelector`] used to skip rows when /// scanning a parquet file @@ -136,9 +133,9 @@ impl RowSelection { Self::from_consecutive_ranges(iter, total_rows) } - + /// Creates a [`RowSelection`] from a slice of [`BooleanArray`] as a bitmap - /// + /// pub fn from_filters_as_bitmap(filters: &[BooleanArray]) -> Self { let arrays: Vec<&dyn Array> = filters.iter().map(|x| x as &dyn Array).collect(); let result = arrow_select::concat::concat(&arrays).unwrap().into_data(); @@ -218,7 +215,8 @@ impl RowSelection { continue; } else { - if row_offset + selector.row_count == next_page.first_row_index as usize { + if row_offset + selector.row_count == next_page.first_row_index as usize + { current_page = pages.next(); current_page_included = false; } @@ -242,8 +240,6 @@ impl RowSelection { unimplemented!("BitMap variant is not yet supported") } } - - } /// Splits off the first `row_count` from this [`RowSelection`] @@ -289,7 +285,7 @@ impl RowSelection { } } } - + /// returns a [`RowSelection`] representing rows that are selected in both /// input [`RowSelection`]s. /// @@ -423,14 +419,15 @@ impl RowSelection { /// returned: NNNNNNNNYYNYN pub fn intersection(&self, other: &Self) -> Self { match (self, other) { - (RowSelection::Ranges(a), RowSelection::Ranges(b)) => - RowSelection::Ranges(intersect_row_selections(a, b)), - (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bit_map)) =>{ + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => { + RowSelection::Ranges(intersect_row_selections(a, b)) + } + (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bit_map)) => { let intersection_selectors = bit_map.values() & other_bit_map.values(); RowSelection::BitMap(BooleanArray::from(intersection_selectors)) - }, + } (RowSelection::BitMap(_), RowSelection::Ranges(_)) => todo!(), - (RowSelection::Ranges(_), RowSelection::BitMap(_)) => todo!() + (RowSelection::Ranges(_), RowSelection::BitMap(_)) => todo!(), } } @@ -442,22 +439,20 @@ impl RowSelection { /// returned: NYYYYYNNYYNYN pub fn union(&self, other: &Self) -> Self { match (self, other) { - (RowSelection::Ranges(a), RowSelection::Ranges(b)) => - RowSelection::Ranges(union_row_selections(a, b)), - (RowSelection::BitMap(_), _) | (_, RowSelection::BitMap(_)) => - unimplemented!("BitMap variant is not yet supported"), + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => { + RowSelection::Ranges(union_row_selections(a, b)) + } + (RowSelection::BitMap(_), _) | (_, RowSelection::BitMap(_)) => { + unimplemented!("BitMap variant is not yet supported") + } } } /// Returns `true` if this [`RowSelection`] selects any rows pub fn selects_any(&self) -> bool { match self { - RowSelection::Ranges(selectors) => { - selectors.iter().any(|x| !x.skip) - } - RowSelection::BitMap(bitmap) => { - bitmap.true_count() > 0 - } + RowSelection::Ranges(selectors) => selectors.iter().any(|x| !x.skip), + RowSelection::BitMap(bitmap) => bitmap.true_count() > 0, } } @@ -490,25 +485,23 @@ impl RowSelection { match self { RowSelection::Ranges(mut selectors) => { if offset == 0 { - return RowSelection::Ranges(selectors) + return RowSelection::Ranges(selectors); } let mut selected_count = 0; let mut skipped_count = 0; // Find the index where the selector exceeds the row count - let find = selectors - .iter() - .position(|selector| match selector.skip { - true => { - skipped_count += selector.row_count; - false - } - false => { - selected_count += selector.row_count; - selected_count > offset - } - }); + let find = selectors.iter().position(|selector| match selector.skip { + true => { + skipped_count += selector.row_count; + false + } + false => { + selected_count += selector.row_count; + selected_count > offset + } + }); let split_idx = match find { Some(idx) => idx, @@ -522,7 +515,7 @@ impl RowSelection { select.push(RowSelector::skip(skipped_count + offset)); select.push(RowSelector::select(selected_count - offset)); select.extend_from_slice(&selectors[split_idx + 1..]); - + RowSelection::Ranges(select) } RowSelection::BitMap(_) => { @@ -571,7 +564,7 @@ impl RowSelection { } } } - + pub fn total_rows(&self) -> usize { match self { RowSelection::Ranges(selectors) => selectors.iter().map(|s| s.row_count).sum(), @@ -579,13 +572,22 @@ impl RowSelection { } } + pub fn len(&self) -> usize { + match self { + RowSelection::Ranges(selectors) => selectors.len(), + RowSelection::BitMap(bitmap) => bitmap.len(), + } + } + /// Returns the number of selected rows pub fn row_count(&self) -> usize { match self { - RowSelection::Ranges(selectors) => selectors.iter().filter(|s| !s.skip).map(|s| s.row_count).sum(), - RowSelection::BitMap(bitmap) => { - bitmap.true_count() - } + RowSelection::Ranges(selectors) => selectors + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(), + RowSelection::BitMap(bitmap) => bitmap.true_count(), } } @@ -609,13 +611,11 @@ fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { run_len += 1; } else { // flush previous run - selectors.push( - if run_skip { - RowSelector::skip(run_len) - } else { - RowSelector::select(run_len) - } - ); + selectors.push(if run_skip { + RowSelector::skip(run_len) + } else { + RowSelector::select(run_len) + }); // start new run run_skip = !run_skip; run_len = 1; @@ -623,13 +623,11 @@ fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { } // flush final run if run_len > 0 { - selectors.push( - if run_skip { - RowSelector::skip(run_len) - } else { - RowSelector::select(run_len) - } - ); + selectors.push(if run_skip { + RowSelector::skip(run_len) + } else { + RowSelector::select(run_len) + }); } selectors } @@ -671,7 +669,7 @@ impl FromIterator for RowSelection { selectors.push(s) } } - + RowSelection::Ranges(selectors) } } @@ -846,8 +844,6 @@ fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec = selection.into(); - assert_eq!( - expected, - vec![RowSelector::skip(3), RowSelector::select(4)] - ); + assert_eq!(expected, vec![RowSelector::skip(3), RowSelector::select(4)]); let selection = RowSelection::from_filters(&filters[..2]); assert!(selection.selects_any()); @@ -953,7 +946,10 @@ mod tests { let split = selection.split_off(8); match split { RowSelection::Ranges(selectors) => { - assert_eq!(selectors, vec![RowSelector::select(7), RowSelector::skip(1)]); + assert_eq!( + selectors, + vec![RowSelector::select(7), RowSelector::skip(1)] + ); } _ => panic!("Expected Ranges variant"), } @@ -985,644 +981,644 @@ mod tests { } } -// #[test] -// fn test_offset() { -// let selection = RowSelection::from(vec![ -// RowSelector::select(5), -// RowSelector::skip(23), -// RowSelector::select(7), -// RowSelector::skip(33), -// RowSelector::select(6), -// ]); -// -// let selection = selection.offset(2); -// assert_eq!( -// selection.selectors, -// vec![ -// RowSelector::skip(2), -// RowSelector::select(3), -// RowSelector::skip(23), -// RowSelector::select(7), -// RowSelector::skip(33), -// RowSelector::select(6), -// ] -// ); -// -// let selection = selection.offset(5); -// assert_eq!( -// selection.selectors, -// vec![ -// RowSelector::skip(30), -// RowSelector::select(5), -// RowSelector::skip(33), -// RowSelector::select(6), -// ] -// ); -// -// let selection = selection.offset(3); -// assert_eq!( -// selection.selectors, -// vec![ -// RowSelector::skip(33), -// RowSelector::select(2), -// RowSelector::skip(33), -// RowSelector::select(6), -// ] -// ); -// -// let selection = selection.offset(2); -// assert_eq!( -// selection.selectors, -// vec![RowSelector::skip(68), RowSelector::select(6),] -// ); -// -// let selection = selection.offset(3); -// assert_eq!( -// selection.selectors, -// vec![RowSelector::skip(71), RowSelector::select(3),] -// ); -// } -// -// #[test] -// fn test_and() { -// let mut a = RowSelection::from(vec![ -// RowSelector::skip(12), -// RowSelector::select(23), -// RowSelector::skip(3), -// RowSelector::select(5), -// ]); -// -// let b = RowSelection::from(vec![ -// RowSelector::select(5), -// RowSelector::skip(4), -// RowSelector::select(15), -// RowSelector::skip(4), -// ]); -// -// let mut expected = RowSelection::from(vec![ -// RowSelector::skip(12), -// RowSelector::select(5), -// RowSelector::skip(4), -// RowSelector::select(14), -// RowSelector::skip(3), -// RowSelector::select(1), -// RowSelector::skip(4), -// ]); -// -// assert_eq!(a.and_then(&b), expected); -// -// a.split_off(7); -// expected.split_off(7); -// assert_eq!(a.and_then(&b), expected); -// -// let a = RowSelection::from(vec![RowSelector::select(5), RowSelector::skip(3)]); -// -// let b = RowSelection::from(vec![ -// RowSelector::select(2), -// RowSelector::skip(1), -// RowSelector::select(1), -// RowSelector::skip(1), -// ]); -// -// assert_eq!( -// a.and_then(&b).selectors, -// vec![ -// RowSelector::select(2), -// RowSelector::skip(1), -// RowSelector::select(1), -// RowSelector::skip(4) -// ] -// ); -// } -// -// #[test] -// fn test_combine() { -// let a = vec![ -// RowSelector::skip(3), -// RowSelector::skip(3), -// RowSelector::select(10), -// RowSelector::skip(4), -// ]; -// -// let b = vec![ -// RowSelector::skip(3), -// RowSelector::skip(3), -// RowSelector::select(10), -// RowSelector::skip(4), -// RowSelector::skip(0), -// ]; -// -// let c = vec![ -// RowSelector::skip(2), -// RowSelector::skip(4), -// RowSelector::select(3), -// RowSelector::select(3), -// RowSelector::select(4), -// RowSelector::skip(3), -// RowSelector::skip(1), -// RowSelector::skip(0), -// ]; -// -// let expected = RowSelection::from(vec![ -// RowSelector::skip(6), -// RowSelector::select(10), -// RowSelector::skip(4), -// ]); -// -// assert_eq!(RowSelection::from_iter(a), expected); -// assert_eq!(RowSelection::from_iter(b), expected); -// assert_eq!(RowSelection::from_iter(c), expected); -// } -// -// #[test] -// fn test_combine_2elements() { -// let a = vec![RowSelector::select(10), RowSelector::select(5)]; -// let a_expect = vec![RowSelector::select(15)]; -// assert_eq!(RowSelection::from_iter(a).selectors, a_expect); -// -// let b = vec![RowSelector::select(10), RowSelector::skip(5)]; -// let b_expect = vec![RowSelector::select(10), RowSelector::skip(5)]; -// assert_eq!(RowSelection::from_iter(b).selectors, b_expect); -// -// let c = vec![RowSelector::skip(10), RowSelector::select(5)]; -// let c_expect = vec![RowSelector::skip(10), RowSelector::select(5)]; -// assert_eq!(RowSelection::from_iter(c).selectors, c_expect); -// -// let d = vec![RowSelector::skip(10), RowSelector::skip(5)]; -// let d_expect = vec![RowSelector::skip(15)]; -// assert_eq!(RowSelection::from_iter(d).selectors, d_expect); -// } -// -// #[test] -// fn test_from_one_and_empty() { -// let a = vec![RowSelector::select(10)]; -// let selection1 = RowSelection::from(a.clone()); -// assert_eq!(selection1.selectors, a); -// -// let b = vec![]; -// let selection1 = RowSelection::from(b.clone()); -// assert_eq!(selection1.selectors, b) -// } -// -// #[test] -// #[should_panic(expected = "selection exceeds the number of selected rows")] -// fn test_and_longer() { -// let a = RowSelection::from(vec![ -// RowSelector::select(3), -// RowSelector::skip(33), -// RowSelector::select(3), -// RowSelector::skip(33), -// ]); -// let b = RowSelection::from(vec![RowSelector::select(36)]); -// a.and_then(&b); -// } -// -// #[test] -// #[should_panic(expected = "selection contains less than the number of selected rows")] -// fn test_and_shorter() { -// let a = RowSelection::from(vec![ -// RowSelector::select(3), -// RowSelector::skip(33), -// RowSelector::select(3), -// RowSelector::skip(33), -// ]); -// let b = RowSelection::from(vec![RowSelector::select(3)]); -// a.and_then(&b); -// } -// -// #[test] -// fn test_intersect_row_selection_and_combine() { -// // a size equal b size -// let a = vec![ -// RowSelector::select(5), -// RowSelector::skip(4), -// RowSelector::select(1), -// ]; -// let b = vec![ -// RowSelector::select(8), -// RowSelector::skip(1), -// RowSelector::select(1), -// ]; -// -// let res = intersect_row_selections(&a, &b); -// assert_eq!( -// res.selectors, -// vec![ -// RowSelector::select(5), -// RowSelector::skip(4), -// RowSelector::select(1), -// ], -// ); -// -// // a size larger than b size -// let a = vec![ -// RowSelector::select(3), -// RowSelector::skip(33), -// RowSelector::select(3), -// RowSelector::skip(33), -// ]; -// let b = vec![RowSelector::select(36), RowSelector::skip(36)]; -// let res = intersect_row_selections(&a, &b); -// assert_eq!( -// res.selectors, -// vec![RowSelector::select(3), RowSelector::skip(69)] -// ); -// -// // a size less than b size -// let a = vec![RowSelector::select(3), RowSelector::skip(7)]; -// let b = vec![ -// RowSelector::select(2), -// RowSelector::skip(2), -// RowSelector::select(2), -// RowSelector::skip(2), -// RowSelector::select(2), -// ]; -// let res = intersect_row_selections(&a, &b); -// assert_eq!( -// res.selectors, -// vec![RowSelector::select(2), RowSelector::skip(8)] -// ); -// -// let a = vec![RowSelector::select(3), RowSelector::skip(7)]; -// let b = vec![ -// RowSelector::select(2), -// RowSelector::skip(2), -// RowSelector::select(2), -// RowSelector::skip(2), -// RowSelector::select(2), -// ]; -// let res = intersect_row_selections(&a, &b); -// assert_eq!( -// res.selectors, -// vec![RowSelector::select(2), RowSelector::skip(8)] -// ); -// } -// -// #[test] -// fn test_and_fuzz() { -// let mut rand = rng(); -// for _ in 0..100 { -// let a_len = rand.random_range(10..100); -// let a_bools: Vec<_> = (0..a_len).map(|_| rand.random_bool(0.2)).collect(); -// let a = RowSelection::from_filters(&[BooleanArray::from(a_bools.clone())]); -// -// let b_len: usize = a_bools.iter().map(|x| *x as usize).sum(); -// let b_bools: Vec<_> = (0..b_len).map(|_| rand.random_bool(0.8)).collect(); -// let b = RowSelection::from_filters(&[BooleanArray::from(b_bools.clone())]); -// -// let mut expected_bools = vec![false; a_len]; -// -// let mut iter_b = b_bools.iter(); -// for (idx, b) in a_bools.iter().enumerate() { -// if *b && *iter_b.next().unwrap() { -// expected_bools[idx] = true; -// } -// } -// -// let expected = RowSelection::from_filters(&[BooleanArray::from(expected_bools)]); -// -// let total_rows: usize = expected.selectors.iter().map(|s| s.row_count).sum(); -// assert_eq!(a_len, total_rows); -// -// assert_eq!(a.and_then(&b), expected); -// } -// } -// -// #[test] -// fn test_iter() { -// // use the iter() API to show it does what is expected and -// // avoid accidental deletion -// let selectors = vec![ -// RowSelector::select(3), -// RowSelector::skip(33), -// RowSelector::select(4), -// ]; -// -// let round_tripped = RowSelection::from(selectors.clone()) -// .iter() -// .cloned() -// .collect::>(); -// assert_eq!(selectors, round_tripped); -// } -// -// #[test] -// fn test_limit() { -// // Limit to existing limit should no-op -// let selection = RowSelection::from(vec![RowSelector::select(10), RowSelector::skip(90)]); -// let limited = selection.limit(10); -// assert_eq!(RowSelection::from(vec![RowSelector::select(10)]), limited); -// -// let selection = RowSelection::from(vec![ -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(10), -// ]); -// -// let limited = selection.clone().limit(5); -// let expected = vec![RowSelector::select(5)]; -// assert_eq!(limited.selectors, expected); -// -// let limited = selection.clone().limit(15); -// let expected = vec![ -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(5), -// ]; -// assert_eq!(limited.selectors, expected); -// -// let limited = selection.clone().limit(0); -// let expected = vec![]; -// assert_eq!(limited.selectors, expected); -// -// let limited = selection.clone().limit(30); -// let expected = vec![ -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(10), -// ]; -// assert_eq!(limited.selectors, expected); -// -// let limited = selection.limit(100); -// let expected = vec![ -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(10), -// ]; -// assert_eq!(limited.selectors, expected); -// } -// -// #[test] -// fn test_scan_ranges() { -// let index = vec![ -// PageLocation { -// offset: 0, -// compressed_page_size: 10, -// first_row_index: 0, -// }, -// PageLocation { -// offset: 10, -// compressed_page_size: 10, -// first_row_index: 10, -// }, -// PageLocation { -// offset: 20, -// compressed_page_size: 10, -// first_row_index: 20, -// }, -// PageLocation { -// offset: 30, -// compressed_page_size: 10, -// first_row_index: 30, -// }, -// PageLocation { -// offset: 40, -// compressed_page_size: 10, -// first_row_index: 40, -// }, -// PageLocation { -// offset: 50, -// compressed_page_size: 10, -// first_row_index: 50, -// }, -// PageLocation { -// offset: 60, -// compressed_page_size: 10, -// first_row_index: 60, -// }, -// ]; -// -// let selection = RowSelection::from(vec![ -// // Skip first page -// RowSelector::skip(10), -// // Multiple selects in same page -// RowSelector::select(3), -// RowSelector::skip(3), -// RowSelector::select(4), -// // Select to page boundary -// RowSelector::skip(5), -// RowSelector::select(5), -// // Skip full page past page boundary -// RowSelector::skip(12), -// // Select across page boundaries -// RowSelector::select(12), -// // Skip final page -// RowSelector::skip(12), -// ]); -// -// let ranges = selection.scan_ranges(&index); -// -// // assert_eq!(mask, vec![false, true, true, false, true, true, false]); -// assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60]); -// -// let selection = RowSelection::from(vec![ -// // Skip first page -// RowSelector::skip(10), -// // Multiple selects in same page -// RowSelector::select(3), -// RowSelector::skip(3), -// RowSelector::select(4), -// // Select to page boundary -// RowSelector::skip(5), -// RowSelector::select(5), -// // Skip full page past page boundary -// RowSelector::skip(12), -// // Select across page boundaries -// RowSelector::select(12), -// RowSelector::skip(1), -// // Select across page boundaries including final page -// RowSelector::select(8), -// ]); -// -// let ranges = selection.scan_ranges(&index); -// -// // assert_eq!(mask, vec![false, true, true, false, true, true, true]); -// assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); -// -// let selection = RowSelection::from(vec![ -// // Skip first page -// RowSelector::skip(10), -// // Multiple selects in same page -// RowSelector::select(3), -// RowSelector::skip(3), -// RowSelector::select(4), -// // Select to page boundary -// RowSelector::skip(5), -// RowSelector::select(5), -// // Skip full page past page boundary -// RowSelector::skip(12), -// // Select to final page boundary -// RowSelector::select(12), -// RowSelector::skip(1), -// // Skip across final page boundary -// RowSelector::skip(8), -// // Select from final page -// RowSelector::select(4), -// ]); -// -// let ranges = selection.scan_ranges(&index); -// -// // assert_eq!(mask, vec![false, true, true, false, true, true, true]); -// assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); -// -// let selection = RowSelection::from(vec![ -// // Skip first page -// RowSelector::skip(10), -// // Multiple selects in same page -// RowSelector::select(3), -// RowSelector::skip(3), -// RowSelector::select(4), -// // Select to remaining in page and first row of next page -// RowSelector::skip(5), -// RowSelector::select(6), -// // Skip remaining -// RowSelector::skip(50), -// ]); -// -// let ranges = selection.scan_ranges(&index); -// -// // assert_eq!(mask, vec![false, true, true, false, true, true, true]); -// assert_eq!(ranges, vec![10..20, 20..30, 30..40]); -// } -// -// #[test] -// fn test_from_ranges() { -// let ranges = [1..3, 4..6, 6..6, 8..8, 9..10]; -// let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), 10); -// assert_eq!( -// selection.selectors, -// vec![ -// RowSelector::skip(1), -// RowSelector::select(2), -// RowSelector::skip(1), -// RowSelector::select(2), -// RowSelector::skip(3), -// RowSelector::select(1) -// ] -// ); -// -// let out_of_order_ranges = [1..3, 8..10, 4..7]; -// let result = std::panic::catch_unwind(|| { -// RowSelection::from_consecutive_ranges(out_of_order_ranges.into_iter(), 10) -// }); -// assert!(result.is_err()); -// } -// -// #[test] -// fn test_empty_selector() { -// let selection = RowSelection::from(vec![ -// RowSelector::skip(0), -// RowSelector::select(2), -// RowSelector::skip(0), -// RowSelector::select(2), -// ]); -// assert_eq!(selection.selectors, vec![RowSelector::select(4)]); -// -// let selection = RowSelection::from(vec![ -// RowSelector::select(0), -// RowSelector::skip(2), -// RowSelector::select(0), -// RowSelector::skip(2), -// ]); -// assert_eq!(selection.selectors, vec![RowSelector::skip(4)]); -// } -// -// #[test] -// fn test_intersection() { -// let selection = RowSelection::from(vec![RowSelector::select(1048576)]); -// let result = selection.intersection(&selection); -// assert_eq!(result, selection); -// -// let a = RowSelection::from(vec![ -// RowSelector::skip(10), -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(20), -// ]); -// -// let b = RowSelection::from(vec![ -// RowSelector::skip(20), -// RowSelector::select(20), -// RowSelector::skip(10), -// ]); -// -// let result = a.intersection(&b); -// assert_eq!( -// result.selectors, -// vec![ -// RowSelector::skip(30), -// RowSelector::select(10), -// RowSelector::skip(10) -// ] -// ); -// } -// -// #[test] -// fn test_union() { -// let selection = RowSelection::from(vec![RowSelector::select(1048576)]); -// let result = selection.union(&selection); -// assert_eq!(result, selection); -// -// // NYNYY -// let a = RowSelection::from(vec![ -// RowSelector::skip(10), -// RowSelector::select(10), -// RowSelector::skip(10), -// RowSelector::select(20), -// ]); -// -// // NNYYNYN -// let b = RowSelection::from(vec![ -// RowSelector::skip(20), -// RowSelector::select(20), -// RowSelector::skip(10), -// RowSelector::select(10), -// RowSelector::skip(10), -// ]); -// -// let result = a.union(&b); -// -// // NYYYYYN -// assert_eq!( -// result.iter().collect::>(), -// vec![ -// &RowSelector::skip(10), -// &RowSelector::select(50), -// &RowSelector::skip(10), -// ] -// ); -// } -// -// #[test] -// fn test_row_count() { -// let selection = RowSelection::from(vec![ -// RowSelector::skip(34), -// RowSelector::select(12), -// RowSelector::skip(3), -// RowSelector::select(35), -// ]); -// -// assert_eq!(selection.row_count(), 12 + 35); -// assert_eq!(selection.skipped_row_count(), 34 + 3); -// -// let selection = RowSelection::from(vec![RowSelector::select(12), RowSelector::select(35)]); -// -// assert_eq!(selection.row_count(), 12 + 35); -// assert_eq!(selection.skipped_row_count(), 0); -// -// let selection = RowSelection::from(vec![RowSelector::skip(34), RowSelector::skip(3)]); -// -// assert_eq!(selection.row_count(), 0); -// assert_eq!(selection.skipped_row_count(), 34 + 3); -// -// let selection = RowSelection::from(vec![]); -// -// assert_eq!(selection.row_count(), 0); -// assert_eq!(selection.skipped_row_count(), 0); -// } + // #[test] + // fn test_offset() { + // let selection = RowSelection::from(vec![ + // RowSelector::select(5), + // RowSelector::skip(23), + // RowSelector::select(7), + // RowSelector::skip(33), + // RowSelector::select(6), + // ]); + // + // let selection = selection.offset(2); + // assert_eq!( + // selection.selectors, + // vec![ + // RowSelector::skip(2), + // RowSelector::select(3), + // RowSelector::skip(23), + // RowSelector::select(7), + // RowSelector::skip(33), + // RowSelector::select(6), + // ] + // ); + // + // let selection = selection.offset(5); + // assert_eq!( + // selection.selectors, + // vec![ + // RowSelector::skip(30), + // RowSelector::select(5), + // RowSelector::skip(33), + // RowSelector::select(6), + // ] + // ); + // + // let selection = selection.offset(3); + // assert_eq!( + // selection.selectors, + // vec![ + // RowSelector::skip(33), + // RowSelector::select(2), + // RowSelector::skip(33), + // RowSelector::select(6), + // ] + // ); + // + // let selection = selection.offset(2); + // assert_eq!( + // selection.selectors, + // vec![RowSelector::skip(68), RowSelector::select(6),] + // ); + // + // let selection = selection.offset(3); + // assert_eq!( + // selection.selectors, + // vec![RowSelector::skip(71), RowSelector::select(3),] + // ); + // } + // + // #[test] + // fn test_and() { + // let mut a = RowSelection::from(vec![ + // RowSelector::skip(12), + // RowSelector::select(23), + // RowSelector::skip(3), + // RowSelector::select(5), + // ]); + // + // let b = RowSelection::from(vec![ + // RowSelector::select(5), + // RowSelector::skip(4), + // RowSelector::select(15), + // RowSelector::skip(4), + // ]); + // + // let mut expected = RowSelection::from(vec![ + // RowSelector::skip(12), + // RowSelector::select(5), + // RowSelector::skip(4), + // RowSelector::select(14), + // RowSelector::skip(3), + // RowSelector::select(1), + // RowSelector::skip(4), + // ]); + // + // assert_eq!(a.and_then(&b), expected); + // + // a.split_off(7); + // expected.split_off(7); + // assert_eq!(a.and_then(&b), expected); + // + // let a = RowSelection::from(vec![RowSelector::select(5), RowSelector::skip(3)]); + // + // let b = RowSelection::from(vec![ + // RowSelector::select(2), + // RowSelector::skip(1), + // RowSelector::select(1), + // RowSelector::skip(1), + // ]); + // + // assert_eq!( + // a.and_then(&b).selectors, + // vec![ + // RowSelector::select(2), + // RowSelector::skip(1), + // RowSelector::select(1), + // RowSelector::skip(4) + // ] + // ); + // } + // + // #[test] + // fn test_combine() { + // let a = vec![ + // RowSelector::skip(3), + // RowSelector::skip(3), + // RowSelector::select(10), + // RowSelector::skip(4), + // ]; + // + // let b = vec![ + // RowSelector::skip(3), + // RowSelector::skip(3), + // RowSelector::select(10), + // RowSelector::skip(4), + // RowSelector::skip(0), + // ]; + // + // let c = vec![ + // RowSelector::skip(2), + // RowSelector::skip(4), + // RowSelector::select(3), + // RowSelector::select(3), + // RowSelector::select(4), + // RowSelector::skip(3), + // RowSelector::skip(1), + // RowSelector::skip(0), + // ]; + // + // let expected = RowSelection::from(vec![ + // RowSelector::skip(6), + // RowSelector::select(10), + // RowSelector::skip(4), + // ]); + // + // assert_eq!(RowSelection::from_iter(a), expected); + // assert_eq!(RowSelection::from_iter(b), expected); + // assert_eq!(RowSelection::from_iter(c), expected); + // } + // + // #[test] + // fn test_combine_2elements() { + // let a = vec![RowSelector::select(10), RowSelector::select(5)]; + // let a_expect = vec![RowSelector::select(15)]; + // assert_eq!(RowSelection::from_iter(a).selectors, a_expect); + // + // let b = vec![RowSelector::select(10), RowSelector::skip(5)]; + // let b_expect = vec![RowSelector::select(10), RowSelector::skip(5)]; + // assert_eq!(RowSelection::from_iter(b).selectors, b_expect); + // + // let c = vec![RowSelector::skip(10), RowSelector::select(5)]; + // let c_expect = vec![RowSelector::skip(10), RowSelector::select(5)]; + // assert_eq!(RowSelection::from_iter(c).selectors, c_expect); + // + // let d = vec![RowSelector::skip(10), RowSelector::skip(5)]; + // let d_expect = vec![RowSelector::skip(15)]; + // assert_eq!(RowSelection::from_iter(d).selectors, d_expect); + // } + // + // #[test] + // fn test_from_one_and_empty() { + // let a = vec![RowSelector::select(10)]; + // let selection1 = RowSelection::from(a.clone()); + // assert_eq!(selection1.selectors, a); + // + // let b = vec![]; + // let selection1 = RowSelection::from(b.clone()); + // assert_eq!(selection1.selectors, b) + // } + // + // #[test] + // #[should_panic(expected = "selection exceeds the number of selected rows")] + // fn test_and_longer() { + // let a = RowSelection::from(vec![ + // RowSelector::select(3), + // RowSelector::skip(33), + // RowSelector::select(3), + // RowSelector::skip(33), + // ]); + // let b = RowSelection::from(vec![RowSelector::select(36)]); + // a.and_then(&b); + // } + // + // #[test] + // #[should_panic(expected = "selection contains less than the number of selected rows")] + // fn test_and_shorter() { + // let a = RowSelection::from(vec![ + // RowSelector::select(3), + // RowSelector::skip(33), + // RowSelector::select(3), + // RowSelector::skip(33), + // ]); + // let b = RowSelection::from(vec![RowSelector::select(3)]); + // a.and_then(&b); + // } + // + // #[test] + // fn test_intersect_row_selection_and_combine() { + // // a size equal b size + // let a = vec![ + // RowSelector::select(5), + // RowSelector::skip(4), + // RowSelector::select(1), + // ]; + // let b = vec![ + // RowSelector::select(8), + // RowSelector::skip(1), + // RowSelector::select(1), + // ]; + // + // let res = intersect_row_selections(&a, &b); + // assert_eq!( + // res.selectors, + // vec![ + // RowSelector::select(5), + // RowSelector::skip(4), + // RowSelector::select(1), + // ], + // ); + // + // // a size larger than b size + // let a = vec![ + // RowSelector::select(3), + // RowSelector::skip(33), + // RowSelector::select(3), + // RowSelector::skip(33), + // ]; + // let b = vec![RowSelector::select(36), RowSelector::skip(36)]; + // let res = intersect_row_selections(&a, &b); + // assert_eq!( + // res.selectors, + // vec![RowSelector::select(3), RowSelector::skip(69)] + // ); + // + // // a size less than b size + // let a = vec![RowSelector::select(3), RowSelector::skip(7)]; + // let b = vec![ + // RowSelector::select(2), + // RowSelector::skip(2), + // RowSelector::select(2), + // RowSelector::skip(2), + // RowSelector::select(2), + // ]; + // let res = intersect_row_selections(&a, &b); + // assert_eq!( + // res.selectors, + // vec![RowSelector::select(2), RowSelector::skip(8)] + // ); + // + // let a = vec![RowSelector::select(3), RowSelector::skip(7)]; + // let b = vec![ + // RowSelector::select(2), + // RowSelector::skip(2), + // RowSelector::select(2), + // RowSelector::skip(2), + // RowSelector::select(2), + // ]; + // let res = intersect_row_selections(&a, &b); + // assert_eq!( + // res.selectors, + // vec![RowSelector::select(2), RowSelector::skip(8)] + // ); + // } + // + // #[test] + // fn test_and_fuzz() { + // let mut rand = rng(); + // for _ in 0..100 { + // let a_len = rand.random_range(10..100); + // let a_bools: Vec<_> = (0..a_len).map(|_| rand.random_bool(0.2)).collect(); + // let a = RowSelection::from_filters(&[BooleanArray::from(a_bools.clone())]); + // + // let b_len: usize = a_bools.iter().map(|x| *x as usize).sum(); + // let b_bools: Vec<_> = (0..b_len).map(|_| rand.random_bool(0.8)).collect(); + // let b = RowSelection::from_filters(&[BooleanArray::from(b_bools.clone())]); + // + // let mut expected_bools = vec![false; a_len]; + // + // let mut iter_b = b_bools.iter(); + // for (idx, b) in a_bools.iter().enumerate() { + // if *b && *iter_b.next().unwrap() { + // expected_bools[idx] = true; + // } + // } + // + // let expected = RowSelection::from_filters(&[BooleanArray::from(expected_bools)]); + // + // let total_rows: usize = expected.selectors.iter().map(|s| s.row_count).sum(); + // assert_eq!(a_len, total_rows); + // + // assert_eq!(a.and_then(&b), expected); + // } + // } + // + // #[test] + // fn test_iter() { + // // use the iter() API to show it does what is expected and + // // avoid accidental deletion + // let selectors = vec![ + // RowSelector::select(3), + // RowSelector::skip(33), + // RowSelector::select(4), + // ]; + // + // let round_tripped = RowSelection::from(selectors.clone()) + // .iter() + // .cloned() + // .collect::>(); + // assert_eq!(selectors, round_tripped); + // } + // + // #[test] + // fn test_limit() { + // // Limit to existing limit should no-op + // let selection = RowSelection::from(vec![RowSelector::select(10), RowSelector::skip(90)]); + // let limited = selection.limit(10); + // assert_eq!(RowSelection::from(vec![RowSelector::select(10)]), limited); + // + // let selection = RowSelection::from(vec![ + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(10), + // ]); + // + // let limited = selection.clone().limit(5); + // let expected = vec![RowSelector::select(5)]; + // assert_eq!(limited.selectors, expected); + // + // let limited = selection.clone().limit(15); + // let expected = vec![ + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(5), + // ]; + // assert_eq!(limited.selectors, expected); + // + // let limited = selection.clone().limit(0); + // let expected = vec![]; + // assert_eq!(limited.selectors, expected); + // + // let limited = selection.clone().limit(30); + // let expected = vec![ + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(10), + // ]; + // assert_eq!(limited.selectors, expected); + // + // let limited = selection.limit(100); + // let expected = vec![ + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(10), + // ]; + // assert_eq!(limited.selectors, expected); + // } + // + // #[test] + // fn test_scan_ranges() { + // let index = vec![ + // PageLocation { + // offset: 0, + // compressed_page_size: 10, + // first_row_index: 0, + // }, + // PageLocation { + // offset: 10, + // compressed_page_size: 10, + // first_row_index: 10, + // }, + // PageLocation { + // offset: 20, + // compressed_page_size: 10, + // first_row_index: 20, + // }, + // PageLocation { + // offset: 30, + // compressed_page_size: 10, + // first_row_index: 30, + // }, + // PageLocation { + // offset: 40, + // compressed_page_size: 10, + // first_row_index: 40, + // }, + // PageLocation { + // offset: 50, + // compressed_page_size: 10, + // first_row_index: 50, + // }, + // PageLocation { + // offset: 60, + // compressed_page_size: 10, + // first_row_index: 60, + // }, + // ]; + // + // let selection = RowSelection::from(vec![ + // // Skip first page + // RowSelector::skip(10), + // // Multiple selects in same page + // RowSelector::select(3), + // RowSelector::skip(3), + // RowSelector::select(4), + // // Select to page boundary + // RowSelector::skip(5), + // RowSelector::select(5), + // // Skip full page past page boundary + // RowSelector::skip(12), + // // Select across page boundaries + // RowSelector::select(12), + // // Skip final page + // RowSelector::skip(12), + // ]); + // + // let ranges = selection.scan_ranges(&index); + // + // // assert_eq!(mask, vec![false, true, true, false, true, true, false]); + // assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60]); + // + // let selection = RowSelection::from(vec![ + // // Skip first page + // RowSelector::skip(10), + // // Multiple selects in same page + // RowSelector::select(3), + // RowSelector::skip(3), + // RowSelector::select(4), + // // Select to page boundary + // RowSelector::skip(5), + // RowSelector::select(5), + // // Skip full page past page boundary + // RowSelector::skip(12), + // // Select across page boundaries + // RowSelector::select(12), + // RowSelector::skip(1), + // // Select across page boundaries including final page + // RowSelector::select(8), + // ]); + // + // let ranges = selection.scan_ranges(&index); + // + // // assert_eq!(mask, vec![false, true, true, false, true, true, true]); + // assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); + // + // let selection = RowSelection::from(vec![ + // // Skip first page + // RowSelector::skip(10), + // // Multiple selects in same page + // RowSelector::select(3), + // RowSelector::skip(3), + // RowSelector::select(4), + // // Select to page boundary + // RowSelector::skip(5), + // RowSelector::select(5), + // // Skip full page past page boundary + // RowSelector::skip(12), + // // Select to final page boundary + // RowSelector::select(12), + // RowSelector::skip(1), + // // Skip across final page boundary + // RowSelector::skip(8), + // // Select from final page + // RowSelector::select(4), + // ]); + // + // let ranges = selection.scan_ranges(&index); + // + // // assert_eq!(mask, vec![false, true, true, false, true, true, true]); + // assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); + // + // let selection = RowSelection::from(vec![ + // // Skip first page + // RowSelector::skip(10), + // // Multiple selects in same page + // RowSelector::select(3), + // RowSelector::skip(3), + // RowSelector::select(4), + // // Select to remaining in page and first row of next page + // RowSelector::skip(5), + // RowSelector::select(6), + // // Skip remaining + // RowSelector::skip(50), + // ]); + // + // let ranges = selection.scan_ranges(&index); + // + // // assert_eq!(mask, vec![false, true, true, false, true, true, true]); + // assert_eq!(ranges, vec![10..20, 20..30, 30..40]); + // } + // + // #[test] + // fn test_from_ranges() { + // let ranges = [1..3, 4..6, 6..6, 8..8, 9..10]; + // let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), 10); + // assert_eq!( + // selection.selectors, + // vec![ + // RowSelector::skip(1), + // RowSelector::select(2), + // RowSelector::skip(1), + // RowSelector::select(2), + // RowSelector::skip(3), + // RowSelector::select(1) + // ] + // ); + // + // let out_of_order_ranges = [1..3, 8..10, 4..7]; + // let result = std::panic::catch_unwind(|| { + // RowSelection::from_consecutive_ranges(out_of_order_ranges.into_iter(), 10) + // }); + // assert!(result.is_err()); + // } + // + // #[test] + // fn test_empty_selector() { + // let selection = RowSelection::from(vec![ + // RowSelector::skip(0), + // RowSelector::select(2), + // RowSelector::skip(0), + // RowSelector::select(2), + // ]); + // assert_eq!(selection.selectors, vec![RowSelector::select(4)]); + // + // let selection = RowSelection::from(vec![ + // RowSelector::select(0), + // RowSelector::skip(2), + // RowSelector::select(0), + // RowSelector::skip(2), + // ]); + // assert_eq!(selection.selectors, vec![RowSelector::skip(4)]); + // } + // + // #[test] + // fn test_intersection() { + // let selection = RowSelection::from(vec![RowSelector::select(1048576)]); + // let result = selection.intersection(&selection); + // assert_eq!(result, selection); + // + // let a = RowSelection::from(vec![ + // RowSelector::skip(10), + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(20), + // ]); + // + // let b = RowSelection::from(vec![ + // RowSelector::skip(20), + // RowSelector::select(20), + // RowSelector::skip(10), + // ]); + // + // let result = a.intersection(&b); + // assert_eq!( + // result.selectors, + // vec![ + // RowSelector::skip(30), + // RowSelector::select(10), + // RowSelector::skip(10) + // ] + // ); + // } + // + // #[test] + // fn test_union() { + // let selection = RowSelection::from(vec![RowSelector::select(1048576)]); + // let result = selection.union(&selection); + // assert_eq!(result, selection); + // + // // NYNYY + // let a = RowSelection::from(vec![ + // RowSelector::skip(10), + // RowSelector::select(10), + // RowSelector::skip(10), + // RowSelector::select(20), + // ]); + // + // // NNYYNYN + // let b = RowSelection::from(vec![ + // RowSelector::skip(20), + // RowSelector::select(20), + // RowSelector::skip(10), + // RowSelector::select(10), + // RowSelector::skip(10), + // ]); + // + // let result = a.union(&b); + // + // // NYYYYYN + // assert_eq!( + // result.iter().collect::>(), + // vec![ + // &RowSelector::skip(10), + // &RowSelector::select(50), + // &RowSelector::skip(10), + // ] + // ); + // } + // + // #[test] + // fn test_row_count() { + // let selection = RowSelection::from(vec![ + // RowSelector::skip(34), + // RowSelector::select(12), + // RowSelector::skip(3), + // RowSelector::select(35), + // ]); + // + // assert_eq!(selection.row_count(), 12 + 35); + // assert_eq!(selection.skipped_row_count(), 34 + 3); + // + // let selection = RowSelection::from(vec![RowSelector::select(12), RowSelector::select(35)]); + // + // assert_eq!(selection.row_count(), 12 + 35); + // assert_eq!(selection.skipped_row_count(), 0); + // + // let selection = RowSelection::from(vec![RowSelector::skip(34), RowSelector::skip(3)]); + // + // assert_eq!(selection.row_count(), 0); + // assert_eq!(selection.skipped_row_count(), 34 + 3); + // + // let selection = RowSelection::from(vec![]); + // + // assert_eq!(selection.row_count(), 0); + // assert_eq!(selection.skipped_row_count(), 0); + // } } diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 2c71822d1f82..313e6395e871 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -20,8 +20,8 @@ use std::collections::HashMap; use std::sync::{Mutex, MutexGuard}; use std::{collections::VecDeque, sync::Arc}; -use arrow_array::{ArrayRef, BooleanArray}; use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; +use arrow_array::{ArrayRef, BooleanArray}; use arrow_buffer::BooleanBufferBuilder; use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; use arrow_select::filter::{filter, filter_record_batch, prep_null_mask_filter}; @@ -60,15 +60,15 @@ fn read_selection( } } reader.consume_batch() - }, + } RowSelection::BitMap(bitmap) => { let to_read = bitmap.len(); reader.read_records(to_read)?; let array = reader.consume_batch()?; - let filtered_array = filter(&array, bitmap) - .map_err(|e| ParquetError::General(e.to_string()))?; + let filtered_array = + filter(&array, bitmap).map_err(|e| ParquetError::General(e.to_string()))?; Ok(filtered_array) } @@ -94,7 +94,9 @@ fn take_next_selection( return Some(RowSelection::BitMap(prefix)); } - let RowSelection::Ranges(runs) = current else { unreachable!() }; + let RowSelection::Ranges(runs) = current else { + unreachable!() + }; let mut queue: VecDeque = runs.into(); let mut taken = Vec::new(); let mut count = 0; @@ -128,8 +130,6 @@ fn take_next_selection( Some(RowSelection::Ranges(taken.into())) } - - impl FilteredParquetRecordBatchReader { pub(crate) fn new( batch_size: usize, @@ -229,7 +229,9 @@ impl Iterator for FilteredParquetRecordBatchReader { // Move acc_skip here so it persists across loop iterations let mut acc_skip = 0; - while let Some(raw_sel) = take_next_selection(&mut self.selection, self.batch_size - rows_accum) { + while let Some(raw_sel) = + take_next_selection(&mut self.selection, self.batch_size - rows_accum) + { let sel = match self.build_predicate_filter(raw_sel) { Ok(s) => s, Err(e) => return Some(Err(e)), @@ -241,8 +243,11 @@ impl Iterator for FilteredParquetRecordBatchReader { let mut total_skip = 0; let mut total_read = 0; for r in &runs { - if r.skip { total_skip += r.row_count; } - else { total_read += r.row_count; } + if r.skip { + total_skip += r.row_count; + } else { + total_read += r.row_count; + } } // If nothing to read, accumulate skip and continue @@ -327,15 +332,11 @@ impl Iterator for FilteredParquetRecordBatchReader { .ok()? }; - let struct_arr = final_array - .as_struct_opt() - .expect("StructArray expected"); + let struct_arr = final_array.as_struct_opt().expect("StructArray expected"); Some(Ok(RecordBatch::from(struct_arr.clone()))) } } - - impl RecordBatchReader for FilteredParquetRecordBatchReader { fn schema(&self) -> SchemaRef { self.schema.clone() @@ -343,7 +344,7 @@ impl RecordBatchReader for FilteredParquetRecordBatchReader { } struct CachedPage { - dict: Option<(usize, Page)>, // page offset -> page + dict: Option<(usize, Page)>, // page offset -> page data: VecDeque<(usize, Page)>, // page offset -> page, use 2 pages, because the batch size will exceed the page size sometimes } @@ -354,14 +355,17 @@ struct PredicatePageCacheInner { impl PredicatePageCacheInner { pub(crate) fn get_page(&self, col_id: usize, offset: usize) -> Option { self.pages.get(&col_id).and_then(|pages| { - if let Some((off, page)) = &pages.dict { if *off == offset { return Some(page.clone()); } } - pages.data.iter().find(|(off, _)| *off == offset).map(|(_, page)| page.clone()) + pages + .data + .iter() + .find(|(off, _)| *off == offset) + .map(|(_, page)| page.clone()) }) } @@ -375,12 +379,7 @@ impl PredicatePageCacheInner { /// shows that 3 pages are enough to cover the batch size when we're setting batch size to 8192. And the 3 data page size /// is not too large, it only uses 3MB in memory, so we can keep 3 pages in the cache. /// TODO, in future we may use adaptive cache size according the dynamic batch size. - pub(crate) fn insert_page( - &mut self, - col_id: usize, - offset: usize, - page: Page, - ) { + pub(crate) fn insert_page(&mut self, col_id: usize, offset: usize, page: Page) { let is_dict = page.page_type() == PageType::DICTIONARY_PAGE; match self.pages.entry(col_id) { @@ -588,17 +587,18 @@ mod tests { #[test] fn test_take_next_selection_exact_match() { - let mut queue = VecDeque::from(vec![ + let selection_vec = vec![ RowSelector::skip(5), RowSelector::select(3), RowSelector::skip(2), RowSelector::select(7), - ]); + ]; + let mut selection = Some(RowSelection::Ranges(selection_vec)); // Request exactly 10 rows (5 skip + 3 select + 2 skip) - let selection = take_next_selection(&mut queue, 3).unwrap(); + let result = take_next_selection(&mut selection, 3).unwrap(); assert_eq!( - selection, + result, vec![ RowSelector::skip(5), RowSelector::select(3), @@ -607,46 +607,46 @@ mod tests { .into() ); - // Check remaining queue - assert_eq!(queue.len(), 1); - assert_eq!(queue[0].row_count, 7); - assert!(!queue[0].skip); + // Remaining: select(7) + assert_eq!( + selection, + Some(RowSelection::Ranges(vec![RowSelector::select(7)])) + ); } #[test] fn test_take_next_selection_split_required() { - let mut queue = VecDeque::from(vec![RowSelector::select(10), RowSelector::select(10)]); + let mut selection = Some(RowSelection::Ranges(vec![ + RowSelector::select(10), + RowSelector::select(10), + ])); - // Request 15 rows, which should split the first selector - let selection = take_next_selection(&mut queue, 15).unwrap(); + // Request 15 rows: should take 10 + 5, leave 5 + let result = take_next_selection(&mut selection, 15).unwrap(); + assert_eq!( + result, + RowSelection::Ranges(vec![RowSelector::select(10), RowSelector::select(5),]) + ); + // Remaining: select(5) assert_eq!( selection, - vec![RowSelector::select(10), RowSelector::select(5)].into() + Some(RowSelection::Ranges(vec![RowSelector::select(5)])) ); - - // Check remaining queue - should have 5 rows from split and original 10 - assert_eq!(queue.len(), 1); - assert!(!queue[0].skip); - assert_eq!(queue[0].row_count, 5); } #[test] fn test_take_next_selection_empty_queue() { - let mut queue = VecDeque::new(); - - // Should return None for empty queue - let selection = take_next_selection(&mut queue, 10); - assert!(selection.is_none()); + let mut selection = None; - // Test with queue that becomes empty - queue.push_back(RowSelector::select(5)); - let selection = take_next_selection(&mut queue, 10).unwrap(); - assert_eq!(selection, vec![RowSelector::select(5)].into()); + // Empty selection + assert!(take_next_selection(&mut selection, 10).is_none()); - // Queue should now be empty - let selection = take_next_selection(&mut queue, 10); - assert!(selection.is_none()); + // One item, smaller than to_select + selection = Some(RowSelection::Ranges(vec![RowSelector::select(5)])); + let result = take_next_selection(&mut selection, 10).unwrap(); + assert_eq!(result, RowSelection::Ranges(vec![RowSelector::select(5)])); + assert_eq!(selection, None); } #[test] diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 345022c77655..d1bc32154a16 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -659,8 +659,6 @@ where .fetch(&mut self.input, &projection, Some(&selection)) .await?; - - // let total_rows = selection.total_rows(); // // let selection = match selection { @@ -688,9 +686,7 @@ where // // } // _ => None, // }; - - let array_reader = build_array_reader(self.fields.as_deref(), &projection, &row_group)?; let reader = FilteredParquetRecordBatchReader::new( batch_size, diff --git a/parquet/src/file/serialized_reader.rs b/parquet/src/file/serialized_reader.rs index 6501950fbb23..94b003358309 100644 --- a/parquet/src/file/serialized_reader.rs +++ b/parquet/src/file/serialized_reader.rs @@ -999,7 +999,9 @@ impl PageReader for SerializedPageReader { SerializedPageReaderState::Pages { page_locations, - dictionary_page, .. } => { + dictionary_page, + .. + } => { if dictionary_page.is_some() { // If a dictionary page exists, consume it by taking it (sets to None) dictionary_page.take(); @@ -1343,7 +1345,7 @@ mod tests { } assert_eq!(page_count, 2); } - + #[test] fn test_file_reader_empty_compressed_datapage_v2() { // this file has a compressed datapage that un-compresses to 0 bytes @@ -1535,7 +1537,7 @@ mod tests { } assert_eq!(page_count, 1); } - + #[cfg(feature = "async")] fn get_serialized_page_reader( file_reader: &SerializedFileReader, From efcc0de297411a3420379477840e075d38a7021b Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 20:59:02 +0800 Subject: [PATCH 65/73] Add new testing --- parquet/src/arrow/arrow_reader/selection.rs | 1282 ++++++++++--------- 1 file changed, 642 insertions(+), 640 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 8fcc5595ab4e..a8c8fcc81fae 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -115,6 +115,15 @@ impl Default for RowSelection { } impl RowSelection { + /// Returns the [`RowSelector`]s for this [`RowSelection`] + /// It only works for [`RowSelection::Ranges`] + pub fn selectors(&self) -> &[RowSelector] { + match self { + RowSelection::Ranges(selectors) => selectors, + RowSelection::BitMap(_) => panic!("called selectors() on RowSelection::BitMap"), + } + } + /// Creates a [`RowSelection`] from a slice of [`BooleanArray`] /// /// # Panic @@ -844,6 +853,8 @@ fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec = (0..a_len).map(|_| rand.random_bool(0.2)).collect(); - // let a = RowSelection::from_filters(&[BooleanArray::from(a_bools.clone())]); - // - // let b_len: usize = a_bools.iter().map(|x| *x as usize).sum(); - // let b_bools: Vec<_> = (0..b_len).map(|_| rand.random_bool(0.8)).collect(); - // let b = RowSelection::from_filters(&[BooleanArray::from(b_bools.clone())]); - // - // let mut expected_bools = vec![false; a_len]; - // - // let mut iter_b = b_bools.iter(); - // for (idx, b) in a_bools.iter().enumerate() { - // if *b && *iter_b.next().unwrap() { - // expected_bools[idx] = true; - // } - // } - // - // let expected = RowSelection::from_filters(&[BooleanArray::from(expected_bools)]); - // - // let total_rows: usize = expected.selectors.iter().map(|s| s.row_count).sum(); - // assert_eq!(a_len, total_rows); - // - // assert_eq!(a.and_then(&b), expected); - // } - // } - // - // #[test] - // fn test_iter() { - // // use the iter() API to show it does what is expected and - // // avoid accidental deletion - // let selectors = vec![ - // RowSelector::select(3), - // RowSelector::skip(33), - // RowSelector::select(4), - // ]; - // - // let round_tripped = RowSelection::from(selectors.clone()) - // .iter() - // .cloned() - // .collect::>(); - // assert_eq!(selectors, round_tripped); - // } - // - // #[test] - // fn test_limit() { - // // Limit to existing limit should no-op - // let selection = RowSelection::from(vec![RowSelector::select(10), RowSelector::skip(90)]); - // let limited = selection.limit(10); - // assert_eq!(RowSelection::from(vec![RowSelector::select(10)]), limited); - // - // let selection = RowSelection::from(vec![ - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(10), - // ]); - // - // let limited = selection.clone().limit(5); - // let expected = vec![RowSelector::select(5)]; - // assert_eq!(limited.selectors, expected); - // - // let limited = selection.clone().limit(15); - // let expected = vec![ - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(5), - // ]; - // assert_eq!(limited.selectors, expected); - // - // let limited = selection.clone().limit(0); - // let expected = vec![]; - // assert_eq!(limited.selectors, expected); - // - // let limited = selection.clone().limit(30); - // let expected = vec![ - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(10), - // ]; - // assert_eq!(limited.selectors, expected); - // - // let limited = selection.limit(100); - // let expected = vec![ - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(10), - // ]; - // assert_eq!(limited.selectors, expected); - // } - // - // #[test] - // fn test_scan_ranges() { - // let index = vec![ - // PageLocation { - // offset: 0, - // compressed_page_size: 10, - // first_row_index: 0, - // }, - // PageLocation { - // offset: 10, - // compressed_page_size: 10, - // first_row_index: 10, - // }, - // PageLocation { - // offset: 20, - // compressed_page_size: 10, - // first_row_index: 20, - // }, - // PageLocation { - // offset: 30, - // compressed_page_size: 10, - // first_row_index: 30, - // }, - // PageLocation { - // offset: 40, - // compressed_page_size: 10, - // first_row_index: 40, - // }, - // PageLocation { - // offset: 50, - // compressed_page_size: 10, - // first_row_index: 50, - // }, - // PageLocation { - // offset: 60, - // compressed_page_size: 10, - // first_row_index: 60, - // }, - // ]; - // - // let selection = RowSelection::from(vec![ - // // Skip first page - // RowSelector::skip(10), - // // Multiple selects in same page - // RowSelector::select(3), - // RowSelector::skip(3), - // RowSelector::select(4), - // // Select to page boundary - // RowSelector::skip(5), - // RowSelector::select(5), - // // Skip full page past page boundary - // RowSelector::skip(12), - // // Select across page boundaries - // RowSelector::select(12), - // // Skip final page - // RowSelector::skip(12), - // ]); - // - // let ranges = selection.scan_ranges(&index); - // - // // assert_eq!(mask, vec![false, true, true, false, true, true, false]); - // assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60]); - // - // let selection = RowSelection::from(vec![ - // // Skip first page - // RowSelector::skip(10), - // // Multiple selects in same page - // RowSelector::select(3), - // RowSelector::skip(3), - // RowSelector::select(4), - // // Select to page boundary - // RowSelector::skip(5), - // RowSelector::select(5), - // // Skip full page past page boundary - // RowSelector::skip(12), - // // Select across page boundaries - // RowSelector::select(12), - // RowSelector::skip(1), - // // Select across page boundaries including final page - // RowSelector::select(8), - // ]); - // - // let ranges = selection.scan_ranges(&index); - // - // // assert_eq!(mask, vec![false, true, true, false, true, true, true]); - // assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); - // - // let selection = RowSelection::from(vec![ - // // Skip first page - // RowSelector::skip(10), - // // Multiple selects in same page - // RowSelector::select(3), - // RowSelector::skip(3), - // RowSelector::select(4), - // // Select to page boundary - // RowSelector::skip(5), - // RowSelector::select(5), - // // Skip full page past page boundary - // RowSelector::skip(12), - // // Select to final page boundary - // RowSelector::select(12), - // RowSelector::skip(1), - // // Skip across final page boundary - // RowSelector::skip(8), - // // Select from final page - // RowSelector::select(4), - // ]); - // - // let ranges = selection.scan_ranges(&index); - // - // // assert_eq!(mask, vec![false, true, true, false, true, true, true]); - // assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); - // - // let selection = RowSelection::from(vec![ - // // Skip first page - // RowSelector::skip(10), - // // Multiple selects in same page - // RowSelector::select(3), - // RowSelector::skip(3), - // RowSelector::select(4), - // // Select to remaining in page and first row of next page - // RowSelector::skip(5), - // RowSelector::select(6), - // // Skip remaining - // RowSelector::skip(50), - // ]); - // - // let ranges = selection.scan_ranges(&index); - // - // // assert_eq!(mask, vec![false, true, true, false, true, true, true]); - // assert_eq!(ranges, vec![10..20, 20..30, 30..40]); - // } - // - // #[test] - // fn test_from_ranges() { - // let ranges = [1..3, 4..6, 6..6, 8..8, 9..10]; - // let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), 10); - // assert_eq!( - // selection.selectors, - // vec![ - // RowSelector::skip(1), - // RowSelector::select(2), - // RowSelector::skip(1), - // RowSelector::select(2), - // RowSelector::skip(3), - // RowSelector::select(1) - // ] - // ); - // - // let out_of_order_ranges = [1..3, 8..10, 4..7]; - // let result = std::panic::catch_unwind(|| { - // RowSelection::from_consecutive_ranges(out_of_order_ranges.into_iter(), 10) - // }); - // assert!(result.is_err()); - // } - // - // #[test] - // fn test_empty_selector() { - // let selection = RowSelection::from(vec![ - // RowSelector::skip(0), - // RowSelector::select(2), - // RowSelector::skip(0), - // RowSelector::select(2), - // ]); - // assert_eq!(selection.selectors, vec![RowSelector::select(4)]); - // - // let selection = RowSelection::from(vec![ - // RowSelector::select(0), - // RowSelector::skip(2), - // RowSelector::select(0), - // RowSelector::skip(2), - // ]); - // assert_eq!(selection.selectors, vec![RowSelector::skip(4)]); - // } - // - // #[test] - // fn test_intersection() { - // let selection = RowSelection::from(vec![RowSelector::select(1048576)]); - // let result = selection.intersection(&selection); - // assert_eq!(result, selection); - // - // let a = RowSelection::from(vec![ - // RowSelector::skip(10), - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(20), - // ]); - // - // let b = RowSelection::from(vec![ - // RowSelector::skip(20), - // RowSelector::select(20), - // RowSelector::skip(10), - // ]); - // - // let result = a.intersection(&b); - // assert_eq!( - // result.selectors, - // vec![ - // RowSelector::skip(30), - // RowSelector::select(10), - // RowSelector::skip(10) - // ] - // ); - // } - // - // #[test] - // fn test_union() { - // let selection = RowSelection::from(vec![RowSelector::select(1048576)]); - // let result = selection.union(&selection); - // assert_eq!(result, selection); - // - // // NYNYY - // let a = RowSelection::from(vec![ - // RowSelector::skip(10), - // RowSelector::select(10), - // RowSelector::skip(10), - // RowSelector::select(20), - // ]); - // - // // NNYYNYN - // let b = RowSelection::from(vec![ - // RowSelector::skip(20), - // RowSelector::select(20), - // RowSelector::skip(10), - // RowSelector::select(10), - // RowSelector::skip(10), - // ]); - // - // let result = a.union(&b); - // - // // NYYYYYN - // assert_eq!( - // result.iter().collect::>(), - // vec![ - // &RowSelector::skip(10), - // &RowSelector::select(50), - // &RowSelector::skip(10), - // ] - // ); - // } - // - // #[test] - // fn test_row_count() { - // let selection = RowSelection::from(vec![ - // RowSelector::skip(34), - // RowSelector::select(12), - // RowSelector::skip(3), - // RowSelector::select(35), - // ]); - // - // assert_eq!(selection.row_count(), 12 + 35); - // assert_eq!(selection.skipped_row_count(), 34 + 3); - // - // let selection = RowSelection::from(vec![RowSelector::select(12), RowSelector::select(35)]); - // - // assert_eq!(selection.row_count(), 12 + 35); - // assert_eq!(selection.skipped_row_count(), 0); - // - // let selection = RowSelection::from(vec![RowSelector::skip(34), RowSelector::skip(3)]); - // - // assert_eq!(selection.row_count(), 0); - // assert_eq!(selection.skipped_row_count(), 34 + 3); - // - // let selection = RowSelection::from(vec![]); - // - // assert_eq!(selection.row_count(), 0); - // assert_eq!(selection.skipped_row_count(), 0); - // } + #[test] + fn test_offset() { + let selection = RowSelection::from(vec![ + RowSelector::select(5), + RowSelector::skip(23), + RowSelector::select(7), + RowSelector::skip(33), + RowSelector::select(6), + ]); + + let selection = selection.offset(2); + assert_eq!( + selection.selectors(), + vec![ + RowSelector::skip(2), + RowSelector::select(3), + RowSelector::skip(23), + RowSelector::select(7), + RowSelector::skip(33), + RowSelector::select(6), + ] + ); + + let selection = selection.offset(5); + assert_eq!( + selection.selectors(), + vec![ + RowSelector::skip(30), + RowSelector::select(5), + RowSelector::skip(33), + RowSelector::select(6), + ] + ); + + let selection = selection.offset(3); + assert_eq!( + selection.selectors(), + vec![ + RowSelector::skip(33), + RowSelector::select(2), + RowSelector::skip(33), + RowSelector::select(6), + ] + ); + + let selection = selection.offset(2); + assert_eq!( + selection.selectors(), + vec![RowSelector::skip(68), RowSelector::select(6),] + ); + + let selection = selection.offset(3); + assert_eq!( + selection.selectors(), + vec![RowSelector::skip(71), RowSelector::select(3),] + ); + } + + #[test] + fn test_and() { + let mut a = RowSelection::from(vec![ + RowSelector::skip(12), + RowSelector::select(23), + RowSelector::skip(3), + RowSelector::select(5), + ]); + + let b = RowSelection::from(vec![ + RowSelector::select(5), + RowSelector::skip(4), + RowSelector::select(15), + RowSelector::skip(4), + ]); + + let mut expected = RowSelection::from(vec![ + RowSelector::skip(12), + RowSelector::select(5), + RowSelector::skip(4), + RowSelector::select(14), + RowSelector::skip(3), + RowSelector::select(1), + RowSelector::skip(4), + ]); + + assert_eq!(a.and_then(&b), expected); + + a.split_off(7); + expected.split_off(7); + assert_eq!(a.and_then(&b), expected); + + let a = RowSelection::from(vec![RowSelector::select(5), RowSelector::skip(3)]); + + let b = RowSelection::from(vec![ + RowSelector::select(2), + RowSelector::skip(1), + RowSelector::select(1), + RowSelector::skip(1), + ]); + + assert_eq!( + a.and_then(&b).selectors(), + vec![ + RowSelector::select(2), + RowSelector::skip(1), + RowSelector::select(1), + RowSelector::skip(4) + ] + ); + } + + #[test] + fn test_combine() { + let a = vec![ + RowSelector::skip(3), + RowSelector::skip(3), + RowSelector::select(10), + RowSelector::skip(4), + ]; + + let b = vec![ + RowSelector::skip(3), + RowSelector::skip(3), + RowSelector::select(10), + RowSelector::skip(4), + RowSelector::skip(0), + ]; + + let c = vec![ + RowSelector::skip(2), + RowSelector::skip(4), + RowSelector::select(3), + RowSelector::select(3), + RowSelector::select(4), + RowSelector::skip(3), + RowSelector::skip(1), + RowSelector::skip(0), + ]; + + let expected = RowSelection::from(vec![ + RowSelector::skip(6), + RowSelector::select(10), + RowSelector::skip(4), + ]); + + assert_eq!(RowSelection::from_iter(a), expected); + assert_eq!(RowSelection::from_iter(b), expected); + assert_eq!(RowSelection::from_iter(c), expected); + } + + #[test] + fn test_combine_2elements() { + let a = vec![RowSelector::select(10), RowSelector::select(5)]; + let a_expect = vec![RowSelector::select(15)]; + assert_eq!(RowSelection::from_iter(a).selectors(), a_expect); + + let b = vec![RowSelector::select(10), RowSelector::skip(5)]; + let b_expect = vec![RowSelector::select(10), RowSelector::skip(5)]; + assert_eq!(RowSelection::from_iter(b).selectors(), b_expect); + + let c = vec![RowSelector::skip(10), RowSelector::select(5)]; + let c_expect = vec![RowSelector::skip(10), RowSelector::select(5)]; + assert_eq!(RowSelection::from_iter(c).selectors(), c_expect); + + let d = vec![RowSelector::skip(10), RowSelector::skip(5)]; + let d_expect = vec![RowSelector::skip(15)]; + assert_eq!(RowSelection::from_iter(d).selectors(), d_expect); + } + + #[test] + fn test_from_one_and_empty() { + let a = vec![RowSelector::select(10)]; + let selection1 = RowSelection::from(a.clone()); + assert_eq!(selection1.selectors(), a); + + let b = vec![]; + let selection1 = RowSelection::from(b.clone()); + assert_eq!(selection1.selectors(), b) + } + + #[test] + #[should_panic(expected = "selection exceeds the number of selected rows")] + fn test_and_longer() { + let a = RowSelection::from(vec![ + RowSelector::select(3), + RowSelector::skip(33), + RowSelector::select(3), + RowSelector::skip(33), + ]); + let b = RowSelection::from(vec![RowSelector::select(36)]); + a.and_then(&b); + } + + #[test] + #[should_panic(expected = "selection contains less than the number of selected rows")] + fn test_and_shorter() { + let a = RowSelection::from(vec![ + RowSelector::select(3), + RowSelector::skip(33), + RowSelector::select(3), + RowSelector::skip(33), + ]); + let b = RowSelection::from(vec![RowSelector::select(3)]); + a.and_then(&b); + } + + #[test] + fn test_intersect_row_selection_and_combine() { + // a size equal b size + let a = vec![ + RowSelector::select(5), + RowSelector::skip(4), + RowSelector::select(1), + ]; + let b = vec![ + RowSelector::select(8), + RowSelector::skip(1), + RowSelector::select(1), + ]; + + let res = intersect_row_selections(&a, &b); + assert_eq!( + res, + vec![ + RowSelector::select(5), + RowSelector::skip(4), + RowSelector::select(1), + ], + ); + + // a size larger than b size + let a = vec![ + RowSelector::select(3), + RowSelector::skip(33), + RowSelector::select(3), + RowSelector::skip(33), + ]; + let b = vec![RowSelector::select(36), RowSelector::skip(36)]; + let res = intersect_row_selections(&a, &b); + assert_eq!(res, vec![RowSelector::select(3), RowSelector::skip(69)]); + + // a size less than b size + let a = vec![RowSelector::select(3), RowSelector::skip(7)]; + let b = vec![ + RowSelector::select(2), + RowSelector::skip(2), + RowSelector::select(2), + RowSelector::skip(2), + RowSelector::select(2), + ]; + let res = intersect_row_selections(&a, &b); + assert_eq!(res, vec![RowSelector::select(2), RowSelector::skip(8)]); + + let a = vec![RowSelector::select(3), RowSelector::skip(7)]; + let b = vec![ + RowSelector::select(2), + RowSelector::skip(2), + RowSelector::select(2), + RowSelector::skip(2), + RowSelector::select(2), + ]; + let res = intersect_row_selections(&a, &b); + assert_eq!(res, vec![RowSelector::select(2), RowSelector::skip(8)]); + } + + #[test] + fn test_and_fuzz() { + let mut rand = rng(); + for _ in 0..100 { + let a_len = rand.random_range(10..100); + let a_bools: Vec<_> = (0..a_len).map(|_| rand.random_bool(0.2)).collect(); + let a = RowSelection::from_filters(&[BooleanArray::from(a_bools.clone())]); + + let b_len: usize = a_bools.iter().map(|x| *x as usize).sum(); + let b_bools: Vec<_> = (0..b_len).map(|_| rand.random_bool(0.8)).collect(); + let b = RowSelection::from_filters(&[BooleanArray::from(b_bools.clone())]); + + let mut expected_bools = vec![false; a_len]; + + let mut iter_b = b_bools.iter(); + for (idx, b) in a_bools.iter().enumerate() { + if *b && *iter_b.next().unwrap() { + expected_bools[idx] = true; + } + } + + let expected = RowSelection::from_filters(&[BooleanArray::from(expected_bools)]); + + let total_rows: usize = expected.iter().map(|s| s.row_count).sum(); + assert_eq!(a_len, total_rows); + + assert_eq!(a.and_then(&b), expected); + } + } + + #[test] + fn test_iter() { + // use the iter() API to show it does what is expected and + // avoid accidental deletion + let selectors = vec![ + RowSelector::select(3), + RowSelector::skip(33), + RowSelector::select(4), + ]; + + let round_tripped = RowSelection::from(selectors.clone()) + .iter() + .cloned() + .collect::>(); + assert_eq!(selectors, round_tripped); + } + + #[test] + fn test_limit() { + // Limit to existing limit should no-op + let selection = RowSelection::from(vec![RowSelector::select(10), RowSelector::skip(90)]); + let limited = selection.limit(10); + assert_eq!(RowSelection::from(vec![RowSelector::select(10)]), limited); + + let selection = RowSelection::from(vec![ + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(10), + ]); + + let limited = selection.clone().limit(5); + let expected = vec![RowSelector::select(5)]; + assert_eq!(limited.selectors(), expected); + + let limited = selection.clone().limit(15); + let expected = vec![ + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(5), + ]; + assert_eq!(limited.selectors(), expected); + + let limited = selection.clone().limit(0); + let expected = vec![]; + assert_eq!(limited.selectors(), expected); + + let limited = selection.clone().limit(30); + let expected = vec![ + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(10), + ]; + assert_eq!(limited.selectors(), expected); + + let limited = selection.limit(100); + let expected = vec![ + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(10), + ]; + assert_eq!(limited.selectors(), expected); + } + + #[test] + fn test_scan_ranges() { + let index = vec![ + PageLocation { + offset: 0, + compressed_page_size: 10, + first_row_index: 0, + }, + PageLocation { + offset: 10, + compressed_page_size: 10, + first_row_index: 10, + }, + PageLocation { + offset: 20, + compressed_page_size: 10, + first_row_index: 20, + }, + PageLocation { + offset: 30, + compressed_page_size: 10, + first_row_index: 30, + }, + PageLocation { + offset: 40, + compressed_page_size: 10, + first_row_index: 40, + }, + PageLocation { + offset: 50, + compressed_page_size: 10, + first_row_index: 50, + }, + PageLocation { + offset: 60, + compressed_page_size: 10, + first_row_index: 60, + }, + ]; + + let selection = RowSelection::from(vec![ + // Skip first page + RowSelector::skip(10), + // Multiple selects in same page + RowSelector::select(3), + RowSelector::skip(3), + RowSelector::select(4), + // Select to page boundary + RowSelector::skip(5), + RowSelector::select(5), + // Skip full page past page boundary + RowSelector::skip(12), + // Select across page boundaries + RowSelector::select(12), + // Skip final page + RowSelector::skip(12), + ]); + + let ranges = selection.scan_ranges(&index); + + // assert_eq!(mask, vec![false, true, true, false, true, true, false]); + assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60]); + + let selection = RowSelection::from(vec![ + // Skip first page + RowSelector::skip(10), + // Multiple selects in same page + RowSelector::select(3), + RowSelector::skip(3), + RowSelector::select(4), + // Select to page boundary + RowSelector::skip(5), + RowSelector::select(5), + // Skip full page past page boundary + RowSelector::skip(12), + // Select across page boundaries + RowSelector::select(12), + RowSelector::skip(1), + // Select across page boundaries including final page + RowSelector::select(8), + ]); + + let ranges = selection.scan_ranges(&index); + + // assert_eq!(mask, vec![false, true, true, false, true, true, true]); + assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); + + let selection = RowSelection::from(vec![ + // Skip first page + RowSelector::skip(10), + // Multiple selects in same page + RowSelector::select(3), + RowSelector::skip(3), + RowSelector::select(4), + // Select to page boundary + RowSelector::skip(5), + RowSelector::select(5), + // Skip full page past page boundary + RowSelector::skip(12), + // Select to final page boundary + RowSelector::select(12), + RowSelector::skip(1), + // Skip across final page boundary + RowSelector::skip(8), + // Select from final page + RowSelector::select(4), + ]); + + let ranges = selection.scan_ranges(&index); + + // assert_eq!(mask, vec![false, true, true, false, true, true, true]); + assert_eq!(ranges, vec![10..20, 20..30, 40..50, 50..60, 60..70]); + + let selection = RowSelection::from(vec![ + // Skip first page + RowSelector::skip(10), + // Multiple selects in same page + RowSelector::select(3), + RowSelector::skip(3), + RowSelector::select(4), + // Select to remaining in page and first row of next page + RowSelector::skip(5), + RowSelector::select(6), + // Skip remaining + RowSelector::skip(50), + ]); + + let ranges = selection.scan_ranges(&index); + + // assert_eq!(mask, vec![false, true, true, false, true, true, true]); + assert_eq!(ranges, vec![10..20, 20..30, 30..40]); + } + + #[test] + fn test_from_ranges() { + let ranges = [1..3, 4..6, 6..6, 8..8, 9..10]; + let selection = RowSelection::from_consecutive_ranges(ranges.into_iter(), 10); + assert_eq!( + selection.selectors(), + vec![ + RowSelector::skip(1), + RowSelector::select(2), + RowSelector::skip(1), + RowSelector::select(2), + RowSelector::skip(3), + RowSelector::select(1) + ] + ); + + let out_of_order_ranges = [1..3, 8..10, 4..7]; + let result = std::panic::catch_unwind(|| { + RowSelection::from_consecutive_ranges(out_of_order_ranges.into_iter(), 10) + }); + assert!(result.is_err()); + } + + #[test] + fn test_empty_selector() { + let selection = RowSelection::from(vec![ + RowSelector::skip(0), + RowSelector::select(2), + RowSelector::skip(0), + RowSelector::select(2), + ]); + assert_eq!(selection.selectors(), vec![RowSelector::select(4)]); + + let selection = RowSelection::from(vec![ + RowSelector::select(0), + RowSelector::skip(2), + RowSelector::select(0), + RowSelector::skip(2), + ]); + assert_eq!(selection.selectors(), vec![RowSelector::skip(4)]); + } + + #[test] + fn test_intersection() { + let selection = RowSelection::from(vec![RowSelector::select(1048576)]); + let result = selection.intersection(&selection); + assert_eq!(result, selection); + + let a = RowSelection::from(vec![ + RowSelector::skip(10), + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(20), + ]); + + let b = RowSelection::from(vec![ + RowSelector::skip(20), + RowSelector::select(20), + RowSelector::skip(10), + ]); + + let result = a.intersection(&b); + assert_eq!( + result.selectors(), + vec![ + RowSelector::skip(30), + RowSelector::select(10), + RowSelector::skip(10) + ] + ); + } + + #[test] + fn test_union() { + let selection = RowSelection::from(vec![RowSelector::select(1048576)]); + let result = selection.union(&selection); + assert_eq!(result, selection); + + // NYNYY + let a = RowSelection::from(vec![ + RowSelector::skip(10), + RowSelector::select(10), + RowSelector::skip(10), + RowSelector::select(20), + ]); + + // NNYYNYN + let b = RowSelection::from(vec![ + RowSelector::skip(20), + RowSelector::select(20), + RowSelector::skip(10), + RowSelector::select(10), + RowSelector::skip(10), + ]); + + let result = a.union(&b); + + // NYYYYYN + assert_eq!( + result.iter().collect::>(), + vec![ + &RowSelector::skip(10), + &RowSelector::select(50), + &RowSelector::skip(10), + ] + ); + } + + #[test] + fn test_row_count() { + let selection = RowSelection::from(vec![ + RowSelector::skip(34), + RowSelector::select(12), + RowSelector::skip(3), + RowSelector::select(35), + ]); + + assert_eq!(selection.row_count(), 12 + 35); + assert_eq!(selection.skipped_row_count(), 34 + 3); + + let selection = RowSelection::from(vec![RowSelector::select(12), RowSelector::select(35)]); + + assert_eq!(selection.row_count(), 12 + 35); + assert_eq!(selection.skipped_row_count(), 0); + + let selection = RowSelection::from(vec![RowSelector::skip(34), RowSelector::skip(3)]); + + assert_eq!(selection.row_count(), 0); + assert_eq!(selection.skipped_row_count(), 34 + 3); + + let selection = RowSelection::from(vec![]); + + assert_eq!(selection.row_count(), 0); + assert_eq!(selection.skipped_row_count(), 0); + } } From 61290ab79a105dc6c968349f01a58c768893b93e Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 21:22:26 +0800 Subject: [PATCH 66/73] Fix clippy --- parquet/src/arrow/arrow_reader/mod.rs | 8 ++-- parquet/src/arrow/arrow_reader/selection.rs | 37 ++++++++++--------- .../src/arrow/async_reader/arrow_reader.rs | 11 +++--- 3 files changed, 28 insertions(+), 28 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 8fb01d3aac99..31eb4f51b10d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -892,12 +892,10 @@ impl Iterator for ParquetRecordBatchReader { Ok(filtered_batch) => Some(Ok(filtered_batch)), Err(e) => Some(Err(e)), } + } else if batch.num_rows() > 0 { + Some(Ok(batch)) } else { - if batch.num_rows() > 0 { - Some(Ok(batch)) - } else { - None - } + None } } } diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index a8c8fcc81fae..2fa0affbac9d 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -91,20 +91,13 @@ impl RowSelector { /// assert_eq!(actual, expected); /// ``` /// -/// A [`RowSelection`] maintains the following invariants: -/// -/// * It contains no [`RowSelector`] of 0 rows -/// * Consecutive [`RowSelector`]s alternate skipping or selecting rows -/// -/// [`PageIndex`]: crate::file::page_index::index::PageIndex -// #[derive(Debug, Clone, Default, Eq, PartialEq)] -// pub struct RowSelection { -// selectors: Vec, -// } - +/// [`RowSelection`] is an enum that can be either a list of [`RowSelector`]s +/// or a [`BooleanArray`] bitmap #[derive(Debug, Clone, PartialEq)] pub enum RowSelection { + /// A list of [`RowSelector`]s Ranges(Vec), + /// A [`BooleanArray`] bitmap BitMap(BooleanArray), } @@ -244,7 +237,7 @@ impl RowSelection { ranges } - RowSelection::BitMap(bitmap) => { + RowSelection::BitMap(_) => { // not implemented yet unimplemented!("BitMap variant is not yet supported") } @@ -466,7 +459,7 @@ impl RowSelection { } /// Trims this [`RowSelection`] removing any trailing skips - pub(crate) fn trim(mut self) -> Self { + pub(crate) fn trim(self) -> Self { match self { RowSelection::Ranges(mut selectors) => { while selectors.last().map(|x| x.skip).unwrap_or(false) { @@ -535,7 +528,7 @@ impl RowSelection { } /// Limit this [`RowSelection`] to only select `limit` rows - pub(crate) fn limit(mut self, mut limit: usize) -> Self { + pub(crate) fn limit(self, mut limit: usize) -> Self { match self { RowSelection::Ranges(mut selectors) => { if limit == 0 { @@ -567,13 +560,14 @@ impl RowSelection { pub fn iter(&self) -> impl Iterator { match self { RowSelection::Ranges(selectors) => selectors.iter(), - RowSelection::BitMap(bitmap) => { + RowSelection::BitMap(_) => { // not implemented yet unimplemented!("BitMap variant is not yet supported") } } } + /// Returns the total number of rows in this [`RowSelection`] pub fn total_rows(&self) -> usize { match self { RowSelection::Ranges(selectors) => selectors.iter().map(|s| s.row_count).sum(), @@ -581,12 +575,21 @@ impl RowSelection { } } + /// Returns the number of selectors in this [`RowSelection`] pub fn len(&self) -> usize { match self { RowSelection::Ranges(selectors) => selectors.len(), RowSelection::BitMap(bitmap) => bitmap.len(), } } + + /// Returns `true` if this [`RowSelection`] is empty + pub fn is_empty(&self) -> bool { + match self { + RowSelection::Ranges(selectors) => selectors.is_empty(), + RowSelection::BitMap(bitmap) => bitmap.is_empty(), + } + } /// Returns the number of selected rows pub fn row_count(&self) -> usize { @@ -615,7 +618,7 @@ fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { for i in 0..bitmap.len() { let bit = bitmap.value(i); - if bit == !run_skip { + if bit != run_skip { // same as current run type (select vs skip) run_len += 1; } else { @@ -696,7 +699,7 @@ impl From for VecDeque { fn from(r: RowSelection) -> Self { match r { RowSelection::Ranges(selectors) => selectors.into(), - RowSelection::BitMap(bitmap) => { + RowSelection::BitMap(_) => { // not implemented yet unimplemented!("BitMap variant is not yet supported") } diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 313e6395e871..92f401f27320 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -24,7 +24,7 @@ use arrow_array::{cast::AsArray, Array, RecordBatch, RecordBatchReader}; use arrow_array::{ArrayRef, BooleanArray}; use arrow_buffer::BooleanBufferBuilder; use arrow_schema::{ArrowError, DataType, Schema, SchemaRef}; -use arrow_select::filter::{filter, filter_record_batch, prep_null_mask_filter}; +use arrow_select::filter::{filter, prep_null_mask_filter}; use crate::basic::PageType; use crate::column::page::{Page, PageMetadata, PageReader}; @@ -81,7 +81,7 @@ fn take_next_selection( ) -> Option { let current = selection.as_ref()?.clone(); - if let RowSelection::BitMap(mut bitmap) = current { + if let RowSelection::BitMap(bitmap) = current { let take = bitmap.len().min(to_select); let prefix = bitmap.slice(0, take); let suffix_len = bitmap.len() - take; @@ -103,7 +103,7 @@ fn take_next_selection( while let Some(run) = queue.pop_front() { if run.skip { - taken.push(run.clone()); + taken.push(run); continue; } let room = to_select.saturating_sub(count); @@ -112,7 +112,7 @@ fn take_next_selection( break; } if run.row_count <= room { - taken.push(run.clone()); + taken.push(run); count += run.row_count; } else { taken.push(RowSelector::select(room)); @@ -127,7 +127,7 @@ fn take_next_selection( Some(RowSelection::Ranges(queue.into_iter().collect())) }; - Some(RowSelection::Ranges(taken.into())) + Some(RowSelection::Ranges(taken)) } impl FilteredParquetRecordBatchReader { @@ -311,7 +311,6 @@ impl Iterator for FilteredParquetRecordBatchReader { // At loop exit, flush any remaining skips before finishing batch if acc_skip > 0 { self.array_reader.skip_records(acc_skip).ok()?; - acc_skip = 0; } if rows_accum == 0 { From b67b92ca55cc40b57f9fc535638454cb3558b8d9 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 22:15:57 +0800 Subject: [PATCH 67/73] Fix test --- parquet/src/arrow/arrow_reader/selection.rs | 47 ++++++++++++++------- 1 file changed, 32 insertions(+), 15 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 2fa0affbac9d..594d684361fc 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -57,7 +57,7 @@ impl RowSelector { /// This is applied prior to reading column data, and can therefore /// be used to skip IO to fetch data into memory /// -/// A typical use-case would be using the [`PageIndex`] to filter out rows +/// A typical use-case would be using the PageIndex to filter out rows /// that don't satisfy a predicate /// /// # Example @@ -421,9 +421,13 @@ impl RowSelection { /// returned: NNNNNNNNYYNYN pub fn intersection(&self, other: &Self) -> Self { match (self, other) { - (RowSelection::Ranges(a), RowSelection::Ranges(b)) => { - RowSelection::Ranges(intersect_row_selections(a, b)) - } + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => RowSelection::Ranges( + intersect_row_selections(a, b) + .selectors() + .iter() + .cloned() + .collect(), + ), (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bit_map)) => { let intersection_selectors = bit_map.values() & other_bit_map.values(); RowSelection::BitMap(BooleanArray::from(intersection_selectors)) @@ -441,9 +445,13 @@ impl RowSelection { /// returned: NYYYYYNNYYNYN pub fn union(&self, other: &Self) -> Self { match (self, other) { - (RowSelection::Ranges(a), RowSelection::Ranges(b)) => { - RowSelection::Ranges(union_row_selections(a, b)) - } + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => RowSelection::Ranges( + union_row_selections(a, b) + .selectors() + .iter() + .cloned() + .collect(), + ), (RowSelection::BitMap(_), _) | (_, RowSelection::BitMap(_)) => { unimplemented!("BitMap variant is not yet supported") } @@ -582,7 +590,7 @@ impl RowSelection { RowSelection::BitMap(bitmap) => bitmap.len(), } } - + /// Returns `true` if this [`RowSelection`] is empty pub fn is_empty(&self) -> bool { match self { @@ -609,7 +617,7 @@ impl RowSelection { } } -/// Convert a BooleanArray (no nulls) into a Vec, +/// Convert a BooleanArray (no nulls) into a `Vec`, /// alternating `select(run_len)` and `skip(run_len)`. fn selectors_from_bitmap(bitmap: &BooleanArray) -> Vec { let mut selectors = Vec::new(); @@ -713,7 +721,7 @@ impl From for VecDeque { /// other: NYNNNNNNY /// /// returned: NNNNNNNNYYNYN -fn intersect_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec { +fn intersect_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelection { let mut l_iter = left.iter().copied().peekable(); let mut r_iter = right.iter().copied().peekable(); @@ -775,7 +783,7 @@ fn intersect_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec< /// returned: NYYYYYNNYYNYN /// /// This can be removed from here once RowSelection::union is in parquet::arrow -fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> Vec { +fn union_row_selections(left: &[RowSelector], right: &[RowSelector]) -> RowSelection { let mut l_iter = left.iter().copied().peekable(); let mut r_iter = right.iter().copied().peekable(); @@ -1216,7 +1224,7 @@ mod tests { let res = intersect_row_selections(&a, &b); assert_eq!( - res, + res.selectors(), vec![ RowSelector::select(5), RowSelector::skip(4), @@ -1233,7 +1241,10 @@ mod tests { ]; let b = vec![RowSelector::select(36), RowSelector::skip(36)]; let res = intersect_row_selections(&a, &b); - assert_eq!(res, vec![RowSelector::select(3), RowSelector::skip(69)]); + assert_eq!( + res.selectors(), + vec![RowSelector::select(3), RowSelector::skip(69)] + ); // a size less than b size let a = vec![RowSelector::select(3), RowSelector::skip(7)]; @@ -1245,7 +1256,10 @@ mod tests { RowSelector::select(2), ]; let res = intersect_row_selections(&a, &b); - assert_eq!(res, vec![RowSelector::select(2), RowSelector::skip(8)]); + assert_eq!( + res.selectors(), + vec![RowSelector::select(2), RowSelector::skip(8)] + ); let a = vec![RowSelector::select(3), RowSelector::skip(7)]; let b = vec![ @@ -1256,7 +1270,10 @@ mod tests { RowSelector::select(2), ]; let res = intersect_row_selections(&a, &b); - assert_eq!(res, vec![RowSelector::select(2), RowSelector::skip(8)]); + assert_eq!( + res.selectors(), + vec![RowSelector::select(2), RowSelector::skip(8)] + ); } #[test] From 774bed51875e989756250105be6e97c76bf67a9e Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 22:20:32 +0800 Subject: [PATCH 68/73] Clippy fix --- parquet/src/arrow/arrow_reader/selection.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 594d684361fc..d83624d888a0 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -423,10 +423,7 @@ impl RowSelection { match (self, other) { (RowSelection::Ranges(a), RowSelection::Ranges(b)) => RowSelection::Ranges( intersect_row_selections(a, b) - .selectors() - .iter() - .cloned() - .collect(), + .selectors().to_vec(), ), (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bit_map)) => { let intersection_selectors = bit_map.values() & other_bit_map.values(); @@ -447,10 +444,7 @@ impl RowSelection { match (self, other) { (RowSelection::Ranges(a), RowSelection::Ranges(b)) => RowSelection::Ranges( union_row_selections(a, b) - .selectors() - .iter() - .cloned() - .collect(), + .selectors().to_vec(), ), (RowSelection::BitMap(_), _) | (_, RowSelection::BitMap(_)) => { unimplemented!("BitMap variant is not yet supported") From 0ad20e5d2b09c3a21eca0d6ec4753abd6e00189c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 22:20:37 +0800 Subject: [PATCH 69/73] Clippy fix --- parquet/src/arrow/arrow_reader/selection.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index d83624d888a0..9ead35cde8b9 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -421,10 +421,9 @@ impl RowSelection { /// returned: NNNNNNNNYYNYN pub fn intersection(&self, other: &Self) -> Self { match (self, other) { - (RowSelection::Ranges(a), RowSelection::Ranges(b)) => RowSelection::Ranges( - intersect_row_selections(a, b) - .selectors().to_vec(), - ), + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => { + RowSelection::Ranges(intersect_row_selections(a, b).selectors().to_vec()) + } (RowSelection::BitMap(bit_map), RowSelection::BitMap(other_bit_map)) => { let intersection_selectors = bit_map.values() & other_bit_map.values(); RowSelection::BitMap(BooleanArray::from(intersection_selectors)) @@ -442,10 +441,9 @@ impl RowSelection { /// returned: NYYYYYNNYYNYN pub fn union(&self, other: &Self) -> Self { match (self, other) { - (RowSelection::Ranges(a), RowSelection::Ranges(b)) => RowSelection::Ranges( - union_row_selections(a, b) - .selectors().to_vec(), - ), + (RowSelection::Ranges(a), RowSelection::Ranges(b)) => { + RowSelection::Ranges(union_row_selections(a, b).selectors().to_vec()) + } (RowSelection::BitMap(_), _) | (_, RowSelection::BitMap(_)) => { unimplemented!("BitMap variant is not yet supported") } From ceceb8e090f59c693e2ade3da85a21e4ef2c8ff1 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 11 May 2025 22:41:27 +0800 Subject: [PATCH 70/73] Fix 3 data page testing --- .../src/arrow/async_reader/arrow_reader.rs | 32 ++++++++++++++++--- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 92f401f27320..01cc63e549e3 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -652,9 +652,13 @@ mod tests { fn test_predicate_page_cache_basic_operations() { use super::*; - let cache = PredicatePageCache::new(2); + // Mock PredicatePageCache with a capacity of 4 + // One dictionary page and 3 data pages + let cache = PredicatePageCache::new(4); let page1 = Page::dummy_page(PageType::DATA_PAGE, 100); let page2 = Page::dummy_page(PageType::DICTIONARY_PAGE, 200); + let page3 = Page::dummy_page(PageType::DATA_PAGE, 100); + let page4 = Page::dummy_page(PageType::DATA_PAGE, 100); // Insert and retrieve a data page cache.get().insert_page(0, 1000, page1.clone()); @@ -668,26 +672,46 @@ mod tests { assert!(retrieved.is_some()); assert_eq!(retrieved.unwrap().page_type(), PageType::DICTIONARY_PAGE); + // Insert and retrieve a data page for same column + cache.get().insert_page(0, 3000, page3.clone()); + let retrieved = cache.get().get_page(0, 3000); + assert!(retrieved.is_some()); + assert_eq!(retrieved.unwrap().page_type(), PageType::DATA_PAGE); + + // Insert and retrieve another data page for same column + cache.get().insert_page(0, 4000, page4.clone()); + let retrieved = cache.get().get_page(0, 4000); + assert!(retrieved.is_some()); + assert_eq!(retrieved.unwrap().page_type(), PageType::DATA_PAGE); + // Both pages should still be accessible assert!(cache.get().get_page(0, 1000).is_some()); assert!(cache.get().get_page(0, 2000).is_some()); + assert!(cache.get().get_page(0, 3000).is_some()); + assert!(cache.get().get_page(0, 4000).is_some()); } #[test] fn test_predicate_page_cache_replacement() { use super::*; - let cache = PredicatePageCache::new(2); + let cache = PredicatePageCache::new(4); let data_page1 = Page::dummy_page(PageType::DATA_PAGE, 100); let data_page2 = Page::dummy_page(PageType::DATA_PAGE_V2, 200); + let data_page3 = Page::dummy_page(PageType::DATA_PAGE, 300); + let data_page4 = Page::dummy_page(PageType::DATA_PAGE, 300); // Insert first data page cache.get().insert_page(0, 1000, data_page1.clone()); assert!(cache.get().get_page(0, 1000).is_some()); - - // Insert second data page - should replace first data page cache.get().insert_page(0, 2000, data_page2.clone()); assert!(cache.get().get_page(0, 2000).is_some()); + cache.get().insert_page(0, 3000, data_page3.clone()); + assert!(cache.get().get_page(0, 3000).is_some()); + + // Insert the 4th data page - should replace the first data page + cache.get().insert_page(0, 4000, data_page4.clone()); + assert!(cache.get().get_page(0, 4000).is_some()); assert!(cache.get().get_page(0, 1000).is_none()); // First page should be gone } From bc02e2a2d4714b80e2d0bb4b823c6e8f01f775a9 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 12 May 2025 17:59:18 +0800 Subject: [PATCH 71/73] Fix encrption error handling for new page cache logic --- parquet-testing | 2 +- .../src/arrow/async_reader/arrow_reader.rs | 35 +++++++++++++++---- .../tests/encryption/encryption_agnostic.rs | 4 +-- 3 files changed, 31 insertions(+), 10 deletions(-) diff --git a/parquet-testing b/parquet-testing index 2dc8bf140ed6..6e851ddd768d 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 2dc8bf140ed6e28652fc347211c7d661714c7f95 +Subproject commit 6e851ddd768d6af741c7b15dc594874399fc3cff diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 01cc63e549e3..6fc363ca7d71 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -260,7 +260,10 @@ impl Iterator for FilteredParquetRecordBatchReader { // Before any read, flush accumulated skips if acc_skip > 0 { - self.array_reader.skip_records(acc_skip).ok()?; + match self.array_reader.skip_records(acc_skip) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; acc_skip = 0; } @@ -269,7 +272,10 @@ impl Iterator for FilteredParquetRecordBatchReader { if total < 10 * select_count { // Bitmap branch let bitmap = self.create_bitmap_from_ranges(&runs); - self.array_reader.read_records(bitmap.len()).ok()?; + match self.array_reader.read_records(bitmap.len()) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; mask_builder.append_buffer(bitmap.values()); rows_accum += bitmap.true_count(); } else { @@ -279,10 +285,16 @@ impl Iterator for FilteredParquetRecordBatchReader { acc_skip += r.row_count; } else { if acc_skip > 0 { - self.array_reader.skip_records(acc_skip).ok()?; + match self.array_reader.skip_records(acc_skip) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; acc_skip = 0; } - self.array_reader.read_records(r.row_count).ok()?; + match self.array_reader.read_records(r.row_count) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; mask_builder.append_n(r.row_count, true); rows_accum += r.row_count; } @@ -293,11 +305,17 @@ impl Iterator for FilteredParquetRecordBatchReader { RowSelection::BitMap(bitmap) => { // Flush any pending skips before bitmap if acc_skip > 0 { - self.array_reader.skip_records(acc_skip).ok()?; + match self.array_reader.skip_records(acc_skip) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; acc_skip = 0; } let n = bitmap.len(); - self.array_reader.read_records(n).ok()?; + match self.array_reader.read_records(n) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; mask_builder.append_buffer(bitmap.values()); rows_accum += bitmap.true_count(); } @@ -310,7 +328,10 @@ impl Iterator for FilteredParquetRecordBatchReader { // At loop exit, flush any remaining skips before finishing batch if acc_skip > 0 { - self.array_reader.skip_records(acc_skip).ok()?; + match self.array_reader.skip_records(acc_skip) { + Ok(_) => {} + Err(e) => return Some(Err(e.into())), + }; } if rows_accum == 0 { diff --git a/parquet/tests/encryption/encryption_agnostic.rs b/parquet/tests/encryption/encryption_agnostic.rs index ffa8a468d820..e071471712f4 100644 --- a/parquet/tests/encryption/encryption_agnostic.rs +++ b/parquet/tests/encryption/encryption_agnostic.rs @@ -139,8 +139,8 @@ pub async fn read_plaintext_footer_file_without_decryption_properties_async() { Some(Err(ParquetError::ArrowError(s))) => { assert!(s.contains("protocol error")); } - e => { - panic!("Expected ArrowError::ParquetError. Got {e:?}"); + _ => { + panic!("Expected ArrowError::ParquetError"); } }; } From a4065cc52fae045d81650d3c146bbd514536c707 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 12 May 2025 18:10:17 +0800 Subject: [PATCH 72/73] Update parquet testing --- parquet-testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-testing b/parquet-testing index 6e851ddd768d..2dc8bf140ed6 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 6e851ddd768d6af741c7b15dc594874399fc3cff +Subproject commit 2dc8bf140ed6e28652fc347211c7d661714c7f95 From 269b396bdb5c8969ad7814ae07af30f6a3ea2ea6 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 12 May 2025 18:18:53 +0800 Subject: [PATCH 73/73] Clean up code --- .../src/arrow/async_reader/arrow_reader.rs | 4 +-- parquet/src/arrow/async_reader/mod.rs | 28 ------------------- 2 files changed, 1 insertion(+), 31 deletions(-) diff --git a/parquet/src/arrow/async_reader/arrow_reader.rs b/parquet/src/arrow/async_reader/arrow_reader.rs index 6fc363ca7d71..6a8a9fc5db38 100644 --- a/parquet/src/arrow/async_reader/arrow_reader.rs +++ b/parquet/src/arrow/async_reader/arrow_reader.rs @@ -256,8 +256,6 @@ impl Iterator for FilteredParquetRecordBatchReader { continue; } - //println!("select_count = {}, read_nums {}, total_skip{}, acc_skip {}", runs.len(), total_read, total_skip, acc_skip); - // Before any read, flush accumulated skips if acc_skip > 0 { match self.array_reader.skip_records(acc_skip) { @@ -340,7 +338,7 @@ impl Iterator for FilteredParquetRecordBatchReader { let array = match self.array_reader.consume_batch() { Ok(a) => a, - Err(_) => return None, + Err(error) => return Some(Err(error.into())), }; let final_array = if mask_builder.is_empty() { diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index d1bc32154a16..a0c8de6bf642 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -659,34 +659,6 @@ where .fetch(&mut self.input, &projection, Some(&selection)) .await?; - // let total_rows = selection.total_rows(); - // - // let selection = match selection { - // RowSelection::Ranges(selectors) => { - // // if total_rows / selectors.len() > 200 { - // // Some(RowSelection::Ranges(selectors)) - // // } else { - // - // println!("selectors len : {:?}", selectors.len()); - // println!("total select rows : {:?}", rows_after); - // println!("total rows : {:?}", row_group.row_count); - // println!("total rows for selectors : {:?}", rows_before); - // - // let mut builder = arrow_array::builder::BooleanBufferBuilder::new(rows_after); - // - // for selector in selectors.iter() { - // if selector.skip { - // builder.append_n(selector.row_count, false); - // } else { - // builder.append_n(selector.row_count, true); - // } - // } - // Some(RowSelection::BitMap( arrow_array::BooleanArray::from(builder.finish()))) - // } - // // } - // _ => None, - // }; - let array_reader = build_array_reader(self.fields.as_deref(), &projection, &row_group)?; let reader = FilteredParquetRecordBatchReader::new( batch_size,