-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
Changes from 11 commits
446d052
99c4f5b
729115f
7f315cd
5db1bc1
5b25ca4
163567e
f92137f
add5c9e
c59c5f9
68bf187
2fa920e
86bb976
6661228
559ece5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| 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::{DataFusionError, Result}; | ||
| use datafusion_execution::memory_pool::MemoryReservation; | ||
| use datafusion_physical_expr_common::sort_expr::LexOrdering; | ||
| use futures::stream::{Fuse, StreamExt}; | ||
|
|
@@ -78,6 +78,8 @@ impl FusedStreams { | |
|
|
||
| /// 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 +90,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: Vec<[Option<Arc<Rows>>; 2]>, | ||
Dandandan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| impl RowCursorStream { | ||
|
|
@@ -105,26 +110,57 @@ 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, | ||
| }) | ||
| } | ||
|
|
||
| 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 = Arc::try_unwrap(self.rows[stream_idx][1].take().unwrap()) | ||
Dandandan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| .map_err(|_| { | ||
| DataFusionError::Internal( | ||
| "Rows from RowCursorStream is still in use by consumer".to_string(), | ||
Dandandan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| ) | ||
| })?; | ||
|
|
||
| rows.clear(); | ||
|
|
||
| self.converter.append(&mut rows, &cols)?; | ||
| self.reservation.try_resize(self.converter.size())?; | ||
|
|
||
| let rows = Arc::new(rows); | ||
|
|
||
| self.rows[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.rows[stream_idx]; | ||
| std::mem::swap(a, b); | ||
Dandandan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| // track the memory in the newly created Rows. | ||
| let mut rows_reservation = self.reservation.new_empty(); | ||
| rows_reservation.try_grow(rows.size())?; | ||
|
|
@@ -146,7 +182,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)) | ||
| }) | ||
| })) | ||
|
|
||
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 !