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


##########
datafusion/physical-plan/src/spill/replayable_spill_input.rs:
##########
@@ -0,0 +1,372 @@
+// 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.
+
+//! Utility for replaying a one-shot input `RecourdBatchStream` through spill.
+//!
+//! See comments in [`ReplayableStreamSource`] for details.
+
+use std::pin::Pin;
+use std::sync::Arc;
+use std::sync::atomic::{AtomicU32, Ordering};
+use std::task::{Context, Poll};
+
+use arrow::datatypes::SchemaRef;
+use arrow::record_batch::RecordBatch;
+use datafusion_common::{Result, internal_err};
+use datafusion_execution::RecordBatchStream;
+use datafusion_execution::SendableRecordBatchStream;
+use datafusion_execution::disk_manager::RefCountedTempFile;
+use futures::Stream;
+use parking_lot::Mutex;
+
+use crate::EmptyRecordBatchStream;
+use crate::spill::in_progress_spill_file::InProgressSpillFile;
+use crate::spill::spill_manager::SpillManager;
+
+const FIRST_PASS_ACTIVE_EPOCH: u32 = 1;
+const POISONED_EPOCH: u32 = u32::MAX;
+
+/// Spill-backed replayable stream source.
+///
+/// [`ReplayableStreamSource`] is constructed from an input stream, usually 
produced
+/// by executing an input `ExecutionPlan`.
+///
+/// - On the first pass, it evaluates the input stream, produces 
`RecordBatch`es,
+///   caches those batches to a local spill file, and also forwards them to the
+///   output.
+/// - On subsequent passes, it reads directly from the spill file.
+///
+/// ```text
+/// first pass:
+///
+/// RecordBatch stream
+///     |
+///     v
+///   [batch] -> output
+///     |
+///     +----> spill file
+///
+///
+/// later passes:
+///
+/// spill file
+///     |
+///     v
+///   [batch] -> output
+/// ```
+///
+/// This is useful when an input stream must be replayed and:
+/// - Re-evaluation is expensive because the input stream may come from a long
+///   and complex pipeline.
+/// - The parent operator is under memory pressure and cannot cache the input 
in
+///   memory for replay.
+pub(crate) struct ReplayableStreamSource {
+    schema: SchemaRef,
+    input: Option<SendableRecordBatchStream>,
+    spill_manager: SpillManager,
+    request_description: String,
+    /// 0 = unopened, 1 = first pass active, 2 = replayable/empty, MAX = 
poisoned
+    /// on execution errors.
+    epoch: Arc<AtomicU32>,
+    spill_file: Arc<Mutex<Option<RefCountedTempFile>>>,
+}
+
+impl ReplayableStreamSource {
+    /// Creates a replayable stream producer over a one-shot input stream.
+    ///
+    /// It caches the input into a local spill file on the first pass, then
+    /// reads directly from that spill file on subsequent passes.
+    pub(crate) fn new(
+        input: SendableRecordBatchStream,
+        spill_manager: SpillManager,
+        request_description: impl Into<String>,
+    ) -> Self {
+        let schema = input.schema();
+        Self {
+            schema,
+            input: Some(input),
+            spill_manager,
+            request_description: request_description.into(),
+            epoch: Arc::new(AtomicU32::new(0)),
+            spill_file: Arc::new(Mutex::new(None)),
+        }
+    }
+
+    /// Opens the next pass over this input.
+    ///
+    /// The first call returns a stream that forwards upstream batches while
+    /// caching them to spill. Later calls return streams that read directly
+    /// from the completed spill file.
+    ///
+    /// # Note
+    /// Subsequent passes MUST be opened only after the initial pass is fully
+    /// consumed; otherwise, an error is returned.
+    pub(crate) fn open_pass(&mut self) -> Result<SendableRecordBatchStream> {
+        match self.epoch.load(Ordering::Relaxed) {
+            0 => {
+                let Some(input) = self.input.take() else {

Review Comment:
   Thank you for the review. I think in the previous implementation, the 
concurrency part is a bit hard to follow, so I refactored it to make the shared 
state controlled by a single mutex in 
[19494c2](https://github.com/apache/datafusion/pull/21742/commits/19494c245d8921967df57f9dadf309b07e4c65a0),
 hope this solves the problem.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to