zhuqi-lucas commented on code in PR #16647: URL: https://github.com/apache/datafusion/pull/16647#discussion_r2179323995
########## datafusion/physical-plan/src/sorts/stream.rs: ########## @@ -105,26 +110,53 @@ 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()) + .unwrap_or_else(|_| self.converter.empty_rows(0, 0)); + + 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 Review Comment: Thank you @Dandandan , this implementation is really clever: Double‐buffer swap for smooth handoff After appending the new data into the “cur” slot, swapping the two slots with std::mem::swap transparently rotates which buffer will be reused next. This means you always have one slot holding the “previous” data for downstream consumers and an idle slot ready for your next try_unwrap. ########## datafusion/physical-plan/src/sorts/cursor.rs: ########## @@ -151,7 +152,7 @@ impl<T: CursorValues> Ord for Cursor<T> { /// Used for sorting when there are multiple columns in the sort key #[derive(Debug)] pub struct RowValues { - rows: Rows, + rows: Arc<Rows>, Review Comment: It makes sense, thank you @Dandandan ! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org