Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
109 changes: 109 additions & 0 deletions datafusion/execution/src/disk_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -473,6 +473,115 @@ fn create_local_dirs(local_dirs: &[PathBuf]) -> Result<Vec<Arc<TempDir>>> {
.collect()
}

pub struct FileSpillWriter {
file: std::fs::File,
disk_manager: Arc<DiskManager>,
current_file_disk_usage: Arc<AtomicU64>,
}

impl SpillWriter for FileSpillWriter {
fn write(&mut self, data: Bytes) -> Result<()> {
use std::io::Write;

let len = data.len() as u64;
if len == 0 {
return Ok(());
}

self.file
.write_all(&data)
.map_err(DataFusionError::IoError)?;
self.current_file_disk_usage
.fetch_add(len, Ordering::Relaxed);
let new_global = self
.disk_manager
.used_disk_space
.fetch_add(len, Ordering::Relaxed)
+ len;

if new_global > self.disk_manager.max_temp_directory_size {
return resources_err!(
"The used disk space during the spilling process has exceeded the allowable limit of {}. \
Please try increasing the config: `datafusion.runtime.max_temp_directory_size`.",
human_readable_size(self.disk_manager.max_temp_directory_size as usize)
);
}

Ok(())
}

fn flush(&mut self) -> Result<()> {
use std::io::Write;
self.file.flush().map_err(DataFusionError::IoError)
}

fn finish(&mut self) -> Result<()> {
// flush() already called by SpillWriteAdapter before finish()
Ok(())
}
}

impl SpillFile for RefCountedTempFile {
fn path(&self) -> Option<&Path> {
Some(self.tempfile.path())
}

fn size(&self) -> Option<u64> {
Some(self.current_disk_usage())
}
#[cfg(not(target_arch = "wasm32"))]
fn read_stream(
&self,
) -> Result<std::pin::Pin<Box<dyn futures::Stream<Item = Result<Bytes>> + Send>>>
{
let path = self.path().to_owned();

let stream =
futures::stream::once(async move {
tokio::fs::File::open(&path)
.await
.map_err(DataFusionError::IoError)
})
.flat_map(
|open_result| -> std::pin::Pin<
Box<dyn futures::Stream<Item = Result<Bytes>> + Send>,
> {
match open_result {
Ok(file) => Box::pin(
tokio_util::io::ReaderStream::new(file)
.map(|r| r.map_err(DataFusionError::IoError)),
),
Err(e) => Box::pin(futures::stream::once(async move { Err(e) })),
}
},
);

Ok(Box::pin(stream))
}

#[cfg(target_arch = "wasm32")]
fn read_stream(
&self,
) -> Result<std::pin::Pin<Box<dyn futures::Stream<Item = Result<Bytes>> + Send>>>
{
datafusion_common::exec_err!(
"Default OS file spilling is not supported on WASM. Configure DiskManager with a Custom TempFileFactory."
)
}

fn open_writer(&self) -> Result<Box<dyn SpillWriter>> {
let file = self
.tempfile
.as_file()
.try_clone()
.map_err(DataFusionError::IoError)?;
Ok(Box::new(FileSpillWriter {
file,
disk_manager: Arc::clone(&self.disk_manager),
current_file_disk_usage: Arc::clone(&self.current_file_disk_usage),
}))
}
}
#[cfg(test)]
mod tests {
use super::*;
Expand Down
60 changes: 60 additions & 0 deletions datafusion/execution/src/spill_file.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
// 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>>;
}

/// 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<()>;
fn flush(&mut self) -> Result<()>;
/// Finalizes the write after all data has been flushed.
///
/// Implementations must not call `flush` internally.
/// Intended for close/sync/commit operations.
fn finish(&mut self) -> Result<()>;
}

/// Factory for creating spill files.
pub trait TempFileFactory:
Send + Sync + std::panic::UnwindSafe + std::panic::RefUnwindSafe
{
fn create_temp_file(&self, description: &str) -> Result<Arc<dyn SpillFile>>;
}
Loading