Skip to content
Merged
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
1 change: 1 addition & 0 deletions vortex-datafusion/src/persistent/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ pub mod metrics;
mod opener;
mod sink;
mod source;
mod stream;

pub use access_plan::VortexAccessPlan;
pub use format::VortexFormat;
Expand Down
7 changes: 6 additions & 1 deletion vortex-datafusion/src/persistent/opener.rs
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ use crate::VortexAccessPlan;
use crate::convert::exprs::ExpressionConvertor;
use crate::convert::exprs::can_be_pushed_down;
use crate::convert::exprs::make_vortex_predicate;
use crate::persistent::stream::PrunableStream;

#[derive(Clone)]
pub(crate) struct VortexOpener {
Expand Down Expand Up @@ -336,7 +337,11 @@ impl FileOpener for VortexOpener {
.map(move |batch| batch.and_then(|b| schema_mapping.map_batch(b)))
.boxed();

Ok(stream)
if let Some(file_pruner) = file_pruner {
Ok(PrunableStream::new(file_pruner, stream).boxed())
} else {
Ok(stream)
}
}
.in_current_span()
.boxed())
Expand Down
40 changes: 40 additions & 0 deletions vortex-datafusion/src/persistent/stream.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright the Vortex contributors

use std::pin::Pin;
use std::task::Context;
use std::task::Poll;

use datafusion_common::Result as DFResult;
use datafusion_common::arrow::array::RecordBatch;
use datafusion_pruning::FilePruner;
use futures::Stream;
use futures::StreamExt;
use futures::stream::BoxStream;

/// Utility to end a stream early if its backing [`PartitionFile`] can be pruned away by an updated dynamic expression.
pub(crate) struct PrunableStream {
file_pruner: FilePruner,
stream: BoxStream<'static, DFResult<RecordBatch>>,
}

impl PrunableStream {
pub fn new(file_pruner: FilePruner, stream: BoxStream<'static, DFResult<RecordBatch>>) -> Self {
Self {
file_pruner,
stream,
}
}
}

impl Stream for PrunableStream {
type Item = DFResult<RecordBatch>;

fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
if self.as_mut().file_pruner.should_prune()? {
Poll::Ready(None)
} else {
self.stream.poll_next_unpin(cx)
}
}
}
Loading