Dandandan commented on code in PR #15380:
URL: https://github.com/apache/datafusion/pull/15380#discussion_r2050293912


##########
datafusion/physical-plan/src/sorts/sort.rs:
##########
@@ -662,53 +665,152 @@ impl ExternalSorter {
         let elapsed_compute = metrics.elapsed_compute().clone();
         let _timer = elapsed_compute.timer();
 
-        // Please pay attention that any operation inside of 
`in_mem_sort_stream` will
-        // not perform any memory reservation. This is for avoiding the need 
of handling
-        // reservation failure and spilling in the middle of the sort/merge. 
The memory
-        // space for batches produced by the resulting stream will be reserved 
by the
-        // consumer of the stream.
+        // Note, in theory in memory batches should have limited size, but 
some testing
+        // cases testing the memory limit use `sort_in_place_threshold_bytes` 
to, so here we
+        // set a larger limit to avoid testing failure.
+        if self.expr.len() <= 2
+            && self.reservation.size() < 1000 * 
self.sort_in_place_threshold_bytes
+        {
+            let interleave_indices = self.build_sorted_indices(
+                self.in_mem_batches.as_slice(),
+                Arc::clone(&self.expr),
+            )?;
 
-        if self.in_mem_batches.len() == 1 {
-            let batch = self.in_mem_batches.swap_remove(0);
-            let reservation = self.reservation.take();
-            return self.sort_batch_stream(batch, metrics, reservation);
-        }
+            let batches: Vec<&RecordBatch> = 
self.in_mem_batches.iter().collect();
+            let sorted_batch = interleave_record_batch(&batches, 
&interleave_indices)?;
 
-        // If less than sort_in_place_threshold_bytes, concatenate and sort in 
place
-        if self.reservation.size() < self.sort_in_place_threshold_bytes {
-            // Concatenate memory batches together and sort
-            let batch = concat_batches(&self.schema, &self.in_mem_batches)?;
             self.in_mem_batches.clear();
             self.reservation
-                .try_resize(get_reserved_byte_for_record_batch(&batch))
+                .try_resize(get_reserved_byte_for_record_batch(&sorted_batch))
                 .map_err(Self::err_with_oom_context)?;
-            let reservation = self.reservation.take();
-            return self.sort_batch_stream(batch, metrics, reservation);
+
+            metrics.record_output(sorted_batch.num_rows());
+
+            Ok(Box::pin(RecordBatchStreamAdapter::new(
+                Arc::clone(&self.schema),
+                futures::stream::once(async { Ok(sorted_batch) }),
+            )) as SendableRecordBatchStream)
+        } else {
+            // Please pay attention that any operation inside of 
`in_mem_sort_stream` will
+            // not perform any memory reservation. This is for avoiding the 
need of handling
+            // reservation failure and spilling in the middle of the 
sort/merge. The memory
+            // space for batches produced by the resulting stream will be 
reserved by the
+            // consumer of the stream.
+
+            if self.in_mem_batches.len() == 1 {
+                let batch = self.in_mem_batches.swap_remove(0);
+                let reservation = self.reservation.take();
+                return self.sort_batch_stream(batch, metrics, reservation);
+            }
+
+            // If less than sort_in_place_threshold_bytes, concatenate and 
sort in place
+            if self.reservation.size() < self.sort_in_place_threshold_bytes {
+                // Concatenate memory batches together and sort

Review Comment:
   I think one explanation can be that row format becomes faster at >2 columns 
(rather than at >1 as is currently used in most places).
   We probably can try as a next step confirm it by sorting by `Row` directly 
instead of sort followed by merge for this case and see if we get some 
performance increase there as well.



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