gruuya commented on code in PR #7180:
URL: https://github.com/apache/arrow-datafusion/pull/7180#discussion_r1285109657


##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -253,11 +251,27 @@ impl ExternalSorter {
     /// Appends an unsorted [`RecordBatch`] to `in_mem_batches`
     ///
     /// Updates memory usage metrics, and possibly triggers spilling to disk
-    async fn insert_batch(&mut self, input: RecordBatch) -> Result<()> {
+    async fn insert_batch(&mut self, mut input: RecordBatch) -> Result<()> {
         if input.num_rows() == 0 {
             return Ok(());
         }
 
+        let mut batch_sorted = false;
+        if self.fetch.map_or(false, |f| f < input.num_rows()) {
+            // Eagerly sort the batch to potentially reduce the number of rows
+            // after applying the fetch parameter; first perform a memory 
reservation
+            // for the sorting procedure.
+            let mut reservation =

Review Comment:
   So I went ahead to add this, but then it seemed to me that the logic of 
`split` (and `take` by extension) is backwards? In particular, the `size` of 
the reservation is not _free_ memory, but instead _used_ memory, right? And 
with `split` we'd effectively deduct some piece of memory in our accounting 
that was previously allocated (without actually freeing that memory). 
   
   Atm I went with skipping reservation updates at all in this case; the reason 
being that we're already holding the input batch in memory and we're are about 
to (try to) allocate memory for it right after sorting it anyway. Hence, the 
sole purpose of that reservation seems to be to account for that extra piece of 
memory due to non-in-place sorting (i.e. having the old and the new bath 
present at the same time briefly), and that piece of memory is guaranteed to be 
less than the batch size. Let me know if you'd like me to do something else.
   
   I also changed that tests output. Now all tests should be passing, so it's 
probably time for the regular benchmarks.



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

Reply via email to