Skip to content

Rewrite FileStream in terms of Morsel API#21342

Merged
alamb merged 13 commits intoapache:mainfrom
alamb:alamb/file_stream_split
Apr 14, 2026
Merged

Rewrite FileStream in terms of Morsel API#21342
alamb merged 13 commits intoapache:mainfrom
alamb:alamb/file_stream_split

Conversation

@alamb
Copy link
Copy Markdown
Contributor

@alamb alamb commented Apr 3, 2026

Stacked on

Which issue does this PR close?

Rationale for this change

The Morsel API allows for finer grain parallelism (and IO). It is important to have the FileStream work in terms of the Morsel API to allow future features (like workstealing, etc)

What changes are included in this PR?

I apologize for the large diff; Note about 1/2 of this PR is tests and a test framework to test the calling sequence of FileStream.

  1. Rewrite FileStream in terms of the MorselAPI
  2. Add snapshot driven test to document the I/O and CPU patterns in FileStream
  3. Add snapshot based tests that show the ordering of files

Are these changes tested?

Yes by existing functional and benchmark tests, as well as new functional snapshot based tests

Are there any user-facing changes?

No (not yet)

@github-actions github-actions bot added the datasource Changes to the datasource crate label Apr 3, 2026
@alamb alamb force-pushed the alamb/file_stream_split branch 3 times, most recently from 816d243 to 3346af7 Compare April 3, 2026 16:14
/// This groups together ready planners, ready morsels, the active reader,
/// pending planner I/O, the remaining files and limit, and the metrics
/// associated with processing that work.
pub(super) struct ScanState {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is the new inner state machine for FileStream

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think some more diagrams in the docstring of the struct and/or fields could help. I'm trying to wrap my head around how the IO queue and such work.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I have added a diagram - let me know if that helps or if there is something else I can do

use std::sync::Arc;
use std::sync::mpsc::{self, Receiver, TryRecvError};

/// Adapt a legacy [`FileOpener`] to the morsel API.
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is an adapter so that existing FileOpeners continue to have the same behavior

@@ -0,0 +1,556 @@
// Licensed to the Apache Software Foundation (ASF) under one
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is testing infrastructure to write the snapshot tests

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Basically it makes a mock morselizer that records its steps so that the control flow of FileStream can be tested / verified

return Poll::Ready(Some(Err(err)));
}
}
FileStreamState::Scan { scan_state: queue } => {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

moved the inner state machine into a separate module/struct to try and keep indenting under control and encapsualte the complexity somewhat

assert!(err.contains("FileStreamBuilder invalid partition index: 1"));
}

/// Verifies the simplest morsel-driven flow: one planner produces one
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Here are tests showing the sequence of calls to the various morsel APIs. I intend to use this framework to show how work can migrate from one stream to the other

@alamb

This comment has been minimized.

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@alamb

This comment has been minimized.

@adriangbot

This comment has been minimized.

all-features = true

[features]
backtrace = ["datafusion-common/backtrace"]
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I added this while debugging why the tests failed on CI and not locally (it was when this feature flag was on the Error messages got mangled).

I added a crate level feature to enable the feature in datafusion-common so I could reproduce the error locally

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@alamb alamb force-pushed the alamb/file_stream_split branch from d5a1f74 to b2c9bd6 Compare April 3, 2026 17:38
@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

@adriangbot

This comment has been minimized.

Copy link
Copy Markdown
Contributor

@adriangb adriangb left a comment

Choose a reason for hiding this comment

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

Ran out of time for the last couple of files. A lot of the comments are just tracking my thought process, I plan to go over them again to clarify my own understanding but maybe they're helpful as input on how the code reads top to bottom for a first time reader.

Comment on lines +78 to +93
/// Creates a `dyn Morselizer` based on given parameters.
///
/// The default implementation preserves existing behavior by adapting the
/// legacy [`FileOpener`] API into a [`Morselizer`].
///
/// It is preferred to implement the [`Morselizer`] API directly by
/// implementing this method.
fn create_morselizer(
&self,
object_store: Arc<dyn ObjectStore>,
base_config: &FileScanConfig,
partition: usize,
) -> Result<Box<dyn Morselizer>> {
let opener = self.create_file_opener(object_store, base_config, partition)?;
Ok(Box::new(FileOpenerMorselizer::new(opener)))
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

🚀

/// Configure the [`FileOpener`] used to open files.
///
/// This will overwrite any setting from [`Self::with_morselizer`]
pub fn with_file_opener(mut self, file_opener: Arc<dyn FileOpener>) -> Self {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

While I think it could make sense to keep FileOpener as a public API for building data sources (if we consider it simpler, for folks who don't care about perf), this method in particular seems like a mostly internal method (even if it is pub) on we might as well deprecate / remove.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This method is the way we could keep using FileOpener (as it is simpler)

I am not sure how we could still allow using FileOpener but not keep this method

Comment on lines +61 to +62
/// The active reader, if any.
reader: Option<BoxStream<'static, Result<RecordBatch>>>,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Is there one ScanState across all partitions or one per partition? I'm guessing the latter: file_iter: VecDeque<PartitionedFile> is the files for this partition, we pump all of the files into one output stream of RecordBatch (reader). But we can have multiple planners / morsels ready and merge those all into a single stream of RecordBatch on the way out.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

One per partition

we pump all of the files into one output stream of RecordBatch (reader). But we can have multiple planners / morsels ready and merge those all into a single stream of RecordBatch on the way out.

My initial proposal (following @Dandandan 's original design" is that when possible the files are put into a shared queue so that when a FileStream is ready it gets the next file

I think once we get that structure in place, we can contemplate more sophisticated designs (like one filestream preparing a parquet file, and then divying up the record batches between other cores)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Is there one ScanState across all partitions or one per partition? I'm guessing the latter: file_iter: VecDeque is the files for this partition

yes, it is one ScanState per partition

we pump all of the files into one output stream of RecordBatch (reader). But we can have multiple planners / morsels ready and merge those all into a single stream of RecordBatch on the way out.

Yes this is right

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

My initial proposal (following @Dandandan 's original design" is that when possible the files are put into a shared queue so that when a FileStream is ready it gets the next file

yes, it is one ScanState per partition

I'm a bit confused then: if there is one ScanState per partition then there is one VecDeque<PartitionedFile>, which means it's not shared between partitions. But that would contradict
"files are put into a shared queue so that when a FileStream is ready it gets the next file"?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

You can see how cross stream sharing works in the next stacked PR:

the ScanState is not shared across partitions, but it has a new work_queue that is (potentially) shared. The relevant change is to replace the file_iter with this work_source thing and then handle setting up the work_source in the DataSource exec

pub(super) struct ScanState {
    /// Files that still need to be planned.
    file_iter: VecDeque<PartitionedFile>,
...

Wth

pub(super) struct ScanState {
    /// Unopened files that still need to be planned for this stream.
    work_source: WorkSource,
...

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 6, 2026

Thanks @adriangb -- I think I am now convinced this will make things faster (see #21351)

Once I finalze that I will then go back and ping / respond to each of these PRs in turn

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 6, 2026

Ok the first PR in the chain is ready for review:

(that is basically 50% of this PR)

Copy link
Copy Markdown
Contributor Author

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Thanks for the comments @adriangb

Once I get this PR merged

I will come back and update this PR / address your comments

/// Configure the [`FileOpener`] used to open files.
///
/// This will overwrite any setting from [`Self::with_morselizer`]
pub fn with_file_opener(mut self, file_opener: Arc<dyn FileOpener>) -> Self {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This method is the way we could keep using FileOpener (as it is simpler)

I am not sure how we could still allow using FileOpener but not keep this method

Comment on lines +61 to +62
/// The active reader, if any.
reader: Option<BoxStream<'static, Result<RecordBatch>>>,
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

One per partition

we pump all of the files into one output stream of RecordBatch (reader). But we can have multiple planners / morsels ready and merge those all into a single stream of RecordBatch on the way out.

My initial proposal (following @Dandandan 's original design" is that when possible the files are put into a shared queue so that when a FileStream is ready it gets the next file

I think once we get that structure in place, we can contemplate more sophisticated designs (like one filestream preparing a parquet file, and then divying up the record batches between other cores)

@alamb alamb requested a review from adriangb April 10, 2026 15:31
@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 10, 2026

My summary of the benchmark results is that this PR does not change performance characteristics (as expected)

The changes come in #21351

/// This mirrors how `FileStream` consumes the morsel APIs: it repeatedly
/// plans CPU work, awaits any discovered I/O futures, and feeds the planner
/// back into the ready queue until a stream morsel is ready.
async fn open_file(
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this is needed so that all the tests in this file don't need to be rewritten to the morselizer API

+ Send,
>,
>,
mut stream: BoxStream<'static, Result<RecordBatch>>,
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

just a simpler way to write this

},
});
Ok(opener)
Ok(Box::new(ParquetMorselizer {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this passes the new morselizer directly rather than wrapping it first

/// This groups together ready planners, ready morsels, the active reader,
/// pending planner I/O, the remaining files and limit, and the metrics
/// associated with processing that work.
pub(super) struct ScanState {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I have added a diagram - let me know if that helps or if there is something else I can do

///
/// The return [`ScanAndReturn`] tells `poll_inner` how to update the
/// outer `FileStreamState`.
pub(super) fn poll_scan(&mut self, cx: &mut Context<'_>) -> ScanAndReturn {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is the main state machine loop that shows how the file stream transitions between states

@@ -0,0 +1,556 @@
// Licensed to the Apache Software Foundation (ASF) under one
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Basically it makes a mock morselizer that records its steps so that the control flow of FileStream can be tested / verified

Comment on lines +152 to +153
// Morsels should ideally only expose ready-to-decode streams,
// but tolerate pending readers here.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Is this another case where we could better encode the behavior constraints into the type system?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

That is a good question... I think we would have to change the inner API to use something other than Stream (perhaps just an iterator). I'll see what I can come up with

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ok, an update here. I tried this out and hit some snags -- see here for deatils

self.ready_morsels.extend(plan.take_morsels());
self.ready_planners.extend(plan.take_ready_planners());
if let Some(pending_planner) = plan.take_pending_planner() {
self.pending_planner = Some(pending_planner);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Should we at least assert the invariant assert!(self.pending_planner.is_none())? Or should we have a queue (VecDequeu) of pending planners?

Copy link
Copy Markdown
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 good call -- I think a queue of pending planners is best. Will do

Copy link
Copy Markdown
Contributor Author

@alamb alamb Apr 13, 2026

Choose a reason for hiding this comment

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

I played around with this and it turns out to be non trivial (mostly due to testing) and the current parquet morselizer doesn't actually return multiple child planners (yet)

So for this PR I would like to proceed with just an assert and I will leave some comments / a note about supporting multiple returned planners in the future

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I played around with this more and changed the core loop here to make things better in 42d8750

The current design of FileStream only allows for a single outstanding IO (to avoid having to deal with backpressure, etc)

Comment on lines +61 to +62
/// The active reader, if any.
reader: Option<BoxStream<'static, Result<RecordBatch>>>,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

My initial proposal (following @Dandandan 's original design" is that when possible the files are put into a shared queue so that when a FileStream is ready it gets the next file

yes, it is one ScanState per partition

I'm a bit confused then: if there is one ScanState per partition then there is one VecDeque<PartitionedFile>, which means it's not shared between partitions. But that would contradict
"files are put into a shared queue so that when a FileStream is ready it gets the next file"?

alamb and others added 2 commits April 13, 2026 08:35
Co-authored-by: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com>
Copy link
Copy Markdown
Contributor Author

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Thank you @adriangb -- I am working on your comments

Comment on lines +61 to +62
/// The active reader, if any.
reader: Option<BoxStream<'static, Result<RecordBatch>>>,
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

You can see how cross stream sharing works in the next stacked PR:

the ScanState is not shared across partitions, but it has a new work_queue that is (potentially) shared. The relevant change is to replace the file_iter with this work_source thing and then handle setting up the work_source in the DataSource exec

pub(super) struct ScanState {
    /// Files that still need to be planned.
    file_iter: VecDeque<PartitionedFile>,
...

Wth

pub(super) struct ScanState {
    /// Unopened files that still need to be planned for this stream.
    work_source: WorkSource,
...

Comment on lines +152 to +153
// Morsels should ideally only expose ready-to-decode streams,
// but tolerate pending readers here.
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

That is a good question... I think we would have to change the inner API to use something other than Stream (perhaps just an iterator). I'll see what I can come up with

self.ready_morsels.extend(plan.take_morsels());
self.ready_planners.extend(plan.take_ready_planners());
if let Some(pending_planner) = plan.take_pending_planner() {
self.pending_planner = Some(pending_planner);
Copy link
Copy Markdown
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 good call -- I think a queue of pending planners is best. Will do

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 13, 2026

I have also updated the testing mocks to more closely follow the morsel API (so that I can test the suggestions from @adriangb )

I ran out of time now, but hopefully soon I'll move on to trying to explore:

Is this another case where we could better encode the behavior constraints into the type system?

@adriangb
Copy link
Copy Markdown
Contributor

Let me know when you want me to do the next (which feels like it might be the final) round of review.

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 13, 2026

Let me know when you want me to do the next (which feels like it might be the final) round of review.

Thanks -- I haven't explored this more yet:

Is this another case where we could better encode the behavior constraints into the type system?

But I can also do it as part of a follow on PR (I think it would need to change the Morsel API)

I don't think it really changes this PR per se

Copy link
Copy Markdown
Contributor

@adriangb adriangb left a comment

Choose a reason for hiding this comment

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

Makes sense to me. We can iterate on the APIs as a followup but I think we should keep that in mind, it feels like there's some improvements we can make.

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 13, 2026

Makes sense to me. We can iterate on the APIs as a followup but I think we should keep that in mind, it feels like there's some improvements we can make.

Yes it is an excellent point actually -- and one I think we can resolve

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 13, 2026

FYI @Dandandan @zhuliquan and @xudong963 in case you would like to review

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 14, 2026

Makes sense to me. We can iterate on the APIs as a followup but I think we should keep that in mind, it feels like there's some improvements we can make.

Yes it is an excellent point actually -- and one I think we can resolve

Update is I filed a ticket to explain what I found here

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 14, 2026

I'll merge this one in and get the final one ready for review

@alamb
Copy link
Copy Markdown
Contributor Author

alamb commented Apr 14, 2026

Thank you for helping this along @adriangb

@alamb alamb added this pull request to the merge queue Apr 14, 2026
Merged via the queue into apache:main with commit 776b723 Apr 14, 2026
34 checks passed
@alamb alamb deleted the alamb/file_stream_split branch April 14, 2026 11:25
@Dandandan
Copy link
Copy Markdown
Contributor

Woah merged!!! 🥳 🥳 🥳

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

datasource Changes to the datasource crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants