Skip to content

Conversation

alamb
Copy link
Contributor

@alamb alamb commented Jul 24, 2025

Which issue does this PR close?

Rationale for this change

This PR is the first part of separating IO and decode operations in the rust parquet decoder.

Decoupling IO and CPU enables several important usecases:

  1. Different IO patterns (e.g. not buffer the entire row group at once)
  2. Different IO APIs e.g. use io_uring, or OpenDAL, etc.
  3. Deliberate prefetching within a file
  4. Avoid code duplication between the ParquetRecordBatchStreamBuilder and ParquetRecordBatchReaderBuilder

What changes are included in this PR?

  1. Add new ParquetDecoderBuilder, and ParquetDecoder and tests

It is effectively an explicit version of the state machine that is used in existing async reader (where the state machine is encoded as Rust async / await structures)

Are these changes tested?

Yes -- there are extensive tests for the new code

Note that this PR actually adds a 3rd path for control flow (when I claim this will remove duplication!) In follow on PRs I will convert the existing readers to use this new pattern, similarly to the sequence I did for the metadata decoder:

Here is a preview of a PR that consolidates the async reader to use the push decoder internally (and removes one duplicate):

Are there any user-facing changes?

Yes, a new API, but now changes to the existing APIs

@github-actions github-actions bot added the parquet Changes to the parquet crate label Jul 24, 2025
@alamb alamb force-pushed the alamb/parquet_decoder branch from e203c17 to 4fe003f Compare July 25, 2025 18:48
@alamb
Copy link
Contributor Author

alamb commented Jul 28, 2025

Update: I spent this morning creating a "push" based ParquetMetadataDecoder as I found metadata is, of course, needed to configure the parquet metadata decoding process. I am quite pleased with the results

@alamb alamb force-pushed the alamb/parquet_decoder branch 6 times, most recently from cf1f993 to 58c033d Compare August 1, 2025 17:04
@alamb alamb changed the title WIP: Sketch Push Parquet Decoder WIP: Implement Push Parquet Decoder Aug 1, 2025
@alamb alamb force-pushed the alamb/parquet_decoder branch from 58c033d to c6385ae Compare August 1, 2025 17:19
@alamb
Copy link
Contributor Author

alamb commented Aug 1, 2025

And the tests pass 🥁 🚀

Now I will do some test PRs to use this one implementation instead of the async reader (and I'll do the sync reader too)

@alamb alamb changed the title WIP: Implement Push Parquet Decoder Implement Push Parquet Decoder Aug 4, 2025
@alamb alamb force-pushed the alamb/parquet_decoder branch from 5fdc785 to 26f4ce4 Compare August 7, 2025 13:33
@github-actions github-actions bot added arrow Changes to the arrow crate arrow-flight Changes to the arrow-flight crate labels Aug 7, 2025
@alamb alamb force-pushed the alamb/parquet_decoder branch from 26f4ce4 to 89cdfde Compare August 7, 2025 14:04
@alamb alamb force-pushed the alamb/parquet_decoder branch from 89cdfde to bf6dd1f Compare August 8, 2025 19:53
@github-actions github-actions bot removed arrow Changes to the arrow crate arrow-flight Changes to the arrow-flight crate labels Aug 8, 2025
@alamb alamb force-pushed the alamb/parquet_decoder branch 4 times, most recently from cd3eea7 to 4a41365 Compare August 15, 2025 19:39
Copy link
Contributor

@adriangb adriangb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm out of time right now, I will come back and take a look at parquet/src/arrow/push_decoder/reader_builder/mod.rs.

My 2 high level questions for now are:

  1. How does this fit into the current decoders? Does it replace them for all users? Is it something users have to opt into via code, or a feature flag? I'm sure the is obvious somewhere and I just haven't seen it.
  2. You mentioned this unlocks the possibility to widen IO by asking the push decoder to "look ahead" at what might need to be decoded next. I want to dig into what that would look like, I guess it would be something like asking RowGroupDecoderState for "what do you think is the next bit of data you will need"? In the StartData or WaitingOnData states that might be obvious / accurate but in the Start, Filters or WaitingOnFilterData that would be either a conservative or aggressive guess.

metadata: &'a ParquetMetaData,
}

impl InMemoryRowGroup<'_> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes was going to comment the same thing. Is there a reason for not moving it? Since it's just an impl ... moving it should not break any imports, etc.

impl InMemoryRowGroup<'_> {
/// Returns the byte ranges to fetch for the columns specified in
/// `projection` and `selection`.
pub(crate) fn fetch_ranges(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Came here looking to confirm this. It looks like you've taken one big async fn fetch and split it up into multiple smaller pub(crate) fn fetch_ranges and pub(crate) fn fill_column_chunks which seems reasonable to me. No new public APIs. Smaller functions. 👍🏻


// If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the
// `RowSelection`
let mut page_start_offsets: Vec<Vec<u64>> = vec![];
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FWIW that was not pre-allocated in the existing code either so I don't think it's required to get this PR across the line.

_ => (),
}

// Expand selection to batch boundaries only for cached columns
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As a first time reader of this code it would be nice to get 1 more line of comments here with a code pointer or explanation as to how the caching works e.g. what is a cached column

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call -- I added some additional comments with context about the predicate cache and links

Comment on lines 227 to 238
/// Column chunk data representing only a subset of data pages
Sparse {
/// Length of the full column chunk
length: usize,
/// Subset of data pages included in this sparse chunk.
///
/// Each element is a tuple of (page offset within file, page data).
/// Each entry is a complete page and the list is ordered by offset.
data: Vec<(usize, Bytes)>,
},
/// Full column chunk and the offset within the original file
Dense { offset: usize, data: Bytes },
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍🏻 very nice structure

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks -- this is just giving a name to what was already happening in

let metadata = self.metadata.row_group(self.row_group_idx);
if let Some((selection, offset_index)) = selection.zip(self.offset_index) {
let expanded_selection =
selection.expand_to_batch_boundaries(batch_size, self.row_count);
// If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the
// `RowSelection`
let mut page_start_offsets: Vec<Vec<u64>> = vec![];
let fetch_ranges = self
.column_chunks
.iter()
.zip(metadata.columns())
.enumerate()
.filter(|&(idx, (chunk, _chunk_meta))| {
chunk.is_none() && projection.leaf_included(idx)
})
.flat_map(|(idx, (_chunk, chunk_meta))| {
// If the first page does not start at the beginning of the column,
// then we need to also fetch a dictionary page.
let mut ranges: Vec<Range<u64>> = vec![];
let (start, _len) = chunk_meta.byte_range();
match offset_index[idx].page_locations.first() {
Some(first) if first.offset as u64 != start => {
ranges.push(start..first.offset as u64);
}
_ => (),
}
// Expand selection to batch boundaries only for cached columns
let use_expanded = cache_mask.map(|m| m.leaf_included(idx)).unwrap_or(false);
if use_expanded {
ranges.extend(
expanded_selection.scan_ranges(&offset_index[idx].page_locations),
);
} else {
ranges.extend(selection.scan_ranges(&offset_index[idx].page_locations));
}
page_start_offsets.push(ranges.iter().map(|range| range.start).collect());
ranges
})
.collect();
let mut chunk_data = input.get_byte_ranges(fetch_ranges).await?.into_iter();
let mut page_start_offsets = page_start_offsets.into_iter();
for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
if chunk.is_some() || !projection.leaf_included(idx) {
continue;
}
if let Some(offsets) = page_start_offsets.next() {
let mut chunks = Vec::with_capacity(offsets.len());
for _ in 0..offsets.len() {
chunks.push(chunk_data.next().unwrap());
}
*chunk = Some(Arc::new(ColumnChunkData::Sparse {
length: metadata.column(idx).byte_range().1 as usize,
data: offsets
.into_iter()
.map(|x| x as usize)
.zip(chunks.into_iter())
.collect(),
}))
}
}
} else {
let fetch_ranges = self
.column_chunks
.iter()
.enumerate()
.filter(|&(idx, chunk)| chunk.is_none() && projection.leaf_included(idx))
.map(|(idx, _chunk)| {
let column = metadata.column(idx);
let (start, length) = column.byte_range();
start..(start + length)
})
.collect();
let mut chunk_data = input.get_byte_ranges(fetch_ranges).await?.into_iter();
for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
if chunk.is_some() || !projection.leaf_included(idx) {
continue;
}
if let Some(data) = chunk_data.next() {
*chunk = Some(Arc::new(ColumnChunkData::Dense {
offset: metadata.column(idx).byte_range().0 as usize,
data,
}));
}
}
}

(but was stored in local variable names)

fn get(&self, start: u64) -> crate::errors::Result<Bytes> {
match &self {
ColumnChunkData::Sparse { data, .. } => data
.binary_search_by_key(&start, |(offset, _)| *offset as u64)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Comment on lines +261 to +269
impl Length for ColumnChunkData {
/// Return the total length of the full column chunk
fn len(&self) -> u64 {
match &self {
ColumnChunkData::Sparse { length, .. } => *length as u64,
ColumnChunkData::Dense { data, .. } => data.len() as u64,
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👏🏻

self.offset = offset;
self
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems like you've added pub fn clear_ranges(&mut self, ranges_to_clear: &[Range<u64>])?

Copy link
Contributor

@adriangb adriangb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall this looks great @alamb!

I left a lot of nits, but nothing major!

One question I have left is performance, but I think the right place to do that is a follow up PR where you replace the current internal machinery with this, then we can do a main vs. PR type benchmark.

Comment on lines 35 to 36
/// These must be owned by FilterInfo because they may be mutated as part of
/// evaluation so there is a bunch of complexity of handing them back and forth
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Having some familiarity with what is happening in the levels above this I believe it needs to be mutable because the mask of one filter is fed into the next / it's a stack of filters we pop from. I will try to confirm as I read through the rest of the code but a comment explaining why / what mutable state is used for would be helpful.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The core driver is that ArrowPredicate takes &mut self: https://docs.rs/parquet/latest/parquet/arrow/arrow_reader/trait.ArrowPredicate.html

pub trait ArrowPredicate: Send + 'static {
    // Required methods
    fn projection(&self) -> &ProjectionMask;
    fn evaluate(
        &mut self,
        batch: RecordBatch,
    ) -> Result<BooleanArray, ArrowError>;
}

This means that to evaluate the predicate there must be exactly one owner and to get the lifetimes to work out I needed to pass ownership around like this. I will clarify this in the commetns

/// evaluation so there is a bunch of complexity of handing them back and forth
filter: RowFilter,
/// The next filter to be evaluated
next_predicate: NonZeroUsize,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seeing now that we just maintain a mutable index into RowFilter (i.e. don't mutate RowFilter itself I'm guessing)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried to clarify this in comments

Comment on lines +295 to +310
let Some(filter) = self.filter.take() else {
// no filter, start trying to read data immediately
return Ok(NextState::again(RowGroupDecoderState::StartData {
row_group_info,
column_chunks,
cache_info: None,
}));
};
// no predicates in filter, so start reading immediately
if filter.predicates.is_empty() {
return Ok(NextState::again(RowGroupDecoderState::StartData {
row_group_info,
column_chunks,
cache_info: None,
}));
};
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we collapse these two states in RowGroupReaderBuilder::new()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since it happens for each row group (aka row_group_info changes on each row group) I don't think it could be in RowGroupReaderBuilder::new()

We could possibly inline it in RowGroupReaderBuilder::next_row_group() 🤔
https://github.com/apache/arrow-rs/blob/10aae0fc6bc8020f77d584a5fa3cc0c5da605211/parquet/src/arrow/push_decoder/reader_builder/mod.rs#L244-L243

But I am not sure how much better that would be as it just moves the code around?

self.exclude_nested_columns_from_cache(&cache_projection)
}

/// Exclude leaves belonging to roots that span multiple parquet leaves (i.e. nested columns)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Curious why this is? Does this mean that if there is a query like select 1 from t where nested["field"] <15 AND nested["field"] > 5 we will read the leaf for nested["field"] twice?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Short answer is I don't know -- this just preserves the existing semantics

https://github.com/apache/arrow-rs/blob/b9c2bf73e792e7cb849f0bd453059ceef45b0b74/parquet/src/arrow/async_reader/mod.rs#L728-L727

Maybe @XiangpengHao can explain in more detail

///
/// // In a loop, ask the decoder what it needs next, and provide it with the required data
/// loop {
/// match decoder.try_decode().unwrap() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm guessing if I wanted to "look ahead" we'd add a method along the lines of try_peek()? It'd be cool if it returned some structure that allowed fine grained control of the peeking:

let max_ranges = 32;
let max_bytes = 1024 * 1024 * 32;
let mut current_bytes = 0;
let mut ranges = Vec::new();
let mut peek = decoder.peek()
loop {
    match peek.next() {
        PeekResult::Range(range) => {
            ranges.push(range);
            current_bytes += range.end - range.start;
            if ranges.len() > max_ranges { break }
            if current_bytes > max_bytes { break }
        PeekResult::End { break }
    }
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Comment on lines +95 to +109
/// DecodeResult::NeedsData(ranges) => {
/// // The decoder needs more data. Fetch the data for the given ranges
/// let data = ranges.iter().map(|r| get_range(r)).collect::<Vec<_>>();
/// // Push the data to the decoder
/// decoder.push_ranges(ranges, data).unwrap();
/// // After pushing the data, we can try to decode again on the next iteration
/// }
/// DecodeResult::Data(batch) => {
/// // Successfully decoded a batch of data
/// assert!(batch.num_rows() > 0);
/// }
/// DecodeResult::Finished => {
/// // The decoder has finished decoding exit the loop
/// break;
/// }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a very nice high level API!

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am also pleased as it mirrors the Metadata decoder one as well: https://docs.rs/parquet/latest/parquet/file/metadata/struct.ParquetMetaDataPushDecoder.html

Comment on lines +115 to +117
/// Methods for building a ParquetDecoder. See the base [`ArrowReaderBuilder`] for
/// more options that can be configured.
impl ParquetPushDecoderBuilder {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any particular reason for not putting the impl in the same file as the struct?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if you mean alongside ArrowReaderBuilder in arrow_reader/mod.rs it is because I felt that file was already pretty big as it has both generic options along with specific options for sync and async readers

https://github.com/apache/arrow-rs/blob/b9c2bf73e792e7cb849f0bd453059ceef45b0b74/parquet/src/arrow/arrow_reader/mod.rs#L710-L709

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

One question I have left is performance, but I think the right place to do that is a follow up PR where you replace the current internal machinery with this, then we can do a main vs. PR type benchmark.

I agree -- I expect this code to have exactly the same performance as the current async decoder -- it is the same algorithm, just the state machine(s) are now explicit rather than implicit via await

follow up PR where you replace the current internal machinery with this

Indeed. Here is that PR:

I'm out of time right now, I will come back and take a look at parquet/src/arrow/push_decoder/reader_builder/mod.rs.

How does this fit into the current decoders? Does it replace them for all users? Is it something users have to opt into via code, or a feature flag? I'm sure the is obvious somewhere and I just haven't seen it.

Currently this is a entirely new (3rd) decoder. My plan (and I have already done it in #8159) is to rewrite the existing async decoder to use this push decoder.

You mentioned this unlocks the possibility to widen IO by asking the push decoder to "look ahead" at what might need to be decoded next. I want to dig into what that would look like, I guess it would be something like asking RowGroupDecoderState for "what do you think is the next bit of data you will need"? In the StartData or WaitingOnData states that might be obvious / accurate but in the Start, Filters or WaitingOnFilterData that would be either a conservative or aggressive guess.

Yes, exactly.

Another thing that I would like to do is to make it possible to buffer only some of the pages needed for a row group rather than all of them (aka what is stored in InMemoryRowGroup). This would reduce memory requirements for files with large row groups. However, it would also increase the number of IO requests (aka object store requests) so it would have to be configurable to let people trade off the IOs and the memory requirements

@adriangb
Copy link
Contributor

adriangb commented Oct 20, 2025

Another thing that I would like to do is to make it possible to buffer only some of the pages needed for a row group rather than all of them (aka what is stored in InMemoryRowGroup). This would reduce memory requirements for files with large row groups. However, it would also increase the number of IO requests (aka object store requests) so it would have to be configurable to let people trade off the IOs and the memory requirements

IMO the parquet decoder should produce as granular as possible ranges of data to read and the object store implementation can handle coalescing them as needed. Currently some object stores (remote ones that talk to S3, GCS, etc.) already coalesce ranges to be more efficient, but to what degree is hidden behind hardcoded constants (https://github.com/apache/arrow-rs-object-store/blob/ad1d70f4876b0c2ea6c6a5e34dc158c63f861384/src/util.rs#L90-L95). Maybe that's what should be used to tweak the tradeoff between IO round-trips and memory? I guess we still need to decide when to go to hit object storage (i.e. how many ranges or bytes we accumulate before making a request?).

Copy link
Contributor Author

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you so much @adriangb and @mbrobbel for your comments on this PR. Super super helpful.

I'll plan to merge this over the next few days, after I:

  1. file a few more follow on tasks
  2. Verify benchmarks on this PR again
  3. get the PR to rewrite the async decoder ready

I also plan a blog post about this work:

metadata: &'a ParquetMetaData,
}

impl InMemoryRowGroup<'_> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Moved!


// If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the
// `RowSelection`
let mut page_start_offsets: Vec<Vec<u64>> = vec![];
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

_ => (),
}

// Expand selection to batch boundaries only for cached columns
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call -- I added some additional comments with context about the predicate cache and links

Comment on lines 227 to 238
/// Column chunk data representing only a subset of data pages
Sparse {
/// Length of the full column chunk
length: usize,
/// Subset of data pages included in this sparse chunk.
///
/// Each element is a tuple of (page offset within file, page data).
/// Each entry is a complete page and the list is ordered by offset.
data: Vec<(usize, Bytes)>,
},
/// Full column chunk and the offset within the original file
Dense { offset: usize, data: Bytes },
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks -- this is just giving a name to what was already happening in

let metadata = self.metadata.row_group(self.row_group_idx);
if let Some((selection, offset_index)) = selection.zip(self.offset_index) {
let expanded_selection =
selection.expand_to_batch_boundaries(batch_size, self.row_count);
// If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the
// `RowSelection`
let mut page_start_offsets: Vec<Vec<u64>> = vec![];
let fetch_ranges = self
.column_chunks
.iter()
.zip(metadata.columns())
.enumerate()
.filter(|&(idx, (chunk, _chunk_meta))| {
chunk.is_none() && projection.leaf_included(idx)
})
.flat_map(|(idx, (_chunk, chunk_meta))| {
// If the first page does not start at the beginning of the column,
// then we need to also fetch a dictionary page.
let mut ranges: Vec<Range<u64>> = vec![];
let (start, _len) = chunk_meta.byte_range();
match offset_index[idx].page_locations.first() {
Some(first) if first.offset as u64 != start => {
ranges.push(start..first.offset as u64);
}
_ => (),
}
// Expand selection to batch boundaries only for cached columns
let use_expanded = cache_mask.map(|m| m.leaf_included(idx)).unwrap_or(false);
if use_expanded {
ranges.extend(
expanded_selection.scan_ranges(&offset_index[idx].page_locations),
);
} else {
ranges.extend(selection.scan_ranges(&offset_index[idx].page_locations));
}
page_start_offsets.push(ranges.iter().map(|range| range.start).collect());
ranges
})
.collect();
let mut chunk_data = input.get_byte_ranges(fetch_ranges).await?.into_iter();
let mut page_start_offsets = page_start_offsets.into_iter();
for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
if chunk.is_some() || !projection.leaf_included(idx) {
continue;
}
if let Some(offsets) = page_start_offsets.next() {
let mut chunks = Vec::with_capacity(offsets.len());
for _ in 0..offsets.len() {
chunks.push(chunk_data.next().unwrap());
}
*chunk = Some(Arc::new(ColumnChunkData::Sparse {
length: metadata.column(idx).byte_range().1 as usize,
data: offsets
.into_iter()
.map(|x| x as usize)
.zip(chunks.into_iter())
.collect(),
}))
}
}
} else {
let fetch_ranges = self
.column_chunks
.iter()
.enumerate()
.filter(|&(idx, chunk)| chunk.is_none() && projection.leaf_included(idx))
.map(|(idx, _chunk)| {
let column = metadata.column(idx);
let (start, length) = column.byte_range();
start..(start + length)
})
.collect();
let mut chunk_data = input.get_byte_ranges(fetch_ranges).await?.into_iter();
for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
if chunk.is_some() || !projection.leaf_included(idx) {
continue;
}
if let Some(data) = chunk_data.next() {
*chunk = Some(Arc::new(ColumnChunkData::Dense {
offset: metadata.column(idx).byte_range().0 as usize,
data,
}));
}
}
}

(but was stored in local variable names)

///
/// // In a loop, ask the decoder what it needs next, and provide it with the required data
/// loop {
/// match decoder.try_decode().unwrap() {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

/// Offset to apply to remaining row groups (decremented as rows are read)
offset: Option<usize>,

/// The size in bytes of the predicate cache
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added a link to RowGroupCache

Comment on lines +253 to +256
pub(crate) fn try_build(
&mut self,
) -> Result<DecodeResult<ParquetRecordBatchReader>, ParquetError> {
loop {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It isn't called concurrently. try_transition can change the state as decoding proceeds. If the decoder already has enough data to potentially move on to the next state it begins doing so immediately via this loop -- I will make this clearer. in comments

self.exclude_nested_columns_from_cache(&cache_projection)
}

/// Exclude leaves belonging to roots that span multiple parquet leaves (i.e. nested columns)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Short answer is I don't know -- this just preserves the existing semantics

https://github.com/apache/arrow-rs/blob/b9c2bf73e792e7cb849f0bd453059ceef45b0b74/parquet/src/arrow/async_reader/mod.rs#L728-L727

Maybe @XiangpengHao can explain in more detail

self.offset = offset;
self
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will file a ticket to track

Comment on lines +295 to +310
let Some(filter) = self.filter.take() else {
// no filter, start trying to read data immediately
return Ok(NextState::again(RowGroupDecoderState::StartData {
row_group_info,
column_chunks,
cache_info: None,
}));
};
// no predicates in filter, so start reading immediately
if filter.predicates.is_empty() {
return Ok(NextState::again(RowGroupDecoderState::StartData {
row_group_info,
column_chunks,
cache_info: None,
}));
};
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since it happens for each row group (aka row_group_info changes on each row group) I don't think it could be in RowGroupReaderBuilder::new()

We could possibly inline it in RowGroupReaderBuilder::next_row_group() 🤔
https://github.com/apache/arrow-rs/blob/10aae0fc6bc8020f77d584a5fa3cc0c5da605211/parquet/src/arrow/push_decoder/reader_builder/mod.rs#L244-L243

But I am not sure how much better that would be as it just moves the code around?

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

🤖 ./gh_compare_arrow.sh Benchmark Script Running
Linux aal-dev 6.14.0-1017-gcp #18~24.04.1-Ubuntu SMP Tue Sep 23 17:51:44 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Comparing alamb/parquet_decoder (10aae0f) to 9d75f87 diff
BENCH_NAME=arrow_reader
BENCH_COMMAND=cargo bench --features=arrow,async,test_common,experimental --bench arrow_reader
BENCH_FILTER=
BENCH_BRANCH_NAME=alamb_parquet_decoder
Results will be posted here when complete

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

Another thing that I would like to do is to make it possible to buffer only some of the pages needed for a row group rather than all of them (aka what is stored in InMemoryRowGroup). This would reduce memory requirements for files with large row groups. However, it would also increase the number of IO requests (aka object store requests) so it would have to be configurable to let people trade off the IOs and the memory requirements

IMO the parquet decoder should produce as granular as possible ranges of data to read and the object store implementation can handle coalescing them as needed. Currently some object stores (remote ones that talk to S3, GCS, etc.) already coalesce ranges to be more efficient, but to what degree is hidden behind hardcoded constants (https://github.com/apache/arrow-rs-object-store/blob/ad1d70f4876b0c2ea6c6a5e34dc158c63f861384/src/util.rs#L90-L95). Maybe that's what should be used to tweak the tradeoff between IO round-trips and memory? I guess we still need to decide when to go to hit object storage (i.e. how many ranges or bytes we accumulate before making a request?).

I agree it would be good to provide granular requests for data, but I think it is orthogonal to the 'what data to wait for until decoding can start'

Right now, the readers (including the push decoder) will wait until all the data for a RowGroup (after filtering) is fetched

Once the decoder can tell the caller how much data it really needs to decode something, then I think we'll be in a much better position to control CPU vs Memory

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

🤖: Benchmark completed

Details

group                                                                                                      alamb_parquet_decoder                  main
-----                                                                                                      ---------------------                  ----
arrow_array_reader/BYTE_ARRAY/Decimal128Array/plain encoded, mandatory, no NULLs                           1.03   1310.3±3.14µs        ? ?/sec    1.00   1275.7±4.34µs        ? ?/sec
arrow_array_reader/BYTE_ARRAY/Decimal128Array/plain encoded, optional, half NULLs                          1.04   1330.3±2.93µs        ? ?/sec    1.00   1279.3±3.07µs        ? ?/sec
arrow_array_reader/BYTE_ARRAY/Decimal128Array/plain encoded, optional, no NULLs                            1.03   1322.8±4.30µs        ? ?/sec    1.00   1278.9±2.86µs        ? ?/sec
arrow_array_reader/BinaryArray/dictionary encoded, mandatory, no NULLs                                     1.01    495.6±3.57µs        ? ?/sec    1.00    491.3±3.88µs        ? ?/sec
arrow_array_reader/BinaryArray/dictionary encoded, optional, half NULLs                                    1.02    668.1±2.21µs        ? ?/sec    1.00    652.5±2.20µs        ? ?/sec
arrow_array_reader/BinaryArray/dictionary encoded, optional, no NULLs                                      1.03    502.8±4.66µs        ? ?/sec    1.00    488.8±3.02µs        ? ?/sec
arrow_array_reader/BinaryArray/plain encoded, mandatory, no NULLs                                          1.01    555.1±2.14µs        ? ?/sec    1.00    548.4±3.98µs        ? ?/sec
arrow_array_reader/BinaryArray/plain encoded, optional, half NULLs                                         1.04    756.0±3.26µs        ? ?/sec    1.00    724.6±7.77µs        ? ?/sec
arrow_array_reader/BinaryArray/plain encoded, optional, no NULLs                                           1.01    564.3±3.42µs        ? ?/sec    1.00    561.5±2.66µs        ? ?/sec
arrow_array_reader/BinaryViewArray/dictionary encoded, mandatory, no NULLs                                 1.03    261.2±2.54µs        ? ?/sec    1.00    253.4±2.94µs        ? ?/sec
arrow_array_reader/BinaryViewArray/dictionary encoded, optional, half NULLs                                1.00    264.2±0.62µs        ? ?/sec    1.01    266.6±1.06µs        ? ?/sec
arrow_array_reader/BinaryViewArray/dictionary encoded, optional, no NULLs                                  1.02    271.9±3.77µs        ? ?/sec    1.00    267.6±2.55µs        ? ?/sec
arrow_array_reader/BinaryViewArray/plain encoded, mandatory, no NULLs                                      1.23    353.9±2.39µs        ? ?/sec    1.00    287.6±1.29µs        ? ?/sec
arrow_array_reader/BinaryViewArray/plain encoded, mandatory, no NULLs, short string                        1.17    329.1±1.45µs        ? ?/sec    1.00    281.3±0.58µs        ? ?/sec
arrow_array_reader/BinaryViewArray/plain encoded, optional, half NULLs                                     1.12    320.4±1.47µs        ? ?/sec    1.00    286.1±3.59µs        ? ?/sec
arrow_array_reader/BinaryViewArray/plain encoded, optional, no NULLs                                       1.22    363.2±1.84µs        ? ?/sec    1.00    296.8±3.18µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Decimal128Array/byte_stream_split encoded, mandatory, no NULLs     1.00   1051.7±4.30µs        ? ?/sec    1.03   1083.8±5.98µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Decimal128Array/byte_stream_split encoded, optional, half NULLs    1.00    895.1±2.61µs        ? ?/sec    1.02    914.5±1.90µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Decimal128Array/byte_stream_split encoded, optional, no NULLs      1.00   1059.5±4.78µs        ? ?/sec    1.03  1092.2±12.36µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Decimal128Array/plain encoded, mandatory, no NULLs                 1.00    428.5±3.77µs        ? ?/sec    1.09    465.2±4.08µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Decimal128Array/plain encoded, optional, half NULLs                1.00    576.9±2.60µs        ? ?/sec    1.05    603.7±1.81µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Decimal128Array/plain encoded, optional, no NULLs                  1.00    437.5±3.95µs        ? ?/sec    1.08    470.6±3.07µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Float16Array/byte_stream_split encoded, mandatory, no NULLs        1.00    153.2±0.78µs        ? ?/sec    1.00    153.4±0.81µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Float16Array/byte_stream_split encoded, optional, half NULLs       1.03    294.9±1.13µs        ? ?/sec    1.00    285.8±0.49µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Float16Array/byte_stream_split encoded, optional, no NULLs         1.00    158.4±0.34µs        ? ?/sec    1.00    158.4±0.30µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Float16Array/plain encoded, mandatory, no NULLs                    1.00     76.0±0.20µs        ? ?/sec    1.01     76.6±0.68µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Float16Array/plain encoded, optional, half NULLs                   1.05    258.1±1.10µs        ? ?/sec    1.00    244.8±0.47µs        ? ?/sec
arrow_array_reader/FIXED_LEN_BYTE_ARRAY/Float16Array/plain encoded, optional, no NULLs                     1.01     81.5±0.23µs        ? ?/sec    1.00     81.0±0.17µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(16)/byte_stream_split encoded, mandatory, no NULLs                    1.01    688.6±2.65µs        ? ?/sec    1.00    680.6±2.46µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(16)/byte_stream_split encoded, optional, half NULLs                   1.01    520.2±1.27µs        ? ?/sec    1.00    513.6±1.47µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(16)/byte_stream_split encoded, optional, no NULLs                     1.01    695.8±1.78µs        ? ?/sec    1.00    688.6±2.05µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(16)/plain encoded, mandatory, no NULLs                                1.13     65.2±5.49µs        ? ?/sec    1.00     57.4±5.47µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(16)/plain encoded, optional, half NULLs                               1.03    213.2±0.76µs        ? ?/sec    1.00    206.6±1.42µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(16)/plain encoded, optional, no NULLs                                 1.10     72.8±7.18µs        ? ?/sec    1.00     66.0±6.06µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(2)/byte_stream_split encoded, mandatory, no NULLs                     1.00     86.1±0.25µs        ? ?/sec    1.00     85.7±0.25µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(2)/byte_stream_split encoded, optional, half NULLs                    1.05    229.1±0.86µs        ? ?/sec    1.00    217.8±0.41µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(2)/byte_stream_split encoded, optional, no NULLs                      1.00     90.9±0.31µs        ? ?/sec    1.00     91.4±0.34µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(2)/plain encoded, mandatory, no NULLs                                 1.00      9.3±0.17µs        ? ?/sec    1.00      9.3±0.14µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(2)/plain encoded, optional, half NULLs                                1.07    190.2±0.34µs        ? ?/sec    1.00    178.1±0.50µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(2)/plain encoded, optional, no NULLs                                  1.00     14.3±0.20µs        ? ?/sec    1.03     14.8±0.23µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(4)/byte_stream_split encoded, mandatory, no NULLs                     1.00    170.6±0.63µs        ? ?/sec    1.00    169.9±0.54µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(4)/byte_stream_split encoded, optional, half NULLs                    1.04    346.9±3.58µs        ? ?/sec    1.00    335.0±1.06µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(4)/byte_stream_split encoded, optional, no NULLs                      1.00    175.5±0.53µs        ? ?/sec    1.00    175.6±0.39µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(4)/plain encoded, mandatory, no NULLs                                 1.00     13.1±0.20µs        ? ?/sec    1.06     13.8±0.31µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(4)/plain encoded, optional, half NULLs                                1.04    269.2±0.61µs        ? ?/sec    1.00    258.0±1.08µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(4)/plain encoded, optional, no NULLs                                  1.00     20.2±0.57µs        ? ?/sec    1.00     20.1±0.57µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(8)/byte_stream_split encoded, mandatory, no NULLs                     1.00    340.7±0.74µs        ? ?/sec    1.00    340.5±2.49µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(8)/byte_stream_split encoded, optional, half NULLs                    1.03    339.7±1.38µs        ? ?/sec    1.00    329.7±5.67µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(8)/byte_stream_split encoded, optional, no NULLs                      1.00    347.0±0.68µs        ? ?/sec    1.00    347.0±0.87µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(8)/plain encoded, mandatory, no NULLs                                 1.01     26.9±0.27µs        ? ?/sec    1.00     26.5±0.37µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(8)/plain encoded, optional, half NULLs                                1.06    180.9±0.46µs        ? ?/sec    1.00    170.7±0.68µs        ? ?/sec
arrow_array_reader/FixedLenByteArray(8)/plain encoded, optional, no NULLs                                  1.01     33.2±0.43µs        ? ?/sec    1.00     32.7±0.39µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/binary packed skip, mandatory, no NULLs                           1.00    125.2±0.23µs        ? ?/sec    1.00    125.3±0.34µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/binary packed skip, optional, half NULLs                          1.01    125.8±0.53µs        ? ?/sec    1.00    124.4±0.57µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/binary packed skip, optional, no NULLs                            1.01    130.0±7.78µs        ? ?/sec    1.00    128.2±0.21µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/binary packed, mandatory, no NULLs                                1.00    179.2±0.77µs        ? ?/sec    1.01    180.4±0.52µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/binary packed, optional, half NULLs                               1.01    208.2±0.43µs        ? ?/sec    1.00    206.7±0.43µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/binary packed, optional, no NULLs                                 1.00    185.4±1.19µs        ? ?/sec    1.00    186.1±2.02µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/byte_stream_split encoded, mandatory, no NULLs                    1.01     76.9±0.22µs        ? ?/sec    1.00     76.0±0.20µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/byte_stream_split encoded, optional, half NULLs                   1.01    154.3±1.71µs        ? ?/sec    1.00    153.3±2.15µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/byte_stream_split encoded, optional, no NULLs                     1.01     82.4±0.35µs        ? ?/sec    1.00     81.7±0.40µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/dictionary encoded, mandatory, no NULLs                           1.00    134.8±0.68µs        ? ?/sec    1.01    136.1±0.49µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/dictionary encoded, optional, half NULLs                          1.01    186.6±0.44µs        ? ?/sec    1.00    185.5±0.40µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/dictionary encoded, optional, no NULLs                            1.00    140.3±0.35µs        ? ?/sec    1.01    141.7±0.47µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/plain encoded, mandatory, no NULLs                                1.00     73.1±0.43µs        ? ?/sec    1.00     73.1±0.24µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/plain encoded, optional, half NULLs                               1.01    151.9±1.77µs        ? ?/sec    1.00    150.5±1.26µs        ? ?/sec
arrow_array_reader/INT32/Decimal128Array/plain encoded, optional, no NULLs                                 1.00     78.6±0.55µs        ? ?/sec    1.00     78.6±0.34µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/binary packed skip, mandatory, no NULLs                           1.02    110.9±0.18µs        ? ?/sec    1.00    109.2±0.32µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/binary packed skip, optional, half NULLs                          1.01    132.1±0.61µs        ? ?/sec    1.00    131.3±0.24µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/binary packed skip, optional, no NULLs                            1.00    112.1±1.62µs        ? ?/sec    1.00    112.2±0.25µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/binary packed, mandatory, no NULLs                                1.00    164.2±1.28µs        ? ?/sec    1.00    163.5±0.47µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/binary packed, optional, half NULLs                               1.02    235.6±2.57µs        ? ?/sec    1.00    231.7±0.59µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/binary packed, optional, no NULLs                                 1.01    169.9±0.89µs        ? ?/sec    1.00    168.6±0.54µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/byte_stream_split encoded, mandatory, no NULLs                    1.00    201.9±0.67µs        ? ?/sec    1.01    203.5±0.44µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/byte_stream_split encoded, optional, half NULLs                   1.00    251.1±0.54µs        ? ?/sec    1.00    250.8±1.04µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/byte_stream_split encoded, optional, no NULLs                     1.00    211.0±0.87µs        ? ?/sec    1.00    210.8±0.59µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/dictionary encoded, mandatory, no NULLs                           1.00    145.1±0.49µs        ? ?/sec    1.01    145.9±0.47µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/dictionary encoded, optional, half NULLs                          1.00    217.2±1.01µs        ? ?/sec    1.01    219.0±0.55µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/dictionary encoded, optional, no NULLs                            1.00    148.0±0.34µs        ? ?/sec    1.01    149.5±0.33µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/plain encoded, mandatory, no NULLs                                1.01    106.2±1.23µs        ? ?/sec    1.00    104.9±1.24µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/plain encoded, optional, half NULLs                               1.00    197.3±0.49µs        ? ?/sec    1.01    198.3±1.17µs        ? ?/sec
arrow_array_reader/INT64/Decimal128Array/plain encoded, optional, no NULLs                                 1.00    113.2±1.56µs        ? ?/sec    1.00    112.9±1.90µs        ? ?/sec
arrow_array_reader/Int16Array/binary packed skip, mandatory, no NULLs                                      1.03    102.0±0.34µs        ? ?/sec    1.00     99.1±0.23µs        ? ?/sec
arrow_array_reader/Int16Array/binary packed skip, optional, half NULLs                                     1.02    103.4±0.46µs        ? ?/sec    1.00    101.8±0.32µs        ? ?/sec
arrow_array_reader/Int16Array/binary packed skip, optional, no NULLs                                       1.03    105.0±0.73µs        ? ?/sec    1.00    101.8±0.33µs        ? ?/sec
arrow_array_reader/Int16Array/binary packed, mandatory, no NULLs                                           1.06    144.6±0.32µs        ? ?/sec    1.00    136.7±0.32µs        ? ?/sec
arrow_array_reader/Int16Array/binary packed, optional, half NULLs                                          1.02    169.4±0.56µs        ? ?/sec    1.00    166.5±0.37µs        ? ?/sec
arrow_array_reader/Int16Array/binary packed, optional, no NULLs                                            1.06    149.7±0.30µs        ? ?/sec    1.00    141.1±0.24µs        ? ?/sec
arrow_array_reader/Int16Array/byte_stream_split encoded, mandatory, no NULLs                               1.00     42.2±0.25µs        ? ?/sec    1.04     44.1±0.77µs        ? ?/sec
arrow_array_reader/Int16Array/byte_stream_split encoded, optional, half NULLs                              1.00    117.4±0.63µs        ? ?/sec    1.00    117.8±0.26µs        ? ?/sec
arrow_array_reader/Int16Array/byte_stream_split encoded, optional, no NULLs                                1.00     47.6±0.16µs        ? ?/sec    1.01     48.1±0.09µs        ? ?/sec
arrow_array_reader/Int16Array/dictionary encoded, mandatory, no NULLs                                      1.00    101.9±0.83µs        ? ?/sec    1.02    103.9±0.18µs        ? ?/sec
arrow_array_reader/Int16Array/dictionary encoded, optional, half NULLs                                     1.00    150.7±0.37µs        ? ?/sec    1.00    150.1±0.31µs        ? ?/sec
arrow_array_reader/Int16Array/dictionary encoded, optional, no NULLs                                       1.00    107.8±1.02µs        ? ?/sec    1.00    107.6±0.20µs        ? ?/sec
arrow_array_reader/Int16Array/plain encoded, mandatory, no NULLs                                           1.01     38.5±0.13µs        ? ?/sec    1.00     38.1±0.15µs        ? ?/sec
arrow_array_reader/Int16Array/plain encoded, optional, half NULLs                                          1.00    114.6±0.28µs        ? ?/sec    1.00    114.6±0.32µs        ? ?/sec
arrow_array_reader/Int16Array/plain encoded, optional, no NULLs                                            1.04     44.3±0.18µs        ? ?/sec    1.00     42.6±0.11µs        ? ?/sec
arrow_array_reader/Int32Array/binary packed skip, mandatory, no NULLs                                      1.00     98.6±0.62µs        ? ?/sec    1.01     99.4±0.44µs        ? ?/sec
arrow_array_reader/Int32Array/binary packed skip, optional, half NULLs                                     1.01     98.9±0.31µs        ? ?/sec    1.00     97.6±0.28µs        ? ?/sec
arrow_array_reader/Int32Array/binary packed skip, optional, no NULLs                                       1.01    102.7±0.48µs        ? ?/sec    1.00    102.1±0.16µs        ? ?/sec
arrow_array_reader/Int32Array/binary packed, mandatory, no NULLs                                           1.00    128.3±0.37µs        ? ?/sec    1.03    132.7±0.36µs        ? ?/sec
arrow_array_reader/Int32Array/binary packed, optional, half NULLs                                          1.00    149.1±0.32µs        ? ?/sec    1.04    154.8±0.38µs        ? ?/sec
arrow_array_reader/Int32Array/binary packed, optional, no NULLs                                            1.00    133.6±0.44µs        ? ?/sec    1.03    137.4±0.30µs        ? ?/sec
arrow_array_reader/Int32Array/byte_stream_split encoded, mandatory, no NULLs                               1.01     26.4±0.24µs        ? ?/sec    1.00     26.2±0.16µs        ? ?/sec
arrow_array_reader/Int32Array/byte_stream_split encoded, optional, half NULLs                              1.00     99.1±1.09µs        ? ?/sec    1.01    100.1±0.63µs        ? ?/sec
arrow_array_reader/Int32Array/byte_stream_split encoded, optional, no NULLs                                1.04     31.5±0.31µs        ? ?/sec    1.00     30.2±0.23µs        ? ?/sec
arrow_array_reader/Int32Array/dictionary encoded, mandatory, no NULLs                                      1.00     85.1±0.36µs        ? ?/sec    1.00     85.5±0.39µs        ? ?/sec
arrow_array_reader/Int32Array/dictionary encoded, optional, half NULLs                                     1.00    130.7±2.17µs        ? ?/sec    1.00    131.1±0.82µs        ? ?/sec
arrow_array_reader/Int32Array/dictionary encoded, optional, no NULLs                                       1.00     88.8±0.17µs        ? ?/sec    1.01     89.9±0.37µs        ? ?/sec
arrow_array_reader/Int32Array/plain encoded, mandatory, no NULLs                                           1.01     17.9±0.61µs        ? ?/sec    1.00     17.7±0.59µs        ? ?/sec
arrow_array_reader/Int32Array/plain encoded, optional, half NULLs                                          1.00     90.8±0.24µs        ? ?/sec    1.06     95.9±0.97µs        ? ?/sec
arrow_array_reader/Int32Array/plain encoded, optional, no NULLs                                            1.04     25.6±0.41µs        ? ?/sec    1.00     24.7±0.38µs        ? ?/sec
arrow_array_reader/Int64Array/binary packed skip, mandatory, no NULLs                                      1.00     82.6±0.19µs        ? ?/sec    1.00     82.4±0.20µs        ? ?/sec
arrow_array_reader/Int64Array/binary packed skip, optional, half NULLs                                     1.01    104.5±1.08µs        ? ?/sec    1.00    103.4±0.39µs        ? ?/sec
arrow_array_reader/Int64Array/binary packed skip, optional, no NULLs                                       1.01     85.5±0.32µs        ? ?/sec    1.00     85.0±0.35µs        ? ?/sec
arrow_array_reader/Int64Array/binary packed, mandatory, no NULLs                                           1.01    112.3±0.57µs        ? ?/sec    1.00    111.2±0.38µs        ? ?/sec
arrow_array_reader/Int64Array/binary packed, optional, half NULLs                                          1.01    172.5±0.63µs        ? ?/sec    1.00    171.0±0.53µs        ? ?/sec
arrow_array_reader/Int64Array/binary packed, optional, no NULLs                                            1.03    115.3±0.64µs        ? ?/sec    1.00    112.4±0.51µs        ? ?/sec
arrow_array_reader/Int64Array/byte_stream_split encoded, mandatory, no NULLs                               1.01    151.0±0.49µs        ? ?/sec    1.00    148.9±0.53µs        ? ?/sec
arrow_array_reader/Int64Array/byte_stream_split encoded, optional, half NULLs                              1.02    196.1±0.72µs        ? ?/sec    1.00    193.0±0.44µs        ? ?/sec
arrow_array_reader/Int64Array/byte_stream_split encoded, optional, no NULLs                                1.02    157.0±0.45µs        ? ?/sec    1.00    153.9±0.36µs        ? ?/sec
arrow_array_reader/Int64Array/dictionary encoded, mandatory, no NULLs                                      1.00     89.8±0.58µs        ? ?/sec    1.01     90.5±0.58µs        ? ?/sec
arrow_array_reader/Int64Array/dictionary encoded, optional, half NULLs                                     1.05    169.9±0.45µs        ? ?/sec    1.00    161.5±0.74µs        ? ?/sec
arrow_array_reader/Int64Array/dictionary encoded, optional, no NULLs                                       1.00     96.1±0.44µs        ? ?/sec    1.00     95.7±0.61µs        ? ?/sec
arrow_array_reader/Int64Array/plain encoded, mandatory, no NULLs                                           1.19     49.2±2.17µs        ? ?/sec    1.00     41.2±0.99µs        ? ?/sec
arrow_array_reader/Int64Array/plain encoded, optional, half NULLs                                          1.03    141.1±1.57µs        ? ?/sec    1.00    136.8±0.36µs        ? ?/sec
arrow_array_reader/Int64Array/plain encoded, optional, no NULLs                                            1.19     56.7±2.07µs        ? ?/sec    1.00     47.7±0.79µs        ? ?/sec
arrow_array_reader/Int8Array/binary packed skip, mandatory, no NULLs                                       1.00     95.8±0.26µs        ? ?/sec    1.02     97.4±0.12µs        ? ?/sec
arrow_array_reader/Int8Array/binary packed skip, optional, half NULLs                                      1.00     98.0±0.92µs        ? ?/sec    1.01     99.0±1.04µs        ? ?/sec
arrow_array_reader/Int8Array/binary packed skip, optional, no NULLs                                        1.00     99.0±0.23µs        ? ?/sec    1.01    100.2±0.17µs        ? ?/sec
arrow_array_reader/Int8Array/binary packed, mandatory, no NULLs                                            1.00    130.6±0.32µs        ? ?/sec    1.04    135.9±0.98µs        ? ?/sec
arrow_array_reader/Int8Array/binary packed, optional, half NULLs                                           1.00    157.9±0.34µs        ? ?/sec    1.02    160.4±0.38µs        ? ?/sec
arrow_array_reader/Int8Array/binary packed, optional, no NULLs                                             1.00    135.9±0.27µs        ? ?/sec    1.03    140.0±0.26µs        ? ?/sec
arrow_array_reader/Int8Array/byte_stream_split encoded, mandatory, no NULLs                                1.00     34.5±0.25µs        ? ?/sec    1.04     35.8±0.11µs        ? ?/sec
arrow_array_reader/Int8Array/byte_stream_split encoded, optional, half NULLs                               1.00    107.6±0.72µs        ? ?/sec    1.03    110.3±0.58µs        ? ?/sec
arrow_array_reader/Int8Array/byte_stream_split encoded, optional, no NULLs                                 1.02     41.2±0.12µs        ? ?/sec    1.00     40.3±0.10µs        ? ?/sec
arrow_array_reader/Int8Array/dictionary encoded, mandatory, no NULLs                                       1.00     94.3±0.25µs        ? ?/sec    1.00     94.6±0.36µs        ? ?/sec
arrow_array_reader/Int8Array/dictionary encoded, optional, half NULLs                                      1.00    142.2±0.53µs        ? ?/sec    1.00    142.6±0.29µs        ? ?/sec
arrow_array_reader/Int8Array/dictionary encoded, optional, no NULLs                                        1.00     99.6±0.86µs        ? ?/sec    1.00     99.4±0.14µs        ? ?/sec
arrow_array_reader/Int8Array/plain encoded, mandatory, no NULLs                                            1.00     30.5±0.11µs        ? ?/sec    1.00     30.5±0.28µs        ? ?/sec
arrow_array_reader/Int8Array/plain encoded, optional, half NULLs                                           1.00    106.1±0.41µs        ? ?/sec    1.00    105.8±0.29µs        ? ?/sec
arrow_array_reader/Int8Array/plain encoded, optional, no NULLs                                             1.04     36.1±0.11µs        ? ?/sec    1.00     34.9±0.13µs        ? ?/sec
arrow_array_reader/ListArray/plain encoded optional strings half NULLs                                     1.02      7.2±0.04ms        ? ?/sec    1.00      7.1±0.04ms        ? ?/sec
arrow_array_reader/ListArray/plain encoded optional strings no NULLs                                       1.03     13.2±0.12ms        ? ?/sec    1.00     12.9±0.12ms        ? ?/sec
arrow_array_reader/StringArray/dictionary encoded, mandatory, no NULLs                                     1.03    503.9±3.42µs        ? ?/sec    1.00    490.3±3.24µs        ? ?/sec
arrow_array_reader/StringArray/dictionary encoded, optional, half NULLs                                    1.02    667.7±2.87µs        ? ?/sec    1.00    652.9±1.97µs        ? ?/sec
arrow_array_reader/StringArray/dictionary encoded, optional, no NULLs                                      1.01    503.7±6.95µs        ? ?/sec    1.00    498.3±5.57µs        ? ?/sec
arrow_array_reader/StringArray/plain encoded, mandatory, no NULLs                                          1.07    710.1±2.31µs        ? ?/sec    1.00    662.8±3.33µs        ? ?/sec
arrow_array_reader/StringArray/plain encoded, optional, half NULLs                                         1.07    823.9±3.83µs        ? ?/sec    1.00    771.9±2.97µs        ? ?/sec
arrow_array_reader/StringArray/plain encoded, optional, no NULLs                                           1.07    719.7±2.20µs        ? ?/sec    1.00    671.1±3.28µs        ? ?/sec
arrow_array_reader/StringDictionary/dictionary encoded, mandatory, no NULLs                                1.01    295.3±1.19µs        ? ?/sec    1.00    292.9±0.70µs        ? ?/sec
arrow_array_reader/StringDictionary/dictionary encoded, optional, half NULLs                               1.08    388.3±1.27µs        ? ?/sec    1.00    358.2±1.32µs        ? ?/sec
arrow_array_reader/StringDictionary/dictionary encoded, optional, no NULLs                                 1.00    301.5±1.16µs        ? ?/sec    1.00    300.3±1.80µs        ? ?/sec
arrow_array_reader/StringViewArray/dictionary encoded, mandatory, no NULLs                                 1.00    252.2±3.48µs        ? ?/sec    1.04    262.1±2.69µs        ? ?/sec
arrow_array_reader/StringViewArray/dictionary encoded, optional, half NULLs                                1.00    265.0±0.76µs        ? ?/sec    1.00    264.5±0.78µs        ? ?/sec
arrow_array_reader/StringViewArray/dictionary encoded, optional, no NULLs                                  1.00    267.1±2.22µs        ? ?/sec    1.00    268.3±2.53µs        ? ?/sec
arrow_array_reader/StringViewArray/plain encoded, mandatory, no NULLs                                      1.00    459.9±4.18µs        ? ?/sec    1.08    497.7±1.44µs        ? ?/sec
arrow_array_reader/StringViewArray/plain encoded, optional, half NULLs                                     1.00    367.8±1.06µs        ? ?/sec    1.05    387.2±1.15µs        ? ?/sec
arrow_array_reader/StringViewArray/plain encoded, optional, no NULLs                                       1.00    467.7±1.76µs        ? ?/sec    1.08    506.4±2.38µs        ? ?/sec
arrow_array_reader/UInt16Array/binary packed skip, mandatory, no NULLs                                     1.00    106.4±0.30µs        ? ?/sec    1.00    106.6±0.17µs        ? ?/sec
arrow_array_reader/UInt16Array/binary packed skip, optional, half NULLs                                    1.00    106.4±0.27µs        ? ?/sec    1.00    106.7±0.54µs        ? ?/sec
arrow_array_reader/UInt16Array/binary packed skip, optional, no NULLs                                      1.00    109.6±0.27µs        ? ?/sec    1.00    109.5±0.19µs        ? ?/sec
arrow_array_reader/UInt16Array/binary packed, mandatory, no NULLs                                          1.00    149.1±0.40µs        ? ?/sec    1.01    150.1±0.49µs        ? ?/sec
arrow_array_reader/UInt16Array/binary packed, optional, half NULLs                                         1.00    174.5±0.78µs        ? ?/sec    1.00    173.7±0.62µs        ? ?/sec
arrow_array_reader/UInt16Array/binary packed, optional, no NULLs                                           1.00    154.9±0.38µs        ? ?/sec    1.00    155.0±0.56µs        ? ?/sec
arrow_array_reader/UInt16Array/byte_stream_split encoded, mandatory, no NULLs                              1.00     42.4±0.14µs        ? ?/sec    1.04     43.9±0.10µs        ? ?/sec
arrow_array_reader/UInt16Array/byte_stream_split encoded, optional, half NULLs                             1.00    117.4±0.75µs        ? ?/sec    1.00    117.6±0.28µs        ? ?/sec
arrow_array_reader/UInt16Array/byte_stream_split encoded, optional, no NULLs                               1.00     47.7±0.12µs        ? ?/sec    1.02     48.4±0.10µs        ? ?/sec
arrow_array_reader/UInt16Array/dictionary encoded, mandatory, no NULLs                                     1.00    101.5±0.36µs        ? ?/sec    1.01    102.6±0.36µs        ? ?/sec
arrow_array_reader/UInt16Array/dictionary encoded, optional, half NULLs                                    1.00    150.4±0.26µs        ? ?/sec    1.00    150.5±0.43µs        ? ?/sec
arrow_array_reader/UInt16Array/dictionary encoded, optional, no NULLs                                      1.00    107.3±0.21µs        ? ?/sec    1.00    107.5±0.28µs        ? ?/sec
arrow_array_reader/UInt16Array/plain encoded, mandatory, no NULLs                                          1.00     38.2±0.16µs        ? ?/sec    1.00     38.1±0.13µs        ? ?/sec
arrow_array_reader/UInt16Array/plain encoded, optional, half NULLs                                         1.00    114.9±0.54µs        ? ?/sec    1.00    114.7±1.13µs        ? ?/sec
arrow_array_reader/UInt16Array/plain encoded, optional, no NULLs                                           1.02     43.9±0.11µs        ? ?/sec    1.00     42.9±0.10µs        ? ?/sec
arrow_array_reader/UInt32Array/binary packed skip, mandatory, no NULLs                                     1.00     99.3±0.25µs        ? ?/sec    1.00     99.7±0.25µs        ? ?/sec
arrow_array_reader/UInt32Array/binary packed skip, optional, half NULLs                                    1.01     98.7±0.29µs        ? ?/sec    1.00     97.5±0.27µs        ? ?/sec
arrow_array_reader/UInt32Array/binary packed skip, optional, no NULLs                                      1.00    102.6±0.17µs        ? ?/sec    1.00    102.4±0.36µs        ? ?/sec
arrow_array_reader/UInt32Array/binary packed, mandatory, no NULLs                                          1.00    129.2±1.01µs        ? ?/sec    1.00    129.8±0.31µs        ? ?/sec
arrow_array_reader/UInt32Array/binary packed, optional, half NULLs                                         1.00    155.0±0.58µs        ? ?/sec    1.00    154.5±0.35µs        ? ?/sec
arrow_array_reader/UInt32Array/binary packed, optional, no NULLs                                           1.00    134.2±0.46µs        ? ?/sec    1.00    134.1±0.49µs        ? ?/sec
arrow_array_reader/UInt32Array/byte_stream_split encoded, mandatory, no NULLs                              1.01     26.5±0.35µs        ? ?/sec    1.00     26.2±0.29µs        ? ?/sec
arrow_array_reader/UInt32Array/byte_stream_split encoded, optional, half NULLs                             1.01    100.7±0.30µs        ? ?/sec    1.00    100.1±0.29µs        ? ?/sec
arrow_array_reader/UInt32Array/byte_stream_split encoded, optional, no NULLs                               1.05     31.8±0.31µs        ? ?/sec    1.00     30.2±0.31µs        ? ?/sec
arrow_array_reader/UInt32Array/dictionary encoded, mandatory, no NULLs                                     1.00     85.0±0.48µs        ? ?/sec    1.00     85.0±0.30µs        ? ?/sec
arrow_array_reader/UInt32Array/dictionary encoded, optional, half NULLs                                    1.01    133.1±0.77µs        ? ?/sec    1.00    131.4±0.49µs        ? ?/sec
arrow_array_reader/UInt32Array/dictionary encoded, optional, no NULLs                                      1.01     90.2±0.39µs        ? ?/sec    1.00     89.6±0.34µs        ? ?/sec
arrow_array_reader/UInt32Array/plain encoded, mandatory, no NULLs                                          1.07     22.1±0.91µs        ? ?/sec    1.00     20.6±0.82µs        ? ?/sec
arrow_array_reader/UInt32Array/plain encoded, optional, half NULLs                                         1.00     96.9±0.29µs        ? ?/sec    1.01     97.6±0.52µs        ? ?/sec
arrow_array_reader/UInt32Array/plain encoded, optional, no NULLs                                           1.10     27.7±1.22µs        ? ?/sec    1.00     25.2±0.70µs        ? ?/sec
arrow_array_reader/UInt64Array/binary packed skip, mandatory, no NULLs                                     1.00     82.7±0.57µs        ? ?/sec    1.00     82.5±0.47µs        ? ?/sec
arrow_array_reader/UInt64Array/binary packed skip, optional, half NULLs                                    1.01    104.8±0.36µs        ? ?/sec    1.00    104.1±0.96µs        ? ?/sec
arrow_array_reader/UInt64Array/binary packed skip, optional, no NULLs                                      1.00     85.6±0.28µs        ? ?/sec    1.01     86.6±0.31µs        ? ?/sec
arrow_array_reader/UInt64Array/binary packed, mandatory, no NULLs                                          1.01    111.6±0.54µs        ? ?/sec    1.00    110.9±0.68µs        ? ?/sec
arrow_array_reader/UInt64Array/binary packed, optional, half NULLs                                         1.01    173.9±2.67µs        ? ?/sec    1.00    171.8±0.72µs        ? ?/sec
arrow_array_reader/UInt64Array/binary packed, optional, no NULLs                                           1.02    115.1±0.40µs        ? ?/sec    1.00    113.0±1.13µs        ? ?/sec
arrow_array_reader/UInt64Array/byte_stream_split encoded, mandatory, no NULLs                              1.00    149.5±0.45µs        ? ?/sec    1.00    149.2±0.41µs        ? ?/sec
arrow_array_reader/UInt64Array/byte_stream_split encoded, optional, half NULLs                             1.00    195.2±0.50µs        ? ?/sec    1.00    194.3±0.60µs        ? ?/sec
arrow_array_reader/UInt64Array/byte_stream_split encoded, optional, no NULLs                               1.01    155.1±0.59µs        ? ?/sec    1.00    154.2±0.45µs        ? ?/sec
arrow_array_reader/UInt64Array/dictionary encoded, mandatory, no NULLs                                     1.00     90.6±0.60µs        ? ?/sec    1.01     91.2±0.63µs        ? ?/sec
arrow_array_reader/UInt64Array/dictionary encoded, optional, half NULLs                                    1.01    163.1±0.51µs        ? ?/sec    1.00    162.0±0.64µs        ? ?/sec
arrow_array_reader/UInt64Array/dictionary encoded, optional, no NULLs                                      1.01     95.8±0.74µs        ? ?/sec    1.00     95.2±0.40µs        ? ?/sec
arrow_array_reader/UInt64Array/plain encoded, mandatory, no NULLs                                          1.05     47.7±2.11µs        ? ?/sec    1.00     45.2±2.70µs        ? ?/sec
arrow_array_reader/UInt64Array/plain encoded, optional, half NULLs                                         1.00    137.5±0.71µs        ? ?/sec    1.01    138.9±0.76µs        ? ?/sec
arrow_array_reader/UInt64Array/plain encoded, optional, no NULLs                                           1.09     56.7±2.84µs        ? ?/sec    1.00     52.2±1.38µs        ? ?/sec
arrow_array_reader/UInt8Array/binary packed skip, mandatory, no NULLs                                      1.01    104.0±0.16µs        ? ?/sec    1.00    103.1±0.64µs        ? ?/sec
arrow_array_reader/UInt8Array/binary packed skip, optional, half NULLs                                     1.00    102.3±0.23µs        ? ?/sec    1.00    101.8±0.29µs        ? ?/sec
arrow_array_reader/UInt8Array/binary packed skip, optional, no NULLs                                       1.01    106.9±0.28µs        ? ?/sec    1.00    105.8±0.28µs        ? ?/sec
arrow_array_reader/UInt8Array/binary packed, mandatory, no NULLs                                           1.00    138.6±0.35µs        ? ?/sec    1.03    143.0±0.66µs        ? ?/sec
arrow_array_reader/UInt8Array/binary packed, optional, half NULLs                                          1.00    163.3±0.35µs        ? ?/sec    1.02    165.8±0.33µs        ? ?/sec
arrow_array_reader/UInt8Array/binary packed, optional, no NULLs                                            1.00    143.9±0.35µs        ? ?/sec    1.02    147.2±0.41µs        ? ?/sec
arrow_array_reader/UInt8Array/byte_stream_split encoded, mandatory, no NULLs                               1.00     36.2±0.10µs        ? ?/sec    1.01     36.4±0.13µs        ? ?/sec
arrow_array_reader/UInt8Array/byte_stream_split encoded, optional, half NULLs                              1.00    107.3±0.26µs        ? ?/sec    1.04    111.8±1.07µs        ? ?/sec
arrow_array_reader/UInt8Array/byte_stream_split encoded, optional, no NULLs                                1.01     41.1±0.14µs        ? ?/sec    1.00     40.6±0.09µs        ? ?/sec
arrow_array_reader/UInt8Array/dictionary encoded, mandatory, no NULLs                                      1.00     94.0±0.12µs        ? ?/sec    1.01     95.3±0.26µs        ? ?/sec
arrow_array_reader/UInt8Array/dictionary encoded, optional, half NULLs                                     1.00    142.0±0.65µs        ? ?/sec    1.01    142.9±0.39µs        ? ?/sec
arrow_array_reader/UInt8Array/dictionary encoded, optional, no NULLs                                       1.00     99.6±0.25µs        ? ?/sec    1.00     99.7±0.19µs        ? ?/sec
arrow_array_reader/UInt8Array/plain encoded, mandatory, no NULLs                                           1.00     30.6±0.35µs        ? ?/sec    1.01     30.8±0.12µs        ? ?/sec
arrow_array_reader/UInt8Array/plain encoded, optional, half NULLs                                          1.00    106.0±0.21µs        ? ?/sec    1.01    107.0±0.60µs        ? ?/sec
arrow_array_reader/UInt8Array/plain encoded, optional, no NULLs                                            1.01     35.7±0.31µs        ? ?/sec    1.00     35.4±0.11µs        ? ?/sec

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

🤖 ./gh_compare_arrow.sh Benchmark Script Running
Linux aal-dev 6.14.0-1017-gcp #18~24.04.1-Ubuntu SMP Tue Sep 23 17:51:44 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Comparing alamb/parquet_decoder (10aae0f) to 9d75f87 diff
BENCH_NAME=arrow_reader_clickbench
BENCH_COMMAND=cargo bench --features=arrow,async,test_common,experimental --bench arrow_reader_clickbench
BENCH_FILTER=
BENCH_BRANCH_NAME=alamb_parquet_decoder
Results will be posted here when complete

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

🤖: Benchmark completed

Details

group                                alamb_parquet_decoder                  main
-----                                ---------------------                  ----
arrow_reader_clickbench/async/Q1     1.01      2.4±0.02ms        ? ?/sec    1.00      2.3±0.01ms        ? ?/sec
arrow_reader_clickbench/async/Q10    1.02     12.8±0.24ms        ? ?/sec    1.00     12.6±0.29ms        ? ?/sec
arrow_reader_clickbench/async/Q11    1.02     14.6±0.28ms        ? ?/sec    1.00     14.4±0.25ms        ? ?/sec
arrow_reader_clickbench/async/Q12    1.00     27.1±0.33ms        ? ?/sec    1.02     27.5±0.28ms        ? ?/sec
arrow_reader_clickbench/async/Q13    1.00     38.4±0.26ms        ? ?/sec    1.02     39.0±0.30ms        ? ?/sec
arrow_reader_clickbench/async/Q14    1.00     35.9±0.25ms        ? ?/sec    1.02     36.6±0.29ms        ? ?/sec
arrow_reader_clickbench/async/Q19    1.00      5.6±0.08ms        ? ?/sec    1.00      5.6±0.07ms        ? ?/sec
arrow_reader_clickbench/async/Q20    1.00    119.6±0.56ms        ? ?/sec    1.10    132.1±7.06ms        ? ?/sec
arrow_reader_clickbench/async/Q21    1.00    153.8±4.87ms        ? ?/sec    1.06    163.5±6.88ms        ? ?/sec
arrow_reader_clickbench/async/Q22    1.00    275.6±5.77ms        ? ?/sec    1.06   293.2±16.30ms        ? ?/sec
arrow_reader_clickbench/async/Q23    1.00    421.5±2.41ms        ? ?/sec    1.02    429.5±2.20ms        ? ?/sec
arrow_reader_clickbench/async/Q24    1.00     42.5±0.32ms        ? ?/sec    1.02     43.4±0.33ms        ? ?/sec
arrow_reader_clickbench/async/Q27    1.00    103.5±0.56ms        ? ?/sec    1.02    105.5±1.46ms        ? ?/sec
arrow_reader_clickbench/async/Q28    1.00    103.3±0.73ms        ? ?/sec    1.02    105.2±0.88ms        ? ?/sec
arrow_reader_clickbench/async/Q30    1.00     51.6±0.34ms        ? ?/sec    1.03     52.9±0.38ms        ? ?/sec
arrow_reader_clickbench/async/Q36    1.00    124.4±0.58ms        ? ?/sec    1.01    126.2±0.55ms        ? ?/sec
arrow_reader_clickbench/async/Q37    1.00     99.2±0.67ms        ? ?/sec    1.01    100.2±0.40ms        ? ?/sec
arrow_reader_clickbench/async/Q38    1.00     36.8±0.23ms        ? ?/sec    1.01     37.2±0.29ms        ? ?/sec
arrow_reader_clickbench/async/Q39    1.00     48.1±0.26ms        ? ?/sec    1.01     48.7±0.27ms        ? ?/sec
arrow_reader_clickbench/async/Q40    1.00     45.0±0.34ms        ? ?/sec    1.02     45.8±0.42ms        ? ?/sec
arrow_reader_clickbench/async/Q41    1.00     35.2±0.49ms        ? ?/sec    1.01     35.7±0.24ms        ? ?/sec
arrow_reader_clickbench/async/Q42    1.00     13.4±0.09ms        ? ?/sec    1.00     13.4±0.09ms        ? ?/sec
arrow_reader_clickbench/sync/Q1      1.00      2.1±0.01ms        ? ?/sec    1.00      2.1±0.01ms        ? ?/sec
arrow_reader_clickbench/sync/Q10     1.03      9.3±0.07ms        ? ?/sec    1.00      9.0±0.06ms        ? ?/sec
arrow_reader_clickbench/sync/Q11     1.02     10.8±0.09ms        ? ?/sec    1.00     10.6±0.07ms        ? ?/sec
arrow_reader_clickbench/sync/Q12     1.00     37.3±0.30ms        ? ?/sec    1.01     37.8±0.33ms        ? ?/sec
arrow_reader_clickbench/sync/Q13     1.00     48.4±0.41ms        ? ?/sec    1.01     48.8±0.36ms        ? ?/sec
arrow_reader_clickbench/sync/Q14     1.00     46.3±0.44ms        ? ?/sec    1.01     47.0±0.29ms        ? ?/sec
arrow_reader_clickbench/sync/Q19     1.00      4.3±0.03ms        ? ?/sec    1.01      4.3±0.01ms        ? ?/sec
arrow_reader_clickbench/sync/Q20     1.00    176.5±0.81ms        ? ?/sec    1.02    179.3±0.84ms        ? ?/sec
arrow_reader_clickbench/sync/Q21     1.00    238.8±1.94ms        ? ?/sec    1.01    242.2±1.78ms        ? ?/sec
arrow_reader_clickbench/sync/Q22     1.00    486.8±4.76ms        ? ?/sec    1.00    487.4±3.92ms        ? ?/sec
arrow_reader_clickbench/sync/Q23     1.00   437.3±15.51ms        ? ?/sec    1.01   440.3±11.99ms        ? ?/sec
arrow_reader_clickbench/sync/Q24     1.00     49.2±0.87ms        ? ?/sec    1.01     49.8±0.66ms        ? ?/sec
arrow_reader_clickbench/sync/Q27     1.00    152.0±0.71ms        ? ?/sec    1.03    156.5±1.38ms        ? ?/sec
arrow_reader_clickbench/sync/Q28     1.00    147.9±0.75ms        ? ?/sec    1.02    150.7±0.77ms        ? ?/sec
arrow_reader_clickbench/sync/Q30     1.00     49.3±0.30ms        ? ?/sec    1.02     50.3±0.34ms        ? ?/sec
arrow_reader_clickbench/sync/Q36     1.00    152.9±0.77ms        ? ?/sec    1.02    155.9±0.97ms        ? ?/sec
arrow_reader_clickbench/sync/Q37     1.00     89.4±0.49ms        ? ?/sec    1.00     89.7±0.51ms        ? ?/sec
arrow_reader_clickbench/sync/Q38     1.00     29.2±0.21ms        ? ?/sec    1.01     29.5±0.22ms        ? ?/sec
arrow_reader_clickbench/sync/Q39     1.00     33.7±0.31ms        ? ?/sec    1.02     34.4±0.27ms        ? ?/sec
arrow_reader_clickbench/sync/Q40     1.00     42.3±0.25ms        ? ?/sec    1.01     42.8±0.24ms        ? ?/sec
arrow_reader_clickbench/sync/Q41     1.00     32.0±0.24ms        ? ?/sec    1.01     32.2±0.19ms        ? ?/sec
arrow_reader_clickbench/sync/Q42     1.00     12.5±0.07ms        ? ?/sec    1.00     12.5±0.07ms        ? ?/sec

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

🤖 ./gh_compare_arrow.sh Benchmark Script Running
Linux aal-dev 6.14.0-1017-gcp #18~24.04.1-Ubuntu SMP Tue Sep 23 17:51:44 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Comparing alamb/parquet_decoder (10aae0f) to 9d75f87 diff
BENCH_NAME=arrow_reader_row_filter
BENCH_COMMAND=cargo bench --features=arrow,async,test_common,experimental --bench arrow_reader_row_filter
BENCH_FILTER=
BENCH_BRANCH_NAME=alamb_parquet_decoder
Results will be posted here when complete

@alamb
Copy link
Contributor Author

alamb commented Oct 20, 2025

🤖: Benchmark completed

Details

group                                                                                alamb_parquet_decoder                  main
-----                                                                                ---------------------                  ----
arrow_reader_row_filter/float64 <= 99.0/all_columns/async                            1.00   1691.0±7.31µs        ? ?/sec    1.00  1698.1±10.16µs        ? ?/sec
arrow_reader_row_filter/float64 <= 99.0/all_columns/sync                             1.00  1957.1±11.71µs        ? ?/sec    1.01  1974.4±12.18µs        ? ?/sec
arrow_reader_row_filter/float64 <= 99.0/exclude_filter_column/async                  1.00  1541.7±10.56µs        ? ?/sec    1.00   1548.1±8.86µs        ? ?/sec
arrow_reader_row_filter/float64 <= 99.0/exclude_filter_column/sync                   1.00  1617.8±12.57µs        ? ?/sec    1.02  1645.1±25.56µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0 AND ts >= 9000/all_columns/async              1.02  1543.1±13.77µs        ? ?/sec    1.00  1513.8±12.16µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0 AND ts >= 9000/all_columns/sync               1.00  1864.3±11.99µs        ? ?/sec    1.00  1860.4±13.50µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0 AND ts >= 9000/exclude_filter_column/async    1.00  1357.6±15.52µs        ? ?/sec    1.00   1355.8±7.94µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0 AND ts >= 9000/exclude_filter_column/sync     1.01   1479.3±7.08µs        ? ?/sec    1.00   1462.0±9.03µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0/all_columns/async                             1.00   1682.9±6.95µs        ? ?/sec    1.01   1707.8±9.98µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0/all_columns/sync                              1.00  1959.2±16.61µs        ? ?/sec    1.02  1990.6±23.77µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0/exclude_filter_column/async                   1.00   1532.7±7.95µs        ? ?/sec    1.02  1561.2±14.39µs        ? ?/sec
arrow_reader_row_filter/float64 > 99.0/exclude_filter_column/sync                    1.00  1612.2±13.48µs        ? ?/sec    1.02  1641.2±13.21µs        ? ?/sec
arrow_reader_row_filter/int64 == 9999/all_columns/async                              1.01    941.9±6.11µs        ? ?/sec    1.00    936.4±4.32µs        ? ?/sec
arrow_reader_row_filter/int64 == 9999/all_columns/sync                               1.00    991.4±3.86µs        ? ?/sec    1.00    990.8±6.26µs        ? ?/sec
arrow_reader_row_filter/int64 == 9999/exclude_filter_column/async                    1.00    845.4±5.36µs        ? ?/sec    1.02    859.7±4.67µs        ? ?/sec
arrow_reader_row_filter/int64 == 9999/exclude_filter_column/sync                     1.00    974.4±7.15µs        ? ?/sec    1.02   989.5±13.84µs        ? ?/sec
arrow_reader_row_filter/int64 > 90/all_columns/async                                 1.00      4.0±0.02ms        ? ?/sec    1.01      4.1±0.02ms        ? ?/sec
arrow_reader_row_filter/int64 > 90/all_columns/sync                                  1.00      4.0±0.01ms        ? ?/sec    1.01      4.0±0.02ms        ? ?/sec
arrow_reader_row_filter/int64 > 90/exclude_filter_column/async                       1.00      3.5±0.01ms        ? ?/sec    1.02      3.6±0.01ms        ? ?/sec
arrow_reader_row_filter/int64 > 90/exclude_filter_column/sync                        1.00      3.4±0.02ms        ? ?/sec    1.01      3.4±0.01ms        ? ?/sec
arrow_reader_row_filter/ts < 9000/all_columns/async                                  1.02      2.0±0.01ms        ? ?/sec    1.00  1974.1±13.08µs        ? ?/sec
arrow_reader_row_filter/ts < 9000/all_columns/sync                                   1.01      2.2±0.02ms        ? ?/sec    1.00      2.2±0.01ms        ? ?/sec
arrow_reader_row_filter/ts < 9000/exclude_filter_column/async                        1.03  1863.4±17.62µs        ? ?/sec    1.00   1808.6±9.30µs        ? ?/sec
arrow_reader_row_filter/ts < 9000/exclude_filter_column/sync                         1.01   1950.3±9.25µs        ? ?/sec    1.00  1924.8±13.81µs        ? ?/sec
arrow_reader_row_filter/ts >= 9000/all_columns/async                                 1.01  1267.6±10.57µs        ? ?/sec    1.00  1256.1±14.84µs        ? ?/sec
arrow_reader_row_filter/ts >= 9000/all_columns/sync                                  1.01  1435.8±14.48µs        ? ?/sec    1.00  1415.4±18.71µs        ? ?/sec
arrow_reader_row_filter/ts >= 9000/exclude_filter_column/async                       1.01   1167.5±3.81µs        ? ?/sec    1.00   1153.3±7.17µs        ? ?/sec
arrow_reader_row_filter/ts >= 9000/exclude_filter_column/sync                        1.02  1300.5±10.24µs        ? ?/sec    1.00   1280.9±9.56µs        ? ?/sec
arrow_reader_row_filter/utf8View <> ''/all_columns/async                             1.00      4.3±0.03ms        ? ?/sec    1.00      4.3±0.02ms        ? ?/sec
arrow_reader_row_filter/utf8View <> ''/all_columns/sync                              1.00      4.9±0.02ms        ? ?/sec    1.00      4.9±0.01ms        ? ?/sec
arrow_reader_row_filter/utf8View <> ''/exclude_filter_column/async                   1.00      3.6±0.01ms        ? ?/sec    1.01      3.6±0.01ms        ? ?/sec
arrow_reader_row_filter/utf8View <> ''/exclude_filter_column/sync                    1.01      3.5±0.01ms        ? ?/sec    1.00      3.5±0.01ms        ? ?/sec

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

parquet Changes to the parquet crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Decouple IO and CPU operations in the Parquet Reader (push decoder)

3 participants