You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/05/18 17:47:58 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6382: Account for memory usage in SortPreservingMerge (#5885)

alamb commented on code in PR #6382:
URL: https://github.com/apache/arrow-datafusion/pull/6382#discussion_r1198076661


##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -238,6 +247,9 @@ impl ExternalSorter {
             return Ok(());
         }
 
+        // Release the memory reserved for merge
+        self.merge_reservation.free();
+
         self.in_mem_batches = self
             .in_mem_sort_stream(self.metrics.baseline.intermediate())?

Review Comment:
   I double checked that `in_mem_sort_stream` correctly respects `self.reservation` 👍  



##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -55,6 +55,9 @@ use tempfile::NamedTempFile;
 use tokio::sync::mpsc::{Receiver, Sender};
 use tokio::task;
 
+/// How much memory to reserve for performing in-memory sorts

Review Comment:
   ```suggestion
   /// How much memory to reserve for performing in-memory sorts prior to spill
   ```



##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -55,6 +55,9 @@ use tempfile::NamedTempFile;
 use tokio::sync::mpsc::{Receiver, Sender};
 use tokio::task;
 
+/// How much memory to reserve for performing in-memory sorts
+const EXTERNAL_SORTER_MERGE_RESERVATION: usize = 10 * 1024 * 1024;

Review Comment:
   The problem with this approach is that even 10MB may not be enough to correctly merge the batches prior to spilling. So some queries that today would succeed (though exceed their memory limits) might fail.
   
   It seems to me better approaches (as follow on PRs) would be:
   1. Make this a config parameter so users can avoid the error by reserving more memory up front if needed
   2. teach SortExec how to write more (smaller) spill files if it doesn't have enough memory to merge the in memory batches. 
   
   However, given the behavior on master today is to simply ignore the reservation and exceed the memory limit this behavior seems better than before. 
    
   I suggest we merge this PR as is and file a follow on ticket for the improved behavior 



##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -94,8 +97,10 @@ struct ExternalSorter {
     expr: Arc<[PhysicalSortExpr]>,
     metrics: ExternalSorterMetrics,
     fetch: Option<usize>,
+    /// Reservation for in_mem_batches
     reservation: MemoryReservation,
-    partition_id: usize,
+    /// Reservation for in memory sorting of batches

Review Comment:
   ```suggestion
       /// Reservation for in memory sorting of batches, prior to spilling.
       /// Without this reservation, when the memory budget is exhausted
       /// it might not be possible to merge the in memory batches as part 
       /// of spilling.
   ```



##########
datafusion/execution/src/memory_pool/pool.rs:
##########
@@ -145,21 +145,21 @@ impl MemoryPool for FairSpillPool {
 
     fn unregister(&self, consumer: &MemoryConsumer) {
         if consumer.can_spill {
-            self.state.lock().num_spill -= 1;
+            self.state.lock().num_spill.checked_sub(1).unwrap();

Review Comment:
   Maybe it would be worth adding some unit tests to the `MemoryReservation` now given it is growing in sophistication



##########
datafusion/core/src/physical_plan/sorts/cursor.rs:
##########
@@ -29,6 +30,9 @@ pub struct RowCursor {
     num_rows: usize,
 
     rows: Rows,
+
+    #[allow(dead_code)]

Review Comment:
   I think it would help to note here in comments why the code needs to keep around a field that is never read (`dead_code`). I think it is to keep the reservation around long enough?



-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org