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


##########
datafusion/core/tests/memory_limit.rs:
##########
@@ -196,6 +205,110 @@ async fn symmetric_hash_join() {
     .await
 }
 
+#[tokio::test]
+async fn sort_preserving_merge() {
+    let partition_size = batches_byte_size(&dict_batches());
+
+    TestCase::new(
+        // This query uses the exact same ordering as the input table
+        // so only a merge is needed
+        "select * from t ORDER BY a ASC NULLS LAST, b ASC NULLS LAST LIMIT 10",
+        vec![
+            "Resources exhausted: Failed to allocate additional",
+            "SortPreservingMergeExec",
+        ],
+        // provide insufficient memory to merge
+        partition_size / 2,
+    )
+        // two partitions of data, so a merge is required
+        .with_scenario(Scenario::DictionaryStrings(2))
+        .with_expected_plan(
+            // It is important that this plan only has
+            // SortPreservingMergeExec (not a Sort which would compete
+            // with the SortPreservingMergeExec for memory)
+            &[
+                
"+---------------+-------------------------------------------------------------------------------------------------------------+",
+                "| plan_type     | plan                                        
                                                                |",
+                
"+---------------+-------------------------------------------------------------------------------------------------------------+",
+                "| logical_plan  | Limit: skip=0, fetch=10                     
                                                                |",
+                "|               |   Sort: t.a ASC NULLS LAST, t.b ASC NULLS 
LAST, fetch=10                                                    |",
+                "|               |     TableScan: t projection=[a, b]          
                                                                |",
+                "| physical_plan | GlobalLimitExec: skip=0, fetch=10           
                                                                |",
+                "|               |   SortPreservingMergeExec: [a@0 ASC NULLS 
LAST,b@1 ASC NULLS LAST], fetch=10                                |",
+                "|               |     MemoryExec: partitions=2, 
partition_sizes=[5, 5], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST 
|",
+                "|               |                                             
                                                                |",
+                
"+---------------+-------------------------------------------------------------------------------------------------------------+",
+            ]
+        )
+        .run()
+        .await
+}
+
+#[tokio::test]
+async fn sort_spill_reservation() {
+    let partition_size = batches_byte_size(&dict_batches());
+
+    let base_config = SessionConfig::new()
+        // do not allow the sort to use the 'concat in place' path
+        .with_sort_in_place_threshold_bytes(10);
+
+    // This test case shows how sort_spill_reservation works by
+    // purposely sorting data that requires non trivial memory to
+    // sort/merge.
+    let test = TestCase::new(
+        // This query uses a different order than the input table to
+        // force a sort. It also needs to have multiple columns to
+        // force RowFormat / interner that makes merge require
+        // substantial memory
+        "select * from t ORDER BY a , b DESC",
+        vec![], // expected errors set below
+        // enough memory to sort if we don't try to merge it all at once
+        (partition_size * 5) / 2,
+    )
+        // use a single partiton so only a sort is needed

Review Comment:
   This test demonstrates the need for reserving memory up front for the spill 
-- and shows that if someone hits the error they can increased the memory set 
aside for the merge and it will work



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