Rewrite FileStream in terms of Morsel API#21342
Conversation
816d243 to
3346af7
Compare
| /// 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 { |
There was a problem hiding this comment.
This is the new inner state machine for FileStream
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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. |
There was a problem hiding this comment.
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 | |||
There was a problem hiding this comment.
This is testing infrastructure to write the snapshot tests
There was a problem hiding this comment.
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 } => { |
There was a problem hiding this comment.
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 |
There was a problem hiding this comment.
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
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
b5c452a to
d5a1f74
Compare
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
| all-features = true | ||
|
|
||
| [features] | ||
| backtrace = ["datafusion-common/backtrace"] |
There was a problem hiding this comment.
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
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
d5a1f74 to
b2c9bd6
Compare
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
adriangb
left a comment
There was a problem hiding this comment.
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.
| /// 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))) | ||
| } |
| /// 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 { |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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
| /// The active reader, if any. | ||
| reader: Option<BoxStream<'static, Result<RecordBatch>>>, |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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)
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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"?
There was a problem hiding this comment.
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,
...|
Ok the first PR in the chain is ready for review: (that is basically 50% of this PR) |
| /// 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 { |
There was a problem hiding this comment.
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
| /// The active reader, if any. | ||
| reader: Option<BoxStream<'static, Result<RecordBatch>>>, |
There was a problem hiding this comment.
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)
| /// The active reader, if any. | ||
| reader: Option<BoxStream<'static, Result<RecordBatch>>>, |
There was a problem hiding this comment.
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,
...| // Morsels should ideally only expose ready-to-decode streams, | ||
| // but tolerate pending readers here. |
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
THis is a good call -- I think a queue of pending planners is best. Will do
|
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:
|
|
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:
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 |
adriangb
left a comment
There was a problem hiding this comment.
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 |
|
FYI @Dandandan @zhuliquan and @xudong963 in case you would like to review |
Update is I filed a ticket to explain what I found here |
|
I'll merge this one in and get the final one ready for review |
|
Thank you for helping this along @adriangb |
|
Woah merged!!! 🥳 🥳 🥳 |
|
Well, this is just one of the refactors -- the real change in behavior (benefit) comes in |
## Which issue does this PR close? - Closes #20529 - Closes #20820 ## Rationale for this change This PR finally enables dynamic work scheduling in the FileStream (so that if a task is done it can look at any remaining work) This improves performance on queries that scan multiple files and the work is not balanced evenly across partitions in the plan (e.g. we have dynamic filtering that reduces work significantly) It is the last of a sequence of several PRs: - #21342 - #21327 - #21340 ## What changes are included in this PR? 1. Add shared state across sibling FileStream's and the wiring to connect them 2. Sibling streams put their file work into a shared queue when it can be reordered 3. Add a bunch of tests sjpw Note there are a bunch of other things that are NOT included in this PR, including 1. Trying to limit concurrent IO (this PR has the same properties as main -- up to one outstanding IO per partition) 2. Trying to issue multiple IOs by the same partition (aka to interleave IO and CPU work more) 4. Splitting files into smaller units (e.g. across row groups) As @Dandandan proposes below, I expect we can work on those changes as follow on PRs. ## Are these changes tested? Yes by existing functional and benchmark tests, as well as new functional tests ## Are there any user-facing changes? Yes, faster performance (see benchmarks): #21351 (comment) --------- Co-authored-by: Oleks V <comphead@users.noreply.github.com>
…er` (apache#21327) ~(Draft until I am sure I can use this API to make FileStream behave better)~ ## Which issue does this PR close? - part of apache#20529 - Needed for apache#21351 - Broken out of apache#20820 - Closes apache#21427 ## Rationale for this change I can get 10% faster on many ClickBench queries by reordeirng files at runtime. You can see it all working together here: apache#21351 To do do, I need to rework the FileStream so that it can reorder operations at runtime. Eventually that will include both CPU and IO. This PR is a step in the direction by introducing the main Morsel API and implementing it for Parquet. The next PR (apache#21342) rewrites FileStream in terms of the Morsel API ## What changes are included in this PR? 1. Add proposed `Morsel` API 2. Rewrite Parquet opener in terms of that API 3. Add an adapter layer (back to FileOpener, so I don't have to rewrite FileStream in the same PR) My next PR will rewrite the FileStream to use the Morsel API ## Are these changes tested? Yes by existing CI. I will work on adding additional tests for just Parquet opener in a follow on PR ## Are there any user-facing changes? No
Stacked on - apache#21327 - apache#21340 ## Which issue does this PR close? - part of apache#20529 - Broken out of apache#20820 ## 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) --------- Co-authored-by: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com>
## Which issue does this PR close? - Closes apache#20529 - Closes apache#20820 ## Rationale for this change This PR finally enables dynamic work scheduling in the FileStream (so that if a task is done it can look at any remaining work) This improves performance on queries that scan multiple files and the work is not balanced evenly across partitions in the plan (e.g. we have dynamic filtering that reduces work significantly) It is the last of a sequence of several PRs: - apache#21342 - apache#21327 - apache#21340 ## What changes are included in this PR? 1. Add shared state across sibling FileStream's and the wiring to connect them 2. Sibling streams put their file work into a shared queue when it can be reordered 3. Add a bunch of tests sjpw Note there are a bunch of other things that are NOT included in this PR, including 1. Trying to limit concurrent IO (this PR has the same properties as main -- up to one outstanding IO per partition) 2. Trying to issue multiple IOs by the same partition (aka to interleave IO and CPU work more) 4. Splitting files into smaller units (e.g. across row groups) As @Dandandan proposes below, I expect we can work on those changes as follow on PRs. ## Are these changes tested? Yes by existing functional and benchmark tests, as well as new functional tests ## Are there any user-facing changes? Yes, faster performance (see benchmarks): apache#21351 (comment) --------- Co-authored-by: Oleks V <comphead@users.noreply.github.com>
Stacked on
ParquetOpenertoParquetMorselizer#21327Which 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.
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)