alamb commented on code in PR #16647:
URL: https://github.com/apache/datafusion/pull/16647#discussion_r2180789490


##########
datafusion/physical-plan/src/sorts/stream.rs:
##########
@@ -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]>,

Review Comment:
   I think calling this `rows` is somewhat confusing. How about `row_buffer`?
   
   Something that might make this code easier to read could be a wrapper,  the 
major benefit could be to explain what is going on
   
   ```suggestion
       rows: ReusedRows,
   ```
   
   ```rust
   
   /// A pair pf Arc<Rows> that can be reused
   struct ReusableRows {
     // inner[stream_idx] holds a two Arcs: 
     // .0 is the currently rows
     // .1 is the currently 
     inner: Vec<[Option<Arc<Rows>>; 2]>,
   }
   
   impl ReusedRows {
     // return a Rows for writing, 
     // does not clone if the existing rows can be reused
     fn take_next(&mut self, stream_idx: usize) -> Rows {
           Arc::try_unwrap(self.inner[stream_idx][1].take().unwrap())
               .unwrap_or_else(|_| self.converter.empty_rows(0, 0));
      }
     // save the Rows
     fn save(&mut self, stream_idx, rows: Arc<Rows>) {
       self.rows[stream_idx][1] = Some(Arc::clone(&rows));
        let [a, b] = &mut self.rows[stream_idx];
        std::mem::swap(a, b);
     }
   
   ```
   
   



##########
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())

Review Comment:
   Would it be better to avoid an unwrap here and instead just use a new rows 
if there is no previous rows?
   
   If we could figure out how to encapsulate it more, maybe it would be clearer 
that stream_idx[1] should always be Some



##########
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
+        let [a, b] = &mut self.rows[stream_idx];
+        std::mem::swap(a, b);

Review Comment:
   I would personally find this less magical to understand (though I realize it 
is a matter of opinion)
   ```suggestion
           std::mem::swap(
             &mut self.rows[stream_idx][0], 
             &mut self.rows[stream_idx][1], 
           )
   ```



-- 
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

Reply via email to