Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
137 changes: 137 additions & 0 deletions parquet/src/arrow/arrow_reader/metrics.rs
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.
Copy link
Contributor Author

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

///
/// 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),
}
}
}
85 changes: 72 additions & 13 deletions parquet/src/arrow/arrow_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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> {
Expand All @@ -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()
}
}
Expand All @@ -146,6 +151,7 @@ impl<T> ArrowReaderBuilder<T> {
selection: None,
limit: None,
offset: None,
metrics: ArrowReaderMetrics::Disabled,
}
}

Expand Down Expand Up @@ -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:
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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
Expand Down Expand Up @@ -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() {
Expand All @@ -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();

Expand Down
8 changes: 8 additions & 0 deletions parquet/src/arrow/async_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ pub use metadata::*;
#[cfg(feature = "object_store")]
mod store;

use crate::arrow::arrow_reader::metrics::ArrowReaderMetrics;
use crate::arrow::arrow_reader::ReadPlanBuilder;
use crate::arrow::schema::ParquetField;
#[cfg(feature = "object_store")]
Expand Down Expand Up @@ -510,6 +511,7 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> {
fields: self.fields,
limit: self.limit,
offset: self.offset,
metrics: self.metrics,
};

// Ensure schema of ParquetRecordBatchStream respects projection, and does
Expand Down Expand Up @@ -560,6 +562,10 @@ struct ReaderFactory<T> {

/// Offset to apply to the next
offset: Option<usize>,

/// Metrics
#[expect(unused)] // until https://github.com/apache/arrow-rs/pull/7850
metrics: ArrowReaderMetrics,
}

impl<T> ReaderFactory<T>
Expand Down Expand Up @@ -1832,6 +1838,7 @@ mod tests {
assert_eq!(total_rows, 730);
}

#[ignore]
#[tokio::test]
async fn test_in_memory_row_group_sparse() {
let testdata = arrow::util::test_util::parquet_test_data();
Expand Down Expand Up @@ -1883,6 +1890,7 @@ mod tests {
filter: None,
limit: None,
offset: None,
metrics: ArrowReaderMetrics::disabled(),
};

let mut skip = true;
Expand Down
7 changes: 7 additions & 0 deletions parquet/src/arrow/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,13 @@ impl ProjectionMask {
Self { mask: None }
}

/// Create a [`ProjectionMask`] which selects no columns
pub fn none(len: usize) -> Self {
Self {
mask: Some(vec![false; len]),
}
}

/// Create a [`ProjectionMask`] which selects only the specified leaf columns
///
/// Note: repeated or out of order indices will not impact the final mask
Expand Down
2 changes: 2 additions & 0 deletions parquet/tests/arrow_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@ use tempfile::NamedTempFile;
mod bad_data;
#[cfg(feature = "crc")]
mod checksum;
#[cfg(feature = "async")]
mod predicate_cache;
mod statistics;

// returns a struct array with columns "int32_col", "float32_col" and "float64_col" with the specified values
Expand Down
Loading
Loading