-
Notifications
You must be signed in to change notification settings - Fork 1.9k
Reuse Rows allocation in RowCursorStream #16647
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
Show all changes
15 commits
Select commit
Hold shift + click to select a range
446d052
Reuse Rows in RowCursorStream
Dandandan 99c4f5b
WIP
Dandandan 729115f
Fmt
Dandandan 7f315cd
Add comment, make it backwards compatible
Dandandan 5db1bc1
Add comment, make it backwards compatible
Dandandan 5b25ca4
Add comment, make it backwards compatible
Dandandan 163567e
Clippy
Dandandan f92137f
Clippy
Dandandan add5c9e
Return error on non-unique reference
Dandandan c59c5f9
Comment
Dandandan 68bf187
Merge remote-tracking branch 'upstream/main' into reuse_rows
Dandandan 2fa920e
Update datafusion/physical-plan/src/sorts/stream.rs
Dandandan 86bb976
Fix
Dandandan 6661228
Extract logic
Dandandan 559ece5
Doc fix
Dandandan 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
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 |
|---|---|---|
|
|
@@ -21,8 +21,8 @@ use crate::{PhysicalExpr, PhysicalSortExpr}; | |
| use arrow::array::Array; | ||
| use arrow::datatypes::Schema; | ||
| use arrow::record_batch::RecordBatch; | ||
| use arrow::row::{RowConverter, SortField}; | ||
| use datafusion_common::Result; | ||
| use arrow::row::{RowConverter, Rows, SortField}; | ||
| use datafusion_common::{internal_datafusion_err, Result}; | ||
| use datafusion_execution::memory_pool::MemoryReservation; | ||
| use datafusion_physical_expr_common::sort_expr::LexOrdering; | ||
| use futures::stream::{Fuse, StreamExt}; | ||
|
|
@@ -76,8 +76,40 @@ impl FusedStreams { | |
| } | ||
| } | ||
|
|
||
| /// A pair of `Arc<Rows>` that can be reused | ||
| #[derive(Debug)] | ||
| struct ReusableRows { | ||
|
Contributor
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. ❤️ |
||
| // inner[stream_idx] holds a two Arcs: | ||
| // at start of a new poll | ||
| // .0 is the rows from the previous poll (at start), | ||
| // .1 is the one that is being written to | ||
| // at end of a poll, .0 will be swapped with .1, | ||
| inner: Vec<[Option<Arc<Rows>>; 2]>, | ||
| } | ||
|
|
||
| impl ReusableRows { | ||
| // return a Rows for writing, | ||
| // does not clone if the existing rows can be reused | ||
| fn take_next(&mut self, stream_idx: usize) -> Result<Rows> { | ||
| Arc::try_unwrap(self.inner[stream_idx][1].take().unwrap()).map_err(|_| { | ||
| internal_datafusion_err!( | ||
| "Rows from RowCursorStream is still in use by consumer" | ||
| ) | ||
| }) | ||
| } | ||
| // save the Rows | ||
| fn save(&mut self, stream_idx: usize, rows: Arc<Rows>) { | ||
| self.inner[stream_idx][1] = Some(Arc::clone(&rows)); | ||
| // swap the curent with the previous one, so that the next poll can reuse the Rows from the previous poll | ||
| let [a, b] = &mut self.inner[stream_idx]; | ||
| std::mem::swap(a, b); | ||
| } | ||
| } | ||
|
|
||
| /// A [`PartitionedStream`] that wraps a set of [`SendableRecordBatchStream`] | ||
| /// and computes [`RowValues`] based on the provided [`PhysicalSortExpr`] | ||
| /// Note: the stream returns an error if the consumer buffers more than one RowValues (i.e. holds on to two RowValues | ||
| /// from the same partition at the same time). | ||
| #[derive(Debug)] | ||
| pub struct RowCursorStream { | ||
| /// Converter to convert output of physical expressions | ||
|
|
@@ -88,6 +120,9 @@ pub struct RowCursorStream { | |
| streams: FusedStreams, | ||
| /// Tracks the memory used by `converter` | ||
| reservation: MemoryReservation, | ||
| /// Allocated rows for each partition, we keep two to allow for buffering one | ||
| /// in the consumer of the stream | ||
| rows: ReusableRows, | ||
| } | ||
|
|
||
| impl RowCursorStream { | ||
|
|
@@ -105,26 +140,48 @@ impl RowCursorStream { | |
| }) | ||
| .collect::<Result<Vec<_>>>()?; | ||
|
|
||
| let streams = streams.into_iter().map(|s| s.fuse()).collect(); | ||
| let streams: Vec<_> = streams.into_iter().map(|s| s.fuse()).collect(); | ||
| let converter = RowConverter::new(sort_fields)?; | ||
| let mut rows = Vec::with_capacity(streams.len()); | ||
| for _ in &streams { | ||
| // Initialize each stream with an empty Rows | ||
| rows.push([ | ||
| Some(Arc::new(converter.empty_rows(0, 0))), | ||
| Some(Arc::new(converter.empty_rows(0, 0))), | ||
| ]); | ||
| } | ||
| Ok(Self { | ||
| converter, | ||
| reservation, | ||
| column_expressions: expressions.iter().map(|x| Arc::clone(&x.expr)).collect(), | ||
| streams: FusedStreams(streams), | ||
| rows: ReusableRows { inner: rows }, | ||
| }) | ||
| } | ||
|
|
||
| fn convert_batch(&mut self, batch: &RecordBatch) -> Result<RowValues> { | ||
| fn convert_batch( | ||
| &mut self, | ||
| batch: &RecordBatch, | ||
| stream_idx: usize, | ||
| ) -> Result<RowValues> { | ||
| let cols = self | ||
| .column_expressions | ||
| .iter() | ||
| .map(|expr| expr.evaluate(batch)?.into_array(batch.num_rows())) | ||
| .collect::<Result<Vec<_>>>()?; | ||
|
|
||
| let rows = self.converter.convert_columns(&cols)?; | ||
| // At this point, ownership should of this Rows should be unique | ||
| let mut rows = self.rows.take_next(stream_idx)?; | ||
|
|
||
| rows.clear(); | ||
|
|
||
| self.converter.append(&mut rows, &cols)?; | ||
| self.reservation.try_resize(self.converter.size())?; | ||
|
|
||
| let rows = Arc::new(rows); | ||
|
|
||
| self.rows.save(stream_idx, Arc::clone(&rows)); | ||
|
|
||
| // track the memory in the newly created Rows. | ||
| let mut rows_reservation = self.reservation.new_empty(); | ||
| rows_reservation.try_grow(rows.size())?; | ||
|
|
@@ -146,7 +203,7 @@ impl PartitionedStream for RowCursorStream { | |
| ) -> Poll<Option<Self::Output>> { | ||
| Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { | ||
| r.and_then(|batch| { | ||
| let cursor = self.convert_batch(&batch)?; | ||
| let cursor = self.convert_batch(&batch, stream_idx)?; | ||
| Ok((cursor, batch)) | ||
| }) | ||
| })) | ||
|
|
||
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.
It makes sense, thank you @Dandandan !