Skip to content
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

FileStream: Open next file in parallel while decoding #5161

Merged
merged 4 commits into from
Feb 7, 2023
Merged
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
86 changes: 67 additions & 19 deletions datafusion/core/src/physical_plan/file_format/file_stream.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
//! compliant with the `SendableRecordBatchStream` trait.

use std::collections::VecDeque;
use std::mem;
use std::pin::Pin;
use std::task::{Context, Poll};
use std::time::Instant;
Expand Down Expand Up @@ -98,6 +99,11 @@ enum FileStreamState {
partition_values: Vec<ScalarValue>,
/// The reader instance
reader: BoxStream<'static, Result<RecordBatch, ArrowError>>,
/// A [`FileOpenFuture`] for the next file to be processed,
/// and its corresponding partition column values, if any.
/// This allows the next file to be opened in parallel while the
/// current file is read.
next: Option<(FileOpenFuture, Vec<ScalarValue>)>,
Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder if we could make it future-proof by potentially prefetching n files instead of 1? I guess in cases where file opening is slower than scanning / processing, this could make a difference (e.g. small files).

Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps a follow on PR could turn this into a stream and use StreamExt::buffered or something

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yeah, that seems like a good idea

},
/// Encountered an error
Error,
Expand Down Expand Up @@ -202,30 +208,39 @@ impl<F: FileOpener> FileStream<F> {
})
}

// Begin opening the next file in parallel while decoding the current file in FileStream.
// Since file opening is mostly IO (and may involve a
// bunch of sequential IO), it can be parallelized with decoding.
fn start_next_file(&mut self) -> Option<Result<(FileOpenFuture, Vec<ScalarValue>)>> {
let part_file = self.file_iter.pop_front()?;

let file_meta = FileMeta {
object_meta: part_file.object_meta,
range: part_file.range,
extensions: part_file.extensions,
};

Some(
self.file_reader
.open(file_meta)
.map(|future| (future, part_file.partition_values)),
)
}

fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll<Option<Result<RecordBatch>>> {
loop {
match &mut self.state {
FileStreamState::Idle => {
let part_file = match self.file_iter.pop_front() {
Some(file) => file,
None => return Poll::Ready(None),
};

let file_meta = FileMeta {
object_meta: part_file.object_meta,
range: part_file.range,
extensions: part_file.extensions,
};

self.file_stream_metrics.time_opening.start();

match self.file_reader.open(file_meta) {
Ok(future) => {
match self.start_next_file().transpose() {
Ok(Some((future, partition_values))) => {
self.state = FileStreamState::Open {
future,
partition_values: part_file.partition_values,
partition_values,
}
}
Ok(None) => return Poll::Ready(None),
Err(e) => {
self.state = FileStreamState::Error;
return Poll::Ready(Some(Err(e)));
Expand All @@ -237,13 +252,34 @@ impl<F: FileOpener> FileStream<F> {
partition_values,
} => match ready!(future.poll_unpin(cx)) {
Ok(reader) => {
let partition_values = mem::take(partition_values);

let next = self.start_next_file().transpose();

self.file_stream_metrics.time_opening.stop();
self.file_stream_metrics.time_scanning_until_data.start();
self.file_stream_metrics.time_scanning_total.start();
self.state = FileStreamState::Scan {
partition_values: std::mem::take(partition_values),
reader,
};

match next {
Ok(Some((next_future, next_partition_values))) => {
self.state = FileStreamState::Scan {
partition_values,
reader,
next: Some((next_future, next_partition_values)),
};
}
Ok(None) => {
self.state = FileStreamState::Scan {
reader,
partition_values,
next: None,
};
}
Err(e) => {
self.state = FileStreamState::Error;
return Poll::Ready(Some(Err(e)));
}
}
}
Err(e) => {
self.state = FileStreamState::Error;
Expand All @@ -253,6 +289,7 @@ impl<F: FileOpener> FileStream<F> {
FileStreamState::Scan {
reader,
partition_values,
next,
} => match ready!(reader.poll_next_unpin(cx)) {
Some(result) => {
self.file_stream_metrics.time_scanning_until_data.stop();
Expand Down Expand Up @@ -287,7 +324,18 @@ impl<F: FileOpener> FileStream<F> {
None => {
self.file_stream_metrics.time_scanning_until_data.stop();
self.file_stream_metrics.time_scanning_total.stop();
self.state = FileStreamState::Idle;

match mem::take(next) {
Some((future, partition_values)) => {
self.file_stream_metrics.time_opening.start();

self.state = FileStreamState::Open {
future,
partition_values,
}
}
None => return Poll::Ready(None),
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
None => return Poll::Ready(None),
// No more input files
None => return Poll::Ready(None),

}
}
},
FileStreamState::Error | FileStreamState::Limit => {
Expand Down