2010YOUY01 commented on code in PR #15700:
URL: https://github.com/apache/datafusion/pull/15700#discussion_r2230075680


##########
datafusion/physical-plan/src/sorts/multi_level_merge.rs:
##########
@@ -0,0 +1,449 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Create a stream that do a multi level merge stream
+
+use crate::metrics::BaselineMetrics;
+use crate::{EmptyRecordBatchStream, SpillManager};
+use arrow::array::RecordBatch;
+use std::fmt::{Debug, Formatter};
+use std::mem;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+
+use arrow::datatypes::SchemaRef;
+use datafusion_common::Result;
+use datafusion_execution::memory_pool::MemoryReservation;
+
+use crate::sorts::sort::get_reserved_byte_for_record_batch_size;
+use crate::sorts::streaming_merge::{SortedSpillFile, StreamingMergeBuilder};
+use crate::stream::RecordBatchStreamAdapter;
+use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream};
+use datafusion_physical_expr_common::sort_expr::LexOrdering;
+use futures::TryStreamExt;
+use futures::{Stream, StreamExt};
+
+/// Merges a stream of sorted cursors and record batches into a single sorted 
stream
+///
+/// This is a wrapper around 
[`SortPreservingMergeStream`](crate::sorts::merge::SortPreservingMergeStream)
+/// that provide it the sorted streams/files to merge while making sure we can 
merge them in memory.
+/// In case we can't merge all of them in a single pass we will spill the 
intermediate results to disk
+/// and repeat the process.
+///
+/// ## High level Algorithm
+/// 1. Get the maximum amount of sorted in-memory streams and spill files we 
can merge with the available memory
+/// 2. Sort them to a sorted stream
+/// 3. Do we have more spill files to merge?
+///  - Yes: write that sorted stream to a spill file,
+///    add that spill file back to the spill files to merge and
+///    repeat the process
+///
+///  - No: return that sorted stream as the final output stream
+///
+/// ```text
+/// Initial State: Multiple sorted streams + spill files
+///      ┌───────────┐
+///      │  Phase 1  │
+///      └───────────┘
+/// ┌──Can hold in memory─┐
+/// │   ┌──────────────┐  │
+/// │   │  In-memory   │
+/// │   │sorted stream │──┼────────┐
+/// │   │      1       │  │        │
+///     └──────────────┘  │        │
+/// │   ┌──────────────┐  │        │
+/// │   │  In-memory   │           │
+/// │   │sorted stream │──┼────────┤
+/// │   │      2       │  │        │
+///     └──────────────┘  │        │
+/// │   ┌──────────────┐  │        │
+/// │   │  In-memory   │           │
+/// │   │sorted stream │──┼────────┤
+/// │   │      3       │  │        │
+///     └──────────────┘  │        │
+/// │   ┌──────────────┐  │        │            ┌───────────┐
+/// │   │ Sorted Spill │           │            │  Phase 2  │
+/// │   │    file 1    │──┼────────┤            └───────────┘
+/// │   └──────────────┘  │        │
+///  ──── ──── ──── ──── ─┘        │       ┌──Can hold in memory─┐
+///                                │       │                     │
+///     ┌──────────────┐           │       │   ┌──────────────┐
+///     │ Sorted Spill │           │       │   │ Sorted Spill │  │
+///     │    file 2    │──────────────────────▶│    file 2    │──┼─────┐
+///     └──────────────┘           │           └──────────────┘  │     │
+///     ┌──────────────┐           │       │   ┌──────────────┐  │     │
+///     │ Sorted Spill │           │       │   │ Sorted Spill │        │
+///     │    file 3    │──────────────────────▶│    file 3    │──┼─────┤
+///     └──────────────┘           │       │   └──────────────┘  │     │
+///     ┌──────────────┐           │           ┌──────────────┐  │     │
+///     │ Sorted Spill │           │       │   │ Sorted Spill │  │     │
+///     │    file 4    │──────────────────────▶│    file 4    │────────┤       
   ┌───────────┐
+///     └──────────────┘           │       │   └──────────────┘  │     │       
   │  Phase 3  │
+///                                │       │                     │     │       
   └───────────┘
+///                                │        ──── ──── ──── ──── ─┘     │     
┌──Can hold in memory─┐
+///                                │                                   │     │ 
                    │
+///     ┌──────────────┐           │           ┌──────────────┐        │     │ 
 ┌──────────────┐
+///     │ Sorted Spill │           │           │ Sorted Spill │        │     │ 
 │ Sorted Spill │   │
+///     │    file 5    │──────────────────────▶│    file 5    
│────────────────▶│    file 5    │───┼───┐
+///     └──────────────┘           │           └──────────────┘        │     │ 
 └──────────────┘   │   │
+///                                │                                   │     │ 
                    │   │
+///                                │           ┌──────────────┐        │     │ 
 ┌──────────────┐       │
+///                                │           │ Sorted Spill │        │     │ 
 │ Sorted Spill │   │   │       ┌── ─── ─── ─── ─── ─── ─── ──┐
+///                                └──────────▶│    file 6    
│────────────────▶│    file 6    │───┼───┼──────▶         Output Stream
+///                                            └──────────────┘        │     │ 
 └──────────────┘   │   │       └── ─── ─── ─── ─── ─── ─── ──┘
+///                                                                    │     │ 
                    │   │
+///                                                                    │     │ 
 ┌──────────────┐       │
+///                                                                    │     │ 
 │ Sorted Spill │   │   │
+///                                                                    
└───────▶│    file 7    │───┼───┘
+///                                                                          │ 
 └──────────────┘   │
+///                                                                          │ 
                    │
+///                                                                          
└─ ──── ──── ──── ────
+/// ```
+///
+/// ## Memory Management Strategy
+///
+/// This multi-level merge make sure that we can handle any amount of data to 
sort as long as
+/// we have enough memory to merge at least 2 streams at a time.
+///
+/// 1. **Worst-Case Memory Reservation**: Reserves memory based on the largest
+///    batch size encountered in each spill file to merge, ensuring sufficient 
memory is always
+///    available during merge operations.
+/// 2. **Adaptive Buffer Sizing**: Reduces buffer sizes when memory is 
constrained
+/// 3. **Spill-to-Disk**: Spill to disk when we cannot merge all files in 
memory
+///
+pub(crate) struct MultiLevelMergeBuilder {
+    spill_manager: SpillManager,
+    schema: SchemaRef,
+    sorted_spill_files: Vec<SortedSpillFile>,
+    sorted_streams: Vec<SendableRecordBatchStream>,
+    expr: LexOrdering,
+    metrics: BaselineMetrics,
+    batch_size: usize,
+    reservation: MemoryReservation,
+    fetch: Option<usize>,
+    enable_round_robin_tie_breaker: bool,
+}
+
+impl Debug for MultiLevelMergeBuilder {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "MultiLevelMergeBuilder")
+    }
+}
+
+impl MultiLevelMergeBuilder {
+    #[allow(clippy::too_many_arguments)]
+    pub(crate) fn new(
+        spill_manager: SpillManager,
+        schema: SchemaRef,
+        sorted_spill_files: Vec<SortedSpillFile>,
+        sorted_streams: Vec<SendableRecordBatchStream>,
+        expr: LexOrdering,
+        metrics: BaselineMetrics,
+        batch_size: usize,
+        reservation: MemoryReservation,
+        fetch: Option<usize>,
+        enable_round_robin_tie_breaker: bool,
+    ) -> Self {
+        Self {
+            spill_manager,
+            schema,
+            sorted_spill_files,
+            sorted_streams,
+            expr,
+            metrics,
+            batch_size,
+            reservation,
+            enable_round_robin_tie_breaker,
+            fetch,
+        }
+    }
+
+    pub(crate) fn create_spillable_merge_stream(self) -> 
SendableRecordBatchStream {
+        Box::pin(RecordBatchStreamAdapter::new(
+            Arc::clone(&self.schema),
+            futures::stream::once(self.create_stream()).try_flatten(),
+        ))
+    }
+
+    async fn create_stream(mut self) -> Result<SendableRecordBatchStream> {
+        loop {
+            let mut stream = self.merge_sorted_runs_within_mem_limit()?;
+
+            // TODO - add a threshold for number of files to disk even if 
empty and reading from disk so
+            //        we can avoid the memory reservation
+
+            // If no spill files are left, we can return the stream as this is 
the last sorted run
+            // TODO - We can write to disk before reading it back to avoid 
having multiple streams in memory
+            if self.sorted_spill_files.is_empty() {
+                assert!(
+                    self.sorted_streams.is_empty(),
+                    "We should not have any sorted streams left"
+                );

Review Comment:
   I think it's okay to keep those `assert`s that is very unlikely to fail, it 
can be useful during development.
   If they're in the hot path, we can use `debug_assert()` instead to reduce 
the runtime overhead.



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