mbutrovich commented on code in PR #21600:
URL: https://github.com/apache/datafusion/pull/21600#discussion_r3080009565


##########
datafusion/physical-plan/src/sorts/sort.rs:
##########
@@ -323,56 +312,185 @@ impl ExternalSorter {
         self.reserve_memory_for_batch_and_maybe_spill(&input)
             .await?;
 
-        self.in_mem_batches.push(input);
+        let coalescer = self
+            .coalescer
+            .as_mut()
+            .expect("coalescer must exist during insert phase");
+        coalescer
+            .push_batch(input)
+            .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
+
+        self.drain_completed_batches()?;
+
+        Ok(())
+    }
+
+    /// Drains completed (full) batches from the coalescer, sorts each,
+    /// and appends the sorted chunks to `sorted_runs`.
+    fn drain_completed_batches(&mut self) -> Result<()> {
+        // Collect completed batches first to avoid borrow conflict
+        let mut completed = vec![];
+        if let Some(coalescer) = self.coalescer.as_mut() {
+            while let Some(batch) = coalescer.next_completed_batch() {
+                completed.push(batch);
+            }
+        }
+        for batch in &completed {
+            self.sort_and_store_run(batch)?;
+        }
+        Ok(())
+    }
+
+    /// Sorts a single coalesced batch and stores the result as a new run.
+    ///
+    /// Uses radix sort when the batch is large enough to amortize encoding
+    /// overhead (more than `batch_size` rows). Otherwise falls back to 
lexsort.
+    fn sort_and_store_run(&mut self, batch: &RecordBatch) -> Result<()> {
+        let use_radix_for_this_batch =
+            self.use_radix && batch.num_rows() > self.batch_size;
+

Review Comment:
   Thanks for the feedback @gratus00! I addressed both of these. Checking 
per-batch is wasteful too. I created an inner function that we can call 
directly because `sort_batch` is a public API and I don't want to change it.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to