From f93d36eff9e13c463d7fe99a58ff3c388e63bdc0 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Sun, 16 Mar 2025 16:46:04 +0100 Subject: [PATCH 01/48] Add support for file row numbers in Parquet readers --- parquet/examples/read_with_rowgroup.rs | 4 + parquet/src/arrow/array_reader/builder.rs | 76 +++-- parquet/src/arrow/array_reader/list_array.rs | 3 +- parquet/src/arrow/array_reader/mod.rs | 9 + parquet/src/arrow/array_reader/row_number.rs | 130 +++++++++ parquet/src/arrow/arrow_reader/mod.rs | 284 ++++++++++++++++++- parquet/src/arrow/async_reader/mod.rs | 77 ++++- parquet/src/errors.rs | 6 + parquet/src/file/metadata/mod.rs | 49 +++- parquet/src/file/metadata/reader.rs | 15 +- 10 files changed, 607 insertions(+), 46 deletions(-) create mode 100644 parquet/src/arrow/array_reader/row_number.rs diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index 8cccc7fe14ac..83c9c4adfa12 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -129,6 +129,10 @@ impl RowGroups for InMemoryRowGroup { } } } + + fn row_groups(&self) -> Box + '_> { + Box::new(std::iter::once(&self.metadata)) + } } impl InMemoryRowGroup { diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 945f62526a7e..bf154b605cd6 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. +use arrow_schema::{DataType, Field, Fields, SchemaBuilder}; use std::sync::Arc; -use arrow_schema::{DataType, Fields, SchemaBuilder}; - use crate::arrow::array_reader::byte_view_array::make_byte_view_array_reader; 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_number::RowNumberReader; use crate::arrow::array_reader::{ make_byte_array_dictionary_reader, make_byte_array_reader, ArrayReader, FixedSizeListArrayReader, ListArrayReader, MapArrayReader, NullArrayReader, @@ -39,10 +39,29 @@ pub fn build_array_reader( field: Option<&ParquetField>, mask: &ProjectionMask, row_groups: &dyn RowGroups, + row_number_column: Option, ) -> Result> { let reader = field - .and_then(|field| build_reader(field, mask, row_groups).transpose()) - .transpose()? + .and_then(|field| { + build_reader(field, mask, row_groups, row_number_column.clone()).transpose() + }) + .or_else(|| { + row_number_column.map(|column| { + let row_number_reader = build_row_number_reader(row_groups)?; + let reader: Box = Box::new(StructArrayReader::new( + DataType::Struct(Fields::from(vec![Field::new( + column, + row_number_reader.get_data_type().clone(), + false, + )])), + vec![row_number_reader], + 0, + 0, + false, + )); + Ok(reader) + }) + }).transpose()? .unwrap_or_else(|| make_empty_array_reader(row_groups.num_rows())); Ok(reader) @@ -52,12 +71,13 @@ fn build_reader( field: &ParquetField, mask: &ProjectionMask, row_groups: &dyn RowGroups, + row_number_column: Option, ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { .. } => build_primitive_reader(field, mask, row_groups), ParquetFieldType::Group { .. } => match &field.arrow_type { DataType::Map(_, _) => build_map_reader(field, mask, row_groups), - DataType::Struct(_) => build_struct_reader(field, mask, row_groups), + DataType::Struct(_) => build_struct_reader(field, mask, row_groups, row_number_column), DataType::List(_) => build_list_reader(field, mask, false, row_groups), DataType::LargeList(_) => build_list_reader(field, mask, true, row_groups), DataType::FixedSizeList(_, _) => build_fixed_size_list_reader(field, mask, row_groups), @@ -66,6 +86,10 @@ fn build_reader( } } +fn build_row_number_reader(row_groups: &dyn RowGroups) -> Result> { + Ok(Box::new(RowNumberReader::try_new(row_groups.row_groups())?)) +} + /// Build array reader for map type. fn build_map_reader( field: &ParquetField, @@ -75,8 +99,8 @@ fn build_map_reader( let children = field.children().unwrap(); assert_eq!(children.len(), 2); - let key_reader = build_reader(&children[0], mask, row_groups)?; - let value_reader = build_reader(&children[1], mask, row_groups)?; + let key_reader = build_reader(&children[0], mask, row_groups, None)?; + let value_reader = build_reader(&children[1], mask, row_groups, None)?; match (key_reader, value_reader) { (Some(key_reader), Some(value_reader)) => { @@ -127,7 +151,7 @@ fn build_list_reader( let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match build_reader(&children[0], mask, row_groups)? { + let reader = match build_reader(&children[0], mask, row_groups, None)? { Some(item_reader) => { // Need to retrieve underlying data type to handle projection let item_type = item_reader.get_data_type().clone(); @@ -173,7 +197,7 @@ fn build_fixed_size_list_reader( let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match build_reader(&children[0], mask, row_groups)? { + let reader = match build_reader(&children[0], mask, row_groups, None)? { Some(item_reader) => { let item_type = item_reader.get_data_type().clone(); let reader = match &field.arrow_type { @@ -300,6 +324,7 @@ fn build_struct_reader( field: &ParquetField, mask: &ProjectionMask, row_groups: &dyn RowGroups, + row_number_column: Option, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -312,7 +337,7 @@ fn build_struct_reader( let mut builder = SchemaBuilder::with_capacity(children.len()); for (arrow, parquet) in arrow_fields.iter().zip(children) { - if let Some(reader) = build_reader(parquet, mask, row_groups)? { + if let Some(reader) = build_reader(parquet, mask, row_groups, None)? { // 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)); @@ -320,6 +345,16 @@ fn build_struct_reader( } } + if let Some(row_number_column) = row_number_column { + let reader = build_row_number_reader(row_groups)?; + builder.push(Field::new( + row_number_column, + reader.get_data_type().clone(), + false, + )); + readers.push(reader); + } + if readers.is_empty() { return Ok(None); } @@ -356,14 +391,23 @@ mod tests { ) .unwrap(); - let array_reader = build_array_reader(fields.as_ref(), &mask, &file_reader).unwrap(); + let array_reader = build_array_reader( + fields.as_ref(), + &mask, + &file_reader, + Some("row_number".to_string()), + ) + .unwrap(); // Create arrow types - let arrow_type = DataType::Struct(Fields::from(vec![Field::new( - "b_struct", - DataType::Struct(vec![Field::new("b_c_int", DataType::Int32, true)].into()), - true, - )])); + let arrow_type = DataType::Struct(Fields::from(vec![ + Field::new( + "b_struct", + DataType::Struct(vec![Field::new("b_c_int", DataType::Int32, true)].into()), + true, + ), + Field::new("row_number", DataType::Int64, false), + ])); assert_eq!(array_reader.get_data_type(), &arrow_type); } diff --git a/parquet/src/arrow/array_reader/list_array.rs b/parquet/src/arrow/array_reader/list_array.rs index 6e583ed00c19..f74a39c0d4c4 100644 --- a/parquet/src/arrow/array_reader/list_array.rs +++ b/parquet/src/arrow/array_reader/list_array.rs @@ -563,7 +563,8 @@ mod tests { ) .unwrap(); - let mut array_reader = build_array_reader(fields.as_ref(), &mask, &file_reader).unwrap(); + let mut array_reader = + build_array_reader(fields.as_ref(), &mask, &file_reader, None).unwrap(); let batch = array_reader.next_batch(100).unwrap(); assert_eq!(batch.data_type(), array_reader.get_data_type()); diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index a5ea426e95bb..f7bcf9746422 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -41,10 +41,12 @@ mod map_array; mod null_array; mod primitive_array; mod struct_array; +mod row_number; #[cfg(test)] mod test_util; +use crate::file::metadata::RowGroupMetaData; pub use builder::build_array_reader; pub use byte_array::make_byte_array_reader; pub use byte_array_dictionary::make_byte_array_dictionary_reader; @@ -113,6 +115,9 @@ pub trait RowGroups { /// Returns a [`PageIterator`] for the column chunks with the given leaf column index fn column_chunks(&self, i: usize) -> Result>; + + /// Returns an iterator over the row groups in this collection + fn row_groups(&self) -> Box + '_>; } impl RowGroups for Arc { @@ -124,6 +129,10 @@ impl RowGroups for Arc { let iterator = FilePageIterator::new(column_index, Arc::clone(self))?; Ok(Box::new(iterator)) } + + fn row_groups(&self) -> Box + '_> { + Box::new(self.metadata().row_groups().iter()) + } } /// Uses `record_reader` to read up to `batch_size` records from `pages` diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs new file mode 100644 index 000000000000..ff6b51992bf0 --- /dev/null +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -0,0 +1,130 @@ +use crate::arrow::array_reader::ArrayReader; +use crate::errors::{ParquetError, Result}; +use crate::file::metadata::RowGroupMetaData; +use arrow_array::{ArrayRef, Int64Array}; +use arrow_schema::DataType; +use std::any::Any; +use std::collections::VecDeque; +use std::sync::Arc; + +pub(crate) struct RowNumberReader { + row_numbers: Vec, + row_groups: RowGroupSizeIterator, +} + +impl RowNumberReader { + pub(crate) fn try_new(row_groups: impl IntoIterator) -> Result + where + I: TryInto, + { + let row_groups = RowGroupSizeIterator::try_new(row_groups)?; + Ok(Self { + row_numbers: Vec::new(), + row_groups, + }) + } +} + +impl ArrayReader for RowNumberReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_data_type(&self) -> &DataType { + &DataType::Int64 + } + + fn read_records(&mut self, batch_size: usize) -> Result { + let read = self + .row_groups + .read_records(batch_size, &mut self.row_numbers); + Ok(read) + } + + fn consume_batch(&mut self) -> Result { + Ok(Arc::new(Int64Array::from_iter(self.row_numbers.drain(..)))) + } + + fn skip_records(&mut self, num_records: usize) -> Result { + let skipped = self.row_groups.skip_records(num_records); + Ok(skipped) + } + + fn get_def_levels(&self) -> Option<&[i16]> { + None + } + + fn get_rep_levels(&self) -> Option<&[i16]> { + None + } +} + +struct RowGroupSizeIterator { + row_groups: VecDeque, +} + +impl RowGroupSizeIterator { + fn try_new(row_groups: impl IntoIterator) -> Result + where + I: TryInto, + { + Ok(Self { + row_groups: VecDeque::from(row_groups.into_iter().map(TryInto::try_into).collect::>>()?), + }) + } +} + +impl RowGroupSizeIterator { + fn read_records(&mut self, mut batch_size: usize, row_numbers: &mut Vec) -> usize { + let mut read = 0; + while batch_size > 0 { + let Some(front) = self.row_groups.front_mut() else { + return read as usize; + }; + let to_read = std::cmp::min(front.num_rows, batch_size as i64); + row_numbers.extend(front.first_row_number..front.first_row_number + to_read); + front.num_rows -= to_read; + front.first_row_number += to_read; + if front.num_rows == 0 { + self.row_groups.pop_front(); + } + batch_size -= to_read as usize; + read += to_read; + } + read as usize + } + + fn skip_records(&mut self, mut num_records: usize) -> usize { + let mut skipped = 0; + while num_records > 0 { + let Some(front) = self.row_groups.front_mut() else { + return skipped as usize; + }; + let to_skip = std::cmp::min(front.num_rows, num_records as i64); + front.num_rows -= to_skip; + front.first_row_number += to_skip; + if front.num_rows == 0 { + self.row_groups.pop_front(); + } + skipped += to_skip; + num_records -= to_skip as usize; + } + skipped as usize + } +} + +pub(crate) struct RowGroupSize { + first_row_number: i64, + num_rows: i64, +} + +impl TryFrom<&RowGroupMetaData> for RowGroupSize { + type Error = ParquetError; + + fn try_from(rg: &RowGroupMetaData) -> Result { + Ok(Self { + first_row_number: rg.first_row_number().ok_or(ParquetError::RowGroupMetaDataMissingRowNumber)?, + num_rows: rg.num_rows(), + }) + } +} diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index d14b94cc7ef7..a41813541820 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -35,7 +35,7 @@ use crate::column::page::{PageIterator, PageReader}; #[cfg(feature = "encryption")] use crate::encryption::decrypt::{CryptoContext, FileDecryptionProperties}; use crate::errors::{ParquetError, Result}; -use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; +use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; @@ -73,6 +73,8 @@ pub struct ArrowReaderBuilder { pub(crate) limit: Option, pub(crate) offset: Option, + + pub(crate) row_number_column: Option, } impl ArrowReaderBuilder { @@ -89,6 +91,7 @@ impl ArrowReaderBuilder { selection: None, limit: None, offset: None, + row_number_column: None, } } @@ -239,6 +242,16 @@ impl ArrowReaderBuilder { ..self } } + + /// Include file row numbers in the output with the given column name + /// + /// This will add a column to the output record batch with the file row number + pub fn with_row_number_column(self, row_number_column: Option) -> Self { + Self { + row_number_column, + ..self + } + } } /// Options that control how metadata is read for a parquet file @@ -621,8 +634,12 @@ impl ParquetRecordBatchReaderBuilder { break; } - let array_reader = - build_array_reader(self.fields.as_deref(), predicate.projection(), &reader)?; + let array_reader = build_array_reader( + self.fields.as_deref(), + predicate.projection(), + &reader, + self.row_number_column.clone(), + )?; selection = Some(evaluate_predicate( batch_size, @@ -633,7 +650,12 @@ impl ParquetRecordBatchReaderBuilder { } } - let array_reader = build_array_reader(self.fields.as_deref(), &self.projection, &reader)?; + let array_reader = build_array_reader( + self.fields.as_deref(), + &self.projection, + &reader, + self.row_number_column, + )?; // If selection is empty, truncate if !selects_any(selection.as_ref()) { @@ -673,6 +695,14 @@ impl RowGroups for ReaderRowGroups { row_groups: self.row_groups.clone().into_iter(), })) } + + fn row_groups(&self) -> Box + '_> { + Box::new( + self.row_groups + .iter() + .map(move |i| self.metadata.row_group(*i)), + ) + } } struct ReaderPageIterator { @@ -860,8 +890,12 @@ impl ParquetRecordBatchReader { batch_size: usize, selection: Option, ) -> Result { - let array_reader = - build_array_reader(levels.levels.as_ref(), &ProjectionMask::all(), row_groups)?; + let array_reader = build_array_reader( + levels.levels.as_ref(), + &ProjectionMask::all(), + row_groups, + None, + )?; Ok(Self { batch_size, @@ -976,7 +1010,7 @@ pub(crate) fn evaluate_predicate( } #[cfg(test)] -mod tests { +pub(crate) mod tests { use std::cmp::min; use std::collections::{HashMap, VecDeque}; use std::fmt::Formatter; @@ -988,7 +1022,8 @@ mod tests { use bytes::Bytes; use half::f16; use num::PrimInt; - use rand::{thread_rng, Rng, RngCore}; + use rand::rngs::StdRng; + use rand::{random, thread_rng, Rng, RngCore, SeedableRng}; use tempfile::tempfile; use arrow_array::builder::*; @@ -1006,8 +1041,9 @@ mod tests { use arrow_select::concat::concat_batches; use crate::arrow::arrow_reader::{ - ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader, - ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, + ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, + ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, + RowSelector, }; use crate::arrow::schema::add_encoded_arrow_schema_to_metadata; use crate::arrow::{ArrowWriter, ProjectionMask}; @@ -1020,6 +1056,7 @@ mod tests { use crate::errors::Result; use crate::file::properties::{EnabledStatistics, WriterProperties, WriterVersion}; use crate::file::writer::SerializedFileWriter; + use crate::format::FileMetaData; use crate::schema::parser::parse_message_type; use crate::schema::types::{Type, TypePtr}; use crate::util::test_common::rand_gen::RandGen; @@ -4395,4 +4432,231 @@ mod tests { assert_eq!(c0.len(), c1.len()); c0.iter().zip(c1.iter()).for_each(|(l, r)| assert_eq!(l, r)); } + + #[test] + fn test_read_row_numbers() { + let file = write_parquet_from_iter(vec![( + "value", + Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, + )]); + let supplied_fields = Fields::from(vec![Field::new("value", ArrowDataType::Int64, false)]); + + let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); + let mut arrow_reader = ParquetRecordBatchReaderBuilder::try_new_with_options( + file.try_clone().unwrap(), + options, + ) + .expect("reader builder with schema") + .with_row_number_column(Some("row_number".to_string())) + .build() + .expect("reader with schema"); + + let batch = arrow_reader.next().unwrap().unwrap(); + let schema = Arc::new(Schema::new(vec![ + Field::new("value", ArrowDataType::Int64, false), + Field::new("row_number", ArrowDataType::Int64, false), + ])); + + assert_eq!(batch.schema(), schema); + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + assert_eq!( + batch + .column(0) + .as_primitive::() + .iter() + .collect::>(), + vec![Some(1), Some(2), Some(3)] + ); + assert_eq!( + batch + .column(1) + .as_primitive::() + .iter() + .collect::>(), + vec![Some(0), Some(1), Some(2)] + ); + } + + #[test] + fn test_read_only_row_numbers() { + let file = write_parquet_from_iter(vec![( + "value", + Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, + )]); + let mut metadata = ArrowReaderMetadata::load(&file, Default::default()).unwrap(); + metadata.fields = None; + + let mut arrow_reader = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata) + .with_row_number_column(Some("row_number".to_string())) + .build() + .expect("reader with schema"); + + let batch = arrow_reader.next().unwrap().unwrap(); + let schema = Arc::new(Schema::new(vec![Field::new( + "row_number", + ArrowDataType::Int64, + false, + )])); + + assert_eq!(batch.schema(), schema); + assert_eq!(batch.num_columns(), 1); + assert_eq!(batch.num_rows(), 3); + assert_eq!( + batch + .column(0) + .as_primitive::() + .iter() + .collect::>(), + vec![Some(0), Some(1), Some(2)] + ); + } + + #[test] + fn test_row_numbers_with_multiple_row_groups() { + test_row_numbers_with_multiple_row_groups_helper( + false, + |path, selection, _row_filter, batch_size| { + let file = File::open(path).unwrap(); + let reader = ParquetRecordBatchReaderBuilder::try_new(file) + .unwrap() + .with_row_selection(selection) + .with_batch_size(batch_size) + .with_row_number_column(Some("row_number".to_string())) + .build() + .expect("Could not create reader"); + reader + .collect::, _>>() + .expect("Could not read") + }, + ); + } + + #[test] + fn test_row_numbers_with_multiple_row_groups_and_filter() { + test_row_numbers_with_multiple_row_groups_helper( + true, + |path, selection, row_filter, batch_size| { + let file = File::open(path).unwrap(); + let reader = ParquetRecordBatchReaderBuilder::try_new(file) + .unwrap() + .with_row_selection(selection) + .with_batch_size(batch_size) + .with_row_filter(row_filter.expect("No filter")) + .with_row_number_column(Some("row_number".to_string())) + .build() + .expect("Could not create reader"); + reader + .collect::, _>>() + .expect("Could not read") + }, + ); + } + + pub(crate) fn test_row_numbers_with_multiple_row_groups_helper( + use_filter: bool, + test_case: F, + ) where + F: FnOnce(PathBuf, RowSelection, Option, usize) -> Vec, + { + let seed: u64 = random(); + println!("test_row_numbers_with_multiple_row_groups seed: {}", seed); + let mut rng = StdRng::seed_from_u64(seed); + + use tempfile::TempDir; + let tempdir = TempDir::new().expect("Could not create temp dir"); + + let (bytes, metadata) = generate_file_with_row_numbers(&mut rng); + + let path = tempdir.path().join("test.parquet"); + std::fs::write(&path, bytes).expect("Could not write file"); + + let mut case = vec![]; + let mut remaining = metadata.num_rows; + while remaining > 0 { + let row_count = rng.gen_range(1..=remaining); + remaining -= row_count; + case.push(RowSelector { + row_count: row_count as usize, + skip: rng.gen_bool(0.5), + }); + } + + let filter = use_filter.then(|| { + let filter = (0..metadata.num_rows) + .map(|_| rng.gen_bool(0.99)) + .collect::>(); + let mut filter_offset = 0; + RowFilter::new(vec![Box::new(ArrowPredicateFn::new( + ProjectionMask::all(), + move |b| { + let array = BooleanArray::from_iter( + filter + .iter() + .skip(filter_offset) + .take(b.num_rows()) + .map(|x| Some(*x)), + ); + filter_offset += b.num_rows(); + Ok(array) + }, + ))]) + }); + + let selection = RowSelection::from(case); + let batches = test_case(path, selection.clone(), filter, rng.gen_range(1..4096)); + + if selection.skipped_row_count() == metadata.num_rows as usize { + assert!(batches.into_iter().all(|batch| batch.num_rows() == 0)); + return; + } + let actual = concat_batches(batches.first().expect("No batches").schema_ref(), &batches) + .expect("Failed to concatenate"); + // assert_eq!(selection.row_count(), actual.num_rows()); + let values = actual + .column(0) + .as_primitive::() + .iter() + .collect::>(); + let row_numbers = actual + .column(1) + .as_primitive::() + .iter() + .collect::>(); + assert_eq!( + row_numbers + .into_iter() + .map(|number| number.map(|number| number + 1)) + .collect::>(), + values + ); + } + + fn generate_file_with_row_numbers(rng: &mut impl Rng) -> (Bytes, FileMetaData) { + let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( + "value", + ArrowDataType::Int64, + false, + )]))); + + let mut buf = Vec::with_capacity(1024); + let mut writer = + ArrowWriter::try_new(&mut buf, schema.clone(), None).expect("Could not create writer"); + + let mut values = 1..=rng.gen_range(1..4096); + while !values.is_empty() { + let batch_values = values + .by_ref() + .take(rng.gen_range(1..4096)) + .collect::>(); + let array = Arc::new(Int64Array::from(batch_values)) as ArrayRef; + let batch = + RecordBatch::try_from_iter([("value", array)]).expect("Could not create batch"); + writer.write(&batch).expect("Could not write batch"); + writer.flush().expect("Could not flush"); + } + let metadata = writer.close().expect("Could not close writer"); + + (Bytes::from(buf), metadata) + } } diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index fd49ad22934d..5c74611a9fcb 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -52,7 +52,7 @@ use crate::bloom_filter::{ }; use crate::column::page::{PageIterator, PageReader}; use crate::errors::{ParquetError, Result}; -use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; +use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::file::reader::{ChunkReader, Length, SerializedPageReader}; use crate::file::FOOTER_SIZE; @@ -564,6 +564,7 @@ impl ParquetRecordBatchStreamBuilder { fields: self.fields, limit: self.limit, offset: self.offset, + row_number_column: self.row_number_column, }; // Ensure schema of ParquetRecordBatchStream respects projection, and does @@ -606,6 +607,8 @@ struct ReaderFactory { limit: Option, offset: Option, + + row_number_column: Option, } impl ReaderFactory @@ -652,8 +655,12 @@ where .fetch(&mut self.input, predicate_projection, selection.as_ref()) .await?; - let array_reader = - build_array_reader(self.fields.as_deref(), predicate_projection, &row_group)?; + let array_reader = build_array_reader( + self.fields.as_deref(), + predicate_projection, + &row_group, + self.row_number_column.clone(), + )?; selection = Some(evaluate_predicate( batch_size, @@ -703,7 +710,12 @@ where let reader = ParquetRecordBatchReader::new( batch_size, - build_array_reader(self.fields.as_deref(), &projection, &row_group)?, + build_array_reader( + self.fields.as_deref(), + &projection, + &row_group, + self.row_number_column.clone(), + )?, selection, ); @@ -1086,6 +1098,10 @@ impl RowGroups for InMemoryRowGroup<'_> { } } } + + fn row_groups(&self) -> Box + '_> { + Box::new(std::iter::once(self.metadata.row_group(self.row_group_idx))) + } } /// An in-memory column chunk @@ -1161,6 +1177,7 @@ impl PageIterator for ColumnChunkIterator {} #[cfg(test)] mod tests { use super::*; + use crate::arrow::arrow_reader::tests::test_row_numbers_with_multiple_row_groups_helper; use crate::arrow::arrow_reader::{ ArrowPredicateFn, ParquetRecordBatchReaderBuilder, RowSelector, }; @@ -1925,6 +1942,7 @@ mod tests { filter: None, limit: None, offset: None, + row_number_column: None, }; let mut skip = true; @@ -2453,4 +2471,55 @@ mod tests { let result = reader.try_collect::>().await.unwrap(); assert_eq!(result.len(), 1); } + + #[test] + fn test_row_numbers_with_multiple_row_groups() { + test_row_numbers_with_multiple_row_groups_helper( + false, + |path, selection, _row_filter, batch_size| { + let runtime = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .expect("Could not create runtime"); + runtime.block_on(async move { + let file = tokio::fs::File::open(path).await.unwrap(); + let reader = ParquetRecordBatchStreamBuilder::new(file) + .await + .unwrap() + .with_row_selection(selection) + .with_batch_size(batch_size) + .with_row_number_column(Some("row_number".to_string())) + .build() + .expect("Could not create reader"); + reader.try_collect::>().await.unwrap() + }) + }, + ); + } + + #[test] + fn test_row_numbers_with_multiple_row_groups_and_filter() { + test_row_numbers_with_multiple_row_groups_helper( + true, + |path, selection, row_filter, batch_size| { + let runtime = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .expect("Could not create runtime"); + runtime.block_on(async move { + let file = tokio::fs::File::open(path).await.unwrap(); + let reader = ParquetRecordBatchStreamBuilder::new(file) + .await + .unwrap() + .with_row_selection(selection) + .with_row_filter(row_filter.expect("No row filter")) + .with_batch_size(batch_size) + .with_row_number_column(Some("row_number".to_string())) + .build() + .expect("Could not create reader"); + reader.try_collect::>().await.unwrap() + }) + }, + ); + } } diff --git a/parquet/src/errors.rs b/parquet/src/errors.rs index 4cb1f99c3cf6..469af6332ee3 100644 --- a/parquet/src/errors.rs +++ b/parquet/src/errors.rs @@ -52,6 +52,9 @@ pub enum ParquetError { /// Returned when a function needs more data to complete properly. The `usize` field indicates /// the total number of bytes required, not the number of additional bytes. NeedMoreData(usize), + /// Returned when an operation needs to know the first row number of a row group, but the row + /// number is unknown. + RowGroupMetaDataMissingRowNumber, } impl std::fmt::Display for ParquetError { @@ -69,6 +72,9 @@ impl std::fmt::Display for ParquetError { } ParquetError::External(e) => write!(fmt, "External: {e}"), ParquetError::NeedMoreData(needed) => write!(fmt, "NeedMoreData: {needed}"), + ParquetError::RowGroupMetaDataMissingRowNumber => { + write!(fmt, "Row group missing row number") + } } } } diff --git a/parquet/src/file/metadata/mod.rs b/parquet/src/file/metadata/mod.rs index 217685049ea9..43084dc83350 100644 --- a/parquet/src/file/metadata/mod.rs +++ b/parquet/src/file/metadata/mod.rs @@ -544,6 +544,7 @@ pub type RowGroupMetaDataPtr = Arc; pub struct RowGroupMetaData { columns: Vec, num_rows: i64, + first_row_number: Option, sorting_columns: Option>, total_byte_size: i64, schema_descr: SchemaDescPtr, @@ -584,6 +585,11 @@ impl RowGroupMetaData { self.num_rows } + /// Returns the first row number in this row group. + pub fn first_row_number(&self) -> Option { + self.first_row_number + } + /// Returns the sort ordering of the rows in this RowGroup if any pub fn sorting_columns(&self) -> Option<&Vec> { self.sorting_columns.as_ref() @@ -629,6 +635,7 @@ impl RowGroupMetaData { fn from_encrypted_thrift( schema_descr: SchemaDescPtr, mut rg: RowGroup, + first_row_number: Option, decryptor: Option<&FileDecryptor>, ) -> Result { if schema_descr.num_columns() != rg.columns.len() { @@ -691,6 +698,7 @@ impl RowGroupMetaData { Ok(RowGroupMetaData { columns, num_rows, + first_row_number, sorting_columns, total_byte_size, schema_descr, @@ -700,7 +708,11 @@ impl RowGroupMetaData { } /// Method to convert from Thrift. - pub fn from_thrift(schema_descr: SchemaDescPtr, mut rg: RowGroup) -> Result { + pub fn from_thrift( + schema_descr: SchemaDescPtr, + mut rg: RowGroup, + first_row_number: Option, + ) -> Result { if schema_descr.num_columns() != rg.columns.len() { return Err(general_err!( "Column count mismatch. Schema has {} columns while Row Group has {}", @@ -720,6 +732,7 @@ impl RowGroupMetaData { Ok(RowGroupMetaData { columns, num_rows, + first_row_number, sorting_columns, total_byte_size, schema_descr, @@ -758,6 +771,7 @@ impl RowGroupMetaDataBuilder { schema_descr, file_offset: None, num_rows: 0, + first_row_number: None, sorting_columns: None, total_byte_size: 0, ordinal: None, @@ -770,6 +784,12 @@ impl RowGroupMetaDataBuilder { self } + /// Sets the first row number in this row group. + pub fn set_first_row_number(mut self, value: i64) -> Self { + self.0.first_row_number = Some(value); + self + } + /// Sets the sorting order for columns pub fn set_sorting_columns(mut self, value: Option>) -> Self { self.0.sorting_columns = value; @@ -1702,13 +1722,15 @@ mod tests { .set_total_byte_size(2000) .set_column_metadata(columns) .set_ordinal(1) + .set_first_row_number(10) .build() .unwrap(); let row_group_exp = row_group_meta.to_thrift(); - let row_group_res = RowGroupMetaData::from_thrift(schema_descr, row_group_exp.clone()) - .unwrap() - .to_thrift(); + let row_group_res = + RowGroupMetaData::from_thrift(schema_descr, row_group_exp.clone(), Some(10)) + .unwrap() + .to_thrift(); assert_eq!(row_group_res, row_group_exp); } @@ -1787,10 +1809,13 @@ mod tests { .build() .unwrap(); - let err = - RowGroupMetaData::from_thrift(schema_descr_3cols, row_group_meta_2cols.to_thrift()) - .unwrap_err() - .to_string(); + let err = RowGroupMetaData::from_thrift( + schema_descr_3cols, + row_group_meta_2cols.to_thrift(), + None, + ) + .unwrap_err() + .to_string(); assert_eq!( err, "Parquet error: Column count mismatch. Schema has 3 columns while Row Group has 2" @@ -1952,9 +1977,9 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let base_expected_size = 2312; + let base_expected_size = 2328; #[cfg(feature = "encryption")] - let base_expected_size = 2448; + let base_expected_size = 2464; assert_eq!(parquet_meta.memory_size(), base_expected_size); @@ -1982,9 +2007,9 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let bigger_expected_size = 2816; + let bigger_expected_size = 2832; #[cfg(feature = "encryption")] - let bigger_expected_size = 2952; + let bigger_expected_size = 2968; // more set fields means more memory usage assert!(bigger_expected_size > base_expected_size); diff --git a/parquet/src/file/metadata/reader.rs b/parquet/src/file/metadata/reader.rs index b80e76d7929a..e984710dad4b 100644 --- a/parquet/src/file/metadata/reader.rs +++ b/parquet/src/file/metadata/reader.rs @@ -754,7 +754,7 @@ impl ParquetMetaDataReader { } } - let t_file_metadata: TFileMetaData = TFileMetaData::read_from_in_protocol(&mut prot) + let mut t_file_metadata: TFileMetaData = TFileMetaData::read_from_in_protocol(&mut prot) .map_err(|e| general_err!("Could not parse metadata: {}", e))?; let schema = types::from_thrift(&t_file_metadata.schema)?; let schema_descr = Arc::new(SchemaDescriptor::new(schema)); @@ -767,13 +767,17 @@ impl ParquetMetaDataReader { file_decryptor = Some(get_file_decryptor(algo, file_decryption_properties)?); } + let mut first_row_number = 0; let mut row_groups = Vec::new(); + t_file_metadata.row_groups.sort_by_key(|rg| rg.ordinal); for rg in t_file_metadata.row_groups { let r = RowGroupMetaData::from_encrypted_thrift( schema_descr.clone(), rg, + Some(first_row_number), file_decryptor.as_ref(), )?; + first_row_number += r.num_rows; row_groups.push(r); } let column_orders = @@ -804,14 +808,19 @@ impl ParquetMetaDataReader { pub fn decode_metadata(buf: &[u8]) -> Result { let mut prot = TCompactSliceInputProtocol::new(buf); - let t_file_metadata: TFileMetaData = TFileMetaData::read_from_in_protocol(&mut prot) + let mut t_file_metadata: TFileMetaData = TFileMetaData::read_from_in_protocol(&mut prot) .map_err(|e| general_err!("Could not parse metadata: {}", e))?; let schema = types::from_thrift(&t_file_metadata.schema)?; let schema_descr = Arc::new(SchemaDescriptor::new(schema)); + let mut first_row_number = 0; let mut row_groups = Vec::new(); + t_file_metadata.row_groups.sort_by_key(|rg| rg.ordinal); for rg in t_file_metadata.row_groups { - row_groups.push(RowGroupMetaData::from_thrift(schema_descr.clone(), rg)?); + let row_group = + RowGroupMetaData::from_thrift(schema_descr.clone(), rg, Some(first_row_number))?; + first_row_number += row_group.num_rows; + row_groups.push(row_group); } let column_orders = Self::parse_column_orders(t_file_metadata.column_orders, &schema_descr)?; From e485c0b3af1fa3d24292bf6740031be675c25280 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Wed, 26 Mar 2025 08:21:16 +0100 Subject: [PATCH 02/48] Add Apache license header to row_number.rs --- parquet/src/arrow/array_reader/row_number.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index ff6b51992bf0..cbb5f49c5e97 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.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::ArrayReader; use crate::errors::{ParquetError, Result}; use crate::file::metadata::RowGroupMetaData; From 2a62009aa67cc6bd29ee6df1cac853e9b1a79089 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Wed, 26 Mar 2025 08:23:23 +0100 Subject: [PATCH 03/48] Run cargo format --- parquet/src/arrow/array_reader/builder.rs | 3 ++- parquet/src/arrow/array_reader/mod.rs | 2 +- parquet/src/arrow/array_reader/row_number.rs | 15 +++++++++++---- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index bf154b605cd6..17130287c722 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -61,7 +61,8 @@ pub fn build_array_reader( )); Ok(reader) }) - }).transpose()? + }) + .transpose()? .unwrap_or_else(|| make_empty_array_reader(row_groups.num_rows())); Ok(reader) diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index f7bcf9746422..d4108fc0e2db 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -40,8 +40,8 @@ mod list_array; mod map_array; mod null_array; mod primitive_array; -mod struct_array; mod row_number; +mod struct_array; #[cfg(test)] mod test_util; diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index cbb5f49c5e97..8dbab1f82a07 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -32,7 +32,7 @@ pub(crate) struct RowNumberReader { impl RowNumberReader { pub(crate) fn try_new(row_groups: impl IntoIterator) -> Result where - I: TryInto, + I: TryInto, { let row_groups = RowGroupSizeIterator::try_new(row_groups)?; Ok(Self { @@ -83,10 +83,15 @@ struct RowGroupSizeIterator { impl RowGroupSizeIterator { fn try_new(row_groups: impl IntoIterator) -> Result where - I: TryInto, + I: TryInto, { Ok(Self { - row_groups: VecDeque::from(row_groups.into_iter().map(TryInto::try_into).collect::>>()?), + row_groups: VecDeque::from( + row_groups + .into_iter() + .map(TryInto::try_into) + .collect::>>()?, + ), }) } } @@ -140,7 +145,9 @@ impl TryFrom<&RowGroupMetaData> for RowGroupSize { fn try_from(rg: &RowGroupMetaData) -> Result { Ok(Self { - first_row_number: rg.first_row_number().ok_or(ParquetError::RowGroupMetaDataMissingRowNumber)?, + first_row_number: rg + .first_row_number() + .ok_or(ParquetError::RowGroupMetaDataMissingRowNumber)?, num_rows: rg.num_rows(), }) } From fb5126fa4ca65e379eff5d269cdd0889524cd92a Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Thu, 27 Mar 2025 18:02:07 +0100 Subject: [PATCH 04/48] Change with_row_number_column to take impl Into --- parquet/src/arrow/arrow_reader/mod.rs | 12 ++++++------ parquet/src/arrow/async_reader/mod.rs | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index a41813541820..1f2d35358c8d 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -246,9 +246,9 @@ impl ArrowReaderBuilder { /// Include file row numbers in the output with the given column name /// /// This will add a column to the output record batch with the file row number - pub fn with_row_number_column(self, row_number_column: Option) -> Self { + pub fn with_row_number_column(self, row_number_column: impl Into) -> Self { Self { - row_number_column, + row_number_column: Some(row_number_column.into()), ..self } } @@ -4447,7 +4447,7 @@ pub(crate) mod tests { options, ) .expect("reader builder with schema") - .with_row_number_column(Some("row_number".to_string())) + .with_row_number_column("row_number") .build() .expect("reader with schema"); @@ -4488,7 +4488,7 @@ pub(crate) mod tests { metadata.fields = None; let mut arrow_reader = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata) - .with_row_number_column(Some("row_number".to_string())) + .with_row_number_column("row_number") .build() .expect("reader with schema"); @@ -4522,7 +4522,7 @@ pub(crate) mod tests { .unwrap() .with_row_selection(selection) .with_batch_size(batch_size) - .with_row_number_column(Some("row_number".to_string())) + .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader @@ -4543,7 +4543,7 @@ pub(crate) mod tests { .with_row_selection(selection) .with_batch_size(batch_size) .with_row_filter(row_filter.expect("No filter")) - .with_row_number_column(Some("row_number".to_string())) + .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 5c74611a9fcb..1b4d81dc133a 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -2488,7 +2488,7 @@ mod tests { .unwrap() .with_row_selection(selection) .with_batch_size(batch_size) - .with_row_number_column(Some("row_number".to_string())) + .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader.try_collect::>().await.unwrap() @@ -2514,7 +2514,7 @@ mod tests { .with_row_selection(selection) .with_row_filter(row_filter.expect("No row filter")) .with_batch_size(batch_size) - .with_row_number_column(Some("row_number".to_string())) + .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader.try_collect::>().await.unwrap() From 5350728bb61a3a0a93a8fe797c969a1e6d264169 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Thu, 27 Mar 2025 18:02:07 +0100 Subject: [PATCH 05/48] Change Option -> Option<&str> in build_array_reader --- parquet/src/arrow/array_reader/builder.rs | 19 ++++++------------- parquet/src/arrow/arrow_reader/mod.rs | 4 ++-- parquet/src/arrow/async_reader/mod.rs | 4 ++-- 3 files changed, 10 insertions(+), 17 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 17130287c722..7cf6717576b6 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -39,12 +39,10 @@ pub fn build_array_reader( field: Option<&ParquetField>, mask: &ProjectionMask, row_groups: &dyn RowGroups, - row_number_column: Option, + row_number_column: Option<&str>, ) -> Result> { let reader = field - .and_then(|field| { - build_reader(field, mask, row_groups, row_number_column.clone()).transpose() - }) + .and_then(|field| build_reader(field, mask, row_groups, row_number_column).transpose()) .or_else(|| { row_number_column.map(|column| { let row_number_reader = build_row_number_reader(row_groups)?; @@ -72,7 +70,7 @@ fn build_reader( field: &ParquetField, mask: &ProjectionMask, row_groups: &dyn RowGroups, - row_number_column: Option, + row_number_column: Option<&str>, ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { .. } => build_primitive_reader(field, mask, row_groups), @@ -325,7 +323,7 @@ fn build_struct_reader( field: &ParquetField, mask: &ProjectionMask, row_groups: &dyn RowGroups, - row_number_column: Option, + row_number_column: Option<&str>, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -392,13 +390,8 @@ mod tests { ) .unwrap(); - let array_reader = build_array_reader( - fields.as_ref(), - &mask, - &file_reader, - Some("row_number".to_string()), - ) - .unwrap(); + let array_reader = + build_array_reader(fields.as_ref(), &mask, &file_reader, Some("row_number")).unwrap(); // Create arrow types let arrow_type = DataType::Struct(Fields::from(vec![ diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 1f2d35358c8d..4066310654e7 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -638,7 +638,7 @@ impl ParquetRecordBatchReaderBuilder { self.fields.as_deref(), predicate.projection(), &reader, - self.row_number_column.clone(), + self.row_number_column.as_deref(), )?; selection = Some(evaluate_predicate( @@ -654,7 +654,7 @@ impl ParquetRecordBatchReaderBuilder { self.fields.as_deref(), &self.projection, &reader, - self.row_number_column, + self.row_number_column.as_deref(), )?; // If selection is empty, truncate diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 1b4d81dc133a..558da8c1c907 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -659,7 +659,7 @@ where self.fields.as_deref(), predicate_projection, &row_group, - self.row_number_column.clone(), + self.row_number_column.as_deref(), )?; selection = Some(evaluate_predicate( @@ -714,7 +714,7 @@ where self.fields.as_deref(), &projection, &row_group, - self.row_number_column.clone(), + self.row_number_column.as_deref(), )?, selection, ); From 188f35047c11c33b18d3adf08b2a5c1836495768 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Thu, 27 Mar 2025 18:02:07 +0100 Subject: [PATCH 06/48] Replace ParquetError::RowGroupMetaDataMissingRowNumber with General --- parquet/src/arrow/array_reader/row_number.rs | 6 +++--- parquet/src/errors.rs | 6 ------ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index 8dbab1f82a07..5e68fe32915b 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -145,9 +145,9 @@ impl TryFrom<&RowGroupMetaData> for RowGroupSize { fn try_from(rg: &RowGroupMetaData) -> Result { Ok(Self { - first_row_number: rg - .first_row_number() - .ok_or(ParquetError::RowGroupMetaDataMissingRowNumber)?, + first_row_number: rg.first_row_number().ok_or(ParquetError::General( + "Row group missing row number".to_string(), + ))?, num_rows: rg.num_rows(), }) } diff --git a/parquet/src/errors.rs b/parquet/src/errors.rs index 469af6332ee3..4cb1f99c3cf6 100644 --- a/parquet/src/errors.rs +++ b/parquet/src/errors.rs @@ -52,9 +52,6 @@ pub enum ParquetError { /// Returned when a function needs more data to complete properly. The `usize` field indicates /// the total number of bytes required, not the number of additional bytes. NeedMoreData(usize), - /// Returned when an operation needs to know the first row number of a row group, but the row - /// number is unknown. - RowGroupMetaDataMissingRowNumber, } impl std::fmt::Display for ParquetError { @@ -72,9 +69,6 @@ impl std::fmt::Display for ParquetError { } ParquetError::External(e) => write!(fmt, "External: {e}"), ParquetError::NeedMoreData(needed) => write!(fmt, "NeedMoreData: {needed}"), - ParquetError::RowGroupMetaDataMissingRowNumber => { - write!(fmt, "Row group missing row number") - } } } } From 37a9d83780f1249b0ab4489f61915f0bceb8bb88 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Thu, 27 Mar 2025 18:02:07 +0100 Subject: [PATCH 07/48] Split test_create_array_reader test into two --- parquet/src/arrow/array_reader/builder.rs | 26 +++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 7cf6717576b6..b1503fd73fbb 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -390,6 +390,32 @@ mod tests { ) .unwrap(); + let array_reader = build_array_reader(fields.as_ref(), &mask, &file_reader, None).unwrap(); + + // Create arrow types + let arrow_type = DataType::Struct(Fields::from(vec![Field::new( + "b_struct", + DataType::Struct(vec![Field::new("b_c_int", DataType::Int32, true)].into()), + true, + )])); + + assert_eq!(array_reader.get_data_type(), &arrow_type); + } + + #[test] + fn test_create_array_reader_with_row_numbers() { + let file = get_test_file("nulls.snappy.parquet"); + let file_reader: Arc = Arc::new(SerializedFileReader::new(file).unwrap()); + + let file_metadata = file_reader.metadata().file_metadata(); + let mask = ProjectionMask::leaves(file_metadata.schema_descr(), [0]); + let (_, fields) = parquet_to_arrow_schema_and_fields( + file_metadata.schema_descr(), + ProjectionMask::all(), + file_metadata.key_value_metadata(), + ) + .unwrap(); + let array_reader = build_array_reader(fields.as_ref(), &mask, &file_reader, Some("row_number")).unwrap(); From 41e38fea0d053fe590656ffe8c8ce956c2252a6e Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Fri, 28 Mar 2025 22:41:12 +0100 Subject: [PATCH 08/48] first_row_number -> first_row_index --- parquet/src/arrow/array_reader/row_number.rs | 2 +- parquet/src/file/metadata/mod.rs | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index 5e68fe32915b..b5dbb75bc19e 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -145,7 +145,7 @@ impl TryFrom<&RowGroupMetaData> for RowGroupSize { fn try_from(rg: &RowGroupMetaData) -> Result { Ok(Self { - first_row_number: rg.first_row_number().ok_or(ParquetError::General( + first_row_number: rg.first_row_index().ok_or(ParquetError::General( "Row group missing row number".to_string(), ))?, num_rows: rg.num_rows(), diff --git a/parquet/src/file/metadata/mod.rs b/parquet/src/file/metadata/mod.rs index 43084dc83350..51d22c3e907f 100644 --- a/parquet/src/file/metadata/mod.rs +++ b/parquet/src/file/metadata/mod.rs @@ -544,7 +544,7 @@ pub type RowGroupMetaDataPtr = Arc; pub struct RowGroupMetaData { columns: Vec, num_rows: i64, - first_row_number: Option, + first_row_index: Option, sorting_columns: Option>, total_byte_size: i64, schema_descr: SchemaDescPtr, @@ -585,9 +585,9 @@ impl RowGroupMetaData { self.num_rows } - /// Returns the first row number in this row group. - pub fn first_row_number(&self) -> Option { - self.first_row_number + /// Returns the global index number for the first row in this row group. + pub fn first_row_index(&self) -> Option { + self.first_row_index } /// Returns the sort ordering of the rows in this RowGroup if any @@ -698,7 +698,7 @@ impl RowGroupMetaData { Ok(RowGroupMetaData { columns, num_rows, - first_row_number, + first_row_index: first_row_number, sorting_columns, total_byte_size, schema_descr, @@ -732,7 +732,7 @@ impl RowGroupMetaData { Ok(RowGroupMetaData { columns, num_rows, - first_row_number, + first_row_index: first_row_number, sorting_columns, total_byte_size, schema_descr, @@ -771,7 +771,7 @@ impl RowGroupMetaDataBuilder { schema_descr, file_offset: None, num_rows: 0, - first_row_number: None, + first_row_index: None, sorting_columns: None, total_byte_size: 0, ordinal: None, @@ -786,7 +786,7 @@ impl RowGroupMetaDataBuilder { /// Sets the first row number in this row group. pub fn set_first_row_number(mut self, value: i64) -> Self { - self.0.first_row_number = Some(value); + self.0.first_row_index = Some(value); self } From 1a1e6b6a3ff25cc5adec5b9aa2f6615159609663 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Sun, 11 May 2025 20:12:41 +0200 Subject: [PATCH 09/48] Simplify RowNumberReader with iterators Co-authored-by: scovich --- parquet/src/arrow/array_reader/row_number.rs | 130 +++++-------------- 1 file changed, 30 insertions(+), 100 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index b5dbb75bc19e..551f4fb061e3 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -21,28 +21,45 @@ use crate::file::metadata::RowGroupMetaData; use arrow_array::{ArrayRef, Int64Array}; use arrow_schema::DataType; use std::any::Any; -use std::collections::VecDeque; use std::sync::Arc; pub(crate) struct RowNumberReader { - row_numbers: Vec, - row_groups: RowGroupSizeIterator, + buffered_row_numbers: Vec, + remaining_row_numbers: std::iter::Flatten>>, } impl RowNumberReader { - pub(crate) fn try_new(row_groups: impl IntoIterator) -> Result - where - I: TryInto, - { - let row_groups = RowGroupSizeIterator::try_new(row_groups)?; + pub(crate) fn try_new<'a>( + row_groups: impl Iterator, + ) -> Result { + let ranges = row_groups + .map(|rg| { + let first_row_number = rg.first_row_index().ok_or(ParquetError::General( + "Row group missing row number".to_string(), + ))?; + Ok(first_row_number..first_row_number + rg.num_rows()) + }) + .collect::>>()?; Ok(Self { - row_numbers: Vec::new(), - row_groups, + buffered_row_numbers: Vec::new(), + remaining_row_numbers: ranges.into_iter().flatten(), }) } } impl ArrayReader for RowNumberReader { + fn read_records(&mut self, batch_size: usize) -> Result { + let starting_len = self.buffered_row_numbers.len(); + self.buffered_row_numbers + .extend((&mut self.remaining_row_numbers).take(batch_size)); + Ok(self.buffered_row_numbers.len() - starting_len) + } + + fn skip_records(&mut self, num_records: usize) -> Result { + // TODO: Use advance_by when it stabilizes to improve performance + Ok((&mut self.remaining_row_numbers).take(num_records).count()) + } + fn as_any(&self) -> &dyn Any { self } @@ -51,20 +68,10 @@ impl ArrayReader for RowNumberReader { &DataType::Int64 } - fn read_records(&mut self, batch_size: usize) -> Result { - let read = self - .row_groups - .read_records(batch_size, &mut self.row_numbers); - Ok(read) - } - fn consume_batch(&mut self) -> Result { - Ok(Arc::new(Int64Array::from_iter(self.row_numbers.drain(..)))) - } - - fn skip_records(&mut self, num_records: usize) -> Result { - let skipped = self.row_groups.skip_records(num_records); - Ok(skipped) + Ok(Arc::new(Int64Array::from_iter( + self.buffered_row_numbers.drain(..), + ))) } fn get_def_levels(&self) -> Option<&[i16]> { @@ -75,80 +82,3 @@ impl ArrayReader for RowNumberReader { None } } - -struct RowGroupSizeIterator { - row_groups: VecDeque, -} - -impl RowGroupSizeIterator { - fn try_new(row_groups: impl IntoIterator) -> Result - where - I: TryInto, - { - Ok(Self { - row_groups: VecDeque::from( - row_groups - .into_iter() - .map(TryInto::try_into) - .collect::>>()?, - ), - }) - } -} - -impl RowGroupSizeIterator { - fn read_records(&mut self, mut batch_size: usize, row_numbers: &mut Vec) -> usize { - let mut read = 0; - while batch_size > 0 { - let Some(front) = self.row_groups.front_mut() else { - return read as usize; - }; - let to_read = std::cmp::min(front.num_rows, batch_size as i64); - row_numbers.extend(front.first_row_number..front.first_row_number + to_read); - front.num_rows -= to_read; - front.first_row_number += to_read; - if front.num_rows == 0 { - self.row_groups.pop_front(); - } - batch_size -= to_read as usize; - read += to_read; - } - read as usize - } - - fn skip_records(&mut self, mut num_records: usize) -> usize { - let mut skipped = 0; - while num_records > 0 { - let Some(front) = self.row_groups.front_mut() else { - return skipped as usize; - }; - let to_skip = std::cmp::min(front.num_rows, num_records as i64); - front.num_rows -= to_skip; - front.first_row_number += to_skip; - if front.num_rows == 0 { - self.row_groups.pop_front(); - } - skipped += to_skip; - num_records -= to_skip as usize; - } - skipped as usize - } -} - -pub(crate) struct RowGroupSize { - first_row_number: i64, - num_rows: i64, -} - -impl TryFrom<&RowGroupMetaData> for RowGroupSize { - type Error = ParquetError; - - fn try_from(rg: &RowGroupMetaData) -> Result { - Ok(Self { - first_row_number: rg.first_row_index().ok_or(ParquetError::General( - "Row group missing row number".to_string(), - ))?, - num_rows: rg.num_rows(), - }) - } -} From 89c1fd1d344d05e84c11a662676f8681f34e6f4f Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 13:17:52 +0200 Subject: [PATCH 10/48] add parquet-testing change from the merge --- parquet-testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-testing b/parquet-testing index a3d96a65e11e..f4d7ed772a62 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit a3d96a65e11e2bbca7d22a894e8313ede90a33a3 +Subproject commit f4d7ed772a62a95111db50fbcad2460833e8c882 From b0d53d00b05778272fdf96aba892e341a69aadf4 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 13:35:57 +0200 Subject: [PATCH 11/48] Fix test_arrow_reader_all_columns --- parquet/src/arrow/array_reader/builder.rs | 22 ++++++++++++---------- parquet/src/arrow/array_reader/mod.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 19 +++++++++---------- 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index ee37b8703777..da6d0df90fdf 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -119,7 +119,7 @@ impl<'a> ArrayReaderBuilder<'a> { .and_then(|field| self.build_reader(field, mask, row_number_column).transpose()) .or_else(|| { row_number_column.map(|column| { - let row_number_reader = build_row_number_reader(row_groups)?; + let row_number_reader = self.build_row_number_reader()?; let reader: Box = Box::new(StructArrayReader::new( DataType::Struct(Fields::from(vec![Field::new( column, @@ -135,10 +135,10 @@ impl<'a> ArrayReaderBuilder<'a> { }) }) .transpose()? - .unwrap_or_else(|| make_empty_array_reader(self.num_rows())); + .unwrap_or_else(|| make_empty_array_reader(self.row_groups.num_rows())); - Ok(reader) -} + Ok(reader) + } fn build_reader( &self, @@ -178,9 +178,9 @@ impl<'a> ArrayReaderBuilder<'a> { } } -fn build_row_number_reader(row_groups: &dyn RowGroups) -> Result> { - Ok(Box::new(RowNumberReader::try_new(row_groups.row_groups())?)) -} + fn build_row_number_reader(&self) -> Result> { + Ok(Box::new(RowNumberReader::try_new(self.row_groups.row_groups())?)) + } /// Build array reader for map type. fn build_map_reader( @@ -441,7 +441,7 @@ fn build_row_number_reader(row_groups: &dyn RowGroups) -> Result ParquetRecordBatchReaderBuilder { metrics, // Not used for the sync reader, see https://github.com/apache/arrow-rs/issues/8000 max_predicate_cache_size: _, + row_number_column } = self; // Try to avoid allocate large buffer @@ -912,14 +913,14 @@ impl ParquetRecordBatchReaderBuilder { cache_projection.intersect(&projection); let array_reader = ArrayReaderBuilder::new(&reader, &metrics) - .build_array_reader(fields.as_deref(), predicate.projection(), self.row_number_column.as_deref(),)?; + .build_array_reader(fields.as_deref(), predicate.projection(), row_number_column.as_deref(),)?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } } let array_reader = ArrayReaderBuilder::new(&reader, &metrics) - .build_array_reader(fields.as_deref(), &projection, self.row_number_column.as_deref())?; + .build_array_reader(fields.as_deref(), &projection, row_number_column.as_deref())?; let read_plan = plan_builder .limited(reader.num_rows()) @@ -1176,7 +1177,7 @@ pub(crate) mod tests { use std::sync::Arc; use rand::rngs::StdRng; - use rand::{random, thread_rng, Rng, RngCore, SeedableRng}; + use rand::{random, rng, thread_rng, Rng, RngCore, SeedableRng}; use tempfile::tempfile; use arrow_array::builder::*; @@ -1196,8 +1197,6 @@ pub(crate) mod tests { use bytes::Bytes; use half::f16; use num_traits::PrimInt; - use rand::{Rng, RngCore, rng}; - use tempfile::tempfile; use crate::arrow::arrow_reader::{ ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions, @@ -2956,7 +2955,7 @@ pub(crate) mod tests { assert_eq!(end - total_read, batch.num_rows()); let a = converter(&expected_data[total_read..end]); - let b = Arc::clone(batch.column(0)); + let b = batch.column(0); assert_eq!(a.data_type(), b.data_type()); assert_eq!(a.to_data(), b.to_data()); @@ -5171,7 +5170,7 @@ pub(crate) mod tests { std::fs::write(&path, bytes).expect("Could not write file"); let mut case = vec![]; - let mut remaining = metadata.num_rows; + let mut remaining = metadata.file_metadata().num_rows(); while remaining > 0 { let row_count = rng.gen_range(1..=remaining); remaining -= row_count; @@ -5182,7 +5181,7 @@ pub(crate) mod tests { } let filter = use_filter.then(|| { - let filter = (0..metadata.num_rows) + let filter = (0..metadata.file_metadata().num_rows()) .map(|_| rng.gen_bool(0.99)) .collect::>(); let mut filter_offset = 0; @@ -5205,7 +5204,7 @@ pub(crate) mod tests { let selection = RowSelection::from(case); let batches = test_case(path, selection.clone(), filter, rng.gen_range(1..4096)); - if selection.skipped_row_count() == metadata.num_rows as usize { + if selection.skipped_row_count() == metadata.file_metadata().num_rows() as usize { assert!(batches.into_iter().all(|batch| batch.num_rows() == 0)); return; } @@ -5231,7 +5230,7 @@ pub(crate) mod tests { ); } - fn generate_file_with_row_numbers(rng: &mut impl Rng) -> (Bytes, FileMetaData) { + fn generate_file_with_row_numbers(rng: &mut impl Rng) -> (Bytes, ParquetMetaData) { let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( "value", ArrowDataType::Int64, From 094ae81bdb49e5af7b41731aa256c7fdff51e4ab Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 15:07:22 +0200 Subject: [PATCH 12/48] Fix first_row_number --- parquet/src/file/metadata/thrift/mod.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/parquet/src/file/metadata/thrift/mod.rs b/parquet/src/file/metadata/thrift/mod.rs index 14774910961f..28e0aa332740 100644 --- a/parquet/src/file/metadata/thrift/mod.rs +++ b/parquet/src/file/metadata/thrift/mod.rs @@ -585,9 +585,11 @@ fn read_column_chunk<'a>( fn read_row_group( prot: &mut ThriftSliceInputProtocol, schema_descr: &Arc, + first_row_number: i64, ) -> Result { // create default initialized RowGroupMetaData let mut row_group = RowGroupMetaDataBuilder::new(schema_descr.clone()).build_unchecked(); + row_group.first_row_index = Some(first_row_number); // mask values for required fields const RG_COLUMNS: u8 = 1 << 1; @@ -726,8 +728,11 @@ pub(crate) fn parquet_metadata_from_bytes(buf: &[u8]) -> Result let schema_descr = schema_descr.as_ref().unwrap(); let list_ident = prot.read_list_begin()?; let mut rg_vec = Vec::with_capacity(list_ident.size as usize); + let mut first_row_number = 0i64; for _ in 0..list_ident.size { - rg_vec.push(read_row_group(&mut prot, schema_descr)?); + let rg = read_row_group(&mut prot, schema_descr, first_row_number)?; + first_row_number += rg.num_rows(); + rg_vec.push(rg); } row_groups = Some(rg_vec); } @@ -1586,7 +1591,7 @@ pub(crate) mod tests { schema_descr: Arc, ) -> Result { let mut reader = ThriftSliceInputProtocol::new(buf); - crate::file::metadata::thrift::read_row_group(&mut reader, &schema_descr) + crate::file::metadata::thrift::read_row_group(&mut reader, &schema_descr, 0) } pub(crate) fn read_column_chunk( From a5858df0b77cb658fe78d747fec46a1b9b411705 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 15:20:22 +0200 Subject: [PATCH 13/48] Rename to first_row_index consistently, remove Option. --- parquet/src/arrow/array_reader/row_number.rs | 6 ++---- parquet/src/file/metadata/mod.rs | 11 +++++------ parquet/src/file/metadata/thrift/mod.rs | 10 +++++----- 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index 551f4fb061e3..5484f8d14186 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -34,10 +34,8 @@ impl RowNumberReader { ) -> Result { let ranges = row_groups .map(|rg| { - let first_row_number = rg.first_row_index().ok_or(ParquetError::General( - "Row group missing row number".to_string(), - ))?; - Ok(first_row_number..first_row_number + rg.num_rows()) + let first_row_index = rg.first_row_index(); + Ok(first_row_index..first_row_index + rg.num_rows()) }) .collect::>>()?; Ok(Self { diff --git a/parquet/src/file/metadata/mod.rs b/parquet/src/file/metadata/mod.rs index 18af946eb819..7d0590ed8013 100644 --- a/parquet/src/file/metadata/mod.rs +++ b/parquet/src/file/metadata/mod.rs @@ -616,7 +616,7 @@ pub type RowGroupMetaDataPtr = Arc; pub struct RowGroupMetaData { columns: Vec, num_rows: i64, - first_row_index: Option, + first_row_index: i64, sorting_columns: Option>, total_byte_size: i64, schema_descr: SchemaDescPtr, @@ -658,7 +658,7 @@ impl RowGroupMetaData { } /// Returns the global index number for the first row in this row group. - pub fn first_row_index(&self) -> Option { + pub fn first_row_index(&self) -> i64 { self.first_row_index } @@ -719,7 +719,7 @@ impl RowGroupMetaDataBuilder { schema_descr, file_offset: None, num_rows: 0, - first_row_index: None, + first_row_index: 0, sorting_columns: None, total_byte_size: 0, ordinal: None, @@ -733,8 +733,8 @@ impl RowGroupMetaDataBuilder { } /// Sets the first row number in this row group. - pub fn set_first_row_number(mut self, value: i64) -> Self { - self.0.first_row_index = Some(value); + pub fn set_first_row_index(mut self, value: i64) -> Self { + self.0.first_row_index = value; self } @@ -1632,7 +1632,6 @@ mod tests { .set_total_byte_size(2000) .set_column_metadata(columns) .set_ordinal(1) - .set_first_row_number(10) .build() .unwrap(); diff --git a/parquet/src/file/metadata/thrift/mod.rs b/parquet/src/file/metadata/thrift/mod.rs index 28e0aa332740..b157c5fe45be 100644 --- a/parquet/src/file/metadata/thrift/mod.rs +++ b/parquet/src/file/metadata/thrift/mod.rs @@ -585,11 +585,11 @@ fn read_column_chunk<'a>( fn read_row_group( prot: &mut ThriftSliceInputProtocol, schema_descr: &Arc, - first_row_number: i64, + first_row_index: i64, ) -> Result { // create default initialized RowGroupMetaData let mut row_group = RowGroupMetaDataBuilder::new(schema_descr.clone()).build_unchecked(); - row_group.first_row_index = Some(first_row_number); + row_group.first_row_index = first_row_index; // mask values for required fields const RG_COLUMNS: u8 = 1 << 1; @@ -728,10 +728,10 @@ pub(crate) fn parquet_metadata_from_bytes(buf: &[u8]) -> Result let schema_descr = schema_descr.as_ref().unwrap(); let list_ident = prot.read_list_begin()?; let mut rg_vec = Vec::with_capacity(list_ident.size as usize); - let mut first_row_number = 0i64; + let mut first_row_index = 0i64; for _ in 0..list_ident.size { - let rg = read_row_group(&mut prot, schema_descr, first_row_number)?; - first_row_number += rg.num_rows(); + let rg = read_row_group(&mut prot, schema_descr, first_row_index)?; + first_row_index += rg.num_rows(); rg_vec.push(rg); } row_groups = Some(rg_vec); From 5e7d9a12b222e7c4a04339761f0bfec9a74c4238 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 15:25:04 +0200 Subject: [PATCH 14/48] revert parquet-testing update --- parquet-testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-testing b/parquet-testing index f4d7ed772a62..a3d96a65e11e 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit f4d7ed772a62a95111db50fbcad2460833e8c882 +Subproject commit a3d96a65e11e2bbca7d22a894e8313ede90a33a3 From 54c22c65317185aedbcd28b1b5210bbe296c8105 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 15:27:34 +0200 Subject: [PATCH 15/48] Fix baselines in file::metadata::tests::test_memory_size --- parquet/src/file/metadata/mod.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/parquet/src/file/metadata/mod.rs b/parquet/src/file/metadata/mod.rs index 7d0590ed8013..7afc8045c40d 100644 --- a/parquet/src/file/metadata/mod.rs +++ b/parquet/src/file/metadata/mod.rs @@ -1888,10 +1888,10 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let base_expected_size = 2248; + let base_expected_size = 2256; #[cfg(feature = "encryption")] // Not as accurate as it should be: https://github.com/apache/arrow-rs/issues/8472 - let base_expected_size = 2416; + let base_expected_size = 2424; assert_eq!(parquet_meta.memory_size(), base_expected_size); @@ -1920,10 +1920,10 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let bigger_expected_size = 2674; + let bigger_expected_size = 2682; #[cfg(feature = "encryption")] // Not as accurate as it should be: https://github.com/apache/arrow-rs/issues/8472 - let bigger_expected_size = 2842; + let bigger_expected_size = 2850; // more set fields means more memory usage assert!(bigger_expected_size > base_expected_size); From f05d470e9ff13cbb0797eb8b9af4c0245f7f4d7a Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 23 Oct 2025 16:40:14 +0200 Subject: [PATCH 16/48] Fix encryption metadata and async tests. Those features and default feature tests pass --- parquet/src/arrow/async_reader/mod.rs | 1 + parquet/src/file/metadata/thrift/encryption.rs | 14 ++++++++++---- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 9b9a38ccf744..0f9df2f77509 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -2464,6 +2464,7 @@ mod tests { offset: None, metrics: ArrowReaderMetrics::disabled(), max_predicate_cache_size: 0, + row_number_column: None, }; // Provide an output projection that also selects the same nested leaf diff --git a/parquet/src/file/metadata/thrift/encryption.rs b/parquet/src/file/metadata/thrift/encryption.rs index 9744f0f7a6b5..240071292680 100644 --- a/parquet/src/file/metadata/thrift/encryption.rs +++ b/parquet/src/file/metadata/thrift/encryption.rs @@ -113,6 +113,7 @@ pub(crate) struct FileCryptoMetaData<'a> { fn row_group_from_encrypted_thrift( mut rg: RowGroupMetaData, decryptor: Option<&FileDecryptor>, + first_row_index: i64, ) -> Result { let schema_descr = rg.schema_descr; @@ -192,6 +193,7 @@ fn row_group_from_encrypted_thrift( Ok(RowGroupMetaData { columns, num_rows, + first_row_index, sorting_columns, total_byte_size, schema_descr, @@ -294,10 +296,14 @@ pub(crate) fn parquet_metadata_with_encryption( } // decrypt column chunk info - let row_groups = row_groups - .into_iter() - .map(|rg| row_group_from_encrypted_thrift(rg, file_decryptor.as_ref())) - .collect::>>()?; + let mut first_row_index = 0i64; + let mut decrypted_row_groups = Vec::with_capacity(row_groups.len()); + for rg in row_groups { + let decrypted_rg = row_group_from_encrypted_thrift(rg, file_decryptor.as_ref(), first_row_index)?; + first_row_index += decrypted_rg.num_rows(); + decrypted_row_groups.push(decrypted_rg); + } + let row_groups = decrypted_row_groups; let metadata = ParquetMetaDataBuilder::new(file_metadata) .set_row_groups(row_groups) From 11e4f398076bac2dd6980b95e84bcb2f40f35ea7 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 24 Oct 2025 13:41:54 +0200 Subject: [PATCH 17/48] RowNumber extension type --- parquet/src/arrow/schema/mod.rs | 1 + parquet/src/arrow/schema/row_number.rs | 131 +++++++++++++++++++++++++ 2 files changed, 132 insertions(+) create mode 100644 parquet/src/arrow/schema/row_number.rs diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index 75603ac86693..2622a8b4afba 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -34,6 +34,7 @@ use crate::schema::types::{ColumnDescriptor, SchemaDescriptor, Type}; mod complex; mod extension; +mod row_number; mod primitive; use super::PARQUET_FIELD_ID_META_KEY; diff --git a/parquet/src/arrow/schema/row_number.rs b/parquet/src/arrow/schema/row_number.rs new file mode 100644 index 000000000000..96059b7ce950 --- /dev/null +++ b/parquet/src/arrow/schema/row_number.rs @@ -0,0 +1,131 @@ +// 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. + +//! RowNumber +//! + +use arrow_schema::{ArrowError, DataType, extension::ExtensionType}; + +/// The extension type for `8-bit Boolean`. +/// +/// Extension name: `arrow.row_number`. +/// +/// The storage type of the extension is `Int64`. +/// +#[derive(Debug, Default, Clone, Copy, PartialEq)] +pub struct RowNumber; + +impl ExtensionType for RowNumber { + const NAME: &'static str = "arrow.row_number"; // TODO @vustef: What should it be named? + + type Metadata = &'static str; + + fn metadata(&self) -> &Self::Metadata { + &"" + } + + fn serialize_metadata(&self) -> Option { + Some(String::default()) + } + + fn deserialize_metadata(metadata: Option<&str>) -> Result { + if metadata.is_some_and(str::is_empty) { + Ok("") + } else { + Err(ArrowError::InvalidArgumentError( + "RowNumber extension type expects an empty string as metadata".to_owned(), + )) + } + } + + fn supports_data_type(&self, data_type: &DataType) -> Result<(), ArrowError> { + match data_type { + DataType::Int64 => Ok(()), + data_type => Err(ArrowError::InvalidArgumentError(format!( + "RowNumber data type mismatch, expected Int64, found {data_type}" + ))), + } + } + + fn try_new(data_type: &DataType, _metadata: Self::Metadata) -> Result { + Self.supports_data_type(data_type).map(|_| Self) + } +} + +#[cfg(test)] +mod tests { + use arrow_schema::{ + ArrowError, DataType, Field, + extension::{EXTENSION_TYPE_METADATA_KEY, EXTENSION_TYPE_NAME_KEY}, + }; + + use super::*; + + #[test] + fn valid() -> Result<(), ArrowError> { + let mut field = Field::new("", DataType::Int64, false); + field.try_with_extension_type(RowNumber)?; + field.try_extension_type::()?; + + Ok(()) + } + + #[test] + #[should_panic(expected = "Field extension type name missing")] + fn missing_name() { + let field = Field::new("", DataType::Int64, false).with_metadata( + [(EXTENSION_TYPE_METADATA_KEY.to_owned(), "".to_owned())] + .into_iter() + .collect(), + ); + field.extension_type::(); + } + + #[test] + #[should_panic(expected = "expected Int64, found Int32")] + fn invalid_type() { + Field::new("", DataType::Int32, false).with_extension_type(RowNumber); + } + + #[test] + #[should_panic(expected = "RowNumber extension type expects an empty string as metadata")] + fn missing_metadata() { + let field = Field::new("", DataType::Int64, false).with_metadata( + [(EXTENSION_TYPE_NAME_KEY.to_owned(), RowNumber::NAME.to_owned())] + .into_iter() + .collect(), + ); + field.extension_type::(); + } + + #[test] + #[should_panic(expected = "RowNumber extension type expects an empty string as metadata")] + fn invalid_metadata() { + let field = Field::new("", DataType::Int64, false).with_metadata( + [ + (EXTENSION_TYPE_NAME_KEY.to_owned(), RowNumber::NAME.to_owned()), + ( + EXTENSION_TYPE_METADATA_KEY.to_owned(), + "non-empty".to_owned(), + ), + ] + .into_iter() + .collect(), + ); + field.extension_type::(); + } +} From d02c97730043a46ab6b2c7bc300923593f92c9b7 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 24 Oct 2025 17:13:34 +0200 Subject: [PATCH 18/48] using supplied_schema works --- parquet/src/arrow/array_reader/builder.rs | 9 +- parquet/src/arrow/arrow_reader/mod.rs | 10 +- parquet/src/arrow/schema/complex.rs | 157 ++++++++++++++---- parquet/src/arrow/schema/mod.rs | 4 +- .../schema/{row_number.rs => virtual_type.rs} | 14 +- 5 files changed, 155 insertions(+), 39 deletions(-) rename parquet/src/arrow/schema/{row_number.rs => virtual_type.rs} (87%) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index da6d0df90fdf..9c765fc9ca95 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -32,7 +32,7 @@ use crate::arrow::array_reader::{ make_byte_array_reader, }; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; -use crate::arrow::schema::{ParquetField, ParquetFieldType}; +use crate::arrow::schema::{ParquetField, ParquetFieldType, VirtualColumnType}; use crate::basic::Type as PhysicalType; use crate::data_type::{BoolType, DoubleType, FloatType, Int32Type, Int64Type, Int96Type}; use crate::errors::{ParquetError, Result}; @@ -167,6 +167,13 @@ impl<'a> ArrayReaderBuilder<'a> { Ok(Some(reader)) } } + ParquetFieldType::Virtual(virtual_type) => { + // Virtual columns don't have data in the parquet file + // They need to be built by specialized readers + match virtual_type { + VirtualColumnType::RowNumber => Ok(Some(self.build_row_number_reader()?)), + } + } ParquetFieldType::Group { .. } => match &field.arrow_type { DataType::Map(_, _) => self.build_map_reader(field, mask), DataType::Struct(_) => self.build_struct_reader(field, mask, row_number_column), diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 1ab2b8bab4a0..902e86dc1f64 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1203,7 +1203,7 @@ pub(crate) mod tests { ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, }; - use crate::arrow::schema::add_encoded_arrow_schema_to_metadata; + use crate::arrow::schema::{add_encoded_arrow_schema_to_metadata, virtual_type::RowNumber}; use crate::arrow::{ArrowWriter, ProjectionMask}; use crate::basic::{ConvertedType, Encoding, Repetition, Type as PhysicalType}; use crate::column::reader::decoder::REPETITION_LEVELS_BATCH_SIZE; @@ -5037,7 +5037,10 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let supplied_fields = Fields::from(vec![Field::new("value", ArrowDataType::Int64, false)]); + let supplied_fields = Fields::from(vec![ + Field::new("value", ArrowDataType::Int64, false), + Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber), + ]); let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); let mut arrow_reader = ParquetRecordBatchReaderBuilder::try_new_with_options( @@ -5045,14 +5048,13 @@ pub(crate) mod tests { options, ) .expect("reader builder with schema") - .with_row_number_column("row_number") .build() .expect("reader with schema"); let batch = arrow_reader.next().unwrap().unwrap(); let schema = Arc::new(Schema::new(vec![ Field::new("value", ArrowDataType::Int64, false), - Field::new("row_number", ArrowDataType::Int64, false), + Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber), ])); assert_eq!(batch.schema(), schema); diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 9622f6270d9e..78bf13a00d4b 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -20,12 +20,14 @@ use std::sync::Arc; use crate::arrow::schema::extension::try_add_extension_type; use crate::arrow::schema::primitive::convert_primitive; +use crate::arrow::schema::virtual_type::{RowNumber, is_virtual_column}; use crate::arrow::{PARQUET_FIELD_ID_META_KEY, ProjectionMask}; use crate::basic::{ConvertedType, Repetition}; use crate::errors::ParquetError; use crate::errors::Result; use crate::schema::types::{SchemaDescriptor, Type, TypePtr}; use arrow_schema::{DataType, Field, Fields, SchemaBuilder}; +use arrow_schema::extension::ExtensionType; fn get_repetition(t: &Type) -> Repetition { let info = t.get_basic_info(); @@ -77,10 +79,18 @@ impl ParquetField { match &self.field_type { ParquetFieldType::Primitive { .. } => None, ParquetFieldType::Group { children } => Some(children), + ParquetFieldType::Virtual(_) => None, } } } +/// Types of virtual columns that can be computed at read time +#[derive(Debug, Clone, Copy, PartialEq)] +pub enum VirtualColumnType { + /// Row number within the file + RowNumber, +} + #[derive(Debug, Clone)] pub enum ParquetFieldType { Primitive { @@ -92,6 +102,9 @@ pub enum ParquetFieldType { Group { children: Vec, }, + /// Virtual column that doesn't exist in the parquet file + /// but is computed at read time (e.g., row_number) + Virtual(VirtualColumnType), } /// Encodes the context of the parent of the field currently under consideration @@ -173,17 +186,35 @@ impl Visitor { let parquet_fields = struct_type.get_fields(); - // Extract any arrow fields from the hints - let arrow_fields = match &context.data_type { + // Virtual columns are only supported at the root level (def_level == 0 && rep_level == 0) + let allow_virtual_columns = def_level == 0 && rep_level == 0; + + // Extract any arrow fields from the hints and compute capacity + let (arrow_fields, capacity) = match &context.data_type { Some(DataType::Struct(fields)) => { - if fields.len() != parquet_fields.len() { + // Check length after filtering out virtual columns (only at root level) + let non_virtual_count = if allow_virtual_columns { + fields.iter() + .filter(|field| !is_virtual_column(field)) + .count() + } else { + // Verify no virtual columns exist at non-root levels + if fields.iter().any(|field| is_virtual_column(field)) { + return Err(arrow_err!( + "virtual columns are only supported at the root level of the schema" + )); + } + fields.len() + }; + + if non_virtual_count != parquet_fields.len() { return Err(arrow_err!( - "incompatible arrow schema, expected {} struct fields got {}", + "incompatible arrow schema, expected {} struct fields got {} (after filtering virtual columns)", parquet_fields.len(), - fields.len() + non_virtual_count )); } - Some(fields) + (Some(fields), fields.len()) } Some(d) => { return Err(arrow_err!( @@ -191,41 +222,67 @@ impl Visitor { d )); } - None => None, + None => (None, parquet_fields.len()), }; - let mut child_fields = SchemaBuilder::with_capacity(parquet_fields.len()); - let mut children = Vec::with_capacity(parquet_fields.len()); + let mut child_fields = SchemaBuilder::with_capacity(capacity); + let mut children = Vec::with_capacity(capacity); // Perform a DFS of children - for (idx, parquet_field) in parquet_fields.iter().enumerate() { - let data_type = match arrow_fields { - Some(fields) => { - let field = &fields[idx]; - if field.name() != parquet_field.name() { + if let Some(fields) = arrow_fields { + let mut parquet_idx = 0; + for arrow_field in fields.iter() { + if is_virtual_column(arrow_field) { + // Handle virtual column - create a ParquetField for it + let virtual_parquet_field = convert_virtual_field(arrow_field, rep_level, def_level)?; + child_fields.push(arrow_field.clone()); + children.push(virtual_parquet_field); + } else { + // Non-virtual column - match with parquet field + if parquet_idx >= parquet_fields.len() { + return Err(arrow_err!( + "incompatible arrow schema, more non-virtual fields than parquet fields" + )); + } + + let parquet_field = &parquet_fields[parquet_idx]; + if arrow_field.name() != parquet_field.name() { return Err(arrow_err!( "incompatible arrow schema, expected field named {} got {}", parquet_field.name(), - field.name() + arrow_field.name() )); } - Some(field.data_type().clone()) - } - None => None, - }; - let arrow_field = arrow_fields.map(|x| &*x[idx]); - let child_ctx = VisitorContext { - rep_level, - def_level, - data_type, - }; + let child_ctx = VisitorContext { + rep_level, + def_level, + data_type: Some(arrow_field.data_type().clone()), + }; + + if let Some(mut child) = self.dispatch(parquet_field, child_ctx)? { + // The child type returned may be different from what is encoded in the arrow + // schema in the event of a mismatch or a projection + child_fields.push(convert_field(parquet_field, &mut child, Some(arrow_field))?); + children.push(child); + } - if let Some(mut child) = self.dispatch(parquet_field, child_ctx)? { - // The child type returned may be different from what is encoded in the arrow - // schema in the event of a mismatch or a projection - child_fields.push(convert_field(parquet_field, &mut child, arrow_field)?); - children.push(child); + parquet_idx += 1; + } + } + } else { + // No arrow fields provided - process all parquet fields + for parquet_field in parquet_fields.iter() { + let child_ctx = VisitorContext { + rep_level, + def_level, + data_type: None, + }; + + if let Some(mut child) = self.dispatch(parquet_field, child_ctx)? { + child_fields.push(convert_field(parquet_field, &mut child, None)?); + children.push(child); + } } } @@ -541,6 +598,46 @@ impl Visitor { } } +/// Converts a virtual Arrow [`Field`] to a [`ParquetField`] +/// +/// Virtual fields don't correspond to any data in the parquet file, +/// but are computed at read time (e.g., row_number) +/// +/// The levels are computed based on the parent context: +/// - If nullable: def_level = parent_def_level + 1 +/// - If required: def_level = parent_def_level +/// - rep_level = parent_rep_level (virtual fields are not repeated) +fn convert_virtual_field( + arrow_field: &Field, + parent_rep_level: i16, + parent_def_level: i16, +) -> Result { + let nullable = arrow_field.is_nullable(); + let def_level = if nullable { + parent_def_level + 1 + } else { + parent_def_level + }; + + // Determine the virtual column type based on the extension type + let virtual_type = if arrow_field.try_extension_type::().is_ok() { + VirtualColumnType::RowNumber + } else { + return Err(ParquetError::ArrowError(format!( + "unsupported virtual column type for field '{}'", + arrow_field.name() + ))); + }; + + Ok(ParquetField { + rep_level: parent_rep_level, + def_level, + nullable, + arrow_type: arrow_field.data_type().clone(), + field_type: ParquetFieldType::Virtual(virtual_type), + }) +} + /// Computes the Arrow [`Field`] for a child column /// /// The resulting Arrow [`Field`] will have the type dictated by the Parquet `field`, a name diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index 2622a8b4afba..1100223fd1a3 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -34,7 +34,7 @@ use crate::schema::types::{ColumnDescriptor, SchemaDescriptor, Type}; mod complex; mod extension; -mod row_number; +pub mod virtual_type; mod primitive; use super::PARQUET_FIELD_ID_META_KEY; @@ -43,7 +43,7 @@ use crate::arrow::schema::extension::{ has_extension_type, logical_type_for_fixed_size_binary, logical_type_for_string, logical_type_for_struct, try_add_extension_type, }; -pub(crate) use complex::{ParquetField, ParquetFieldType}; +pub(crate) use complex::{ParquetField, ParquetFieldType, VirtualColumnType}; /// Convert Parquet schema to Arrow schema including optional metadata /// diff --git a/parquet/src/arrow/schema/row_number.rs b/parquet/src/arrow/schema/virtual_type.rs similarity index 87% rename from parquet/src/arrow/schema/row_number.rs rename to parquet/src/arrow/schema/virtual_type.rs index 96059b7ce950..df8fbc013ccc 100644 --- a/parquet/src/arrow/schema/row_number.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -18,7 +18,7 @@ //! RowNumber //! -use arrow_schema::{ArrowError, DataType, extension::ExtensionType}; +use arrow_schema::{ArrowError, DataType, Field, extension::ExtensionType}; /// The extension type for `8-bit Boolean`. /// @@ -30,7 +30,7 @@ use arrow_schema::{ArrowError, DataType, extension::ExtensionType}; pub struct RowNumber; impl ExtensionType for RowNumber { - const NAME: &'static str = "arrow.row_number"; // TODO @vustef: What should it be named? + const NAME: &'static str = "arrow.virtual.row_number"; // TODO @vustef: What should it be named? type Metadata = &'static str; @@ -129,3 +129,13 @@ mod tests { field.extension_type::(); } } + +/// Returns `true` if the field is a virtual column. +/// +/// Virtual columns have extension type names starting with `arrow.virtual.`. +pub fn is_virtual_column(field: &Field) -> bool { + // TODO @vustef: Make this more typed through another approach that doesn't rely on a naming convention. + field.extension_type_name() + .map(|name| name.starts_with("arrow.virtual.")) + .unwrap_or(false) +} From 6fecc17e7952ea3918b6131b223bed7ea4734c11 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 24 Oct 2025 20:48:55 +0200 Subject: [PATCH 19/48] Don't modify parsing of parquet schema, virtual columns can only be added later --- parquet/src/arrow/arrow_reader/mod.rs | 155 ++++++++++++++++++++++++-- parquet/src/arrow/schema/complex.rs | 104 +++++------------ 2 files changed, 174 insertions(+), 85 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 902e86dc1f64..743d03b1a410 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -20,7 +20,7 @@ use arrow_array::Array; use arrow_array::cast::AsArray; use arrow_array::{RecordBatch, RecordBatchReader}; -use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef}; +use arrow_schema::{ArrowError, DataType as ArrowType, Field, Fields, Schema, SchemaRef}; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; @@ -28,7 +28,7 @@ use std::sync::Arc; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; -use crate::arrow::schema::{ParquetField, parquet_to_arrow_schema_and_fields}; +use crate::arrow::schema::{ParquetField, ParquetFieldType, parquet_to_arrow_schema_and_fields}; use crate::arrow::{FieldLevels, ProjectionMask, parquet_to_arrow_field_levels}; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; use crate::bloom_filter::{ @@ -382,6 +382,134 @@ impl ArrowReaderBuilder { ..self } } + + /// Include additional fields by appending them at the end of the fields. + /// + /// This will add columns to the output record batch. + pub fn with_fields_appended(self, fields_to_append: Vec) -> Result { + use crate::arrow::schema::virtual_type::{RowNumber, is_virtual_column}; + use crate::arrow::schema::VirtualColumnType; + use arrow_schema::extension::ExtensionType; + + if fields_to_append.is_empty() { + return Ok(self); + } + + // Verify all fields to append are virtual columns + for field in &fields_to_append { + if !is_virtual_column(field) { + return Err(ParquetError::ArrowError( + "Only virtual columns can be appended via with_fields_appended".to_string() + )); + } + } + + // Create extended schema + let mut all_fields: Vec = Vec::new(); + all_fields.extend(self.schema.fields().iter().map(|f| f.as_ref().clone())); + all_fields.extend(fields_to_append.iter().cloned()); + // TODO @vustef: Preserve metadata in the schema... + let extended_schema = Arc::new(Schema::new(all_fields)); + + // Get or create root ParquetField + let mut root_field = if self.schema.fields().is_empty() { + // If schema is empty, create an empty root group + ParquetField { + rep_level: 0, + def_level: 0, + nullable: false, + arrow_type: ArrowType::Struct(Fields::empty()), + field_type: ParquetFieldType::Group { children: vec![] }, + } + } else if let Some(fields) = self.fields { + // Unwrap Arc and clone if needed + Arc::try_unwrap(fields).unwrap_or_else(|arc| (*arc).clone()) + } else { + // If no fields exist yet, create root group from parquet schema + let parquet_schema = self.metadata.file_metadata().schema_descr(); + let field_levels = parquet_to_arrow_field_levels( + parquet_schema, + ProjectionMask::all(), + None, + )?; + field_levels.levels.ok_or_else(|| { + ParquetError::ArrowError("Failed to create ParquetField from schema".to_string()) + })? + }; + + // Convert virtual fields to ParquetFields and append them + match &mut root_field.field_type { + ParquetFieldType::Group { children } => { + for field in &fields_to_append { + // Determine virtual column type + let virtual_type = if field.try_extension_type::().is_ok() { + VirtualColumnType::RowNumber + } else { + return Err(ParquetError::ArrowError(format!( + "Unsupported virtual column type for field '{}'", + field.name() + ))); + }; + + let nullable = field.is_nullable(); + // TODO @vustef: Assert def_level and rep_level are 0? + let parquet_field = ParquetField { + rep_level: 0, + def_level: if nullable { 1 } else { 0 }, + nullable, + arrow_type: field.data_type().clone(), + field_type: ParquetFieldType::Virtual(virtual_type), + }; + + children.push(parquet_field); + } + + // Update the root field's arrow_type to match the extended schema + root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); + } + ParquetFieldType::Primitive { .. } => { + // Root should never be primitive, but handle it by wrapping in a Group + let existing_field = root_field.clone(); + let mut children = vec![existing_field]; + + for field in &fields_to_append { + let virtual_type = if field.try_extension_type::().is_ok() { + VirtualColumnType::RowNumber + } else { + return Err(ParquetError::ArrowError(format!( + "Unsupported virtual column type for field '{}'", + field.name() + ))); + }; + + let nullable = field.is_nullable(); + let parquet_field = ParquetField { + rep_level: 0, + def_level: if nullable { 1 } else { 0 }, + nullable, + arrow_type: field.data_type().clone(), + field_type: ParquetFieldType::Virtual(virtual_type), + }; + + children.push(parquet_field); + } + + root_field.field_type = ParquetFieldType::Group { children }; + root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); + } + ParquetFieldType::Virtual(_) => { + return Err(ParquetError::ArrowError( + "Root field cannot be a Virtual column".to_string() + )); + } + } + + Ok(Self { + schema: extended_schema, + fields: Some(Arc::new(root_field)), + ..self + }) + } } /// Options that control how metadata is read for a parquet file @@ -5039,22 +5167,25 @@ pub(crate) mod tests { )]); let supplied_fields = Fields::from(vec![ Field::new("value", ArrowDataType::Int64, false), - Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber), ]); + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); let mut arrow_reader = ParquetRecordBatchReaderBuilder::try_new_with_options( file.try_clone().unwrap(), options, ) .expect("reader builder with schema") + .with_fields_appended(vec![row_number_field.clone()]) + .expect("with_fields_appended should succeed") .build() .expect("reader with schema"); let batch = arrow_reader.next().unwrap().unwrap(); let schema = Arc::new(Schema::new(vec![ Field::new("value", ArrowDataType::Int64, false), - Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber), + row_number_field, ])); assert_eq!(batch.schema(), schema); @@ -5085,19 +5216,21 @@ pub(crate) mod tests { Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); let mut metadata = ArrowReaderMetadata::load(&file, Default::default()).unwrap(); - metadata.fields = None; + metadata.fields = None; // TODO @vustef: Work out how to best trigger empty schema, only virtual columns...what would return empty results? And are both needed? + metadata.schema = Arc::new(Schema::empty()); + + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); let mut arrow_reader = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata) - .with_row_number_column("row_number") + .with_fields_appended(vec![row_number_field.clone()]) + .expect("with_fields_appended should succeed") .build() .expect("reader with schema"); let batch = arrow_reader.next().unwrap().unwrap(); - let schema = Arc::new(Schema::new(vec![Field::new( - "row_number", - ArrowDataType::Int64, - false, - )])); + let schema = Arc::new(Schema::new(vec![ + row_number_field, + ])); assert_eq!(batch.schema(), schema); assert_eq!(batch.num_columns(), 1); diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 78bf13a00d4b..3cab3081dae3 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -186,35 +186,17 @@ impl Visitor { let parquet_fields = struct_type.get_fields(); - // Virtual columns are only supported at the root level (def_level == 0 && rep_level == 0) - let allow_virtual_columns = def_level == 0 && rep_level == 0; - - // Extract any arrow fields from the hints and compute capacity - let (arrow_fields, capacity) = match &context.data_type { + // Extract any arrow fields from the hints + let arrow_fields = match &context.data_type { Some(DataType::Struct(fields)) => { - // Check length after filtering out virtual columns (only at root level) - let non_virtual_count = if allow_virtual_columns { - fields.iter() - .filter(|field| !is_virtual_column(field)) - .count() - } else { - // Verify no virtual columns exist at non-root levels - if fields.iter().any(|field| is_virtual_column(field)) { - return Err(arrow_err!( - "virtual columns are only supported at the root level of the schema" - )); - } - fields.len() - }; - - if non_virtual_count != parquet_fields.len() { + if fields.len() != parquet_fields.len() { return Err(arrow_err!( - "incompatible arrow schema, expected {} struct fields got {} (after filtering virtual columns)", + "incompatible arrow schema, expected {} struct fields got {}", parquet_fields.len(), - non_virtual_count + fields.len() )); } - (Some(fields), fields.len()) + Some(fields) } Some(d) => { return Err(arrow_err!( @@ -222,67 +204,41 @@ impl Visitor { d )); } - None => (None, parquet_fields.len()), + None => None, }; - let mut child_fields = SchemaBuilder::with_capacity(capacity); - let mut children = Vec::with_capacity(capacity); + let mut child_fields = SchemaBuilder::with_capacity(parquet_fields.len()); + let mut children = Vec::with_capacity(parquet_fields.len()); // Perform a DFS of children - if let Some(fields) = arrow_fields { - let mut parquet_idx = 0; - for arrow_field in fields.iter() { - if is_virtual_column(arrow_field) { - // Handle virtual column - create a ParquetField for it - let virtual_parquet_field = convert_virtual_field(arrow_field, rep_level, def_level)?; - child_fields.push(arrow_field.clone()); - children.push(virtual_parquet_field); - } else { - // Non-virtual column - match with parquet field - if parquet_idx >= parquet_fields.len() { - return Err(arrow_err!( - "incompatible arrow schema, more non-virtual fields than parquet fields" - )); - } - - let parquet_field = &parquet_fields[parquet_idx]; - if arrow_field.name() != parquet_field.name() { + for (idx, parquet_field) in parquet_fields.iter().enumerate() { + let data_type = match arrow_fields { + Some(fields) => { + let field = &fields[idx]; + if field.name() != parquet_field.name() { return Err(arrow_err!( "incompatible arrow schema, expected field named {} got {}", parquet_field.name(), - arrow_field.name() + field.name() )); } - - let child_ctx = VisitorContext { - rep_level, - def_level, - data_type: Some(arrow_field.data_type().clone()), - }; - - if let Some(mut child) = self.dispatch(parquet_field, child_ctx)? { - // The child type returned may be different from what is encoded in the arrow - // schema in the event of a mismatch or a projection - child_fields.push(convert_field(parquet_field, &mut child, Some(arrow_field))?); - children.push(child); - } - - parquet_idx += 1; + Some(field.data_type().clone()) } - } - } else { - // No arrow fields provided - process all parquet fields - for parquet_field in parquet_fields.iter() { - let child_ctx = VisitorContext { - rep_level, - def_level, - data_type: None, - }; + None => None, + }; - if let Some(mut child) = self.dispatch(parquet_field, child_ctx)? { - child_fields.push(convert_field(parquet_field, &mut child, None)?); - children.push(child); - } + let arrow_field = arrow_fields.map(|x| &*x[idx]); + let child_ctx = VisitorContext { + rep_level, + def_level, + data_type, + }; + + if let Some(mut child) = self.dispatch(parquet_field, child_ctx)? { + // The child type returned may be different from what is encoded in the arrow + // schema in the event of a mismatch or a projection + child_fields.push(convert_field(parquet_field, &mut child, arrow_field)?); + children.push(child); } } From 1414421aacd7b46e7fce100371a3dd8d82569653 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 09:09:54 +0100 Subject: [PATCH 20/48] Reworked with_virtual_columns in options --- parquet/examples/read_with_rowgroup.rs | 1 + parquet/src/arrow/array_reader/builder.rs | 4 +- parquet/src/arrow/array_reader/list_array.rs | 1 + parquet/src/arrow/array_reader/row_number.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 311 ++++++++++--------- parquet/src/arrow/mod.rs | 2 +- parquet/src/arrow/schema/complex.rs | 24 +- parquet/src/arrow/schema/mod.rs | 8 +- 8 files changed, 199 insertions(+), 154 deletions(-) diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index e2f021d1c97f..8cd6b1a4777c 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -155,6 +155,7 @@ impl InMemoryRowGroup { &self.metadata.schema_descr_ptr(), self.mask.clone(), None, + vec![], )?; ParquetRecordBatchReader::try_new_with_row_groups(&levels, self, batch_size, selection) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 9c765fc9ca95..803dfd586c1c 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -425,7 +425,7 @@ impl<'a> ArrayReaderBuilder<'a> { fn build_struct_reader( &self, field: &ParquetField, - mask: &ProjectionMask, + mask: &ProjectionMask, // TODO @vustef:Don't apply to the virtual columns. row_number_column: Option<&str>, ) -> Result>> { let arrow_fields = match &field.arrow_type { @@ -491,6 +491,7 @@ mod tests { file_metadata.schema_descr(), ProjectionMask::all(), file_metadata.key_value_metadata(), + vec![], ) .unwrap(); @@ -520,6 +521,7 @@ mod tests { file_metadata.schema_descr(), ProjectionMask::all(), file_metadata.key_value_metadata(), + vec![], ) .unwrap(); diff --git a/parquet/src/arrow/array_reader/list_array.rs b/parquet/src/arrow/array_reader/list_array.rs index 3e2bfaa506cc..bf7093a912c2 100644 --- a/parquet/src/arrow/array_reader/list_array.rs +++ b/parquet/src/arrow/array_reader/list_array.rs @@ -561,6 +561,7 @@ mod tests { schema, ProjectionMask::all(), file_metadata.key_value_metadata(), + vec![], ) .unwrap(); diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index 5484f8d14186..fbf47b6b5e23 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -16,7 +16,7 @@ // under the License. use crate::arrow::array_reader::ArrayReader; -use crate::errors::{ParquetError, Result}; +use crate::errors::Result; use crate::file::metadata::RowGroupMetaData; use arrow_array::{ArrayRef, Int64Array}; use arrow_schema::DataType; diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 743d03b1a410..e6d2af3a9fd6 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -20,7 +20,7 @@ use arrow_array::Array; use arrow_array::cast::AsArray; use arrow_array::{RecordBatch, RecordBatchReader}; -use arrow_schema::{ArrowError, DataType as ArrowType, Field, Fields, Schema, SchemaRef}; +use arrow_schema::{ArrowError, DataType as ArrowType, Field, Schema, SchemaRef}; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; @@ -28,7 +28,7 @@ use std::sync::Arc; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; -use crate::arrow::schema::{ParquetField, ParquetFieldType, parquet_to_arrow_schema_and_fields}; +use crate::arrow::schema::{ParquetField, parquet_to_arrow_schema_and_fields}; use crate::arrow::{FieldLevels, ProjectionMask, parquet_to_arrow_field_levels}; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; use crate::bloom_filter::{ @@ -383,133 +383,135 @@ impl ArrowReaderBuilder { } } - /// Include additional fields by appending them at the end of the fields. - /// - /// This will add columns to the output record batch. - pub fn with_fields_appended(self, fields_to_append: Vec) -> Result { - use crate::arrow::schema::virtual_type::{RowNumber, is_virtual_column}; - use crate::arrow::schema::VirtualColumnType; - use arrow_schema::extension::ExtensionType; - - if fields_to_append.is_empty() { - return Ok(self); - } - - // Verify all fields to append are virtual columns - for field in &fields_to_append { - if !is_virtual_column(field) { - return Err(ParquetError::ArrowError( - "Only virtual columns can be appended via with_fields_appended".to_string() - )); - } - } - - // Create extended schema - let mut all_fields: Vec = Vec::new(); - all_fields.extend(self.schema.fields().iter().map(|f| f.as_ref().clone())); - all_fields.extend(fields_to_append.iter().cloned()); - // TODO @vustef: Preserve metadata in the schema... - let extended_schema = Arc::new(Schema::new(all_fields)); - - // Get or create root ParquetField - let mut root_field = if self.schema.fields().is_empty() { - // If schema is empty, create an empty root group - ParquetField { - rep_level: 0, - def_level: 0, - nullable: false, - arrow_type: ArrowType::Struct(Fields::empty()), - field_type: ParquetFieldType::Group { children: vec![] }, - } - } else if let Some(fields) = self.fields { - // Unwrap Arc and clone if needed - Arc::try_unwrap(fields).unwrap_or_else(|arc| (*arc).clone()) - } else { - // If no fields exist yet, create root group from parquet schema - let parquet_schema = self.metadata.file_metadata().schema_descr(); - let field_levels = parquet_to_arrow_field_levels( - parquet_schema, - ProjectionMask::all(), - None, - )?; - field_levels.levels.ok_or_else(|| { - ParquetError::ArrowError("Failed to create ParquetField from schema".to_string()) - })? - }; - - // Convert virtual fields to ParquetFields and append them - match &mut root_field.field_type { - ParquetFieldType::Group { children } => { - for field in &fields_to_append { - // Determine virtual column type - let virtual_type = if field.try_extension_type::().is_ok() { - VirtualColumnType::RowNumber - } else { - return Err(ParquetError::ArrowError(format!( - "Unsupported virtual column type for field '{}'", - field.name() - ))); - }; - - let nullable = field.is_nullable(); - // TODO @vustef: Assert def_level and rep_level are 0? - let parquet_field = ParquetField { - rep_level: 0, - def_level: if nullable { 1 } else { 0 }, - nullable, - arrow_type: field.data_type().clone(), - field_type: ParquetFieldType::Virtual(virtual_type), - }; - - children.push(parquet_field); - } - - // Update the root field's arrow_type to match the extended schema - root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); - } - ParquetFieldType::Primitive { .. } => { - // Root should never be primitive, but handle it by wrapping in a Group - let existing_field = root_field.clone(); - let mut children = vec![existing_field]; - - for field in &fields_to_append { - let virtual_type = if field.try_extension_type::().is_ok() { - VirtualColumnType::RowNumber - } else { - return Err(ParquetError::ArrowError(format!( - "Unsupported virtual column type for field '{}'", - field.name() - ))); - }; - - let nullable = field.is_nullable(); - let parquet_field = ParquetField { - rep_level: 0, - def_level: if nullable { 1 } else { 0 }, - nullable, - arrow_type: field.data_type().clone(), - field_type: ParquetFieldType::Virtual(virtual_type), - }; - - children.push(parquet_field); - } - - root_field.field_type = ParquetFieldType::Group { children }; - root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); - } - ParquetFieldType::Virtual(_) => { - return Err(ParquetError::ArrowError( - "Root field cannot be a Virtual column".to_string() - )); - } - } - - Ok(Self { - schema: extended_schema, - fields: Some(Arc::new(root_field)), - ..self - }) - } + // /// Include additional fields by appending them at the end of the fields. + // /// + // /// This will add columns to the output record batch. + // pub fn with_fields_appended(self, fields_to_append: Vec) -> Result { + // use crate::arrow::schema::virtual_type::{RowNumber, is_virtual_column}; + // use crate::arrow::schema::VirtualColumnType; + // use arrow_schema::extension::ExtensionType; + + // if fields_to_append.is_empty() { + // return Ok(self); + // } + + // // Verify all fields to append are virtual columns + // for field in &fields_to_append { + // if !is_virtual_column(field) { + // return Err(ParquetError::ArrowError( + // "Only virtual columns can be appended via with_fields_appended".to_string() + // )); + // } + // } + + // // Create extended schema + // let mut all_fields: Vec = Vec::new(); + // all_fields.extend(self.schema.fields().iter().map(|f| f.as_ref().clone())); + // all_fields.extend(fields_to_append.iter().cloned()); + // // TODO @vustef: Preserve metadata in the schema... + // let extended_schema = Arc::new(Schema::new(all_fields)); + + // // Get or create root ParquetField + // let mut root_field = if self.schema.fields().is_empty() { + // // If schema is empty, create an empty root group + // ParquetField { + // rep_level: 0, + // def_level: 0, + // nullable: false, + // arrow_type: ArrowType::Struct(Fields::empty()), + // field_type: ParquetFieldType::Group { children: vec![] }, + // } + // } else if let Some(fields) = self.fields { + // // Unwrap Arc and clone if needed + // Arc::try_unwrap(fields).unwrap_or_else(|arc| (*arc).clone()) + // } else { + // // If no fields exist yet, create root group from parquet schema + // let parquet_schema = self.metadata.file_metadata().schema_descr(); + // let field_levels = parquet_to_arrow_field_levels( + // parquet_schema, + // ProjectionMask::all(), + // None, + // )?; + // field_levels.levels.ok_or_else(|| { + // ParquetError::ArrowError("Failed to create ParquetField from schema".to_string()) + // })? + // }; + + // // Convert virtual fields to ParquetFields and append them + // match &mut root_field.field_type { + // ParquetFieldType::Group { children } => { + // for field in &fields_to_append { + // // Determine virtual column type + // let virtual_type = if field.try_extension_type::().is_ok() { + // VirtualColumnType::RowNumber + // } else { + // return Err(ParquetError::ArrowError(format!( + // "Unsupported virtual column type for field '{}'", + // field.name() + // ))); + // }; + + // let nullable = field.is_nullable(); + // // TODO @vustef: Assert def_level and rep_level are 0? + // let parquet_field = ParquetField { + // rep_level: 0, + // def_level: if nullable { 1 } else { 0 }, + // nullable, + // arrow_type: field.data_type().clone(), + // field_type: ParquetFieldType::Virtual(virtual_type), + // }; + + // children.push(parquet_field); + // } + + // // Update the root field's arrow_type to match the extended schema + // root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); + // } + // ParquetFieldType::Primitive { .. } => { + // // Root should never be primitive, but handle it by wrapping in a Group + // let existing_field = root_field.clone(); + // let mut children = vec![existing_field]; + + // for field in &fields_to_append { + // let virtual_type = if field.try_extension_type::().is_ok() { + // VirtualColumnType::RowNumber + // } else { + // return Err(ParquetError::ArrowError(format!( + // "Unsupported virtual column type for field '{}'", + // field.name() + // ))); + // }; + + // let nullable = field.is_nullable(); + // let parquet_field = ParquetField { + // rep_level: 0, + // def_level: if nullable { 1 } else { 0 }, + // nullable, + // arrow_type: field.data_type().clone(), + // field_type: ParquetFieldType::Virtual(virtual_type), + // }; + + // children.push(parquet_field); + // } + + // root_field.field_type = ParquetFieldType::Group { children }; + // root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); + // } + // ParquetFieldType::Virtual(_) => { + // return Err(ParquetError::ArrowError( + // "Root field cannot be a Virtual column".to_string() + // )); + // } + // } + + // // TODO @vustef: this makes virtual columns part of the schema, but when creating ProjectionMasks, they wouldn't easily be constructed to include them, since they are based on ParquetMetadata. + // // Maybe masks shouldn't apply to virtual columns at the end of the schema. + // Ok(Self { + // schema: extended_schema, + // fields: Some(Arc::new(root_field)), + // ..self + // }) + // } } /// Options that control how metadata is read for a parquet file @@ -530,6 +532,8 @@ pub struct ArrowReaderOptions { /// If encryption is enabled, the file decryption properties can be provided #[cfg(feature = "encryption")] pub(crate) file_decryption_properties: Option>, + + virtual_columns: Vec } impl ArrowReaderOptions { @@ -658,6 +662,22 @@ impl ArrowReaderOptions { } } + /// Include virtual columns in the output. + /// + /// Virtual columns are columns that are not part of the Parquet schema, but are added to the output by the reader. + /// + /// # Example + /// ``` + /// let virtual_columns = vec![Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)]; + /// let options = ArrowReaderOptions::new().with_virtual_columns(virtual_columns); + pub fn with_virtual_columns(self, virtual_columns: Vec) -> Self { + // TODO @vustef: Make sure the passed fields are virtual columns. + Self { + virtual_columns, + ..self + } + } + /// Retrieve the currently set page index behavior. /// /// This can be set via [`with_page_index`][Self::with_page_index]. @@ -729,7 +749,7 @@ impl ArrowReaderMetadata { /// See [`Self::load`] for more details. pub fn try_new(metadata: Arc, options: ArrowReaderOptions) -> Result { match options.supplied_schema { - Some(supplied_schema) => Self::with_supplied_schema(metadata, supplied_schema.clone()), + Some(supplied_schema) => Self::with_supplied_schema(metadata, supplied_schema.clone(), options.virtual_columns), None => { let kv_metadata = match options.skip_arrow_metadata { true => None, @@ -740,6 +760,7 @@ impl ArrowReaderMetadata { metadata.file_metadata().schema_descr(), ProjectionMask::all(), kv_metadata, + options.virtual_columns, )?; Ok(Self { @@ -754,16 +775,19 @@ impl ArrowReaderMetadata { fn with_supplied_schema( metadata: Arc, supplied_schema: SchemaRef, + virtual_columns: Vec, ) -> Result { let parquet_schema = metadata.file_metadata().schema_descr(); + let virtual_columns_len = virtual_columns.len(); let field_levels = parquet_to_arrow_field_levels( parquet_schema, ProjectionMask::all(), Some(supplied_schema.fields()), + virtual_columns, )?; let fields = field_levels.fields; let inferred_len = fields.len(); - let supplied_len = supplied_schema.fields().len(); + let supplied_len = supplied_schema.fields().len() + virtual_columns_len; // Ensure the supplied schema has the same number of columns as the parquet schema. // parquet_to_arrow_field_levels is expected to throw an error if the schemas have // different lengths, but we check here to be safe. @@ -1305,7 +1329,7 @@ pub(crate) mod tests { use std::sync::Arc; use rand::rngs::StdRng; - use rand::{random, rng, thread_rng, Rng, RngCore, SeedableRng}; + use rand::{random, rng, Rng, RngCore, SeedableRng}; use tempfile::tempfile; use arrow_array::builder::*; @@ -5172,13 +5196,12 @@ pub(crate) mod tests { let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); + let options = options.with_virtual_columns(vec![row_number_field.clone()]); let mut arrow_reader = ParquetRecordBatchReaderBuilder::try_new_with_options( file.try_clone().unwrap(), options, ) .expect("reader builder with schema") - .with_fields_appended(vec![row_number_field.clone()]) - .expect("with_fields_appended should succeed") .build() .expect("reader with schema"); @@ -5215,15 +5238,13 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let mut metadata = ArrowReaderMetadata::load(&file, Default::default()).unwrap(); - metadata.fields = None; // TODO @vustef: Work out how to best trigger empty schema, only virtual columns...what would return empty results? And are both needed? - metadata.schema = Arc::new(Schema::empty()); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field.clone()]); + let metadata = ArrowReaderMetadata::load(&file, options).unwrap(); + let num_columns = metadata.metadata.file_metadata().schema_descr().num_columns(); let mut arrow_reader = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata) - .with_fields_appended(vec![row_number_field.clone()]) - .expect("with_fields_appended should succeed") + .with_projection(ProjectionMask::none(num_columns)) .build() .expect("reader with schema"); @@ -5251,11 +5272,12 @@ pub(crate) mod tests { false, |path, selection, _row_filter, batch_size| { let file = File::open(path).unwrap(); - let reader = ParquetRecordBatchReaderBuilder::try_new(file) + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() .with_row_selection(selection) .with_batch_size(batch_size) - .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader @@ -5271,12 +5293,13 @@ pub(crate) mod tests { true, |path, selection, row_filter, batch_size| { let file = File::open(path).unwrap(); - let reader = ParquetRecordBatchReaderBuilder::try_new(file) + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() .with_row_selection(selection) .with_batch_size(batch_size) .with_row_filter(row_filter.expect("No filter")) - .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 94e359065107..ad14da0e5a45 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -258,7 +258,7 @@ pub struct ProjectionMask { /// A mask of `[true, false, true, false]` will result in a schema 2 /// elements long: /// * `fields[0]`: `a` - /// * `fields[1]`: `c` + /// * `fields[1]`: `c` /// /// A mask of `None` will result in a schema 4 elements long: /// * `fields[0]`: `a` diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 3cab3081dae3..a47e576fe3c0 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use crate::arrow::schema::extension::try_add_extension_type; use crate::arrow::schema::primitive::convert_primitive; -use crate::arrow::schema::virtual_type::{RowNumber, is_virtual_column}; +use crate::arrow::schema::virtual_type::RowNumber; use crate::arrow::{PARQUET_FIELD_ID_META_KEY, ProjectionMask}; use crate::basic::{ConvertedType, Repetition}; use crate::errors::ParquetError; @@ -138,6 +138,8 @@ struct Visitor { /// Mask of columns to include mask: ProjectionMask, + + virtual_columns: Vec, } impl Visitor { @@ -207,8 +209,8 @@ impl Visitor { None => None, }; - let mut child_fields = SchemaBuilder::with_capacity(parquet_fields.len()); - let mut children = Vec::with_capacity(parquet_fields.len()); + let mut child_fields = SchemaBuilder::with_capacity(parquet_fields.len() + self.virtual_columns.len()); + let mut children = Vec::with_capacity(parquet_fields.len() + self.virtual_columns.len()); // Perform a DFS of children for (idx, parquet_field) in parquet_fields.iter().enumerate() { @@ -242,6 +244,17 @@ impl Visitor { } } + // TODO @vustef: Are all parquet schemas going to start with a struct? I.e. is this the only place where + // we need to handle virtual columns? + if rep_level == 0 && def_level == 0 { + // TODO @vustef: assert is_virtual_column ? Or use types to our advantage somehow. + for virtual_column in &self.virtual_columns { + child_fields.push(virtual_column.clone()); + let child = convert_virtual_field(virtual_column, rep_level, def_level)?; + children.push(child); + } + } + if children.is_empty() { return Ok(None); } @@ -577,7 +590,7 @@ fn convert_virtual_field( // Determine the virtual column type based on the extension type let virtual_type = if arrow_field.try_extension_type::().is_ok() { - VirtualColumnType::RowNumber + VirtualColumnType::RowNumber // TODO @vustef: Don't like the ifelse approach... } else { return Err(ParquetError::ArrowError(format!( "unsupported virtual column type for field '{}'", @@ -647,10 +660,12 @@ pub fn convert_schema( schema: &SchemaDescriptor, mask: ProjectionMask, embedded_arrow_schema: Option<&Fields>, + virtual_columns: Vec, ) -> Result> { let mut visitor = Visitor { next_col_idx: 0, mask, + virtual_columns, }; let context = VisitorContext { @@ -667,6 +682,7 @@ pub fn convert_type(parquet_type: &TypePtr) -> Result { let mut visitor = Visitor { next_col_idx: 0, mask: ProjectionMask::all(), + virtual_columns: vec![], // TODO @vustef: Maybe should be None rather? }; let context = VisitorContext { diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index 1100223fd1a3..b03b389f5bb7 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -63,7 +63,7 @@ pub fn parquet_to_arrow_schema_by_columns( mask: ProjectionMask, key_value_metadata: Option<&Vec>, ) -> Result { - Ok(parquet_to_arrow_schema_and_fields(parquet_schema, mask, key_value_metadata)?.0) + Ok(parquet_to_arrow_schema_and_fields(parquet_schema, mask, key_value_metadata, vec![])?.0) } /// Determines the Arrow Schema from a Parquet schema @@ -75,6 +75,7 @@ pub(crate) fn parquet_to_arrow_schema_and_fields( parquet_schema: &SchemaDescriptor, mask: ProjectionMask, key_value_metadata: Option<&Vec>, + virtual_columns: Vec, ) -> Result<(Schema, Option)> { let mut metadata = parse_key_value_metadata(key_value_metadata).unwrap_or_default(); let maybe_schema = metadata @@ -90,7 +91,7 @@ pub(crate) fn parquet_to_arrow_schema_and_fields( } let hint = maybe_schema.as_ref().map(|s| s.fields()); - let field_levels = parquet_to_arrow_field_levels(parquet_schema, mask, hint)?; + let field_levels = parquet_to_arrow_field_levels(parquet_schema, mask, hint, virtual_columns)?; let schema = Schema::new_with_metadata(field_levels.fields, metadata); Ok((schema, field_levels.levels)) } @@ -132,8 +133,9 @@ pub fn parquet_to_arrow_field_levels( schema: &SchemaDescriptor, mask: ProjectionMask, hint: Option<&Fields>, + virtual_columns: Vec, // TODO @vustef: This is a public method, maybe preserve its signature. Or maybe it's good to change it, to be able to construct readers from pub API. ) -> Result { - match complex::convert_schema(schema, mask, hint)? { + match complex::convert_schema(schema, mask, hint, virtual_columns)? { Some(field) => match &field.arrow_type { DataType::Struct(fields) => Ok(FieldLevels { fields: fields.clone(), From 07eb467fff0d0c7d519394a671884185e4806885 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 09:50:23 +0100 Subject: [PATCH 21/48] switch to ref to slice; cleanup with_row_number_columns; async tests passing --- parquet/examples/read_with_rowgroup.rs | 2 +- parquet/src/arrow/array_reader/builder.rs | 67 +++----- parquet/src/arrow/array_reader/list_array.rs | 4 +- parquet/src/arrow/arrow_reader/mod.rs | 160 +------------------ parquet/src/arrow/async_reader/mod.rs | 21 ++- parquet/src/arrow/schema/complex.rs | 12 +- parquet/src/arrow/schema/mod.rs | 6 +- 7 files changed, 51 insertions(+), 221 deletions(-) diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index 8cd6b1a4777c..b4a9a4be1fc5 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -155,7 +155,7 @@ impl InMemoryRowGroup { &self.metadata.schema_descr_ptr(), self.mask.clone(), None, - vec![], + &[], )?; ParquetRecordBatchReader::try_new_with_row_groups(&levels, self, batch_size, selection) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 803dfd586c1c..7b06ebb7ae6d 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -33,6 +33,7 @@ use crate::arrow::array_reader::{ }; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; use crate::arrow::schema::{ParquetField, ParquetFieldType, VirtualColumnType}; +use crate::arrow::schema::virtual_type::RowNumber; use crate::basic::Type as PhysicalType; use crate::data_type::{BoolType, DoubleType, FloatType, Int32Type, Int64Type, Int96Type}; use crate::errors::{ParquetError, Result}; @@ -113,38 +114,24 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: Option<&ParquetField>, mask: &ProjectionMask, - row_number_column: Option<&str>, ) -> Result> { let reader = field - .and_then(|field| self.build_reader(field, mask, row_number_column).transpose()) - .or_else(|| { - row_number_column.map(|column| { - let row_number_reader = self.build_row_number_reader()?; - let reader: Box = Box::new(StructArrayReader::new( - DataType::Struct(Fields::from(vec![Field::new( - column, - row_number_reader.get_data_type().clone(), - false, - )])), - vec![row_number_reader], - 0, - 0, - false, - )); - Ok(reader) - }) - }) - .transpose()? - .unwrap_or_else(|| make_empty_array_reader(self.row_groups.num_rows())); + .and_then(|field| self.build_reader(field, mask).transpose()) + .transpose()? + .unwrap_or_else(|| make_empty_array_reader(self.num_rows())); Ok(reader) } + /// Return the total number of rows + fn num_rows(&self) -> usize { + self.row_groups.num_rows() + } + fn build_reader( &self, field: &ParquetField, mask: &ProjectionMask, - row_number_column: Option<&str>, ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { col_idx, .. } => { @@ -176,7 +163,7 @@ impl<'a> ArrayReaderBuilder<'a> { } ParquetFieldType::Group { .. } => match &field.arrow_type { DataType::Map(_, _) => self.build_map_reader(field, mask), - DataType::Struct(_) => self.build_struct_reader(field, mask, row_number_column), + 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), @@ -198,8 +185,8 @@ impl<'a> ArrayReaderBuilder<'a> { let children = field.children().unwrap(); assert_eq!(children.len(), 2); - let key_reader = self.build_reader(&children[0], mask, None)?; - let value_reader = self.build_reader(&children[1], mask, None)?; + 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)) => { @@ -250,7 +237,7 @@ impl<'a> ArrayReaderBuilder<'a> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, None)? { + 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(); @@ -296,7 +283,7 @@ impl<'a> ArrayReaderBuilder<'a> { let children = field.children().unwrap(); assert_eq!(children.len(), 1); - let reader = match self.build_reader(&children[0], mask, None)? { + 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 { @@ -425,8 +412,7 @@ impl<'a> ArrayReaderBuilder<'a> { fn build_struct_reader( &self, field: &ParquetField, - mask: &ProjectionMask, // TODO @vustef:Don't apply to the virtual columns. - row_number_column: Option<&str>, + mask: &ProjectionMask, ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, @@ -439,7 +425,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, None)? { + 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)); @@ -447,16 +433,6 @@ impl<'a> ArrayReaderBuilder<'a> { } } - if let Some(row_number_column) = row_number_column { - let reader = self.build_row_number_reader()?; - builder.push(Field::new( - row_number_column, - reader.get_data_type().clone(), - false, - )); - readers.push(reader); - } - if readers.is_empty() { return Ok(None); } @@ -491,13 +467,13 @@ mod tests { file_metadata.schema_descr(), ProjectionMask::all(), file_metadata.key_value_metadata(), - vec![], + &[], ) .unwrap(); let metrics = ArrowReaderMetrics::disabled(); let array_reader = ArrayReaderBuilder::new(&file_reader, &metrics) - .build_array_reader(fields.as_ref(), &mask, None) + .build_array_reader(fields.as_ref(), &mask) .unwrap(); // Create arrow types @@ -517,17 +493,18 @@ mod tests { let file_metadata = file_reader.metadata().file_metadata(); let mask = ProjectionMask::leaves(file_metadata.schema_descr(), [0]); + let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber); let (_, fields) = parquet_to_arrow_schema_and_fields( file_metadata.schema_descr(), ProjectionMask::all(), file_metadata.key_value_metadata(), - vec![], + std::slice::from_ref(&row_number_field), ) .unwrap(); let metrics = ArrowReaderMetrics::disabled(); let array_reader = ArrayReaderBuilder::new(&file_reader, &metrics) - .build_array_reader(fields.as_ref(), &mask, Some("row_number")) + .build_array_reader(fields.as_ref(), &mask) .unwrap(); // Create arrow types @@ -537,7 +514,7 @@ mod tests { DataType::Struct(vec![Field::new("b_c_int", DataType::Int32, true)].into()), true, ), - Field::new("row_number", DataType::Int64, false), + row_number_field, ])); assert_eq!(array_reader.get_data_type(), &arrow_type); diff --git a/parquet/src/arrow/array_reader/list_array.rs b/parquet/src/arrow/array_reader/list_array.rs index bf7093a912c2..ff1b414c27bb 100644 --- a/parquet/src/arrow/array_reader/list_array.rs +++ b/parquet/src/arrow/array_reader/list_array.rs @@ -561,13 +561,13 @@ mod tests { schema, ProjectionMask::all(), file_metadata.key_value_metadata(), - vec![], + &[], ) .unwrap(); let metrics = ArrowReaderMetrics::disabled(); let mut array_reader = ArrayReaderBuilder::new(&file_reader, &metrics) - .build_array_reader(fields.as_ref(), &mask, None) + .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 e6d2af3a9fd6..d6476e455f4a 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -122,8 +122,6 @@ pub struct ArrowReaderBuilder { pub(crate) metrics: ArrowReaderMetrics, pub(crate) max_predicate_cache_size: usize, - - pub(crate) row_number_column: Option, } impl Debug for ArrowReaderBuilder { @@ -141,7 +139,6 @@ impl Debug for ArrowReaderBuilder { .field("limit", &self.limit) .field("offset", &self.offset) .field("metrics", &self.metrics) - .field("row_number_column", &self.row_number_column) .finish() } } @@ -162,7 +159,6 @@ impl ArrowReaderBuilder { offset: None, metrics: ArrowReaderMetrics::Disabled, max_predicate_cache_size: 100 * 1024 * 1024, // 100MB default cache size - row_number_column: None, } } @@ -372,146 +368,6 @@ impl ArrowReaderBuilder { ..self } } - - /// Include file row numbers in the output with the given column name - /// - /// This will add a column to the output record batch with the file row number - pub fn with_row_number_column(self, row_number_column: impl Into) -> Self { - Self { - row_number_column: Some(row_number_column.into()), - ..self - } - } - - // /// Include additional fields by appending them at the end of the fields. - // /// - // /// This will add columns to the output record batch. - // pub fn with_fields_appended(self, fields_to_append: Vec) -> Result { - // use crate::arrow::schema::virtual_type::{RowNumber, is_virtual_column}; - // use crate::arrow::schema::VirtualColumnType; - // use arrow_schema::extension::ExtensionType; - - // if fields_to_append.is_empty() { - // return Ok(self); - // } - - // // Verify all fields to append are virtual columns - // for field in &fields_to_append { - // if !is_virtual_column(field) { - // return Err(ParquetError::ArrowError( - // "Only virtual columns can be appended via with_fields_appended".to_string() - // )); - // } - // } - - // // Create extended schema - // let mut all_fields: Vec = Vec::new(); - // all_fields.extend(self.schema.fields().iter().map(|f| f.as_ref().clone())); - // all_fields.extend(fields_to_append.iter().cloned()); - // // TODO @vustef: Preserve metadata in the schema... - // let extended_schema = Arc::new(Schema::new(all_fields)); - - // // Get or create root ParquetField - // let mut root_field = if self.schema.fields().is_empty() { - // // If schema is empty, create an empty root group - // ParquetField { - // rep_level: 0, - // def_level: 0, - // nullable: false, - // arrow_type: ArrowType::Struct(Fields::empty()), - // field_type: ParquetFieldType::Group { children: vec![] }, - // } - // } else if let Some(fields) = self.fields { - // // Unwrap Arc and clone if needed - // Arc::try_unwrap(fields).unwrap_or_else(|arc| (*arc).clone()) - // } else { - // // If no fields exist yet, create root group from parquet schema - // let parquet_schema = self.metadata.file_metadata().schema_descr(); - // let field_levels = parquet_to_arrow_field_levels( - // parquet_schema, - // ProjectionMask::all(), - // None, - // )?; - // field_levels.levels.ok_or_else(|| { - // ParquetError::ArrowError("Failed to create ParquetField from schema".to_string()) - // })? - // }; - - // // Convert virtual fields to ParquetFields and append them - // match &mut root_field.field_type { - // ParquetFieldType::Group { children } => { - // for field in &fields_to_append { - // // Determine virtual column type - // let virtual_type = if field.try_extension_type::().is_ok() { - // VirtualColumnType::RowNumber - // } else { - // return Err(ParquetError::ArrowError(format!( - // "Unsupported virtual column type for field '{}'", - // field.name() - // ))); - // }; - - // let nullable = field.is_nullable(); - // // TODO @vustef: Assert def_level and rep_level are 0? - // let parquet_field = ParquetField { - // rep_level: 0, - // def_level: if nullable { 1 } else { 0 }, - // nullable, - // arrow_type: field.data_type().clone(), - // field_type: ParquetFieldType::Virtual(virtual_type), - // }; - - // children.push(parquet_field); - // } - - // // Update the root field's arrow_type to match the extended schema - // root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); - // } - // ParquetFieldType::Primitive { .. } => { - // // Root should never be primitive, but handle it by wrapping in a Group - // let existing_field = root_field.clone(); - // let mut children = vec![existing_field]; - - // for field in &fields_to_append { - // let virtual_type = if field.try_extension_type::().is_ok() { - // VirtualColumnType::RowNumber - // } else { - // return Err(ParquetError::ArrowError(format!( - // "Unsupported virtual column type for field '{}'", - // field.name() - // ))); - // }; - - // let nullable = field.is_nullable(); - // let parquet_field = ParquetField { - // rep_level: 0, - // def_level: if nullable { 1 } else { 0 }, - // nullable, - // arrow_type: field.data_type().clone(), - // field_type: ParquetFieldType::Virtual(virtual_type), - // }; - - // children.push(parquet_field); - // } - - // root_field.field_type = ParquetFieldType::Group { children }; - // root_field.arrow_type = ArrowType::Struct(extended_schema.fields().clone()); - // } - // ParquetFieldType::Virtual(_) => { - // return Err(ParquetError::ArrowError( - // "Root field cannot be a Virtual column".to_string() - // )); - // } - // } - - // // TODO @vustef: this makes virtual columns part of the schema, but when creating ProjectionMasks, they wouldn't easily be constructed to include them, since they are based on ParquetMetadata. - // // Maybe masks shouldn't apply to virtual columns at the end of the schema. - // Ok(Self { - // schema: extended_schema, - // fields: Some(Arc::new(root_field)), - // ..self - // }) - // } } /// Options that control how metadata is read for a parquet file @@ -749,7 +605,7 @@ impl ArrowReaderMetadata { /// See [`Self::load`] for more details. pub fn try_new(metadata: Arc, options: ArrowReaderOptions) -> Result { match options.supplied_schema { - Some(supplied_schema) => Self::with_supplied_schema(metadata, supplied_schema.clone(), options.virtual_columns), + Some(supplied_schema) => Self::with_supplied_schema(metadata, supplied_schema.clone(), &options.virtual_columns), None => { let kv_metadata = match options.skip_arrow_metadata { true => None, @@ -760,7 +616,7 @@ impl ArrowReaderMetadata { metadata.file_metadata().schema_descr(), ProjectionMask::all(), kv_metadata, - options.virtual_columns, + &options.virtual_columns, )?; Ok(Self { @@ -775,10 +631,9 @@ impl ArrowReaderMetadata { fn with_supplied_schema( metadata: Arc, supplied_schema: SchemaRef, - virtual_columns: Vec, + virtual_columns: &[Field], ) -> Result { let parquet_schema = metadata.file_metadata().schema_descr(); - let virtual_columns_len = virtual_columns.len(); let field_levels = parquet_to_arrow_field_levels( parquet_schema, ProjectionMask::all(), @@ -787,7 +642,7 @@ impl ArrowReaderMetadata { )?; let fields = field_levels.fields; let inferred_len = fields.len(); - let supplied_len = supplied_schema.fields().len() + virtual_columns_len; + let supplied_len = supplied_schema.fields().len() + virtual_columns.len(); // Ensure the supplied schema has the same number of columns as the parquet schema. // parquet_to_arrow_field_levels is expected to throw an error if the schemas have // different lengths, but we check here to be safe. @@ -1035,7 +890,6 @@ impl ParquetRecordBatchReaderBuilder { metrics, // Not used for the sync reader, see https://github.com/apache/arrow-rs/issues/8000 max_predicate_cache_size: _, - row_number_column } = self; // Try to avoid allocate large buffer @@ -1065,14 +919,14 @@ impl ParquetRecordBatchReaderBuilder { cache_projection.intersect(&projection); let array_reader = ArrayReaderBuilder::new(&reader, &metrics) - .build_array_reader(fields.as_deref(), predicate.projection(), row_number_column.as_deref(),)?; + .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, &metrics) - .build_array_reader(fields.as_deref(), &projection, row_number_column.as_deref())?; + .build_array_reader(fields.as_deref(), &projection)?; let read_plan = plan_builder .limited(reader.num_rows()) @@ -1283,7 +1137,7 @@ impl ParquetRecordBatchReader { // 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(), None)?; + .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 0f9df2f77509..6e48f2629443 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -516,7 +516,6 @@ impl ParquetRecordBatchStreamBuilder { offset: self.offset, metrics: self.metrics, max_predicate_cache_size: self.max_predicate_cache_size, - row_number_column: self.row_number_column, }; // Ensure schema of ParquetRecordBatchStream respects projection, and does @@ -573,8 +572,6 @@ struct ReaderFactory { /// Maximum size of the predicate cache max_predicate_cache_size: usize, - - row_number_column: Option, } impl ReaderFactory @@ -652,7 +649,7 @@ where let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) .with_cache_options(Some(&cache_options)) - .build_array_reader(self.fields.as_deref(), predicate.projection(), self.row_number_column.as_deref())?; + .build_array_reader(self.fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; } @@ -709,7 +706,7 @@ where let cache_options = cache_options_builder.consumer(); let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) .with_cache_options(Some(&cache_options)) - .build_array_reader(self.fields.as_deref(), &projection, self.row_number_column.as_deref())?; + .build_array_reader(self.fields.as_deref(), &projection)?; let reader = ParquetRecordBatchReader::new(array_reader, plan); @@ -1223,6 +1220,7 @@ mod tests { }; use crate::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; + use crate::arrow::schema::virtual_type::RowNumber; use crate::file::metadata::ParquetMetaDataReader; use crate::file::properties::WriterProperties; use arrow::compute::kernels::cmp::eq; @@ -1982,6 +1980,7 @@ mod tests { metadata.file_metadata().schema_descr(), ProjectionMask::all(), None, + &[], ) .unwrap(); @@ -1998,7 +1997,6 @@ mod tests { offset: None, metrics: ArrowReaderMetrics::disabled(), max_predicate_cache_size: 0, - row_number_column: None, }; let mut skip = true; @@ -2464,7 +2462,6 @@ mod tests { offset: None, metrics: ArrowReaderMetrics::disabled(), max_predicate_cache_size: 0, - row_number_column: None, }; // Provide an output projection that also selects the same nested leaf @@ -2730,12 +2727,13 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let reader = ParquetRecordBatchStreamBuilder::new(file) + let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber); + let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await .unwrap() .with_row_selection(selection) .with_batch_size(batch_size) - .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader.try_collect::>().await.unwrap() @@ -2755,13 +2753,14 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let reader = ParquetRecordBatchStreamBuilder::new(file) + let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber); + let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await .unwrap() .with_row_selection(selection) .with_row_filter(row_filter.expect("No row filter")) .with_batch_size(batch_size) - .with_row_number_column("row_number") .build() .expect("Could not create reader"); reader.try_collect::>().await.unwrap() diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index a47e576fe3c0..ec0fdfdc6690 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -132,17 +132,17 @@ impl VisitorContext { /// See [Logical Types] for more information on the conversion algorithm /// /// [Logical Types]: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md -struct Visitor { +struct Visitor<'a> { /// The column index of the next leaf column next_col_idx: usize, /// Mask of columns to include mask: ProjectionMask, - virtual_columns: Vec, + virtual_columns: &'a [Field], } -impl Visitor { +impl<'a> Visitor<'a> { fn visit_primitive( &mut self, primitive_type: &TypePtr, @@ -248,7 +248,7 @@ impl Visitor { // we need to handle virtual columns? if rep_level == 0 && def_level == 0 { // TODO @vustef: assert is_virtual_column ? Or use types to our advantage somehow. - for virtual_column in &self.virtual_columns { + for virtual_column in self.virtual_columns { child_fields.push(virtual_column.clone()); let child = convert_virtual_field(virtual_column, rep_level, def_level)?; children.push(child); @@ -660,7 +660,7 @@ pub fn convert_schema( schema: &SchemaDescriptor, mask: ProjectionMask, embedded_arrow_schema: Option<&Fields>, - virtual_columns: Vec, + virtual_columns: &[Field], // TODO @vustef: Also a pub API change... ) -> Result> { let mut visitor = Visitor { next_col_idx: 0, @@ -682,7 +682,7 @@ pub fn convert_type(parquet_type: &TypePtr) -> Result { let mut visitor = Visitor { next_col_idx: 0, mask: ProjectionMask::all(), - virtual_columns: vec![], // TODO @vustef: Maybe should be None rather? + virtual_columns: &[], }; let context = VisitorContext { diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index b03b389f5bb7..eed1cceff9ce 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -63,7 +63,7 @@ pub fn parquet_to_arrow_schema_by_columns( mask: ProjectionMask, key_value_metadata: Option<&Vec>, ) -> Result { - Ok(parquet_to_arrow_schema_and_fields(parquet_schema, mask, key_value_metadata, vec![])?.0) + Ok(parquet_to_arrow_schema_and_fields(parquet_schema, mask, key_value_metadata, &[])?.0) } /// Determines the Arrow Schema from a Parquet schema @@ -75,7 +75,7 @@ pub(crate) fn parquet_to_arrow_schema_and_fields( parquet_schema: &SchemaDescriptor, mask: ProjectionMask, key_value_metadata: Option<&Vec>, - virtual_columns: Vec, + virtual_columns: &[Field], ) -> Result<(Schema, Option)> { let mut metadata = parse_key_value_metadata(key_value_metadata).unwrap_or_default(); let maybe_schema = metadata @@ -133,7 +133,7 @@ pub fn parquet_to_arrow_field_levels( schema: &SchemaDescriptor, mask: ProjectionMask, hint: Option<&Fields>, - virtual_columns: Vec, // TODO @vustef: This is a public method, maybe preserve its signature. Or maybe it's good to change it, to be able to construct readers from pub API. + virtual_columns: &[Field], // TODO @vustef: This is a public method, maybe preserve its signature. Or maybe it's good to change it, to be able to construct readers from pub API. ) -> Result { match complex::convert_schema(schema, mask, hint, virtual_columns)? { Some(field) => match &field.arrow_type { From af0e0f92d96c8f2c31c536c3d937f9a60d4410d6 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 10:29:30 +0100 Subject: [PATCH 22/48] Bring back optionality to first_row_index, for future consideration when metadata parsing may skip row groups --- parquet/src/arrow/array_reader/row_number.rs | 6 ++++-- parquet/src/file/metadata/mod.rs | 17 +++++++++-------- parquet/src/file/metadata/thrift/encryption.rs | 2 +- parquet/src/file/metadata/thrift/mod.rs | 2 +- 4 files changed, 15 insertions(+), 12 deletions(-) diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index fbf47b6b5e23..3a942277a91a 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -16,7 +16,7 @@ // under the License. use crate::arrow::array_reader::ArrayReader; -use crate::errors::Result; +use crate::errors::{ParquetError, Result}; use crate::file::metadata::RowGroupMetaData; use arrow_array::{ArrayRef, Int64Array}; use arrow_schema::DataType; @@ -34,7 +34,9 @@ impl RowNumberReader { ) -> Result { let ranges = row_groups .map(|rg| { - let first_row_index = rg.first_row_index(); + let first_row_index = rg.first_row_index().ok_or(ParquetError::General( + "Row group missing row number".to_string(), + ))?; Ok(first_row_index..first_row_index + rg.num_rows()) }) .collect::>>()?; diff --git a/parquet/src/file/metadata/mod.rs b/parquet/src/file/metadata/mod.rs index 7afc8045c40d..a97df859fdf1 100644 --- a/parquet/src/file/metadata/mod.rs +++ b/parquet/src/file/metadata/mod.rs @@ -616,7 +616,7 @@ pub type RowGroupMetaDataPtr = Arc; pub struct RowGroupMetaData { columns: Vec, num_rows: i64, - first_row_index: i64, + first_row_index: Option, sorting_columns: Option>, total_byte_size: i64, schema_descr: SchemaDescPtr, @@ -658,7 +658,7 @@ impl RowGroupMetaData { } /// Returns the global index number for the first row in this row group. - pub fn first_row_index(&self) -> i64 { + pub fn first_row_index(&self) -> Option { self.first_row_index } @@ -719,7 +719,7 @@ impl RowGroupMetaDataBuilder { schema_descr, file_offset: None, num_rows: 0, - first_row_index: 0, + first_row_index: None, sorting_columns: None, total_byte_size: 0, ordinal: None, @@ -734,7 +734,7 @@ impl RowGroupMetaDataBuilder { /// Sets the first row number in this row group. pub fn set_first_row_index(mut self, value: i64) -> Self { - self.0.first_row_index = value; + self.0.first_row_index = Some(value); self } @@ -1631,6 +1631,7 @@ mod tests { .set_num_rows(1000) .set_total_byte_size(2000) .set_column_metadata(columns) + .set_first_row_index(0) .set_ordinal(1) .build() .unwrap(); @@ -1888,10 +1889,10 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let base_expected_size = 2256; + let base_expected_size = 2264; #[cfg(feature = "encryption")] // Not as accurate as it should be: https://github.com/apache/arrow-rs/issues/8472 - let base_expected_size = 2424; + let base_expected_size = 2432; assert_eq!(parquet_meta.memory_size(), base_expected_size); @@ -1920,10 +1921,10 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let bigger_expected_size = 2682; + let bigger_expected_size = 2690; #[cfg(feature = "encryption")] // Not as accurate as it should be: https://github.com/apache/arrow-rs/issues/8472 - let bigger_expected_size = 2850; + let bigger_expected_size = 2858; // more set fields means more memory usage assert!(bigger_expected_size > base_expected_size); diff --git a/parquet/src/file/metadata/thrift/encryption.rs b/parquet/src/file/metadata/thrift/encryption.rs index 240071292680..43537ad3ea03 100644 --- a/parquet/src/file/metadata/thrift/encryption.rs +++ b/parquet/src/file/metadata/thrift/encryption.rs @@ -193,7 +193,7 @@ fn row_group_from_encrypted_thrift( Ok(RowGroupMetaData { columns, num_rows, - first_row_index, + first_row_index: Some(first_row_index), sorting_columns, total_byte_size, schema_descr, diff --git a/parquet/src/file/metadata/thrift/mod.rs b/parquet/src/file/metadata/thrift/mod.rs index b157c5fe45be..3a866fc14472 100644 --- a/parquet/src/file/metadata/thrift/mod.rs +++ b/parquet/src/file/metadata/thrift/mod.rs @@ -589,7 +589,7 @@ fn read_row_group( ) -> Result { // create default initialized RowGroupMetaData let mut row_group = RowGroupMetaDataBuilder::new(schema_descr.clone()).build_unchecked(); - row_group.first_row_index = first_row_index; + row_group.first_row_index = Some(first_row_index); // mask values for required fields const RG_COLUMNS: u8 = 1 << 1; From 8bccd22aa23719e4b661d1e4db75f3382d8683f7 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 11:44:34 +0100 Subject: [PATCH 23/48] Reexport --- parquet/src/arrow/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index ad14da0e5a45..d36e6036a468 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -206,6 +206,9 @@ pub use self::schema::{ parquet_to_arrow_field_levels, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, }; +// Re-export virtual_type directly from the schema module. +pub use self::schema::virtual_type; + /// Schema metadata key used to store serialized Arrow schema /// /// The Arrow schema is encoded using the Arrow IPC format, and then base64 From 65679ba3aa052c6fda3aa7715a9102e09f4818cf Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 11:49:37 +0100 Subject: [PATCH 24/48] reexport all within virtual_type --- parquet/src/arrow/mod.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index d36e6036a468..0732733dbbdb 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -204,11 +204,9 @@ use arrow_schema::{FieldRef, Schema}; pub use self::schema::{ ArrowSchemaConverter, FieldLevels, add_encoded_arrow_schema_to_metadata, encode_arrow_schema, parquet_to_arrow_field_levels, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, + virtual_type::*, }; -// Re-export virtual_type directly from the schema module. -pub use self::schema::virtual_type; - /// Schema metadata key used to store serialized Arrow schema /// /// The Arrow schema is encoded using the Arrow IPC format, and then base64 From 968d4613409af9b71bb65654f057022c30ccc0be Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 11:55:10 +0100 Subject: [PATCH 25/48] pub mod virtual_type skipping experimental schema --- parquet/src/arrow/mod.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 0732733dbbdb..27bdd1f75ef2 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -193,6 +193,10 @@ pub mod async_writer; mod record_reader; experimental!(mod schema); +// Make virtual_type publicly available without experimental flag +#[path = "schema/virtual_type.rs"] +pub mod virtual_type; + pub use self::arrow_writer::ArrowWriter; #[cfg(feature = "async")] pub use self::async_reader::ParquetRecordBatchStreamBuilder; @@ -204,7 +208,6 @@ use arrow_schema::{FieldRef, Schema}; pub use self::schema::{ ArrowSchemaConverter, FieldLevels, add_encoded_arrow_schema_to_metadata, encode_arrow_schema, parquet_to_arrow_field_levels, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, - virtual_type::*, }; /// Schema metadata key used to store serialized Arrow schema From 6144967b99c077376555f186c1104479abd97934 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 12:06:51 +0100 Subject: [PATCH 26/48] Switch back to `virtual_type::*` for now; fix warnings on cargo test --- parquet/src/arrow/array_reader/builder.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 13 ++++++------- parquet/src/arrow/mod.rs | 4 +--- parquet/src/arrow/schema/complex.rs | 1 - 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 7b06ebb7ae6d..f03e75f81502 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::{DataType, Field, Fields, SchemaBuilder}; +use arrow_schema::{DataType, Fields, SchemaBuilder}; use std::sync::{Arc, Mutex}; use crate::arrow::ProjectionMask; diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index d6476e455f4a..96fbc6d8b515 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1221,7 +1221,6 @@ pub(crate) mod tests { use crate::file::metadata::ParquetMetaData; use crate::file::properties::{EnabledStatistics, WriterProperties, WriterVersion}; use crate::file::writer::SerializedFileWriter; - use crate::format::FileMetaData; use crate::schema::parser::parse_message_type; use crate::schema::types::{Type, TypePtr}; use crate::util::test_common::rand_gen::RandGen; @@ -5184,17 +5183,17 @@ pub(crate) mod tests { let mut case = vec![]; let mut remaining = metadata.file_metadata().num_rows(); while remaining > 0 { - let row_count = rng.gen_range(1..=remaining); + let row_count = rng.random_range(1..=remaining); remaining -= row_count; case.push(RowSelector { row_count: row_count as usize, - skip: rng.gen_bool(0.5), + skip: rng.random_bool(0.5), }); } let filter = use_filter.then(|| { let filter = (0..metadata.file_metadata().num_rows()) - .map(|_| rng.gen_bool(0.99)) + .map(|_| rng.random_bool(0.99)) .collect::>(); let mut filter_offset = 0; RowFilter::new(vec![Box::new(ArrowPredicateFn::new( @@ -5214,7 +5213,7 @@ pub(crate) mod tests { }); let selection = RowSelection::from(case); - let batches = test_case(path, selection.clone(), filter, rng.gen_range(1..4096)); + let batches = test_case(path, selection.clone(), filter, rng.random_range(1..4096)); if selection.skipped_row_count() == metadata.file_metadata().num_rows() as usize { assert!(batches.into_iter().all(|batch| batch.num_rows() == 0)); @@ -5253,11 +5252,11 @@ pub(crate) mod tests { let mut writer = ArrowWriter::try_new(&mut buf, schema.clone(), None).expect("Could not create writer"); - let mut values = 1..=rng.gen_range(1..4096); + let mut values = 1..=rng.random_range(1..4096); while !values.is_empty() { let batch_values = values .by_ref() - .take(rng.gen_range(1..4096)) + .take(rng.random_range(1..4096)) .collect::>(); let array = Arc::new(Int64Array::from(batch_values)) as ArrayRef; let batch = diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 27bdd1f75ef2..6675f98cb1a2 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -193,9 +193,6 @@ pub mod async_writer; mod record_reader; experimental!(mod schema); -// Make virtual_type publicly available without experimental flag -#[path = "schema/virtual_type.rs"] -pub mod virtual_type; pub use self::arrow_writer::ArrowWriter; #[cfg(feature = "async")] @@ -208,6 +205,7 @@ use arrow_schema::{FieldRef, Schema}; pub use self::schema::{ ArrowSchemaConverter, FieldLevels, add_encoded_arrow_schema_to_metadata, encode_arrow_schema, parquet_to_arrow_field_levels, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, + virtual_type::*, }; /// Schema metadata key used to store serialized Arrow schema diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index ec0fdfdc6690..9edba8ee2367 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -27,7 +27,6 @@ use crate::errors::ParquetError; use crate::errors::Result; use crate::schema::types::{SchemaDescriptor, Type, TypePtr}; use arrow_schema::{DataType, Field, Fields, SchemaBuilder}; -use arrow_schema::extension::ExtensionType; fn get_repetition(t: &Type) -> Repetition { let info = t.get_basic_info(); From 3af3ad7b6ef758796cee6b8b73b1612ccc4a360b Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 27 Oct 2025 13:53:32 +0100 Subject: [PATCH 27/48] Fix `projected_fields` assertion in async reader --- parquet/src/arrow/async_reader/mod.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 6e48f2629443..bd848c797c5c 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -522,7 +522,8 @@ impl ParquetRecordBatchStreamBuilder { // not store metadata (same as for ParquetRecordBatchReader and emitted RecordBatches) let projected_fields = match reader_factory.fields.as_deref().map(|pf| &pf.arrow_type) { Some(DataType::Struct(fields)) => { - fields.filter_leaves(|idx, _| self.projection.leaf_included(idx)) + let projection_len = self.projection.mask.as_ref().map_or(usize::MAX, |m| m.len()); + fields.filter_leaves(|idx, _| idx < projection_len && self.projection.leaf_included(idx)) } None => Fields::empty(), _ => unreachable!("Must be Struct for root type"), From fad0ea1f752631e1955cc000fbdf226ed0b1210e Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 6 Nov 2025 12:57:14 +0100 Subject: [PATCH 28/48] common virtual column struct --- parquet/src/arrow/array_reader/builder.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 30 ++++++++++---- parquet/src/arrow/async_reader/mod.rs | 4 +- parquet/src/arrow/schema/virtual_type.rs | 50 ++++++++++++++++------- 4 files changed, 62 insertions(+), 24 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index f03e75f81502..6dde646c0430 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -493,7 +493,7 @@ mod tests { let file_metadata = file_reader.metadata().file_metadata(); let mask = ProjectionMask::leaves(file_metadata.schema_descr(), [0]); - let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default()); let (_, fields) = parquet_to_arrow_schema_and_fields( file_metadata.schema_descr(), ProjectionMask::all(), diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 96fbc6d8b515..fe06d0cf53b0 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -28,7 +28,7 @@ use std::sync::Arc; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; -use crate::arrow::schema::{ParquetField, parquet_to_arrow_schema_and_fields}; +use crate::arrow::schema::{ParquetField, parquet_to_arrow_schema_and_fields, virtual_type::is_virtual_column}; use crate::arrow::{FieldLevels, ProjectionMask, parquet_to_arrow_field_levels}; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; use crate::bloom_filter::{ @@ -524,10 +524,18 @@ impl ArrowReaderOptions { /// /// # Example /// ``` - /// let virtual_columns = vec![Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)]; + /// let virtual_columns = vec![Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())]; /// let options = ArrowReaderOptions::new().with_virtual_columns(virtual_columns); pub fn with_virtual_columns(self, virtual_columns: Vec) -> Self { - // TODO @vustef: Make sure the passed fields are virtual columns. + // Validate that all fields are virtual columns + for field in &virtual_columns { + if !is_virtual_column(field) { + panic!( + "Field '{}' is not a virtual column. Virtual columns must have extension type names starting with 'arrow.virtual.'", + field.name() + ); + } + } Self { virtual_columns, ..self @@ -5046,7 +5054,7 @@ pub(crate) mod tests { Field::new("value", ArrowDataType::Int64, false), ]); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); let options = options.with_virtual_columns(vec![row_number_field.clone()]); @@ -5091,7 +5099,7 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field.clone()]); let metadata = ArrowReaderMetadata::load(&file, options).unwrap(); let num_columns = metadata.metadata.file_metadata().schema_descr().num_columns(); @@ -5119,13 +5127,21 @@ pub(crate) mod tests { ); } + #[test] + #[should_panic(expected = "is not a virtual column")] + fn test_with_virtual_columns_rejects_non_virtual_fields() { + // Try to pass a regular field (not a virtual column) to with_virtual_columns + let regular_field = Field::new("regular_column", ArrowDataType::Int64, false); + let _options = ArrowReaderOptions::new().with_virtual_columns(vec![regular_field]); + } + #[test] fn test_row_numbers_with_multiple_row_groups() { test_row_numbers_with_multiple_row_groups_helper( false, |path, selection, _row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() @@ -5146,7 +5162,7 @@ pub(crate) mod tests { true, |path, selection, row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index bd848c797c5c..e4d7400268c0 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -2728,7 +2728,7 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default()); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await @@ -2754,7 +2754,7 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber); + let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default()); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index df8fbc013ccc..f4c5e5757318 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -20,17 +20,35 @@ use arrow_schema::{ArrowError, DataType, Field, extension::ExtensionType}; -/// The extension type for `8-bit Boolean`. +/// Prefix for virtual column extension type names. +const VIRTUAL_PREFIX: &str = "arrow.virtual."; + +/// Macro to concatenate VIRTUAL_PREFIX with a suffix. +macro_rules! virtual_name { + ($suffix:literal) => { + concat!("arrow.virtual.", $suffix) + }; +} + +/// Constants for virtual column type identifiers. +mod virtual_column_type { + /// Row number virtual column. + pub(super) const ROW_NUMBER: u8 = 0; +} + +/// Generic virtual column extension type. /// -/// Extension name: `arrow.row_number`. +/// This struct provides a common implementation for all virtual column types. /// /// The storage type of the extension is `Int64`. -/// #[derive(Debug, Default, Clone, Copy, PartialEq)] -pub struct RowNumber; +pub struct VirtualColumn; -impl ExtensionType for RowNumber { - const NAME: &'static str = "arrow.virtual.row_number"; // TODO @vustef: What should it be named? +impl ExtensionType for VirtualColumn { + const NAME: &'static str = match TYPE { + virtual_column_type::ROW_NUMBER => virtual_name!("row_number"), + _ => panic!("Unknown virtual column type"), + }; type Metadata = &'static str; @@ -47,7 +65,7 @@ impl ExtensionType for RowNumber { Ok("") } else { Err(ArrowError::InvalidArgumentError( - "RowNumber extension type expects an empty string as metadata".to_owned(), + "Virtual column extension type expects an empty string as metadata".to_owned(), )) } } @@ -56,7 +74,7 @@ impl ExtensionType for RowNumber { match data_type { DataType::Int64 => Ok(()), data_type => Err(ArrowError::InvalidArgumentError(format!( - "RowNumber data type mismatch, expected Int64, found {data_type}" + "Virtual column data type mismatch, expected Int64, found {data_type}" ))), } } @@ -66,6 +84,11 @@ impl ExtensionType for RowNumber { } } +/// The extension type for row numbers. +/// +/// Extension name: `arrow.virtual.row_number`. +pub type RowNumber = VirtualColumn<{ virtual_column_type::ROW_NUMBER }>; + #[cfg(test)] mod tests { use arrow_schema::{ @@ -78,7 +101,7 @@ mod tests { #[test] fn valid() -> Result<(), ArrowError> { let mut field = Field::new("", DataType::Int64, false); - field.try_with_extension_type(RowNumber)?; + field.try_with_extension_type(RowNumber::default())?; field.try_extension_type::()?; Ok(()) @@ -98,11 +121,11 @@ mod tests { #[test] #[should_panic(expected = "expected Int64, found Int32")] fn invalid_type() { - Field::new("", DataType::Int32, false).with_extension_type(RowNumber); + Field::new("", DataType::Int32, false).with_extension_type(RowNumber::default()); } #[test] - #[should_panic(expected = "RowNumber extension type expects an empty string as metadata")] + #[should_panic(expected = "Virtual column extension type expects an empty string as metadata")] fn missing_metadata() { let field = Field::new("", DataType::Int64, false).with_metadata( [(EXTENSION_TYPE_NAME_KEY.to_owned(), RowNumber::NAME.to_owned())] @@ -113,7 +136,7 @@ mod tests { } #[test] - #[should_panic(expected = "RowNumber extension type expects an empty string as metadata")] + #[should_panic(expected = "Virtual column extension type expects an empty string as metadata")] fn invalid_metadata() { let field = Field::new("", DataType::Int64, false).with_metadata( [ @@ -134,8 +157,7 @@ mod tests { /// /// Virtual columns have extension type names starting with `arrow.virtual.`. pub fn is_virtual_column(field: &Field) -> bool { - // TODO @vustef: Make this more typed through another approach that doesn't rely on a naming convention. field.extension_type_name() - .map(|name| name.starts_with("arrow.virtual.")) + .map(|name| name.starts_with(VIRTUAL_PREFIX)) .unwrap_or(false) } From ca6c7a607ddc1b93eb097b001f1ca7341db718ae Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 6 Nov 2025 13:20:17 +0100 Subject: [PATCH 29/48] assert that column is virtual --- parquet/src/arrow/schema/complex.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 9edba8ee2367..31cafbc726cc 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -243,11 +243,14 @@ impl<'a> Visitor<'a> { } } - // TODO @vustef: Are all parquet schemas going to start with a struct? I.e. is this the only place where - // we need to handle virtual columns? if rep_level == 0 && def_level == 0 { - // TODO @vustef: assert is_virtual_column ? Or use types to our advantage somehow. for virtual_column in self.virtual_columns { + // Ensure this is actually a virtual column + assert!( + super::virtual_type::is_virtual_column(virtual_column), + "Field '{}' is not a virtual column. Virtual columns must have extension type names starting with 'arrow.virtual.'", + virtual_column.name() + ); child_fields.push(virtual_column.clone()); let child = convert_virtual_field(virtual_column, rep_level, def_level)?; children.push(child); From da9245d9181f8f0e4f0d6d9c56f4fba9a7d3a91b Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 6 Nov 2025 13:20:31 +0100 Subject: [PATCH 30/48] don't change pub API --- parquet/src/arrow/arrow_reader/mod.rs | 4 +- parquet/src/arrow/mod.rs | 2 +- parquet/src/arrow/schema/mod.rs | 74 ++++++++++++++++++++++++++- 3 files changed, 75 insertions(+), 5 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index fe06d0cf53b0..8e40f0e00422 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -29,7 +29,7 @@ use std::sync::Arc; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; use crate::arrow::schema::{ParquetField, parquet_to_arrow_schema_and_fields, virtual_type::is_virtual_column}; -use crate::arrow::{FieldLevels, ProjectionMask, parquet_to_arrow_field_levels}; +use crate::arrow::{FieldLevels, ProjectionMask, parquet_to_arrow_field_levels_with_virtual}; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; use crate::bloom_filter::{ SBBF_HEADER_SIZE_ESTIMATE, Sbbf, chunk_read_bloom_filter_header_and_offset, @@ -642,7 +642,7 @@ impl ArrowReaderMetadata { virtual_columns: &[Field], ) -> Result { let parquet_schema = metadata.file_metadata().schema_descr(); - let field_levels = parquet_to_arrow_field_levels( + let field_levels = parquet_to_arrow_field_levels_with_virtual( parquet_schema, ProjectionMask::all(), Some(supplied_schema.fields()), diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 6675f98cb1a2..f3e249da2522 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -204,7 +204,7 @@ use arrow_schema::{FieldRef, Schema}; pub use self::schema::{ ArrowSchemaConverter, FieldLevels, add_encoded_arrow_schema_to_metadata, encode_arrow_schema, - parquet_to_arrow_field_levels, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, + parquet_to_arrow_field_levels, parquet_to_arrow_field_levels_with_virtual, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, virtual_type::*, }; diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index eed1cceff9ce..b16318bddfef 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -91,7 +91,7 @@ pub(crate) fn parquet_to_arrow_schema_and_fields( } let hint = maybe_schema.as_ref().map(|s| s.fields()); - let field_levels = parquet_to_arrow_field_levels(parquet_schema, mask, hint, virtual_columns)?; + let field_levels = parquet_to_arrow_field_levels_with_virtual(parquet_schema, mask, hint, virtual_columns)?; let schema = Schema::new_with_metadata(field_levels.fields, metadata); Ok((schema, field_levels.levels)) } @@ -133,8 +133,55 @@ pub fn parquet_to_arrow_field_levels( schema: &SchemaDescriptor, mask: ProjectionMask, hint: Option<&Fields>, - virtual_columns: &[Field], // TODO @vustef: This is a public method, maybe preserve its signature. Or maybe it's good to change it, to be able to construct readers from pub API. ) -> Result { + parquet_to_arrow_field_levels_with_virtual(schema, mask, hint, &[]) +} + +/// Convert a parquet [`SchemaDescriptor`] to [`FieldLevels`] with support for virtual columns +/// +/// Columns not included within [`ProjectionMask`] will be ignored. +/// +/// The optional `hint` parameter is the desired Arrow schema. See the +/// [`arrow`] module documentation for more information. +/// +/// [`arrow`]: crate::arrow +/// +/// # Arguments +/// * `schema` - The Parquet schema descriptor +/// * `mask` - Projection mask to select which columns to include +/// * `hint` - Optional hint for Arrow field types to use instead of defaults +/// * `virtual_columns` - Virtual columns to append to the schema (e.g., row numbers) +/// +/// # Notes: +/// Where a field type in `hint` is compatible with the corresponding parquet type in `schema`, it +/// will be used, otherwise the default arrow type for the given parquet column type will be used. +/// +/// Virtual columns are columns that don't exist in the Parquet file but are generated during reading. +/// They must have extension type names starting with "arrow.virtual.". +/// +/// This is to accommodate arrow types that cannot be round-tripped through parquet natively. +/// Depending on the parquet writer, this can lead to a mismatch between a file's parquet schema +/// and its embedded arrow schema. The parquet `schema` must be treated as authoritative in such +/// an event. See [#1663](https://github.com/apache/arrow-rs/issues/1663) for more information +/// +/// Note: this is a low-level API, most users will want to make use of the higher-level +/// [`parquet_to_arrow_schema`] for decoding metadata from a parquet file. +pub fn parquet_to_arrow_field_levels_with_virtual( + schema: &SchemaDescriptor, + mask: ProjectionMask, + hint: Option<&Fields>, + virtual_columns: &[Field], +) -> Result { + // Validate that all fields are virtual columns + for field in virtual_columns { + if !virtual_type::is_virtual_column(field) { + return Err(ParquetError::General(format!( + "Field '{}' is not a virtual column. Virtual columns must have extension type names starting with 'arrow.virtual.'", + field.name() + ))); + } + } + match complex::convert_schema(schema, mask, hint, virtual_columns)? { Some(field) => match &field.arrow_type { DataType::Struct(fields) => Ok(FieldLevels { @@ -2252,4 +2299,27 @@ mod tests { Ok(()) } + + #[test] + fn test_parquet_to_arrow_field_levels_with_virtual_rejects_non_virtual() { + let message_type = " + message test_schema { + REQUIRED INT32 id; + } + "; + let parquet_schema = Arc::new(parse_message_type(message_type).unwrap()); + let descriptor = SchemaDescriptor::new(parquet_schema); + + // Try to pass a regular field (not a virtual column) + let regular_field = Field::new("regular_column", DataType::Int64, false); + let result = parquet_to_arrow_field_levels_with_virtual( + &descriptor, + ProjectionMask::all(), + None, + &[regular_field], + ); + + assert!(result.is_err()); + assert!(result.unwrap_err().to_string().contains("is not a virtual column")); + } } From 031c6d56952f74b1eb33cf2a95f6dd5efac0c303 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 6 Nov 2025 13:33:10 +0100 Subject: [PATCH 31/48] complex_schema rename --- parquet/src/arrow/schema/complex.rs | 59 +++++++++++++++++++++++++---- parquet/src/arrow/schema/mod.rs | 2 +- 2 files changed, 52 insertions(+), 9 deletions(-) diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 31cafbc726cc..09f6eae092ec 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -27,6 +27,7 @@ use crate::errors::ParquetError; use crate::errors::Result; use crate::schema::types::{SchemaDescriptor, Type, TypePtr}; use arrow_schema::{DataType, Field, Fields, SchemaBuilder}; +use arrow_schema::extension::ExtensionType; fn get_repetition(t: &Type) -> Repetition { let info = t.get_basic_info(); @@ -590,14 +591,23 @@ fn convert_virtual_field( parent_def_level }; - // Determine the virtual column type based on the extension type - let virtual_type = if arrow_field.try_extension_type::().is_ok() { - VirtualColumnType::RowNumber // TODO @vustef: Don't like the ifelse approach... - } else { - return Err(ParquetError::ArrowError(format!( - "unsupported virtual column type for field '{}'", + // Determine the virtual column type based on the extension type name + let extension_name = arrow_field.extension_type_name().ok_or_else(|| { + ParquetError::ArrowError(format!( + "virtual column field '{}' must have an extension type", arrow_field.name() - ))); + )) + })?; + + let virtual_type = match extension_name { + RowNumber::NAME => VirtualColumnType::RowNumber, + _ => { + return Err(ParquetError::ArrowError(format!( + "unsupported virtual column type '{}' for field '{}'", + extension_name, + arrow_field.name() + ))) + } }; Ok(ParquetField { @@ -662,8 +672,41 @@ pub fn convert_schema( schema: &SchemaDescriptor, mask: ProjectionMask, embedded_arrow_schema: Option<&Fields>, - virtual_columns: &[Field], // TODO @vustef: Also a pub API change... ) -> Result> { + convert_schema_with_virtual(schema, mask, embedded_arrow_schema, &[]) +} + +/// Computes the [`ParquetField`] for the provided [`SchemaDescriptor`] with support for virtual columns +/// +/// This function is similar to [`convert_schema`] but allows specifying virtual columns that should be +/// included in the schema. Virtual columns are columns that don't exist in the Parquet file but are +/// generated during reading (e.g., row numbers). +/// +/// # Arguments +/// * `schema` - The Parquet schema descriptor +/// * `mask` - Projection mask to select which columns to include +/// * `embedded_arrow_schema` - Optional embedded Arrow schema from metadata +/// * `virtual_columns` - Virtual columns to append to the schema +/// +/// # Notes +/// - Virtual columns must have extension type names starting with "arrow.virtual." +/// - This does not support out of order column projection +pub fn convert_schema_with_virtual( + schema: &SchemaDescriptor, + mask: ProjectionMask, + embedded_arrow_schema: Option<&Fields>, + virtual_columns: &[Field], +) -> Result> { + // Validate that all fields are virtual columns + for field in virtual_columns { + if !super::virtual_type::is_virtual_column(field) { + return Err(ParquetError::ArrowError(format!( + "Field '{}' is not a virtual column. Virtual columns must have extension type names starting with 'arrow.virtual.'", + field.name() + ))); + } + } + let mut visitor = Visitor { next_col_idx: 0, mask, diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index b16318bddfef..1ebd24375421 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -182,7 +182,7 @@ pub fn parquet_to_arrow_field_levels_with_virtual( } } - match complex::convert_schema(schema, mask, hint, virtual_columns)? { + match complex::convert_schema_with_virtual(schema, mask, hint, virtual_columns)? { Some(field) => match &field.arrow_type { DataType::Struct(fields) => Ok(FieldLevels { fields: fields.clone(), From 079a78d414738de376794e223ae99ea5dec7b25c Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 6 Nov 2025 14:10:01 +0100 Subject: [PATCH 32/48] passing docstring test --- parquet/src/arrow/arrow_reader/mod.rs | 47 +++++++++++++++++++++++++-- 1 file changed, 45 insertions(+), 2 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 8e40f0e00422..61e71af751b4 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -524,8 +524,51 @@ impl ArrowReaderOptions { /// /// # Example /// ``` - /// let virtual_columns = vec![Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())]; - /// let options = ArrowReaderOptions::new().with_virtual_columns(virtual_columns); + /// # use std::sync::Arc; + /// # use arrow_array::{ArrayRef, Int64Array, RecordBatch}; + /// # use arrow_schema::{DataType, Field, Schema}; + /// # use parquet::arrow::{ArrowWriter, RowNumber}; + /// # use parquet::arrow::arrow_reader::{ArrowReaderOptions, ParquetRecordBatchReaderBuilder}; + /// # use tempfile::tempfile; + /// # + /// # fn main() -> Result<(), Box> { + /// // Create a simple record batch with some data + /// let values = Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef; + /// let batch = RecordBatch::try_from_iter(vec![("value", values)])?; + /// + /// // Write the batch to a temporary parquet file + /// let file = tempfile()?; + /// let mut writer = ArrowWriter::try_new( + /// file.try_clone()?, + /// batch.schema(), + /// None + /// )?; + /// writer.write(&batch)?; + /// writer.close()?; + /// + /// // Create a virtual column for row numbers + /// let row_number_field = Field::new("row_number", DataType::Int64, false) + /// .with_extension_type(RowNumber::default()); + /// + /// // Configure options with virtual columns + /// let options = ArrowReaderOptions::new() + /// .with_virtual_columns(vec![row_number_field]); + /// + /// // Create a reader with the options + /// let mut reader = ParquetRecordBatchReaderBuilder::try_new_with_options( + /// file, + /// options + /// )? + /// .build()?; + /// + /// // Read the batch - it will include both the original column and the virtual row_number column + /// let result_batch = reader.next().unwrap()?; + /// assert_eq!(result_batch.num_columns(), 2); // "value" + "row_number" + /// assert_eq!(result_batch.num_rows(), 3); + /// # + /// # Ok(()) + /// # } + /// ``` pub fn with_virtual_columns(self, virtual_columns: Vec) -> Self { // Validate that all fields are virtual columns for field in &virtual_columns { From f2a4f457cab2148ae00768ccb9e18a8e1a797018 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Thu, 6 Nov 2025 15:34:15 +0100 Subject: [PATCH 33/48] Pass parquet metadata to array reader builder --- parquet/src/arrow/array_reader/builder.rs | 52 ++++++++++++++++--- parquet/src/arrow/array_reader/row_number.rs | 37 ++++++++++--- parquet/src/arrow/arrow_reader/mod.rs | 2 + parquet/src/arrow/async_reader/mod.rs | 2 + parquet/src/file/metadata/mod.rs | 22 ++------ .../src/file/metadata/thrift/encryption.rs | 14 ++--- parquet/src/file/metadata/thrift/mod.rs | 9 +--- 7 files changed, 89 insertions(+), 49 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 6dde646c0430..4a14f74e4ea2 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -37,6 +37,7 @@ use crate::arrow::schema::virtual_type::RowNumber; use crate::basic::Type as PhysicalType; use crate::data_type::{BoolType, DoubleType, FloatType, Int32Type, Int64Type, Int96Type}; use crate::errors::{ParquetError, Result}; +use crate::file::metadata::ParquetMetaData; use crate::schema::types::{ColumnDescriptor, ColumnPath, Type}; /// Builder for [`CacheOptions`] @@ -90,6 +91,8 @@ pub struct ArrayReaderBuilder<'a> { row_groups: &'a dyn RowGroups, /// Optional cache options for the array reader cache_options: Option<&'a CacheOptions<'a>>, + /// Parquet metadata for computing virtual column values + parquet_metadata: Option<&'a ParquetMetaData>, /// metrics metrics: &'a ArrowReaderMetrics, } @@ -99,6 +102,7 @@ impl<'a> ArrayReaderBuilder<'a> { Self { row_groups, cache_options: None, + parquet_metadata: None, metrics, } } @@ -109,6 +113,12 @@ impl<'a> ArrayReaderBuilder<'a> { self } + /// Add parquet metadata to the builder for computing virtual column values + pub fn with_parquet_metadata(mut self, parquet_metadata: &'a ParquetMetaData) -> Self { + self.parquet_metadata = Some(parquet_metadata); + self + } + /// Create [`ArrayReader`] from parquet schema, projection mask, and parquet file reader. pub fn build_array_reader( &self, @@ -173,7 +183,16 @@ impl<'a> ArrayReaderBuilder<'a> { } fn build_row_number_reader(&self) -> Result> { - Ok(Box::new(RowNumberReader::try_new(self.row_groups.row_groups())?)) + let parquet_metadata = self.parquet_metadata.ok_or_else(|| { + ParquetError::General( + "ParquetMetaData is required to read virtual row number columns. \ + Use ArrayReaderBuilder::with_parquet_metadata()".to_string() + ) + })?; + Ok(Box::new(RowNumberReader::try_new( + parquet_metadata, + self.row_groups.row_groups(), + )?)) } /// Build array reader for map type. @@ -488,7 +507,29 @@ mod tests { #[test] fn test_create_array_reader_with_row_numbers() { - let file = get_test_file("nulls.snappy.parquet"); + use tempfile::tempfile; + use crate::arrow::ArrowWriter; + use crate::file::serialized_reader::SerializedFileReader; + use arrow_array::{ArrayRef, Int32Array}; + use std::sync::Arc; + + // Create a simple test file with just an int column + let schema = Arc::new(arrow_schema::Schema::new(vec![ + arrow_schema::Field::new("value", DataType::Int32, false), + ])); + + let batch = arrow_array::RecordBatch::try_new( + schema.clone(), + vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef], + ).unwrap(); + + // Write to a temp file + let file = tempfile().unwrap(); + let mut writer = ArrowWriter::try_new(file.try_clone().unwrap(), schema, None).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + // Now read it back let file_reader: Arc = Arc::new(SerializedFileReader::new(file).unwrap()); let file_metadata = file_reader.metadata().file_metadata(); @@ -504,16 +545,13 @@ mod tests { let metrics = ArrowReaderMetrics::disabled(); let array_reader = ArrayReaderBuilder::new(&file_reader, &metrics) + .with_parquet_metadata(file_reader.metadata()) .build_array_reader(fields.as_ref(), &mask) .unwrap(); // Create arrow types let arrow_type = DataType::Struct(Fields::from(vec![ - Field::new( - "b_struct", - DataType::Struct(vec![Field::new("b_c_int", DataType::Int32, true)].into()), - true, - ), + Field::new("value", DataType::Int32, false), row_number_field, ])); diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index 3a942277a91a..c687f5785ea8 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -17,10 +17,11 @@ use crate::arrow::array_reader::ArrayReader; use crate::errors::{ParquetError, Result}; -use crate::file::metadata::RowGroupMetaData; +use crate::file::metadata::{ParquetMetaData, RowGroupMetaData}; use arrow_array::{ArrayRef, Int64Array}; use arrow_schema::DataType; use std::any::Any; +use std::collections::HashSet; use std::sync::Arc; pub(crate) struct RowNumberReader { @@ -30,16 +31,38 @@ pub(crate) struct RowNumberReader { impl RowNumberReader { pub(crate) fn try_new<'a>( + parquet_metadata: &'a ParquetMetaData, row_groups: impl Iterator, ) -> Result { - let ranges = row_groups + // Collect ordinals from the selected row groups + let selected_ordinals: HashSet = row_groups .map(|rg| { - let first_row_index = rg.first_row_index().ok_or(ParquetError::General( - "Row group missing row number".to_string(), - ))?; - Ok(first_row_index..first_row_index + rg.num_rows()) + rg.ordinal().ok_or_else(|| { + ParquetError::General( + "Row group missing ordinal field, required to compute row numbers".to_string() + ) + }) }) - .collect::>>()?; + .collect::>()?; + + // Iterate through all row groups once, computing first_row_index and creating ranges + // This is O(M) where M is total row groups, much better than O(N * O) where N is selected + let mut first_row_index: i64 = 0; + let mut ranges = Vec::new(); + + for rg in parquet_metadata.row_groups() { + if let Some(ordinal) = rg.ordinal() { + if selected_ordinals.contains(&ordinal) { + ranges.push((ordinal, first_row_index..first_row_index + rg.num_rows())); + } + } + first_row_index += rg.num_rows(); + } + + // Sort ranges by ordinal to maintain original row group order + ranges.sort_by_key(|(ordinal, _)| *ordinal); + let ranges: Vec<_> = ranges.into_iter().map(|(_, range)| range).collect(); + Ok(Self { buffered_row_numbers: Vec::new(), remaining_row_numbers: ranges.into_iter().flatten(), diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 61e71af751b4..b7eaab7e8f51 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -970,6 +970,7 @@ impl ParquetRecordBatchReaderBuilder { cache_projection.intersect(&projection); let array_reader = ArrayReaderBuilder::new(&reader, &metrics) + .with_parquet_metadata(&reader.metadata) .build_array_reader(fields.as_deref(), predicate.projection())?; plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; @@ -977,6 +978,7 @@ impl ParquetRecordBatchReaderBuilder { } let array_reader = ArrayReaderBuilder::new(&reader, &metrics) + .with_parquet_metadata(&reader.metadata) .build_array_reader(fields.as_deref(), &projection)?; let read_plan = plan_builder diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index e4d7400268c0..df49656e6b09 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -649,6 +649,7 @@ where .await?; let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) + .with_parquet_metadata(row_group.metadata) .with_cache_options(Some(&cache_options)) .build_array_reader(self.fields.as_deref(), predicate.projection())?; @@ -706,6 +707,7 @@ where let cache_options = cache_options_builder.consumer(); let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) + .with_parquet_metadata(row_group.metadata) .with_cache_options(Some(&cache_options)) .build_array_reader(self.fields.as_deref(), &projection)?; diff --git a/parquet/src/file/metadata/mod.rs b/parquet/src/file/metadata/mod.rs index a97df859fdf1..763025fe142b 100644 --- a/parquet/src/file/metadata/mod.rs +++ b/parquet/src/file/metadata/mod.rs @@ -616,7 +616,6 @@ pub type RowGroupMetaDataPtr = Arc; pub struct RowGroupMetaData { columns: Vec, num_rows: i64, - first_row_index: Option, sorting_columns: Option>, total_byte_size: i64, schema_descr: SchemaDescPtr, @@ -657,11 +656,6 @@ impl RowGroupMetaData { self.num_rows } - /// Returns the global index number for the first row in this row group. - pub fn first_row_index(&self) -> Option { - self.first_row_index - } - /// Returns the sort ordering of the rows in this RowGroup if any pub fn sorting_columns(&self) -> Option<&Vec> { self.sorting_columns.as_ref() @@ -719,7 +713,6 @@ impl RowGroupMetaDataBuilder { schema_descr, file_offset: None, num_rows: 0, - first_row_index: None, sorting_columns: None, total_byte_size: 0, ordinal: None, @@ -732,12 +725,6 @@ impl RowGroupMetaDataBuilder { self } - /// Sets the first row number in this row group. - pub fn set_first_row_index(mut self, value: i64) -> Self { - self.0.first_row_index = Some(value); - self - } - /// Sets the sorting order for columns pub fn set_sorting_columns(mut self, value: Option>) -> Self { self.0.sorting_columns = value; @@ -1631,7 +1618,6 @@ mod tests { .set_num_rows(1000) .set_total_byte_size(2000) .set_column_metadata(columns) - .set_first_row_index(0) .set_ordinal(1) .build() .unwrap(); @@ -1889,10 +1875,10 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let base_expected_size = 2264; + let base_expected_size = 2248; #[cfg(feature = "encryption")] // Not as accurate as it should be: https://github.com/apache/arrow-rs/issues/8472 - let base_expected_size = 2432; + let base_expected_size = 2416; assert_eq!(parquet_meta.memory_size(), base_expected_size); @@ -1921,10 +1907,10 @@ mod tests { .build(); #[cfg(not(feature = "encryption"))] - let bigger_expected_size = 2690; + let bigger_expected_size = 2674; #[cfg(feature = "encryption")] // Not as accurate as it should be: https://github.com/apache/arrow-rs/issues/8472 - let bigger_expected_size = 2858; + let bigger_expected_size = 2842; // more set fields means more memory usage assert!(bigger_expected_size > base_expected_size); diff --git a/parquet/src/file/metadata/thrift/encryption.rs b/parquet/src/file/metadata/thrift/encryption.rs index 43537ad3ea03..9744f0f7a6b5 100644 --- a/parquet/src/file/metadata/thrift/encryption.rs +++ b/parquet/src/file/metadata/thrift/encryption.rs @@ -113,7 +113,6 @@ pub(crate) struct FileCryptoMetaData<'a> { fn row_group_from_encrypted_thrift( mut rg: RowGroupMetaData, decryptor: Option<&FileDecryptor>, - first_row_index: i64, ) -> Result { let schema_descr = rg.schema_descr; @@ -193,7 +192,6 @@ fn row_group_from_encrypted_thrift( Ok(RowGroupMetaData { columns, num_rows, - first_row_index: Some(first_row_index), sorting_columns, total_byte_size, schema_descr, @@ -296,14 +294,10 @@ pub(crate) fn parquet_metadata_with_encryption( } // decrypt column chunk info - let mut first_row_index = 0i64; - let mut decrypted_row_groups = Vec::with_capacity(row_groups.len()); - for rg in row_groups { - let decrypted_rg = row_group_from_encrypted_thrift(rg, file_decryptor.as_ref(), first_row_index)?; - first_row_index += decrypted_rg.num_rows(); - decrypted_row_groups.push(decrypted_rg); - } - let row_groups = decrypted_row_groups; + let row_groups = row_groups + .into_iter() + .map(|rg| row_group_from_encrypted_thrift(rg, file_decryptor.as_ref())) + .collect::>>()?; let metadata = ParquetMetaDataBuilder::new(file_metadata) .set_row_groups(row_groups) diff --git a/parquet/src/file/metadata/thrift/mod.rs b/parquet/src/file/metadata/thrift/mod.rs index 3a866fc14472..14774910961f 100644 --- a/parquet/src/file/metadata/thrift/mod.rs +++ b/parquet/src/file/metadata/thrift/mod.rs @@ -585,11 +585,9 @@ fn read_column_chunk<'a>( fn read_row_group( prot: &mut ThriftSliceInputProtocol, schema_descr: &Arc, - first_row_index: i64, ) -> Result { // create default initialized RowGroupMetaData let mut row_group = RowGroupMetaDataBuilder::new(schema_descr.clone()).build_unchecked(); - row_group.first_row_index = Some(first_row_index); // mask values for required fields const RG_COLUMNS: u8 = 1 << 1; @@ -728,11 +726,8 @@ pub(crate) fn parquet_metadata_from_bytes(buf: &[u8]) -> Result let schema_descr = schema_descr.as_ref().unwrap(); let list_ident = prot.read_list_begin()?; let mut rg_vec = Vec::with_capacity(list_ident.size as usize); - let mut first_row_index = 0i64; for _ in 0..list_ident.size { - let rg = read_row_group(&mut prot, schema_descr, first_row_index)?; - first_row_index += rg.num_rows(); - rg_vec.push(rg); + rg_vec.push(read_row_group(&mut prot, schema_descr)?); } row_groups = Some(rg_vec); } @@ -1591,7 +1586,7 @@ pub(crate) mod tests { schema_descr: Arc, ) -> Result { let mut reader = ThriftSliceInputProtocol::new(buf); - crate::file::metadata::thrift::read_row_group(&mut reader, &schema_descr, 0) + crate::file::metadata::thrift::read_row_group(&mut reader, &schema_descr) } pub(crate) fn read_column_chunk( From 3933d8e109144726a125918728178a9918c3bb5e Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 12:30:03 +0100 Subject: [PATCH 34/48] Add virtual fields outside of the visitor --- parquet/src/arrow/schema/complex.rs | 65 +++---------------------- parquet/src/arrow/schema/mod.rs | 73 ++++++++++++++++++++++++----- 2 files changed, 68 insertions(+), 70 deletions(-) diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 09f6eae092ec..79bd85485434 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -26,8 +26,7 @@ use crate::basic::{ConvertedType, Repetition}; use crate::errors::ParquetError; use crate::errors::Result; use crate::schema::types::{SchemaDescriptor, Type, TypePtr}; -use arrow_schema::{DataType, Field, Fields, SchemaBuilder}; -use arrow_schema::extension::ExtensionType; +use arrow_schema::{DataType, Field, Fields, SchemaBuilder, extension::ExtensionType}; fn get_repetition(t: &Type) -> Repetition { let info = t.get_basic_info(); @@ -132,17 +131,15 @@ impl VisitorContext { /// See [Logical Types] for more information on the conversion algorithm /// /// [Logical Types]: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md -struct Visitor<'a> { +struct Visitor { /// The column index of the next leaf column next_col_idx: usize, /// Mask of columns to include mask: ProjectionMask, - - virtual_columns: &'a [Field], } -impl<'a> Visitor<'a> { +impl Visitor { fn visit_primitive( &mut self, primitive_type: &TypePtr, @@ -209,8 +206,8 @@ impl<'a> Visitor<'a> { None => None, }; - let mut child_fields = SchemaBuilder::with_capacity(parquet_fields.len() + self.virtual_columns.len()); - let mut children = Vec::with_capacity(parquet_fields.len() + self.virtual_columns.len()); + let mut child_fields = SchemaBuilder::with_capacity(parquet_fields.len()); + let mut children = Vec::with_capacity(parquet_fields.len()); // Perform a DFS of children for (idx, parquet_field) in parquet_fields.iter().enumerate() { @@ -244,20 +241,6 @@ impl<'a> Visitor<'a> { } } - if rep_level == 0 && def_level == 0 { - for virtual_column in self.virtual_columns { - // Ensure this is actually a virtual column - assert!( - super::virtual_type::is_virtual_column(virtual_column), - "Field '{}' is not a virtual column. Virtual columns must have extension type names starting with 'arrow.virtual.'", - virtual_column.name() - ); - child_fields.push(virtual_column.clone()); - let child = convert_virtual_field(virtual_column, rep_level, def_level)?; - children.push(child); - } - } - if children.is_empty() { return Ok(None); } @@ -579,7 +562,7 @@ impl<'a> Visitor<'a> { /// - If nullable: def_level = parent_def_level + 1 /// - If required: def_level = parent_def_level /// - rep_level = parent_rep_level (virtual fields are not repeated) -fn convert_virtual_field( +pub(super) fn convert_virtual_field( arrow_field: &Field, parent_rep_level: i16, parent_def_level: i16, @@ -673,44 +656,9 @@ pub fn convert_schema( mask: ProjectionMask, embedded_arrow_schema: Option<&Fields>, ) -> Result> { - convert_schema_with_virtual(schema, mask, embedded_arrow_schema, &[]) -} - -/// Computes the [`ParquetField`] for the provided [`SchemaDescriptor`] with support for virtual columns -/// -/// This function is similar to [`convert_schema`] but allows specifying virtual columns that should be -/// included in the schema. Virtual columns are columns that don't exist in the Parquet file but are -/// generated during reading (e.g., row numbers). -/// -/// # Arguments -/// * `schema` - The Parquet schema descriptor -/// * `mask` - Projection mask to select which columns to include -/// * `embedded_arrow_schema` - Optional embedded Arrow schema from metadata -/// * `virtual_columns` - Virtual columns to append to the schema -/// -/// # Notes -/// - Virtual columns must have extension type names starting with "arrow.virtual." -/// - This does not support out of order column projection -pub fn convert_schema_with_virtual( - schema: &SchemaDescriptor, - mask: ProjectionMask, - embedded_arrow_schema: Option<&Fields>, - virtual_columns: &[Field], -) -> Result> { - // Validate that all fields are virtual columns - for field in virtual_columns { - if !super::virtual_type::is_virtual_column(field) { - return Err(ParquetError::ArrowError(format!( - "Field '{}' is not a virtual column. Virtual columns must have extension type names starting with 'arrow.virtual.'", - field.name() - ))); - } - } - let mut visitor = Visitor { next_col_idx: 0, mask, - virtual_columns, }; let context = VisitorContext { @@ -727,7 +675,6 @@ pub fn convert_type(parquet_type: &TypePtr) -> Result { let mut visitor = Visitor { next_col_idx: 0, mask: ProjectionMask::all(), - virtual_columns: &[], }; let context = VisitorContext { diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index 1ebd24375421..1ded88dcb725 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -182,18 +182,69 @@ pub fn parquet_to_arrow_field_levels_with_virtual( } } - match complex::convert_schema_with_virtual(schema, mask, hint, virtual_columns)? { - Some(field) => match &field.arrow_type { - DataType::Struct(fields) => Ok(FieldLevels { - fields: fields.clone(), - levels: Some(field), - }), - _ => unreachable!(), - }, - None => Ok(FieldLevels { - fields: Fields::empty(), - levels: None, + // Convert the regular schema first + let mut parquet_field = match complex::convert_schema(schema, mask, hint)? { + Some(field) => field, + None if virtual_columns.is_empty() => { + return Ok(FieldLevels { + fields: Fields::empty(), + levels: None, + }); + } + None => { + // No regular fields, but we have virtual columns - create empty root struct + ParquetField { + rep_level: 0, + def_level: 0, + nullable: false, + arrow_type: DataType::Struct(Fields::empty()), + field_type: ParquetFieldType::Group { + children: Vec::new(), + }, + } + } + }; + + // Append virtual columns if any + if !virtual_columns.is_empty() { + match &mut parquet_field.field_type { + ParquetFieldType::Group { children } => { + // Get the mutable fields from the struct type + let DataType::Struct(ref mut fields) = parquet_field.arrow_type else { + unreachable!("Root field must be a struct"); + }; + + // Convert to mutable Vec to append + let mut fields_vec: Vec> = fields.iter().cloned().collect(); + + // Append each virtual column + for virtual_column in virtual_columns { + // Virtual columns can only be added at the root level + assert_eq!(parquet_field.rep_level, 0, "Virtual columns can only be added at rep level 0"); + assert_eq!(parquet_field.def_level, 0, "Virtual columns can only be added at def level 0"); + + fields_vec.push(Arc::new(virtual_column.clone())); + let virtual_parquet_field = complex::convert_virtual_field( + virtual_column, + parquet_field.rep_level, + parquet_field.def_level, + )?; + children.push(virtual_parquet_field); + } + + // Update the fields + parquet_field.arrow_type = DataType::Struct(Fields::from(fields_vec)); + } + _ => unreachable!("Root field must be a group"), + } + } + + match &parquet_field.arrow_type { + DataType::Struct(fields) => Ok(FieldLevels { + fields: fields.clone(), + levels: Some(parquet_field), }), + _ => unreachable!(), } } From e5449e1a5bb4b27849921c636961eaa029805d1f Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 12:31:09 +0100 Subject: [PATCH 35/48] use parquet.virtual instead of arrow.virtual --- parquet/src/arrow/schema/virtual_type.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index f4c5e5757318..0601c74cd557 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -21,12 +21,12 @@ use arrow_schema::{ArrowError, DataType, Field, extension::ExtensionType}; /// Prefix for virtual column extension type names. -const VIRTUAL_PREFIX: &str = "arrow.virtual."; +const VIRTUAL_PREFIX: &str = "parquet.virtual."; /// Macro to concatenate VIRTUAL_PREFIX with a suffix. macro_rules! virtual_name { ($suffix:literal) => { - concat!("arrow.virtual.", $suffix) + concat!("parquet.virtual.", $suffix) }; } From a2c55dc025d0819c477aab86afb1812c29a30a64 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 13:11:42 +0100 Subject: [PATCH 36/48] more struct based approach to virtual type reuse --- parquet/src/arrow/schema/virtual_type.rs | 45 ++++++++++++++++-------- 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index 0601c74cd557..7bfc102e232c 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -19,6 +19,7 @@ //! use arrow_schema::{ArrowError, DataType, Field, extension::ExtensionType}; +use core::marker::PhantomData; /// Prefix for virtual column extension type names. const VIRTUAL_PREFIX: &str = "parquet.virtual."; @@ -30,10 +31,11 @@ macro_rules! virtual_name { }; } -/// Constants for virtual column type identifiers. -mod virtual_column_type { - /// Row number virtual column. - pub(super) const ROW_NUMBER: u8 = 0; +/// Trait for virtual column name constants. +/// +/// Implementors must provide a unique name suffix for their virtual column type. +trait VirtualColumnName: Default { + const NAME: &'static str; } /// Generic virtual column extension type. @@ -42,13 +44,19 @@ mod virtual_column_type { /// /// The storage type of the extension is `Int64`. #[derive(Debug, Default, Clone, Copy, PartialEq)] -pub struct VirtualColumn; +pub struct VirtualColumn { + _m: PhantomData, +} -impl ExtensionType for VirtualColumn { - const NAME: &'static str = match TYPE { - virtual_column_type::ROW_NUMBER => virtual_name!("row_number"), - _ => panic!("Unknown virtual column type"), - }; +// Constructors & helpers +impl VirtualColumn { + fn new() -> Self { + Self { _m: PhantomData } + } +} + +impl ExtensionType for VirtualColumn { + const NAME: &'static str = N::NAME; type Metadata = &'static str; @@ -80,14 +88,23 @@ impl ExtensionType for VirtualColumn { } fn try_new(data_type: &DataType, _metadata: Self::Metadata) -> Result { - Self.supports_data_type(data_type).map(|_| Self) + Self::default().supports_data_type(data_type).map(|_| Self::default()) } } + +/// Marker type for row number virtual column. +#[derive(Debug, Default, Clone, Copy, PartialEq)] +pub(crate) struct RowNumberName; + +impl VirtualColumnName for RowNumberName { + const NAME: &'static str = virtual_name!("row_number"); +} + /// The extension type for row numbers. /// -/// Extension name: `arrow.virtual.row_number`. -pub type RowNumber = VirtualColumn<{ virtual_column_type::ROW_NUMBER }>; +/// Extension name: `parquet.virtual.row_number`. +pub type RowNumber = VirtualColumn; #[cfg(test)] mod tests { @@ -155,7 +172,7 @@ mod tests { /// Returns `true` if the field is a virtual column. /// -/// Virtual columns have extension type names starting with `arrow.virtual.`. +/// Virtual columns have extension type names starting with `parquet.virtual.`. pub fn is_virtual_column(field: &Field) -> bool { field.extension_type_name() .map(|name| name.starts_with(VIRTUAL_PREFIX)) From 688ce7bd8e4bbb4228200e6853e4fdd318c07d30 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 13:27:49 +0100 Subject: [PATCH 37/48] Switch to directly implementing ExtensionType for RowNumber, no common struct for virtual types --- parquet/src/arrow/schema/virtual_type.rs | 53 ++++-------------------- 1 file changed, 7 insertions(+), 46 deletions(-) diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index 7bfc102e232c..d9c4f941834e 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -19,45 +19,20 @@ //! use arrow_schema::{ArrowError, DataType, Field, extension::ExtensionType}; -use core::marker::PhantomData; /// Prefix for virtual column extension type names. const VIRTUAL_PREFIX: &str = "parquet.virtual."; -/// Macro to concatenate VIRTUAL_PREFIX with a suffix. -macro_rules! virtual_name { - ($suffix:literal) => { - concat!("parquet.virtual.", $suffix) - }; -} - -/// Trait for virtual column name constants. -/// -/// Implementors must provide a unique name suffix for their virtual column type. -trait VirtualColumnName: Default { - const NAME: &'static str; -} - -/// Generic virtual column extension type. +/// The extension type for row numbers. /// -/// This struct provides a common implementation for all virtual column types. +/// Extension name: `parquet.virtual.row_number`. /// -/// The storage type of the extension is `Int64`. +/// This virtual column has storage type `Int64` and uses empty string metadata. #[derive(Debug, Default, Clone, Copy, PartialEq)] -pub struct VirtualColumn { - _m: PhantomData, -} - -// Constructors & helpers -impl VirtualColumn { - fn new() -> Self { - Self { _m: PhantomData } - } -} - -impl ExtensionType for VirtualColumn { - const NAME: &'static str = N::NAME; +pub struct RowNumber; +impl ExtensionType for RowNumber { + const NAME: &'static str = concat!(VIRTUAL_PREFIX, "row_number"); type Metadata = &'static str; fn metadata(&self) -> &Self::Metadata { @@ -88,24 +63,10 @@ impl ExtensionType for VirtualColumn { } fn try_new(data_type: &DataType, _metadata: Self::Metadata) -> Result { - Self::default().supports_data_type(data_type).map(|_| Self::default()) + Self.supports_data_type(data_type).map(|_| Self) } } - -/// Marker type for row number virtual column. -#[derive(Debug, Default, Clone, Copy, PartialEq)] -pub(crate) struct RowNumberName; - -impl VirtualColumnName for RowNumberName { - const NAME: &'static str = virtual_name!("row_number"); -} - -/// The extension type for row numbers. -/// -/// Extension name: `parquet.virtual.row_number`. -pub type RowNumber = VirtualColumn; - #[cfg(test)] mod tests { use arrow_schema::{ From 8e7f66808f4cf7192e27a9fa71e706af20ec6660 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 14:02:29 +0100 Subject: [PATCH 38/48] Use FieldRef --- parquet/src/arrow/array_reader/builder.rs | 4 ++-- parquet/src/arrow/arrow_reader/mod.rs | 20 ++++++++++---------- parquet/src/arrow/async_reader/mod.rs | 4 ++-- parquet/src/arrow/schema/mod.rs | 12 ++++++------ 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 4a14f74e4ea2..8ed02b37a30c 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -534,7 +534,7 @@ mod tests { let file_metadata = file_reader.metadata().file_metadata(); let mask = ProjectionMask::leaves(file_metadata.schema_descr(), [0]); - let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default())); let (_, fields) = parquet_to_arrow_schema_and_fields( file_metadata.schema_descr(), ProjectionMask::all(), @@ -552,7 +552,7 @@ mod tests { // Create arrow types let arrow_type = DataType::Struct(Fields::from(vec![ Field::new("value", DataType::Int32, false), - row_number_field, + (*row_number_field).clone(), ])); assert_eq!(array_reader.get_data_type(), &arrow_type); diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index b7eaab7e8f51..8e4a4ddb8d82 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -20,7 +20,7 @@ use arrow_array::Array; use arrow_array::cast::AsArray; use arrow_array::{RecordBatch, RecordBatchReader}; -use arrow_schema::{ArrowError, DataType as ArrowType, Field, Schema, SchemaRef}; +use arrow_schema::{ArrowError, DataType as ArrowType, Field, FieldRef, Schema, SchemaRef}; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; @@ -389,7 +389,7 @@ pub struct ArrowReaderOptions { #[cfg(feature = "encryption")] pub(crate) file_decryption_properties: Option>, - virtual_columns: Vec + virtual_columns: Vec } impl ArrowReaderOptions { @@ -569,7 +569,7 @@ impl ArrowReaderOptions { /// # Ok(()) /// # } /// ``` - pub fn with_virtual_columns(self, virtual_columns: Vec) -> Self { + pub fn with_virtual_columns(self, virtual_columns: Vec) -> Self { // Validate that all fields are virtual columns for field in &virtual_columns { if !is_virtual_column(field) { @@ -682,7 +682,7 @@ impl ArrowReaderMetadata { fn with_supplied_schema( metadata: Arc, supplied_schema: SchemaRef, - virtual_columns: &[Field], + virtual_columns: &[FieldRef], ) -> Result { let parquet_schema = metadata.file_metadata().schema_descr(); let field_levels = parquet_to_arrow_field_levels_with_virtual( @@ -5099,7 +5099,7 @@ pub(crate) mod tests { Field::new("value", ArrowDataType::Int64, false), ]); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); let options = options.with_virtual_columns(vec![row_number_field.clone()]); @@ -5114,7 +5114,7 @@ pub(crate) mod tests { let batch = arrow_reader.next().unwrap().unwrap(); let schema = Arc::new(Schema::new(vec![ Field::new("value", ArrowDataType::Int64, false), - row_number_field, + (*row_number_field).clone(), ])); assert_eq!(batch.schema(), schema); @@ -5144,7 +5144,7 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field.clone()]); let metadata = ArrowReaderMetadata::load(&file, options).unwrap(); let num_columns = metadata.metadata.file_metadata().schema_descr().num_columns(); @@ -5176,7 +5176,7 @@ pub(crate) mod tests { #[should_panic(expected = "is not a virtual column")] fn test_with_virtual_columns_rejects_non_virtual_fields() { // Try to pass a regular field (not a virtual column) to with_virtual_columns - let regular_field = Field::new("regular_column", ArrowDataType::Int64, false); + let regular_field = Arc::new(Field::new("regular_column", ArrowDataType::Int64, false)); let _options = ArrowReaderOptions::new().with_virtual_columns(vec![regular_field]); } @@ -5186,7 +5186,7 @@ pub(crate) mod tests { false, |path, selection, _row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() @@ -5207,7 +5207,7 @@ pub(crate) mod tests { true, |path, selection, row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index df49656e6b09..018893ac39c4 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -2730,7 +2730,7 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default())); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await @@ -2756,7 +2756,7 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default()); + let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default())); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index 1ded88dcb725..adfad47f3d9a 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -23,7 +23,7 @@ use std::collections::HashMap; use std::sync::Arc; use arrow_ipc::writer; -use arrow_schema::{DataType, Field, Fields, Schema, TimeUnit}; +use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, TimeUnit}; use crate::basic::{ ConvertedType, LogicalType, Repetition, TimeUnit as ParquetTimeUnit, Type as PhysicalType, @@ -75,7 +75,7 @@ pub(crate) fn parquet_to_arrow_schema_and_fields( parquet_schema: &SchemaDescriptor, mask: ProjectionMask, key_value_metadata: Option<&Vec>, - virtual_columns: &[Field], + virtual_columns: &[FieldRef], ) -> Result<(Schema, Option)> { let mut metadata = parse_key_value_metadata(key_value_metadata).unwrap_or_default(); let maybe_schema = metadata @@ -170,7 +170,7 @@ pub fn parquet_to_arrow_field_levels_with_virtual( schema: &SchemaDescriptor, mask: ProjectionMask, hint: Option<&Fields>, - virtual_columns: &[Field], + virtual_columns: &[FieldRef], ) -> Result { // Validate that all fields are virtual columns for field in virtual_columns { @@ -215,7 +215,7 @@ pub fn parquet_to_arrow_field_levels_with_virtual( }; // Convert to mutable Vec to append - let mut fields_vec: Vec> = fields.iter().cloned().collect(); + let mut fields_vec: Vec = fields.iter().cloned().collect(); // Append each virtual column for virtual_column in virtual_columns { @@ -223,7 +223,7 @@ pub fn parquet_to_arrow_field_levels_with_virtual( assert_eq!(parquet_field.rep_level, 0, "Virtual columns can only be added at rep level 0"); assert_eq!(parquet_field.def_level, 0, "Virtual columns can only be added at def level 0"); - fields_vec.push(Arc::new(virtual_column.clone())); + fields_vec.push(virtual_column.clone()); let virtual_parquet_field = complex::convert_virtual_field( virtual_column, parquet_field.rep_level, @@ -2362,7 +2362,7 @@ mod tests { let descriptor = SchemaDescriptor::new(parquet_schema); // Try to pass a regular field (not a virtual column) - let regular_field = Field::new("regular_column", DataType::Int64, false); + let regular_field = Arc::new(Field::new("regular_column", DataType::Int64, false)); let result = parquet_to_arrow_field_levels_with_virtual( &descriptor, ProjectionMask::all(), From 3aeced133bdd14415acfa17fee447b6b4138ff6d Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 14:03:27 +0100 Subject: [PATCH 39/48] row number virtual_prefix sharing --- parquet/src/arrow/schema/virtual_type.rs | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index d9c4f941834e..14fd062f119e 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -21,7 +21,11 @@ use arrow_schema::{ArrowError, DataType, Field, extension::ExtensionType}; /// Prefix for virtual column extension type names. -const VIRTUAL_PREFIX: &str = "parquet.virtual."; +macro_rules! VIRTUAL_PREFIX { + () => { + "parquet.virtual." + }; +} /// The extension type for row numbers. /// @@ -32,7 +36,7 @@ const VIRTUAL_PREFIX: &str = "parquet.virtual."; pub struct RowNumber; impl ExtensionType for RowNumber { - const NAME: &'static str = concat!(VIRTUAL_PREFIX, "row_number"); + const NAME: &'static str = concat!(VIRTUAL_PREFIX!(), "row_number"); type Metadata = &'static str; fn metadata(&self) -> &Self::Metadata { @@ -79,7 +83,7 @@ mod tests { #[test] fn valid() -> Result<(), ArrowError> { let mut field = Field::new("", DataType::Int64, false); - field.try_with_extension_type(RowNumber::default())?; + field.try_with_extension_type(RowNumber)?; field.try_extension_type::()?; Ok(()) @@ -99,7 +103,7 @@ mod tests { #[test] #[should_panic(expected = "expected Int64, found Int32")] fn invalid_type() { - Field::new("", DataType::Int32, false).with_extension_type(RowNumber::default()); + Field::new("", DataType::Int32, false).with_extension_type(RowNumber); } #[test] @@ -136,6 +140,6 @@ mod tests { /// Virtual columns have extension type names starting with `parquet.virtual.`. pub fn is_virtual_column(field: &Field) -> bool { field.extension_type_name() - .map(|name| name.starts_with(VIRTUAL_PREFIX)) + .map(|name| name.starts_with(VIRTUAL_PREFIX!())) .unwrap_or(false) } From 31679f191bd2bf5edb0212c55dcf82acce34e604 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 14:03:43 +0100 Subject: [PATCH 40/48] RowNumber instead of RowNumber::default() --- parquet/src/arrow/array_reader/builder.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 10 +++++----- parquet/src/arrow/async_reader/mod.rs | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 8ed02b37a30c..4b00442e0e97 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -534,7 +534,7 @@ mod tests { let file_metadata = file_reader.metadata().file_metadata(); let mask = ProjectionMask::leaves(file_metadata.schema_descr(), [0]); - let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber)); let (_, fields) = parquet_to_arrow_schema_and_fields( file_metadata.schema_descr(), ProjectionMask::all(), diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 8e4a4ddb8d82..b6ad2e63ccc6 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -548,7 +548,7 @@ impl ArrowReaderOptions { /// /// // Create a virtual column for row numbers /// let row_number_field = Field::new("row_number", DataType::Int64, false) - /// .with_extension_type(RowNumber::default()); + /// .with_extension_type(RowNumber); /// /// // Configure options with virtual columns /// let options = ArrowReaderOptions::new() @@ -5099,7 +5099,7 @@ pub(crate) mod tests { Field::new("value", ArrowDataType::Int64, false), ]); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); let options = options.with_virtual_columns(vec![row_number_field.clone()]); @@ -5144,7 +5144,7 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field.clone()]); let metadata = ArrowReaderMetadata::load(&file, options).unwrap(); let num_columns = metadata.metadata.file_metadata().schema_descr().num_columns(); @@ -5186,7 +5186,7 @@ pub(crate) mod tests { false, |path, selection, _row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() @@ -5207,7 +5207,7 @@ pub(crate) mod tests { true, |path, selection, row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index 018893ac39c4..218d4302e510 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -2730,7 +2730,7 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber)); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await @@ -2756,7 +2756,7 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber::default())); + let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber)); let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await From 83a20c63fa21203a789061f214c3fdbbcdfaf198 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Fri, 7 Nov 2025 14:24:51 +0100 Subject: [PATCH 41/48] Default ordinals --- parquet/src/arrow/array_reader/builder.rs | 30 ++++--------------- .../src/file/metadata/thrift/encryption.rs | 27 +++++++++++++++-- parquet/src/file/metadata/thrift/mod.rs | 26 ++++++++++++++-- 3 files changed, 55 insertions(+), 28 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 4b00442e0e97..7bd7a46982f2 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -507,29 +507,7 @@ mod tests { #[test] fn test_create_array_reader_with_row_numbers() { - use tempfile::tempfile; - use crate::arrow::ArrowWriter; - use crate::file::serialized_reader::SerializedFileReader; - use arrow_array::{ArrayRef, Int32Array}; - use std::sync::Arc; - - // Create a simple test file with just an int column - let schema = Arc::new(arrow_schema::Schema::new(vec![ - arrow_schema::Field::new("value", DataType::Int32, false), - ])); - - let batch = arrow_array::RecordBatch::try_new( - schema.clone(), - vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef], - ).unwrap(); - - // Write to a temp file - let file = tempfile().unwrap(); - let mut writer = ArrowWriter::try_new(file.try_clone().unwrap(), schema, None).unwrap(); - writer.write(&batch).unwrap(); - writer.close().unwrap(); - - // Now read it back + let file = get_test_file("nulls.snappy.parquet"); let file_reader: Arc = Arc::new(SerializedFileReader::new(file).unwrap()); let file_metadata = file_reader.metadata().file_metadata(); @@ -551,7 +529,11 @@ mod tests { // Create arrow types let arrow_type = DataType::Struct(Fields::from(vec![ - Field::new("value", DataType::Int32, false), + Field::new( + "b_struct", + DataType::Struct(vec![Field::new("b_c_int", DataType::Int32, true)].into()), + true, + ), (*row_number_field).clone(), ])); diff --git a/parquet/src/file/metadata/thrift/encryption.rs b/parquet/src/file/metadata/thrift/encryption.rs index 9744f0f7a6b5..e5e7be914920 100644 --- a/parquet/src/file/metadata/thrift/encryption.rs +++ b/parquet/src/file/metadata/thrift/encryption.rs @@ -293,10 +293,33 @@ pub(crate) fn parquet_metadata_with_encryption( file_decryptor = Some(file_decryptor_value); } - // decrypt column chunk info + // decrypt column chunk info and handle ordinal assignment + let mut first_has_ordinal = false; let row_groups = row_groups .into_iter() - .map(|rg| row_group_from_encrypted_thrift(rg, file_decryptor.as_ref())) + .enumerate() + .map(|(ordinal, rg)| { + let mut rg = row_group_from_encrypted_thrift(rg, file_decryptor.as_ref())?; + + // Check first row group to determine ordinal strategy + let rg_has_ordinal = rg.ordinal.is_some(); + if ordinal == 0 { + first_has_ordinal = rg_has_ordinal; + } + + if !first_has_ordinal && !rg_has_ordinal { + rg.ordinal = Some(ordinal as i16); + } else if first_has_ordinal != rg_has_ordinal { + return Err(general_err!( + "Inconsistent ordinal assignment: first_has_ordinal is set to {} but first_has_ordinal for row-group {} is set to{}", + first_has_ordinal, + ordinal, + rg_has_ordinal + )); + } + + Ok(rg) + }) .collect::>>()?; let metadata = ParquetMetaDataBuilder::new(file_metadata) diff --git a/parquet/src/file/metadata/thrift/mod.rs b/parquet/src/file/metadata/thrift/mod.rs index 14774910961f..8577b45719ab 100644 --- a/parquet/src/file/metadata/thrift/mod.rs +++ b/parquet/src/file/metadata/thrift/mod.rs @@ -726,8 +726,30 @@ pub(crate) fn parquet_metadata_from_bytes(buf: &[u8]) -> Result let schema_descr = schema_descr.as_ref().unwrap(); let list_ident = prot.read_list_begin()?; let mut rg_vec = Vec::with_capacity(list_ident.size as usize); - for _ in 0..list_ident.size { - rg_vec.push(read_row_group(&mut prot, schema_descr)?); + + // Read row groups and handle ordinal assignment + let mut first_has_ordinal = false; + for ordinal in 0..list_ident.size { + let mut rg = read_row_group(&mut prot, schema_descr)?; + + // Check first row group to determine ordinal strategy + let rg_has_ordinal = rg.ordinal.is_some(); + if ordinal == 0 { + first_has_ordinal = rg_has_ordinal; + } + + if !first_has_ordinal && !rg_has_ordinal { + rg.ordinal = Some(ordinal as i16); + } else if first_has_ordinal != rg_has_ordinal { + return Err(general_err!( + "Inconsistent ordinal assignment: first_has_ordinal is set to {} but first_has_ordinal for row-group {} is set to{}", + first_has_ordinal, + ordinal, + rg_has_ordinal + )); + } + + rg_vec.push(rg); } row_groups = Some(rg_vec); } From 3e3b90f60235330ac695bc48876c9b8faddb2e05 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 14:34:20 +0100 Subject: [PATCH 42/48] merge fixes --- parquet/src/arrow/array_reader/builder.rs | 3 +- parquet/src/arrow/arrow_reader/mod.rs | 2 +- parquet/src/arrow/async_reader/mod.rs | 67 +++++++++---------- parquet/src/arrow/in_memory_row_group.rs | 6 +- .../arrow/push_decoder/reader_builder/mod.rs | 4 +- 5 files changed, 42 insertions(+), 40 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 7bd7a46982f2..ed4abd59432b 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -185,8 +185,7 @@ impl<'a> ArrayReaderBuilder<'a> { fn build_row_number_reader(&self) -> Result> { let parquet_metadata = self.parquet_metadata.ok_or_else(|| { ParquetError::General( - "ParquetMetaData is required to read virtual row number columns. \ - Use ArrayReaderBuilder::with_parquet_metadata()".to_string() + "ParquetMetaData is required to read virtual row number columns.".to_string() ) })?; Ok(Box::new(RowNumberReader::try_new( diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index e7aeda786f28..64d92005039a 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1231,7 +1231,7 @@ impl ParquetRecordBatchReader { // 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())?; + .build_array_reader(levels.levels.as_ref(), &ProjectionMask::all())?; // TODO: .with_parquet_metadata(&reader.metadata) 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 df8c58b9d6dd..d03471f7913d 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -45,7 +45,7 @@ use crate::bloom_filter::{ SBBF_HEADER_SIZE_ESTIMATE, Sbbf, chunk_read_bloom_filter_header_and_offset, }; use crate::errors::{ParquetError, Result}; -use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; +use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader}; mod metadata; pub use metadata::*; @@ -510,45 +510,42 @@ impl ParquetRecordBatchStreamBuilder { selection, batch_size, row_groups, - projection: self.projection, - selection: self.selection, - schema, - reader_factory: Some(reader_factory), - state: StreamState::Init, + limit, + offset, + metrics, + max_predicate_cache_size, + } + .build()?; + + let request_state = RequestState::None { input: input.0 }; + + Ok(ParquetRecordBatchStream { + schema: projected_schema, + decoder, + request_state, }) } } -/// Returns a [`ReaderFactory`] and an optional [`ParquetRecordBatchReader`] for the next row group +/// State machine that tracks outstanding requests to fetch data /// -/// Note: If all rows are filtered out in the row group (e.g by filters, limit or -/// offset), returns `None` for the reader. -type ReadResult = Result<(ReaderFactory, Option)>; - -/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create -/// [`ParquetRecordBatchReader`] -struct ReaderFactory { - metadata: Arc, - - /// Top level parquet schema - fields: Option>, - - input: T, - - /// Optional filter - filter: Option, - - /// Limit to apply to remaining row groups. - limit: Option, - - /// Offset to apply to the next - offset: Option, - - /// Metrics - metrics: ArrowReaderMetrics, - - /// Maximum size of the predicate cache - max_predicate_cache_size: usize, +/// The parameter `T` is the input, typically an `AsyncFileReader` +enum RequestState { + /// No outstanding requests + None { + input: T, + }, + /// There is an outstanding request for data + Outstanding { + /// Ranges that have been requested + ranges: Vec>, + /// Future that will resolve (input, requested_ranges) + /// + /// Note the future owns the reader while the request is outstanding + /// and returns it upon completion + future: BoxFuture<'static, Result<(T, Vec)>>, + }, + Done, } impl RequestState diff --git a/parquet/src/arrow/in_memory_row_group.rs b/parquet/src/arrow/in_memory_row_group.rs index 34e46cd34e91..b77626e94bc2 100644 --- a/parquet/src/arrow/in_memory_row_group.rs +++ b/parquet/src/arrow/in_memory_row_group.rs @@ -20,7 +20,7 @@ use crate::arrow::array_reader::RowGroups; use crate::arrow::arrow_reader::RowSelection; use crate::column::page::{PageIterator, PageReader}; use crate::errors::ParquetError; -use crate::file::metadata::ParquetMetaData; +use crate::file::metadata::{ParquetMetaData, RowGroupMetaData}; use crate::file::page_index::offset_index::OffsetIndexMetaData; use crate::file::reader::{ChunkReader, Length, SerializedPageReader}; use bytes::{Buf, Bytes}; @@ -226,6 +226,10 @@ impl RowGroups for InMemoryRowGroup<'_> { } } } + + fn row_groups(&self) -> Box + '_> { + Box::new(std::iter::once(self.metadata.row_group(self.row_group_idx))) + } } /// An in-memory column chunk. diff --git a/parquet/src/arrow/push_decoder/reader_builder/mod.rs b/parquet/src/arrow/push_decoder/reader_builder/mod.rs index a0ced8aa8522..150093d4f327 100644 --- a/parquet/src/arrow/push_decoder/reader_builder/mod.rs +++ b/parquet/src/arrow/push_decoder/reader_builder/mod.rs @@ -425,6 +425,7 @@ impl RowGroupReaderBuilder { let array_reader = ArrayReaderBuilder::new(&row_group, &self.metrics) .with_cache_options(Some(&cache_options)) + .with_parquet_metadata(&self.metadata) .build_array_reader(self.fields.as_deref(), predicate.projection())?; plan_builder = @@ -573,7 +574,8 @@ impl RowGroupReaderBuilder { let plan = plan_builder.build(); // if we have any cached results, connect them up - let array_reader_builder = ArrayReaderBuilder::new(&row_group, &self.metrics); + let array_reader_builder = ArrayReaderBuilder::new(&row_group, &self.metrics) + .with_parquet_metadata(&self.metadata); let array_reader = if let Some(cache_info) = cache_info.as_ref() { let cache_options = cache_info.builder().consumer(); array_reader_builder From 5db91133db7ee8d43a94cf1ccf9c1b6232601dd0 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 15:07:22 +0100 Subject: [PATCH 43/48] Fix example --- parquet/examples/read_with_rowgroup.rs | 41 ++++++++++++++++-------- parquet/src/arrow/array_reader/mod.rs | 7 ++++ parquet/src/arrow/arrow_reader/mod.rs | 7 +++- parquet/src/arrow/in_memory_row_group.rs | 4 +++ 4 files changed, 44 insertions(+), 15 deletions(-) diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index b4a9a4be1fc5..a2dabe94c009 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -22,7 +22,7 @@ use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ProjectionMask, parquet_to_arrow_field_levels}; use parquet::column::page::{PageIterator, PageReader}; use parquet::errors::{ParquetError, Result}; -use parquet::file::metadata::RowGroupMetaData; +use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData}; use parquet::file::reader::{ChunkReader, Length}; use parquet::file::serialized_reader::SerializedPageReader; use std::sync::Arc; @@ -35,10 +35,14 @@ async fn main() -> Result<()> { let mut file = File::open(&path).await.unwrap(); // The metadata could be cached in other places, this example only shows how to read - let metadata = file.get_metadata(None).await?; - - for rg in metadata.row_groups() { - let mut rowgroup = InMemoryRowGroup::create(rg.clone(), ProjectionMask::all()); + let metadata = Arc::try_unwrap(file.get_metadata(None).await?).unwrap(); + + for row_group_idx in 0..metadata.row_groups().len() { + let mut rowgroup = InMemoryRowGroup::create( + metadata.clone(), + row_group_idx, + ProjectionMask::all(), + ); rowgroup.async_fetch_data(&mut file, None).await?; let reader = rowgroup.build_reader(1024, None)?; @@ -100,14 +104,15 @@ impl ChunkReader for ColumnChunkData { #[derive(Clone)] pub struct InMemoryRowGroup { - pub metadata: RowGroupMetaData, + metadata: ParquetMetaData, + row_group_idx: usize, mask: ProjectionMask, column_chunks: Vec>>, } impl RowGroups for InMemoryRowGroup { fn num_rows(&self) -> usize { - self.metadata.num_rows() as usize + self.row_group_metadata().num_rows() as usize } fn column_chunks(&self, i: usize) -> Result> { @@ -118,7 +123,7 @@ impl RowGroups for InMemoryRowGroup { Some(data) => { let page_reader: Box = Box::new(SerializedPageReader::new( data.clone(), - self.metadata.column(i), + self.row_group_metadata().column(i), self.num_rows(), None, )?); @@ -131,31 +136,39 @@ impl RowGroups for InMemoryRowGroup { } fn row_groups(&self) -> Box + '_> { - Box::new(std::iter::once(&self.metadata)) + Box::new(std::iter::once(self.row_group_metadata())) + } + + fn metadata(&self) -> &ParquetMetaData { + &self.metadata } } impl InMemoryRowGroup { - pub fn create(metadata: RowGroupMetaData, mask: ProjectionMask) -> Self { - let column_chunks = metadata.columns().iter().map(|_| None).collect::>(); + pub fn create(metadata: ParquetMetaData, row_group_idx: usize, mask: ProjectionMask) -> Self { + let column_chunks = metadata.row_group(row_group_idx).columns().iter().map(|_| None).collect::>(); Self { metadata, + row_group_idx, mask, column_chunks, } } + pub fn row_group_metadata(&self) -> &RowGroupMetaData { + self.metadata.row_group(self.row_group_idx) + } + pub fn build_reader( &self, batch_size: usize, selection: Option, ) -> Result { let levels = parquet_to_arrow_field_levels( - &self.metadata.schema_descr_ptr(), + &self.row_group_metadata().schema_descr_ptr(), self.mask.clone(), None, - &[], )?; ParquetRecordBatchReader::try_new_with_row_groups(&levels, self, batch_size, selection) @@ -168,7 +181,7 @@ impl InMemoryRowGroup { _selection: Option<&RowSelection>, ) -> Result<()> { let mut vs = std::mem::take(&mut self.column_chunks); - for (leaf_idx, meta) in self.metadata.columns().iter().enumerate() { + for (leaf_idx, meta) in self.row_group_metadata().columns().iter().enumerate() { if self.mask.leaf_included(leaf_idx) { let (start, len) = meta.byte_range(); let data = reader.get_bytes(start..(start + len)).await?; diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 7fef792a8722..4840bf9763ed 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -27,6 +27,7 @@ use crate::arrow::record_reader::GenericRecordReader; use crate::arrow::record_reader::buffer::ValuesBuffer; use crate::column::page::PageIterator; use crate::column::reader::decoder::ColumnValueDecoder; +use crate::file::metadata::ParquetMetaData; use crate::file::reader::{FilePageIterator, FileReader}; mod builder; @@ -143,6 +144,8 @@ pub trait RowGroups { /// Returns an iterator over the row groups in this collection fn row_groups(&self) -> Box + '_>; + + fn metadata(&self) -> &ParquetMetaData; } impl RowGroups for Arc { @@ -158,6 +161,10 @@ impl RowGroups for Arc { fn row_groups(&self) -> Box + '_> { Box::new(self.metadata().row_groups().iter()) } + + fn metadata(&self) -> &ParquetMetaData { + self.metadata() + } } /// Uses `record_reader` to read up to `batch_size` records from `pages` diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 64d92005039a..ced2c54b153a 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -1052,6 +1052,10 @@ impl RowGroups for ReaderRowGroups { .map(move |i| self.metadata.row_group(*i)), ) } + + fn metadata(&self) -> &ParquetMetaData { + self.metadata.as_ref() + } } struct ReaderPageIterator { @@ -1231,7 +1235,8 @@ impl ParquetRecordBatchReader { // 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())?; // TODO: .with_parquet_metadata(&reader.metadata) + .with_parquet_metadata(row_groups.metadata()) + .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/in_memory_row_group.rs b/parquet/src/arrow/in_memory_row_group.rs index b77626e94bc2..0014bb391630 100644 --- a/parquet/src/arrow/in_memory_row_group.rs +++ b/parquet/src/arrow/in_memory_row_group.rs @@ -230,6 +230,10 @@ impl RowGroups for InMemoryRowGroup<'_> { fn row_groups(&self) -> Box + '_> { Box::new(std::iter::once(self.metadata.row_group(self.row_group_idx))) } + + fn metadata(&self) -> &ParquetMetaData { + self.metadata + } } /// An in-memory column chunk. From b00373b34c7fab16b2d9e6d309d4e9a6adb38b4b Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 15:13:33 +0100 Subject: [PATCH 44/48] cargo fmt --- parquet/examples/read_with_rowgroup.rs | 14 +++--- parquet/src/arrow/array_reader/mod.rs | 1 + parquet/src/arrow/arrow_reader/mod.rs | 59 +++++++++++++++++--------- parquet/src/arrow/async_reader/mod.rs | 18 +++++--- parquet/src/arrow/mod.rs | 4 +- 5 files changed, 64 insertions(+), 32 deletions(-) diff --git a/parquet/examples/read_with_rowgroup.rs b/parquet/examples/read_with_rowgroup.rs index a2dabe94c009..e3c714c2807a 100644 --- a/parquet/examples/read_with_rowgroup.rs +++ b/parquet/examples/read_with_rowgroup.rs @@ -38,11 +38,8 @@ async fn main() -> Result<()> { let metadata = Arc::try_unwrap(file.get_metadata(None).await?).unwrap(); for row_group_idx in 0..metadata.row_groups().len() { - let mut rowgroup = InMemoryRowGroup::create( - metadata.clone(), - row_group_idx, - ProjectionMask::all(), - ); + let mut rowgroup = + InMemoryRowGroup::create(metadata.clone(), row_group_idx, ProjectionMask::all()); rowgroup.async_fetch_data(&mut file, None).await?; let reader = rowgroup.build_reader(1024, None)?; @@ -146,7 +143,12 @@ impl RowGroups for InMemoryRowGroup { impl InMemoryRowGroup { pub fn create(metadata: ParquetMetaData, row_group_idx: usize, mask: ProjectionMask) -> Self { - let column_chunks = metadata.row_group(row_group_idx).columns().iter().map(|_| None).collect::>(); + let column_chunks = metadata + .row_group(row_group_idx) + .columns() + .iter() + .map(|_| None) + .collect::>(); Self { metadata, diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 4840bf9763ed..588f27c82a94 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -145,6 +145,7 @@ pub trait RowGroups { /// Returns an iterator over the row groups in this collection fn row_groups(&self) -> Box + '_>; + /// Returns the parquet metadata fn metadata(&self) -> &ParquetMetaData; } diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index ced2c54b153a..18482ea60b80 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -28,7 +28,9 @@ use std::sync::Arc; pub use crate::arrow::array_reader::RowGroups; use crate::arrow::array_reader::{ArrayReader, ArrayReaderBuilder}; -use crate::arrow::schema::{ParquetField, parquet_to_arrow_schema_and_fields, virtual_type::is_virtual_column}; +use crate::arrow::schema::{ + ParquetField, parquet_to_arrow_schema_and_fields, virtual_type::is_virtual_column, +}; use crate::arrow::{FieldLevels, ProjectionMask, parquet_to_arrow_field_levels_with_virtual}; use crate::basic::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash}; use crate::bloom_filter::{ @@ -38,7 +40,9 @@ use crate::column::page::{PageIterator, PageReader}; #[cfg(feature = "encryption")] use crate::encryption::decrypt::FileDecryptionProperties; use crate::errors::{ParquetError, Result}; -use crate::file::metadata::{PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; +use crate::file::metadata::{ + PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData, +}; use crate::file::reader::{ChunkReader, SerializedPageReader}; use crate::schema::types::SchemaDescriptor; @@ -408,7 +412,7 @@ pub struct ArrowReaderOptions { #[cfg(feature = "encryption")] pub(crate) file_decryption_properties: Option>, - virtual_columns: Vec + virtual_columns: Vec, } impl ArrowReaderOptions { @@ -679,7 +683,11 @@ impl ArrowReaderMetadata { /// of the settings in `options`. See [`Self::load`] to load metadata including the page index if needed. pub fn try_new(metadata: Arc, options: ArrowReaderOptions) -> Result { match options.supplied_schema { - Some(supplied_schema) => Self::with_supplied_schema(metadata, supplied_schema.clone(), &options.virtual_columns), + Some(supplied_schema) => Self::with_supplied_schema( + metadata, + supplied_schema.clone(), + &options.virtual_columns, + ), None => { let kv_metadata = match options.skip_arrow_metadata { true => None, @@ -1282,7 +1290,7 @@ pub(crate) mod tests { use std::sync::Arc; use rand::rngs::StdRng; - use rand::{random, rng, Rng, RngCore, SeedableRng}; + use rand::{Rng, RngCore, SeedableRng, random, rng}; use tempfile::tempfile; use arrow_array::builder::*; @@ -5164,11 +5172,11 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let supplied_fields = Fields::from(vec![ - Field::new("value", ArrowDataType::Int64, false), - ]); + let supplied_fields = Fields::from(vec![Field::new("value", ArrowDataType::Int64, false)]); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); + let row_number_field = Arc::new( + Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber), + ); let options = ArrowReaderOptions::new().with_schema(Arc::new(Schema::new(supplied_fields))); let options = options.with_virtual_columns(vec![row_number_field.clone()]); @@ -5213,10 +5221,17 @@ pub(crate) mod tests { "value", Arc::new(Int64Array::from(vec![1, 2, 3])) as ArrayRef, )]); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); - let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field.clone()]); + let row_number_field = Arc::new( + Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber), + ); + let options = + ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field.clone()]); let metadata = ArrowReaderMetadata::load(&file, options).unwrap(); - let num_columns = metadata.metadata.file_metadata().schema_descr().num_columns(); + let num_columns = metadata + .metadata + .file_metadata() + .schema_descr() + .num_columns(); let mut arrow_reader = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata) .with_projection(ProjectionMask::none(num_columns)) @@ -5224,9 +5239,7 @@ pub(crate) mod tests { .expect("reader with schema"); let batch = arrow_reader.next().unwrap().unwrap(); - let schema = Arc::new(Schema::new(vec![ - row_number_field, - ])); + let schema = Arc::new(Schema::new(vec![row_number_field])); assert_eq!(batch.schema(), schema); assert_eq!(batch.num_columns(), 1); @@ -5255,8 +5268,12 @@ pub(crate) mod tests { false, |path, selection, _row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); - let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let row_number_field = Arc::new( + Field::new("row_number", ArrowDataType::Int64, false) + .with_extension_type(RowNumber), + ); + let options = + ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() .with_row_selection(selection) @@ -5276,8 +5293,12 @@ pub(crate) mod tests { true, |path, selection, row_filter, batch_size| { let file = File::open(path).unwrap(); - let row_number_field = Arc::new(Field::new("row_number", ArrowDataType::Int64, false).with_extension_type(RowNumber)); - let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let row_number_field = Arc::new( + Field::new("row_number", ArrowDataType::Int64, false) + .with_extension_type(RowNumber), + ); + let options = + ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options) .unwrap() .with_row_selection(selection) diff --git a/parquet/src/arrow/async_reader/mod.rs b/parquet/src/arrow/async_reader/mod.rs index d03471f7913d..6eec7d5cb017 100644 --- a/parquet/src/arrow/async_reader/mod.rs +++ b/parquet/src/arrow/async_reader/mod.rs @@ -767,8 +767,8 @@ mod tests { ArrowPredicateFn, ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector, }; use crate::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; - use crate::arrow::{ArrowWriter, ProjectionMask}; use crate::arrow::schema::virtual_type::RowNumber; + use crate::arrow::{ArrowWriter, ProjectionMask}; use crate::file::metadata::ParquetMetaDataReader; use crate::file::properties::WriterProperties; use arrow::compute::kernels::cmp::eq; @@ -2114,8 +2114,12 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber)); - let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let row_number_field = Arc::new( + Field::new("row_number", DataType::Int64, false) + .with_extension_type(RowNumber), + ); + let options = + ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await .unwrap() @@ -2140,8 +2144,12 @@ mod tests { .expect("Could not create runtime"); runtime.block_on(async move { let file = tokio::fs::File::open(path).await.unwrap(); - let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber)); - let options = ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); + let row_number_field = Arc::new( + Field::new("row_number", DataType::Int64, false) + .with_extension_type(RowNumber), + ); + let options = + ArrowReaderOptions::new().with_virtual_columns(vec![row_number_field]); let reader = ParquetRecordBatchStreamBuilder::new_with_options(file, options) .await .unwrap() diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 260b1b0a41e0..244364d8d161 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -209,8 +209,8 @@ use arrow_schema::{FieldRef, Schema}; pub use self::schema::{ ArrowSchemaConverter, FieldLevels, add_encoded_arrow_schema_to_metadata, encode_arrow_schema, - parquet_to_arrow_field_levels, parquet_to_arrow_field_levels_with_virtual, parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, - virtual_type::*, + parquet_to_arrow_field_levels, parquet_to_arrow_field_levels_with_virtual, + parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, virtual_type::*, }; /// Schema metadata key used to store serialized Arrow schema From 6651017d35b62e275535d89d3012f18dd503502e Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 15:19:31 +0100 Subject: [PATCH 45/48] fix infinite loop --- parquet/src/arrow/array_reader/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index 588f27c82a94..d8d975764ed8 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -151,7 +151,7 @@ pub trait RowGroups { impl RowGroups for Arc { fn num_rows(&self) -> usize { - self.metadata().file_metadata().num_rows() as usize + FileReader::metadata(self.as_ref()).file_metadata().num_rows() as usize } fn column_chunks(&self, column_index: usize) -> Result> { @@ -160,11 +160,11 @@ impl RowGroups for Arc { } fn row_groups(&self) -> Box + '_> { - Box::new(self.metadata().row_groups().iter()) + Box::new(FileReader::metadata(self.as_ref()).row_groups().iter()) } fn metadata(&self) -> &ParquetMetaData { - self.metadata() + FileReader::metadata(self.as_ref()) } } From 5ff1cc966d70b0e31441952964914df9b7404776 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 15:22:26 +0100 Subject: [PATCH 46/48] cargo fmt -p parquet ... --- parquet/src/arrow/array_reader/builder.rs | 14 +++++++------ parquet/src/arrow/array_reader/mod.rs | 4 +++- parquet/src/arrow/array_reader/row_number.rs | 3 ++- parquet/src/arrow/schema/complex.rs | 2 +- parquet/src/arrow/schema/mod.rs | 22 +++++++++++++++----- parquet/src/arrow/schema/virtual_type.rs | 17 ++++++++++----- 6 files changed, 43 insertions(+), 19 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index ed4abd59432b..0e71c91be452 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -32,8 +32,8 @@ use crate::arrow::array_reader::{ make_byte_array_reader, }; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; -use crate::arrow::schema::{ParquetField, ParquetFieldType, VirtualColumnType}; use crate::arrow::schema::virtual_type::RowNumber; +use crate::arrow::schema::{ParquetField, ParquetFieldType, VirtualColumnType}; use crate::basic::Type as PhysicalType; use crate::data_type::{BoolType, DoubleType, FloatType, Int32Type, Int64Type, Int96Type}; use crate::errors::{ParquetError, Result}; @@ -124,7 +124,7 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: Option<&ParquetField>, mask: &ProjectionMask, -) -> Result> { + ) -> Result> { let reader = field .and_then(|field| self.build_reader(field, mask).transpose()) .transpose()? @@ -142,7 +142,7 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, -) -> Result>> { + ) -> Result>> { match field.field_type { ParquetFieldType::Primitive { col_idx, .. } => { let Some(reader) = self.build_primitive_reader(field, mask)? else { @@ -185,7 +185,7 @@ impl<'a> ArrayReaderBuilder<'a> { fn build_row_number_reader(&self) -> Result> { let parquet_metadata = self.parquet_metadata.ok_or_else(|| { ParquetError::General( - "ParquetMetaData is required to read virtual row number columns.".to_string() + "ParquetMetaData is required to read virtual row number columns.".to_string(), ) })?; Ok(Box::new(RowNumberReader::try_new( @@ -431,7 +431,7 @@ impl<'a> ArrayReaderBuilder<'a> { &self, field: &ParquetField, mask: &ProjectionMask, -) -> Result>> { + ) -> Result>> { let arrow_fields = match &field.arrow_type { DataType::Struct(children) => children, _ => unreachable!(), @@ -511,7 +511,9 @@ mod tests { let file_metadata = file_reader.metadata().file_metadata(); let mask = ProjectionMask::leaves(file_metadata.schema_descr(), [0]); - let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber)); + let row_number_field = Arc::new( + Field::new("row_number", DataType::Int64, false).with_extension_type(RowNumber), + ); let (_, fields) = parquet_to_arrow_schema_and_fields( file_metadata.schema_descr(), ProjectionMask::all(), diff --git a/parquet/src/arrow/array_reader/mod.rs b/parquet/src/arrow/array_reader/mod.rs index d8d975764ed8..474e1fbeef95 100644 --- a/parquet/src/arrow/array_reader/mod.rs +++ b/parquet/src/arrow/array_reader/mod.rs @@ -151,7 +151,9 @@ pub trait RowGroups { impl RowGroups for Arc { fn num_rows(&self) -> usize { - FileReader::metadata(self.as_ref()).file_metadata().num_rows() as usize + FileReader::metadata(self.as_ref()) + .file_metadata() + .num_rows() as usize } fn column_chunks(&self, column_index: usize) -> Result> { diff --git a/parquet/src/arrow/array_reader/row_number.rs b/parquet/src/arrow/array_reader/row_number.rs index c687f5785ea8..e8116b2936b8 100644 --- a/parquet/src/arrow/array_reader/row_number.rs +++ b/parquet/src/arrow/array_reader/row_number.rs @@ -39,7 +39,8 @@ impl RowNumberReader { .map(|rg| { rg.ordinal().ok_or_else(|| { ParquetError::General( - "Row group missing ordinal field, required to compute row numbers".to_string() + "Row group missing ordinal field, required to compute row numbers" + .to_string(), ) }) }) diff --git a/parquet/src/arrow/schema/complex.rs b/parquet/src/arrow/schema/complex.rs index 79bd85485434..8b85cac479c1 100644 --- a/parquet/src/arrow/schema/complex.rs +++ b/parquet/src/arrow/schema/complex.rs @@ -589,7 +589,7 @@ pub(super) fn convert_virtual_field( "unsupported virtual column type '{}' for field '{}'", extension_name, arrow_field.name() - ))) + ))); } }; diff --git a/parquet/src/arrow/schema/mod.rs b/parquet/src/arrow/schema/mod.rs index adfad47f3d9a..4c98cb6b2998 100644 --- a/parquet/src/arrow/schema/mod.rs +++ b/parquet/src/arrow/schema/mod.rs @@ -34,8 +34,8 @@ use crate::schema::types::{ColumnDescriptor, SchemaDescriptor, Type}; mod complex; mod extension; -pub mod virtual_type; mod primitive; +pub mod virtual_type; use super::PARQUET_FIELD_ID_META_KEY; use crate::arrow::ProjectionMask; @@ -91,7 +91,8 @@ pub(crate) fn parquet_to_arrow_schema_and_fields( } let hint = maybe_schema.as_ref().map(|s| s.fields()); - let field_levels = parquet_to_arrow_field_levels_with_virtual(parquet_schema, mask, hint, virtual_columns)?; + let field_levels = + parquet_to_arrow_field_levels_with_virtual(parquet_schema, mask, hint, virtual_columns)?; let schema = Schema::new_with_metadata(field_levels.fields, metadata); Ok((schema, field_levels.levels)) } @@ -220,8 +221,14 @@ pub fn parquet_to_arrow_field_levels_with_virtual( // Append each virtual column for virtual_column in virtual_columns { // Virtual columns can only be added at the root level - assert_eq!(parquet_field.rep_level, 0, "Virtual columns can only be added at rep level 0"); - assert_eq!(parquet_field.def_level, 0, "Virtual columns can only be added at def level 0"); + assert_eq!( + parquet_field.rep_level, 0, + "Virtual columns can only be added at rep level 0" + ); + assert_eq!( + parquet_field.def_level, 0, + "Virtual columns can only be added at def level 0" + ); fields_vec.push(virtual_column.clone()); let virtual_parquet_field = complex::convert_virtual_field( @@ -2371,6 +2378,11 @@ mod tests { ); assert!(result.is_err()); - assert!(result.unwrap_err().to_string().contains("is not a virtual column")); + assert!( + result + .unwrap_err() + .to_string() + .contains("is not a virtual column") + ); } } diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index 14fd062f119e..7b0f124ba1bd 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -110,9 +110,12 @@ mod tests { #[should_panic(expected = "Virtual column extension type expects an empty string as metadata")] fn missing_metadata() { let field = Field::new("", DataType::Int64, false).with_metadata( - [(EXTENSION_TYPE_NAME_KEY.to_owned(), RowNumber::NAME.to_owned())] - .into_iter() - .collect(), + [( + EXTENSION_TYPE_NAME_KEY.to_owned(), + RowNumber::NAME.to_owned(), + )] + .into_iter() + .collect(), ); field.extension_type::(); } @@ -122,7 +125,10 @@ mod tests { fn invalid_metadata() { let field = Field::new("", DataType::Int64, false).with_metadata( [ - (EXTENSION_TYPE_NAME_KEY.to_owned(), RowNumber::NAME.to_owned()), + ( + EXTENSION_TYPE_NAME_KEY.to_owned(), + RowNumber::NAME.to_owned(), + ), ( EXTENSION_TYPE_METADATA_KEY.to_owned(), "non-empty".to_owned(), @@ -139,7 +145,8 @@ mod tests { /// /// Virtual columns have extension type names starting with `parquet.virtual.`. pub fn is_virtual_column(field: &Field) -> bool { - field.extension_type_name() + field + .extension_type_name() .map(|name| name.starts_with(VIRTUAL_PREFIX!())) .unwrap_or(false) } From 8da925cea6d5b74cdb9738c5f059dd9e7d1381e5 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 15:26:09 +0100 Subject: [PATCH 47/48] Fix clippy too --- parquet/src/arrow/array_reader/builder.rs | 2 +- parquet/src/arrow/arrow_reader/mod.rs | 2 +- parquet/src/arrow/schema/virtual_type.rs | 20 ++++++++++---------- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/parquet/src/arrow/array_reader/builder.rs b/parquet/src/arrow/array_reader/builder.rs index 0e71c91be452..b631f07cac2e 100644 --- a/parquet/src/arrow/array_reader/builder.rs +++ b/parquet/src/arrow/array_reader/builder.rs @@ -32,7 +32,6 @@ use crate::arrow::array_reader::{ make_byte_array_reader, }; use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; -use crate::arrow::schema::virtual_type::RowNumber; use crate::arrow::schema::{ParquetField, ParquetFieldType, VirtualColumnType}; use crate::basic::Type as PhysicalType; use crate::data_type::{BoolType, DoubleType, FloatType, Int32Type, Int64Type, Int96Type}; @@ -469,6 +468,7 @@ impl<'a> ArrayReaderBuilder<'a> { mod tests { use super::*; use crate::arrow::schema::parquet_to_arrow_schema_and_fields; + use crate::arrow::schema::virtual_type::RowNumber; use crate::file::reader::{FileReader, SerializedFileReader}; use crate::util::test_common::file_util::get_test_file; use arrow::datatypes::Field; diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 18482ea60b80..0dc345061ce3 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -20,7 +20,7 @@ use arrow_array::Array; use arrow_array::cast::AsArray; use arrow_array::{RecordBatch, RecordBatchReader}; -use arrow_schema::{ArrowError, DataType as ArrowType, Field, FieldRef, Schema, SchemaRef}; +use arrow_schema::{ArrowError, DataType as ArrowType, FieldRef, Schema, SchemaRef}; pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter}; pub use selection::{RowSelection, RowSelector}; use std::fmt::{Debug, Formatter}; diff --git a/parquet/src/arrow/schema/virtual_type.rs b/parquet/src/arrow/schema/virtual_type.rs index 7b0f124ba1bd..eca2aef08dca 100644 --- a/parquet/src/arrow/schema/virtual_type.rs +++ b/parquet/src/arrow/schema/virtual_type.rs @@ -71,6 +71,16 @@ impl ExtensionType for RowNumber { } } +/// Returns `true` if the field is a virtual column. +/// +/// Virtual columns have extension type names starting with `parquet.virtual.`. +pub fn is_virtual_column(field: &Field) -> bool { + field + .extension_type_name() + .map(|name| name.starts_with(VIRTUAL_PREFIX!())) + .unwrap_or(false) +} + #[cfg(test)] mod tests { use arrow_schema::{ @@ -140,13 +150,3 @@ mod tests { field.extension_type::(); } } - -/// Returns `true` if the field is a virtual column. -/// -/// Virtual columns have extension type names starting with `parquet.virtual.`. -pub fn is_virtual_column(field: &Field) -> bool { - field - .extension_type_name() - .map(|name| name.starts_with(VIRTUAL_PREFIX!())) - .unwrap_or(false) -} From 40db3d6013a2195ef69cf45e0eca26c2a4ed5908 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Mon, 10 Nov 2025 15:34:48 +0100 Subject: [PATCH 48/48] fix doctest too --- parquet/src/arrow/arrow_reader/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 0dc345061ce3..f7225d1dc432 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -572,8 +572,8 @@ impl ArrowReaderOptions { /// writer.close()?; /// /// // Create a virtual column for row numbers - /// let row_number_field = Field::new("row_number", DataType::Int64, false) - /// .with_extension_type(RowNumber); + /// let row_number_field = Arc::new(Field::new("row_number", DataType::Int64, false) + /// .with_extension_type(RowNumber)); /// /// // Configure options with virtual columns /// let options = ArrowReaderOptions::new()