alamb commented on code in PR #21882:
URL: https://github.com/apache/datafusion/pull/21882#discussion_r3204748777


##########
datafusion/execution/src/disk_manager.rs:
##########
@@ -89,19 +100,28 @@ impl DiskManagerBuilder {
                     max_temp_directory_size: self.max_temp_directory_size,
                     used_disk_space: Arc::new(AtomicU64::new(0)),
                     active_files_count: Arc::new(AtomicUsize::new(0)),
+                    custom_factory: None,
                 })
             }
             DiskManagerMode::Disabled => Ok(DiskManager {
                 local_dirs: Mutex::new(None),
                 max_temp_directory_size: self.max_temp_directory_size,
                 used_disk_space: Arc::new(AtomicU64::new(0)),
                 active_files_count: Arc::new(AtomicUsize::new(0)),
+                custom_factory: None,
+            }),
+            DiskManagerMode::Custom(factory) => Ok(DiskManager {
+                local_dirs: Mutex::new(None),
+                max_temp_directory_size: self.max_temp_directory_size,
+                used_disk_space: Arc::new(AtomicU64::new(0)),
+                active_files_count: Arc::new(AtomicUsize::new(0)),
+                custom_factory: Some(factory),
             }),
         }
     }
 }
 
-#[derive(Clone, Debug, Default)]
+#[derive(Clone, Default)]
 pub enum DiskManagerMode {

Review Comment:
   If we are going to permit a TempFileFactory here, I wonder if it would make 
sense (maybe as a follow on PR) here to move everything over
   
   So like
   
   ```rust
   pub enum DiskManagerMode {
      Custom(Arc<dyn TempFileFactory>),
   }
   ```
   
   And then implement a basic `DirectoriesTempFileFactor` and a 
`NoTempFilesFactory` that are provided along with datafusion
   
   That might simplify the code, and it would also ensure the `TempFileFactory` 
API is rich enough for the existing temp file strategies



##########
datafusion/execution/src/spill_file.rs:
##########
@@ -0,0 +1,71 @@
+// 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.
+
+use bytes::Bytes;
+use datafusion_common::Result;
+use futures::Stream;
+use std::path::Path;
+use std::pin::Pin;
+use std::sync::Arc;
+
+/// Abstraction over a spill file backend.
+/// Implementations handle their own quota enforcement and blocking concerns.
+pub trait SpillFile: Send + Sync {
+    /// Returns the OS path if this is a local file, None otherwise.
+    fn path(&self) -> Option<&Path> {
+        None
+    }
+
+    /// Returns current size in bytes if cheaply available.
+    fn size(&self) -> Option<u64>;
+
+    /// Returns file contents as an async stream of byte chunks.
+    fn read_stream(&self) -> Result<Pin<Box<dyn Stream<Item = Result<Bytes>> + 
Send>>>;
+
+    /// Opens a writer for appending data to this file.
+    fn open_writer(&self) -> Result<Box<dyn SpillWriter>>;
+
+    /// Opens a synchronous reader for this file.
+    /// Used by legacy operators (like SortMergeJoin) that haven't been fully 
migrated to async.
+    ///
+    /// Backends that only support async reads should leave this default 
implementation,
+    /// which will safely return a NotImplemented error if used in synchronous 
contexts.
+    fn open_sync_reader(&self) -> Result<Box<dyn std::io::Read + Send>> {
+        datafusion_common::exec_err!(
+            "Synchronous reads are not supported by this spill backend. \
+            This backend cannot be used with synchronous operators like 
SortMergeJoin \
+            until they are refactored to be fully asynchronous."
+        )
+    }
+}
+
+/// Writer for spill file backends.
+/// Receives zero-copy `Bytes` payloads from the IPCStreamWriter adapter.
+pub trait SpillWriter: Send {
+    fn write(&mut self, data: Bytes) -> Result<()>;

Review Comment:
   This is pretty similar to https://doc.rust-lang.org/std/io/trait.Write.html 
🤔 



##########
datafusion/execution/src/disk_manager.rs:
##########
@@ -473,6 +528,120 @@ fn create_local_dirs(local_dirs: &[PathBuf]) -> 
Result<Vec<Arc<TempDir>>> {
         .collect()
 }
 
+pub struct OsSpillWriter {

Review Comment:
   maybe "file spill writer"?



##########
datafusion/execution/src/disk_manager.rs:
##########
@@ -174,8 +207,20 @@ pub struct DiskManager {
     used_disk_space: Arc<AtomicU64>,
     /// Number of active temporary files created by this disk manager
     active_files_count: Arc<AtomicUsize>,
+    /// Custom Backend

Review Comment:
   A small nit: I think "custom" is a somewhat unecessary term here . Perhaps 
this 
   
   ```rust
       factory: Option<Arc<dyn TempFileFactory>>,
   ```
   
   or 
   ```rust
       temp_file_factory: Option<Arc<dyn TempFileFactory>>,
   ```
   
   would be more consistent with the rest of the codebase



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