From 1f93a9352becc864bd3487cc8abd4b1520c2939a Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 1 Jul 2025 11:18:04 -0500 Subject: [PATCH 01/38] update --- parquet/src/arrow/array_reader/builder.rs | 62 ++- .../arrow/array_reader/cached_array_reader.rs | 420 ++++++++++++++++++ parquet/src/arrow/array_reader/list_array.rs | 6 +- parquet/src/arrow/array_reader/mod.rs | 3 + .../src/arrow/array_reader/row_group_cache.rs | 80 ++++ parquet/src/arrow/arrow_reader/mod.rs | 45 +- parquet/src/arrow/async_reader/mod.rs | 37 +- parquet/src/arrow/mod.rs | 6 + 8 files changed, 631 insertions(+), 28 deletions(-) create mode 100644 parquet/src/arrow/array_reader/cached_array_reader.rs create mode 100644 parquet/src/arrow/array_reader/row_group_cache.rs diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 6dcf05ccf8ad..4229d349ef6e 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use arrow_schema::{DataType, Fields, SchemaBuilder}; use crate::arrow::array_reader::byte_view_array::make_byte_view_array_reader; +use crate::arrow::array_reader::cached_array_reader::CachedArrayReader; use crate::arrow::array_reader::empty_array::make_empty_array_reader; use crate::arrow::array_reader::fixed_len_byte_array::make_fixed_len_byte_array_reader; +use crate::arrow::array_reader::row_group_cache::RowGroupCache; use crate::arrow::array_reader::{ make_byte_array_dictionary_reader, make_byte_array_reader, ArrayReader, FixedSizeListArrayReader, ListArrayReader, MapArrayReader, NullArrayReader, @@ -49,9 +51,14 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: Option<&ParquetField>, mask: &ProjectionMask, + cache_mask: &ProjectionMask, + cache: Arc>, ) -> Result> { let reader = field - .and_then(|field| self.build_reader(field, mask).transpose()) + .and_then(|field| { + self.build_reader(field, mask, cache_mask, cache) + .transpose() + }) .transpose()? .unwrap_or_else(|| make_empty_array_reader(self.num_rows())); @@ -67,15 +74,33 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, + cache_mask: &ProjectionMask, + cache: Arc>, ) -> Result>> { match field.field_type { - ParquetFieldType::Primitive { .. } => self.build_primitive_reader(field, mask), + ParquetFieldType::Primitive { col_idx, .. } => { + if let Some(reader) = self.build_primitive_reader(field, mask)? { + if cache_mask.leaf_included(col_idx) { + Ok(Some(Box::new(CachedArrayReader::new( + reader, cache, col_idx, + )))) + } else { + Ok(Some(reader)) + } + } else { + Ok(None) + } + } ParquetFieldType::Group { .. } => match &field.arrow_type { - DataType::Map(_, _) => self.build_map_reader(field, mask), - DataType::Struct(_) => self.build_struct_reader(field, mask), - DataType::List(_) => self.build_list_reader(field, mask, false), - DataType::LargeList(_) => self.build_list_reader(field, mask, true), - DataType::FixedSizeList(_, _) => self.build_fixed_size_list_reader(field, mask), + DataType::Map(_, _) => self.build_map_reader(field, mask, cache_mask, cache), + DataType::Struct(_) => self.build_struct_reader(field, mask, cache_mask, cache), + DataType::List(_) => self.build_list_reader(field, mask, cache_mask, cache, false), + DataType::LargeList(_) => { + self.build_list_reader(field, mask, cache_mask, cache, true) + } + DataType::FixedSizeList(_, _) => { + self.build_fixed_size_list_reader(field, mask, cache_mask, cache) + } d => unimplemented!("reading group type {} not implemented", d), }, } @@ -86,12 +111,14 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, + cache_mask: &ProjectionMask, + cache: Arc>, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 2); - let key_reader = self.build_reader(&children[0], mask)?; - let value_reader = self.build_reader(&children[1], mask)?; + let key_reader = self.build_reader(&children[0], mask, cache_mask, cache.clone())?; + let value_reader = self.build_reader(&children[1], mask, cache_mask, cache)?; match (key_reader, value_reader) { (Some(key_reader), Some(value_reader)) => { @@ -137,12 +164,14 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, + cache_mask: &ProjectionMask, + cache: Arc>, is_large: bool, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask)? { + let reader = match self.build_reader(&children[0], mask, cache_mask, cache)? { Some(item_reader) => { // Need to retrieve underlying data type to handle projection let item_type = item_reader.get_data_type().clone(); @@ -184,11 +213,13 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, + cache_mask: &ProjectionMask, + cache: Arc>, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask)? { + let reader = match self.build_reader(&children[0], mask, cache_mask, cache)? { Some(item_reader) => { let item_type = item_reader.get_data_type().clone(); let reader = match &field.arrow_type { @@ -318,6 +349,8 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, + cache_mask: &ProjectionMask, + cache: Arc>, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -330,7 +363,7 @@ impl<'a> ArrayReaderBuilder<'a> { let mut builder = SchemaBuilder::with_capacity(children.len()); for (arrow, parquet) in arrow_fields.iter().zip(children) { - if let Some(reader) = self.build_reader(parquet, mask)? { + if let Some(reader) = self.build_reader(parquet, mask, cache_mask, cache.clone())? { // Need to retrieve underlying data type to handle projection let child_type = reader.get_data_type().clone(); builder.push(arrow.as_ref().clone().with_data_type(child_type)); @@ -374,9 +407,10 @@ mod tests { file_metadata.key_value_metadata(), ) .unwrap(); + let cache = Arc::new(Mutex::new(RowGroupCache::new(1000))); let array_reader = ArrayReaderBuilder::new(&file_reader) - .build_array_reader(fields.as_ref(), &mask) + .build_array_reader(fields.as_ref(), &mask, &ProjectionMask::all(), cache) .unwrap(); // Create arrow types diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs new file mode 100644 index 000000000000..99c05000e472 --- /dev/null +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -0,0 +1,420 @@ +use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; +use crate::errors::Result; +use arrow_array::{new_empty_array, ArrayRef, BooleanArray}; +use arrow_buffer::BooleanBufferBuilder; +use arrow_schema::DataType as ArrowType; +use arrow_select::concat::concat; +use arrow_select::filter::filter; +use std::any::Any; +use std::collections::VecDeque; +use std::sync::{Arc, Mutex}; + +/// Row selector indicating whether to read or skip rows +#[derive(Debug, Clone)] +struct RowSelector { + /// Number of rows in this selection + row_count: usize, + /// Whether to skip (true) or read (false) these rows + skip: bool, +} + +impl RowSelector { + fn select(row_count: usize) -> Self { + Self { + row_count, + skip: false, + } + } + + fn skip(row_count: usize) -> Self { + Self { + row_count, + skip: true, + } + } +} + +/// A cached wrapper around an ArrayReader that avoids duplicate decoding +/// when the same column appears in both filter predicates and output projection. +/// +/// This reader accumulates read/skip selections and processes them efficiently +/// using cached batch data when consume_batch() is called. +pub struct CachedArrayReader { + /// The underlying array reader + inner: Box, + /// Shared cache for this row group + cache: Arc>, + /// Column index for cache key generation + column_idx: usize, + /// Current row position in the row group + current_row: usize, + /// Inner reader's row position (to keep it in sync) + inner_reader_position: usize, + /// Queue of read/skip selections to process + selection: VecDeque, +} + +impl CachedArrayReader { + /// Creates a new cached array reader + pub fn new( + inner: Box, + cache: Arc>, + column_idx: usize, + ) -> Self { + Self { + inner, + cache, + column_idx, + current_row: 0, + inner_reader_position: 0, + selection: VecDeque::new(), + } + } + + /// Gets the batch size from the cache + fn batch_size(&self) -> usize { + self.cache.lock().unwrap().batch_size() + } + + /// Calculates the batch ID for a given row position + fn batch_id_from_row(&self, row: usize) -> usize { + row / self.batch_size() + } + + /// Ensures a batch is cached by fetching it from the inner reader if needed. + /// + /// Returns the number of rows cached for this batch. If `0` is returned it + /// indicates the inner reader has no more data (EOF). + fn ensure_cached(&mut self, batch_id: usize) -> Result { + let batch_start_row = batch_id * self.batch_size(); + + // Check if already cached + if let Some(array) = self + .cache + .lock() + .unwrap() + .get(self.column_idx, batch_start_row) + { + return Ok(array.len()); + } + + // Sync inner reader to batch start if needed + if self.inner_reader_position < batch_start_row { + let to_skip = batch_start_row - self.inner_reader_position; + let skipped = self.inner.skip_records(to_skip)?; + self.inner_reader_position += skipped; + + // Reached EOF before desired position + if skipped < to_skip { + return Ok(0); + } + } + + // Read the batch + let expected_batch = self.batch_size(); + let records_read = self.inner.read_records(expected_batch)?; + + // If no more rows, signal EOF + if records_read == 0 { + return Ok(0); + } + + let array = self.inner.consume_batch()?; + + // Cache it + self.cache + .lock() + .unwrap() + .insert(self.column_idx, batch_start_row, array.clone()); + self.inner_reader_position += records_read; + + Ok(array.len()) + } + + /// Converts selection queue to boolean buffer for filtering + fn selection_to_boolean_buffer(&self, total_rows: usize) -> arrow_buffer::BooleanBuffer { + let mut builder = BooleanBufferBuilder::new(total_rows); + for selector in &self.selection { + builder.append_n(selector.row_count, !selector.skip); + } + builder.finish() + } +} + +impl ArrayReader for CachedArrayReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_data_type(&self) -> &ArrowType { + self.inner.get_data_type() + } + + fn read_records(&mut self, mut requested: usize) -> Result { + let mut total_read = 0; + + while requested > 0 { + let batch_id = self.batch_id_from_row(self.current_row); + + // Ensure the batch is cached and determine its length + let cached_len = self.ensure_cached(batch_id)?; + + // EOF reached + if cached_len == 0 { + break; + } + + let batch_start = batch_id * self.batch_size(); + let offset_in_batch = self.current_row - batch_start; + + // If current position is past the cached length, advance to next batch. + // If this batch was partial (i.e. smaller than the configured batch size) + // this indicates we have reached EOF and should stop. + if offset_in_batch >= cached_len { + self.current_row = batch_start + cached_len; + + // Partial batch implies EOF has been reached + if cached_len < self.batch_size() { + break; + } + + continue; + } + + let available = cached_len - offset_in_batch; + let to_read = available.min(requested); + + // Record the selection + if to_read > 0 { + self.selection.push_back(RowSelector::select(to_read)); + self.current_row += to_read; + requested -= to_read; + total_read += to_read; + } + } + + Ok(total_read) + } + + fn skip_records(&mut self, num_records: usize) -> Result { + // Add selection for skipping these records + self.selection.push_back(RowSelector::skip(num_records)); + self.current_row += num_records; + // Note: we don't cache or sync inner reader for skips + Ok(num_records) + } + + fn consume_batch(&mut self) -> Result { + if self.selection.is_empty() { + // Return empty array of correct type + return Ok(new_empty_array(self.get_data_type())); + } + + // Calculate total row count and starting position + let total_rows: usize = self.selection.iter().map(|s| s.row_count).sum(); + let start_row = self.current_row - total_rows; + + if total_rows == 0 { + self.selection.clear(); + return Ok(new_empty_array(self.get_data_type())); + } + + // Create boolean selection buffer + let selection_buffer = self.selection_to_boolean_buffer(total_rows); + + // Determine batch range + let batch_size = self.batch_size(); + let start_batch = start_row / batch_size; + let end_batch = (start_row + total_rows - 1) / batch_size; + + let mut result_arrays = Vec::new(); + + for batch_id in start_batch..=end_batch { + let batch_start = batch_id * batch_size; + let batch_end = batch_start + batch_size - 1; + + // Calculate overlap between our selection and this batch + let overlap_start = start_row.max(batch_start); + let overlap_end = (start_row + total_rows - 1).min(batch_end); + + if overlap_start > overlap_end { + continue; // No overlap + } + + // Get selection slice for this batch + let selection_start = overlap_start - start_row; + let selection_length = overlap_end - overlap_start + 1; + let batch_selection = selection_buffer.slice(selection_start, selection_length); + + if batch_selection.count_set_bits() == 0 { + continue; // Nothing selected in this batch + } + + // Get cached array and apply filter + let cached_array = self + .cache + .lock() + .unwrap() + .get(self.column_idx, batch_start) + .unwrap(); + let mask = BooleanArray::from(batch_selection); + + // Calculate offset within the batch + let batch_offset = overlap_start - batch_start; + let batch_slice = if batch_offset > 0 || selection_length < batch_size { + cached_array.slice(batch_offset, selection_length) + } else { + cached_array + }; + + let filtered = filter(&batch_slice, &mask)?; + if filtered.len() > 0 { + result_arrays.push(filtered); + } + } + + // Clear selection queue + self.selection.clear(); + + // Concatenate results + match result_arrays.len() { + 0 => Ok(new_empty_array(self.get_data_type())), + 1 => Ok(result_arrays.into_iter().next().unwrap()), + _ => { + let array_refs: Vec<&dyn arrow_array::Array> = + result_arrays.iter().map(|a| a.as_ref()).collect(); + Ok(concat(&array_refs)?) + } + } + } + + fn get_def_levels(&self) -> Option<&[i16]> { + self.inner.get_def_levels() + } + + fn get_rep_levels(&self) -> Option<&[i16]> { + self.inner.get_rep_levels() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::arrow::array_reader::row_group_cache::RowGroupCache; + use crate::arrow::array_reader::ArrayReader; + use arrow_array::{ArrayRef, Int32Array}; + use std::sync::{Arc, Mutex}; + + // Mock ArrayReader for testing + struct MockArrayReader { + data: Vec, + position: usize, + read_position: usize, + data_type: ArrowType, + } + + impl MockArrayReader { + fn new(data: Vec) -> Self { + Self { + data, + position: 0, + read_position: 0, + data_type: ArrowType::Int32, + } + } + } + + impl ArrayReader for MockArrayReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_data_type(&self) -> &ArrowType { + &self.data_type + } + + fn read_records(&mut self, batch_size: usize) -> Result { + let remaining = self.data.len() - self.position; + let to_read = std::cmp::min(batch_size, remaining); + self.read_position = self.position; + self.position += to_read; + Ok(to_read) + } + + fn consume_batch(&mut self) -> Result { + // Return data from read_position to current position + let slice = &self.data[self.read_position..self.position]; + Ok(Arc::new(Int32Array::from(slice.to_vec()))) + } + + fn skip_records(&mut self, num_records: usize) -> Result { + let remaining = self.data.len() - self.position; + let to_skip = std::cmp::min(num_records, remaining); + self.position += to_skip; + Ok(to_skip) + } + + fn get_def_levels(&self) -> Option<&[i16]> { + None + } + + fn get_rep_levels(&self) -> Option<&[i16]> { + None + } + } + + #[test] + fn test_cached_reader_basic() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + + // Read 3 records + let records_read = cached_reader.read_records(3).unwrap(); + assert_eq!(records_read, 3); + + // Consume batch should return those 3 records + let array = cached_reader.consume_batch().unwrap(); + assert_eq!(array.len(), 3); + } + + #[test] + fn test_read_skip_pattern() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + + // Read 2, skip 2, read 1 (total 5 records from first batch) + cached_reader.read_records(2).unwrap(); + cached_reader.skip_records(2).unwrap(); + cached_reader.read_records(1).unwrap(); + + // Should get [1,2] and [5] (skipped 3,4) + let array = cached_reader.consume_batch().unwrap(); + assert_eq!(array.len(), 3); // 2 + 1 reads + } + + #[test] + fn test_cache_efficiency() { + // Test that cache avoids duplicate reads for the same batch + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + + // First reader - populate cache + let mut cached_reader1 = CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0); + cached_reader1.read_records(3).unwrap(); + let array1 = cached_reader1.consume_batch().unwrap(); + assert_eq!(array1.len(), 3); + + // Second reader - should use cache (no inner reader created) + let mock_reader2 = MockArrayReader::new(vec![10, 20, 30, 40, 50]); // Different data + let mut cached_reader2 = CachedArrayReader::new(Box::new(mock_reader2), cache.clone(), 0); + cached_reader2.read_records(2).unwrap(); + let array2 = cached_reader2.consume_batch().unwrap(); + assert_eq!(array2.len(), 2); + + // The second reader should get data from cache (original [1,2,3,4,5]) + // not from its mock reader ([10,20,30,40,50]) + // This tests that cache is working + } +} diff --git a/parquet/src/arrow/array_reader/list_array.rs b/parquet/src/arrow/array_reader/list_array.rs index 66c4f30b3c29..06b17211d187 100644 --- a/parquet/src/arrow/array_reader/list_array.rs +++ b/parquet/src/arrow/array_reader/list_array.rs @@ -247,6 +247,7 @@ impl ArrayReader for ListArrayReader { mod tests { use super::*; use crate::arrow::array_reader::list_array::ListArrayReader; + use crate::arrow::array_reader::row_group_cache::RowGroupCache; use crate::arrow::array_reader::test_util::InMemoryArrayReader; use crate::arrow::array_reader::ArrayReaderBuilder; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; @@ -259,7 +260,7 @@ mod tests { use arrow_array::{Array, PrimitiveArray}; use arrow_data::ArrayDataBuilder; use arrow_schema::Fields; - use std::sync::Arc; + use std::sync::{Arc, Mutex}; fn list_type( data_type: ArrowType, @@ -562,9 +563,10 @@ mod tests { file_metadata.key_value_metadata(), ) .unwrap(); + let cache = Arc::new(Mutex::new(RowGroupCache::new(1000))); let mut array_reader = ArrayReaderBuilder::new(&file_reader) - .build_array_reader(fields.as_ref(), &mask) + .build_array_reader(fields.as_ref(), &mask, &ProjectionMask::all(), cache) .unwrap(); let batch = array_reader.next_batch(100).unwrap(); diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index ec461a7cccb1..208799f5ff71 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -33,6 +33,7 @@ mod builder; mod byte_array; mod byte_array_dictionary; mod byte_view_array; +mod cached_array_reader; mod empty_array; mod fixed_len_byte_array; mod fixed_size_list_array; @@ -40,6 +41,7 @@ mod list_array; mod map_array; mod null_array; mod primitive_array; +mod row_group_cache; mod struct_array; #[cfg(test)] @@ -58,6 +60,7 @@ pub use list_array::ListArrayReader; pub use map_array::MapArrayReader; pub use null_array::NullArrayReader; pub use primitive_array::PrimitiveArrayReader; +pub use row_group_cache::RowGroupCache; pub use struct_array::StructArrayReader; /// Array reader reads parquet data into arrow array. diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs new file mode 100644 index 000000000000..3fb21f41a771 --- /dev/null +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -0,0 +1,80 @@ +use arrow_array::ArrayRef; +use std::collections::HashMap; + +/// Cache key that uniquely identifies a batch within a row group +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct CacheKey { + /// Column index in the row group + pub column_idx: usize, + /// Starting row ID for this batch + pub row_id: usize, +} + +/// Row group cache that stores decoded arrow arrays at batch granularity +/// +/// This cache is designed to avoid duplicate decoding when the same column +/// appears in both filter predicates and output projection. +#[derive(Debug, Default)] +pub struct RowGroupCache { + /// Cache storage mapping (column_idx, row_id) -> ArrayRef + cache: HashMap, + /// Batch size used for cache entries + batch_size: usize, +} + +impl RowGroupCache { + /// Creates a new empty row group cache + pub fn new(batch_size: usize) -> Self { + Self { + cache: HashMap::new(), + batch_size, + } + } + + /// Inserts an array into the cache for the given column and starting row ID + pub fn insert(&mut self, column_idx: usize, row_id: usize, array: ArrayRef) { + let key = CacheKey { column_idx, row_id }; + self.cache.insert(key, array); + } + + /// Retrieves a cached array for the given column and row ID + /// Returns None if not found in cache + pub fn get(&self, column_idx: usize, row_id: usize) -> Option { + let key = CacheKey { column_idx, row_id }; + self.cache.get(&key).cloned() + } + + /// Gets the batch size for this cache + pub fn batch_size(&self) -> usize { + self.batch_size + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::{ArrayRef, Int32Array}; + use std::sync::Arc; + + #[test] + fn test_cache_basic_operations() { + let mut cache = RowGroupCache::new(1000); + + // Create test array + let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); + + // Test insert and get + cache.insert(0, 0, array.clone()); + let retrieved = cache.get(0, 0); + assert!(retrieved.is_some()); + assert_eq!(retrieved.unwrap().len(), 5); + + // Test miss + let miss = cache.get(1, 0); + assert!(miss.is_none()); + + // Test different row_id + let miss = cache.get(0, 1000); + assert!(miss.is_none()); + } +} diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 9127423efe4b..6f15e08443e7 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -24,10 +24,10 @@ use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; pub use crate::arrow::array_reader::RowGroups; -use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; +use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder, RowGroupCache}; use crate::arrow::schema::{parquet_to_arrow_schema_and_fields, ParquetField}; use crate::arrow::{parquet_to_arrow_field_levels, FieldLevels, ProjectionMask}; use crate::column::page::{PageIterator, PageReader}; @@ -711,6 +711,10 @@ impl ParquetRecordBatchReaderBuilder { let batch_size = self .batch_size .min(self.metadata.file_metadata().num_rows() as usize); + let cache_projection = match self.compute_cache_projection(&self.projection) { + Some(projection) => projection, + None => ProjectionMask::all(), + }; let row_groups = self .row_groups @@ -721,6 +725,7 @@ impl ParquetRecordBatchReaderBuilder { metadata: self.metadata, row_groups, }; + let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new(batch_size))); let mut filter = self.filter; let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(self.selection); @@ -733,15 +738,26 @@ impl ParquetRecordBatchReaderBuilder { break; } - let array_reader = ArrayReaderBuilder::new(&reader) - .build_array_reader(self.fields.as_deref(), predicate.projection())?; + let mut cache_projection = predicate.projection().clone(); + cache_projection.intersect(&self.projection); + + let array_reader = ArrayReaderBuilder::new(&reader).build_array_reader( + self.fields.as_deref(), + predicate.projection(), + &cache_projection, + row_group_cache.clone(), + )?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } } - let array_reader = ArrayReaderBuilder::new(&reader) - .build_array_reader(self.fields.as_deref(), &self.projection)?; + let array_reader = ArrayReaderBuilder::new(&reader).build_array_reader( + self.fields.as_deref(), + &self.projection, + &cache_projection, + row_group_cache.clone(), + )?; let read_plan = plan_builder .limited(reader.num_rows()) @@ -752,6 +768,16 @@ impl ParquetRecordBatchReaderBuilder { Ok(ParquetRecordBatchReader::new(array_reader, read_plan)) } + + fn compute_cache_projection(&self, projection: &ProjectionMask) -> Option { + let filters = self.filter.as_ref()?; + let mut cache_projection = filters.predicates.first()?.projection().clone(); + for predicate in filters.predicates.iter() { + cache_projection.union(&predicate.projection()); + } + cache_projection.intersect(projection); + Some(cache_projection) + } } struct ReaderRowGroups { @@ -942,7 +968,12 @@ impl ParquetRecordBatchReader { selection: Option, ) -> Result { let array_reader = ArrayReaderBuilder::new(row_groups) - .build_array_reader(levels.levels.as_ref(), &ProjectionMask::all())?; + .build_array_reader( + levels.levels.as_ref(), + &ProjectionMask::all(), + &ProjectionMask::all(), + Arc::new(Mutex::new(RowGroupCache::new(batch_size))), + )?; let read_plan = ReadPlanBuilder::new(batch_size) .with_selection(selection) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 611d6999e07e..2eb71fca14d9 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -26,7 +26,7 @@ use std::fmt::Formatter; use std::io::SeekFrom; use std::ops::Range; use std::pin::Pin; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use std::task::{Context, Poll}; use bytes::{Buf, Bytes}; @@ -38,7 +38,7 @@ use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt}; use arrow_array::RecordBatch; use arrow_schema::{DataType, Fields, Schema, SchemaRef}; -use crate::arrow::array_reader::{ArrayReaderBuilder, RowGroups}; +use crate::arrow::array_reader::{ArrayReaderBuilder, RowGroupCache, RowGroups}; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection, @@ -588,6 +588,12 @@ where .filter(|index| !index.is_empty()) .map(|x| x[row_group_idx].as_slice()); + let cache_projection = match self.compute_cache_projection(&projection) { + Some(projection) => projection, + None => ProjectionMask::none(meta.columns().len()), + }; + let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new(batch_size))); + let mut row_group = InMemoryRowGroup { // schema: meta.schema_descr_ptr(), row_count: meta.num_rows() as usize, @@ -613,8 +619,14 @@ where .fetch(&mut self.input, predicate.projection(), selection) .await?; - let array_reader = ArrayReaderBuilder::new(&row_group) - .build_array_reader(self.fields.as_deref(), predicate.projection())?; + let mut cache_projection = predicate.projection().clone(); + cache_projection.intersect(&projection); + let array_reader = ArrayReaderBuilder::new(&row_group).build_array_reader( + self.fields.as_deref(), + predicate.projection(), + &cache_projection, + row_group_cache.clone(), + )?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } @@ -662,12 +674,27 @@ where let plan = plan_builder.build(); let array_reader = ArrayReaderBuilder::new(&row_group) - .build_array_reader(self.fields.as_deref(), &projection)?; + .build_array_reader( + self.fields.as_deref(), + &projection, + &cache_projection, + row_group_cache.clone(), + )?; let reader = ParquetRecordBatchReader::new(array_reader, plan); Ok((self, Some(reader))) } + + fn compute_cache_projection(&self, projection: &ProjectionMask) -> Option { + let filters = self.filter.as_ref()?; + let mut cache_projection = filters.predicates.first()?.projection().clone(); + for predicate in filters.predicates.iter() { + cache_projection.union(&predicate.projection()); + } + cache_projection.intersect(projection); + Some(cache_projection) + } } enum StreamState { diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 33010f480898..c3debe9b1cb1 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -276,6 +276,12 @@ impl ProjectionMask { Self { mask: None } } + pub fn none(len: usize) -> Self { + Self { + mask: Some(vec![false; len]), + } + } + /// Create a [`ProjectionMask`] which selects only the specified leaf columns /// /// Note: repeated or out of order indices will not impact the final mask From 2e01e56770cf7497a8381d72ebca36a49aa26dc9 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 1 Jul 2025 15:06:17 -0500 Subject: [PATCH 02/38] update --- .../arrow/array_reader/cached_array_reader.rs | 420 ++++++++---------- 1 file changed, 186 insertions(+), 234 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 99c05000e472..4b81e19881a3 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -1,44 +1,17 @@ use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; +use crate::arrow::arrow_reader::RowSelector; use crate::errors::Result; -use arrow_array::{new_empty_array, ArrayRef, BooleanArray}; -use arrow_buffer::BooleanBufferBuilder; +use arrow_array::{new_empty_array, ArrayRef}; use arrow_schema::DataType as ArrowType; -use arrow_select::concat::concat; -use arrow_select::filter::filter; use std::any::Any; use std::collections::VecDeque; use std::sync::{Arc, Mutex}; -/// Row selector indicating whether to read or skip rows -#[derive(Debug, Clone)] -struct RowSelector { - /// Number of rows in this selection - row_count: usize, - /// Whether to skip (true) or read (false) these rows - skip: bool, -} - -impl RowSelector { - fn select(row_count: usize) -> Self { - Self { - row_count, - skip: false, - } - } - - fn skip(row_count: usize) -> Self { - Self { - row_count, - skip: true, - } - } -} - /// A cached wrapper around an ArrayReader that avoids duplicate decoding /// when the same column appears in both filter predicates and output projection. /// -/// This reader accumulates read/skip selections and processes them efficiently -/// using cached batch data when consume_batch() is called. +/// This reader acts as a transparent layer over the inner reader, using a cache +/// to avoid redundant work when the same data is needed multiple times. pub struct CachedArrayReader { /// The underlying array reader inner: Box, @@ -46,12 +19,17 @@ pub struct CachedArrayReader { cache: Arc>, /// Column index for cache key generation column_idx: usize, - /// Current row position in the row group - current_row: usize, - /// Inner reader's row position (to keep it in sync) - inner_reader_position: usize, - /// Queue of read/skip selections to process - selection: VecDeque, + /// Current logical position in the data stream (for cache key generation) + outer_position: usize, + /// Current position in the inner reader + inner_position: usize, + /// Number of records accumulated to be returned in next consume_batch() + pending_records: usize, + /// Number of records to skip in next batch operation + pending_skips: usize, + /// Batch size for the cache + batch_size: usize, + selections: VecDeque, } impl CachedArrayReader { @@ -61,83 +39,47 @@ impl CachedArrayReader { cache: Arc>, column_idx: usize, ) -> Self { + let batch_size = cache.lock().unwrap().batch_size(); Self { inner, cache, column_idx, - current_row: 0, - inner_reader_position: 0, - selection: VecDeque::new(), + outer_position: 0, + inner_position: 0, + pending_records: 0, + pending_skips: 0, + batch_size, + selections: VecDeque::new(), } } /// Gets the batch size from the cache fn batch_size(&self) -> usize { - self.cache.lock().unwrap().batch_size() + self.batch_size } - /// Calculates the batch ID for a given row position - fn batch_id_from_row(&self, row: usize) -> usize { - row / self.batch_size() + fn get_batch_id_from_position(&self, position: usize) -> usize { + position / self.batch_size } - /// Ensures a batch is cached by fetching it from the inner reader if needed. - /// - /// Returns the number of rows cached for this batch. If `0` is returned it - /// indicates the inner reader has no more data (EOF). - fn ensure_cached(&mut self, batch_id: usize) -> Result { - let batch_start_row = batch_id * self.batch_size(); - - // Check if already cached - if let Some(array) = self - .cache - .lock() - .unwrap() - .get(self.column_idx, batch_start_row) - { - return Ok(array.len()); - } + fn sync_inner_position(&mut self) {} - // Sync inner reader to batch start if needed - if self.inner_reader_position < batch_start_row { - let to_skip = batch_start_row - self.inner_reader_position; + fn fetch_batch(&mut self, batch_id: usize) -> Result { + let row_id = batch_id * self.batch_size; + if self.inner_position < row_id { + let to_skip = row_id - self.inner_position; let skipped = self.inner.skip_records(to_skip)?; - self.inner_reader_position += skipped; - - // Reached EOF before desired position - if skipped < to_skip { - return Ok(0); - } - } - - // Read the batch - let expected_batch = self.batch_size(); - let records_read = self.inner.read_records(expected_batch)?; - - // If no more rows, signal EOF - if records_read == 0 { - return Ok(0); + self.inner_position += skipped; } + let read = self.inner.read_records(self.batch_size)?; let array = self.inner.consume_batch()?; - - // Cache it self.cache .lock() .unwrap() - .insert(self.column_idx, batch_start_row, array.clone()); - self.inner_reader_position += records_read; - - Ok(array.len()) - } - - /// Converts selection queue to boolean buffer for filtering - fn selection_to_boolean_buffer(&self, total_rows: usize) -> arrow_buffer::BooleanBuffer { - let mut builder = BooleanBufferBuilder::new(total_rows); - for selector in &self.selection { - builder.append_n(selector.row_count, !selector.skip); - } - builder.finish() + .insert(self.column_idx, batch_id, array); + self.inner_position += read; + Ok(read) } } @@ -150,141 +92,90 @@ impl ArrayReader for CachedArrayReader { self.inner.get_data_type() } - fn read_records(&mut self, mut requested: usize) -> Result { - let mut total_read = 0; - - while requested > 0 { - let batch_id = self.batch_id_from_row(self.current_row); - - // Ensure the batch is cached and determine its length - let cached_len = self.ensure_cached(batch_id)?; - - // EOF reached - if cached_len == 0 { - break; - } - - let batch_start = batch_id * self.batch_size(); - let offset_in_batch = self.current_row - batch_start; - - // If current position is past the cached length, advance to next batch. - // If this batch was partial (i.e. smaller than the configured batch size) - // this indicates we have reached EOF and should stop. - if offset_in_batch >= cached_len { - self.current_row = batch_start + cached_len; - - // Partial batch implies EOF has been reached - if cached_len < self.batch_size() { - break; + fn read_records(&mut self, num_records: usize) -> Result { + let mut read = 0; + while read < num_records { + let batch_id = self.get_batch_id_from_position(self.outer_position + read + 1); // +1 because we want to read the next batch + let cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); + + match cached { + Some(array) => { + let array_len = array.len(); + if array_len + batch_id * self.batch_size - self.outer_position > 0 { + // the cache batch has some records that we can select + let v = array_len + batch_id * self.batch_size - self.outer_position; + let select_cnt = std::cmp::min(num_records - read, v); + read += select_cnt; + self.selections.push_back(RowSelector::select(select_cnt)); + } else { + // this is last batch and we have used all records from it + break; + } + } + None => { + let read_from_inner = self.fetch_batch(batch_id)?; + + let select_from_this_batch = std::cmp::min(num_records - read, read_from_inner); + read += select_from_this_batch; + self.selections + .push_back(RowSelector::select(select_from_this_batch)); + if read_from_inner < self.batch_size { + // this is last batch from inner reader + break; + } } - - continue; - } - - let available = cached_len - offset_in_batch; - let to_read = available.min(requested); - - // Record the selection - if to_read > 0 { - self.selection.push_back(RowSelector::select(to_read)); - self.current_row += to_read; - requested -= to_read; - total_read += to_read; } } - - Ok(total_read) + self.outer_position += read; + Ok(read) } fn skip_records(&mut self, num_records: usize) -> Result { - // Add selection for skipping these records - self.selection.push_back(RowSelector::skip(num_records)); - self.current_row += num_records; - // Note: we don't cache or sync inner reader for skips + let mut skipped = 0; + while skipped < num_records { + let size = std::cmp::min(num_records - skipped, self.batch_size); + skipped += size; + self.selections.push_back(RowSelector::skip(size)); + self.outer_position += size; + } Ok(num_records) } fn consume_batch(&mut self) -> Result { - if self.selection.is_empty() { - // Return empty array of correct type - return Ok(new_empty_array(self.get_data_type())); + let row_count = self.selections.iter().map(|s| s.row_count).sum::(); + if row_count == 0 { + return Ok(new_empty_array(&self.inner.get_data_type())); } - // Calculate total row count and starting position - let total_rows: usize = self.selection.iter().map(|s| s.row_count).sum(); - let start_row = self.current_row - total_rows; + let mut start_position = self.outer_position - row_count; - if total_rows == 0 { - self.selection.clear(); - return Ok(new_empty_array(self.get_data_type())); - } - - // Create boolean selection buffer - let selection_buffer = self.selection_to_boolean_buffer(total_rows); - - // Determine batch range - let batch_size = self.batch_size(); - let start_batch = start_row / batch_size; - let end_batch = (start_row + total_rows - 1) / batch_size; - - let mut result_arrays = Vec::new(); - - for batch_id in start_batch..=end_batch { - let batch_start = batch_id * batch_size; - let batch_end = batch_start + batch_size - 1; + let mut selected_arrays = Vec::new(); + for selector in self.selections.iter() { + if !selector.skip { + let batch_id = self.get_batch_id_from_position(start_position); + let batch_start = batch_id * self.batch_size; - // Calculate overlap between our selection and this batch - let overlap_start = start_row.max(batch_start); - let overlap_end = (start_row + total_rows - 1).min(batch_end); - - if overlap_start > overlap_end { - continue; // No overlap - } - - // Get selection slice for this batch - let selection_start = overlap_start - start_row; - let selection_length = overlap_end - overlap_start + 1; - let batch_selection = selection_buffer.slice(selection_start, selection_length); - - if batch_selection.count_set_bits() == 0 { - continue; // Nothing selected in this batch - } + let cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); + let cached = cached.expect("data must be already cached in the read_records call"); - // Get cached array and apply filter - let cached_array = self - .cache - .lock() - .unwrap() - .get(self.column_idx, batch_start) - .unwrap(); - let mask = BooleanArray::from(batch_selection); - - // Calculate offset within the batch - let batch_offset = overlap_start - batch_start; - let batch_slice = if batch_offset > 0 || selection_length < batch_size { - cached_array.slice(batch_offset, selection_length) - } else { - cached_array - }; - - let filtered = filter(&batch_slice, &mask)?; - if filtered.len() > 0 { - result_arrays.push(filtered); + let slice_start = start_position - batch_start; + let sliced = cached.slice(slice_start, selector.row_count); + selected_arrays.push(sliced); } + start_position += selector.row_count; } - // Clear selection queue - self.selection.clear(); - - // Concatenate results - match result_arrays.len() { - 0 => Ok(new_empty_array(self.get_data_type())), - 1 => Ok(result_arrays.into_iter().next().unwrap()), - _ => { - let array_refs: Vec<&dyn arrow_array::Array> = - result_arrays.iter().map(|a| a.as_ref()).collect(); - Ok(concat(&array_refs)?) - } + self.selections.clear(); + + match selected_arrays.len() { + 0 => Ok(new_empty_array(&self.inner.get_data_type())), + 1 => Ok(selected_arrays.into_iter().next().unwrap()), + _ => Ok(arrow_select::concat::concat( + &selected_arrays + .iter() + .map(|a| a.as_ref()) + .collect::>(), + )?), } } @@ -309,7 +200,7 @@ mod tests { struct MockArrayReader { data: Vec, position: usize, - read_position: usize, + records_to_consume: usize, data_type: ArrowType, } @@ -318,7 +209,7 @@ mod tests { Self { data, position: 0, - read_position: 0, + records_to_consume: 0, data_type: ArrowType::Int32, } } @@ -336,14 +227,16 @@ mod tests { fn read_records(&mut self, batch_size: usize) -> Result { let remaining = self.data.len() - self.position; let to_read = std::cmp::min(batch_size, remaining); - self.read_position = self.position; - self.position += to_read; + self.records_to_consume += to_read; Ok(to_read) } fn consume_batch(&mut self) -> Result { - // Return data from read_position to current position - let slice = &self.data[self.read_position..self.position]; + let start = self.position; + let end = start + self.records_to_consume; + let slice = &self.data[start..end]; + self.position = end; + self.records_to_consume = 0; Ok(Arc::new(Int32Array::from(slice.to_vec()))) } @@ -366,16 +259,22 @@ mod tests { #[test] fn test_cached_reader_basic() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); // Read 3 records let records_read = cached_reader.read_records(3).unwrap(); assert_eq!(records_read, 3); - // Consume batch should return those 3 records let array = cached_reader.consume_batch().unwrap(); assert_eq!(array.len(), 3); + + let int32_array = array.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[1, 2, 3]); + + // Read 3 more records + let records_read = cached_reader.read_records(3).unwrap(); + assert_eq!(records_read, 2); } #[test] @@ -384,37 +283,90 @@ mod tests { let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); - // Read 2, skip 2, read 1 (total 5 records from first batch) - cached_reader.read_records(2).unwrap(); - cached_reader.skip_records(2).unwrap(); - cached_reader.read_records(1).unwrap(); + let read1 = cached_reader.read_records(2).unwrap(); + assert_eq!(read1, 2); + + let array1 = cached_reader.consume_batch().unwrap(); + assert_eq!(array1.len(), 2); + let int32_array = array1.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[1, 2]); + + let skipped = cached_reader.skip_records(2).unwrap(); + assert_eq!(skipped, 2); + + let read2 = cached_reader.read_records(1).unwrap(); + assert_eq!(read2, 1); + + // Consume it (should be the 5th element after skipping 3,4) + let array2 = cached_reader.consume_batch().unwrap(); + assert_eq!(array2.len(), 1); + let int32_array = array2.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[5]); + } - // Should get [1,2] and [5] (skipped 3,4) + #[test] + fn test_multiple_reads_before_consume() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + + // Multiple reads should accumulate + let read1 = cached_reader.read_records(2).unwrap(); + assert_eq!(read1, 2); + + let read2 = cached_reader.read_records(1).unwrap(); + assert_eq!(read2, 1); + + // Consume should return all accumulated records let array = cached_reader.consume_batch().unwrap(); - assert_eq!(array.len(), 3); // 2 + 1 reads + assert_eq!(array.len(), 3); + let int32_array = array.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[1, 2, 3]); } #[test] - fn test_cache_efficiency() { - // Test that cache avoids duplicate reads for the same batch - let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); + fn test_eof_behavior() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + + // Try to read more than available + let read1 = cached_reader.read_records(5).unwrap(); + assert_eq!(read1, 3); // Should only get 3 records (all available) + + let array1 = cached_reader.consume_batch().unwrap(); + assert_eq!(array1.len(), 3); + + // Further reads should return 0 + let read2 = cached_reader.read_records(1).unwrap(); + assert_eq!(read2, 0); + + let array2 = cached_reader.consume_batch().unwrap(); + assert_eq!(array2.len(), 0); + } + + #[test] + fn test_cache_sharing() { let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 // First reader - populate cache - let mut cached_reader1 = CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0); + let mock_reader1 = MockArrayReader::new(vec![1, 2, 3, 4, 5]); + let mut cached_reader1 = CachedArrayReader::new(Box::new(mock_reader1), cache.clone(), 0); + cached_reader1.read_records(3).unwrap(); let array1 = cached_reader1.consume_batch().unwrap(); assert_eq!(array1.len(), 3); - // Second reader - should use cache (no inner reader created) - let mock_reader2 = MockArrayReader::new(vec![10, 20, 30, 40, 50]); // Different data - let mut cached_reader2 = CachedArrayReader::new(Box::new(mock_reader2), cache.clone(), 0); + // Second reader with different column index should not interfere + let mock_reader2 = MockArrayReader::new(vec![10, 20, 30, 40, 50]); + let mut cached_reader2 = CachedArrayReader::new(Box::new(mock_reader2), cache.clone(), 1); + cached_reader2.read_records(2).unwrap(); let array2 = cached_reader2.consume_batch().unwrap(); assert_eq!(array2.len(), 2); - // The second reader should get data from cache (original [1,2,3,4,5]) - // not from its mock reader ([10,20,30,40,50]) - // This tests that cache is working + // Verify the second reader got its own data, not from cache + let int32_array = array2.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[10, 20]); } } From 0bd08c3bb5c6e0b75520de9794300254813f6a8e Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 1 Jul 2025 15:51:41 -0500 Subject: [PATCH 03/38] update --- .../arrow/array_reader/cached_array_reader.rs | 61 +++++++++++++++---- 1 file changed, 48 insertions(+), 13 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 4b81e19881a3..41c087957c1a 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -1,7 +1,8 @@ use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; use crate::arrow::arrow_reader::RowSelector; use crate::errors::Result; -use arrow_array::{new_empty_array, ArrayRef}; +use arrow_array::{new_empty_array, ArrayRef, BooleanArray}; +use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; use arrow_schema::DataType as ArrowType; use std::any::Any; use std::collections::VecDeque; @@ -95,7 +96,7 @@ impl ArrayReader for CachedArrayReader { fn read_records(&mut self, num_records: usize) -> Result { let mut read = 0; while read < num_records { - let batch_id = self.get_batch_id_from_position(self.outer_position + read + 1); // +1 because we want to read the next batch + let batch_id = self.get_batch_id_from_position(self.outer_position + read); let cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); match cached { @@ -147,22 +148,45 @@ impl ArrayReader for CachedArrayReader { return Ok(new_empty_array(&self.inner.get_data_type())); } - let mut start_position = self.outer_position - row_count; + let start_position = self.outer_position - row_count; + + let selection_buffer = row_selection_to_boolean_buffer(row_count, self.selections.iter()); + + let start_batch = start_position / self.batch_size; + let end_batch = (start_position + row_count - 1) / self.batch_size; let mut selected_arrays = Vec::new(); - for selector in self.selections.iter() { - if !selector.skip { - let batch_id = self.get_batch_id_from_position(start_position); - let batch_start = batch_id * self.batch_size; + for batch_id in start_batch..=end_batch { + let batch_start = batch_id * self.batch_size; + let batch_end = batch_start + self.batch_size - 1; + let batch_id = self.get_batch_id_from_position(batch_start); + + // Calculate the overlap between the start_position and the batch + let overlap_start = start_position.max(batch_start); + let overlap_end = (start_position + row_count - 1).min(batch_end); + + if overlap_start > overlap_end { + continue; + } - let cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); - let cached = cached.expect("data must be already cached in the read_records call"); + let selection_start = overlap_start - start_position; + let selection_length = overlap_end - overlap_start + 1; + let mask = selection_buffer.slice(selection_start, selection_length); - let slice_start = start_position - batch_start; - let sliced = cached.slice(slice_start, selector.row_count); - selected_arrays.push(sliced); + if mask.count_set_bits() == 0 { + continue; } - start_position += selector.row_count; + + let mask_array = BooleanArray::from(mask); + let cached = self + .cache + .lock() + .unwrap() + .get(self.column_idx, batch_id) + .expect("data must be already cached in the read_records call"); + let cached = cached.slice(overlap_start - batch_start, selection_length); + let filtered = arrow_select::filter::filter(&cached, &mask_array)?; + selected_arrays.push(filtered); } self.selections.clear(); @@ -188,6 +212,17 @@ impl ArrayReader for CachedArrayReader { } } +fn row_selection_to_boolean_buffer<'a>( + row_count: usize, + selection: impl Iterator, +) -> BooleanBuffer { + let mut buffer = BooleanBufferBuilder::new(row_count); + for selector in selection { + buffer.append_n(selector.row_count, !selector.skip); + } + buffer.finish() +} + #[cfg(test)] mod tests { use super::*; From d6ecbd4fb61b41afcc4f60b3d639f61f3551eed4 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 1 Jul 2025 17:22:37 -0500 Subject: [PATCH 04/38] update --- parquet/src/arrow/array_reader/builder.rs | 82 +++++++++++-------- .../arrow/array_reader/cached_array_reader.rs | 4 +- parquet/src/arrow/array_reader/list_array.rs | 6 +- parquet/src/arrow/arrow_reader/mod.rs | 32 ++------ parquet/src/arrow/async_reader/mod.rs | 24 +++--- 5 files changed, 68 insertions(+), 80 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 4229d349ef6e..8deed73756d5 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -41,6 +41,8 @@ pub struct ArrayReaderBuilder<'a> { row_groups: &'a dyn RowGroups, } +type CacheContext<'a> = (&'a ProjectionMask, Arc>); + impl<'a> ArrayReaderBuilder<'a> { pub fn new(row_groups: &'a dyn RowGroups) -> Self { Self { row_groups } @@ -51,12 +53,25 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: Option<&ParquetField>, mask: &ProjectionMask, - cache_mask: &ProjectionMask, - cache: Arc>, + ) -> Result> { + let reader = field + .and_then(|field| self.build_reader(field, mask, None).transpose()) + .transpose()? + .unwrap_or_else(|| make_empty_array_reader(self.num_rows())); + + Ok(reader) + } + + /// Create [`ArrayReader`] from parquet schema, projection mask, and parquet file reader. + pub fn build_array_reader_with_cache( + &self, + field: Option<&ParquetField>, + mask: &ProjectionMask, + cache_context: CacheContext, ) -> Result> { let reader = field .and_then(|field| { - self.build_reader(field, mask, cache_mask, cache) + self.build_reader(field, mask, Some(cache_context)) .transpose() }) .transpose()? @@ -74,32 +89,32 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_mask: &ProjectionMask, - cache: Arc>, + cache_context: Option, ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { col_idx, .. } => { - if let Some(reader) = self.build_primitive_reader(field, mask)? { - if cache_mask.leaf_included(col_idx) { - Ok(Some(Box::new(CachedArrayReader::new( - reader, cache, col_idx, - )))) - } else { - Ok(Some(reader)) - } + let Some(reader) = self.build_primitive_reader(field, mask)? else { + return Ok(None); + }; + let Some((cache_mask, cache)) = cache_context else { + return Ok(Some(reader)); + }; + + if cache_mask.leaf_included(col_idx) { + Ok(Some(Box::new(CachedArrayReader::new( + reader, cache, col_idx, + )))) } else { - Ok(None) + Ok(Some(reader)) } } ParquetFieldType::Group { .. } => match &field.arrow_type { - DataType::Map(_, _) => self.build_map_reader(field, mask, cache_mask, cache), - DataType::Struct(_) => self.build_struct_reader(field, mask, cache_mask, cache), - DataType::List(_) => self.build_list_reader(field, mask, cache_mask, cache, false), - DataType::LargeList(_) => { - self.build_list_reader(field, mask, cache_mask, cache, true) - } + DataType::Map(_, _) => self.build_map_reader(field, mask, cache_context), + DataType::Struct(_) => self.build_struct_reader(field, mask, cache_context), + DataType::List(_) => self.build_list_reader(field, mask, cache_context, false), + DataType::LargeList(_) => self.build_list_reader(field, mask, cache_context, true), DataType::FixedSizeList(_, _) => { - self.build_fixed_size_list_reader(field, mask, cache_mask, cache) + self.build_fixed_size_list_reader(field, mask, cache_context) } d => unimplemented!("reading group type {} not implemented", d), }, @@ -111,14 +126,13 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_mask: &ProjectionMask, - cache: Arc>, + cache_context: Option, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 2); - let key_reader = self.build_reader(&children[0], mask, cache_mask, cache.clone())?; - let value_reader = self.build_reader(&children[1], mask, cache_mask, cache)?; + let key_reader = self.build_reader(&children[0], mask, cache_context.clone())?; + let value_reader = self.build_reader(&children[1], mask, cache_context)?; match (key_reader, value_reader) { (Some(key_reader), Some(value_reader)) => { @@ -164,14 +178,13 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_mask: &ProjectionMask, - cache: Arc>, + cache_context: Option, is_large: bool, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, cache_mask, cache)? { + let reader = match self.build_reader(&children[0], mask, cache_context)? { Some(item_reader) => { // Need to retrieve underlying data type to handle projection let item_type = item_reader.get_data_type().clone(); @@ -213,13 +226,12 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_mask: &ProjectionMask, - cache: Arc>, + cache_context: Option, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, cache_mask, cache)? { + let reader = match self.build_reader(&children[0], mask, cache_context)? { Some(item_reader) => { let item_type = item_reader.get_data_type().clone(); let reader = match &field.arrow_type { @@ -349,8 +361,7 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_mask: &ProjectionMask, - cache: Arc>, + cache_context: Option, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -363,7 +374,7 @@ impl<'a> ArrayReaderBuilder<'a> { let mut builder = SchemaBuilder::with_capacity(children.len()); for (arrow, parquet) in arrow_fields.iter().zip(children) { - if let Some(reader) = self.build_reader(parquet, mask, cache_mask, cache.clone())? { + if let Some(reader) = self.build_reader(parquet, mask, cache_context.clone())? { // Need to retrieve underlying data type to handle projection let child_type = reader.get_data_type().clone(); builder.push(arrow.as_ref().clone().with_data_type(child_type)); @@ -407,10 +418,9 @@ mod tests { file_metadata.key_value_metadata(), ) .unwrap(); - let cache = Arc::new(Mutex::new(RowGroupCache::new(1000))); let array_reader = ArrayReaderBuilder::new(&file_reader) - .build_array_reader(fields.as_ref(), &mask, &ProjectionMask::all(), cache) + .build_array_reader(fields.as_ref(), &mask) .unwrap(); // Create arrow types diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 41c087957c1a..c2a2b7260da0 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -204,11 +204,11 @@ impl ArrayReader for CachedArrayReader { } fn get_def_levels(&self) -> Option<&[i16]> { - self.inner.get_def_levels() + None // we don't allow nullable parent for now. } fn get_rep_levels(&self) -> Option<&[i16]> { - self.inner.get_rep_levels() + None } } diff --git a/parquet/src/arrow/array_reader/list_array.rs b/parquet/src/arrow/array_reader/list_array.rs index 06b17211d187..66c4f30b3c29 100644 --- a/parquet/src/arrow/array_reader/list_array.rs +++ b/parquet/src/arrow/array_reader/list_array.rs @@ -247,7 +247,6 @@ impl ArrayReader for ListArrayReader { mod tests { use super::*; use crate::arrow::array_reader::list_array::ListArrayReader; - use crate::arrow::array_reader::row_group_cache::RowGroupCache; use crate::arrow::array_reader::test_util::InMemoryArrayReader; use crate::arrow::array_reader::ArrayReaderBuilder; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; @@ -260,7 +259,7 @@ mod tests { use arrow_array::{Array, PrimitiveArray}; use arrow_data::ArrayDataBuilder; use arrow_schema::Fields; - use std::sync::{Arc, Mutex}; + use std::sync::Arc; fn list_type( data_type: ArrowType, @@ -563,10 +562,9 @@ mod tests { file_metadata.key_value_metadata(), ) .unwrap(); - let cache = Arc::new(Mutex::new(RowGroupCache::new(1000))); let mut array_reader = ArrayReaderBuilder::new(&file_reader) - .build_array_reader(fields.as_ref(), &mask, &ProjectionMask::all(), cache) + .build_array_reader(fields.as_ref(), &mask) .unwrap(); let batch = array_reader.next_batch(100).unwrap(); diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 6f15e08443e7..040126ec16cc 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -24,10 +24,10 @@ use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; pub use crate::arrow::array_reader::RowGroups; -use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder, RowGroupCache}; +use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; use crate::arrow::schema::{parquet_to_arrow_schema_and_fields, ParquetField}; use crate::arrow::{parquet_to_arrow_field_levels, FieldLevels, ProjectionMask}; use crate::column::page::{PageIterator, PageReader}; @@ -711,10 +711,6 @@ impl ParquetRecordBatchReaderBuilder { let batch_size = self .batch_size .min(self.metadata.file_metadata().num_rows() as usize); - let cache_projection = match self.compute_cache_projection(&self.projection) { - Some(projection) => projection, - None => ProjectionMask::all(), - }; let row_groups = self .row_groups @@ -725,7 +721,6 @@ impl ParquetRecordBatchReaderBuilder { metadata: self.metadata, row_groups, }; - let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new(batch_size))); let mut filter = self.filter; let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(self.selection); @@ -741,23 +736,15 @@ impl ParquetRecordBatchReaderBuilder { let mut cache_projection = predicate.projection().clone(); cache_projection.intersect(&self.projection); - let array_reader = ArrayReaderBuilder::new(&reader).build_array_reader( - self.fields.as_deref(), - predicate.projection(), - &cache_projection, - row_group_cache.clone(), - )?; + let array_reader = ArrayReaderBuilder::new(&reader) + .build_array_reader(self.fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } } - let array_reader = ArrayReaderBuilder::new(&reader).build_array_reader( - self.fields.as_deref(), - &self.projection, - &cache_projection, - row_group_cache.clone(), - )?; + let array_reader = ArrayReaderBuilder::new(&reader) + .build_array_reader(self.fields.as_deref(), &self.projection)?; let read_plan = plan_builder .limited(reader.num_rows()) @@ -968,12 +955,7 @@ impl ParquetRecordBatchReader { selection: Option, ) -> Result { let array_reader = ArrayReaderBuilder::new(row_groups) - .build_array_reader( - levels.levels.as_ref(), - &ProjectionMask::all(), - &ProjectionMask::all(), - Arc::new(Mutex::new(RowGroupCache::new(batch_size))), - )?; + .build_array_reader(levels.levels.as_ref(), &ProjectionMask::all())?; let read_plan = ReadPlanBuilder::new(batch_size) .with_selection(selection) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 2eb71fca14d9..4b249e9ffdd1 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -621,12 +621,12 @@ where let mut cache_projection = predicate.projection().clone(); cache_projection.intersect(&projection); - let array_reader = ArrayReaderBuilder::new(&row_group).build_array_reader( - self.fields.as_deref(), - predicate.projection(), - &cache_projection, - row_group_cache.clone(), - )?; + let array_reader = ArrayReaderBuilder::new(&row_group) + .build_array_reader_with_cache( + self.fields.as_deref(), + predicate.projection(), + (&cache_projection, row_group_cache.clone()), + )?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } @@ -673,13 +673,11 @@ where let plan = plan_builder.build(); - let array_reader = ArrayReaderBuilder::new(&row_group) - .build_array_reader( - self.fields.as_deref(), - &projection, - &cache_projection, - row_group_cache.clone(), - )?; + let array_reader = ArrayReaderBuilder::new(&row_group).build_array_reader_with_cache( + self.fields.as_deref(), + &projection, + (&cache_projection, row_group_cache.clone()), + )?; let reader = ParquetRecordBatchReader::new(array_reader, plan); From 7cd5518157574ff14c3ad0a388fa9c7395182e5e Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 08:52:41 -0500 Subject: [PATCH 05/38] cleanup --- .../src/arrow/array_reader/cached_array_reader.rs | 15 ++------------- parquet/src/arrow/array_reader/row_group_cache.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 10 ---------- parquet/src/arrow/mod.rs | 1 + 4 files changed, 4 insertions(+), 24 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index c2a2b7260da0..8eab24a3c2af 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -24,12 +24,9 @@ pub struct CachedArrayReader { outer_position: usize, /// Current position in the inner reader inner_position: usize, - /// Number of records accumulated to be returned in next consume_batch() - pending_records: usize, - /// Number of records to skip in next batch operation - pending_skips: usize, /// Batch size for the cache batch_size: usize, + /// Selections to be applied to the next consume_batch() selections: VecDeque, } @@ -41,30 +38,22 @@ impl CachedArrayReader { column_idx: usize, ) -> Self { let batch_size = cache.lock().unwrap().batch_size(); + Self { inner, cache, column_idx, outer_position: 0, inner_position: 0, - pending_records: 0, - pending_skips: 0, batch_size, selections: VecDeque::new(), } } - /// Gets the batch size from the cache - fn batch_size(&self) -> usize { - self.batch_size - } - fn get_batch_id_from_position(&self, position: usize) -> usize { position / self.batch_size } - fn sync_inner_position(&mut self) {} - fn fetch_batch(&mut self, batch_id: usize) -> Result { let row_id = batch_id * self.batch_size; if self.inner_position < row_id { diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 3fb21f41a771..671bbdb78412 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -18,7 +18,7 @@ pub struct CacheKey { pub struct RowGroupCache { /// Cache storage mapping (column_idx, row_id) -> ArrayRef cache: HashMap, - /// Batch size used for cache entries + /// Cache granularity batch_size: usize, } diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 040126ec16cc..ed390ed7fd9e 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -755,16 +755,6 @@ impl ParquetRecordBatchReaderBuilder { Ok(ParquetRecordBatchReader::new(array_reader, read_plan)) } - - fn compute_cache_projection(&self, projection: &ProjectionMask) -> Option { - let filters = self.filter.as_ref()?; - let mut cache_projection = filters.predicates.first()?.projection().clone(); - for predicate in filters.predicates.iter() { - cache_projection.union(&predicate.projection()); - } - cache_projection.intersect(projection); - Some(cache_projection) - } } struct ReaderRowGroups { diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index c3debe9b1cb1..72626d70e0e5 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -276,6 +276,7 @@ impl ProjectionMask { Self { mask: None } } + /// Create a [`ProjectionMask`] which selects no columns pub fn none(len: usize) -> Self { Self { mask: Some(vec![false; len]), From 452004819fb8acdb5cbd0790b09907ef3d2a12a2 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 09:46:23 -0500 Subject: [PATCH 06/38] update --- parquet/src/arrow/array_reader/builder.rs | 8 +- .../arrow/array_reader/cached_array_reader.rs | 161 +++++++++++++++++- parquet/src/arrow/array_reader/mod.rs | 1 + .../src/arrow/array_reader/row_group_cache.rs | 47 +++++ parquet/src/arrow/async_reader/mod.rs | 4 +- 5 files changed, 209 insertions(+), 12 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 8deed73756d5..91fd25944097 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -21,6 +21,7 @@ use arrow_schema::{DataType, Fields, SchemaBuilder}; use crate::arrow::array_reader::byte_view_array::make_byte_view_array_reader; use crate::arrow::array_reader::cached_array_reader::CachedArrayReader; +use crate::arrow::array_reader::cached_array_reader::CacheRole; use crate::arrow::array_reader::empty_array::make_empty_array_reader; use crate::arrow::array_reader::fixed_len_byte_array::make_fixed_len_byte_array_reader; use crate::arrow::array_reader::row_group_cache::RowGroupCache; @@ -41,7 +42,8 @@ pub struct ArrayReaderBuilder<'a> { row_groups: &'a dyn RowGroups, } -type CacheContext<'a> = (&'a ProjectionMask, Arc>); +/// Cache context type containing projection mask, cache, and role +type CacheContext<'a> = (&'a ProjectionMask, Arc>, CacheRole); impl<'a> ArrayReaderBuilder<'a> { pub fn new(row_groups: &'a dyn RowGroups) -> Self { @@ -96,13 +98,13 @@ impl<'a> ArrayReaderBuilder<'a> { let Some(reader) = self.build_primitive_reader(field, mask)? else { return Ok(None); }; - let Some((cache_mask, cache)) = cache_context else { + let Some((cache_mask, cache, role)) = cache_context else { return Ok(Some(reader)); }; if cache_mask.leaf_included(col_idx) { Ok(Some(Box::new(CachedArrayReader::new( - reader, cache, col_idx, + reader, cache, col_idx, role, )))) } else { Ok(Some(reader)) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 8eab24a3c2af..2905b6fcb622 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -8,11 +8,40 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::{Arc, Mutex}; +/// Role of the cached array reader +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum CacheRole { + /// Producer role: inserts data into the cache during filter phase + Producer, + /// Consumer role: removes consumed data from the cache during output building phase + Consumer, +} + /// A cached wrapper around an ArrayReader that avoids duplicate decoding /// when the same column appears in both filter predicates and output projection. /// /// This reader acts as a transparent layer over the inner reader, using a cache /// to avoid redundant work when the same data is needed multiple times. +/// +/// The reader can operate in two roles: +/// - Producer: During filter phase, inserts decoded data into the cache +/// - Consumer: During output building, consumes and removes data from the cache +/// +/// This means the memory consumption of the cache has two stages: +/// 1. During the filter phase, the memory increases as the cache is populated +/// 2. It peaks when filters are built. +/// 3. It decreases as the cached data is consumed. +/// ▲ +/// │ ╭─╮ +/// │ ╱ ╲ +/// │ ╱ ╲ +/// │ ╱ ╲ +/// │ ╱ ╲ +/// │╱ ╲ +/// └─────────────╲──────► Time +/// │ │ │ +/// Filter Peak Consume +/// Phase (Built) (Decrease) pub struct CachedArrayReader { /// The underlying array reader inner: Box, @@ -28,14 +57,17 @@ pub struct CachedArrayReader { batch_size: usize, /// Selections to be applied to the next consume_batch() selections: VecDeque, + /// Role of this reader (Producer or Consumer) + role: CacheRole, } impl CachedArrayReader { - /// Creates a new cached array reader + /// Creates a new cached array reader with the specified role pub fn new( inner: Box, cache: Arc>, column_idx: usize, + role: CacheRole, ) -> Self { let batch_size = cache.lock().unwrap().batch_size(); @@ -47,6 +79,7 @@ impl CachedArrayReader { inner_position: 0, batch_size, selections: VecDeque::new(), + role, } } @@ -64,13 +97,33 @@ impl CachedArrayReader { let read = self.inner.read_records(self.batch_size)?; let array = self.inner.consume_batch()?; + + // Both producers and consumers insert data into the cache when fetching from underlying reader + // The difference is that consumers will remove consumed batches later self.cache .lock() .unwrap() .insert(self.column_idx, batch_id, array); + self.inner_position += read; Ok(read) } + + /// Remove batches from cache that have been completely consumed + /// This is only called for Consumer role readers + fn cleanup_consumed_batches(&mut self) { + let current_batch_id = self.get_batch_id_from_position(self.outer_position); + + // Remove batches that are at least one batch behind the current position + // This ensures we don't remove batches that might still be needed for the current batch + // We can safely remove batch_id if current_batch_id > batch_id + 1 + if current_batch_id > 1 { + let mut cache = self.cache.lock().unwrap(); + for batch_id_to_remove in 0..(current_batch_id - 1) { + cache.remove(self.column_idx, batch_id_to_remove); + } + } + } } impl ArrayReader for CachedArrayReader { @@ -180,6 +233,12 @@ impl ArrayReader for CachedArrayReader { self.selections.clear(); + // For consumers, cleanup batches that have been completely consumed + // This reduces the memory usage of the cache + if self.role == CacheRole::Consumer { + self.cleanup_consumed_batches(); + } + match selected_arrays.len() { 0 => Ok(new_empty_array(&self.inner.get_data_type())), 1 => Ok(selected_arrays.into_iter().next().unwrap()), @@ -284,7 +343,8 @@ mod tests { fn test_cached_reader_basic() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 - let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + let mut cached_reader = + CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Producer); // Read 3 records let records_read = cached_reader.read_records(3).unwrap(); @@ -305,7 +365,8 @@ mod tests { fn test_read_skip_pattern() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 - let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + let mut cached_reader = + CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); let read1 = cached_reader.read_records(2).unwrap(); assert_eq!(read1, 2); @@ -332,7 +393,8 @@ mod tests { fn test_multiple_reads_before_consume() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 - let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + let mut cached_reader = + CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); // Multiple reads should accumulate let read1 = cached_reader.read_records(2).unwrap(); @@ -352,7 +414,8 @@ mod tests { fn test_eof_behavior() { let mock_reader = MockArrayReader::new(vec![1, 2, 3]); let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 - let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0); + let mut cached_reader = + CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); // Try to read more than available let read1 = cached_reader.read_records(5).unwrap(); @@ -375,7 +438,12 @@ mod tests { // First reader - populate cache let mock_reader1 = MockArrayReader::new(vec![1, 2, 3, 4, 5]); - let mut cached_reader1 = CachedArrayReader::new(Box::new(mock_reader1), cache.clone(), 0); + let mut cached_reader1 = CachedArrayReader::new( + Box::new(mock_reader1), + cache.clone(), + 0, + CacheRole::Producer, + ); cached_reader1.read_records(3).unwrap(); let array1 = cached_reader1.consume_batch().unwrap(); @@ -383,7 +451,12 @@ mod tests { // Second reader with different column index should not interfere let mock_reader2 = MockArrayReader::new(vec![10, 20, 30, 40, 50]); - let mut cached_reader2 = CachedArrayReader::new(Box::new(mock_reader2), cache.clone(), 1); + let mut cached_reader2 = CachedArrayReader::new( + Box::new(mock_reader2), + cache.clone(), + 1, + CacheRole::Consumer, + ); cached_reader2.read_records(2).unwrap(); let array2 = cached_reader2.consume_batch().unwrap(); @@ -393,4 +466,78 @@ mod tests { let int32_array = array2.as_any().downcast_ref::().unwrap(); assert_eq!(int32_array.values(), &[10, 20]); } + + #[test] + fn test_consumer_removes_batches() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let mut consumer_reader = + CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + + // Read first batch (positions 0-2, batch 0) + let read1 = consumer_reader.read_records(3).unwrap(); + assert_eq!(read1, 3); + assert_eq!(consumer_reader.outer_position, 3); + // Check that batch 0 is in cache after read_records + assert!(cache.lock().unwrap().get(0, 0).is_some()); + + let array1 = consumer_reader.consume_batch().unwrap(); + assert_eq!(array1.len(), 3); + + // After first consume_batch, batch 0 should still be in cache + // (current_batch_id = 3/3 = 1, cleanup only happens if current_batch_id > 1) + assert!(cache.lock().unwrap().get(0, 0).is_some()); + + // Read second batch (positions 3-5, batch 1) + let read2 = consumer_reader.read_records(3).unwrap(); + assert_eq!(read2, 3); + assert_eq!(consumer_reader.outer_position, 6); + let array2 = consumer_reader.consume_batch().unwrap(); + assert_eq!(array2.len(), 3); + + // After second consume_batch, batch 0 should be removed + // (current_batch_id = 6/3 = 2, cleanup removes batches 0..(2-1) = 0..1, so removes batch 0) + assert!(cache.lock().unwrap().get(0, 0).is_none()); + assert!(cache.lock().unwrap().get(0, 1).is_some()); + + // Read third batch (positions 6-8, batch 2) + let read3 = consumer_reader.read_records(3).unwrap(); + assert_eq!(read3, 3); + assert_eq!(consumer_reader.outer_position, 9); + let array3 = consumer_reader.consume_batch().unwrap(); + assert_eq!(array3.len(), 3); + + // After third consume_batch, batches 0 and 1 should be removed + // (current_batch_id = 9/3 = 3, cleanup removes batches 0..(3-1) = 0..2, so removes batches 0 and 1) + assert!(cache.lock().unwrap().get(0, 0).is_none()); + assert!(cache.lock().unwrap().get(0, 1).is_none()); + assert!(cache.lock().unwrap().get(0, 2).is_some()); + } + + #[test] + fn test_producer_keeps_batches() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let mut producer_reader = + CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Producer); + + // Read first batch (positions 0-2) + let read1 = producer_reader.read_records(3).unwrap(); + assert_eq!(read1, 3); + let array1 = producer_reader.consume_batch().unwrap(); + assert_eq!(array1.len(), 3); + + // Verify batch 0 is in cache + assert!(cache.lock().unwrap().get(0, 0).is_some()); + + // Read second batch (positions 3-5) - producer should NOT remove batch 0 + let read2 = producer_reader.read_records(3).unwrap(); + assert_eq!(read2, 3); + let array2 = producer_reader.consume_batch().unwrap(); + assert_eq!(array2.len(), 3); + + // Verify both batch 0 and batch 1 are still present (no removal for producer) + assert!(cache.lock().unwrap().get(0, 0).is_some()); + assert!(cache.lock().unwrap().get(0, 1).is_some()); + } } diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 208799f5ff71..8e1b9d7d37f8 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -53,6 +53,7 @@ pub use byte_array::make_byte_array_reader; pub use byte_array_dictionary::make_byte_array_dictionary_reader; #[allow(unused_imports)] // Only used for benchmarks pub use byte_view_array::make_byte_view_array_reader; +pub use cached_array_reader::CacheRole; #[allow(unused_imports)] // Only used for benchmarks pub use fixed_len_byte_array::make_fixed_len_byte_array_reader; pub use fixed_size_list_array::FixedSizeListArrayReader; diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 671bbdb78412..150f9edd406d 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -48,6 +48,13 @@ impl RowGroupCache { pub fn batch_size(&self) -> usize { self.batch_size } + + /// Removes a cached array for the given column and row ID + /// Returns true if the entry was found and removed, false otherwise + pub fn remove(&mut self, column_idx: usize, row_id: usize) -> bool { + let key = CacheKey { column_idx, row_id }; + self.cache.remove(&key).is_some() + } } #[cfg(test)] @@ -77,4 +84,44 @@ mod tests { let miss = cache.get(0, 1000); assert!(miss.is_none()); } + + #[test] + fn test_cache_remove() { + let mut cache = RowGroupCache::new(1000); + + // Create test arrays + let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); + let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); + + // Insert arrays + cache.insert(0, 0, array1.clone()); + cache.insert(0, 1000, array2.clone()); + cache.insert(1, 0, array1.clone()); + + // Verify they're there + assert!(cache.get(0, 0).is_some()); + assert!(cache.get(0, 1000).is_some()); + assert!(cache.get(1, 0).is_some()); + + // Remove one entry + let removed = cache.remove(0, 0); + assert!(removed); + assert!(cache.get(0, 0).is_none()); + + // Other entries should still be there + assert!(cache.get(0, 1000).is_some()); + assert!(cache.get(1, 0).is_some()); + + // Try to remove non-existent entry + let not_removed = cache.remove(0, 0); + assert!(!not_removed); + + // Remove remaining entries + assert!(cache.remove(0, 1000)); + assert!(cache.remove(1, 0)); + + // Cache should be empty + assert!(cache.get(0, 1000).is_none()); + assert!(cache.get(1, 0).is_none()); + } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 4b249e9ffdd1..aeba334ec431 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -625,7 +625,7 @@ where .build_array_reader_with_cache( self.fields.as_deref(), predicate.projection(), - (&cache_projection, row_group_cache.clone()), + (&cache_projection, row_group_cache.clone(), crate::arrow::array_reader::CacheRole::Producer), )?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; @@ -676,7 +676,7 @@ where let array_reader = ArrayReaderBuilder::new(&row_group).build_array_reader_with_cache( self.fields.as_deref(), &projection, - (&cache_projection, row_group_cache.clone()), + (&cache_projection, row_group_cache.clone(), crate::arrow::array_reader::CacheRole::Consumer), )?; let reader = ParquetRecordBatchReader::new(array_reader, plan); From e6281bc7ef92e730947cac396865fedcf51a5f57 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 10:16:28 -0500 Subject: [PATCH 07/38] update --- .../arrow/array_reader/cached_array_reader.rs | 72 ++++++++++++++++--- 1 file changed, 61 insertions(+), 11 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 2905b6fcb622..3bad82475f85 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -5,7 +5,7 @@ use arrow_array::{new_empty_array, ArrayRef, BooleanArray}; use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; use arrow_schema::DataType as ArrowType; use std::any::Any; -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::sync::{Arc, Mutex}; /// Role of the cached array reader @@ -59,6 +59,9 @@ pub struct CachedArrayReader { selections: VecDeque, /// Role of this reader (Producer or Consumer) role: CacheRole, + /// Local cache to store batches between read_records and consume_batch calls + /// This ensures data is available even if the shared cache evicts items + local_cache: HashMap, } impl CachedArrayReader { @@ -80,6 +83,7 @@ impl CachedArrayReader { batch_size, selections: VecDeque::new(), role, + local_cache: HashMap::new(), } } @@ -98,12 +102,15 @@ impl CachedArrayReader { let read = self.inner.read_records(self.batch_size)?; let array = self.inner.consume_batch()?; - // Both producers and consumers insert data into the cache when fetching from underlying reader - // The difference is that consumers will remove consumed batches later + // Store in both shared cache and local cache + // The shared cache is for coordination between readers + // The local cache ensures data is available for our consume_batch call self.cache .lock() .unwrap() - .insert(self.column_idx, batch_id, array); + .insert(self.column_idx, batch_id, array.clone()); + + self.local_cache.insert(batch_id, array); self.inner_position += read; Ok(read) @@ -139,7 +146,19 @@ impl ArrayReader for CachedArrayReader { let mut read = 0; while read < num_records { let batch_id = self.get_batch_id_from_position(self.outer_position + read); - let cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); + + // Check local cache first + let cached = if let Some(array) = self.local_cache.get(&batch_id) { + Some(array.clone()) + } else { + // If not in local cache, check shared cache + let shared_cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); + if let Some(array) = shared_cached.as_ref() { + // Store in local cache for later use in consume_batch + self.local_cache.insert(batch_id, array.clone()); + } + shared_cached + }; match cached { Some(array) => { @@ -220,21 +239,21 @@ impl ArrayReader for CachedArrayReader { } let mask_array = BooleanArray::from(mask); + // Read from local cache instead of shared cache to avoid cache eviction issues let cached = self - .cache - .lock() - .unwrap() - .get(self.column_idx, batch_id) - .expect("data must be already cached in the read_records call"); + .local_cache + .get(&batch_id) + .expect("data must be already cached in the read_records call, this is a bug"); let cached = cached.slice(overlap_start - batch_start, selection_length); let filtered = arrow_select::filter::filter(&cached, &mask_array)?; selected_arrays.push(filtered); } self.selections.clear(); + self.local_cache.clear(); // For consumers, cleanup batches that have been completely consumed - // This reduces the memory usage of the cache + // This reduces the memory usage of the shared cache if self.role == CacheRole::Consumer { self.cleanup_consumed_batches(); } @@ -540,4 +559,35 @@ mod tests { assert!(cache.lock().unwrap().get(0, 0).is_some()); assert!(cache.lock().unwrap().get(0, 1).is_some()); } + + #[test] + fn test_local_cache_protects_against_eviction() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let mut cached_reader = + CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + + // Read records which should populate both shared and local cache + let records_read = cached_reader.read_records(3).unwrap(); + assert_eq!(records_read, 3); + + // Verify data is in both caches + assert!(cache.lock().unwrap().get(0, 0).is_some()); + assert!(cached_reader.local_cache.get(&0).is_some()); + + // Simulate cache eviction by manually removing from shared cache + cache.lock().unwrap().remove(0, 0); + assert!(cache.lock().unwrap().get(0, 0).is_none()); + + // Even though shared cache was evicted, consume_batch should still work + // because data is preserved in local cache + let array = cached_reader.consume_batch().unwrap(); + assert_eq!(array.len(), 3); + + let int32_array = array.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[1, 2, 3]); + + // Local cache should be cleared after consume_batch + assert!(cached_reader.local_cache.is_empty()); + } } From 6b6d4fceb43c9f90bf564cd28865f9461b99f9c0 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 10:39:14 -0500 Subject: [PATCH 08/38] update --- .../arrow/array_reader/cached_array_reader.rs | 5 +- .../src/arrow/array_reader/row_group_cache.rs | 220 +++++++++++++++++- 2 files changed, 214 insertions(+), 11 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 3bad82475f85..4688271da38a 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -105,10 +105,13 @@ impl CachedArrayReader { // Store in both shared cache and local cache // The shared cache is for coordination between readers // The local cache ensures data is available for our consume_batch call - self.cache + let _cached = self + .cache .lock() .unwrap() .insert(self.column_idx, batch_id, array.clone()); + // Note: if the shared cache is full (_cached == false), we continue without caching + // The local cache will still store the data for this reader's use self.local_cache.insert(batch_id, array); diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 150f9edd406d..5a65c8637ada 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -14,12 +14,22 @@ pub struct CacheKey { /// /// This cache is designed to avoid duplicate decoding when the same column /// appears in both filter predicates and output projection. -#[derive(Debug, Default)] +#[derive(Debug)] pub struct RowGroupCache { /// Cache storage mapping (column_idx, row_id) -> ArrayRef cache: HashMap, /// Cache granularity batch_size: usize, + /// Maximum cache size in bytes (None means unlimited) + max_cache_size: Option, + /// Current cache size in bytes + current_cache_size: usize, +} + +impl Default for RowGroupCache { + fn default() -> Self { + Self::new(1000) + } } impl RowGroupCache { @@ -28,13 +38,44 @@ impl RowGroupCache { Self { cache: HashMap::new(), batch_size, + max_cache_size: None, + current_cache_size: 0, + } + } + + /// Creates a new empty row group cache with a maximum cache size in bytes + pub fn new_with_max_size(batch_size: usize, max_cache_size: usize) -> Self { + Self { + cache: HashMap::new(), + batch_size, + max_cache_size: Some(max_cache_size), + current_cache_size: 0, } } + /// Returns the memory size of an ArrayRef in bytes + fn array_memory_size(array: &ArrayRef) -> usize { + array.get_array_memory_size() + } + /// Inserts an array into the cache for the given column and starting row ID - pub fn insert(&mut self, column_idx: usize, row_id: usize, array: ArrayRef) { + /// Returns true if the array was inserted, false if it would exceed the cache size limit + pub fn insert(&mut self, column_idx: usize, row_id: usize, array: ArrayRef) -> bool { + let array_size = Self::array_memory_size(&array); + + // Check if adding this array would exceed the cache size limit + if let Some(max_size) = self.max_cache_size { + if self.current_cache_size + array_size > max_size { + return false; // Cache is full, don't insert + } + } + let key = CacheKey { column_idx, row_id }; - self.cache.insert(key, array); + + let existing = self.cache.insert(key, array); + assert!(existing.is_none()); + self.current_cache_size += array_size; + true } /// Retrieves a cached array for the given column and row ID @@ -49,11 +90,40 @@ impl RowGroupCache { self.batch_size } + /// Gets the maximum cache size in bytes (None means unlimited) + pub fn max_cache_size(&self) -> Option { + self.max_cache_size + } + + /// Gets the current cache size in bytes + pub fn current_cache_size(&self) -> usize { + self.current_cache_size + } + + /// Returns true if the cache has reached its maximum size + pub fn is_full(&self) -> bool { + match self.max_cache_size { + Some(max_size) => self.current_cache_size >= max_size, + None => false, + } + } + /// Removes a cached array for the given column and row ID /// Returns true if the entry was found and removed, false otherwise pub fn remove(&mut self, column_idx: usize, row_id: usize) -> bool { let key = CacheKey { column_idx, row_id }; - self.cache.remove(&key).is_some() + if let Some(array) = self.cache.remove(&key) { + self.current_cache_size -= Self::array_memory_size(&array); + true + } else { + false + } + } + + /// Clears all entries from the cache + pub fn clear(&mut self) { + self.cache.clear(); + self.current_cache_size = 0; } } @@ -71,7 +141,7 @@ mod tests { let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); // Test insert and get - cache.insert(0, 0, array.clone()); + assert!(cache.insert(0, 0, array.clone())); let retrieved = cache.get(0, 0); assert!(retrieved.is_some()); assert_eq!(retrieved.unwrap().len(), 5); @@ -94,9 +164,9 @@ mod tests { let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); // Insert arrays - cache.insert(0, 0, array1.clone()); - cache.insert(0, 1000, array2.clone()); - cache.insert(1, 0, array1.clone()); + assert!(cache.insert(0, 0, array1.clone())); + assert!(cache.insert(0, 1000, array2.clone())); + assert!(cache.insert(1, 0, array1.clone())); // Verify they're there assert!(cache.get(0, 0).is_some()); @@ -107,7 +177,7 @@ mod tests { let removed = cache.remove(0, 0); assert!(removed); assert!(cache.get(0, 0).is_none()); - + // Other entries should still be there assert!(cache.get(0, 1000).is_some()); assert!(cache.get(1, 0).is_some()); @@ -119,9 +189,139 @@ mod tests { // Remove remaining entries assert!(cache.remove(0, 1000)); assert!(cache.remove(1, 0)); - + // Cache should be empty assert!(cache.get(0, 1000).is_none()); assert!(cache.get(1, 0).is_none()); } + + #[test] + fn test_cache_with_max_size() { + // Create a cache with a very small max size + let mut cache = RowGroupCache::new_with_max_size(1000, 100); + + assert_eq!(cache.max_cache_size(), Some(100)); + assert_eq!(cache.current_cache_size(), 0); + assert!(!cache.is_full()); + + // Create a test array + let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); + let array_size = RowGroupCache::array_memory_size(&array); + + // If array is larger than max cache size, insertion should fail + if array_size > 100 { + assert!(!cache.insert(0, 0, array.clone())); + assert_eq!(cache.current_cache_size(), 0); + assert!(cache.get(0, 0).is_none()); + } else { + // If array fits, insertion should succeed + assert!(cache.insert(0, 0, array.clone())); + assert_eq!(cache.current_cache_size(), array_size); + assert!(cache.get(0, 0).is_some()); + + // Try to insert another array that would exceed the limit + let array2: ArrayRef = Arc::new(Int32Array::from(vec![6, 7, 8, 9, 10])); + let array2_size = RowGroupCache::array_memory_size(&array2); + + if array_size + array2_size > 100 { + assert!(!cache.insert(0, 1000, array2.clone())); + assert_eq!(cache.current_cache_size(), array_size); + assert!(cache.get(0, 1000).is_none()); + } + } + } + + #[test] + fn test_cache_unlimited_size() { + let mut cache = RowGroupCache::new(1000); + + assert_eq!(cache.max_cache_size(), None); + assert_eq!(cache.current_cache_size(), 0); + assert!(!cache.is_full()); + + // Should be able to insert multiple arrays without size limit + for i in 0..10 { + let array: ArrayRef = Arc::new(Int32Array::from(vec![i, i + 1, i + 2])); + assert!(cache.insert(0, (i * 1000) as usize, array)); + } + + assert_eq!(cache.cache.len(), 10); + assert!(!cache.is_full()); + } + + #[test] + fn test_cache_size_tracking() { + let mut cache = RowGroupCache::new(1000); + + let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); + let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6, 7])); + + let size1 = RowGroupCache::array_memory_size(&array1); + let size2 = RowGroupCache::array_memory_size(&array2); + + // Insert first array + assert!(cache.insert(0, 0, array1.clone())); + assert_eq!(cache.current_cache_size(), size1); + + // Insert second array + assert!(cache.insert(1, 0, array2.clone())); + assert_eq!(cache.current_cache_size(), size1 + size2); + + // Replace first array with second array + assert!(cache.insert(0, 0, array2.clone())); + assert_eq!(cache.current_cache_size(), size2 + size2); + + // Remove one entry + assert!(cache.remove(0, 0)); + assert_eq!(cache.current_cache_size(), size2); + + // Remove remaining entry + assert!(cache.remove(1, 0)); + assert_eq!(cache.current_cache_size(), 0); + } + + #[test] + fn test_cache_clear() { + let mut cache = RowGroupCache::new_with_max_size(1000, 1000); + + // Insert some arrays + for i in 0..5 { + let array: ArrayRef = Arc::new(Int32Array::from(vec![i, i + 1, i + 2])); + assert!(cache.insert(0, (i * 1000) as usize, array)); + } + + assert!(cache.current_cache_size() > 0); + assert_eq!(cache.cache.len(), 5); + + // Clear the cache + cache.clear(); + + assert_eq!(cache.current_cache_size(), 0); + assert_eq!(cache.cache.len(), 0); + assert!(!cache.is_full()); + } + + #[test] + fn test_cache_full_detection() { + // Create a cache that can hold approximately one small array + let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); + let array_size = RowGroupCache::array_memory_size(&array); + + let mut cache = RowGroupCache::new_with_max_size(1000, array_size); + + assert!(!cache.is_full()); + + // Insert array - should succeed and make cache full + assert!(cache.insert(0, 0, array.clone())); + assert!(cache.is_full()); + + // Try to insert another array - should fail + let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); + assert!(!cache.insert(1, 0, array2)); + + // Cache should still be full with original array + assert!(cache.is_full()); + assert!(cache.get(0, 0).is_some()); + assert!(cache.get(1, 0).is_none()); + } } From b696b66d53f432fc6cb5b491dec7a862f58f879f Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 10:54:18 -0500 Subject: [PATCH 09/38] update --- parquet/src/arrow/array_reader/builder.rs | 54 +++++++++++-------- .../arrow/array_reader/cached_array_reader.rs | 16 +++--- parquet/src/arrow/array_reader/mod.rs | 2 +- .../src/arrow/array_reader/row_group_cache.rs | 49 ++++++----------- parquet/src/arrow/async_reader/mod.rs | 16 ++++-- 5 files changed, 69 insertions(+), 68 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 91fd25944097..883e46b584e5 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -20,8 +20,8 @@ use std::sync::{Arc, Mutex}; use arrow_schema::{DataType, Fields, SchemaBuilder}; use crate::arrow::array_reader::byte_view_array::make_byte_view_array_reader; -use crate::arrow::array_reader::cached_array_reader::CachedArrayReader; use crate::arrow::array_reader::cached_array_reader::CacheRole; +use crate::arrow::array_reader::cached_array_reader::CachedArrayReader; use crate::arrow::array_reader::empty_array::make_empty_array_reader; use crate::arrow::array_reader::fixed_len_byte_array::make_fixed_len_byte_array_reader; use crate::arrow::array_reader::row_group_cache::RowGroupCache; @@ -42,8 +42,13 @@ pub struct ArrayReaderBuilder<'a> { row_groups: &'a dyn RowGroups, } -/// Cache context type containing projection mask, cache, and role -type CacheContext<'a> = (&'a ProjectionMask, Arc>, CacheRole); +/// Cache options containing projection mask, cache, and role +#[derive(Clone)] +pub struct CacheOptions<'a> { + pub projection_mask: &'a ProjectionMask, + pub cache: Arc>, + pub role: CacheRole, +} impl<'a> ArrayReaderBuilder<'a> { pub fn new(row_groups: &'a dyn RowGroups) -> Self { @@ -69,11 +74,11 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: Option<&ParquetField>, mask: &ProjectionMask, - cache_context: CacheContext, + cache_options: CacheOptions, ) -> Result> { let reader = field .and_then(|field| { - self.build_reader(field, mask, Some(cache_context)) + self.build_reader(field, mask, Some(cache_options)) .transpose() }) .transpose()? @@ -91,32 +96,35 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_context: Option, + cache_options: Option, ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { col_idx, .. } => { let Some(reader) = self.build_primitive_reader(field, mask)? else { return Ok(None); }; - let Some((cache_mask, cache, role)) = cache_context else { + let Some(cache_options) = cache_options else { return Ok(Some(reader)); }; - if cache_mask.leaf_included(col_idx) { + if cache_options.projection_mask.leaf_included(col_idx) { Ok(Some(Box::new(CachedArrayReader::new( - reader, cache, col_idx, role, + reader, + cache_options.cache, + col_idx, + cache_options.role, )))) } else { Ok(Some(reader)) } } ParquetFieldType::Group { .. } => match &field.arrow_type { - DataType::Map(_, _) => self.build_map_reader(field, mask, cache_context), - DataType::Struct(_) => self.build_struct_reader(field, mask, cache_context), - DataType::List(_) => self.build_list_reader(field, mask, cache_context, false), - DataType::LargeList(_) => self.build_list_reader(field, mask, cache_context, true), + DataType::Map(_, _) => self.build_map_reader(field, mask, cache_options), + DataType::Struct(_) => self.build_struct_reader(field, mask, cache_options), + DataType::List(_) => self.build_list_reader(field, mask, cache_options, false), + DataType::LargeList(_) => self.build_list_reader(field, mask, cache_options, true), DataType::FixedSizeList(_, _) => { - self.build_fixed_size_list_reader(field, mask, cache_context) + self.build_fixed_size_list_reader(field, mask, cache_options) } d => unimplemented!("reading group type {} not implemented", d), }, @@ -128,13 +136,13 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_context: Option, + cache_options: Option, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 2); - let key_reader = self.build_reader(&children[0], mask, cache_context.clone())?; - let value_reader = self.build_reader(&children[1], mask, cache_context)?; + let key_reader = self.build_reader(&children[0], mask, cache_options.clone())?; + let value_reader = self.build_reader(&children[1], mask, cache_options)?; match (key_reader, value_reader) { (Some(key_reader), Some(value_reader)) => { @@ -180,13 +188,13 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_context: Option, + cache_options: Option, is_large: bool, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, cache_context)? { + let reader = match self.build_reader(&children[0], mask, cache_options)? { Some(item_reader) => { // Need to retrieve underlying data type to handle projection let item_type = item_reader.get_data_type().clone(); @@ -228,12 +236,12 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_context: Option, + cache_options: Option, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, cache_context)? { + let reader = match self.build_reader(&children[0], mask, cache_options)? { Some(item_reader) => { let item_type = item_reader.get_data_type().clone(); let reader = match &field.arrow_type { @@ -363,7 +371,7 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_context: Option, + cache_options: Option, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -376,7 +384,7 @@ impl<'a> ArrayReaderBuilder<'a> { let mut builder = SchemaBuilder::with_capacity(children.len()); for (arrow, parquet) in arrow_fields.iter().zip(children) { - if let Some(reader) = self.build_reader(parquet, mask, cache_context.clone())? { + if let Some(reader) = self.build_reader(parquet, mask, cache_options.clone())? { // Need to retrieve underlying data type to handle projection let child_type = reader.get_data_type().clone(); builder.push(arrow.as_ref().clone().with_data_type(child_type)); diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 4688271da38a..e964f41c4b87 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -364,7 +364,7 @@ mod tests { #[test] fn test_cached_reader_basic() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Producer); @@ -386,7 +386,7 @@ mod tests { #[test] fn test_read_skip_pattern() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let cache = Arc::new(Mutex::new(RowGroupCache::new(5, None))); // Batch size 5 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); @@ -414,7 +414,7 @@ mod tests { #[test] fn test_multiple_reads_before_consume() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); @@ -435,7 +435,7 @@ mod tests { #[test] fn test_eof_behavior() { let mock_reader = MockArrayReader::new(vec![1, 2, 3]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let cache = Arc::new(Mutex::new(RowGroupCache::new(5, None))); // Batch size 5 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); @@ -456,7 +456,7 @@ mod tests { #[test] fn test_cache_sharing() { - let cache = Arc::new(Mutex::new(RowGroupCache::new(5))); // Batch size 5 + let cache = Arc::new(Mutex::new(RowGroupCache::new(5, None))); // Batch size 5 // First reader - populate cache let mock_reader1 = MockArrayReader::new(vec![1, 2, 3, 4, 5]); @@ -492,7 +492,7 @@ mod tests { #[test] fn test_consumer_removes_batches() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 let mut consumer_reader = CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); @@ -539,7 +539,7 @@ mod tests { #[test] fn test_producer_keeps_batches() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 let mut producer_reader = CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Producer); @@ -566,7 +566,7 @@ mod tests { #[test] fn test_local_cache_protects_against_eviction() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3))); // Batch size 3 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 let mut cached_reader = CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 8e1b9d7d37f8..85301ef87697 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -48,7 +48,7 @@ mod struct_array; mod test_util; // Note that this crate is public under the `experimental` feature flag. -pub use builder::ArrayReaderBuilder; +pub use builder::{ArrayReaderBuilder, CacheOptions}; pub use byte_array::make_byte_array_reader; pub use byte_array_dictionary::make_byte_array_dictionary_reader; #[allow(unused_imports)] // Only used for benchmarks diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 5a65c8637ada..913cffa0412e 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -28,40 +28,25 @@ pub struct RowGroupCache { impl Default for RowGroupCache { fn default() -> Self { - Self::new(1000) + Self::new(1000, None) } } impl RowGroupCache { /// Creates a new empty row group cache - pub fn new(batch_size: usize) -> Self { + pub fn new(batch_size: usize, max_cache_size: Option) -> Self { Self { cache: HashMap::new(), batch_size, - max_cache_size: None, + max_cache_size, current_cache_size: 0, } } - /// Creates a new empty row group cache with a maximum cache size in bytes - pub fn new_with_max_size(batch_size: usize, max_cache_size: usize) -> Self { - Self { - cache: HashMap::new(), - batch_size, - max_cache_size: Some(max_cache_size), - current_cache_size: 0, - } - } - - /// Returns the memory size of an ArrayRef in bytes - fn array_memory_size(array: &ArrayRef) -> usize { - array.get_array_memory_size() - } - /// Inserts an array into the cache for the given column and starting row ID /// Returns true if the array was inserted, false if it would exceed the cache size limit pub fn insert(&mut self, column_idx: usize, row_id: usize, array: ArrayRef) -> bool { - let array_size = Self::array_memory_size(&array); + let array_size = array.get_array_memory_size(); // Check if adding this array would exceed the cache size limit if let Some(max_size) = self.max_cache_size { @@ -113,7 +98,7 @@ impl RowGroupCache { pub fn remove(&mut self, column_idx: usize, row_id: usize) -> bool { let key = CacheKey { column_idx, row_id }; if let Some(array) = self.cache.remove(&key) { - self.current_cache_size -= Self::array_memory_size(&array); + self.current_cache_size -= array.get_array_memory_size(); true } else { false @@ -135,7 +120,7 @@ mod tests { #[test] fn test_cache_basic_operations() { - let mut cache = RowGroupCache::new(1000); + let mut cache = RowGroupCache::new(1000, None); // Create test array let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); @@ -157,7 +142,7 @@ mod tests { #[test] fn test_cache_remove() { - let mut cache = RowGroupCache::new(1000); + let mut cache = RowGroupCache::new(1000, None); // Create test arrays let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); @@ -198,7 +183,7 @@ mod tests { #[test] fn test_cache_with_max_size() { // Create a cache with a very small max size - let mut cache = RowGroupCache::new_with_max_size(1000, 100); + let mut cache = RowGroupCache::new(1000, Some(100)); assert_eq!(cache.max_cache_size(), Some(100)); assert_eq!(cache.current_cache_size(), 0); @@ -206,7 +191,7 @@ mod tests { // Create a test array let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); - let array_size = RowGroupCache::array_memory_size(&array); + let array_size = array.get_array_memory_size(); // If array is larger than max cache size, insertion should fail if array_size > 100 { @@ -221,7 +206,7 @@ mod tests { // Try to insert another array that would exceed the limit let array2: ArrayRef = Arc::new(Int32Array::from(vec![6, 7, 8, 9, 10])); - let array2_size = RowGroupCache::array_memory_size(&array2); + let array2_size = array2.get_array_memory_size(); if array_size + array2_size > 100 { assert!(!cache.insert(0, 1000, array2.clone())); @@ -233,7 +218,7 @@ mod tests { #[test] fn test_cache_unlimited_size() { - let mut cache = RowGroupCache::new(1000); + let mut cache = RowGroupCache::new(1000, None); assert_eq!(cache.max_cache_size(), None); assert_eq!(cache.current_cache_size(), 0); @@ -251,13 +236,13 @@ mod tests { #[test] fn test_cache_size_tracking() { - let mut cache = RowGroupCache::new(1000); + let mut cache = RowGroupCache::new(1000, None); let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6, 7])); - let size1 = RowGroupCache::array_memory_size(&array1); - let size2 = RowGroupCache::array_memory_size(&array2); + let size1 = array1.get_array_memory_size(); + let size2 = array2.get_array_memory_size(); // Insert first array assert!(cache.insert(0, 0, array1.clone())); @@ -282,7 +267,7 @@ mod tests { #[test] fn test_cache_clear() { - let mut cache = RowGroupCache::new_with_max_size(1000, 1000); + let mut cache = RowGroupCache::new(1000, Some(1000)); // Insert some arrays for i in 0..5 { @@ -305,9 +290,9 @@ mod tests { fn test_cache_full_detection() { // Create a cache that can hold approximately one small array let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); - let array_size = RowGroupCache::array_memory_size(&array); + let array_size = array.get_array_memory_size(); - let mut cache = RowGroupCache::new_with_max_size(1000, array_size); + let mut cache = RowGroupCache::new(1000, Some(array_size)); assert!(!cache.is_full()); diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index aeba334ec431..e289e051d736 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -38,7 +38,7 @@ use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt}; use arrow_array::RecordBatch; use arrow_schema::{DataType, Fields, Schema, SchemaRef}; -use crate::arrow::array_reader::{ArrayReaderBuilder, RowGroupCache, RowGroups}; +use crate::arrow::array_reader::{ArrayReaderBuilder, CacheOptions, RowGroupCache, RowGroups}; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection, @@ -592,7 +592,7 @@ where Some(projection) => projection, None => ProjectionMask::none(meta.columns().len()), }; - let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new(batch_size))); + let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new(batch_size, None))); let mut row_group = InMemoryRowGroup { // schema: meta.schema_descr_ptr(), @@ -625,7 +625,11 @@ where .build_array_reader_with_cache( self.fields.as_deref(), predicate.projection(), - (&cache_projection, row_group_cache.clone(), crate::arrow::array_reader::CacheRole::Producer), + CacheOptions { + projection_mask: &cache_projection, + cache: row_group_cache.clone(), + role: crate::arrow::array_reader::CacheRole::Producer, + }, )?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; @@ -676,7 +680,11 @@ where let array_reader = ArrayReaderBuilder::new(&row_group).build_array_reader_with_cache( self.fields.as_deref(), &projection, - (&cache_projection, row_group_cache.clone(), crate::arrow::array_reader::CacheRole::Consumer), + CacheOptions { + projection_mask: &cache_projection, + cache: row_group_cache.clone(), + role: crate::arrow::array_reader::CacheRole::Consumer, + }, )?; let reader = ParquetRecordBatchReader::new(array_reader, plan); From f60581fffd600dda7cd4641671d6e7a2cba09ea0 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 14:44:16 -0500 Subject: [PATCH 10/38] update --- .../arrow/array_reader/cached_array_reader.rs | 93 ++++--- .../src/arrow/array_reader/row_group_cache.rs | 227 ++++-------------- parquet/src/arrow/async_reader/mod.rs | 8 +- 3 files changed, 106 insertions(+), 222 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index e964f41c4b87..d34817a53cd2 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -1,3 +1,4 @@ +use crate::arrow::array_reader::row_group_cache::BatchID; use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; use crate::arrow::arrow_reader::RowSelector; use crate::errors::Result; @@ -59,9 +60,9 @@ pub struct CachedArrayReader { selections: VecDeque, /// Role of this reader (Producer or Consumer) role: CacheRole, - /// Local cache to store batches between read_records and consume_batch calls + /// Local buffer to store batches between read_records and consume_batch calls /// This ensures data is available even if the shared cache evicts items - local_cache: HashMap, + local_buffer: HashMap, } impl CachedArrayReader { @@ -83,23 +84,34 @@ impl CachedArrayReader { batch_size, selections: VecDeque::new(), role, - local_cache: HashMap::new(), + local_buffer: HashMap::new(), } } - fn get_batch_id_from_position(&self, position: usize) -> usize { - position / self.batch_size + fn get_batch_id_from_position(&self, row_id: usize) -> BatchID { + BatchID { + val: row_id / self.batch_size, + } } - fn fetch_batch(&mut self, batch_id: usize) -> Result { - let row_id = batch_id * self.batch_size; + fn fetch_batch(&mut self, batch_id: BatchID) -> Result { + let row_id = batch_id.val * self.batch_size; if self.inner_position < row_id { let to_skip = row_id - self.inner_position; let skipped = self.inner.skip_records(to_skip)?; + assert_eq!(skipped, to_skip); self.inner_position += skipped; } let read = self.inner.read_records(self.batch_size)?; + + // If there are no remaining records (EOF), return immediately without + // attempting to cache an empty batch. This prevents inserting zero-length + // arrays into the cache which can later cause panics when slicing. + if read == 0 { + return Ok(0); + } + let array = self.inner.consume_batch()?; // Store in both shared cache and local cache @@ -113,7 +125,7 @@ impl CachedArrayReader { // Note: if the shared cache is full (_cached == false), we continue without caching // The local cache will still store the data for this reader's use - self.local_cache.insert(batch_id, array); + self.local_buffer.insert(batch_id, array); self.inner_position += read; Ok(read) @@ -127,10 +139,15 @@ impl CachedArrayReader { // Remove batches that are at least one batch behind the current position // This ensures we don't remove batches that might still be needed for the current batch // We can safely remove batch_id if current_batch_id > batch_id + 1 - if current_batch_id > 1 { + if current_batch_id.val > 1 { let mut cache = self.cache.lock().unwrap(); - for batch_id_to_remove in 0..(current_batch_id - 1) { - cache.remove(self.column_idx, batch_id_to_remove); + for batch_id_to_remove in 0..(current_batch_id.val - 1) { + cache.remove( + self.column_idx, + BatchID { + val: batch_id_to_remove, + }, + ); } } } @@ -151,14 +168,14 @@ impl ArrayReader for CachedArrayReader { let batch_id = self.get_batch_id_from_position(self.outer_position + read); // Check local cache first - let cached = if let Some(array) = self.local_cache.get(&batch_id) { + let cached = if let Some(array) = self.local_buffer.get(&batch_id) { Some(array.clone()) } else { // If not in local cache, check shared cache let shared_cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); if let Some(array) = shared_cached.as_ref() { // Store in local cache for later use in consume_batch - self.local_cache.insert(batch_id, array.clone()); + self.local_buffer.insert(batch_id, array.clone()); } shared_cached }; @@ -166,9 +183,9 @@ impl ArrayReader for CachedArrayReader { match cached { Some(array) => { let array_len = array.len(); - if array_len + batch_id * self.batch_size - self.outer_position > 0 { + if array_len + batch_id.val * self.batch_size - self.outer_position > 0 { // the cache batch has some records that we can select - let v = array_len + batch_id * self.batch_size - self.outer_position; + let v = array_len + batch_id.val * self.batch_size - self.outer_position; let select_cnt = std::cmp::min(num_records - read, v); read += select_cnt; self.selections.push_back(RowSelector::select(select_cnt)); @@ -180,6 +197,11 @@ impl ArrayReader for CachedArrayReader { None => { let read_from_inner = self.fetch_batch(batch_id)?; + // Reached end-of-file, no more records to read + if read_from_inner == 0 { + break; + } + let select_from_this_batch = std::cmp::min(num_records - read, read_from_inner); read += select_from_this_batch; self.selections @@ -209,7 +231,7 @@ impl ArrayReader for CachedArrayReader { fn consume_batch(&mut self) -> Result { let row_count = self.selections.iter().map(|s| s.row_count).sum::(); if row_count == 0 { - return Ok(new_empty_array(&self.inner.get_data_type())); + return Ok(new_empty_array(self.inner.get_data_type())); } let start_position = self.outer_position - row_count; @@ -244,7 +266,7 @@ impl ArrayReader for CachedArrayReader { let mask_array = BooleanArray::from(mask); // Read from local cache instead of shared cache to avoid cache eviction issues let cached = self - .local_cache + .local_buffer .get(&batch_id) .expect("data must be already cached in the read_records call, this is a bug"); let cached = cached.slice(overlap_start - batch_start, selection_length); @@ -253,7 +275,7 @@ impl ArrayReader for CachedArrayReader { } self.selections.clear(); - self.local_cache.clear(); + self.local_buffer.clear(); // For consumers, cleanup batches that have been completely consumed // This reduces the memory usage of the shared cache @@ -262,7 +284,7 @@ impl ArrayReader for CachedArrayReader { } match selected_arrays.len() { - 0 => Ok(new_empty_array(&self.inner.get_data_type())), + 0 => Ok(new_empty_array(self.inner.get_data_type())), 1 => Ok(selected_arrays.into_iter().next().unwrap()), _ => Ok(arrow_select::concat::concat( &selected_arrays @@ -501,14 +523,14 @@ mod tests { assert_eq!(read1, 3); assert_eq!(consumer_reader.outer_position, 3); // Check that batch 0 is in cache after read_records - assert!(cache.lock().unwrap().get(0, 0).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); let array1 = consumer_reader.consume_batch().unwrap(); assert_eq!(array1.len(), 3); // After first consume_batch, batch 0 should still be in cache // (current_batch_id = 3/3 = 1, cleanup only happens if current_batch_id > 1) - assert!(cache.lock().unwrap().get(0, 0).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); // Read second batch (positions 3-5, batch 1) let read2 = consumer_reader.read_records(3).unwrap(); @@ -519,8 +541,8 @@ mod tests { // After second consume_batch, batch 0 should be removed // (current_batch_id = 6/3 = 2, cleanup removes batches 0..(2-1) = 0..1, so removes batch 0) - assert!(cache.lock().unwrap().get(0, 0).is_none()); - assert!(cache.lock().unwrap().get(0, 1).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_none()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 1 }).is_some()); // Read third batch (positions 6-8, batch 2) let read3 = consumer_reader.read_records(3).unwrap(); @@ -531,9 +553,9 @@ mod tests { // After third consume_batch, batches 0 and 1 should be removed // (current_batch_id = 9/3 = 3, cleanup removes batches 0..(3-1) = 0..2, so removes batches 0 and 1) - assert!(cache.lock().unwrap().get(0, 0).is_none()); - assert!(cache.lock().unwrap().get(0, 1).is_none()); - assert!(cache.lock().unwrap().get(0, 2).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_none()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 1 }).is_none()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 2 }).is_some()); } #[test] @@ -550,7 +572,7 @@ mod tests { assert_eq!(array1.len(), 3); // Verify batch 0 is in cache - assert!(cache.lock().unwrap().get(0, 0).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); // Read second batch (positions 3-5) - producer should NOT remove batch 0 let read2 = producer_reader.read_records(3).unwrap(); @@ -559,8 +581,8 @@ mod tests { assert_eq!(array2.len(), 3); // Verify both batch 0 and batch 1 are still present (no removal for producer) - assert!(cache.lock().unwrap().get(0, 0).is_some()); - assert!(cache.lock().unwrap().get(0, 1).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 1 }).is_some()); } #[test] @@ -575,12 +597,15 @@ mod tests { assert_eq!(records_read, 3); // Verify data is in both caches - assert!(cache.lock().unwrap().get(0, 0).is_some()); - assert!(cached_reader.local_cache.get(&0).is_some()); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); + assert!(cached_reader + .local_buffer + .get(&BatchID { val: 0 }) + .is_some()); // Simulate cache eviction by manually removing from shared cache - cache.lock().unwrap().remove(0, 0); - assert!(cache.lock().unwrap().get(0, 0).is_none()); + cache.lock().unwrap().remove(0, BatchID { val: 0 }); + assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_none()); // Even though shared cache was evicted, consume_batch should still work // because data is preserved in local cache @@ -591,6 +616,6 @@ mod tests { assert_eq!(int32_array.values(), &[1, 2, 3]); // Local cache should be cleared after consume_batch - assert!(cached_reader.local_cache.is_empty()); + assert!(cached_reader.local_buffer.is_empty()); } } diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 913cffa0412e..4e335869503a 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -1,13 +1,18 @@ use arrow_array::ArrayRef; use std::collections::HashMap; +#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +pub struct BatchID { + pub val: usize, // batch id is row id / batch_size +} + /// Cache key that uniquely identifies a batch within a row group #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct CacheKey { /// Column index in the row group pub column_idx: usize, /// Starting row ID for this batch - pub row_id: usize, + pub batch_id: BatchID, } /// Row group cache that stores decoded arrow arrays at batch granularity @@ -26,12 +31,6 @@ pub struct RowGroupCache { current_cache_size: usize, } -impl Default for RowGroupCache { - fn default() -> Self { - Self::new(1000, None) - } -} - impl RowGroupCache { /// Creates a new empty row group cache pub fn new(batch_size: usize, max_cache_size: Option) -> Self { @@ -45,7 +44,7 @@ impl RowGroupCache { /// Inserts an array into the cache for the given column and starting row ID /// Returns true if the array was inserted, false if it would exceed the cache size limit - pub fn insert(&mut self, column_idx: usize, row_id: usize, array: ArrayRef) -> bool { + pub fn insert(&mut self, column_idx: usize, batch_id: BatchID, array: ArrayRef) -> bool { let array_size = array.get_array_memory_size(); // Check if adding this array would exceed the cache size limit @@ -55,7 +54,10 @@ impl RowGroupCache { } } - let key = CacheKey { column_idx, row_id }; + let key = CacheKey { + column_idx, + batch_id, + }; let existing = self.cache.insert(key, array); assert!(existing.is_none()); @@ -65,8 +67,11 @@ impl RowGroupCache { /// Retrieves a cached array for the given column and row ID /// Returns None if not found in cache - pub fn get(&self, column_idx: usize, row_id: usize) -> Option { - let key = CacheKey { column_idx, row_id }; + pub fn get(&self, column_idx: usize, batch_id: BatchID) -> Option { + let key = CacheKey { + column_idx, + batch_id, + }; self.cache.get(&key).cloned() } @@ -75,28 +80,13 @@ impl RowGroupCache { self.batch_size } - /// Gets the maximum cache size in bytes (None means unlimited) - pub fn max_cache_size(&self) -> Option { - self.max_cache_size - } - - /// Gets the current cache size in bytes - pub fn current_cache_size(&self) -> usize { - self.current_cache_size - } - - /// Returns true if the cache has reached its maximum size - pub fn is_full(&self) -> bool { - match self.max_cache_size { - Some(max_size) => self.current_cache_size >= max_size, - None => false, - } - } - /// Removes a cached array for the given column and row ID /// Returns true if the entry was found and removed, false otherwise - pub fn remove(&mut self, column_idx: usize, row_id: usize) -> bool { - let key = CacheKey { column_idx, row_id }; + pub fn remove(&mut self, column_idx: usize, batch_id: BatchID) -> bool { + let key = CacheKey { + column_idx, + batch_id, + }; if let Some(array) = self.cache.remove(&key) { self.current_cache_size -= array.get_array_memory_size(); true @@ -104,12 +94,6 @@ impl RowGroupCache { false } } - - /// Clears all entries from the cache - pub fn clear(&mut self) { - self.cache.clear(); - self.current_cache_size = 0; - } } #[cfg(test)] @@ -126,17 +110,18 @@ mod tests { let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); // Test insert and get - assert!(cache.insert(0, 0, array.clone())); - let retrieved = cache.get(0, 0); + let batch_id = BatchID { val: 0 }; + assert!(cache.insert(0, batch_id, array.clone())); + let retrieved = cache.get(0, batch_id); assert!(retrieved.is_some()); assert_eq!(retrieved.unwrap().len(), 5); // Test miss - let miss = cache.get(1, 0); + let miss = cache.get(1, batch_id); assert!(miss.is_none()); // Test different row_id - let miss = cache.get(0, 1000); + let miss = cache.get(0, BatchID { val: 1000 }); assert!(miss.is_none()); } @@ -149,164 +134,34 @@ mod tests { let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); // Insert arrays - assert!(cache.insert(0, 0, array1.clone())); - assert!(cache.insert(0, 1000, array2.clone())); - assert!(cache.insert(1, 0, array1.clone())); + assert!(cache.insert(0, BatchID { val: 0 }, array1.clone())); + assert!(cache.insert(0, BatchID { val: 1000 }, array2.clone())); + assert!(cache.insert(1, BatchID { val: 0 }, array1.clone())); // Verify they're there - assert!(cache.get(0, 0).is_some()); - assert!(cache.get(0, 1000).is_some()); - assert!(cache.get(1, 0).is_some()); + assert!(cache.get(0, BatchID { val: 0 }).is_some()); + assert!(cache.get(0, BatchID { val: 1000 }).is_some()); + assert!(cache.get(1, BatchID { val: 0 }).is_some()); // Remove one entry - let removed = cache.remove(0, 0); + let removed = cache.remove(0, BatchID { val: 0 }); assert!(removed); - assert!(cache.get(0, 0).is_none()); + assert!(cache.get(0, BatchID { val: 0 }).is_none()); // Other entries should still be there - assert!(cache.get(0, 1000).is_some()); - assert!(cache.get(1, 0).is_some()); + assert!(cache.get(0, BatchID { val: 1000 }).is_some()); + assert!(cache.get(1, BatchID { val: 0 }).is_some()); // Try to remove non-existent entry - let not_removed = cache.remove(0, 0); + let not_removed = cache.remove(0, BatchID { val: 0 }); assert!(!not_removed); // Remove remaining entries - assert!(cache.remove(0, 1000)); - assert!(cache.remove(1, 0)); + assert!(cache.remove(0, BatchID { val: 1000 })); + assert!(cache.remove(1, BatchID { val: 0 })); // Cache should be empty - assert!(cache.get(0, 1000).is_none()); - assert!(cache.get(1, 0).is_none()); - } - - #[test] - fn test_cache_with_max_size() { - // Create a cache with a very small max size - let mut cache = RowGroupCache::new(1000, Some(100)); - - assert_eq!(cache.max_cache_size(), Some(100)); - assert_eq!(cache.current_cache_size(), 0); - assert!(!cache.is_full()); - - // Create a test array - let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); - let array_size = array.get_array_memory_size(); - - // If array is larger than max cache size, insertion should fail - if array_size > 100 { - assert!(!cache.insert(0, 0, array.clone())); - assert_eq!(cache.current_cache_size(), 0); - assert!(cache.get(0, 0).is_none()); - } else { - // If array fits, insertion should succeed - assert!(cache.insert(0, 0, array.clone())); - assert_eq!(cache.current_cache_size(), array_size); - assert!(cache.get(0, 0).is_some()); - - // Try to insert another array that would exceed the limit - let array2: ArrayRef = Arc::new(Int32Array::from(vec![6, 7, 8, 9, 10])); - let array2_size = array2.get_array_memory_size(); - - if array_size + array2_size > 100 { - assert!(!cache.insert(0, 1000, array2.clone())); - assert_eq!(cache.current_cache_size(), array_size); - assert!(cache.get(0, 1000).is_none()); - } - } - } - - #[test] - fn test_cache_unlimited_size() { - let mut cache = RowGroupCache::new(1000, None); - - assert_eq!(cache.max_cache_size(), None); - assert_eq!(cache.current_cache_size(), 0); - assert!(!cache.is_full()); - - // Should be able to insert multiple arrays without size limit - for i in 0..10 { - let array: ArrayRef = Arc::new(Int32Array::from(vec![i, i + 1, i + 2])); - assert!(cache.insert(0, (i * 1000) as usize, array)); - } - - assert_eq!(cache.cache.len(), 10); - assert!(!cache.is_full()); - } - - #[test] - fn test_cache_size_tracking() { - let mut cache = RowGroupCache::new(1000, None); - - let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); - let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6, 7])); - - let size1 = array1.get_array_memory_size(); - let size2 = array2.get_array_memory_size(); - - // Insert first array - assert!(cache.insert(0, 0, array1.clone())); - assert_eq!(cache.current_cache_size(), size1); - - // Insert second array - assert!(cache.insert(1, 0, array2.clone())); - assert_eq!(cache.current_cache_size(), size1 + size2); - - // Replace first array with second array - assert!(cache.insert(0, 0, array2.clone())); - assert_eq!(cache.current_cache_size(), size2 + size2); - - // Remove one entry - assert!(cache.remove(0, 0)); - assert_eq!(cache.current_cache_size(), size2); - - // Remove remaining entry - assert!(cache.remove(1, 0)); - assert_eq!(cache.current_cache_size(), 0); - } - - #[test] - fn test_cache_clear() { - let mut cache = RowGroupCache::new(1000, Some(1000)); - - // Insert some arrays - for i in 0..5 { - let array: ArrayRef = Arc::new(Int32Array::from(vec![i, i + 1, i + 2])); - assert!(cache.insert(0, (i * 1000) as usize, array)); - } - - assert!(cache.current_cache_size() > 0); - assert_eq!(cache.cache.len(), 5); - - // Clear the cache - cache.clear(); - - assert_eq!(cache.current_cache_size(), 0); - assert_eq!(cache.cache.len(), 0); - assert!(!cache.is_full()); - } - - #[test] - fn test_cache_full_detection() { - // Create a cache that can hold approximately one small array - let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); - let array_size = array.get_array_memory_size(); - - let mut cache = RowGroupCache::new(1000, Some(array_size)); - - assert!(!cache.is_full()); - - // Insert array - should succeed and make cache full - assert!(cache.insert(0, 0, array.clone())); - assert!(cache.is_full()); - - // Try to insert another array - should fail - let array2: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); - assert!(!cache.insert(1, 0, array2)); - - // Cache should still be full with original array - assert!(cache.is_full()); - assert!(cache.get(0, 0).is_some()); - assert!(cache.get(1, 0).is_none()); + assert!(cache.get(0, BatchID { val: 1000 }).is_none()); + assert!(cache.get(1, BatchID { val: 0 }).is_none()); } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index e289e051d736..f83bdf87cb27 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -592,7 +592,11 @@ where Some(projection) => projection, None => ProjectionMask::none(meta.columns().len()), }; - let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new(batch_size, None))); + let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new( + batch_size, + // None, + Some(1024 * 1024 * 100), + ))); let mut row_group = InMemoryRowGroup { // schema: meta.schema_descr_ptr(), @@ -696,7 +700,7 @@ where let filters = self.filter.as_ref()?; let mut cache_projection = filters.predicates.first()?.projection().clone(); for predicate in filters.predicates.iter() { - cache_projection.union(&predicate.projection()); + cache_projection.union(predicate.projection()); } cache_projection.intersect(projection); Some(cache_projection) From 1851f0b6345a58d904e4a18e3f9b0b09fbf99b8e Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Wed, 2 Jul 2025 15:35:02 -0500 Subject: [PATCH 11/38] clippy and license --- .../arrow/array_reader/cached_array_reader.rs | 22 +++++++++++++++---- .../src/arrow/array_reader/row_group_cache.rs | 17 ++++++++++++++ 2 files changed, 35 insertions(+), 4 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index d34817a53cd2..b57f7a86ba46 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -1,3 +1,20 @@ +// 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 crate::arrow::array_reader::row_group_cache::BatchID; use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; use crate::arrow::arrow_reader::RowSelector; @@ -598,10 +615,7 @@ mod tests { // Verify data is in both caches assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); - assert!(cached_reader - .local_buffer - .get(&BatchID { val: 0 }) - .is_some()); + assert!(cached_reader.local_buffer.contains_key(&BatchID { val: 0 })); // Simulate cache eviction by manually removing from shared cache cache.lock().unwrap().remove(0, BatchID { val: 0 }); diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 4e335869503a..a86b618c8dcb 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use arrow_array::ArrayRef; use std::collections::HashMap; From 58add510c99c4b47f686feb5439042a0b56fb4ab Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 8 Jul 2025 11:17:42 -0500 Subject: [PATCH 12/38] bug fix --- .../arrow/array_reader/cached_array_reader.rs | 6 +- parquet/src/arrow/arrow_reader/selection.rs | 102 ++++++++++++++++++ parquet/src/arrow/async_reader/mod.rs | 67 +++++++++++- 3 files changed, 171 insertions(+), 4 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index b57f7a86ba46..62851f4b6f1d 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -218,8 +218,10 @@ impl ArrayReader for CachedArrayReader { if read_from_inner == 0 { break; } - - let select_from_this_batch = std::cmp::min(num_records - read, read_from_inner); + let select_from_this_batch = std::cmp::min( + num_records - read, + self.inner_position - self.outer_position, + ); read += select_from_this_batch; self.selections .push_back(RowSelector::select(select_from_this_batch)); diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index c53d47be2e56..90d8763dfe73 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -441,6 +441,59 @@ impl RowSelection { pub fn skipped_row_count(&self) -> usize { self.iter().filter(|s| s.skip).map(|s| s.row_count).sum() } + + /// Expands the selection to align with batch boundaries. + /// This is needed when using cached array readers to ensure that + /// the cached data covers full batches. + #[cfg(feature = "async")] + pub(crate) fn expand_to_batch_boundaries(&self, batch_size: usize, total_rows: usize) -> Self { + if batch_size == 0 { + return self.clone(); + } + + let mut expanded_ranges = Vec::new(); + let mut row_offset = 0; + + for selector in &self.selectors { + if selector.skip { + row_offset += selector.row_count; + } else { + let start = row_offset; + let end = row_offset + selector.row_count; + + // Expand start to batch boundary + let expanded_start = (start / batch_size) * batch_size; + // Expand end to batch boundary + let expanded_end = ((end + batch_size - 1) / batch_size) * batch_size; + let expanded_end = expanded_end.min(total_rows); + + expanded_ranges.push(expanded_start..expanded_end); + row_offset += selector.row_count; + } + } + + // Sort ranges by start position + expanded_ranges.sort_by_key(|range| range.start); + + // Merge overlapping or consecutive ranges + let mut merged_ranges: Vec> = Vec::new(); + for range in expanded_ranges { + if let Some(last) = merged_ranges.last_mut() { + if range.start <= last.end { + // Overlapping or consecutive - merge them + last.end = last.end.max(range.end); + } else { + // No overlap - add new range + merged_ranges.push(range); + } + } else { + // First range + merged_ranges.push(range); + } + } + + Self::from_consecutive_ranges(merged_ranges.into_iter(), total_rows) + } } impl From> for RowSelection { @@ -1378,4 +1431,53 @@ mod tests { assert_eq!(selection.row_count(), 0); assert_eq!(selection.skipped_row_count(), 0); } + + #[test] + #[cfg(feature = "async")] + fn test_expand_to_batch_boundaries() { + // Test case that reproduces the overlapping ranges bug + let selection = RowSelection::from(vec![ + RowSelector::skip(21), // Skip first page + RowSelector::select(21), // Select page to boundary + RowSelector::skip(41), // Skip multiple pages + RowSelector::select(41), // Select multiple pages + RowSelector::skip(25), // Skip page across boundary + RowSelector::select(25), // Select across page boundary + RowSelector::skip(7116), // Skip to final page boundary + RowSelector::select(10), // Select final page + ]); + + let total_rows = 7300; + let batch_size = 1024; + + // This should not panic with "out of order" + let expanded = selection.expand_to_batch_boundaries(batch_size, total_rows); + + // Verify that the expanded selection is valid + assert!(expanded.selects_any()); + assert!(expanded.row_count() >= selection.row_count()); + + // Test with smaller batch size that would cause more overlaps + let batch_size = 32; + let expanded = selection.expand_to_batch_boundaries(batch_size, total_rows); + assert!(expanded.selects_any()); + + // Test edge case with batch_size = 0 + let expanded = selection.expand_to_batch_boundaries(0, total_rows); + assert_eq!(expanded, selection); + + // Test simple case with two adjacent selectors + let selection = RowSelection::from(vec![ + RowSelector::select(10), // 0-10 + RowSelector::skip(5), // 10-15 + RowSelector::select(10), // 15-25 + ]); + + let expanded = selection.expand_to_batch_boundaries(32, 100); + // Both selectors should expand to 0-32 + assert_eq!( + expanded.selectors, + vec![RowSelector::select(32), RowSelector::skip(68)] + ); + } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index f83bdf87cb27..56ce7ad796ca 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -620,7 +620,12 @@ where // (pre) Fetch only the columns that are selected by the predicate let selection = plan_builder.selection(); row_group - .fetch(&mut self.input, predicate.projection(), selection) + .fetch( + &mut self.input, + predicate.projection(), + selection, + batch_size, + ) .await?; let mut cache_projection = predicate.projection().clone(); @@ -676,7 +681,12 @@ where } // fetch the pages needed for decoding row_group - .fetch(&mut self.input, &projection, plan_builder.selection()) + .fetch( + &mut self.input, + &projection, + plan_builder.selection(), + batch_size, + ) .await?; let plan = plan_builder.build(); @@ -696,6 +706,7 @@ where Ok((self, Some(reader))) } + /// Compute which columns are used in filters and the final (output) projection fn compute_cache_projection(&self, projection: &ProjectionMask) -> Option { let filters = self.filter.as_ref()?; let mut cache_projection = filters.predicates.first()?.projection().clone(); @@ -934,9 +945,11 @@ impl InMemoryRowGroup<'_> { input: &mut T, projection: &ProjectionMask, selection: Option<&RowSelection>, + batch_size: usize, ) -> Result<()> { let metadata = self.metadata.row_group(self.row_group_idx); if let Some((selection, offset_index)) = selection.zip(self.offset_index) { + let selection = selection.expand_to_batch_boundaries(batch_size, self.row_count); // If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the // `RowSelection` let mut page_start_offsets: Vec> = vec![]; @@ -1869,6 +1882,7 @@ mod tests { assert_eq!(total_rows, 730); } + #[ignore] #[tokio::test] async fn test_in_memory_row_group_sparse() { let testdata = arrow::util::test_util::parquet_test_data(); @@ -2423,4 +2437,53 @@ mod tests { let result = reader.try_collect::>().await.unwrap(); assert_eq!(result.len(), 1); } + + #[tokio::test] + async fn test_cached_array_reader_sparse_offset_error() { + use futures::TryStreamExt; + + use crate::arrow::arrow_reader::{ArrowPredicateFn, RowFilter, RowSelection, RowSelector}; + use arrow_array::{BooleanArray, RecordBatch}; + + let testdata = arrow::util::test_util::parquet_test_data(); + let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet"); + let data = Bytes::from(std::fs::read(path).unwrap()); + + let async_reader = TestReader::new(data); + + // Enable page index so the fetch logic loads only required pages + let options = ArrowReaderOptions::new().with_page_index(true); + let builder = ParquetRecordBatchStreamBuilder::new_with_options(async_reader, options) + .await + .unwrap(); + + // Skip the first 22 rows (entire first Parquet page) and then select the + // next 3 rows (22, 23, 24). This means the fetch step will not include + // the first page starting at file offset 0. + let selection = RowSelection::from(vec![RowSelector::skip(22), RowSelector::select(3)]); + + // Trivial predicate on column 0 that always returns `true`. Using the + // same column in both predicate and projection activates the caching + // layer (Producer/Consumer pattern). + let parquet_schema = builder.parquet_schema(); + let proj = ProjectionMask::leaves(parquet_schema, vec![0]); + let always_true = ArrowPredicateFn::new(proj.clone(), |batch: RecordBatch| { + Ok(BooleanArray::from(vec![true; batch.num_rows()])) + }); + let filter = RowFilter::new(vec![Box::new(always_true)]); + + // Build the stream with batch size 8 so the cache reads whole batches + // that straddle the requested row range (rows 0-7, 8-15, 16-23, …). + let stream = builder + .with_batch_size(8) + .with_projection(proj) + .with_row_selection(selection) + .with_row_filter(filter) + .build() + .unwrap(); + + // Collecting the stream should fail with the sparse column chunk offset + // error we want to reproduce. + let _result: Vec<_> = stream.try_collect().await.unwrap(); + } } From 2bf3d38094f65168387c44adcf16db5995c2b83b Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 8 Jul 2025 11:29:58 -0500 Subject: [PATCH 13/38] clippy --- parquet/src/arrow/arrow_reader/selection.rs | 51 +-------------------- 1 file changed, 1 insertion(+), 50 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/selection.rs b/parquet/src/arrow/arrow_reader/selection.rs index 90d8763dfe73..229eae4c5bb6 100644 --- a/parquet/src/arrow/arrow_reader/selection.rs +++ b/parquet/src/arrow/arrow_reader/selection.rs @@ -464,7 +464,7 @@ impl RowSelection { // Expand start to batch boundary let expanded_start = (start / batch_size) * batch_size; // Expand end to batch boundary - let expanded_end = ((end + batch_size - 1) / batch_size) * batch_size; + let expanded_end = end.div_ceil(batch_size) * batch_size; let expanded_end = expanded_end.min(total_rows); expanded_ranges.push(expanded_start..expanded_end); @@ -1431,53 +1431,4 @@ mod tests { assert_eq!(selection.row_count(), 0); assert_eq!(selection.skipped_row_count(), 0); } - - #[test] - #[cfg(feature = "async")] - fn test_expand_to_batch_boundaries() { - // Test case that reproduces the overlapping ranges bug - let selection = RowSelection::from(vec![ - RowSelector::skip(21), // Skip first page - RowSelector::select(21), // Select page to boundary - RowSelector::skip(41), // Skip multiple pages - RowSelector::select(41), // Select multiple pages - RowSelector::skip(25), // Skip page across boundary - RowSelector::select(25), // Select across page boundary - RowSelector::skip(7116), // Skip to final page boundary - RowSelector::select(10), // Select final page - ]); - - let total_rows = 7300; - let batch_size = 1024; - - // This should not panic with "out of order" - let expanded = selection.expand_to_batch_boundaries(batch_size, total_rows); - - // Verify that the expanded selection is valid - assert!(expanded.selects_any()); - assert!(expanded.row_count() >= selection.row_count()); - - // Test with smaller batch size that would cause more overlaps - let batch_size = 32; - let expanded = selection.expand_to_batch_boundaries(batch_size, total_rows); - assert!(expanded.selects_any()); - - // Test edge case with batch_size = 0 - let expanded = selection.expand_to_batch_boundaries(0, total_rows); - assert_eq!(expanded, selection); - - // Test simple case with two adjacent selectors - let selection = RowSelection::from(vec![ - RowSelector::select(10), // 0-10 - RowSelector::skip(5), // 10-15 - RowSelector::select(10), // 15-25 - ]); - - let expanded = selection.expand_to_batch_boundaries(32, 100); - // Both selectors should expand to 0-32 - assert_eq!( - expanded.selectors, - vec![RowSelector::select(32), RowSelector::skip(68)] - ); - } } From 2cf1a8f82f722e1c7e4857d7b07ba726f67d9f2f Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 8 Jul 2025 17:00:56 -0500 Subject: [PATCH 14/38] bug fix --- .../arrow/array_reader/cached_array_reader.rs | 37 ++++++++++++++++--- 1 file changed, 31 insertions(+), 6 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 62851f4b6f1d..336464cce6b0 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -188,7 +188,7 @@ impl ArrayReader for CachedArrayReader { let cached = if let Some(array) = self.local_buffer.get(&batch_id) { Some(array.clone()) } else { - // If not in local cache, check shared cache + // If not in local cache, i.e., we are consumer, check shared cache let shared_cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); if let Some(array) = shared_cached.as_ref() { // Store in local cache for later use in consume_batch @@ -200,12 +200,14 @@ impl ArrayReader for CachedArrayReader { match cached { Some(array) => { let array_len = array.len(); - if array_len + batch_id.val * self.batch_size - self.outer_position > 0 { + if array_len + batch_id.val * self.batch_size - self.outer_position - read > 0 { // the cache batch has some records that we can select - let v = array_len + batch_id.val * self.batch_size - self.outer_position; + let v = + array_len + batch_id.val * self.batch_size - self.outer_position - read; let select_cnt = std::cmp::min(num_records - read, v); read += select_cnt; self.selections.push_back(RowSelector::select(select_cnt)); + self.outer_position += select_cnt; } else { // this is last batch and we have used all records from it break; @@ -213,7 +215,6 @@ impl ArrayReader for CachedArrayReader { } None => { let read_from_inner = self.fetch_batch(batch_id)?; - // Reached end-of-file, no more records to read if read_from_inner == 0 { break; @@ -225,6 +226,7 @@ impl ArrayReader for CachedArrayReader { read += select_from_this_batch; self.selections .push_back(RowSelector::select(select_from_this_batch)); + self.outer_position += select_from_this_batch; if read_from_inner < self.batch_size { // this is last batch from inner reader break; @@ -232,7 +234,6 @@ impl ArrayReader for CachedArrayReader { } } } - self.outer_position += read; Ok(read) } @@ -294,7 +295,12 @@ impl ArrayReader for CachedArrayReader { } self.selections.clear(); - self.local_buffer.clear(); + + // Only remove batches from local buffer that are completely behind current position + // Keep the current batch and any future batches as they might still be needed + let current_batch_id = self.get_batch_id_from_position(self.outer_position); + self.local_buffer + .retain(|batch_id, _| batch_id.val >= current_batch_id.val); // For consumers, cleanup batches that have been completely consumed // This reduces the memory usage of the shared cache @@ -634,4 +640,23 @@ mod tests { // Local cache should be cleared after consume_batch assert!(cached_reader.local_buffer.is_empty()); } + + #[test] + fn test_local_cache_is_cleared_properly() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, Some(0)))); // Batch size 3, cache 0 + let mut cached_reader = + CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + + // Read records which should populate both shared and local cache + let records_read = cached_reader.read_records(1).unwrap(); + assert_eq!(records_read, 1); + let array = cached_reader.consume_batch().unwrap(); + assert_eq!(array.len(), 1); + + let records_read = cached_reader.read_records(3).unwrap(); + assert_eq!(records_read, 3); + let array = cached_reader.consume_batch().unwrap(); + assert_eq!(array.len(), 3); + } } From 86e149c01caea9e8b61fbdc6a74309452922e4d7 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 15 Jul 2025 11:09:48 -0500 Subject: [PATCH 15/38] switch to boolean array for row selection --- .../arrow/array_reader/cached_array_reader.rs | 35 ++++++------------- .../src/arrow/array_reader/row_group_cache.rs | 8 ++--- 2 files changed, 15 insertions(+), 28 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 336464cce6b0..d15b79fbab2a 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -17,13 +17,12 @@ use crate::arrow::array_reader::row_group_cache::BatchID; use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; -use crate::arrow::arrow_reader::RowSelector; use crate::errors::Result; use arrow_array::{new_empty_array, ArrayRef, BooleanArray}; -use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder}; +use arrow_buffer::BooleanBufferBuilder; use arrow_schema::DataType as ArrowType; use std::any::Any; -use std::collections::{HashMap, VecDeque}; +use std::collections::HashMap; use std::sync::{Arc, Mutex}; /// Role of the cached array reader @@ -73,8 +72,8 @@ pub struct CachedArrayReader { inner_position: usize, /// Batch size for the cache batch_size: usize, - /// Selections to be applied to the next consume_batch() - selections: VecDeque, + /// Boolean buffer builder to track selections for the next consume_batch() + selections: BooleanBufferBuilder, /// Role of this reader (Producer or Consumer) role: CacheRole, /// Local buffer to store batches between read_records and consume_batch calls @@ -99,7 +98,7 @@ impl CachedArrayReader { outer_position: 0, inner_position: 0, batch_size, - selections: VecDeque::new(), + selections: BooleanBufferBuilder::new(0), role, local_buffer: HashMap::new(), } @@ -206,7 +205,7 @@ impl ArrayReader for CachedArrayReader { array_len + batch_id.val * self.batch_size - self.outer_position - read; let select_cnt = std::cmp::min(num_records - read, v); read += select_cnt; - self.selections.push_back(RowSelector::select(select_cnt)); + self.selections.append_n(select_cnt, true); self.outer_position += select_cnt; } else { // this is last batch and we have used all records from it @@ -224,8 +223,7 @@ impl ArrayReader for CachedArrayReader { self.inner_position - self.outer_position, ); read += select_from_this_batch; - self.selections - .push_back(RowSelector::select(select_from_this_batch)); + self.selections.append_n(select_from_this_batch, true); self.outer_position += select_from_this_batch; if read_from_inner < self.batch_size { // this is last batch from inner reader @@ -242,21 +240,21 @@ impl ArrayReader for CachedArrayReader { while skipped < num_records { let size = std::cmp::min(num_records - skipped, self.batch_size); skipped += size; - self.selections.push_back(RowSelector::skip(size)); + self.selections.append_n(size, false); self.outer_position += size; } Ok(num_records) } fn consume_batch(&mut self) -> Result { - let row_count = self.selections.iter().map(|s| s.row_count).sum::(); + let row_count = self.selections.len(); if row_count == 0 { return Ok(new_empty_array(self.inner.get_data_type())); } let start_position = self.outer_position - row_count; - let selection_buffer = row_selection_to_boolean_buffer(row_count, self.selections.iter()); + let selection_buffer = self.selections.finish(); let start_batch = start_position / self.batch_size; let end_batch = (start_position + row_count - 1) / self.batch_size; @@ -294,7 +292,7 @@ impl ArrayReader for CachedArrayReader { selected_arrays.push(filtered); } - self.selections.clear(); + self.selections = BooleanBufferBuilder::new(0); // Only remove batches from local buffer that are completely behind current position // Keep the current batch and any future batches as they might still be needed @@ -329,17 +327,6 @@ impl ArrayReader for CachedArrayReader { } } -fn row_selection_to_boolean_buffer<'a>( - row_count: usize, - selection: impl Iterator, -) -> BooleanBuffer { - let mut buffer = BooleanBufferBuilder::new(row_count); - for selector in selection { - buffer.append_n(selector.row_count, !selector.skip); - } - buffer.finish() -} - #[cfg(test)] mod tests { use super::*; diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index a86b618c8dcb..19c1f2cce124 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -43,18 +43,18 @@ pub struct RowGroupCache { /// Cache granularity batch_size: usize, /// Maximum cache size in bytes (None means unlimited) - max_cache_size: Option, + max_cache_bytes: Option, /// Current cache size in bytes current_cache_size: usize, } impl RowGroupCache { /// Creates a new empty row group cache - pub fn new(batch_size: usize, max_cache_size: Option) -> Self { + pub fn new(batch_size: usize, max_cache_bytes: Option) -> Self { Self { cache: HashMap::new(), batch_size, - max_cache_size, + max_cache_bytes, current_cache_size: 0, } } @@ -65,7 +65,7 @@ impl RowGroupCache { let array_size = array.get_array_memory_size(); // Check if adding this array would exceed the cache size limit - if let Some(max_size) = self.max_cache_size { + if let Some(max_size) = self.max_cache_bytes { if self.current_cache_size + array_size > max_size { return false; // Cache is full, don't insert } From be134d63486d983d20539d1a9e3cfa60caf5e34d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 15 Jul 2025 15:07:13 -0400 Subject: [PATCH 16/38] Add comments (OCD) and rename some fields --- .../arrow/array_reader/cached_array_reader.rs | 56 +++++++++++-------- .../src/arrow/array_reader/row_group_cache.rs | 14 ++++- 2 files changed, 45 insertions(+), 25 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index d15b79fbab2a..d9d0a2c0bd92 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -48,6 +48,8 @@ pub enum CacheRole { /// 1. During the filter phase, the memory increases as the cache is populated /// 2. It peaks when filters are built. /// 3. It decreases as the cached data is consumed. +/// +/// ```text /// ▲ /// │ ╭─╮ /// │ ╱ ╲ @@ -59,16 +61,17 @@ pub enum CacheRole { /// │ │ │ /// Filter Peak Consume /// Phase (Built) (Decrease) +/// ``` pub struct CachedArrayReader { /// The underlying array reader inner: Box, /// Shared cache for this row group - cache: Arc>, + shared_cache: Arc>, /// Column index for cache key generation column_idx: usize, - /// Current logical position in the data stream (for cache key generation) + /// Current logical position in the data stream for this reader (for cache key generation) outer_position: usize, - /// Current position in the inner reader + /// Current position in `inner` inner_position: usize, /// Batch size for the cache batch_size: usize, @@ -76,9 +79,9 @@ pub struct CachedArrayReader { selections: BooleanBufferBuilder, /// Role of this reader (Producer or Consumer) role: CacheRole, - /// Local buffer to store batches between read_records and consume_batch calls + /// Local cache to store batches between read_records and consume_batch calls /// This ensures data is available even if the shared cache evicts items - local_buffer: HashMap, + local_cache: HashMap, } impl CachedArrayReader { @@ -93,27 +96,34 @@ impl CachedArrayReader { Self { inner, - cache, + shared_cache: cache, column_idx, outer_position: 0, inner_position: 0, batch_size, selections: BooleanBufferBuilder::new(0), role, - local_buffer: HashMap::new(), + local_cache: HashMap::new(), } } + fn get_batch_id_from_position(&self, row_id: usize) -> BatchID { BatchID { val: row_id / self.batch_size, } } + /// Loads the batch with the given ID (first row offset) from the inner + /// reader + /// + /// After this call the required batch will be available in + /// `self.local_cache` and may also be stored in `self.shared_cache`. + /// fn fetch_batch(&mut self, batch_id: BatchID) -> Result { - let row_id = batch_id.val * self.batch_size; - if self.inner_position < row_id { - let to_skip = row_id - self.inner_position; + let first_row_offset = batch_id.val * self.batch_size; + if self.inner_position < first_row_offset { + let to_skip = first_row_offset - self.inner_position; let skipped = self.inner.skip_records(to_skip)?; assert_eq!(skipped, to_skip); self.inner_position += skipped; @@ -131,17 +141,17 @@ impl CachedArrayReader { let array = self.inner.consume_batch()?; // Store in both shared cache and local cache - // The shared cache is for coordination between readers + // The shared cache is used to reuse results between readers // The local cache ensures data is available for our consume_batch call let _cached = self - .cache + .shared_cache .lock() .unwrap() .insert(self.column_idx, batch_id, array.clone()); // Note: if the shared cache is full (_cached == false), we continue without caching // The local cache will still store the data for this reader's use - self.local_buffer.insert(batch_id, array); + self.local_cache.insert(batch_id, array); self.inner_position += read; Ok(read) @@ -156,7 +166,7 @@ impl CachedArrayReader { // This ensures we don't remove batches that might still be needed for the current batch // We can safely remove batch_id if current_batch_id > batch_id + 1 if current_batch_id.val > 1 { - let mut cache = self.cache.lock().unwrap(); + let mut cache = self.shared_cache.lock().unwrap(); for batch_id_to_remove in 0..(current_batch_id.val - 1) { cache.remove( self.column_idx, @@ -184,16 +194,16 @@ impl ArrayReader for CachedArrayReader { let batch_id = self.get_batch_id_from_position(self.outer_position + read); // Check local cache first - let cached = if let Some(array) = self.local_buffer.get(&batch_id) { + let cached = if let Some(array) = self.local_cache.get(&batch_id) { Some(array.clone()) } else { // If not in local cache, i.e., we are consumer, check shared cache - let shared_cached = self.cache.lock().unwrap().get(self.column_idx, batch_id); - if let Some(array) = shared_cached.as_ref() { + let cache_content = self.shared_cache.lock().unwrap().get(self.column_idx, batch_id); + if let Some(array) = cache_content.as_ref() { // Store in local cache for later use in consume_batch - self.local_buffer.insert(batch_id, array.clone()); + self.local_cache.insert(batch_id, array.clone()); } - shared_cached + cache_content }; match cached { @@ -284,7 +294,7 @@ impl ArrayReader for CachedArrayReader { let mask_array = BooleanArray::from(mask); // Read from local cache instead of shared cache to avoid cache eviction issues let cached = self - .local_buffer + .local_cache .get(&batch_id) .expect("data must be already cached in the read_records call, this is a bug"); let cached = cached.slice(overlap_start - batch_start, selection_length); @@ -297,7 +307,7 @@ impl ArrayReader for CachedArrayReader { // Only remove batches from local buffer that are completely behind current position // Keep the current batch and any future batches as they might still be needed let current_batch_id = self.get_batch_id_from_position(self.outer_position); - self.local_buffer + self.local_cache .retain(|batch_id, _| batch_id.val >= current_batch_id.val); // For consumers, cleanup batches that have been completely consumed @@ -610,7 +620,7 @@ mod tests { // Verify data is in both caches assert!(cache.lock().unwrap().get(0, BatchID { val: 0 }).is_some()); - assert!(cached_reader.local_buffer.contains_key(&BatchID { val: 0 })); + assert!(cached_reader.local_cache.contains_key(&BatchID { val: 0 })); // Simulate cache eviction by manually removing from shared cache cache.lock().unwrap().remove(0, BatchID { val: 0 }); @@ -625,7 +635,7 @@ mod tests { assert_eq!(int32_array.values(), &[1, 2, 3]); // Local cache should be cleared after consume_batch - assert!(cached_reader.local_buffer.is_empty()); + assert!(cached_reader.local_cache.is_empty()); } #[test] diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 19c1f2cce124..0629f9ad9e61 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -18,9 +18,19 @@ use arrow_array::ArrayRef; use std::collections::HashMap; -#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +/// Starting row ID for this batch +/// +/// The `BatchID` is used to identify batches of rows within a row group. +/// +/// The row_index in the id are relative to the rows being read from the +/// underlying column reader (which might already have a RowSelection applied) +/// +/// The `BatchID` for any particular row is `row_index / batch_size`. The +/// integer division ensures that rows in the same batch share the same +/// the BatchID which can be calculated quickly from the row index +# [derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] pub struct BatchID { - pub val: usize, // batch id is row id / batch_size + pub val: usize, } /// Cache key that uniquely identifies a batch within a row group From 5537bcb0870ba21549e72b58b65237ba823eec50 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 15 Jul 2025 15:28:56 -0500 Subject: [PATCH 17/38] fmt --- parquet/src/arrow/array_reader/row_group_cache.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 0629f9ad9e61..cc3e956da73c 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -28,7 +28,7 @@ use std::collections::HashMap; /// The `BatchID` for any particular row is `row_index / batch_size`. The /// integer division ensures that rows in the same batch share the same /// the BatchID which can be calculated quickly from the row index -# [derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] pub struct BatchID { pub val: usize, } From b8351639569fe49d4aa88eab4b4efd2270859ec0 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 16 Jul 2025 09:30:25 -0400 Subject: [PATCH 18/38] fmt --- .../arrow/array_reader/cached_array_reader.rs | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index d9d0a2c0bd92..280fcedc66a5 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -107,7 +107,6 @@ impl CachedArrayReader { } } - fn get_batch_id_from_position(&self, row_id: usize) -> BatchID { BatchID { val: row_id / self.batch_size, @@ -143,11 +142,11 @@ impl CachedArrayReader { // Store in both shared cache and local cache // The shared cache is used to reuse results between readers // The local cache ensures data is available for our consume_batch call - let _cached = self - .shared_cache - .lock() - .unwrap() - .insert(self.column_idx, batch_id, array.clone()); + let _cached = + self.shared_cache + .lock() + .unwrap() + .insert(self.column_idx, batch_id, array.clone()); // Note: if the shared cache is full (_cached == false), we continue without caching // The local cache will still store the data for this reader's use @@ -198,7 +197,11 @@ impl ArrayReader for CachedArrayReader { Some(array.clone()) } else { // If not in local cache, i.e., we are consumer, check shared cache - let cache_content = self.shared_cache.lock().unwrap().get(self.column_idx, batch_id); + let cache_content = self + .shared_cache + .lock() + .unwrap() + .get(self.column_idx, batch_id); if let Some(array) = cache_content.as_ref() { // Store in local cache for later use in consume_batch self.local_cache.insert(batch_id, array.clone()); From 5132de88171e201dc12ff247ba416ef8e6de06e2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 16 Jul 2025 09:53:25 -0400 Subject: [PATCH 19/38] Simplify projection caching --- parquet/src/arrow/async_reader/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 56ce7ad796ca..d097b6d9965a 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -588,6 +588,7 @@ where .filter(|index| !index.is_empty()) .map(|x| x[row_group_idx].as_slice()); + // Reuse columns that are selected and used by the filters let cache_projection = match self.compute_cache_projection(&projection) { Some(projection) => projection, None => ProjectionMask::none(meta.columns().len()), @@ -628,8 +629,6 @@ where ) .await?; - let mut cache_projection = predicate.projection().clone(); - cache_projection.intersect(&projection); let array_reader = ArrayReaderBuilder::new(&row_group) .build_array_reader_with_cache( self.fields.as_deref(), From 253dad30324db714cfa7353608d6bd233dc28f38 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 16 Jul 2025 11:14:55 -0400 Subject: [PATCH 20/38] Move cache options construction to ArrayReaderBuilder, add builders --- parquet/src/arrow/array_reader/builder.rs | 100 ++++++++++++++-------- parquet/src/arrow/array_reader/mod.rs | 2 +- parquet/src/arrow/async_reader/mod.rs | 33 +++---- 3 files changed, 77 insertions(+), 58 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 883e46b584e5..69bfcb9929c0 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -39,7 +39,47 @@ use crate::schema::types::{ColumnDescriptor, ColumnPath, Type}; /// Builds [`ArrayReader`]s from parquet schema, projection mask, and RowGroups reader pub struct ArrayReaderBuilder<'a> { + /// Source of row group data row_groups: &'a dyn RowGroups, + /// Optional cache options for the array reader + cache_options: Option<&'a CacheOptions<'a>>, +} + +/// Builder for [`CacheOptions`] +#[derive(Debug, Clone)] +pub struct CacheOptionsBuilder<'a> { + /// Projection mask to apply to the cache + pub projection_mask: &'a ProjectionMask, + /// Cache to use for storing row groups + pub cache: Arc>, +} + +impl<'a> CacheOptionsBuilder<'a> { + /// create a new cache options builder + pub fn new(projection_mask: &'a ProjectionMask, cache: Arc>) -> Self { + Self { + projection_mask, + cache, + } + } + + /// Return a new [`CacheOptions`] for producing (populating) the cache + pub fn producer(self) -> CacheOptions<'a> { + CacheOptions { + projection_mask: self.projection_mask, + cache: self.cache, + role: CacheRole::Producer, + } + } + + /// return a new [`CacheOptions`] for consuming (reading) the cache + pub fn consumer(self) -> CacheOptions<'a> { + CacheOptions { + projection_mask: self.projection_mask, + cache: self.cache, + role: CacheRole::Consumer, + } + } } /// Cache options containing projection mask, cache, and role @@ -52,35 +92,26 @@ pub struct CacheOptions<'a> { impl<'a> ArrayReaderBuilder<'a> { pub fn new(row_groups: &'a dyn RowGroups) -> Self { - Self { row_groups } + Self { + row_groups, + cache_options: None, + } } - /// Create [`ArrayReader`] from parquet schema, projection mask, and parquet file reader. - pub fn build_array_reader( - &self, - field: Option<&ParquetField>, - mask: &ProjectionMask, - ) -> Result> { - let reader = field - .and_then(|field| self.build_reader(field, mask, None).transpose()) - .transpose()? - .unwrap_or_else(|| make_empty_array_reader(self.num_rows())); - - Ok(reader) + /// Add cache options to the builder + pub fn with_cache_options(mut self, cache_options: Option<&'a CacheOptions<'a>>) -> Self { + self.cache_options = cache_options; + self } /// Create [`ArrayReader`] from parquet schema, projection mask, and parquet file reader. - pub fn build_array_reader_with_cache( + pub fn build_array_reader( &self, field: Option<&ParquetField>, mask: &ProjectionMask, - cache_options: CacheOptions, ) -> Result> { let reader = field - .and_then(|field| { - self.build_reader(field, mask, Some(cache_options)) - .transpose() - }) + .and_then(|field| self.build_reader(field, mask).transpose()) .transpose()? .unwrap_or_else(|| make_empty_array_reader(self.num_rows())); @@ -96,21 +127,20 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_options: Option, ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { col_idx, .. } => { let Some(reader) = self.build_primitive_reader(field, mask)? else { return Ok(None); }; - let Some(cache_options) = cache_options else { + let Some(cache_options) = self.cache_options.as_ref() else { return Ok(Some(reader)); }; if cache_options.projection_mask.leaf_included(col_idx) { Ok(Some(Box::new(CachedArrayReader::new( reader, - cache_options.cache, + Arc::clone(&cache_options.cache), col_idx, cache_options.role, )))) @@ -119,13 +149,11 @@ impl<'a> ArrayReaderBuilder<'a> { } } ParquetFieldType::Group { .. } => match &field.arrow_type { - DataType::Map(_, _) => self.build_map_reader(field, mask, cache_options), - DataType::Struct(_) => self.build_struct_reader(field, mask, cache_options), - DataType::List(_) => self.build_list_reader(field, mask, cache_options, false), - DataType::LargeList(_) => self.build_list_reader(field, mask, cache_options, true), - DataType::FixedSizeList(_, _) => { - self.build_fixed_size_list_reader(field, mask, cache_options) - } + DataType::Map(_, _) => self.build_map_reader(field, mask), + DataType::Struct(_) => self.build_struct_reader(field, mask), + DataType::List(_) => self.build_list_reader(field, mask, false), + DataType::LargeList(_) => self.build_list_reader(field, mask, true), + DataType::FixedSizeList(_, _) => self.build_fixed_size_list_reader(field, mask), d => unimplemented!("reading group type {} not implemented", d), }, } @@ -136,13 +164,12 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_options: Option, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 2); - let key_reader = self.build_reader(&children[0], mask, cache_options.clone())?; - let value_reader = self.build_reader(&children[1], mask, cache_options)?; + let key_reader = self.build_reader(&children[0], mask)?; + let value_reader = self.build_reader(&children[1], mask)?; match (key_reader, value_reader) { (Some(key_reader), Some(value_reader)) => { @@ -188,13 +215,12 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_options: Option, is_large: bool, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, cache_options)? { + let reader = match self.build_reader(&children[0], mask)? { Some(item_reader) => { // Need to retrieve underlying data type to handle projection let item_type = item_reader.get_data_type().clone(); @@ -236,12 +262,11 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_options: Option, ) -> Result>> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, cache_options)? { + let reader = match self.build_reader(&children[0], mask)? { Some(item_reader) => { let item_type = item_reader.get_data_type().clone(); let reader = match &field.arrow_type { @@ -371,7 +396,6 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, - cache_options: Option, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -384,7 +408,7 @@ impl<'a> ArrayReaderBuilder<'a> { let mut builder = SchemaBuilder::with_capacity(children.len()); for (arrow, parquet) in arrow_fields.iter().zip(children) { - if let Some(reader) = self.build_reader(parquet, mask, cache_options.clone())? { + if let Some(reader) = self.build_reader(parquet, mask)? { // Need to retrieve underlying data type to handle projection let child_type = reader.get_data_type().clone(); builder.push(arrow.as_ref().clone().with_data_type(child_type)); diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 85301ef87697..171c446f413b 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -48,7 +48,7 @@ mod struct_array; mod test_util; // Note that this crate is public under the `experimental` feature flag. -pub use builder::{ArrayReaderBuilder, CacheOptions}; +pub use builder::{ArrayReaderBuilder, CacheOptions, CacheOptionsBuilder}; pub use byte_array::make_byte_array_reader; pub use byte_array_dictionary::make_byte_array_dictionary_reader; #[allow(unused_imports)] // Only used for benchmarks diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 56ce7ad796ca..596f7a50be7e 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -38,7 +38,9 @@ use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt}; use arrow_array::RecordBatch; use arrow_schema::{DataType, Fields, Schema, SchemaRef}; -use crate::arrow::array_reader::{ArrayReaderBuilder, CacheOptions, RowGroupCache, RowGroups}; +use crate::arrow::array_reader::{ + ArrayReaderBuilder, CacheOptionsBuilder, RowGroupCache, RowGroups, +}; use crate::arrow::arrow_reader::{ ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection, @@ -607,11 +609,16 @@ where metadata: self.metadata.as_ref(), }; + let cache_options_builder = + CacheOptionsBuilder::new(&cache_projection, row_group_cache.clone()); + let filter = self.filter.as_mut(); let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); // Update selection based on any filters if let Some(filter) = filter { + let cache_options = cache_options_builder.clone().producer(); + for predicate in filter.predicates.iter_mut() { if !plan_builder.selects_any() { return Ok((self, None)); // ruled out entire row group @@ -631,15 +638,8 @@ where let mut cache_projection = predicate.projection().clone(); cache_projection.intersect(&projection); let array_reader = ArrayReaderBuilder::new(&row_group) - .build_array_reader_with_cache( - self.fields.as_deref(), - predicate.projection(), - CacheOptions { - projection_mask: &cache_projection, - cache: row_group_cache.clone(), - role: crate::arrow::array_reader::CacheRole::Producer, - }, - )?; + .with_cache_options(Some(&cache_options)) + .build_array_reader(self.fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } @@ -691,15 +691,10 @@ where let plan = plan_builder.build(); - let array_reader = ArrayReaderBuilder::new(&row_group).build_array_reader_with_cache( - self.fields.as_deref(), - &projection, - CacheOptions { - projection_mask: &cache_projection, - cache: row_group_cache.clone(), - role: crate::arrow::array_reader::CacheRole::Consumer, - }, - )?; + let cache_options = cache_options_builder.consumer(); + let array_reader = ArrayReaderBuilder::new(&row_group) + .with_cache_options(Some(&cache_options)) + .build_array_reader(self.fields.as_deref(), &projection)?; let reader = ParquetRecordBatchReader::new(array_reader, plan); From 5d9781e554ad9ed8ee29d9ac393f7d862bccc7d4 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 17 Jul 2025 12:08:49 -0500 Subject: [PATCH 21/38] update memory accounting --- parquet/src/arrow/array_reader/row_group_cache.rs | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index cc3e956da73c..efeb9ed1e29c 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use arrow_array::ArrayRef; +use arrow_array::{Array, ArrayRef}; +use arrow_schema::DataType; use std::collections::HashMap; /// Starting row ID for this batch @@ -72,7 +73,16 @@ impl RowGroupCache { /// Inserts an array into the cache for the given column and starting row ID /// Returns true if the array was inserted, false if it would exceed the cache size limit pub fn insert(&mut self, column_idx: usize, batch_id: BatchID, array: ArrayRef) -> bool { - let array_size = array.get_array_memory_size(); + let array_size = match array.data_type() { + // TODO: this is temporary workaround. It's very difficult to measure the actual memory usage of one StringViewArray, + // because the underlying buffer is shared with multiple StringViewArrays. + DataType::Utf8View => { + use arrow_array::cast::AsArray; + let array = array.as_string_view(); + array.len() * 16 + array.total_buffer_bytes_used() + std::mem::size_of_val(&array) + } + _ => array.get_array_memory_size(), + }; // Check if adding this array would exceed the cache size limit if let Some(max_size) = self.max_cache_bytes { From 884b5919d8651f20e03989f28950d2077fcd4e2b Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 17 Jul 2025 12:25:02 -0500 Subject: [PATCH 22/38] update --- parquet/src/arrow/array_reader/mod.rs | 3 +-- parquet/src/arrow/array_reader/row_group_cache.rs | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 171c446f413b..5033bbc1b9b5 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -48,12 +48,11 @@ mod struct_array; mod test_util; // Note that this crate is public under the `experimental` feature flag. -pub use builder::{ArrayReaderBuilder, CacheOptions, CacheOptionsBuilder}; +pub use builder::{ArrayReaderBuilder, CacheOptionsBuilder}; pub use byte_array::make_byte_array_reader; pub use byte_array_dictionary::make_byte_array_dictionary_reader; #[allow(unused_imports)] // Only used for benchmarks pub use byte_view_array::make_byte_view_array_reader; -pub use cached_array_reader::CacheRole; #[allow(unused_imports)] // Only used for benchmarks pub use fixed_len_byte_array::make_fixed_len_byte_array_reader; pub use fixed_size_list_array::FixedSizeListArrayReader; diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index efeb9ed1e29c..32838cafd906 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -79,7 +79,7 @@ impl RowGroupCache { DataType::Utf8View => { use arrow_array::cast::AsArray; let array = array.as_string_view(); - array.len() * 16 + array.total_buffer_bytes_used() + std::mem::size_of_val(&array) + array.len() * 16 + array.total_buffer_bytes_used() + std::mem::size_of_val(array) } _ => array.get_array_memory_size(), }; From 4f6b918437c3bd139e89711b42e91f754ea03304 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 17 Jul 2025 13:26:11 -0500 Subject: [PATCH 23/38] array size --- .../src/arrow/array_reader/row_group_cache.rs | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 32838cafd906..583ad736bca9 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -43,6 +43,19 @@ pub struct CacheKey { pub batch_id: BatchID, } +fn get_array_memory_size_for_cache(array: &ArrayRef) -> usize { + match array.data_type() { + // TODO: this is temporary workaround. It's very difficult to measure the actual memory usage of one StringViewArray, + // because the underlying buffer is shared with multiple StringViewArrays. + DataType::Utf8View => { + use arrow_array::cast::AsArray; + let array = array.as_string_view(); + array.len() * 16 + array.total_buffer_bytes_used() + std::mem::size_of_val(array) + } + _ => array.get_array_memory_size(), + } +} + /// Row group cache that stores decoded arrow arrays at batch granularity /// /// This cache is designed to avoid duplicate decoding when the same column @@ -73,16 +86,7 @@ impl RowGroupCache { /// Inserts an array into the cache for the given column and starting row ID /// Returns true if the array was inserted, false if it would exceed the cache size limit pub fn insert(&mut self, column_idx: usize, batch_id: BatchID, array: ArrayRef) -> bool { - let array_size = match array.data_type() { - // TODO: this is temporary workaround. It's very difficult to measure the actual memory usage of one StringViewArray, - // because the underlying buffer is shared with multiple StringViewArrays. - DataType::Utf8View => { - use arrow_array::cast::AsArray; - let array = array.as_string_view(); - array.len() * 16 + array.total_buffer_bytes_used() + std::mem::size_of_val(array) - } - _ => array.get_array_memory_size(), - }; + let array_size = get_array_memory_size_for_cache(&array); // Check if adding this array would exceed the cache size limit if let Some(max_size) = self.max_cache_bytes { @@ -125,7 +129,7 @@ impl RowGroupCache { batch_id, }; if let Some(array) = self.cache.remove(&key) { - self.current_cache_size -= array.get_array_memory_size(); + self.current_cache_size -= get_array_memory_size_for_cache(&array); true } else { false From 6c53bfd045d532ef7395a253a97b14bfa2f8f767 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 17 Jul 2025 13:28:11 -0500 Subject: [PATCH 24/38] add test case --- .../arrow/array_reader/cached_array_reader.rs | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 280fcedc66a5..0388b0e9377b 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -659,4 +659,43 @@ mod tests { let array = cached_reader.consume_batch().unwrap(); assert_eq!(array.len(), 3); } + + + #[test] + fn test_batch_id_calculation_with_incremental_reads() { + let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 + + // Create a producer to populate cache + let mut producer = CachedArrayReader::new( + Box::new(MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9])), + cache.clone(), + 0, + CacheRole::Producer, + ); + + // Populate cache with first batch (1, 2, 3) + producer.read_records(3).unwrap(); + producer.consume_batch().unwrap(); + + // Now create a consumer that will try to read from cache + let mut consumer = CachedArrayReader::new( + Box::new(mock_reader), + cache.clone(), + 0, + CacheRole::Consumer, + ); + + // - We want to read 4 records starting from position 0 + // - First 3 records (positions 0-2) should come from cache (batch 0) + // - The 4th record (position 3) should come from the next batch + let records_read = consumer.read_records(4).unwrap(); + assert_eq!(records_read, 4); + + let array = consumer.consume_batch().unwrap(); + assert_eq!(array.len(), 4); + + let int32_array = array.as_any().downcast_ref::().unwrap(); + assert_eq!(int32_array.values(), &[1, 2, 3, 4]); + } } From 8ebe57918fab8049357e0051c17b6387d76fdaf9 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 17 Jul 2025 13:29:17 -0500 Subject: [PATCH 25/38] fix bug --- parquet/src/arrow/array_reader/cached_array_reader.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 0388b0e9377b..0609637ce422 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -190,7 +190,7 @@ impl ArrayReader for CachedArrayReader { fn read_records(&mut self, num_records: usize) -> Result { let mut read = 0; while read < num_records { - let batch_id = self.get_batch_id_from_position(self.outer_position + read); + let batch_id = self.get_batch_id_from_position(self.outer_position); // Check local cache first let cached = if let Some(array) = self.local_cache.get(&batch_id) { @@ -212,14 +212,13 @@ impl ArrayReader for CachedArrayReader { match cached { Some(array) => { let array_len = array.len(); - if array_len + batch_id.val * self.batch_size - self.outer_position - read > 0 { + if array_len + batch_id.val * self.batch_size > self.outer_position { // the cache batch has some records that we can select - let v = - array_len + batch_id.val * self.batch_size - self.outer_position - read; + let v = array_len + batch_id.val * self.batch_size - self.outer_position; let select_cnt = std::cmp::min(num_records - read, v); read += select_cnt; - self.selections.append_n(select_cnt, true); self.outer_position += select_cnt; + self.selections.append_n(select_cnt, true); } else { // this is last batch and we have used all records from it break; @@ -236,8 +235,8 @@ impl ArrayReader for CachedArrayReader { self.inner_position - self.outer_position, ); read += select_from_this_batch; - self.selections.append_n(select_from_this_batch, true); self.outer_position += select_from_this_batch; + self.selections.append_n(select_from_this_batch, true); if read_from_inner < self.batch_size { // this is last batch from inner reader break; From c240a52b2fd31af18f1e8f9c0885d76eae872322 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 17 Jul 2025 13:34:14 -0500 Subject: [PATCH 26/38] clippy & fmt --- .../arrow/array_reader/cached_array_reader.rs | 21 +++++++------------ parquet/src/arrow/array_reader/mod.rs | 2 +- 2 files changed, 9 insertions(+), 14 deletions(-) diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index 0609637ce422..dc416ca792fc 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -659,12 +659,11 @@ mod tests { assert_eq!(array.len(), 3); } - #[test] fn test_batch_id_calculation_with_incremental_reads() { let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 - + // Create a producer to populate cache let mut producer = CachedArrayReader::new( Box::new(MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9])), @@ -672,28 +671,24 @@ mod tests { 0, CacheRole::Producer, ); - + // Populate cache with first batch (1, 2, 3) producer.read_records(3).unwrap(); producer.consume_batch().unwrap(); - + // Now create a consumer that will try to read from cache - let mut consumer = CachedArrayReader::new( - Box::new(mock_reader), - cache.clone(), - 0, - CacheRole::Consumer, - ); - + let mut consumer = + CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + // - We want to read 4 records starting from position 0 // - First 3 records (positions 0-2) should come from cache (batch 0) // - The 4th record (position 3) should come from the next batch let records_read = consumer.read_records(4).unwrap(); assert_eq!(records_read, 4); - + let array = consumer.consume_batch().unwrap(); assert_eq!(array.len(), 4); - + let int32_array = array.as_any().downcast_ref::().unwrap(); assert_eq!(int32_array.values(), &[1, 2, 3, 4]); } diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 5033bbc1b9b5..662439742bdf 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -48,7 +48,7 @@ mod struct_array; mod test_util; // Note that this crate is public under the `experimental` feature flag. -pub use builder::{ArrayReaderBuilder, CacheOptionsBuilder}; +pub use builder::{ArrayReaderBuilder, CacheOptions, CacheOptionsBuilder}; pub use byte_array::make_byte_array_reader; pub use byte_array_dictionary::make_byte_array_dictionary_reader; #[allow(unused_imports)] // Only used for benchmarks From 30a0d1cd2539447fe6d8750457ef08aefe18c91f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 23 Jul 2025 10:23:48 -0400 Subject: [PATCH 27/38] Add config option for predicate cache memory limit --- parquet/tests/arrow_reader/mod.rs | 1 + parquet/tests/arrow_reader/predicate_cache.rs | 144 ++++++++++++++++++ 2 files changed, 145 insertions(+) create mode 100644 parquet/tests/arrow_reader/predicate_cache.rs diff --git a/parquet/tests/arrow_reader/mod.rs b/parquet/tests/arrow_reader/mod.rs index 739aa5666230..7d001a9832fa 100644 --- a/parquet/tests/arrow_reader/mod.rs +++ b/parquet/tests/arrow_reader/mod.rs @@ -38,6 +38,7 @@ use std::sync::Arc; use tempfile::NamedTempFile; mod bad_data; +mod predicate_cache; #[cfg(feature = "crc")] mod checksum; mod statistics; diff --git a/parquet/tests/arrow_reader/predicate_cache.rs b/parquet/tests/arrow_reader/predicate_cache.rs new file mode 100644 index 000000000000..be1a1c811b41 --- /dev/null +++ b/parquet/tests/arrow_reader/predicate_cache.rs @@ -0,0 +1,144 @@ +// 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. + +//! Test for predicate cache in Parquet Arrow reader + + + +use arrow::array::ArrayRef; +use std::sync::Arc; +use parquet::arrow::arrow_reader::ArrowReaderOptions; +use std::sync::LazyLock; +use arrow::array::Int64Array; +use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; +use bytes::Bytes; +use arrow_array::{RecordBatch, StringViewArray}; +use parquet::arrow::ArrowWriter; +use parquet::file::properties::WriterProperties; + + +// 1. the predicate cache is not used when there are no filters +#[test] +fn test() { + let test = ParquetPredicateCacheTest::new() + .with_expected_cache_used(false); + let builder = test.sync_builder(ArrowReaderOptions::default()); + test.run(builder); +} + + +// Test: +// 2. the predicate cache is used when there are filters but the cache size is 0 +// 3. the predicate cache is used when there are filters and the cache size is greater than 0 + + + + + +/// A test parquet file +struct ParquetPredicateCacheTest { + bytes: Bytes, + expected_cache_used: bool, +} +impl ParquetPredicateCacheTest { + /// Create a new `TestParquetFile` with: + /// 3 columns: "a", "b", "c" + /// + /// 2 row groups, each with 200 rows + /// each data page has 100 rows + /// + /// Values of column "a" are 0..399 + /// Values of column "b" are 400..799 + /// Values of column "c" are alternating strings of length 12 and longer + fn new() -> Self { + Self { + bytes: TEST_FILE_DATA.clone(), + expected_cache_used: false, + } + } + + /// Set whether the predicate cache is expected to be used + fn with_expected_cache_used(mut self, used: bool) -> Self{ + self.expected_cache_used = used; + self + } + + /// Return a [`ParquetRecordBatchReaderBuilder`] for reading this file + fn sync_builder( + &self, + options: ArrowReaderOptions, + ) -> ParquetRecordBatchReaderBuilder { + let reader = self.bytes.clone(); + ParquetRecordBatchReaderBuilder::try_new_with_options(reader, options) + .expect("ParquetRecordBatchReaderBuilder") + } + + + /// Build the reader from the specified builder, reading all batches from it, + /// and asserts the + fn run( + &self, + builder: ParquetRecordBatchReaderBuilder, + ) { + let reader = builder.build().unwrap(); + for batch in reader { + match batch { + Ok(_) => {} + Err(e) => panic!("Error reading batch: {e}"), + } + } + // TODO check if the cache was used + } +} + +/// Create a parquet file in memory for testing. See [`test_file`] for details. +static TEST_FILE_DATA: LazyLock = LazyLock::new(|| { + // Input batch has 400 rows, with 3 columns: "a", "b", "c" + // Note c is a different types (so the data page sizes will be different) + let a: ArrayRef = Arc::new(Int64Array::from_iter_values(0..400)); + let b: ArrayRef = Arc::new(Int64Array::from_iter_values(400..800)); + let c: ArrayRef = Arc::new(StringViewArray::from_iter_values((0..400).map(|i| { + if i % 2 == 0 { + format!("string_{i}") + } else { + format!("A string larger than 12 bytes and thus not inlined {i}") + } + }))); + + let input_batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); + + let mut output = Vec::new(); + + let writer_options = WriterProperties::builder() + .set_max_row_group_size(200) + .set_data_page_row_count_limit(100) + .build(); + let mut writer = + ArrowWriter::try_new(&mut output, input_batch.schema(), Some(writer_options)).unwrap(); + + // since the limits are only enforced on batch boundaries, write the input + // batch in chunks of 50 + let mut row_remain = input_batch.num_rows(); + while row_remain > 0 { + let chunk_size = row_remain.min(50); + let chunk = input_batch.slice(input_batch.num_rows() - row_remain, chunk_size); + writer.write(&chunk).unwrap(); + row_remain -= chunk_size; + } + writer.close().unwrap(); + Bytes::from(output) +}); From ed3ce1354c4bab8b681fe88b557e06f29081ecaf Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 23 Jul 2025 10:36:04 -0400 Subject: [PATCH 28/38] Add option to control predicate cache, documentation, ArrowReaderMetrics and tests --- parquet/examples/read_with_rowgroup.rs | 23 +- parquet/src/arrow/array_reader/builder.rs | 26 ++- .../arrow/array_reader/cached_array_reader.rs | 123 ++++++++--- parquet/src/arrow/array_reader/list_array.rs | 4 +- .../src/arrow/array_reader/row_group_cache.rs | 16 +- parquet/src/arrow/arrow_reader/metrics.rs | 135 ++++++++++++ parquet/src/arrow/arrow_reader/mod.rs | 109 ++++++++-- parquet/src/arrow/async_reader/mod.rs | 18 +- parquet/tests/arrow_reader/mod.rs | 3 +- parquet/tests/arrow_reader/predicate_cache.rs | 203 ++++++++++++++---- 10 files changed, 552 insertions(+), 108 deletions(-) create mode 100644 parquet/src/arrow/arrow_reader/metrics.rs diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index 5d1ff0770f9e..de33db1de816 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -17,6 +17,7 @@ use arrow::util::pretty::print_batches; use bytes::{Buf, Bytes}; +use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, RowGroups, RowSelection}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{parquet_to_arrow_field_levels, ProjectionMask}; @@ -38,7 +39,11 @@ async fn main() -> Result<()> { let metadata = file.get_metadata(None).await?; for rg in metadata.row_groups() { - let mut rowgroup = InMemoryRowGroup::create(rg.clone(), ProjectionMask::all()); + let mut rowgroup = InMemoryRowGroup::create( + rg.clone(), + ProjectionMask::all(), + ArrowReaderMetrics::disabled(), + ); rowgroup.async_fetch_data(&mut file, None).await?; let reader = rowgroup.build_reader(1024, None)?; @@ -103,6 +108,7 @@ pub struct InMemoryRowGroup { pub metadata: RowGroupMetaData, mask: ProjectionMask, column_chunks: Vec>>, + metrics: ArrowReaderMetrics, } impl RowGroups for InMemoryRowGroup { @@ -132,13 +138,18 @@ impl RowGroups for InMemoryRowGroup { } impl InMemoryRowGroup { - pub fn create(metadata: RowGroupMetaData, mask: ProjectionMask) -> Self { + pub fn create( + metadata: RowGroupMetaData, + mask: ProjectionMask, + metrics: ArrowReaderMetrics, + ) -> Self { let column_chunks = metadata.columns().iter().map(|_| None).collect::>(); Self { metadata, mask, column_chunks, + metrics, } } @@ -153,7 +164,13 @@ impl InMemoryRowGroup { None, )?; - ParquetRecordBatchReader::try_new_with_row_groups(&levels, self, batch_size, selection) + ParquetRecordBatchReader::try_new_with_row_groups( + &levels, + self, + batch_size, + selection, + &self.metrics, + ) } /// fetch data from a reader in sync mode diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 69bfcb9929c0..d5e36fbcb486 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -30,6 +30,7 @@ use crate::arrow::array_reader::{ FixedSizeListArrayReader, ListArrayReader, MapArrayReader, NullArrayReader, PrimitiveArrayReader, RowGroups, StructArrayReader, }; +use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; use crate::arrow::schema::{ParquetField, ParquetFieldType}; use crate::arrow::ProjectionMask; use crate::basic::Type as PhysicalType; @@ -37,14 +38,6 @@ use crate::data_type::{BoolType, DoubleType, FloatType, Int32Type, Int64Type, In use crate::errors::{ParquetError, Result}; use crate::schema::types::{ColumnDescriptor, ColumnPath, Type}; -/// Builds [`ArrayReader`]s from parquet schema, projection mask, and RowGroups reader -pub struct ArrayReaderBuilder<'a> { - /// Source of row group data - row_groups: &'a dyn RowGroups, - /// Optional cache options for the array reader - cache_options: Option<&'a CacheOptions<'a>>, -} - /// Builder for [`CacheOptions`] #[derive(Debug, Clone)] pub struct CacheOptionsBuilder<'a> { @@ -90,11 +83,22 @@ pub struct CacheOptions<'a> { pub role: CacheRole, } +/// Builds [`ArrayReader`]s from parquet schema, projection mask, and RowGroups reader +pub struct ArrayReaderBuilder<'a> { + /// Source of row group data + row_groups: &'a dyn RowGroups, + /// Optional cache options for the array reader + cache_options: Option<&'a CacheOptions<'a>>, + /// metrics + metrics: &'a ArrowReaderMetrics, +} + impl<'a> ArrayReaderBuilder<'a> { - pub fn new(row_groups: &'a dyn RowGroups) -> Self { + pub fn new(row_groups: &'a dyn RowGroups, metrics: &'a ArrowReaderMetrics) -> Self { Self { row_groups, cache_options: None, + metrics, } } @@ -143,6 +147,7 @@ impl<'a> ArrayReaderBuilder<'a> { Arc::clone(&cache_options.cache), col_idx, cache_options.role, + self.metrics.clone(), // cheap clone )))) } else { Ok(Some(reader)) @@ -453,7 +458,8 @@ mod tests { ) .unwrap(); - let array_reader = ArrayReaderBuilder::new(&file_reader) + let metrics = ArrowReaderMetrics::disabled(); + let array_reader = ArrayReaderBuilder::new(&file_reader, &metrics) .build_array_reader(fields.as_ref(), &mask) .unwrap(); diff --git a/parquet/src/arrow/array_reader/cached_array_reader.rs b/parquet/src/arrow/array_reader/cached_array_reader.rs index dc416ca792fc..0e837782faf5 100644 --- a/parquet/src/arrow/array_reader/cached_array_reader.rs +++ b/parquet/src/arrow/array_reader/cached_array_reader.rs @@ -15,8 +15,11 @@ // specific language governing permissions and limitations // under the License. +//! [`CachedArrayReader`] wrapper around [`ArrayReader`] + use crate::arrow::array_reader::row_group_cache::BatchID; use crate::arrow::array_reader::{row_group_cache::RowGroupCache, ArrayReader}; +use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; use crate::errors::Result; use arrow_array::{new_empty_array, ArrayRef, BooleanArray}; use arrow_buffer::BooleanBufferBuilder; @@ -82,6 +85,8 @@ pub struct CachedArrayReader { /// Local cache to store batches between read_records and consume_batch calls /// This ensures data is available even if the shared cache evicts items local_cache: HashMap, + /// Statistics to report on the Cache behavior + metrics: ArrowReaderMetrics, } impl CachedArrayReader { @@ -91,6 +96,7 @@ impl CachedArrayReader { cache: Arc>, column_idx: usize, role: CacheRole, + metrics: ArrowReaderMetrics, ) -> Self { let batch_size = cache.lock().unwrap().batch_size(); @@ -104,6 +110,7 @@ impl CachedArrayReader { selections: BooleanBufferBuilder::new(0), role, local_cache: HashMap::new(), + metrics, } } @@ -217,6 +224,7 @@ impl ArrayReader for CachedArrayReader { let v = array_len + batch_id.val * self.batch_size - self.outer_position; let select_cnt = std::cmp::min(num_records - read, v); read += select_cnt; + self.metrics.increment_cache_reads(select_cnt); self.outer_position += select_cnt; self.selections.append_n(select_cnt, true); } else { @@ -230,6 +238,7 @@ impl ArrayReader for CachedArrayReader { if read_from_inner == 0 { break; } + self.metrics.increment_inner_reads(read_from_inner); let select_from_this_batch = std::cmp::min( num_records - read, self.inner_position - self.outer_position, @@ -409,10 +418,16 @@ mod tests { #[test] fn test_cached_reader_basic() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 - let mut cached_reader = - CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Producer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, usize::MAX))); // Batch size 3 + let mut cached_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache, + 0, + CacheRole::Producer, + metrics, + ); // Read 3 records let records_read = cached_reader.read_records(3).unwrap(); @@ -431,10 +446,16 @@ mod tests { #[test] fn test_read_skip_pattern() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(5, None))); // Batch size 5 - let mut cached_reader = - CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5, usize::MAX))); // Batch size 5 + let mut cached_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache, + 0, + CacheRole::Consumer, + metrics, + ); let read1 = cached_reader.read_records(2).unwrap(); assert_eq!(read1, 2); @@ -459,10 +480,16 @@ mod tests { #[test] fn test_multiple_reads_before_consume() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 - let mut cached_reader = - CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, usize::MAX))); // Batch size 3 + let mut cached_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache, + 0, + CacheRole::Consumer, + metrics, + ); // Multiple reads should accumulate let read1 = cached_reader.read_records(2).unwrap(); @@ -480,10 +507,16 @@ mod tests { #[test] fn test_eof_behavior() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(5, None))); // Batch size 5 - let mut cached_reader = - CachedArrayReader::new(Box::new(mock_reader), cache, 0, CacheRole::Consumer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5, usize::MAX))); // Batch size 5 + let mut cached_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache, + 0, + CacheRole::Consumer, + metrics, + ); // Try to read more than available let read1 = cached_reader.read_records(5).unwrap(); @@ -502,7 +535,8 @@ mod tests { #[test] fn test_cache_sharing() { - let cache = Arc::new(Mutex::new(RowGroupCache::new(5, None))); // Batch size 5 + let metrics = ArrowReaderMetrics::disabled(); + let cache = Arc::new(Mutex::new(RowGroupCache::new(5, usize::MAX))); // Batch size 5 // First reader - populate cache let mock_reader1 = MockArrayReader::new(vec![1, 2, 3, 4, 5]); @@ -511,6 +545,7 @@ mod tests { cache.clone(), 0, CacheRole::Producer, + metrics.clone(), ); cached_reader1.read_records(3).unwrap(); @@ -524,6 +559,7 @@ mod tests { cache.clone(), 1, CacheRole::Consumer, + metrics.clone(), ); cached_reader2.read_records(2).unwrap(); @@ -537,10 +573,16 @@ mod tests { #[test] fn test_consumer_removes_batches() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 - let mut consumer_reader = - CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, usize::MAX))); // Batch size 3 + let mut consumer_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache.clone(), + 0, + CacheRole::Consumer, + metrics, + ); // Read first batch (positions 0-2, batch 0) let read1 = consumer_reader.read_records(3).unwrap(); @@ -584,10 +626,16 @@ mod tests { #[test] fn test_producer_keeps_batches() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 - let mut producer_reader = - CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Producer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, usize::MAX))); // Batch size 3 + let mut producer_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache.clone(), + 0, + CacheRole::Producer, + metrics, + ); // Read first batch (positions 0-2) let read1 = producer_reader.read_records(3).unwrap(); @@ -611,10 +659,16 @@ mod tests { #[test] fn test_local_cache_protects_against_eviction() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 - let mut cached_reader = - CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, usize::MAX))); // Batch size 3 + let mut cached_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache.clone(), + 0, + CacheRole::Consumer, + metrics, + ); // Read records which should populate both shared and local cache let records_read = cached_reader.read_records(3).unwrap(); @@ -642,10 +696,16 @@ mod tests { #[test] fn test_local_cache_is_cleared_properly() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, Some(0)))); // Batch size 3, cache 0 - let mut cached_reader = - CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, 0))); // Batch size 3, cache 0 + let mut cached_reader = CachedArrayReader::new( + Box::new(mock_reader), + cache.clone(), + 0, + CacheRole::Consumer, + metrics, + ); // Read records which should populate both shared and local cache let records_read = cached_reader.read_records(1).unwrap(); @@ -661,8 +721,9 @@ mod tests { #[test] fn test_batch_id_calculation_with_incremental_reads() { + let metrics = ArrowReaderMetrics::disabled(); let mock_reader = MockArrayReader::new(vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); - let cache = Arc::new(Mutex::new(RowGroupCache::new(3, None))); // Batch size 3 + let cache = Arc::new(Mutex::new(RowGroupCache::new(3, usize::MAX))); // Batch size 3 // Create a producer to populate cache let mut producer = CachedArrayReader::new( @@ -670,6 +731,7 @@ mod tests { cache.clone(), 0, CacheRole::Producer, + metrics.clone(), ); // Populate cache with first batch (1, 2, 3) @@ -677,8 +739,13 @@ mod tests { producer.consume_batch().unwrap(); // Now create a consumer that will try to read from cache - let mut consumer = - CachedArrayReader::new(Box::new(mock_reader), cache.clone(), 0, CacheRole::Consumer); + let mut consumer = CachedArrayReader::new( + Box::new(mock_reader), + cache.clone(), + 0, + CacheRole::Consumer, + metrics, + ); // - We want to read 4 records starting from position 0 // - First 3 records (positions 0-2) should come from cache (batch 0) diff --git a/parquet/src/arrow/array_reader/list_array.rs b/parquet/src/arrow/array_reader/list_array.rs index 66c4f30b3c29..e28c93cf624d 100644 --- a/parquet/src/arrow/array_reader/list_array.rs +++ b/parquet/src/arrow/array_reader/list_array.rs @@ -249,6 +249,7 @@ mod tests { use crate::arrow::array_reader::list_array::ListArrayReader; use crate::arrow::array_reader::test_util::InMemoryArrayReader; use crate::arrow::array_reader::ArrayReaderBuilder; + use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; use crate::arrow::{parquet_to_arrow_schema, ArrowWriter, ProjectionMask}; use crate::file::properties::WriterProperties; @@ -563,7 +564,8 @@ mod tests { ) .unwrap(); - let mut array_reader = ArrayReaderBuilder::new(&file_reader) + let metrics = ArrowReaderMetrics::disabled(); + let mut array_reader = ArrayReaderBuilder::new(&file_reader, &metrics) .build_array_reader(fields.as_ref(), &mask) .unwrap(); diff --git a/parquet/src/arrow/array_reader/row_group_cache.rs b/parquet/src/arrow/array_reader/row_group_cache.rs index 583ad736bca9..ef726e16495f 100644 --- a/parquet/src/arrow/array_reader/row_group_cache.rs +++ b/parquet/src/arrow/array_reader/row_group_cache.rs @@ -66,15 +66,15 @@ pub struct RowGroupCache { cache: HashMap, /// Cache granularity batch_size: usize, - /// Maximum cache size in bytes (None means unlimited) - max_cache_bytes: Option, + /// Maximum cache size in bytes + max_cache_bytes: usize, /// Current cache size in bytes current_cache_size: usize, } impl RowGroupCache { /// Creates a new empty row group cache - pub fn new(batch_size: usize, max_cache_bytes: Option) -> Self { + pub fn new(batch_size: usize, max_cache_bytes: usize) -> Self { Self { cache: HashMap::new(), batch_size, @@ -89,10 +89,8 @@ impl RowGroupCache { let array_size = get_array_memory_size_for_cache(&array); // Check if adding this array would exceed the cache size limit - if let Some(max_size) = self.max_cache_bytes { - if self.current_cache_size + array_size > max_size { - return false; // Cache is full, don't insert - } + if self.current_cache_size + array_size > self.max_cache_bytes { + return false; // Cache is full, don't insert } let key = CacheKey { @@ -145,7 +143,7 @@ mod tests { #[test] fn test_cache_basic_operations() { - let mut cache = RowGroupCache::new(1000, None); + let mut cache = RowGroupCache::new(1000, usize::MAX); // Create test array let array: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); @@ -168,7 +166,7 @@ mod tests { #[test] fn test_cache_remove() { - let mut cache = RowGroupCache::new(1000, None); + let mut cache = RowGroupCache::new(1000, usize::MAX); // Create test arrays let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); diff --git a/parquet/src/arrow/arrow_reader/metrics.rs b/parquet/src/arrow/arrow_reader/metrics.rs new file mode 100644 index 000000000000..05c7a5180193 --- /dev/null +++ b/parquet/src/arrow/arrow_reader/metrics.rs @@ -0,0 +1,135 @@ +// 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. + +//! [ArrowReaderMetrics] for collecting metrics about the Arrow reader + +use std::sync::atomic::AtomicUsize; +use std::sync::Arc; + +/// This enum represents the state of Arrow reader metrics collection. +/// +/// The inner metrics are stored in an `Arc` +/// so cloning the `ArrowReaderMetrics` enum will not clone the inner metrics. +/// +/// To access metrics, create an `ArrowReaderMetrics` via [`ArrowReaderMetrics::enabled()`] +/// and configure the `ArrowReaderBuilder` with a clone. +#[derive(Debug, Clone)] +pub enum ArrowReaderMetrics { + /// Metrics are not collected (default) + Disabled, + /// Metrics are collected and stored in an `Arc`. + /// + /// Create this via [`ArrowReaderMetrics::enabled()`]. + Enabled(Arc), +} + +impl ArrowReaderMetrics { + /// Creates a new instance of [`ArrowReaderMetrics::Disabled`] + pub fn disabled() -> Self { + Self::Disabled + } + + /// Creates a new instance of [`ArrowReaderMetrics::Enabled`] + pub fn enabled() -> Self { + Self::Enabled(Arc::new(ArrowReaderMetricsInner::new())) + } + + /// Predicate Cache: number of records read directly from the inner reader + /// + /// This is the total number of records read from the inner reader (that is + /// actually decoding). It measures the amount of work that could not be + /// avoided with caching. + /// + /// It returns the number of records read across all columns, so if you read + /// 2 columns each with 100 records, this will return 200. + /// + /// + /// Returns None if metrics are disabled. + pub fn records_read_from_inner(&self) -> Option { + match self { + Self::Disabled => None, + Self::Enabled(inner) => Some( + inner + .records_read_from_inner + .load(std::sync::atomic::Ordering::Relaxed), + ), + } + } + + /// Predicate Cache: number of records read from the cache + /// + /// This is the total number of records read from the cache actually + /// decoding). It measures the amount of work that was avoided with caching. + /// + /// It returns the number of records read across all columns, so if you read + /// 2 columns each with 100 records from the cache, this will return 200. + /// + /// Returns None if metrics are disabled. + pub fn records_read_from_cache(&self) -> Option { + match self { + Self::Disabled => None, + Self::Enabled(inner) => Some( + inner + .records_read_from_cache + .load(std::sync::atomic::Ordering::Relaxed), + ), + } + } + + /// Increments the count of records read from the inner reader + pub(crate) fn increment_inner_reads(&self, count: usize) { + let Self::Enabled(inner) = self else { + return; + }; + inner + .records_read_from_inner + .fetch_add(count, std::sync::atomic::Ordering::Relaxed); + } + + /// Increments the count of records read from the cache + pub(crate) fn increment_cache_reads(&self, count: usize) { + let Self::Enabled(inner) = self else { + return; + }; + + inner + .records_read_from_cache + .fetch_add(count, std::sync::atomic::Ordering::Relaxed); + } +} + +/// Holds the actual metrics for the Arrow reader. +/// +/// Please see [`ArrowReaderMetrics`] for the public interface. +#[derive(Debug)] +pub struct ArrowReaderMetricsInner { + // Metrics for Predicate Cache + /// Total number of records read from the inner reader (uncached) + records_read_from_inner: AtomicUsize, + /// Total number of records read from previously cached pages + records_read_from_cache: AtomicUsize, +} + +impl ArrowReaderMetricsInner { + /// Creates a new instance of `ArrowReaderMetricsInner` + pub(crate) fn new() -> Self { + Self { + records_read_from_inner: AtomicUsize::new(0), + records_read_from_cache: AtomicUsize::new(0), + } + } +} diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index ed390ed7fd9e..3a815f3e34d6 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -38,9 +38,11 @@ use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; +use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; pub(crate) use read_plan::{ReadPlan, ReadPlanBuilder}; mod filter; +pub mod metrics; mod read_plan; mod selection; pub mod statistics; @@ -112,6 +114,10 @@ pub struct ArrowReaderBuilder { pub(crate) limit: Option, pub(crate) offset: Option, + + pub(crate) metrics: ArrowReaderMetrics, + + pub(crate) max_predicate_cache_size: usize, } impl Debug for ArrowReaderBuilder { @@ -128,6 +134,7 @@ impl Debug for ArrowReaderBuilder { .field("selection", &self.selection) .field("limit", &self.limit) .field("offset", &self.offset) + .field("metrics", &self.metrics) .finish() } } @@ -146,6 +153,8 @@ impl ArrowReaderBuilder { selection: None, limit: None, offset: None, + metrics: ArrowReaderMetrics::Disabled, + max_predicate_cache_size: 100 * 1024 * 1024, // 100MB default cache size } } @@ -296,6 +305,59 @@ impl ArrowReaderBuilder { ..self } } + + /// Specify metrics collection during reading + /// + /// To access the metrics, create an [`ArrowReaderMetrics`] and pass a + /// clone of the provided metrics to the builder. + /// + /// For example: + /// + /// ```rust + /// # use std::sync::Arc; + /// # use bytes::Bytes; + /// # use arrow_array::{Int32Array, RecordBatch}; + /// # use arrow_schema::{DataType, Field, Schema}; + /// # use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder}; + /// use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; + /// # use parquet::arrow::ArrowWriter; + /// # let mut file: Vec = Vec::with_capacity(1024); + /// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)])); + /// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap(); + /// # let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]).unwrap(); + /// # writer.write(&batch).unwrap(); + /// # writer.close().unwrap(); + /// # let file = Bytes::from(file); + /// // Create metrics object to pass into the reader + /// let metrics = ArrowReaderMetrics::enabled(); + /// let reader = ParquetRecordBatchReaderBuilder::try_new(file).unwrap() + /// // Configure the builder to use the metrics by passing a clone + /// .with_metrics(metrics.clone()) + /// // Build the reader + /// .build().unwrap(); + /// // .. read data from the reader .. + /// + /// // check the metrics + /// assert!(metrics.records_read_from_inner().is_some()); + /// ``` + pub fn with_metrics(self, metrics: ArrowReaderMetrics) -> Self { + Self { metrics, ..self } + } + + /// Set the maximum size of the predicate cache in bytes. + /// + /// Defaults to 100MB + /// + /// This cache is used to store decoded arrays that are used in + /// predicate evaluation ([`Self::with_row_filter`]). + /// + /// Set to `usize::MAX` to use unlimited cache size. + pub fn with_max_predicate_cache_size(self, max_predicate_cache_size: usize) -> Self { + Self { + max_predicate_cache_size, + ..self + } + } } /// Options that control how metadata is read for a parquet file @@ -707,23 +769,37 @@ impl ParquetRecordBatchReaderBuilder { /// /// Note: this will eagerly evaluate any `RowFilter` before returning pub fn build(self) -> Result { + let Self { + input, + metadata, + schema: _, + fields, + batch_size: _, + row_groups, + projection, + mut filter, + selection, + limit, + offset, + metrics, + // TODO: need to implement this for the sync reader + max_predicate_cache_size: _, + } = self; + // Try to avoid allocate large buffer let batch_size = self .batch_size - .min(self.metadata.file_metadata().num_rows() as usize); + .min(metadata.file_metadata().num_rows() as usize); - let row_groups = self - .row_groups - .unwrap_or_else(|| (0..self.metadata.num_row_groups()).collect()); + let row_groups = row_groups.unwrap_or_else(|| (0..metadata.num_row_groups()).collect()); let reader = ReaderRowGroups { - reader: Arc::new(self.input.0), - metadata: self.metadata, + reader: Arc::new(input.0), + metadata, row_groups, }; - let mut filter = self.filter; - let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(self.selection); + let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); // Update selection based on any filters if let Some(filter) = filter.as_mut() { @@ -734,22 +810,22 @@ impl ParquetRecordBatchReaderBuilder { } let mut cache_projection = predicate.projection().clone(); - cache_projection.intersect(&self.projection); + cache_projection.intersect(&projection); - let array_reader = ArrayReaderBuilder::new(&reader) - .build_array_reader(self.fields.as_deref(), predicate.projection())?; + let array_reader = ArrayReaderBuilder::new(&reader, &metrics) + .build_array_reader(fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } } - let array_reader = ArrayReaderBuilder::new(&reader) - .build_array_reader(self.fields.as_deref(), &self.projection)?; + let array_reader = ArrayReaderBuilder::new(&reader, &metrics) + .build_array_reader(fields.as_deref(), &projection)?; let read_plan = plan_builder .limited(reader.num_rows()) - .with_offset(self.offset) - .with_limit(self.limit) + .with_offset(offset) + .with_limit(limit) .build_limited() .build(); @@ -943,8 +1019,9 @@ impl ParquetRecordBatchReader { row_groups: &dyn RowGroups, batch_size: usize, selection: Option, + metrics: &ArrowReaderMetrics, ) -> Result { - let array_reader = ArrayReaderBuilder::new(row_groups) + let array_reader = ArrayReaderBuilder::new(row_groups, metrics) .build_array_reader(levels.levels.as_ref(), &ProjectionMask::all())?; let read_plan = ReadPlanBuilder::new(batch_size) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 356e22899755..843ad766e9a7 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -63,6 +63,7 @@ pub use metadata::*; #[cfg(feature = "object_store")] mod store; +use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; use crate::arrow::arrow_reader::ReadPlanBuilder; use crate::arrow::schema::ParquetField; #[cfg(feature = "object_store")] @@ -512,6 +513,8 @@ impl ParquetRecordBatchStreamBuilder { fields: self.fields, limit: self.limit, offset: self.offset, + metrics: self.metrics, + max_predicate_cache_size: self.max_predicate_cache_size, }; // Ensure schema of ParquetRecordBatchStream respects projection, and does @@ -562,6 +565,12 @@ struct ReaderFactory { /// Offset to apply to the next offset: Option, + + /// Metrics + metrics: ArrowReaderMetrics, + + /// Maximum size of the predicate cache + max_predicate_cache_size: usize, } impl ReaderFactory @@ -597,8 +606,7 @@ where }; let row_group_cache = Arc::new(Mutex::new(RowGroupCache::new( batch_size, - // None, - Some(1024 * 1024 * 100), + self.max_predicate_cache_size, ))); let mut row_group = InMemoryRowGroup { @@ -636,7 +644,7 @@ where ) .await?; - let array_reader = ArrayReaderBuilder::new(&row_group) + let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) .with_cache_options(Some(&cache_options)) .build_array_reader(self.fields.as_deref(), predicate.projection())?; @@ -691,7 +699,7 @@ where let plan = plan_builder.build(); let cache_options = cache_options_builder.consumer(); - let array_reader = ArrayReaderBuilder::new(&row_group) + let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) .with_cache_options(Some(&cache_options)) .build_array_reader(self.fields.as_deref(), &projection)?; @@ -1928,6 +1936,8 @@ mod tests { filter: None, limit: None, offset: None, + metrics: ArrowReaderMetrics::disabled(), + max_predicate_cache_size: 0, }; let mut skip = true; diff --git a/parquet/tests/arrow_reader/mod.rs b/parquet/tests/arrow_reader/mod.rs index 7d001a9832fa..beaa82e1c825 100644 --- a/parquet/tests/arrow_reader/mod.rs +++ b/parquet/tests/arrow_reader/mod.rs @@ -38,9 +38,10 @@ use std::sync::Arc; use tempfile::NamedTempFile; mod bad_data; -mod predicate_cache; #[cfg(feature = "crc")] mod checksum; +#[cfg(feature = "async")] +mod predicate_cache; mod statistics; // returns a struct array with columns "int32_col", "float32_col" and "float64_col" with the specified values diff --git a/parquet/tests/arrow_reader/predicate_cache.rs b/parquet/tests/arrow_reader/predicate_cache.rs index be1a1c811b41..b167ef6343a2 100644 --- a/parquet/tests/arrow_reader/predicate_cache.rs +++ b/parquet/tests/arrow_reader/predicate_cache.rs @@ -17,42 +17,71 @@ //! Test for predicate cache in Parquet Arrow reader - - use arrow::array::ArrayRef; -use std::sync::Arc; -use parquet::arrow::arrow_reader::ArrowReaderOptions; -use std::sync::LazyLock; use arrow::array::Int64Array; -use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; -use bytes::Bytes; +use arrow::compute::and; +use arrow::compute::kernels::cmp::{gt, lt}; +use arrow_array::cast::AsArray; +use arrow_array::types::Int64Type; use arrow_array::{RecordBatch, StringViewArray}; -use parquet::arrow::ArrowWriter; +use bytes::Bytes; +use futures::future::BoxFuture; +use futures::{FutureExt, StreamExt}; +use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; +use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::arrow_reader::{ArrowReaderBuilder, ParquetRecordBatchReaderBuilder}; +use parquet::arrow::async_reader::AsyncFileReader; +use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; use parquet::file::properties::WriterProperties; +use std::ops::Range; +use std::sync::Arc; +use std::sync::LazyLock; +// TODO file a ticket about the duplication here -// 1. the predicate cache is not used when there are no filters -#[test] -fn test() { - let test = ParquetPredicateCacheTest::new() - .with_expected_cache_used(false); - let builder = test.sync_builder(ArrowReaderOptions::default()); - test.run(builder); +#[tokio::test] +async fn test_default_read() { + // The cache is not used without predicates, so we expect 0 records read from cache + let test = ParquetPredicateCacheTest::new().with_expected_records_read_from_cache(0); + let sync_builder = test.sync_builder(); + test.run_sync(sync_builder); + let async_builder = test.async_builder().await; + test.run_async(async_builder).await; } +#[tokio::test] +async fn test_cache_with_filters() { + let test = ParquetPredicateCacheTest::new().with_expected_records_read_from_cache(49); + // TODO The sync reader does not use the cache yet.... + // let sync_builder = test.sync_builder(); + // let sync_builder = test.add_project_ab_and_filter_b(sync_builder); + // test.run_sync(sync_builder); -// Test: -// 2. the predicate cache is used when there are filters but the cache size is 0 -// 3. the predicate cache is used when there are filters and the cache size is greater than 0 - + let async_builder = test.async_builder().await; + let async_builder = test.add_project_ab_and_filter_b(async_builder); + test.run_async(async_builder).await; +} +#[tokio::test] +async fn test_cache_disabled_with_filters() { + // expect no records to be read from cache, because the cache is disabled + let test = ParquetPredicateCacheTest::new().with_expected_records_read_from_cache(0); + let sync_builder = test.sync_builder().with_max_predicate_cache_size(0); + let sync_builder = test.add_project_ab_and_filter_b(sync_builder); + test.run_sync(sync_builder); + let async_builder = test.async_builder().await.with_max_predicate_cache_size(0); + let async_builder = test.add_project_ab_and_filter_b(async_builder); + test.run_async(async_builder).await; +} +// -- Begin test infrastructure -- /// A test parquet file struct ParquetPredicateCacheTest { bytes: Bytes, - expected_cache_used: bool, + expected_records_read_from_cache: usize, } impl ParquetPredicateCacheTest { /// Create a new `TestParquetFile` with: @@ -64,44 +93,107 @@ impl ParquetPredicateCacheTest { /// Values of column "a" are 0..399 /// Values of column "b" are 400..799 /// Values of column "c" are alternating strings of length 12 and longer - fn new() -> Self { + fn new() -> Self { Self { bytes: TEST_FILE_DATA.clone(), - expected_cache_used: false, + expected_records_read_from_cache: 0, } } - /// Set whether the predicate cache is expected to be used - fn with_expected_cache_used(mut self, used: bool) -> Self{ - self.expected_cache_used = used; + /// Set the expected number of records read from the cache + fn with_expected_records_read_from_cache( + mut self, + expected_records_read_from_cache: usize, + ) -> Self { + self.expected_records_read_from_cache = expected_records_read_from_cache; self } /// Return a [`ParquetRecordBatchReaderBuilder`] for reading this file - fn sync_builder( - &self, - options: ArrowReaderOptions, - ) -> ParquetRecordBatchReaderBuilder { + fn sync_builder(&self) -> ParquetRecordBatchReaderBuilder { let reader = self.bytes.clone(); - ParquetRecordBatchReaderBuilder::try_new_with_options(reader, options) + ParquetRecordBatchReaderBuilder::try_new_with_options(reader, ArrowReaderOptions::default()) .expect("ParquetRecordBatchReaderBuilder") } + /// Return a [`ParquetRecordBatchReaderBuilder`] for reading this file + async fn async_builder(&self) -> ParquetRecordBatchStreamBuilder { + let reader = TestReader::new(self.bytes.clone()); + ParquetRecordBatchStreamBuilder::new_with_options(reader, ArrowReaderOptions::default()) + .await + .unwrap() + } + + /// Return a [`ParquetRecordBatchReaderBuilder`] for reading the file with + /// + /// 1. a projection selecting the "a" and "b" column + /// 2. a row_filter applied to "b": 575 < "b" < 625 (select 1 data page from each row group) + fn add_project_ab_and_filter_b( + &self, + builder: ArrowReaderBuilder, + ) -> ArrowReaderBuilder { + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + // "b" > 575 and "b" < 625 + let row_filter = ArrowPredicateFn::new( + ProjectionMask::columns(&schema_descr, ["b"]), + |batch: RecordBatch| { + let scalar_575 = Int64Array::new_scalar(575); + let scalar_625 = Int64Array::new_scalar(625); + let column = batch.column(0).as_primitive::(); + and(>(column, &scalar_575)?, <(column, &scalar_625)?) + }, + ); + + builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a", "b"])) + .with_row_filter(RowFilter::new(vec![Box::new(row_filter)])) + } /// Build the reader from the specified builder, reading all batches from it, /// and asserts the - fn run( - &self, - builder: ParquetRecordBatchReaderBuilder, - ) { - let reader = builder.build().unwrap(); + fn run_sync(&self, builder: ParquetRecordBatchReaderBuilder) { + let metrics = ArrowReaderMetrics::enabled(); + + let reader = builder.with_metrics(metrics.clone()).build().unwrap(); for batch in reader { match batch { Ok(_) => {} Err(e) => panic!("Error reading batch: {e}"), } } - // TODO check if the cache was used + self.verify_metrics(metrics) + } + + /// Build the reader from the specified builder, reading all batches from it, + /// and asserts the + async fn run_async(&self, builder: ParquetRecordBatchStreamBuilder) { + let metrics = ArrowReaderMetrics::enabled(); + + let mut stream = builder.with_metrics(metrics.clone()).build().unwrap(); + while let Some(batch) = stream.next().await { + match batch { + Ok(_) => {} + Err(e) => panic!("Error reading batch: {e}"), + } + } + self.verify_metrics(metrics) + } + + fn verify_metrics(&self, metrics: ArrowReaderMetrics) { + let Self { + bytes: _, + expected_records_read_from_cache, + } = self; + + let read_from_cache = metrics + .records_read_from_cache() + .expect("Metrics enabled, so should have metrics"); + + assert_eq!( + &read_from_cache, expected_records_read_from_cache, + "Expected {expected_records_read_from_cache} records read from cache, but got {read_from_cache}" + ); } } @@ -142,3 +234,42 @@ static TEST_FILE_DATA: LazyLock = LazyLock::new(|| { writer.close().unwrap(); Bytes::from(output) }); + +/// Copy paste version of the `AsyncFileReader` trait for testing purposes 🤮 +/// TODO put this in a common place +#[derive(Clone)] +struct TestReader { + data: Bytes, + metadata: Option>, +} + +impl TestReader { + fn new(data: Bytes) -> Self { + Self { + data, + metadata: Default::default(), + } + } +} + +impl AsyncFileReader for TestReader { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + let range = range.clone(); + futures::future::ready(Ok(self + .data + .slice(range.start as usize..range.end as usize))) + .boxed() + } + + fn get_metadata<'a>( + &'a mut self, + options: Option<&'a ArrowReaderOptions>, + ) -> BoxFuture<'a, parquet::errors::Result>> { + let metadata_reader = + ParquetMetaDataReader::new().with_page_indexes(options.is_some_and(|o| o.page_index())); + self.metadata = Some(Arc::new( + metadata_reader.parse_and_finish(&self.data).unwrap(), + )); + futures::future::ready(Ok(self.metadata.clone().unwrap().clone())).boxed() + } +} From 42d5520f1baba701dddde6d8f7464acc40c7c3c5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Jul 2025 16:05:59 -0400 Subject: [PATCH 29/38] Update parquet/src/arrow/arrow_reader/mod.rs Co-authored-by: Xiangpeng Hao --- parquet/src/arrow/arrow_reader/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 3a815f3e34d6..19ae1579fc7d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -344,7 +344,7 @@ impl ArrowReaderBuilder { Self { metrics, ..self } } - /// Set the maximum size of the predicate cache in bytes. + /// Set the maximum size (per row group) of the predicate cache in bytes. /// /// Defaults to 100MB /// From f70e46af43d17d0dfa2e3d44a8561352cebe0fff Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 25 Jul 2025 12:43:19 -0400 Subject: [PATCH 30/38] Clarify in documentation that cache is only for async decoder --- parquet/src/arrow/arrow_reader/mod.rs | 12 ++++++++--- parquet/tests/arrow_reader/predicate_cache.rs | 20 +++++++++++-------- 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 19ae1579fc7d..2afaffdd473d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -344,14 +344,20 @@ impl ArrowReaderBuilder { Self { metrics, ..self } } - /// Set the maximum size (per row group) of the predicate cache in bytes. + /// Set the maximum size (per row group) of the predicate cache in bytes for + /// the async decoder. /// - /// Defaults to 100MB + /// Defaults to 100MB (across all columns). Set to `usize::MAX` to use + /// unlimited cache size. /// /// This cache is used to store decoded arrays that are used in /// predicate evaluation ([`Self::with_row_filter`]). /// - /// Set to `usize::MAX` to use unlimited cache size. + /// This cache is only used for the "async" decoder, [`ParquetRecordBatchStream`]. See + /// [this ticket] for more details and alternatives. + /// + /// [`ParquetRecordBatchStream`]: https://docs.rs/parquet/latest/parquet/arrow/async_reader/struct.ParquetRecordBatchStream.html + /// [this ticket]: https://github.com/apache/arrow-rs/issues/8000 pub fn with_max_predicate_cache_size(self, max_predicate_cache_size: usize) -> Self { Self { max_predicate_cache_size, diff --git a/parquet/tests/arrow_reader/predicate_cache.rs b/parquet/tests/arrow_reader/predicate_cache.rs index b167ef6343a2..44d43113cbf5 100644 --- a/parquet/tests/arrow_reader/predicate_cache.rs +++ b/parquet/tests/arrow_reader/predicate_cache.rs @@ -38,8 +38,6 @@ use std::ops::Range; use std::sync::Arc; use std::sync::LazyLock; -// TODO file a ticket about the duplication here - #[tokio::test] async fn test_default_read() { // The cache is not used without predicates, so we expect 0 records read from cache @@ -51,18 +49,24 @@ async fn test_default_read() { } #[tokio::test] -async fn test_cache_with_filters() { +async fn test_async_cache_with_filters() { let test = ParquetPredicateCacheTest::new().with_expected_records_read_from_cache(49); - // TODO The sync reader does not use the cache yet.... - // let sync_builder = test.sync_builder(); - // let sync_builder = test.add_project_ab_and_filter_b(sync_builder); - // test.run_sync(sync_builder); - let async_builder = test.async_builder().await; let async_builder = test.add_project_ab_and_filter_b(async_builder); test.run_async(async_builder).await; } +#[tokio::test] +async fn test_sync_cache_with_filters() { + let test = ParquetPredicateCacheTest::new() + // The sync reader does not use the cache. See https://github.com/apache/arrow-rs/issues/8000 + .with_expected_records_read_from_cache(0); + + let sync_builder = test.sync_builder(); + let sync_builder = test.add_project_ab_and_filter_b(sync_builder); + test.run_sync(sync_builder); +} + #[tokio::test] async fn test_cache_disabled_with_filters() { // expect no records to be read from cache, because the cache is disabled From 15d6826576ef0972ce0639e0bc4b8fe3a48cee79 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 25 Jul 2025 13:09:17 -0400 Subject: [PATCH 31/38] add comment --- parquet/src/arrow/arrow_reader/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 2afaffdd473d..0556f1a7dc1a 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -788,7 +788,7 @@ impl ParquetRecordBatchReaderBuilder { limit, offset, metrics, - // TODO: need to implement this for the sync reader + // Not used for the sync reader, see https://github.com/apache/arrow-rs/issues/8000 max_predicate_cache_size: _, } = self; From bec6d9ce6034e1ebddcbe03b237b272ee59ad6bb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 25 Jul 2025 13:18:35 -0400 Subject: [PATCH 32/38] Revert backwards incompatible changes to the Parquet reader API --- parquet/examples/read_with_rowgroup.rs | 23 +++-------------------- parquet/src/arrow/arrow_reader/mod.rs | 5 +++-- 2 files changed, 6 insertions(+), 22 deletions(-) diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index de33db1de816..5d1ff0770f9e 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -17,7 +17,6 @@ use arrow::util::pretty::print_batches; use bytes::{Buf, Bytes}; -use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, RowGroups, RowSelection}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{parquet_to_arrow_field_levels, ProjectionMask}; @@ -39,11 +38,7 @@ async fn main() -> Result<()> { let metadata = file.get_metadata(None).await?; for rg in metadata.row_groups() { - let mut rowgroup = InMemoryRowGroup::create( - rg.clone(), - ProjectionMask::all(), - ArrowReaderMetrics::disabled(), - ); + let mut rowgroup = InMemoryRowGroup::create(rg.clone(), ProjectionMask::all()); rowgroup.async_fetch_data(&mut file, None).await?; let reader = rowgroup.build_reader(1024, None)?; @@ -108,7 +103,6 @@ pub struct InMemoryRowGroup { pub metadata: RowGroupMetaData, mask: ProjectionMask, column_chunks: Vec>>, - metrics: ArrowReaderMetrics, } impl RowGroups for InMemoryRowGroup { @@ -138,18 +132,13 @@ impl RowGroups for InMemoryRowGroup { } impl InMemoryRowGroup { - pub fn create( - metadata: RowGroupMetaData, - mask: ProjectionMask, - metrics: ArrowReaderMetrics, - ) -> Self { + pub fn create(metadata: RowGroupMetaData, mask: ProjectionMask) -> Self { let column_chunks = metadata.columns().iter().map(|_| None).collect::>(); Self { metadata, mask, column_chunks, - metrics, } } @@ -164,13 +153,7 @@ impl InMemoryRowGroup { None, )?; - ParquetRecordBatchReader::try_new_with_row_groups( - &levels, - self, - batch_size, - selection, - &self.metrics, - ) + ParquetRecordBatchReader::try_new_with_row_groups(&levels, self, batch_size, selection) } /// fetch data from a reader in sync mode diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 19ae1579fc7d..ab1e6d9a902d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1019,9 +1019,10 @@ impl ParquetRecordBatchReader { row_groups: &dyn RowGroups, batch_size: usize, selection: Option, - metrics: &ArrowReaderMetrics, ) -> Result { - let array_reader = ArrayReaderBuilder::new(row_groups, metrics) + // note metrics are not supported in this API + let metrics = ArrowReaderMetrics::disabled(); + let array_reader = ArrayReaderBuilder::new(row_groups, &metrics) .build_array_reader(levels.levels.as_ref(), &ProjectionMask::all())?; let read_plan = ReadPlanBuilder::new(batch_size) From cb891021d87bc3215b59ca06ffeaeba5508778e2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 21 Jul 2025 10:05:12 -0400 Subject: [PATCH 33/38] Add Parquet IO test --- parquet/src/file/reader.rs | 9 +- parquet/tests/arrow_reader/io/async_reader.rs | 434 +++++++++++ parquet/tests/arrow_reader/io/mod.rs | 712 ++++++++++++++++++ parquet/tests/arrow_reader/io/sync_reader.rs | 454 +++++++++++ parquet/tests/arrow_reader/mod.rs | 1 + 5 files changed, 1606 insertions(+), 4 deletions(-) create mode 100644 parquet/tests/arrow_reader/io/async_reader.rs create mode 100644 parquet/tests/arrow_reader/io/mod.rs create mode 100644 parquet/tests/arrow_reader/io/sync_reader.rs diff --git a/parquet/src/file/reader.rs b/parquet/src/file/reader.rs index 400441f0c9cd..e6d2ac494d09 100644 --- a/parquet/src/file/reader.rs +++ b/parquet/src/file/reader.rs @@ -48,11 +48,12 @@ pub trait Length { /// Generates [`Read`]ers to read chunks of a Parquet data source. /// /// The Parquet reader uses [`ChunkReader`] to access Parquet data, allowing -/// multiple decoders to read concurrently from different locations in the same file. +/// multiple decoders to read concurrently from different locations in the same +/// file. /// -/// The trait provides: -/// * random access (via [`Self::get_bytes`]) -/// * sequential (via [`Self::get_read`]) +/// The trait functions both as a reader and a factory for readers. +/// * random access via [`Self::get_bytes`] +/// * sequential access via the reader returned via factory method [`Self::get_read`] /// /// # Provided Implementations /// * [`File`] for reading from local file system diff --git a/parquet/tests/arrow_reader/io/async_reader.rs b/parquet/tests/arrow_reader/io/async_reader.rs new file mode 100644 index 000000000000..5d1051f24f16 --- /dev/null +++ b/parquet/tests/arrow_reader/io/async_reader.rs @@ -0,0 +1,434 @@ +// 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. + +//! Tests for the async reader ([`ParquetRecordBatchStreamBuilder`]) + +use crate::io::{ + filter_a_175_b_625, filter_b_575_625, filter_b_false, test_file, test_options, LogEntry, + OperationLog, TestParquetFile, +}; +use bytes::Bytes; +use futures::future::BoxFuture; +use futures::{FutureExt, StreamExt}; +use parquet::arrow::arrow_reader::{ArrowReaderOptions, RowSelection, RowSelector}; +use parquet::arrow::async_reader::AsyncFileReader; +use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::errors::Result; +use parquet::file::metadata::ParquetMetaData; +use std::ops::Range; +use std::sync::Arc; + +#[tokio::test] +async fn test_read_entire_file() { + // read entire file without any filtering or projection + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 0, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7346 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 1, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7456 bytes, 1 requests) [data]", + ] + ) + .await; +} + +#[tokio::test] +async fn test_read_single_group() { + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()) + .await + // read only second row group + .with_row_groups(vec![1]); + + // Expect to see only IO for Row Group 1. Should see no IO for Row Group 0. + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 1, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7456 bytes, 1 requests) [data]", + ] + ).await; +} + +#[tokio::test] +async fn test_read_single_column() { + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder.with_projection(ProjectionMask::columns(&schema_descr, ["b"])); + // Expect to see only IO for column "b". Should see no IO for columns "a" or "c". + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + ] + ).await; +} + +#[tokio::test] +async fn test_read_row_selection() { + // There are 400 total rows spread across 4 data pages (100 rows each) + // select rows 175..225 (i.e. DataPage(1) of row group 0 and DataPage(0) of row group 1) + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a", "b"])) + .with_row_selection(RowSelection::from(vec![ + RowSelector::skip(175), + RowSelector::select(50), + ])); + + // Expect to see only data IO for one page for each column for each row group + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + " Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + " Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + ], + ) + .await; +} + +#[tokio::test] +async fn test_read_limit() { + // There are 400 total rows spread across 4 data pages (100 rows each) + // a limit of 125 rows should only fetch the first two data pages (DataPage(0) and DataPage(1)) from row group 0 + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a"])) + .with_limit(125); + + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + ], + ) + .await; +} + +#[tokio::test] +async fn test_read_single_row_filter() { + // Values from column "b" range 400..799 + // filter "b" > 575 and < than 625 + // (last data page in Row Group 0 and first DataPage in Row Group 1) + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a", "b"])) + .with_row_filter(filter_b_575_625(&schema_descr)); + + // Expect to see I/O for column b in both row groups to evaluate filter, + // then a single pages for the "a" column in each row group + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + ] + ).await; +} + +#[tokio::test] +async fn test_read_single_row_filter_no_page_index() { + // Values from column "b" range 400..799 + // Apply a filter "b" > 575 and than 625 + // (last data page in Row Group 0 and first DataPage in Row Group 1) + let test_file = test_file(); + let options = test_options().with_page_index(false); + let builder = async_builder(&test_file, options).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a", "b"])) + .with_row_filter(filter_b_575_625(&schema_descr)); + + // Since we don't have the page index, expect to see: + // 1. I/O for all pages of column b to evaluate the filter + // 2. IO for all pages of column a as the reader doesn't know where the page + // boundaries are so needs to scan them. + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 0, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + ] + ).await; +} + +#[tokio::test] +async fn test_read_multiple_row_filter() { + // Values in column "a" range 0..399 + // Values in column "b" range 400..799 + // First filter: "a" > 175 (last data page in Row Group 0) + // Second filter: "b" < 625 (last data page in Row Group 0 and first DataPage in RowGroup 1) + // Read column "c" + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["c"])) + .with_row_filter(filter_a_175_b_625(&schema_descr)); + + // Expect that we will see + // 1. IO for all pages of column A (to evaluate the first filter) + // 2. IO for pages of column b that passed the first filter (to evaluate the second filter) + // 3. IO after reader is built only for column c for the rows that passed both filters + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Read Multi:", + " Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", + " Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + " Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + " Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + ] + ).await +} + +#[tokio::test] +async fn test_read_single_row_filter_all() { + // Apply a filter that filters out all rows + + let test_file = test_file(); + let builder = async_builder(&test_file, test_options()).await; + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a", "b"])) + .with_row_filter(filter_b_false(&schema_descr)); + + // Expect to see reads for column "b" to evaluate the filter, but no reads + // for column "a" as no rows pass the filter + run_test( + &test_file, + builder, + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + ] + ).await; +} + +/// Return a [`ParquetRecordBatchStreamBuilder`] for reading this file +async fn async_builder( + test_file: &TestParquetFile, + options: ArrowReaderOptions, +) -> ParquetRecordBatchStreamBuilder { + let parquet_meta_data = if options.page_index() { + Arc::clone(test_file.parquet_metadata()) + } else { + // strip out the page index from the metadata + let metadata = test_file + .parquet_metadata() + .as_ref() + .clone() + .into_builder() + .set_column_index(None) + .set_offset_index(None) + .build(); + Arc::new(metadata) + }; + + let reader = RecordingAsyncFileReader { + bytes: test_file.bytes().clone(), + ops: Arc::clone(test_file.ops()), + parquet_meta_data, + }; + + ParquetRecordBatchStreamBuilder::new_with_options(reader, options) + .await + .unwrap() +} + +/// Build the reader from the specified builder and read all batches from it, +/// and assert that the operations log contains the expected entries. +async fn run_test<'a>( + test_file: &TestParquetFile, + builder: ParquetRecordBatchStreamBuilder, + expected: impl IntoIterator, +) { + let ops = test_file.ops(); + ops.add_entry(LogEntry::event("Builder Configured")); + let mut stream = builder.build().unwrap(); + ops.add_entry(LogEntry::event("Reader Built")); + while let Some(batch) = stream.next().await { + match batch { + Ok(_) => {} + Err(e) => panic!("Error reading batch: {e}"), + } + } + ops.assert(expected) +} + +struct RecordingAsyncFileReader { + bytes: Bytes, + ops: Arc, + parquet_meta_data: Arc, +} + +impl AsyncFileReader for RecordingAsyncFileReader { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + let ops = Arc::clone(&self.ops); + let data = self + .bytes + .slice(range.start as usize..range.end as usize) + .clone(); + + // translate to usize from u64 + let logged_range = Range { + start: range.start as usize, + end: range.end as usize, + }; + async move { + ops.add_entry_for_range(&logged_range); + Ok(data) + } + .boxed() + } + + fn get_byte_ranges(&mut self, ranges: Vec>) -> BoxFuture<'_, Result>> { + let ops = Arc::clone(&self.ops); + let datas = ranges + .iter() + .map(|range| { + self.bytes + .slice(range.start as usize..range.end as usize) + .clone() + }) + .collect::>(); + // translate to usize from u64 + let logged_ranges = ranges + .into_iter() + .map(|r| Range { + start: r.start as usize, + end: r.end as usize, + }) + .collect::>(); + + async move { + ops.add_entry_for_ranges(&logged_ranges); + Ok(datas) + } + .boxed() + } + + fn get_metadata<'a>( + &'a mut self, + _options: Option<&'a ArrowReaderOptions>, + ) -> BoxFuture<'a, Result>> { + let ops = Arc::clone(&self.ops); + let parquet_meta_data = Arc::clone(&self.parquet_meta_data); + async move { + ops.add_entry(LogEntry::GetProvidedMetadata); + Ok(parquet_meta_data) + } + .boxed() + } +} diff --git a/parquet/tests/arrow_reader/io/mod.rs b/parquet/tests/arrow_reader/io/mod.rs new file mode 100644 index 000000000000..dab6cb87aa26 --- /dev/null +++ b/parquet/tests/arrow_reader/io/mod.rs @@ -0,0 +1,712 @@ +// 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. + +//! Tests for IO read patterns in the Parquet Reader +//! +//! Each test: +//! 1. Creates a temporary Parquet file with a known row group structure +//! 2. Reads data from that file using the Arrow Parquet Reader, recording the IO operations +//! 3. Asserts the expected IO patterns based on the read operations +//! +//! Note this module contains test infrastructure only. The actual tests are in the +//! sub-modules [`sync_reader`] and [`async_reader`]. +//! +//! Key components: +//! - [`TestParquetFile`] - Represents a Parquet file and its layout +//! - [`OperationLog`] - Records IO operations performed on the file +//! - [`LogEntry`] - Represents a single IO operation in the log + +mod sync_reader; + +#[cfg(feature = "async")] +mod async_reader; + +use arrow::compute::and; +use arrow::compute::kernels::cmp::{gt, lt}; +use arrow_array::cast::AsArray; +use arrow_array::types::Int64Type; +use arrow_array::{ArrayRef, BooleanArray, Int64Array, RecordBatch, StringViewArray}; +use bytes::Bytes; +use parquet::arrow::arrow_reader::{ + ArrowPredicateFn, ArrowReaderOptions, ParquetRecordBatchReaderBuilder, RowFilter, +}; +use parquet::arrow::{ArrowWriter, ProjectionMask}; +use parquet::data_type::AsBytes; +use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, ParquetOffsetIndex}; +use parquet::file::properties::WriterProperties; +use parquet::file::FOOTER_SIZE; +use parquet::format::PageLocation; +use parquet::schema::types::SchemaDescriptor; +use std::collections::BTreeMap; +use std::fmt::Display; +use std::ops::Range; +use std::sync::{Arc, LazyLock, Mutex}; + +/// Create a new `TestParquetFile` with: +/// 3 columns: "a", "b", "c" +/// +/// 2 row groups, each with 200 rows +/// each data page has 100 rows +/// +/// Values of column "a" are 0..399 +/// Values of column "b" are 400..799 +/// Values of column "c" are alternating strings of length 12 and longer +fn test_file() -> TestParquetFile { + TestParquetFile::new(TEST_FILE_DATA.clone()) +} + +/// Default options for tests +/// +/// Note these tests use the PageIndex to reduce IO +fn test_options() -> ArrowReaderOptions { + ArrowReaderOptions::default().with_page_index(true) +} + +/// Return a row filter that evaluates "b > 575" AND "b < 625" +/// +/// last data page in Row Group 0 and first DataPage in Row Group 1 +fn filter_b_575_625(schema_descr: &SchemaDescriptor) -> RowFilter { + // "b" > 575 and "b" < 625 + let predicate = ArrowPredicateFn::new( + ProjectionMask::columns(schema_descr, ["b"]), + |batch: RecordBatch| { + let scalar_575 = Int64Array::new_scalar(575); + let scalar_625 = Int64Array::new_scalar(625); + let column = batch.column(0).as_primitive::(); + and(>(column, &scalar_575)?, <(column, &scalar_625)?) + }, + ); + RowFilter::new(vec![Box::new(predicate)]) +} + +/// Filter a > 175 and b < 625 +/// First filter: "a" > 175 (last data page in Row Group 0) +/// Second filter: "b" < 625 (last data page in Row Group 0 and first DataPage in RowGroup 1) +fn filter_a_175_b_625(schema_descr: &SchemaDescriptor) -> RowFilter { + // "a" > 175 and "b" < 625 + let predicate_a = ArrowPredicateFn::new( + ProjectionMask::columns(schema_descr, ["a"]), + |batch: RecordBatch| { + let scalar_175 = Int64Array::new_scalar(175); + let column = batch.column(0).as_primitive::(); + gt(column, &scalar_175) + }, + ); + + let predicate_b = ArrowPredicateFn::new( + ProjectionMask::columns(schema_descr, ["b"]), + |batch: RecordBatch| { + let scalar_625 = Int64Array::new_scalar(625); + let column = batch.column(0).as_primitive::(); + lt(column, &scalar_625) + }, + ); + + RowFilter::new(vec![Box::new(predicate_a), Box::new(predicate_b)]) +} + +/// Filter FALSE (no rows) with b +/// Entirely filters out both row groups +/// Note it selects "b" +fn filter_b_false(schema_descr: &SchemaDescriptor) -> RowFilter { + // "false" + let predicate = ArrowPredicateFn::new( + ProjectionMask::columns(schema_descr, ["b"]), + |batch: RecordBatch| { + let result = + BooleanArray::from_iter(std::iter::repeat_n(Some(false), batch.num_rows())); + Ok(result) + }, + ); + RowFilter::new(vec![Box::new(predicate)]) +} + +/// Create a parquet file in memory for testing. See [`test_file`] for details. +static TEST_FILE_DATA: LazyLock = LazyLock::new(|| { + // Input batch has 400 rows, with 3 columns: "a", "b", "c" + // Note c is a different types (so the data page sizes will be different) + let a: ArrayRef = Arc::new(Int64Array::from_iter_values(0..400)); + let b: ArrayRef = Arc::new(Int64Array::from_iter_values(400..800)); + let c: ArrayRef = Arc::new(StringViewArray::from_iter_values((0..400).map(|i| { + if i % 2 == 0 { + format!("string_{i}") + } else { + format!("A string larger than 12 bytes and thus not inlined {i}") + } + }))); + + let input_batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); + + let mut output = Vec::new(); + + let writer_options = WriterProperties::builder() + .set_max_row_group_size(200) + .set_data_page_row_count_limit(100) + .build(); + let mut writer = + ArrowWriter::try_new(&mut output, input_batch.schema(), Some(writer_options)).unwrap(); + + // since the limits are only enforced on batch boundaries, write the input + // batch in chunks of 50 + let mut row_remain = input_batch.num_rows(); + while row_remain > 0 { + let chunk_size = row_remain.min(50); + let chunk = input_batch.slice(input_batch.num_rows() - row_remain, chunk_size); + writer.write(&chunk).unwrap(); + row_remain -= chunk_size; + } + writer.close().unwrap(); + Bytes::from(output) +}); + +/// A test parquet file and its layout. +struct TestParquetFile { + bytes: Bytes, + /// The operation log for IO operations performed on this file + ops: Arc, + /// The (pre-parsed) parquet metadata for this file + parquet_metadata: Arc, +} + +impl TestParquetFile { + /// Create a new `TestParquetFile` with the specified temporary directory and path + /// and determines the row group layout. + fn new(bytes: Bytes) -> Self { + // Read the parquet file to determine its layout + let builder = ParquetRecordBatchReaderBuilder::try_new_with_options( + bytes.clone(), + ArrowReaderOptions::default().with_page_index(true), + ) + .unwrap(); + + let parquet_metadata = Arc::clone(builder.metadata()); + + let offset_index = parquet_metadata + .offset_index() + .expect("Parquet metadata should have a page index"); + + let row_groups = TestRowGroups::new(&parquet_metadata, offset_index); + + // figure out the footer location in the file + let footer_location = bytes.len() - FOOTER_SIZE..bytes.len(); + let footer = bytes.slice(footer_location.clone()); + let footer: &[u8; FOOTER_SIZE] = footer + .as_bytes() + .try_into() // convert to a fixed size array + .unwrap(); + + // figure out the metadata location + let footer = ParquetMetaDataReader::decode_footer_tail(footer).unwrap(); + let metadata_len = footer.metadata_length(); + let metadata_location = footer_location.start - metadata_len..footer_location.start; + + let ops = Arc::new(OperationLog::new( + footer_location, + metadata_location, + row_groups, + )); + + TestParquetFile { + bytes, + ops, + parquet_metadata, + } + } + + /// Return the internal bytes of the parquet file + fn bytes(&self) -> &Bytes { + &self.bytes + } + + /// Return the operation log for this file + fn ops(&self) -> &Arc { + &self.ops + } + + /// Return the parquet metadata for this file + fn parquet_metadata(&self) -> &Arc { + &self.parquet_metadata + } +} + +/// Information about a column chunk +#[derive(Debug)] +struct TestColumnChunk { + /// The name of the column + name: String, + + /// The location of the entire column chunk in the file including dictionary pages + /// and data pages. + location: Range, + + /// The offset of the start of of the dictionary page if any + dictionary_page_location: Option, + + /// The location of the data pages in the file + page_locations: Vec, +} + +/// Information about the pages in a single row group +#[derive(Debug)] +struct TestRowGroup { + /// Maps column_name -> Information about the column chunk + columns: BTreeMap, +} + +/// Information about all the row groups in a Parquet file, extracted from its metadata +#[derive(Debug)] +struct TestRowGroups { + /// List of row groups, each containing information about its columns and page locations + row_groups: Vec, +} + +impl TestRowGroups { + fn new(parquet_metadata: &ParquetMetaData, offset_index: &ParquetOffsetIndex) -> Self { + let row_groups = parquet_metadata + .row_groups() + .iter() + .enumerate() + .map(|(rg_index, rg_meta)| { + let columns = rg_meta + .columns() + .iter() + .enumerate() + .map(|(col_idx, col_meta)| { + let column_name = col_meta.column_descr().name().to_string(); + let page_locations = + offset_index[rg_index][col_idx].page_locations().to_vec(); + let dictionary_page_location = col_meta.dictionary_page_offset(); + + // We can find the byte range of the entire column chunk + let (start_offset, length) = col_meta.byte_range(); + let start_offset = start_offset as usize; + let end_offset = start_offset + length as usize; + + TestColumnChunk { + name: column_name.clone(), + location: start_offset..end_offset, + dictionary_page_location, + page_locations, + } + }) + .map(|test_column_chunk| { + // make key=value pairs to insert into the BTreeMap + (test_column_chunk.name.clone(), test_column_chunk) + }) + .collect::>(); + TestRowGroup { columns } + }) + .collect(); + + Self { row_groups } + } + + fn iter(&self) -> impl Iterator { + self.row_groups.iter() + } +} + +/// Type of data read +#[derive(Debug, PartialEq)] +enum PageType { + /// The data page with the specified index + Data { + data_page_index: usize, + }, + Dictionary, + /// Multiple pages read together + Multi { + /// Was the dictionary page included? + dictionary_page: bool, + /// The data pages included + data_page_indices: Vec, + }, +} + +impl Display for PageType { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + PageType::Data { data_page_index } => { + write!(f, "DataPage({data_page_index})") + } + PageType::Dictionary => write!(f, "DictionaryPage"), + PageType::Multi { + dictionary_page, + data_page_indices, + } => { + let dictionary_page = if *dictionary_page { + "dictionary_page: true, " + } else { + "" + }; + write!( + f, + "MultiPage({dictionary_page}data_pages: {data_page_indices:?})", + ) + } + } + } +} + +/// Read single logical data object (data page or dictionary page) +/// in one or more requests +#[derive(Debug)] +struct ReadInfo { + row_group_index: usize, + column_name: String, + range: Range, + read_type: PageType, + /// Number of distinct requests (function calls) that were used + num_requests: usize, +} + +impl Display for ReadInfo { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let Self { + row_group_index, + column_name, + range, + read_type, + num_requests, + } = self; + + // If the average read size is less than 10 bytes, assume it is the thrift + // decoder reading the page headers and add an annotation + let annotation = if (range.len() / num_requests) < 10 { + " [header]" + } else { + " [data]" + }; + + // align the read type to 20 characters for better readability, not sure why + // this does not work inline with write! macro below + write!( + f, + "Row Group {row_group_index}, column '{column_name}': {:15} ({:10}, {:8}){annotation}", + // convert to strings so alignment works + format!("{read_type}"), + format!("{} bytes", range.len()), + format!("{num_requests} requests"), + ) + } +} + +/// Store structured entries in the log to make it easier to combine multiple entries +#[derive(Debug)] +enum LogEntry { + /// Read the footer (last 8 bytes) of the parquet file + ReadFooter(Range), + /// Read the metadata of the parquet file + ReadMetadata(Range), + /// Access previously parsed metadata + GetProvidedMetadata, + /// Read a single logical data object + ReadData(ReadInfo), + /// Read one or more logical data objects in a single operation + ReadMultipleData(Vec), + /// Not known where the read came from + Unknown(Range), + /// A user defined event + Event(String), +} + +impl LogEntry { + fn event(event: impl Into) -> Self { + LogEntry::Event(event.into()) + } + + /// Appends a string representation of this log entry to the output vector + fn append_string(&self, output: &mut Vec, indent: usize) { + let indent_str = " ".repeat(indent); + match self { + LogEntry::ReadFooter(range) => { + output.push(format!("{indent_str}Footer: {} bytes", range.len())) + } + LogEntry::ReadMetadata(range) => { + output.push(format!("{indent_str}Metadata: {}", range.len())) + } + LogEntry::GetProvidedMetadata => { + output.push(format!("{indent_str}Get Provided Metadata")) + } + LogEntry::ReadData(read_info) => output.push(format!("{indent_str}{read_info}")), + LogEntry::ReadMultipleData(read_infos) => { + output.push(format!("{indent_str}Read Multi:")); + for read_info in read_infos { + let new_indent = indent + 2; + read_info.append_string(output, new_indent); + } + } + LogEntry::Unknown(range) => { + output.push(format!("{indent_str}UNKNOWN: {range:?} (maybe Page Index)")) + } + LogEntry::Event(event) => output.push(format!("Event: {event}")), + } + } +} + +#[derive(Debug)] +struct OperationLog { + /// The operations performed on the file + ops: Mutex>, + + /// Footer location in the parquet file + footer_location: Range, + + /// Metadata location in the parquet file + metadata_location: Range, + + /// Information about the row group layout in the parquet file, used to + /// translate read operations into human understandable IO operations + /// Path to the parquet file + row_groups: TestRowGroups, +} + +impl OperationLog { + fn new( + footer_location: Range, + metadata_location: Range, + row_groups: TestRowGroups, + ) -> Self { + OperationLog { + ops: Mutex::new(Vec::new()), + metadata_location, + footer_location, + row_groups, + } + } + + /// Add an operation to the log + fn add_entry(&self, entry: LogEntry) { + let mut ops = self.ops.lock().unwrap(); + ops.push(entry); + } + + /// Adds an entry to the operation log for the interesting object that is + /// accessed by the specified range + /// + /// This function checks the ranges in order against possible locations + /// and adds the appropriate operation to the log for the first match found. + fn add_entry_for_range(&self, range: &Range) { + self.add_entry(self.entry_for_range(range)); + } + + /// Adds entries to the operation log for each interesting object that is + /// accessed by the specified range + /// + /// It behaves the same as [`add_entry_for_range`] but for multiple ranges. + fn add_entry_for_ranges<'a>(&self, ranges: impl IntoIterator>) { + let entries = ranges + .into_iter() + .map(|range| self.entry_for_range(range)) + .collect::>(); + self.add_entry(LogEntry::ReadMultipleData(entries)); + } + + /// Create an appropriate LogEntry for the specified range + fn entry_for_range(&self, range: &Range) -> LogEntry { + let start = range.start as i64; + let end = range.end as i64; + + // figure out what logical part of the file this range corresponds to + if self.metadata_location.contains(&range.start) + || self.metadata_location.contains(&(range.end - 1)) + { + return LogEntry::ReadMetadata(range.clone()); + } + + if self.footer_location.contains(&range.start) + || self.footer_location.contains(&(range.end - 1)) + { + return LogEntry::ReadFooter(range.clone()); + } + + // Search for the location in each column chunk. + // + // The actual parquet reader must in general decode the page headers + // and determine the byte ranges of the pages. However, for this test + // we assume the following layout: + // + // ```text + // (Dictionary Page) + // (Data Page) + // ... + // (Data Page) + // ``` + // + // We also assume that `self.page_locations` holds the location of all + // data pages, so any read operation that overlaps with a data page + // location is considered a read of that page, and any other read must + // be a dictionary page read. + for (row_group_index, row_group) in self.row_groups.iter().enumerate() { + for (column_name, test_column_chunk) in &row_group.columns { + // Check if the range overlaps with any data page locations + let page_locations = test_column_chunk.page_locations.iter(); + + // What data pages does this range overlap with? + let mut data_page_indices = vec![]; + + for (data_page_index, page_location) in page_locations.enumerate() { + let page_offset = page_location.offset; + let page_end = page_offset + page_location.compressed_page_size as i64; + + // if the range fully contains the page, consider it a read of that page + if start >= page_offset && end <= page_end { + let read_info = ReadInfo { + row_group_index, + column_name: column_name.clone(), + range: range.clone(), + read_type: PageType::Data { data_page_index }, + num_requests: 1, + }; + return LogEntry::ReadData(read_info); + } + + // if the range overlaps with the page, add it to the list of overlapping pages + if start < page_end && end > page_offset { + data_page_indices.push(data_page_index); + } + } + + // was the dictionary page read? + let mut dictionary_page = false; + + // Check if the range overlaps with the dictionary page location + if let Some(dict_page_offset) = test_column_chunk.dictionary_page_location { + let dict_page_end = dict_page_offset + test_column_chunk.location.len() as i64; + if start >= dict_page_offset && end < dict_page_end { + let read_info = ReadInfo { + row_group_index, + column_name: column_name.clone(), + range: range.clone(), + read_type: PageType::Dictionary, + num_requests: 1, + }; + + return LogEntry::ReadData(read_info); + } + + // if the range overlaps with the dictionary page, add it to the list of overlapping pages + if start < dict_page_end && end > dict_page_offset { + dictionary_page = true; + } + } + + // If we can't find a page, but the range overlaps with the + // column chunk location, use the column chunk location + let column_byte_range = &test_column_chunk.location; + if column_byte_range.contains(&range.start) + && column_byte_range.contains(&(range.end - 1)) + { + let read_data_entry = ReadInfo { + row_group_index, + column_name: column_name.clone(), + range: range.clone(), + read_type: PageType::Multi { + data_page_indices, + dictionary_page, + }, + num_requests: 1, + }; + + return LogEntry::ReadData(read_data_entry); + } + } + } + + // If we reach here, the range does not match any known logical part of the file + LogEntry::Unknown(range.clone()) + } + + // Combine entries in the log that are similar to reduce noise in the log. + fn coalesce_entries(&self) { + let mut ops = self.ops.lock().unwrap(); + + // Coalesce entries with the same read type + let prev_ops = std::mem::take(&mut *ops); + for entry in prev_ops { + let Some(last) = ops.last_mut() else { + ops.push(entry); + continue; + }; + + let LogEntry::ReadData(ReadInfo { + row_group_index: last_rg_index, + column_name: last_column_name, + range: last_range, + read_type: last_read_type, + num_requests: last_num_reads, + }) = last + else { + // If the last entry is not a ReadColumnChunk, just push it + ops.push(entry); + continue; + }; + + // If the entry is not a ReadColumnChunk, just push it + let LogEntry::ReadData(ReadInfo { + row_group_index, + column_name, + range, + read_type, + num_requests: num_reads, + }) = &entry + else { + ops.push(entry); + continue; + }; + + // Combine the entries if they are the same and this read is less than 10b. + // + // This heuristic is used to combine small reads (typically 1-2 + // byte) made by the thrift decoder when reading the data/dictionary + // page headers. + if *row_group_index != *last_rg_index + || column_name != last_column_name + || read_type != last_read_type + || (range.start > last_range.end) + || (range.end < last_range.start) + || range.len() > 10 + { + ops.push(entry); + continue; + } + // combine + *last_range = last_range.start.min(range.start)..last_range.end.max(range.end); + *last_num_reads += num_reads; + } + } + + /// Assert that the operations in the log match the expected operations + /// with an error message that can be copy/pasted to update a test on failure. + fn assert<'a>(&self, expected: impl IntoIterator) { + let expected = expected.into_iter().collect::>(); + + self.coalesce_entries(); + let ops = self.ops.lock().unwrap(); + + let mut actual = vec![]; + let indent = 0; + ops.iter() + .for_each(|s| s.append_string(&mut actual, indent)); + assert_eq!( + // use pretty print for easier diff comparison + format!("{actual:#?}"), + format!("{expected:#?}"), + "Operation log mismatch\n\nactual:\n{actual:#?}\nexpected:\n{expected:#?}" + ); + } +} diff --git a/parquet/tests/arrow_reader/io/sync_reader.rs b/parquet/tests/arrow_reader/io/sync_reader.rs new file mode 100644 index 000000000000..be4b164f0194 --- /dev/null +++ b/parquet/tests/arrow_reader/io/sync_reader.rs @@ -0,0 +1,454 @@ +// 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. + +//! Tests for the sync reader - [`ParquetRecordBatchReaderBuilder`] + +use crate::io::{ + filter_a_175_b_625, filter_b_575_625, filter_b_false, test_file, test_options, LogEntry, + OperationLog, TestParquetFile, +}; + +use bytes::Bytes; +use parquet::arrow::arrow_reader::{ + ArrowReaderOptions, ParquetRecordBatchReaderBuilder, RowSelection, RowSelector, +}; +use parquet::arrow::ProjectionMask; +use parquet::file::reader::{ChunkReader, Length}; +use std::io::Read; +use std::sync::Arc; + +#[test] +fn test_read_entire_file() { + // read entire file without any filtering or projection + let test_file = test_file(); + // Expect to see IO for all data pages for each row group and column + let builder = sync_builder(&test_file, test_options()); + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", + "Row Group 0, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_single_group() { + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()).with_row_groups(vec![1]); // read only second row group + + // Expect to see only IO for Row Group 1. Should see no IO for Row Group 0. + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_single_column() { + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder.with_projection(ProjectionMask::columns(&schema_descr, ["b"])); + // Expect to see only IO for column "b". Should see no IO for columns "a" or "c". + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_single_column_no_page_index() { + let test_file = test_file(); + let options = test_options().with_page_index(false); + let builder = sync_builder(&test_file, options); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder.with_projection(ProjectionMask::columns(&schema_descr, ["b"])); + // Expect to see only IO for column "b", should see no IO for columns "a" or "c". + // + // Note that we need to read all data page headers to find the pages for column b + // so there are many more small reads than in the test_read_single_column test above + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'b': DictionaryPage (17 bytes , 17 requests) [header]", + "Row Group 0, column 'b': DictionaryPage (1600 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (20 bytes , 20 requests) [header]", + "Row Group 0, column 'b': DataPage(0) (93 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (20 bytes , 20 requests) [header]", + "Row Group 0, column 'b': DataPage(1) (106 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (17 bytes , 17 requests) [header]", + "Row Group 1, column 'b': DictionaryPage (1600 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (20 bytes , 20 requests) [header]", + "Row Group 1, column 'b': DataPage(0) (93 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (20 bytes , 20 requests) [header]", + "Row Group 1, column 'b': DataPage(1) (106 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_row_selection() { + // There are 400 total rows spread across 4 data pages (100 rows each) + // select rows 175..225 (i.e. DataPage(1) of row group 0 and DataPage(0) of row group 1) + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder + .with_projection( + // read both "a" and "b" + ProjectionMask::columns(&schema_descr, ["a", "b"]), + ) + .with_row_selection(RowSelection::from(vec![ + RowSelector::skip(175), + RowSelector::select(50), + ])); + + // Expect to see only data IO for one page for each column for each row group + // Note the data page headers for all pages need to be read to find the correct pages + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_limit() { + // There are 400 total rows spread across 4 data pages (100 rows each) + // a limit of 125 rows should only fetch the first two data pages (DataPage(0) and DataPage(1)) from row group 0 + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a"])) + .with_limit(125); + + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_single_row_filter() { + // Values from column "b" range 400..799 + // filter "b" > 575 and < 625 + // (last data page in Row Group 0 and first DataPage in Row Group 1) + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection( + // read both "a" and "b" + ProjectionMask::columns(&schema_descr, ["a", "b"]), + ) + // "b" > 575 and "b" < 625 + .with_row_filter(filter_b_575_625(&schema_descr)); + + // Expect to see I/O for column b in both row groups and then reading just a + // single pages for a in each row group + // + // Note there is significant IO that happens during the construction of the + // reader (between "Builder Configured" and "Reader Built") + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_multiple_row_filter() { + // Values in column "a" range 0..399 + // Values in column "b" range 400..799 + // First filter: "a" > 175 (last data page in Row Group 0) + // Second filter: "b" < 625 (last data page in Row Group 0 and first DataPage in RowGroup 1) + // Read column "c" + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection( + ProjectionMask::columns(&schema_descr, ["c"]), // read "c" + ) + // a > 175 and b < 625 + .with_row_filter(filter_a_175_b_625(&schema_descr)); + + // Expect that we will see + // 1. IO for all pages of column A + // 2. IO for pages of column b that passed 1. + // 3. IO after reader is built only for column c + // + // Note there is significant IO that happens during the construction of the + // reader (between "Builder Configured" and "Reader Built") + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Event: Reader Built", + "Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", + "Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + ], + ); +} + +#[test] +fn test_read_single_row_filter_all() { + // Apply a filter that entirely filters out rows based on a predicate from one column + // should not read any data pages for any other column + + let test_file = test_file(); + let builder = sync_builder(&test_file, test_options()); + let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); + + let builder = builder + .with_projection(ProjectionMask::columns(&schema_descr, ["a", "b"])) + .with_row_filter(filter_b_false(&schema_descr)); + + // Expect to see the Footer and Metadata, then I/O for column b + // in both row groups but then nothing for column "a" + // since the row filter entirely filters out all rows. + // + // Note that all IO that happens during the construction of the reader + // (between "Builder Configured" and "Reader Built") + run_test( + &test_file, + builder, + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Event: Reader Built", + ], + ); +} + +/// Return a [`ParquetRecordBatchReaderBuilder`] for reading this file +fn sync_builder( + test_file: &TestParquetFile, + options: ArrowReaderOptions, +) -> ParquetRecordBatchReaderBuilder { + let reader = RecordingChunkReader { + inner: test_file.bytes().clone(), + ops: Arc::clone(test_file.ops()), + }; + ParquetRecordBatchReaderBuilder::try_new_with_options(reader, options) + .expect("ParquetRecordBatchReaderBuilder") +} + +/// Build the reader from the specified builder and read all batches from it, +/// and assert that the operations log contains the expected entries. +fn run_test<'a>( + test_file: &TestParquetFile, + builder: ParquetRecordBatchReaderBuilder, + expected: impl IntoIterator, +) { + let ops = test_file.ops(); + ops.add_entry(LogEntry::event("Builder Configured")); + let reader = builder.build().unwrap(); + ops.add_entry(LogEntry::event("Reader Built")); + for batch in reader { + match batch { + Ok(_) => {} + Err(e) => panic!("Error reading batch: {e}"), + } + } + ops.assert(expected) +} + +/// Records IO operations on an in-memory chunk reader +struct RecordingChunkReader { + inner: Bytes, + ops: Arc, +} + +impl Length for RecordingChunkReader { + fn len(&self) -> u64 { + self.inner.len() as u64 + } +} + +impl ChunkReader for RecordingChunkReader { + type T = RecordingStdIoReader; + + fn get_read(&self, start: u64) -> parquet::errors::Result { + let reader = RecordingStdIoReader { + start: start as usize, + inner: self.inner.clone(), + ops: Arc::clone(&self.ops), + }; + Ok(reader) + } + + fn get_bytes(&self, start: u64, length: usize) -> parquet::errors::Result { + let start = start as usize; + let range = start..start + length; + self.ops.add_entry_for_range(&range); + Ok(self.inner.slice(start..start + length)) + } +} + +/// Wrapper around a `Bytes` object that implements `Read` +struct RecordingStdIoReader { + /// current offset in the inner `Bytes` that this reader is reading from + start: usize, + inner: Bytes, + ops: Arc, +} + +impl Read for RecordingStdIoReader { + fn read(&mut self, buf: &mut [u8]) -> std::io::Result { + let remain = self.inner.len() - self.start; + let start = self.start; + let read_length = buf.len().min(remain); + let read_range = start..start + read_length; + + self.ops.add_entry_for_range(&read_range); + + buf.copy_from_slice(self.inner.slice(read_range).as_ref()); + // Update the inner position + self.start += read_length; + Ok(read_length) + } +} diff --git a/parquet/tests/arrow_reader/mod.rs b/parquet/tests/arrow_reader/mod.rs index 48d732f17f21..bb19b47b52e5 100644 --- a/parquet/tests/arrow_reader/mod.rs +++ b/parquet/tests/arrow_reader/mod.rs @@ -42,6 +42,7 @@ mod bad_data; #[cfg(feature = "crc")] mod checksum; mod int96_stats_roundtrip; +mod io; mod statistics; // returns a struct array with columns "int32_col", "float32_col" and "float64_col" with the specified values From 851495cdea672d1337b789cc345d5854e1b16033 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 7 Aug 2025 11:57:19 -0400 Subject: [PATCH 34/38] use insta for sync reader --- parquet/Cargo.toml | 1 + parquet/tests/arrow_reader/io/mod.rs | 11 + parquet/tests/arrow_reader/io/sync_reader.rs | 379 +++++++++---------- 3 files changed, 196 insertions(+), 195 deletions(-) diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index 05557069aa7d..f601ac7cefdc 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -78,6 +78,7 @@ 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 } +insta = "1.43.1" brotli = { version = "8.0", default-features = false, features = ["std"] } flate2 = { version = "1.0", default-features = false, features = ["rust_backend"] } lz4_flex = { version = "0.11", default-features = false, features = ["std", "frame"] } diff --git a/parquet/tests/arrow_reader/io/mod.rs b/parquet/tests/arrow_reader/io/mod.rs index dab6cb87aa26..fefb37abf938 100644 --- a/parquet/tests/arrow_reader/io/mod.rs +++ b/parquet/tests/arrow_reader/io/mod.rs @@ -690,6 +690,17 @@ impl OperationLog { } } + /// return a snapshot of the current operations in the log. + fn snapshot(&self) -> Vec { + self.coalesce_entries(); + let ops = self.ops.lock().unwrap(); + let mut actual = vec![]; + let indent = 0; + ops.iter() + .for_each(|s| s.append_string(&mut actual, indent)); + actual + } + /// Assert that the operations in the log match the expected operations /// with an error message that can be copy/pasted to update a test on failure. fn assert<'a>(&self, expected: impl IntoIterator) { diff --git a/parquet/tests/arrow_reader/io/sync_reader.rs b/parquet/tests/arrow_reader/io/sync_reader.rs index be4b164f0194..2523e17848fc 100644 --- a/parquet/tests/arrow_reader/io/sync_reader.rs +++ b/parquet/tests/arrow_reader/io/sync_reader.rs @@ -37,35 +37,34 @@ fn test_read_entire_file() { let test_file = test_file(); // Expect to see IO for all data pages for each row group and column let builder = sync_builder(&test_file, test_options()); - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Event: Reader Built", - "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", - "Row Group 0, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", - "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", + "Row Group 0, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -74,26 +73,25 @@ fn test_read_single_group() { let builder = sync_builder(&test_file, test_options()).with_row_groups(vec![1]); // read only second row group // Expect to see only IO for Row Group 1. Should see no IO for Row Group 0. - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Event: Reader Built", - "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", - "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -103,23 +101,22 @@ fn test_read_single_column() { let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); let builder = builder.with_projection(ProjectionMask::columns(&schema_descr, ["b"])); // Expect to see only IO for column "b". Should see no IO for columns "a" or "c". - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Event: Reader Built", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -133,28 +130,27 @@ fn test_read_single_column_no_page_index() { // // Note that we need to read all data page headers to find the pages for column b // so there are many more small reads than in the test_read_single_column test above - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "Event: Builder Configured", - "Event: Reader Built", - "Row Group 0, column 'b': DictionaryPage (17 bytes , 17 requests) [header]", - "Row Group 0, column 'b': DictionaryPage (1600 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(0) (20 bytes , 20 requests) [header]", - "Row Group 0, column 'b': DataPage(0) (93 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (20 bytes , 20 requests) [header]", - "Row Group 0, column 'b': DataPage(1) (106 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (17 bytes , 17 requests) [header]", - "Row Group 1, column 'b': DictionaryPage (1600 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (20 bytes , 20 requests) [header]", - "Row Group 1, column 'b': DataPage(0) (93 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (20 bytes , 20 requests) [header]", - "Row Group 1, column 'b': DataPage(1) (106 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'b': DictionaryPage (17 bytes , 17 requests) [header]", + "Row Group 0, column 'b': DictionaryPage (1600 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (20 bytes , 20 requests) [header]", + "Row Group 0, column 'b': DataPage(0) (93 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (20 bytes , 20 requests) [header]", + "Row Group 0, column 'b': DataPage(1) (106 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (17 bytes , 17 requests) [header]", + "Row Group 1, column 'b': DictionaryPage (1600 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (20 bytes , 20 requests) [header]", + "Row Group 1, column 'b': DataPage(0) (93 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (20 bytes , 20 requests) [header]", + "Row Group 1, column 'b': DataPage(1) (106 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -176,25 +172,24 @@ fn test_read_row_selection() { // Expect to see only data IO for one page for each column for each row group // Note the data page headers for all pages need to be read to find the correct pages - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Event: Reader Built", - "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -208,20 +203,19 @@ fn test_read_limit() { .with_projection(ProjectionMask::columns(&schema_descr, ["a"])) .with_limit(125); - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Event: Reader Built", - "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -246,31 +240,30 @@ fn test_read_single_row_filter() { // // Note there is significant IO that happens during the construction of the // reader (between "Builder Configured" and "Reader Built") - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Event: Reader Built", - "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Event: Reader Built", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -298,32 +291,31 @@ fn test_read_multiple_row_filter() { // // Note there is significant IO that happens during the construction of the // reader (between "Builder Configured" and "Reader Built") - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Event: Reader Built", - "Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", - "Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", - "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Event: Reader Built", + "Row Group 0, column 'c': DictionaryPage (7107 bytes, 1 requests) [data]", + "Row Group 0, column 'c': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", + "Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", + ] + "#); } #[test] @@ -345,23 +337,22 @@ fn test_read_single_row_filter_all() { // // Note that all IO that happens during the construction of the reader // (between "Builder Configured" and "Reader Built") - run_test( - &test_file, - builder, - [ - "Footer: 8 bytes", - "Metadata: 1162", - "UNKNOWN: 22230..22877 (maybe Page Index)", - "Event: Builder Configured", - "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Event: Reader Built", - ], - ); + insta::assert_debug_snapshot!(run(&test_file, builder), + @r#" + [ + "Footer: 8 bytes", + "Metadata: 1162", + "UNKNOWN: 22230..22877 (maybe Page Index)", + "Event: Builder Configured", + "Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 0, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + "Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + "Row Group 1, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Event: Reader Built", + ] + "#); } /// Return a [`ParquetRecordBatchReaderBuilder`] for reading this file @@ -377,13 +368,11 @@ fn sync_builder( .expect("ParquetRecordBatchReaderBuilder") } -/// Build the reader from the specified builder and read all batches from it, -/// and assert that the operations log contains the expected entries. -fn run_test<'a>( +/// run +fn run( test_file: &TestParquetFile, builder: ParquetRecordBatchReaderBuilder, - expected: impl IntoIterator, -) { +) -> Vec { let ops = test_file.ops(); ops.add_entry(LogEntry::event("Builder Configured")); let reader = builder.build().unwrap(); @@ -394,7 +383,7 @@ fn run_test<'a>( Err(e) => panic!("Error reading batch: {e}"), } } - ops.assert(expected) + ops.snapshot() } /// Records IO operations on an in-memory chunk reader From e239704b90eb88b34d67e69d78777ec00922847b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 7 Aug 2025 12:07:17 -0400 Subject: [PATCH 35/38] Update to use insta --- parquet/tests/arrow_reader/io/async_reader.rs | 140 +++++++++--------- parquet/tests/arrow_reader/io/mod.rs | 20 --- parquet/tests/arrow_reader/io/sync_reader.rs | 2 +- 3 files changed, 69 insertions(+), 93 deletions(-) diff --git a/parquet/tests/arrow_reader/io/async_reader.rs b/parquet/tests/arrow_reader/io/async_reader.rs index 5d1051f24f16..f2d3ce07234b 100644 --- a/parquet/tests/arrow_reader/io/async_reader.rs +++ b/parquet/tests/arrow_reader/io/async_reader.rs @@ -37,24 +37,23 @@ async fn test_read_entire_file() { // read entire file without any filtering or projection let test_file = test_file(); let builder = async_builder(&test_file, test_options()).await; - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, - [ - "Get Provided Metadata", - "Event: Builder Configured", - "Event: Reader Built", - "Read Multi:", - " Row Group 0, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", - " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", - " Row Group 0, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7346 bytes, 1 requests) [data]", - "Read Multi:", - " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", - " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", - " Row Group 1, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7456 bytes, 1 requests) [data]", - ] - ) - .await; + builder).await, @r#" + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 0, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 0, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7346 bytes, 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", + " Row Group 1, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7456 bytes, 1 requests) [data]", + ] + "#); } #[tokio::test] @@ -66,9 +65,9 @@ async fn test_read_single_group() { .with_row_groups(vec![1]); // Expect to see only IO for Row Group 1. Should see no IO for Row Group 0. - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, + builder).await, @r#" [ "Get Provided Metadata", "Event: Builder Configured", @@ -78,7 +77,7 @@ async fn test_read_single_group() { " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", " Row Group 1, column 'c': MultiPage(dictionary_page: true, data_pages: [0, 1]) (7456 bytes, 1 requests) [data]", ] - ).await; + "#); } #[tokio::test] @@ -88,9 +87,9 @@ async fn test_read_single_column() { let schema_descr = builder.metadata().file_metadata().schema_descr_ptr(); let builder = builder.with_projection(ProjectionMask::columns(&schema_descr, ["b"])); // Expect to see only IO for column "b". Should see no IO for columns "a" or "c". - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, + builder).await, @r#" [ "Get Provided Metadata", "Event: Builder Configured", @@ -100,7 +99,7 @@ async fn test_read_single_column() { "Read Multi:", " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", ] - ).await; + "#); } #[tokio::test] @@ -118,26 +117,25 @@ async fn test_read_row_selection() { ])); // Expect to see only data IO for one page for each column for each row group - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, - [ - "Get Provided Metadata", - "Event: Builder Configured", - "Event: Reader Built", - "Read Multi:", - " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - " Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - " Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", - "Read Multi:", - " Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - " Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - " Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", - " Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", - ], - ) - .await; + builder).await, @r#" + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + " Row Group 0, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'b': DataPage(1) (126 bytes , 1 requests) [data]", + "Read Multi:", + " Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + " Row Group 1, column 'b': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 1, column 'b': DataPage(0) (113 bytes , 1 requests) [data]", + ] + "#); } #[tokio::test] @@ -151,20 +149,19 @@ async fn test_read_limit() { .with_projection(ProjectionMask::columns(&schema_descr, ["a"])) .with_limit(125); - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, - [ - "Get Provided Metadata", - "Event: Builder Configured", - "Event: Reader Built", - "Read Multi:", - " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", - " Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", - " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", - ], - ) - .await; + builder).await, @r#" + [ + "Get Provided Metadata", + "Event: Builder Configured", + "Event: Reader Built", + "Read Multi:", + " Row Group 0, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", + " Row Group 0, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", + " Row Group 0, column 'a': DataPage(1) (126 bytes , 1 requests) [data]", + ] + "#); } #[tokio::test] @@ -182,9 +179,9 @@ async fn test_read_single_row_filter() { // Expect to see I/O for column b in both row groups to evaluate filter, // then a single pages for the "a" column in each row group - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, + builder).await, @r#" [ "Get Provided Metadata", "Event: Builder Configured", @@ -200,7 +197,7 @@ async fn test_read_single_row_filter() { " Row Group 1, column 'a': DictionaryPage (1617 bytes, 1 requests) [data]", " Row Group 1, column 'a': DataPage(0) (113 bytes , 1 requests) [data]", ] - ).await; + "#); } #[tokio::test] @@ -221,9 +218,9 @@ async fn test_read_single_row_filter_no_page_index() { // 1. I/O for all pages of column b to evaluate the filter // 2. IO for all pages of column a as the reader doesn't know where the page // boundaries are so needs to scan them. - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, + builder).await, @r#" [ "Get Provided Metadata", "Event: Builder Configured", @@ -237,7 +234,7 @@ async fn test_read_single_row_filter_no_page_index() { "Read Multi:", " Row Group 1, column 'a': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", ] - ).await; + "#); } #[tokio::test] @@ -259,9 +256,9 @@ async fn test_read_multiple_row_filter() { // 1. IO for all pages of column A (to evaluate the first filter) // 2. IO for pages of column b that passed the first filter (to evaluate the second filter) // 3. IO after reader is built only for column c for the rows that passed both filters - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, + builder).await, @r#" [ "Get Provided Metadata", "Event: Builder Configured", @@ -284,7 +281,7 @@ async fn test_read_multiple_row_filter() { " Row Group 1, column 'c': DictionaryPage (7217 bytes, 1 requests) [data]", " Row Group 1, column 'c': DataPage(0) (113 bytes , 1 requests) [data]", ] - ).await + "#); } #[tokio::test] @@ -301,9 +298,9 @@ async fn test_read_single_row_filter_all() { // Expect to see reads for column "b" to evaluate the filter, but no reads // for column "a" as no rows pass the filter - run_test( + insta::assert_debug_snapshot!(run( &test_file, - builder, + builder).await, @r#" [ "Get Provided Metadata", "Event: Builder Configured", @@ -313,7 +310,7 @@ async fn test_read_single_row_filter_all() { "Read Multi:", " Row Group 1, column 'b': MultiPage(dictionary_page: true, data_pages: [0, 1]) (1856 bytes, 1 requests) [data]", ] - ).await; + "#); } /// Return a [`ParquetRecordBatchStreamBuilder`] for reading this file @@ -348,12 +345,11 @@ async fn async_builder( } /// Build the reader from the specified builder and read all batches from it, -/// and assert that the operations log contains the expected entries. -async fn run_test<'a>( +/// and return the operations log. +async fn run( test_file: &TestParquetFile, builder: ParquetRecordBatchStreamBuilder, - expected: impl IntoIterator, -) { +) -> Vec { let ops = test_file.ops(); ops.add_entry(LogEntry::event("Builder Configured")); let mut stream = builder.build().unwrap(); @@ -364,7 +360,7 @@ async fn run_test<'a>( Err(e) => panic!("Error reading batch: {e}"), } } - ops.assert(expected) + ops.snapshot() } struct RecordingAsyncFileReader { diff --git a/parquet/tests/arrow_reader/io/mod.rs b/parquet/tests/arrow_reader/io/mod.rs index fefb37abf938..b31f295755b0 100644 --- a/parquet/tests/arrow_reader/io/mod.rs +++ b/parquet/tests/arrow_reader/io/mod.rs @@ -700,24 +700,4 @@ impl OperationLog { .for_each(|s| s.append_string(&mut actual, indent)); actual } - - /// Assert that the operations in the log match the expected operations - /// with an error message that can be copy/pasted to update a test on failure. - fn assert<'a>(&self, expected: impl IntoIterator) { - let expected = expected.into_iter().collect::>(); - - self.coalesce_entries(); - let ops = self.ops.lock().unwrap(); - - let mut actual = vec![]; - let indent = 0; - ops.iter() - .for_each(|s| s.append_string(&mut actual, indent)); - assert_eq!( - // use pretty print for easier diff comparison - format!("{actual:#?}"), - format!("{expected:#?}"), - "Operation log mismatch\n\nactual:\n{actual:#?}\nexpected:\n{expected:#?}" - ); - } } diff --git a/parquet/tests/arrow_reader/io/sync_reader.rs b/parquet/tests/arrow_reader/io/sync_reader.rs index 2523e17848fc..685f251a9e2b 100644 --- a/parquet/tests/arrow_reader/io/sync_reader.rs +++ b/parquet/tests/arrow_reader/io/sync_reader.rs @@ -368,7 +368,7 @@ fn sync_builder( .expect("ParquetRecordBatchReaderBuilder") } -/// run +/// build the reader, and read all batches from it, returning the recorded IO operations fn run( test_file: &TestParquetFile, builder: ParquetRecordBatchReaderBuilder, From f73a7c169ae4b0d1c5c5b5bf9eb5200015a98969 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 7 Aug 2025 12:37:11 -0400 Subject: [PATCH 36/38] fix windows tests --- parquet/tests/arrow_reader/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet/tests/arrow_reader/mod.rs b/parquet/tests/arrow_reader/mod.rs index bb19b47b52e5..8b23be13a9d2 100644 --- a/parquet/tests/arrow_reader/mod.rs +++ b/parquet/tests/arrow_reader/mod.rs @@ -335,9 +335,9 @@ fn make_uint_batches(start: u8, end: u8) -> RecordBatch { Field::new("u64", DataType::UInt64, true), ])); let v8: Vec = (start..end).collect(); - let v16: Vec = (start as _..end as _).collect(); - let v32: Vec = (start as _..end as _).collect(); - let v64: Vec = (start as _..end as _).collect(); + let v16: Vec = (start as _..end as u16).collect(); + let v32: Vec = (start as _..end as u32).collect(); + let v64: Vec = (start as _..end as u64).collect(); RecordBatch::try_new( schema, vec![ From 315e4637b4b4baff2b019a629f7f335ed8b5f518 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 7 Aug 2025 14:39:42 -0500 Subject: [PATCH 37/38] exclude nested column from cache --- parquet/src/arrow/async_reader/mod.rs | 109 +++++++++++++++++++++++++- 1 file changed, 108 insertions(+), 1 deletion(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 843ad766e9a7..8b59ac1d516c 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -716,7 +716,43 @@ where cache_projection.union(predicate.projection()); } cache_projection.intersect(projection); - Some(cache_projection) + self.exclude_nested_columns_from_cache(&cache_projection) + } +} + +impl ReaderFactory +where + T: AsyncFileReader + Send, +{ + /// Exclude leaves belonging to roots that span multiple parquet leaves (i.e. nested columns) + fn exclude_nested_columns_from_cache(&self, mask: &ProjectionMask) -> Option { + let schema = self.metadata.file_metadata().schema_descr(); + let num_leaves = schema.num_columns(); + + // Count how many leaves each root column has + let num_roots = schema.root_schema().get_fields().len(); + let mut root_leaf_counts = vec![0usize; num_roots]; + for leaf_idx in 0..num_leaves { + let root_idx = schema.get_column_root_idx(leaf_idx); + root_leaf_counts[root_idx] += 1; + } + + // Keep only leaves whose root has exactly one leaf (non-nested) + let mut included_leaves = Vec::new(); + for leaf_idx in 0..num_leaves { + if mask.leaf_included(leaf_idx) { + let root_idx = schema.get_column_root_idx(leaf_idx); + if root_leaf_counts[root_idx] == 1 { + included_leaves.push(leaf_idx); + } + } + } + + if included_leaves.is_empty() { + None + } else { + Some(ProjectionMask::leaves(schema, included_leaves)) + } } } @@ -2341,6 +2377,77 @@ mod tests { assert_eq!(requests.lock().unwrap().len(), 3); } + #[tokio::test] + async fn test_cache_projection_excludes_nested_columns() { + use arrow_array::{ArrayRef, StringArray}; + + // Build a simple RecordBatch with a primitive column `a` and a nested struct column `b { aa, bb }` + let a = StringArray::from_iter_values(["r1", "r2"]); + let b = StructArray::from(vec![ + ( + Arc::new(Field::new("aa", DataType::Utf8, true)), + Arc::new(StringArray::from_iter_values(["v1", "v2"])) as ArrayRef, + ), + ( + Arc::new(Field::new("bb", DataType::Utf8, true)), + Arc::new(StringArray::from_iter_values(["w1", "w2"])) as ArrayRef, + ), + ]); + + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, true), + Field::new("b", b.data_type().clone(), true), + ])); + + let mut buf = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, None).unwrap(); + let batch = RecordBatch::try_from_iter([ + ("a", Arc::new(a) as ArrayRef), + ("b", Arc::new(b) as ArrayRef), + ]) + .unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + // Load Parquet metadata + let data: Bytes = buf.into(); + let metadata = ParquetMetaDataReader::new() + .parse_and_finish(&data) + .unwrap(); + let metadata = Arc::new(metadata); + + // Build a RowFilter whose predicate projects a leaf under the nested root `b` + // Leaf indices are depth-first; with schema [a, b.aa, b.bb] we pick index 1 (b.aa) + let parquet_schema = metadata.file_metadata().schema_descr(); + let nested_leaf_mask = ProjectionMask::leaves(parquet_schema, vec![1]); + + let always_true = ArrowPredicateFn::new(nested_leaf_mask.clone(), |batch: RecordBatch| { + Ok(arrow_array::BooleanArray::from(vec![ + true; + batch.num_rows() + ])) + }); + let filter = RowFilter::new(vec![Box::new(always_true)]); + + // Construct a ReaderFactory and compute cache projection + let reader_factory = ReaderFactory { + metadata: Arc::clone(&metadata), + fields: None, + input: TestReader::new(data), + filter: Some(filter), + limit: None, + offset: None, + metrics: ArrowReaderMetrics::disabled(), + max_predicate_cache_size: 0, + }; + + // Provide an output projection that also selects the same nested leaf + let cache_projection = reader_factory.compute_cache_projection(&nested_leaf_mask); + + // Expect None since nested columns should be excluded from cache projection + assert!(cache_projection.is_none()); + } + #[tokio::test] async fn empty_offset_index_doesnt_panic_in_read_row_group() { use tokio::fs::File; From 1db701ac8eec61d63b34e090ed666b2cefd565b7 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Thu, 7 Aug 2025 14:52:42 -0500 Subject: [PATCH 38/38] only use expanded selection when the column is one of cache column --- parquet/src/arrow/async_reader/mod.rs | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 8b59ac1d516c..eea6176b766b 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -635,12 +635,15 @@ where // (pre) Fetch only the columns that are selected by the predicate let selection = plan_builder.selection(); + // Fetch predicate columns; expand selection only for cached predicate columns + let cache_mask = Some(&cache_projection); row_group .fetch( &mut self.input, predicate.projection(), selection, batch_size, + cache_mask, ) .await?; @@ -688,11 +691,13 @@ where } // fetch the pages needed for decoding row_group + // Final projection fetch shouldn't expand selection for cache; pass None .fetch( &mut self.input, &projection, plan_builder.selection(), batch_size, + None, ) .await?; @@ -718,12 +723,7 @@ where cache_projection.intersect(projection); self.exclude_nested_columns_from_cache(&cache_projection) } -} -impl ReaderFactory -where - T: AsyncFileReader + Send, -{ /// Exclude leaves belonging to roots that span multiple parquet leaves (i.e. nested columns) fn exclude_nested_columns_from_cache(&self, mask: &ProjectionMask) -> Option { let schema = self.metadata.file_metadata().schema_descr(); @@ -984,10 +984,12 @@ impl InMemoryRowGroup<'_> { projection: &ProjectionMask, selection: Option<&RowSelection>, batch_size: usize, + cache_mask: Option<&ProjectionMask>, ) -> Result<()> { let metadata = self.metadata.row_group(self.row_group_idx); if let Some((selection, offset_index)) = selection.zip(self.offset_index) { - let selection = selection.expand_to_batch_boundaries(batch_size, self.row_count); + let expanded_selection = + selection.expand_to_batch_boundaries(batch_size, self.row_count); // If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the // `RowSelection` let mut page_start_offsets: Vec> = vec![]; @@ -1012,7 +1014,15 @@ impl InMemoryRowGroup<'_> { _ => (), } - ranges.extend(selection.scan_ranges(&offset_index[idx].page_locations)); + // Expand selection to batch boundaries only for cached columns + let use_expanded = cache_mask.map(|m| m.leaf_included(idx)).unwrap_or(false); + if use_expanded { + ranges.extend( + expanded_selection.scan_ranges(&offset_index[idx].page_locations), + ); + } else { + ranges.extend(selection.scan_ranges(&offset_index[idx].page_locations)); + } page_start_offsets.push(ranges.iter().map(|range| range.start).collect()); ranges @@ -1920,7 +1930,6 @@ mod tests { assert_eq!(total_rows, 730); } - #[ignore] #[tokio::test] async fn test_in_memory_row_group_sparse() { let testdata = arrow::util::test_util::parquet_test_data();