rluvaton commented on code in PR #15700:
URL: https://github.com/apache/datafusion/pull/15700#discussion_r2213137522


##########
datafusion/physical-plan/src/sorts/multi_level_merge.rs:
##########
@@ -0,0 +1,342 @@
+// 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::{
+    MemoryConsumer, MemoryPool, MemoryReservation, UnboundedMemoryPool,
+};
+
+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
+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,
+
+    // This is for avoiding double reservation of memory from our side and the 
sort preserving merge stream
+    // side.
+    // and doing a lot of code changes to avoid accounting for the memory used 
by the streams
+    unbounded_memory_pool: Arc<dyn MemoryPool>,
+}
+
+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,
+            unbounded_memory_pool: Arc::new(UnboundedMemoryPool::default()),
+        }
+    }
+
+    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 {
+            // Hold this for the lifetime of the stream
+            let mut current_memory_reservation = self.reservation.new_empty();
+            let mut stream =
+                self.create_sorted_stream(&mut current_memory_reservation)?;
+
+            // 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() {
+                // Attach the memory reservation to the stream as we are done 
with it
+                // but because we replaced the memory reservation of the merge 
stream, we must hold
+                // this to make sure we have enough memory
+                return Ok(Box::pin(StreamAttachedReservation::new(
+                    stream,
+                    current_memory_reservation,
+                )));
+            }

Review Comment:
   Never mind, this can't happen as we take the largest record batch * merge 
degree



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