-
Notifications
You must be signed in to change notification settings - Fork 1.6k
Encapsulate early File pruning in parquet opener in its own stream #17293
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
Merged
Merged
Changes from all commits
Commits
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 |
---|---|---|
|
@@ -17,17 +17,19 @@ | |
|
||
//! [`ParquetOpener`] for opening Parquet files | ||
|
||
use std::sync::Arc; | ||
|
||
use crate::page_filter::PagePruningAccessPlanFilter; | ||
use crate::row_group_filter::RowGroupAccessPlanFilter; | ||
use crate::{ | ||
apply_file_schema_type_coercions, coerce_int96_to_resolution, row_filter, | ||
ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory, | ||
}; | ||
use arrow::array::RecordBatch; | ||
use datafusion_datasource::file_meta::FileMeta; | ||
use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; | ||
use datafusion_datasource::schema_adapter::SchemaAdapterFactory; | ||
use std::pin::Pin; | ||
use std::sync::Arc; | ||
use std::task::{Context, Poll}; | ||
|
||
use arrow::datatypes::{FieldRef, SchemaRef, TimeUnit}; | ||
use arrow::error::ArrowError; | ||
|
@@ -47,7 +49,7 @@ use datafusion_pruning::{build_pruning_predicate, FilePruner, PruningPredicate}; | |
use datafusion_common::config::EncryptionFactoryOptions; | ||
#[cfg(feature = "parquet_encryption")] | ||
use datafusion_execution::parquet_encryption::EncryptionFactory; | ||
use futures::{StreamExt, TryStreamExt}; | ||
use futures::{ready, Stream, StreamExt, TryStreamExt}; | ||
use itertools::Itertools; | ||
use log::debug; | ||
use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; | ||
|
@@ -409,64 +411,103 @@ impl FileOpener for ParquetOpener { | |
.with_row_groups(row_group_indexes) | ||
.build()?; | ||
|
||
// Create a stateful stream that can check pruning after each batch | ||
let adapted = { | ||
use futures::stream; | ||
let schema_mapping = Some(schema_mapping); | ||
let file_pruner = file_pruner; | ||
let stream = stream.map_err(|e| ArrowError::ExternalError(Box::new(e))); | ||
let files_ranges_pruned_statistics = | ||
file_metrics.files_ranges_pruned_statistics.clone(); | ||
|
||
stream::try_unfold( | ||
( | ||
stream, | ||
schema_mapping, | ||
file_pruner, | ||
files_ranges_pruned_statistics, | ||
), | ||
move |( | ||
mut stream, | ||
schema_mapping_opt, | ||
mut file_pruner, | ||
files_ranges_pruned_statistics, | ||
)| async move { | ||
match stream.try_next().await? { | ||
Some(batch) => { | ||
let schema_mapping = schema_mapping_opt.as_ref().unwrap(); | ||
let mapped_batch = schema_mapping.map_batch(batch)?; | ||
|
||
// Check if we can prune the file now | ||
if let Some(ref mut pruner) = file_pruner { | ||
if pruner.should_prune()? { | ||
// File can now be pruned based on updated dynamic filters | ||
files_ranges_pruned_statistics.add(1); | ||
// Terminate the stream early | ||
return Ok(None); | ||
} | ||
} | ||
|
||
Ok(Some(( | ||
mapped_batch, | ||
( | ||
stream, | ||
schema_mapping_opt, | ||
file_pruner, | ||
files_ranges_pruned_statistics, | ||
), | ||
))) | ||
} | ||
None => Ok(None), | ||
} | ||
}, | ||
) | ||
}; | ||
let stream = stream | ||
.map_err(|e| ArrowError::ExternalError(Box::new(e))) | ||
.map(move |b| b.and_then(|b| Ok(schema_mapping.map_batch(b)?))); | ||
|
||
Ok(adapted.boxed()) | ||
if let Some(file_pruner) = file_pruner { | ||
Ok(EarlyStoppingStream::new( | ||
stream, | ||
file_pruner, | ||
file_metrics.files_ranges_pruned_statistics.clone(), | ||
) | ||
.boxed()) | ||
} else { | ||
Ok(stream.boxed()) | ||
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. I think this is also a bit easier to understand that the pruning only happens when we have an file pruner |
||
} | ||
})) | ||
} | ||
} | ||
|
||
/// Wraps an inner RecordBatchStream and a [`FilePruner`] | ||
/// | ||
/// This can terminate the scan early when some dynamic filters is updated after | ||
/// the scan starts, so we discover after the scan starts that the file can be | ||
/// pruned (can't have matching rows). | ||
struct EarlyStoppingStream<S> { | ||
/// Has the stream finished processing? All subsequent polls will return | ||
/// None | ||
done: bool, | ||
file_pruner: FilePruner, | ||
files_ranges_pruned_statistics: Count, | ||
/// The inner stream | ||
inner: S, | ||
} | ||
|
||
impl<S> EarlyStoppingStream<S> { | ||
pub fn new( | ||
stream: S, | ||
file_pruner: FilePruner, | ||
files_ranges_pruned_statistics: Count, | ||
) -> Self { | ||
Self { | ||
done: false, | ||
inner: stream, | ||
file_pruner, | ||
files_ranges_pruned_statistics, | ||
} | ||
} | ||
} | ||
impl<S> EarlyStoppingStream<S> | ||
where | ||
S: Stream<Item = Result<RecordBatch, ArrowError>> + Unpin, | ||
{ | ||
fn check_prune( | ||
&mut self, | ||
input: Result<RecordBatch, ArrowError>, | ||
) -> Result<Option<RecordBatch>, ArrowError> { | ||
let batch = input?; | ||
|
||
// Since dynamic filters may have been updated, see if we can stop | ||
// reading this stream entirely. | ||
if self.file_pruner.should_prune()? { | ||
self.files_ranges_pruned_statistics.add(1); | ||
self.done = true; | ||
Ok(None) | ||
} else { | ||
// Return the adapted batch | ||
Ok(Some(batch)) | ||
} | ||
} | ||
} | ||
|
||
impl<S> Stream for EarlyStoppingStream<S> | ||
where | ||
S: Stream<Item = Result<RecordBatch, ArrowError>> + Unpin, | ||
{ | ||
type Item = Result<RecordBatch, ArrowError>; | ||
|
||
fn poll_next( | ||
mut self: Pin<&mut Self>, | ||
cx: &mut Context<'_>, | ||
) -> Poll<Option<Self::Item>> { | ||
if self.done { | ||
return Poll::Ready(None); | ||
} | ||
match ready!(self.inner.poll_next_unpin(cx)) { | ||
None => { | ||
// input done | ||
self.done = true; | ||
Poll::Ready(None) | ||
} | ||
Some(input_batch) => { | ||
let output = self.check_prune(input_batch); | ||
Poll::Ready(output.transpose()) | ||
} | ||
} | ||
} | ||
} | ||
|
||
#[cfg(feature = "parquet_encryption")] | ||
impl ParquetOpener { | ||
fn get_file_decryption_properties( | ||
|
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.
the core idea is to move this try_unfold closure into a real structure -- all the logic is the same, but I think it is easier to understand in its own, documented Stream struct