-
Notifications
You must be signed in to change notification settings - Fork 1k
[Parquet Predicate Cache]: Add ArrowReaderMetrics and tests for caching #8003
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,137 @@ | ||
| // Licensed to the Apache Software Foundation (ASF) under one | ||
| // or more contributor license agreements. See the NOTICE file | ||
| // distributed with this work for additional information | ||
| // regarding copyright ownership. The ASF licenses this file | ||
| // to you under the Apache License, Version 2.0 (the | ||
| // "License"); you may not use this file except in compliance | ||
| // with the License. You may obtain a copy of the License at | ||
| // | ||
| // http://www.apache.org/licenses/LICENSE-2.0 | ||
| // | ||
| // Unless required by applicable law or agreed to in writing, | ||
| // software distributed under the License is distributed on an | ||
| // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| // KIND, either express or implied. See the License for the | ||
| // specific language governing permissions and limitations | ||
| // under the License. | ||
|
|
||
| //! [ArrowReaderMetrics] for collecting metrics about the Arrow reader | ||
| use std::sync::atomic::AtomicUsize; | ||
| use std::sync::Arc; | ||
|
|
||
| /// This enum represents the state of Arrow reader metrics collection. | ||
| /// | ||
| /// The inner metrics are stored in an `Arc<ArrowReaderMetricsInner>` | ||
| /// so cloning the `ArrowReaderMetrics` enum will not clone the inner metrics. | ||
| /// | ||
| /// To access metrics, create an `ArrowReaderMetrics` via [`ArrowReaderMetrics::enabled()`] | ||
| /// and configure the `ArrowReaderBuilder` with a clone. | ||
| #[derive(Debug, Clone)] | ||
| pub enum ArrowReaderMetrics { | ||
| /// Metrics are not collected (default) | ||
| Disabled, | ||
| /// Metrics are collected and stored in an `Arc`. | ||
| /// | ||
| /// Create this via [`ArrowReaderMetrics::enabled()`]. | ||
| Enabled(Arc<ArrowReaderMetricsInner>), | ||
| } | ||
|
|
||
| impl ArrowReaderMetrics { | ||
| /// Creates a new instance of [`ArrowReaderMetrics::Disabled`] | ||
| pub fn disabled() -> Self { | ||
| Self::Disabled | ||
| } | ||
|
|
||
| /// Creates a new instance of [`ArrowReaderMetrics::Enabled`] | ||
| pub fn enabled() -> Self { | ||
| Self::Enabled(Arc::new(ArrowReaderMetricsInner::new())) | ||
| } | ||
|
|
||
| /// Predicate Cache: number of records read directly from the inner reader | ||
| /// | ||
| /// This is the total number of records read from the inner reader (that is | ||
| /// actually decoding). It measures the amount of work that could not be | ||
| /// avoided with caching. | ||
| /// | ||
| /// It returns the number of records read across all columns, so if you read | ||
| /// 2 columns each with 100 records, this will return 200. | ||
| /// | ||
| /// | ||
| /// Returns None if metrics are disabled. | ||
| pub fn records_read_from_inner(&self) -> Option<usize> { | ||
| match self { | ||
| Self::Disabled => None, | ||
| Self::Enabled(inner) => Some( | ||
| inner | ||
| .records_read_from_inner | ||
| .load(std::sync::atomic::Ordering::Relaxed), | ||
| ), | ||
| } | ||
| } | ||
|
|
||
| /// Predicate Cache: number of records read from the cache | ||
| /// | ||
| /// This is the total number of records read from the cache actually | ||
| /// decoding). It measures the amount of work that was avoided with caching. | ||
| /// | ||
| /// It returns the number of records read across all columns, so if you read | ||
| /// 2 columns each with 100 records from the cache, this will return 200. | ||
| /// | ||
| /// Returns None if metrics are disabled. | ||
| pub fn records_read_from_cache(&self) -> Option<usize> { | ||
| match self { | ||
| Self::Disabled => None, | ||
| Self::Enabled(inner) => Some( | ||
| inner | ||
| .records_read_from_cache | ||
| .load(std::sync::atomic::Ordering::Relaxed), | ||
| ), | ||
| } | ||
| } | ||
|
|
||
| /// Increments the count of records read from the inner reader | ||
| #[expect(unused)] // until https://github.com/apache/arrow-rs/pull/7850 | ||
| pub(crate) fn increment_inner_reads(&self, count: usize) { | ||
| let Self::Enabled(inner) = self else { | ||
| return; | ||
| }; | ||
| inner | ||
| .records_read_from_inner | ||
| .fetch_add(count, std::sync::atomic::Ordering::Relaxed); | ||
| } | ||
|
|
||
| /// Increments the count of records read from the cache | ||
| #[expect(unused)] // until https://github.com/apache/arrow-rs/pull/7850 | ||
| pub(crate) fn increment_cache_reads(&self, count: usize) { | ||
| let Self::Enabled(inner) = self else { | ||
| return; | ||
| }; | ||
|
|
||
| inner | ||
| .records_read_from_cache | ||
| .fetch_add(count, std::sync::atomic::Ordering::Relaxed); | ||
| } | ||
| } | ||
|
|
||
| /// Holds the actual metrics for the Arrow reader. | ||
| /// | ||
| /// Please see [`ArrowReaderMetrics`] for the public interface. | ||
| #[derive(Debug)] | ||
| pub struct ArrowReaderMetricsInner { | ||
| // Metrics for Predicate Cache | ||
| /// Total number of records read from the inner reader (uncached) | ||
| records_read_from_inner: AtomicUsize, | ||
| /// Total number of records read from previously cached pages | ||
| records_read_from_cache: AtomicUsize, | ||
| } | ||
|
|
||
| impl ArrowReaderMetricsInner { | ||
| /// Creates a new instance of `ArrowReaderMetricsInner` | ||
| pub(crate) fn new() -> Self { | ||
| Self { | ||
| records_read_from_inner: AtomicUsize::new(0), | ||
| records_read_from_cache: AtomicUsize::new(0), | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,9 +38,11 @@ use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; | |
| use crate::file::reader::{ChunkReader, SerializedPageReader}; | ||
| use crate::schema::types::SchemaDescriptor; | ||
|
|
||
| use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics; | ||
| pub(crate) use read_plan::{ReadPlan, ReadPlanBuilder}; | ||
|
|
||
| mod filter; | ||
| pub mod metrics; | ||
| mod read_plan; | ||
| mod selection; | ||
| pub mod statistics; | ||
|
|
@@ -112,6 +114,8 @@ pub struct ArrowReaderBuilder<T> { | |
| pub(crate) limit: Option<usize>, | ||
|
|
||
| pub(crate) offset: Option<usize>, | ||
|
|
||
| pub(crate) metrics: ArrowReaderMetrics, | ||
| } | ||
|
|
||
| impl<T: Debug> Debug for ArrowReaderBuilder<T> { | ||
|
|
@@ -128,6 +132,7 @@ impl<T: Debug> Debug for ArrowReaderBuilder<T> { | |
| .field("selection", &self.selection) | ||
| .field("limit", &self.limit) | ||
| .field("offset", &self.offset) | ||
| .field("metrics", &self.metrics) | ||
| .finish() | ||
| } | ||
| } | ||
|
|
@@ -146,6 +151,7 @@ impl<T> ArrowReaderBuilder<T> { | |
| selection: None, | ||
| limit: None, | ||
| offset: None, | ||
| metrics: ArrowReaderMetrics::Disabled, | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -296,6 +302,44 @@ impl<T> ArrowReaderBuilder<T> { | |
| ..self | ||
| } | ||
| } | ||
|
|
||
| /// Specify metrics collection during reading | ||
| /// | ||
| /// To access the metrics, create an [`ArrowReaderMetrics`] and pass a | ||
| /// clone of the provided metrics to the builder. | ||
| /// | ||
| /// For example: | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this shows how the metrics API is used |
||
| /// | ||
| /// ```rust | ||
| /// # use std::sync::Arc; | ||
| /// # use bytes::Bytes; | ||
| /// # use arrow_array::{Int32Array, RecordBatch}; | ||
| /// # use arrow_schema::{DataType, Field, Schema}; | ||
| /// # use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder}; | ||
| /// use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; | ||
| /// # use parquet::arrow::ArrowWriter; | ||
| /// # let mut file: Vec<u8> = Vec::with_capacity(1024); | ||
| /// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)])); | ||
| /// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap(); | ||
| /// # let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]).unwrap(); | ||
| /// # writer.write(&batch).unwrap(); | ||
| /// # writer.close().unwrap(); | ||
| /// # let file = Bytes::from(file); | ||
| /// // Create metrics object to pass into the reader | ||
| /// let metrics = ArrowReaderMetrics::enabled(); | ||
| /// let reader = ParquetRecordBatchReaderBuilder::try_new(file).unwrap() | ||
| /// // Configure the builder to use the metrics by passing a clone | ||
| /// .with_metrics(metrics.clone()) | ||
| /// // Build the reader | ||
| /// .build().unwrap(); | ||
| /// // .. read data from the reader .. | ||
| /// | ||
| /// // check the metrics | ||
| /// assert!(metrics.records_read_from_inner().is_some()); | ||
| /// ``` | ||
| pub fn with_metrics(self, metrics: ArrowReaderMetrics) -> Self { | ||
| Self { metrics, ..self } | ||
| } | ||
| } | ||
|
|
||
| /// Options that control how metadata is read for a parquet file | ||
|
|
@@ -707,23 +751,35 @@ impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> { | |
| /// | ||
| /// Note: this will eagerly evaluate any `RowFilter` before returning | ||
| pub fn build(self) -> Result<ParquetRecordBatchReader> { | ||
| let Self { | ||
| input, | ||
| metadata, | ||
| schema: _, | ||
| fields, | ||
| batch_size: _, | ||
| row_groups, | ||
| projection, | ||
| mut filter, | ||
| selection, | ||
| limit, | ||
| offset, | ||
| metrics: _, // used in https://github.com/apache/arrow-rs/pull/7850 | ||
| } = self; | ||
|
|
||
| // Try to avoid allocate large buffer | ||
| let batch_size = self | ||
| .batch_size | ||
| .min(self.metadata.file_metadata().num_rows() as usize); | ||
| .min(metadata.file_metadata().num_rows() as usize); | ||
|
|
||
| let row_groups = self | ||
| .row_groups | ||
| .unwrap_or_else(|| (0..self.metadata.num_row_groups()).collect()); | ||
| let row_groups = row_groups.unwrap_or_else(|| (0..metadata.num_row_groups()).collect()); | ||
|
|
||
| let reader = ReaderRowGroups { | ||
| reader: Arc::new(self.input.0), | ||
| metadata: self.metadata, | ||
| reader: Arc::new(input.0), | ||
| metadata, | ||
| row_groups, | ||
| }; | ||
|
|
||
| let mut filter = self.filter; | ||
| let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(self.selection); | ||
| let mut plan_builder = ReadPlanBuilder::new(batch_size).with_selection(selection); | ||
|
|
||
| // Update selection based on any filters | ||
| if let Some(filter) = filter.as_mut() { | ||
|
|
@@ -733,20 +789,23 @@ impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> { | |
| break; | ||
| } | ||
|
|
||
| let mut cache_projection = predicate.projection().clone(); | ||
| cache_projection.intersect(&projection); | ||
|
|
||
| let array_reader = ArrayReaderBuilder::new(&reader) | ||
| .build_array_reader(self.fields.as_deref(), predicate.projection())?; | ||
| .build_array_reader(fields.as_deref(), predicate.projection())?; | ||
|
|
||
| plan_builder = plan_builder.with_predicate(array_reader, predicate.as_mut())?; | ||
| } | ||
| } | ||
|
|
||
| let array_reader = ArrayReaderBuilder::new(&reader) | ||
| .build_array_reader(self.fields.as_deref(), &self.projection)?; | ||
| let array_reader = | ||
| ArrayReaderBuilder::new(&reader).build_array_reader(fields.as_deref(), &projection)?; | ||
|
|
||
| let read_plan = plan_builder | ||
| .limited(reader.num_rows()) | ||
| .with_offset(self.offset) | ||
| .with_limit(self.limit) | ||
| .with_offset(offset) | ||
| .with_limit(limit) | ||
| .build_limited() | ||
| .build(); | ||
|
|
||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a new API I am proposing to help write end to end tests and report out on the status of the parquet reader. The first actual usecase is reporting how many rows are read from the cache vs not the cache