From 69ab5d22cf9d0299a9c66161f29bf041194ed65e Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 21 Oct 2025 11:57:59 -0500 Subject: [PATCH 01/44] implement SpillingPool --- datafusion/common/src/config.rs | 14 + .../physical-plan/src/repartition/mod.rs | 160 ++--- datafusion/physical-plan/src/spill/mod.rs | 1 + .../physical-plan/src/spill/spill_pool.rs | 621 ++++++++++++++++++ 4 files changed, 703 insertions(+), 93 deletions(-) create mode 100644 datafusion/physical-plan/src/spill/spill_pool.rs diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index bc321b227ee52..7953493d787e0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -518,6 +518,20 @@ config_namespace! { /// batches and merged. pub sort_in_place_threshold_bytes: usize, default = 1024 * 1024 + /// Maximum size in bytes for individual spill files before rotating to a new file. + /// + /// When operators spill data to disk (e.g., RepartitionExec, SortExec), they write + /// multiple batches to the same file until this size limit is reached, then rotate + /// to a new file. This reduces syscall overhead compared to one-file-per-batch + /// while preventing files from growing too large. + /// + /// A larger value reduces file creation overhead but may hold more disk space. + /// A smaller value creates more files but allows finer-grained space reclamation + /// (especially in LIFO mode where files are truncated after reading). + /// + /// Default: 100 MB + pub max_spill_file_size_bytes: usize, default = 100 * 1024 * 1024 + /// Number of files to read in parallel when inferring schema and statistics pub meta_fetch_concurrency: usize, default = 32 diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 8174f71c31afa..d14ba4b488a82 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -39,6 +39,7 @@ use crate::repartition::distributor_channels::{ }; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; +use crate::spill::spill_pool::SpillPool; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -51,7 +52,6 @@ use datafusion_common::utils::transpose; use datafusion_common::{internal_err, ColumnStatistics, HashMap}; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; -use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; @@ -73,13 +73,9 @@ mod distributor_channels; enum RepartitionBatch { /// Batch held in memory (counts against memory reservation) Memory(RecordBatch), - /// Batch spilled to disk (one file per batch for queue semantics) - /// File automatically deleted when dropped via reference counting - /// The size field stores the original batch size for validation when reading back - Spilled { - spill_file: RefCountedTempFile, - size: usize, - }, + /// Marker indicating a batch was spilled to the partition's SpillPool + /// The actual batch can be retrieved via SpillPool::pop_batch() + Spilled, } type MaybeBatch = Option>; @@ -87,7 +83,6 @@ type InputPartitionsToCurrentPartitionSender = Vec>; /// Channels and resources for a single output partition -#[derive(Debug)] struct PartitionChannels { /// Senders for each input partition to send data to this output partition tx: InputPartitionsToCurrentPartitionSender, @@ -95,11 +90,11 @@ struct PartitionChannels { rx: InputPartitionsToCurrentPartitionReceiver, /// Memory reservation for this output partition reservation: SharedMemoryReservation, - /// Spill manager for handling disk spills for this output partition - spill_manager: Arc, + /// SpillPool for batched spilling with file handle reuse (FIFO semantics) + /// Wrapped in Arc so it can be shared between input tasks and output streams + spill_pool: Arc>, } -#[derive(Debug)] struct ConsumingInputStreamsState { /// Channels for sending batches from input partitions to output partitions. /// Key is the partition number. @@ -109,6 +104,15 @@ struct ConsumingInputStreamsState { abort_helper: Arc>>, } +impl Debug for ConsumingInputStreamsState { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("ConsumingInputStreamsState") + .field("num_channels", &self.channels.len()) + .field("abort_helper", &self.abort_helper) + .finish() + } +} + /// Inner state of [`RepartitionExec`]. enum RepartitionExecState { /// Not initialized yet. This is the default state stored in the RepartitionExec node @@ -236,13 +240,22 @@ impl RepartitionExecState { spill_metrics, input.schema(), )); + + // Create SpillPool with configured max file size + let max_file_size = context.session_config().options().execution.max_spill_file_size_bytes; + let spill_pool = SpillPool::new( + max_file_size, + Arc::clone(&spill_manager), + input.schema(), + ); + channels.insert( partition, PartitionChannels { tx, rx, reservation, - spill_manager, + spill_pool: Arc::new(Mutex::new(spill_pool)), }, ); } @@ -260,7 +273,7 @@ impl RepartitionExecState { ( channels.tx[i].clone(), Arc::clone(&channels.reservation), - Arc::clone(&channels.spill_manager), + Arc::clone(&channels.spill_pool), ), ) }) @@ -721,7 +734,7 @@ impl ExecutionPlan for RepartitionExec { let num_input_partitions = input.output_partitioning().partition_count(); // lock scope - let (mut rx, reservation, spill_manager, abort_helper) = { + let (mut rx, reservation, spill_pool, abort_helper) = { // lock mutexes let mut state = state.lock(); let state = state.consume_input_streams( @@ -738,7 +751,7 @@ impl ExecutionPlan for RepartitionExec { let PartitionChannels { rx, reservation, - spill_manager, + spill_pool, .. } = state .channels @@ -748,7 +761,7 @@ impl ExecutionPlan for RepartitionExec { ( rx, reservation, - spill_manager, + spill_pool, Arc::clone(&state.abort_helper), ) }; @@ -759,6 +772,7 @@ impl ExecutionPlan for RepartitionExec { if preserve_order { // Store streams from all the input partitions: + // All streams share the same SpillPool from PartitionChannels let input_streams = rx .into_iter() .map(|receiver| { @@ -767,7 +781,7 @@ impl ExecutionPlan for RepartitionExec { receiver, _drop_helper: Arc::clone(&abort_helper), reservation: Arc::clone(&reservation), - spill_manager: Arc::clone(&spill_manager), + spill_pool: Arc::clone(&spill_pool), state: RepartitionStreamState::ReceivingFromChannel, }) as SendableRecordBatchStream }) @@ -797,7 +811,7 @@ impl ExecutionPlan for RepartitionExec { input: rx.swap_remove(0), _drop_helper: abort_helper, reservation, - spill_manager, + spill_pool, state: RepartitionStreamState::ReceivingFromChannel, }) as SendableRecordBatchStream) } @@ -1042,7 +1056,7 @@ impl RepartitionExec { ( DistributionSender, SharedMemoryReservation, - Arc, + Arc>, ), >, partitioning: Partitioning, @@ -1076,7 +1090,7 @@ impl RepartitionExec { let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it - if let Some((tx, reservation, spill_manager)) = + if let Some((tx, reservation, spill_pool)) = output_channels.get_mut(&partition) { let (batch_to_send, is_memory_batch) = @@ -1086,19 +1100,12 @@ impl RepartitionExec { (RepartitionBatch::Memory(batch), true) } Err(_) => { - // We're memory limited - spill this single batch to its own file - let spill_file = spill_manager - .spill_record_batch_and_finish( - &[batch], - &format!( - "RepartitionExec spill partition {partition}" - ), - )? - // Note that we handled empty batch above, so this is safe - .expect("non-empty batch should produce spill file"); - - // Store size for validation when reading back - (RepartitionBatch::Spilled { spill_file, size }, false) + // We're memory limited - spill to SpillPool + // SpillPool handles file handle reuse and rotation + spill_pool.lock().push_batch(&batch)?; + + // Send marker indicating batch was spilled + (RepartitionBatch::Spilled, false) } }; @@ -1191,8 +1198,6 @@ impl RepartitionExec { enum RepartitionStreamState { /// Waiting for next item from channel ReceivingFromChannel, - /// Reading a spilled batch from disk (stream reads via tokio::fs) - ReadingSpilledBatch(SendableRecordBatchStream), } struct RepartitionStream { @@ -1214,8 +1219,8 @@ struct RepartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, - /// Spill manager for reading spilled batches - spill_manager: Arc, + /// SpillPool for batched spilling with FIFO semantics + spill_pool: Arc>, /// Current state of the stream state: RepartitionStreamState, @@ -1241,15 +1246,17 @@ impl Stream for RepartitionStream { .shrink(batch.get_array_memory_size()); return Poll::Ready(Some(Ok(batch))); } - Ok(RepartitionBatch::Spilled { spill_file, size }) => { - // Read from disk - SpillReaderStream uses tokio::fs internally - // Pass the original size for validation - let stream = self - .spill_manager - .read_spill_as_stream(spill_file, Some(size))?; - self.state = - RepartitionStreamState::ReadingSpilledBatch(stream); - // Continue loop to poll the stream immediately + Ok(RepartitionBatch::Spilled) => { + // Read from SpillPool (FIFO order) + match self.spill_pool.lock().pop_batch()? { + Some(batch) => { + return Poll::Ready(Some(Ok(batch))); + } + None => { + // No spilled batches available, continue receiving + continue; + } + } } Err(e) => { return Poll::Ready(Some(Err(e))); @@ -1273,23 +1280,6 @@ impl Stream for RepartitionStream { } } } - RepartitionStreamState::ReadingSpilledBatch(stream) => { - match futures::ready!(stream.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - // Return batch and stay in ReadingSpilledBatch state to read more batches - return Poll::Ready(Some(Ok(batch))); - } - Some(Err(e)) => { - self.state = RepartitionStreamState::ReceivingFromChannel; - return Poll::Ready(Some(Err(e))); - } - None => { - // Spill stream ended - go back to receiving from channel - self.state = RepartitionStreamState::ReceivingFromChannel; - continue; - } - } - } } } } @@ -1317,8 +1307,8 @@ struct PerPartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, - /// Spill manager for reading spilled batches - spill_manager: Arc, + /// SpillPool for batched spilling with FIFO semantics (shared across streams) + spill_pool: Arc>, /// Current state of the stream state: RepartitionStreamState, @@ -1344,15 +1334,17 @@ impl Stream for PerPartitionStream { .shrink(batch.get_array_memory_size()); return Poll::Ready(Some(Ok(batch))); } - Ok(RepartitionBatch::Spilled { spill_file, size }) => { - // Read from disk - SpillReaderStream uses tokio::fs internally - // Pass the original size for validation - let stream = self - .spill_manager - .read_spill_as_stream(spill_file, Some(size))?; - self.state = - RepartitionStreamState::ReadingSpilledBatch(stream); - // Continue loop to poll the stream immediately + Ok(RepartitionBatch::Spilled) => { + // Read from SpillPool (FIFO order) + match self.spill_pool.lock().pop_batch()? { + Some(batch) => { + return Poll::Ready(Some(Ok(batch))); + } + None => { + // No spilled batches available, continue receiving + continue; + } + } } Err(e) => { return Poll::Ready(Some(Err(e))); @@ -1365,24 +1357,6 @@ impl Stream for PerPartitionStream { None => return Poll::Ready(None), } } - - RepartitionStreamState::ReadingSpilledBatch(stream) => { - match futures::ready!(stream.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - // Return batch and stay in ReadingSpilledBatch state to read more batches - return Poll::Ready(Some(Ok(batch))); - } - Some(Err(e)) => { - self.state = RepartitionStreamState::ReceivingFromChannel; - return Poll::Ready(Some(Err(e))); - } - None => { - // Spill stream ended - go back to receiving from channel - self.state = RepartitionStreamState::ReceivingFromChannel; - continue; - } - } - } } } } diff --git a/datafusion/physical-plan/src/spill/mod.rs b/datafusion/physical-plan/src/spill/mod.rs index 5b9a91e781b16..a1a7bbee05c8a 100644 --- a/datafusion/physical-plan/src/spill/mod.rs +++ b/datafusion/physical-plan/src/spill/mod.rs @@ -19,6 +19,7 @@ pub(crate) mod in_progress_spill_file; pub(crate) mod spill_manager; +pub mod spill_pool; use std::fs::File; use std::io::BufReader; diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs new file mode 100644 index 0000000000000..9557535ef797f --- /dev/null +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -0,0 +1,621 @@ +// 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. + +//! SpillPool: A reusable abstraction for managing spill files with FIFO semantics. +//! +//! # Overview +//! +//! The `SpillPool` provides a centralized mechanism for spilling record batches to disk +//! when memory is constrained. It manages a collection of spill files, each containing +//! multiple batches, with configurable maximum file sizes. +//! +//! # Design +//! +//! - **FIFO (Queue) semantics**: Batches are read in the order they were spilled +//! - **File handle reuse**: Multiple batches are written to the same file to minimize syscalls +//! - **Automatic file rotation**: When a file exceeds `max_file_size_bytes`, rotate to a new file +//! - **Sequential reading**: Uses IPC Stream format's natural sequential access pattern +//! - **Automatic cleanup**: Files are deleted once fully consumed +//! +//! # Usage Example +//! +//! ```ignore +//! let pool = SpillPool::new( +//! 100 * 1024 * 1024, // 100MB max per file +//! spill_manager, +//! schema, +//! ); +//! +//! // Spill batches - automatically rotates files when size limit reached +//! pool.push_batch(batch1)?; +//! pool.push_batch(batch2)?; +//! pool.flush()?; // Finalize current file +//! +//! // Read back in FIFO order +//! let batch = pool.pop_batch()?.unwrap(); // Returns batch1 +//! let batch = pool.pop_batch()?.unwrap(); // Returns batch2 +//! ``` + +use std::collections::VecDeque; +use std::fs::File; +use std::io::BufReader; +use std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use arrow::ipc::reader::StreamReader; +use arrow::record_batch::RecordBatch; +use datafusion_common::{exec_datafusion_err, Result}; +use datafusion_execution::disk_manager::RefCountedTempFile; + +use super::in_progress_spill_file::InProgressSpillFile; +use super::spill_manager::SpillManager; + +/// A single spill file containing one or more record batches. +struct SpillFile { + /// The temp file handle (auto-deletes when dropped) + file: RefCountedTempFile, + /// Number of batches originally written to this file + total_batches: usize, + /// Number of batches already read from this file + batches_read: usize, + /// Total size of this file in bytes (kept for potential debugging/metrics) + #[allow(dead_code)] + total_size: usize, + /// Sequential reader for this file (lazily initialized on first read) + reader: Option>>, +} + +impl SpillFile { + fn new(file: RefCountedTempFile, total_batches: usize, total_size: usize) -> Self { + Self { + file, + total_batches, + batches_read: 0, + total_size, + reader: None, + } + } + + /// Returns true if all batches have been read from this file + fn is_fully_consumed(&self) -> bool { + self.batches_read >= self.total_batches + } + + /// Returns the number of unread batches remaining + fn remaining_batches(&self) -> usize { + self.total_batches.saturating_sub(self.batches_read) + } + + /// Reads the next batch from this file sequentially. + /// + /// Initializes the reader on first call. Returns None when all batches consumed. + fn read_next_batch(&mut self) -> Result> { + if self.is_fully_consumed() { + return Ok(None); + } + + // Initialize reader on first use + if self.reader.is_none() { + let file_handle = File::open(self.file.path()).map_err(|e| { + exec_datafusion_err!( + "Failed to open spill file {:?} for reading: {}", + self.file.path(), + e + ) + })?; + let buf_reader = BufReader::new(file_handle); + // SAFETY: DataFusion's spill writer strictly follows Arrow IPC specifications + let reader = unsafe { + StreamReader::try_new(buf_reader, None)?.with_skip_validation(true) + }; + self.reader = Some(reader); + } + + // Read next batch from sequential stream + if let Some(reader) = &mut self.reader { + match reader.next() { + Some(Ok(batch)) => { + self.batches_read += 1; + Ok(Some(batch)) + } + Some(Err(e)) => Err(e.into()), + None => { + // Stream ended - this shouldn't happen if batch count is correct + if !self.is_fully_consumed() { + return Err(exec_datafusion_err!( + "Unexpected end of spill file: read {} batches but expected {}", + self.batches_read, + self.total_batches + )); + } + Ok(None) + } + } + } else { + unreachable!("Reader should be initialized above") + } + } +} + +/// A pool of spill files that manages batch-level spilling with FIFO semantics. +/// +/// Batches are written sequentially to files, with automatic rotation when the +/// configured size limit is reached. Reading is done in FIFO order (oldest batch first). +/// +/// # Thread Safety +/// +/// `SpillPool` is not thread-safe and should be used from a single thread or +/// protected with appropriate synchronization. +pub struct SpillPool { + /// Maximum size in bytes before rotating to a new file + max_file_size_bytes: usize, + /// Queue of spill files (front = oldest, back = newest) + files: VecDeque, + /// Current file being written to (if any) + current_write_file: Option, + /// Size of current write file in bytes (estimated) + current_write_size: usize, + /// Number of batches written to current file + current_batch_count: usize, + /// SpillManager for creating files and tracking metrics + spill_manager: Arc, + /// Schema for batches (kept for potential validation in debug builds) + #[allow(dead_code)] + schema: SchemaRef, +} + +impl SpillPool { + /// Creates a new SpillPool with FIFO semantics. + /// + /// # Arguments + /// + /// * `max_file_size_bytes` - Maximum size per file before rotation (e.g., 100MB) + /// * `spill_manager` - Manager for file creation and metrics + /// * `schema` - Schema for record batches + pub fn new( + max_file_size_bytes: usize, + spill_manager: Arc, + schema: SchemaRef, + ) -> Self { + Self { + max_file_size_bytes, + files: VecDeque::new(), + current_write_file: None, + current_write_size: 0, + current_batch_count: 0, + spill_manager, + schema, + } + } + + /// Returns the number of files currently in the pool + pub fn file_count(&self) -> usize { + self.files.len() + if self.current_write_file.is_some() { 1 } else { 0 } + } + + /// Returns the total number of unread batches across all files + pub fn batch_count(&self) -> usize { + self.files.iter().map(|f| f.remaining_batches()).sum::() + + self.current_batch_count + } + + /// Returns true if the pool is empty (no batches to read) + pub fn is_empty(&self) -> bool { + self.batch_count() == 0 + } + + /// Spills a batch to the pool, rotating files when necessary. + /// + /// If the current file would exceed `max_file_size_bytes` after adding + /// this batch, the file is finalized and a new one is started. + /// + /// # Errors + /// + /// Returns an error if disk I/O fails or disk quota is exceeded. + pub fn push_batch(&mut self, batch: &RecordBatch) -> Result<()> { + if batch.num_rows() == 0 { + // Skip empty batches + return Ok(()); + } + + let batch_size = batch.get_array_memory_size(); + + // Check if we need to rotate to a new file + let needs_rotation = if self.current_write_file.is_some() { + // Rotate if adding this batch would exceed the max file size + self.current_write_size + batch_size > self.max_file_size_bytes + } else { + // No current file, need to create one + true + }; + + if needs_rotation && self.current_write_file.is_some() { + // Finish current file and add to queue + self.finish_current_file()?; + } + + // Create new file if needed + if self.current_write_file.is_none() { + self.current_write_file = + Some(self.spill_manager.create_in_progress_file("SpillPool")?); + self.current_write_size = 0; + self.current_batch_count = 0; + } + + // Append batch to current file + if let Some(ref mut file) = self.current_write_file { + file.append_batch(batch)?; + } + + self.current_write_size += batch_size; + self.current_batch_count += 1; + + Ok(()) + } + + /// Reads the next batch from the pool in FIFO order. + /// + /// Returns the oldest unread batch, or None if the pool is empty. + /// + /// # Errors + /// + /// Returns an error if disk I/O fails during read. + pub fn pop_batch(&mut self) -> Result> { + // Ensure any pending writes are flushed first + if self.current_write_file.is_some() { + self.flush()?; + } + + loop { + // Get the oldest file (front of queue) + let spill_file = match self.files.front_mut() { + Some(file) => file, + None => return Ok(None), // No files available + }; + + // Try to read next batch from this file + match spill_file.read_next_batch()? { + Some(batch) => { + // Check if file is now fully consumed after reading this batch + let is_consumed = spill_file.is_fully_consumed(); + if is_consumed { + // Remove the file from the queue + self.files.pop_front(); + } + return Ok(Some(batch)); + } + None => { + // File is fully consumed, remove it and try next file + self.files.pop_front(); + continue; + } + } + } + } + + /// Finalizes the current write file and adds it to the files queue. + /// + /// Called automatically by `push_batch` when rotating files, but can + /// also be called explicitly to ensure all pending data is flushed. + pub fn flush(&mut self) -> Result<()> { + if self.current_write_file.is_some() { + self.finish_current_file()?; + } + Ok(()) + } + + // Private helper methods + + /// Finishes the current write file and moves it to the files queue. + fn finish_current_file(&mut self) -> Result<()> { + if let Some(mut file) = self.current_write_file.take() { + // Finish writing to get the final file + let finished_file = file.finish()?; + + if let Some(temp_file) = finished_file { + // Get actual file size + let actual_size = temp_file.current_disk_usage() as usize; + + // Create SpillFile and add to queue + let spill_file = + SpillFile::new(temp_file, self.current_batch_count, actual_size); + self.files.push_back(spill_file); + } + + // Reset write state + self.current_write_size = 0; + self.current_batch_count = 0; + } + + Ok(()) + } +} + +impl Drop for SpillPool { + fn drop(&mut self) { + // Flush any pending writes to ensure metrics are accurate + // We ignore errors here since Drop doesn't allow returning errors + let _ = self.flush(); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; + use arrow::array::{ArrayRef, Int32Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_execution::runtime_env::RuntimeEnv; + + fn create_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ])) + } + + fn create_test_batch(start: i32, count: usize) -> RecordBatch { + let schema = create_test_schema(); + let a: ArrayRef = Arc::new(Int32Array::from( + (start..start + count as i32).collect::>(), + )); + let b: ArrayRef = Arc::new(Int32Array::from( + (start * 10..start * 10 + count as i32 * 10) + .step_by(10) + .collect::>(), + )); + RecordBatch::try_new(schema, vec![a, b]).unwrap() + } + + fn create_spill_pool(max_file_size: usize) -> SpillPool { + let env = Arc::new(RuntimeEnv::default()); + let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let schema = create_test_schema(); + let spill_manager = Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); + + SpillPool::new(max_file_size, spill_manager, schema) + } + + #[test] + fn test_empty_pool() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + assert!(pool.is_empty()); + assert_eq!(pool.file_count(), 0); + assert_eq!(pool.batch_count(), 0); + assert!(pool.pop_batch()?.is_none()); + + Ok(()) + } + + #[test] + fn test_single_batch() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Push one batch + let batch1 = create_test_batch(0, 10); + pool.push_batch(&batch1)?; + pool.flush()?; + + assert!(!pool.is_empty()); + assert_eq!(pool.file_count(), 1); + assert_eq!(pool.batch_count(), 1); + + // Pop and verify + let result = pool.pop_batch()?.unwrap(); + assert_eq!(result.num_rows(), 10); + assert_eq!(result.num_columns(), 2); + + // Pool should be empty now + assert!(pool.is_empty()); + assert_eq!(pool.file_count(), 0); + + Ok(()) + } + + #[test] + fn test_multiple_batches_single_file() -> Result<()> { + let mut pool = create_spill_pool(10 * 1024 * 1024); // Large file size + + // Push multiple batches + for i in 0..5 { + let batch = create_test_batch(i * 10, 10); + pool.push_batch(&batch)?; + } + pool.flush()?; + + assert_eq!(pool.file_count(), 1); + assert_eq!(pool.batch_count(), 5); + + // Pop in FIFO order + for i in 0..5 { + let result = pool.pop_batch()?.unwrap(); + assert_eq!(result.num_rows(), 10); + + // Verify first value is correct (FIFO order) + let col_a = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), i * 10); + } + + assert!(pool.is_empty()); + Ok(()) + } + + #[test] + fn test_file_rotation() -> Result<()> { + // Small file size to force rotation + let mut pool = create_spill_pool(500); // ~500 bytes + + // Push multiple batches - should create multiple files + for i in 0..10 { + let batch = create_test_batch(i * 5, 5); + pool.push_batch(&batch)?; + } + pool.flush()?; + + // Should have multiple files due to size limit + assert!(pool.file_count() > 1, "Expected file rotation to occur"); + assert_eq!(pool.batch_count(), 10); + + // Pop all batches in FIFO order + for i in 0..10 { + let result = pool.pop_batch()?.unwrap(); + let col_a = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), i * 5); + } + + assert!(pool.is_empty()); + Ok(()) + } + + #[test] + fn test_empty_batch_skipped() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + let batch1 = create_test_batch(0, 10); + let empty_batch = RecordBatch::new_empty(create_test_schema()); + let batch2 = create_test_batch(10, 10); + + pool.push_batch(&batch1)?; + pool.push_batch(&empty_batch)?; // Should be skipped + pool.push_batch(&batch2)?; + pool.flush()?; + + // Should only have 2 batches (empty one skipped) + assert_eq!(pool.batch_count(), 2); + + Ok(()) + } + + #[test] + fn test_interleaved_push_pop() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Push batch 0 + pool.push_batch(&create_test_batch(0, 5))?; + pool.flush()?; + + // Pop batch 0 + let result = pool.pop_batch()?.unwrap(); + assert_eq!(result.num_rows(), 5); + + // Push batches 1, 2 + pool.push_batch(&create_test_batch(10, 5))?; + pool.push_batch(&create_test_batch(20, 5))?; + pool.flush()?; + + // Pop batch 1 + let result = pool.pop_batch()?.unwrap(); + let col_a = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), 10); + + // Push batch 3 + pool.push_batch(&create_test_batch(30, 5))?; + pool.flush()?; + + // Pop remaining: should get 2, 3 + let result = pool.pop_batch()?.unwrap(); + let col_a = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), 20); + + let result = pool.pop_batch()?.unwrap(); + let col_a = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), 30); + + assert!(pool.is_empty()); + Ok(()) + } + + #[test] + fn test_auto_flush_on_pop() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Push without explicit flush + pool.push_batch(&create_test_batch(0, 10))?; + + // Pop should auto-flush + let result = pool.pop_batch()?.unwrap(); + assert_eq!(result.num_rows(), 10); + + Ok(()) + } + + #[test] + fn test_large_batches() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Create larger batches + let batch1 = create_test_batch(0, 1000); + let batch2 = create_test_batch(1000, 1000); + + pool.push_batch(&batch1)?; + pool.push_batch(&batch2)?; + pool.flush()?; + + let result1 = pool.pop_batch()?.unwrap(); + assert_eq!(result1.num_rows(), 1000); + + let result2 = pool.pop_batch()?.unwrap(); + assert_eq!(result2.num_rows(), 1000); + + Ok(()) + } + + #[test] + fn test_file_cleanup() -> Result<()> { + let mut pool = create_spill_pool(500); + + // Create multiple files + for i in 0..10 { + pool.push_batch(&create_test_batch(i * 5, 5))?; + } + pool.flush()?; + + let initial_file_count = pool.file_count(); + assert!(initial_file_count > 1); + + // Pop some batches - should cleanup fully consumed files + for _ in 0..5 { + pool.pop_batch()?; + } + + // File count should decrease as old files are consumed + assert!(pool.file_count() <= initial_file_count); + + Ok(()) + } +} From 2c9bb584bc6e5614fc8db98263bffb74799419c0 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 21 Oct 2025 12:30:49 -0500 Subject: [PATCH 02/44] clippy --- .../physical-plan/src/repartition/mod.rs | 63 +++++++++---------- .../physical-plan/src/spill/spill_pool.rs | 15 ++++- .../test_files/information_schema.slt | 2 + 3 files changed, 42 insertions(+), 38 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d14ba4b488a82..9e6e3d1d42920 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -82,6 +82,14 @@ type MaybeBatch = Option>; type InputPartitionsToCurrentPartitionSender = Vec>; type InputPartitionsToCurrentPartitionReceiver = Vec>; +/// Output channel with its associated memory reservation and spill pool +#[derive(Clone)] +struct OutputChannel { + sender: DistributionSender, + reservation: SharedMemoryReservation, + spill_pool: Arc>, +} + /// Channels and resources for a single output partition struct PartitionChannels { /// Senders for each input partition to send data to this output partition @@ -242,12 +250,13 @@ impl RepartitionExecState { )); // Create SpillPool with configured max file size - let max_file_size = context.session_config().options().execution.max_spill_file_size_bytes; - let spill_pool = SpillPool::new( - max_file_size, - Arc::clone(&spill_manager), - input.schema(), - ); + let max_file_size = context + .session_config() + .options() + .execution + .max_spill_file_size_bytes; + let spill_pool = + SpillPool::new(max_file_size, Arc::clone(&spill_manager), input.schema()); channels.insert( partition, @@ -270,11 +279,11 @@ impl RepartitionExecState { .map(|(partition, channels)| { ( *partition, - ( - channels.tx[i].clone(), - Arc::clone(&channels.reservation), - Arc::clone(&channels.spill_pool), - ), + OutputChannel { + sender: channels.tx[i].clone(), + reservation: Arc::clone(&channels.reservation), + spill_pool: Arc::clone(&channels.spill_pool), + }, ) }) .collect(); @@ -291,9 +300,7 @@ impl RepartitionExecState { let wait_for_task = SpawnedTask::spawn(RepartitionExec::wait_for_task( input_task, txs.into_iter() - .map(|(partition, (tx, _reservation, _spill_manager))| { - (partition, tx) - }) + .map(|(partition, channel)| (partition, channel.sender)) .collect(), )); spawned_tasks.push(wait_for_task); @@ -758,12 +765,7 @@ impl ExecutionPlan for RepartitionExec { .remove(&partition) .expect("partition not used yet"); - ( - rx, - reservation, - spill_pool, - Arc::clone(&state.abort_helper), - ) + (rx, reservation, spill_pool, Arc::clone(&state.abort_helper)) }; trace!( @@ -1051,14 +1053,7 @@ impl RepartitionExec { /// txs hold the output sending channels for each output partition async fn pull_from_input( mut stream: SendableRecordBatchStream, - mut output_channels: HashMap< - usize, - ( - DistributionSender, - SharedMemoryReservation, - Arc>, - ), - >, + mut output_channels: HashMap, partitioning: Partitioning, metrics: RepartitionMetrics, ) -> Result<()> { @@ -1090,11 +1085,9 @@ impl RepartitionExec { let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it - if let Some((tx, reservation, spill_pool)) = - output_channels.get_mut(&partition) - { + if let Some(channel) = output_channels.get_mut(&partition) { let (batch_to_send, is_memory_batch) = - match reservation.lock().try_grow(size) { + match channel.reservation.lock().try_grow(size) { Ok(_) => { // Memory available - send in-memory batch (RepartitionBatch::Memory(batch), true) @@ -1102,18 +1095,18 @@ impl RepartitionExec { Err(_) => { // We're memory limited - spill to SpillPool // SpillPool handles file handle reuse and rotation - spill_pool.lock().push_batch(&batch)?; + channel.spill_pool.lock().push_batch(&batch)?; // Send marker indicating batch was spilled (RepartitionBatch::Spilled, false) } }; - if tx.send(Some(Ok(batch_to_send))).await.is_err() { + if channel.sender.send(Some(Ok(batch_to_send))).await.is_err() { // If the other end has hung up, it was an early shutdown (e.g. LIMIT) // Only shrink memory if it was a memory batch if is_memory_batch { - reservation.lock().shrink(size); + channel.reservation.lock().shrink(size); } output_channels.remove(&partition); } diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 9557535ef797f..ded66bba84c10 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -204,12 +204,20 @@ impl SpillPool { /// Returns the number of files currently in the pool pub fn file_count(&self) -> usize { - self.files.len() + if self.current_write_file.is_some() { 1 } else { 0 } + self.files.len() + + if self.current_write_file.is_some() { + 1 + } else { + 0 + } } /// Returns the total number of unread batches across all files pub fn batch_count(&self) -> usize { - self.files.iter().map(|f| f.remaining_batches()).sum::() + self.files + .iter() + .map(|f| f.remaining_batches()) + .sum::() + self.current_batch_count } @@ -385,7 +393,8 @@ mod tests { let env = Arc::new(RuntimeEnv::default()); let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); let schema = create_test_schema(); - let spill_manager = Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); + let spill_manager = + Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); SpillPool::new(max_file_size, spill_manager, schema) } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index c674057151492..8aeac175620a5 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -223,6 +223,7 @@ datafusion.execution.keep_partition_by_columns false datafusion.execution.listing_table_factory_infer_partitions true datafusion.execution.listing_table_ignore_subdirectory true datafusion.execution.max_buffered_batches_per_output_file 2 +datafusion.execution.max_spill_file_size_bytes 104857600 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 datafusion.execution.objectstore_writer_buffer_size 10485760 @@ -343,6 +344,7 @@ datafusion.execution.keep_partition_by_columns false Should DataFusion keep the datafusion.execution.listing_table_factory_infer_partitions true Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). datafusion.execution.listing_table_ignore_subdirectory true Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption +datafusion.execution.max_spill_file_size_bytes 104857600 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec, SortExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation (especially in LIFO mode where files are truncated after reading). Default: 100 MB datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.objectstore_writer_buffer_size 10485760 Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. From bfabc5eab8e7c653ad18c538d002aebe2e2be3b9 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 21 Oct 2025 14:07:20 -0500 Subject: [PATCH 03/44] use buffered stream --- .../physical-plan/src/repartition/mod.rs | 122 ++++++++++++++++-- .../physical-plan/src/spill/spill_pool.rs | 21 +++ 2 files changed, 131 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 9e6e3d1d42920..2cf07dc1e5484 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -101,6 +101,8 @@ struct PartitionChannels { /// SpillPool for batched spilling with file handle reuse (FIFO semantics) /// Wrapped in Arc so it can be shared between input tasks and output streams spill_pool: Arc>, + /// SpillManager for creating streams from spill files + spill_manager: Arc, } struct ConsumingInputStreamsState { @@ -265,6 +267,7 @@ impl RepartitionExecState { rx, reservation, spill_pool: Arc::new(Mutex::new(spill_pool)), + spill_manager, }, ); } @@ -741,7 +744,7 @@ impl ExecutionPlan for RepartitionExec { let num_input_partitions = input.output_partitioning().partition_count(); // lock scope - let (mut rx, reservation, spill_pool, abort_helper) = { + let (mut rx, reservation, spill_pool, spill_manager, abort_helper) = { // lock mutexes let mut state = state.lock(); let state = state.consume_input_streams( @@ -759,13 +762,20 @@ impl ExecutionPlan for RepartitionExec { rx, reservation, spill_pool, + spill_manager, .. } = state .channels .remove(&partition) .expect("partition not used yet"); - (rx, reservation, spill_pool, Arc::clone(&state.abort_helper)) + ( + rx, + reservation, + spill_pool, + spill_manager, + Arc::clone(&state.abort_helper), + ) }; trace!( @@ -784,6 +794,7 @@ impl ExecutionPlan for RepartitionExec { _drop_helper: Arc::clone(&abort_helper), reservation: Arc::clone(&reservation), spill_pool: Arc::clone(&spill_pool), + spill_manager: Arc::clone(&spill_manager), state: RepartitionStreamState::ReceivingFromChannel, }) as SendableRecordBatchStream }) @@ -814,6 +825,7 @@ impl ExecutionPlan for RepartitionExec { _drop_helper: abort_helper, reservation, spill_pool, + spill_manager, state: RepartitionStreamState::ReceivingFromChannel, }) as SendableRecordBatchStream) } @@ -1191,6 +1203,8 @@ impl RepartitionExec { enum RepartitionStreamState { /// Waiting for next item from channel ReceivingFromChannel, + /// Reading a spilled batch from disk via SpillReaderStream (spawned blocking tasks) + ReadingSpilledBatch(SendableRecordBatchStream), } struct RepartitionStream { @@ -1215,6 +1229,9 @@ struct RepartitionStream { /// SpillPool for batched spilling with FIFO semantics spill_pool: Arc>, + /// SpillManager for creating streams from spill files + spill_manager: Arc, + /// Current state of the stream state: RepartitionStreamState, } @@ -1240,13 +1257,20 @@ impl Stream for RepartitionStream { return Poll::Ready(Some(Ok(batch))); } Ok(RepartitionBatch::Spilled) => { - // Read from SpillPool (FIFO order) - match self.spill_pool.lock().pop_batch()? { - Some(batch) => { - return Poll::Ready(Some(Ok(batch))); + // Get next file from SpillPool and create a stream + let next_file = self.spill_pool.lock().take_next_file()?; + match next_file { + Some(spill_file) => { + // Create stream using SpillReaderStream + spawn_buffered + let stream = self + .spill_manager + .read_spill_as_stream(spill_file, None)?; + self.state = + RepartitionStreamState::ReadingSpilledBatch(stream); + continue; } None => { - // No spilled batches available, continue receiving + // No spilled files available, continue receiving from channel continue; } } @@ -1273,6 +1297,38 @@ impl Stream for RepartitionStream { } } } + RepartitionStreamState::ReadingSpilledBatch(stream) => { + match futures::ready!(stream.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + // Return batch and stay in ReadingSpilledBatch state + return Poll::Ready(Some(Ok(batch))); + } + Some(Err(e)) => { + // Error reading spilled batch + return Poll::Ready(Some(Err(e))); + } + None => { + // Current spill file exhausted, check if there are more + let next_file = self.spill_pool.lock().take_next_file()?; + match next_file { + Some(spill_file) => { + // Create stream for next file + let new_stream = self + .spill_manager + .read_spill_as_stream(spill_file, None)?; + self.state = + RepartitionStreamState::ReadingSpilledBatch(new_stream); + continue; + } + None => { + // No more spilled files, go back to receiving from channel + self.state = RepartitionStreamState::ReceivingFromChannel; + continue; + } + } + } + } + } } } } @@ -1303,6 +1359,9 @@ struct PerPartitionStream { /// SpillPool for batched spilling with FIFO semantics (shared across streams) spill_pool: Arc>, + /// SpillManager for creating streams from spill files + spill_manager: Arc, + /// Current state of the stream state: RepartitionStreamState, } @@ -1328,13 +1387,20 @@ impl Stream for PerPartitionStream { return Poll::Ready(Some(Ok(batch))); } Ok(RepartitionBatch::Spilled) => { - // Read from SpillPool (FIFO order) - match self.spill_pool.lock().pop_batch()? { - Some(batch) => { - return Poll::Ready(Some(Ok(batch))); + // Get next file from SpillPool and create a stream + let next_file = self.spill_pool.lock().take_next_file()?; + match next_file { + Some(spill_file) => { + // Create stream using SpillReaderStream + spawn_buffered + let stream = self + .spill_manager + .read_spill_as_stream(spill_file, None)?; + self.state = + RepartitionStreamState::ReadingSpilledBatch(stream); + continue; } None => { - // No spilled batches available, continue receiving + // No spilled files available, continue receiving from channel continue; } } @@ -1350,6 +1416,38 @@ impl Stream for PerPartitionStream { None => return Poll::Ready(None), } } + RepartitionStreamState::ReadingSpilledBatch(stream) => { + match futures::ready!(stream.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + // Return batch and stay in ReadingSpilledBatch state + return Poll::Ready(Some(Ok(batch))); + } + Some(Err(e)) => { + // Error reading spilled batch + return Poll::Ready(Some(Err(e))); + } + None => { + // Current spill file exhausted, check if there are more + let next_file = self.spill_pool.lock().take_next_file()?; + match next_file { + Some(spill_file) => { + // Create stream for next file + let new_stream = self + .spill_manager + .read_spill_as_stream(spill_file, None)?; + self.state = + RepartitionStreamState::ReadingSpilledBatch(new_stream); + continue; + } + None => { + // No more spilled files, go back to receiving from channel + self.state = RepartitionStreamState::ReceivingFromChannel; + continue; + } + } + } + } + } } } } diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index ded66bba84c10..5a9d9717afafc 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -315,6 +315,27 @@ impl SpillPool { } } + /// Takes the next spill file from the pool for reading. + /// + /// Returns the oldest unread file, or None if the pool is empty. + /// The file is removed from the pool and should be read using + /// `SpillManager::read_spill_as_stream()`. + /// + /// This method flushes any pending writes before returning a file. + /// + /// # Errors + /// + /// Returns an error if flushing pending writes fails. + pub fn take_next_file(&mut self) -> Result> { + // Ensure any pending writes are flushed first + if self.current_write_file.is_some() { + self.flush()?; + } + + // Take the oldest file from the queue + Ok(self.files.pop_front().map(|spill_file| spill_file.file)) + } + /// Finalizes the current write file and adds it to the files queue. /// /// Called automatically by `push_batch` when rotating files, but can From f183b9e282f7d31eb25d61f5d470183346f33ace Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 21 Oct 2025 14:49:28 -0500 Subject: [PATCH 04/44] lint --- .../physical-plan/src/repartition/mod.rs | 128 ++++++++++-------- docs/source/user-guide/configs.md | 1 + 2 files changed, 71 insertions(+), 58 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 2cf07dc1e5484..5951abfabab11 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1248,37 +1248,40 @@ impl Stream for RepartitionStream { RepartitionStreamState::ReceivingFromChannel => { let value = futures::ready!(self.input.recv().poll_unpin(cx)); match value { - Some(Some(v)) => match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Get next file from SpillPool and create a stream - let next_file = self.spill_pool.lock().take_next_file()?; - match next_file { - Some(spill_file) => { - // Create stream using SpillReaderStream + spawn_buffered - let stream = self - .spill_manager - .read_spill_as_stream(spill_file, None)?; - self.state = + Some(Some(v)) => { + match v { + Ok(RepartitionBatch::Memory(batch)) => { + // Release memory and return + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + return Poll::Ready(Some(Ok(batch))); + } + Ok(RepartitionBatch::Spilled) => { + // Get next file from SpillPool and create a stream + let next_file = + self.spill_pool.lock().take_next_file()?; + match next_file { + Some(spill_file) => { + // Create stream using SpillReaderStream + spawn_buffered + let stream = self + .spill_manager + .read_spill_as_stream(spill_file, None)?; + self.state = RepartitionStreamState::ReadingSpilledBatch(stream); - continue; - } - None => { - // No spilled files available, continue receiving from channel - continue; + continue; + } + None => { + // No spilled files available, continue receiving from channel + continue; + } } } + Err(e) => { + return Poll::Ready(Some(Err(e))); + } } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - }, + } Some(None) => { self.num_input_partitions_processed += 1; @@ -1317,12 +1320,15 @@ impl Stream for RepartitionStream { .spill_manager .read_spill_as_stream(spill_file, None)?; self.state = - RepartitionStreamState::ReadingSpilledBatch(new_stream); + RepartitionStreamState::ReadingSpilledBatch( + new_stream, + ); continue; } None => { // No more spilled files, go back to receiving from channel - self.state = RepartitionStreamState::ReceivingFromChannel; + self.state = + RepartitionStreamState::ReceivingFromChannel; continue; } } @@ -1378,37 +1384,40 @@ impl Stream for PerPartitionStream { RepartitionStreamState::ReceivingFromChannel => { let value = futures::ready!(self.receiver.recv().poll_unpin(cx)); match value { - Some(Some(v)) => match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Get next file from SpillPool and create a stream - let next_file = self.spill_pool.lock().take_next_file()?; - match next_file { - Some(spill_file) => { - // Create stream using SpillReaderStream + spawn_buffered - let stream = self - .spill_manager - .read_spill_as_stream(spill_file, None)?; - self.state = + Some(Some(v)) => { + match v { + Ok(RepartitionBatch::Memory(batch)) => { + // Release memory and return + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + return Poll::Ready(Some(Ok(batch))); + } + Ok(RepartitionBatch::Spilled) => { + // Get next file from SpillPool and create a stream + let next_file = + self.spill_pool.lock().take_next_file()?; + match next_file { + Some(spill_file) => { + // Create stream using SpillReaderStream + spawn_buffered + let stream = self + .spill_manager + .read_spill_as_stream(spill_file, None)?; + self.state = RepartitionStreamState::ReadingSpilledBatch(stream); - continue; - } - None => { - // No spilled files available, continue receiving from channel - continue; + continue; + } + None => { + // No spilled files available, continue receiving from channel + continue; + } } } + Err(e) => { + return Poll::Ready(Some(Err(e))); + } } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - }, + } Some(None) => { // Input partition has finished sending batches return Poll::Ready(None); @@ -1436,12 +1445,15 @@ impl Stream for PerPartitionStream { .spill_manager .read_spill_as_stream(spill_file, None)?; self.state = - RepartitionStreamState::ReadingSpilledBatch(new_stream); + RepartitionStreamState::ReadingSpilledBatch( + new_stream, + ); continue; } None => { // No more spilled files, go back to receiving from channel - self.state = RepartitionStreamState::ReceivingFromChannel; + self.state = + RepartitionStreamState::ReceivingFromChannel; continue; } } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 7ca5eb8f7be45..949bcb201749a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -114,6 +114,7 @@ The following configuration settings are available: | datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.max_spill_file_size_bytes | 104857600 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec, SortExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation (especially in LIFO mode where files are truncated after reading). Default: 100 MB | | datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | | datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | From 0cc7a63fc0559b104fd761b0166badca5ce1f96a Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 21 Oct 2025 17:17:43 -0500 Subject: [PATCH 05/44] rework --- .../physical-plan/src/repartition/mod.rs | 351 +++++------ .../physical-plan/src/spill/spill_pool.rs | 577 +++++------------- 2 files changed, 333 insertions(+), 595 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 5951abfabab11..0daa8a5d185bd 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -39,7 +39,7 @@ use crate::repartition::distributor_channels::{ }; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; -use crate::spill::spill_pool::SpillPool; +use crate::spill::spill_pool::{SpillPool, SpillPoolStream}; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -788,14 +788,19 @@ impl ExecutionPlan for RepartitionExec { let input_streams = rx .into_iter() .map(|receiver| { + let spill_stream = SpillPool::reader( + Arc::clone(&spill_pool), + Arc::clone(&spill_manager), + ); + Box::pin(PerPartitionStream { schema: Arc::clone(&schema_captured), receiver, _drop_helper: Arc::clone(&abort_helper), reservation: Arc::clone(&reservation), spill_pool: Arc::clone(&spill_pool), - spill_manager: Arc::clone(&spill_manager), - state: RepartitionStreamState::ReceivingFromChannel, + spill_stream, + input_finished: false, }) as SendableRecordBatchStream }) .collect::>(); @@ -817,6 +822,11 @@ impl ExecutionPlan for RepartitionExec { .with_reservation(merge_reservation) .build() } else { + let spill_stream = SpillPool::reader( + Arc::clone(&spill_pool), + Arc::clone(&spill_manager), + ); + Ok(Box::pin(RepartitionStream { num_input_partitions, num_input_partitions_processed: 0, @@ -825,8 +835,8 @@ impl ExecutionPlan for RepartitionExec { _drop_helper: abort_helper, reservation, spill_pool, - spill_manager, - state: RepartitionStreamState::ReceivingFromChannel, + spill_stream, + all_inputs_finished: false, }) as SendableRecordBatchStream) } }) @@ -1200,13 +1210,6 @@ impl RepartitionExec { } } -enum RepartitionStreamState { - /// Waiting for next item from channel - ReceivingFromChannel, - /// Reading a spilled batch from disk via SpillReaderStream (spawned blocking tasks) - ReadingSpilledBatch(SendableRecordBatchStream), -} - struct RepartitionStream { /// Number of input partitions that will be sending batches to this output channel num_input_partitions: usize, @@ -1226,14 +1229,14 @@ struct RepartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, - /// SpillPool for batched spilling with FIFO semantics + /// SpillPool for batched spilling with FIFO semantics (shared for writing) spill_pool: Arc>, - /// SpillManager for creating streams from spill files - spill_manager: Arc, + /// Infinite stream for reading from the spill pool + spill_stream: SpillPoolStream, - /// Current state of the stream - state: RepartitionStreamState, + /// Flag indicating all inputs have finished + all_inputs_finished: bool, } impl Stream for RepartitionStream { @@ -1243,97 +1246,85 @@ impl Stream for RepartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { + use futures::StreamExt; + loop { - match &mut self.state { - RepartitionStreamState::ReceivingFromChannel => { - let value = futures::ready!(self.input.recv().poll_unpin(cx)); - match value { - Some(Some(v)) => { - match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Get next file from SpillPool and create a stream - let next_file = - self.spill_pool.lock().take_next_file()?; - match next_file { - Some(spill_file) => { - // Create stream using SpillReaderStream + spawn_buffered - let stream = self - .spill_manager - .read_spill_as_stream(spill_file, None)?; - self.state = - RepartitionStreamState::ReadingSpilledBatch(stream); - continue; - } - None => { - // No spilled files available, continue receiving from channel - continue; - } - } - } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - } - } - Some(None) => { - self.num_input_partitions_processed += 1; - - if self.num_input_partitions - == self.num_input_partitions_processed - { - // all input partitions have finished sending batches - return Poll::Ready(None); - } else { - // other partitions still have data to send - continue; - } - } - None => { - return Poll::Ready(None); - } + // First, check if there's a spilled batch available + match self.spill_stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + // Got a spilled batch + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Spill stream ended - all spilled data has been read + return Poll::Ready(None); + } + Poll::Pending => { + // No spilled data available right now + if self.all_inputs_finished { + // All inputs finished, wait for spill stream to have more data or finish + return Poll::Pending; } + // Otherwise check the channel } - RepartitionStreamState::ReadingSpilledBatch(stream) => { - match futures::ready!(stream.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - // Return batch and stay in ReadingSpilledBatch state - return Poll::Ready(Some(Ok(batch))); - } - Some(Err(e)) => { - // Error reading spilled batch - return Poll::Ready(Some(Err(e))); - } - None => { - // Current spill file exhausted, check if there are more - let next_file = self.spill_pool.lock().take_next_file()?; - match next_file { - Some(spill_file) => { - // Create stream for next file - let new_stream = self - .spill_manager - .read_spill_as_stream(spill_file, None)?; - self.state = - RepartitionStreamState::ReadingSpilledBatch( - new_stream, - ); - continue; - } - None => { - // No more spilled files, go back to receiving from channel - self.state = - RepartitionStreamState::ReceivingFromChannel; - continue; - } - } - } + } + + // If all inputs are finished, don't poll channel anymore, just wait for spill_stream + if self.all_inputs_finished { + return Poll::Pending; + } + + // Try to get next item from channel + let value = match self.input.recv().poll_unpin(cx) { + Poll::Ready(v) => v, + Poll::Pending => { + // Nothing from channel either, wait + return Poll::Pending; + } + }; + + match value { + Some(Some(v)) => match v { + Ok(RepartitionBatch::Memory(batch)) => { + // Release memory and return + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + return Poll::Ready(Some(Ok(batch))); + } + Ok(RepartitionBatch::Spilled) => { + // Batch was spilled, it's available in spill_stream + // Loop back to poll spill_stream again + continue; } + Err(e) => { + return Poll::Ready(Some(Err(e))); + } + }, + Some(None) => { + self.num_input_partitions_processed += 1; + + if self.num_input_partitions == self.num_input_partitions_processed { + // All input partitions have finished sending batches + // Flush and finalize the SpillPool + { + let mut pool = self.spill_pool.lock(); + pool.flush().ok(); + pool.finalize(); + } // Drop the lock before continuing + self.all_inputs_finished = true; + // Continue to drain any remaining spilled batches + continue; + } else { + // other partitions still have data to send + continue; + } + } + None => { + return Poll::Ready(None); } } } @@ -1362,14 +1353,14 @@ struct PerPartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, - /// SpillPool for batched spilling with FIFO semantics (shared across streams) + /// SpillPool for batched spilling with FIFO semantics (shared for writing) spill_pool: Arc>, - /// SpillManager for creating streams from spill files - spill_manager: Arc, + /// Infinite stream for reading from the spill pool + spill_stream: SpillPoolStream, - /// Current state of the stream - state: RepartitionStreamState, + /// Flag indicating input partition has finished + input_finished: bool, } impl Stream for PerPartitionStream { @@ -1379,87 +1370,77 @@ impl Stream for PerPartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { + use futures::StreamExt; + loop { - match &mut self.state { - RepartitionStreamState::ReceivingFromChannel => { - let value = futures::ready!(self.receiver.recv().poll_unpin(cx)); - match value { - Some(Some(v)) => { - match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Get next file from SpillPool and create a stream - let next_file = - self.spill_pool.lock().take_next_file()?; - match next_file { - Some(spill_file) => { - // Create stream using SpillReaderStream + spawn_buffered - let stream = self - .spill_manager - .read_spill_as_stream(spill_file, None)?; - self.state = - RepartitionStreamState::ReadingSpilledBatch(stream); - continue; - } - None => { - // No spilled files available, continue receiving from channel - continue; - } - } - } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - } - } - Some(None) => { - // Input partition has finished sending batches - return Poll::Ready(None); - } - None => return Poll::Ready(None), + // First, check if there's a spilled batch available + match self.spill_stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + // Got a spilled batch + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Spill stream never ends, this shouldn't happen + unreachable!("SpillPoolStream should never end"); + } + Poll::Pending => { + // No spilled data available + if self.input_finished { + // Input finished and no more spilled data - we're done + return Poll::Ready(None); } + // Otherwise check the channel } - RepartitionStreamState::ReadingSpilledBatch(stream) => { - match futures::ready!(stream.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - // Return batch and stay in ReadingSpilledBatch state - return Poll::Ready(Some(Ok(batch))); - } - Some(Err(e)) => { - // Error reading spilled batch - return Poll::Ready(Some(Err(e))); - } - None => { - // Current spill file exhausted, check if there are more - let next_file = self.spill_pool.lock().take_next_file()?; - match next_file { - Some(spill_file) => { - // Create stream for next file - let new_stream = self - .spill_manager - .read_spill_as_stream(spill_file, None)?; - self.state = - RepartitionStreamState::ReadingSpilledBatch( - new_stream, - ); - continue; - } - None => { - // No more spilled files, go back to receiving from channel - self.state = - RepartitionStreamState::ReceivingFromChannel; - continue; - } - } - } + } + + // If input is finished, don't poll channel anymore + if self.input_finished { + continue; + } + + // Try to get next item from channel + let value = match self.receiver.recv().poll_unpin(cx) { + Poll::Ready(v) => v, + Poll::Pending => { + // Nothing from channel either, wait + return Poll::Pending; + } + }; + + match value { + Some(Some(v)) => match v { + Ok(RepartitionBatch::Memory(batch)) => { + // Release memory and return + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + return Poll::Ready(Some(Ok(batch))); } + Ok(RepartitionBatch::Spilled) => { + // Batch was spilled, it's available in spill_stream + // Loop back to poll spill_stream again + continue; + } + Err(e) => { + return Poll::Ready(Some(Err(e))); + } + }, + Some(None) => { + // Input partition has finished sending batches + // Flush and finalize the SpillPool + { + let mut pool = self.spill_pool.lock(); + pool.flush().ok(); + pool.finalize(); + } // Drop the lock before continuing + self.input_finished = true; + // Continue to drain any remaining spilled batches + continue; } + None => return Poll::Ready(None), } } } diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 5a9d9717afafc..c481b2e36a7d3 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -51,15 +51,16 @@ //! ``` use std::collections::VecDeque; -use std::fs::File; -use std::io::BufReader; use std::sync::Arc; +use std::task::Waker; + +use parking_lot::Mutex; use arrow::datatypes::SchemaRef; -use arrow::ipc::reader::StreamReader; use arrow::record_batch::RecordBatch; -use datafusion_common::{exec_datafusion_err, Result}; +use datafusion_common::Result; use datafusion_execution::disk_manager::RefCountedTempFile; +use datafusion_execution::SendableRecordBatchStream; use super::in_progress_spill_file::InProgressSpillFile; use super::spill_manager::SpillManager; @@ -68,98 +69,24 @@ use super::spill_manager::SpillManager; struct SpillFile { /// The temp file handle (auto-deletes when dropped) file: RefCountedTempFile, - /// Number of batches originally written to this file - total_batches: usize, - /// Number of batches already read from this file - batches_read: usize, - /// Total size of this file in bytes (kept for potential debugging/metrics) - #[allow(dead_code)] - total_size: usize, - /// Sequential reader for this file (lazily initialized on first read) - reader: Option>>, } impl SpillFile { - fn new(file: RefCountedTempFile, total_batches: usize, total_size: usize) -> Self { - Self { - file, - total_batches, - batches_read: 0, - total_size, - reader: None, - } - } - - /// Returns true if all batches have been read from this file - fn is_fully_consumed(&self) -> bool { - self.batches_read >= self.total_batches - } - - /// Returns the number of unread batches remaining - fn remaining_batches(&self) -> usize { - self.total_batches.saturating_sub(self.batches_read) - } - - /// Reads the next batch from this file sequentially. - /// - /// Initializes the reader on first call. Returns None when all batches consumed. - fn read_next_batch(&mut self) -> Result> { - if self.is_fully_consumed() { - return Ok(None); - } - - // Initialize reader on first use - if self.reader.is_none() { - let file_handle = File::open(self.file.path()).map_err(|e| { - exec_datafusion_err!( - "Failed to open spill file {:?} for reading: {}", - self.file.path(), - e - ) - })?; - let buf_reader = BufReader::new(file_handle); - // SAFETY: DataFusion's spill writer strictly follows Arrow IPC specifications - let reader = unsafe { - StreamReader::try_new(buf_reader, None)?.with_skip_validation(true) - }; - self.reader = Some(reader); - } - - // Read next batch from sequential stream - if let Some(reader) = &mut self.reader { - match reader.next() { - Some(Ok(batch)) => { - self.batches_read += 1; - Ok(Some(batch)) - } - Some(Err(e)) => Err(e.into()), - None => { - // Stream ended - this shouldn't happen if batch count is correct - if !self.is_fully_consumed() { - return Err(exec_datafusion_err!( - "Unexpected end of spill file: read {} batches but expected {}", - self.batches_read, - self.total_batches - )); - } - Ok(None) - } - } - } else { - unreachable!("Reader should be initialized above") - } + fn new(file: RefCountedTempFile, _total_batches: usize, _total_size: usize) -> Self { + Self { file } } } /// A pool of spill files that manages batch-level spilling with FIFO semantics. /// /// Batches are written sequentially to files, with automatic rotation when the -/// configured size limit is reached. Reading is done in FIFO order (oldest batch first). +/// configured size limit is reached. Reading is done via an infinite stream +/// that can read concurrently while writes continue. /// /// # Thread Safety /// /// `SpillPool` is not thread-safe and should be used from a single thread or -/// protected with appropriate synchronization. +/// protected with appropriate synchronization (e.g., `Arc>`). pub struct SpillPool { /// Maximum size in bytes before rotating to a new file max_file_size_bytes: usize, @@ -176,6 +103,10 @@ pub struct SpillPool { /// Schema for batches (kept for potential validation in debug builds) #[allow(dead_code)] schema: SchemaRef, + /// Wakers to notify when new data is available for readers + wakers: Vec, + /// Flag indicating no more writes will occur + finalized: bool, } impl SpillPool { @@ -199,31 +130,42 @@ impl SpillPool { current_batch_count: 0, spill_manager, schema, + wakers: Vec::new(), + finalized: false, } } - /// Returns the number of files currently in the pool - pub fn file_count(&self) -> usize { - self.files.len() - + if self.current_write_file.is_some() { - 1 - } else { - 0 - } + /// Marks the pool as finalized, indicating no more writes will occur. + /// This allows readers to know when to stop waiting for more data. + pub fn finalize(&mut self) { + self.finalized = true; + self.wake(); // Wake readers to check finalized status } - /// Returns the total number of unread batches across all files - pub fn batch_count(&self) -> usize { - self.files - .iter() - .map(|f| f.remaining_batches()) - .sum::() - + self.current_batch_count + /// Returns true if the pool has been finalized + pub fn is_finalized(&self) -> bool { + self.finalized } - /// Returns true if the pool is empty (no batches to read) - pub fn is_empty(&self) -> bool { - self.batch_count() == 0 + /// Creates an infinite stream reader for this pool. + /// + /// The stream automatically handles file rotation and can read concurrently + /// while writes continue to the pool. When the stream catches up to the writer, + /// it will return `Poll::Pending` and wait for more data. + /// + /// # Arguments + /// + /// * `pool` - Shared reference to the SpillPool + /// * `spill_manager` - Manager for creating streams from spill files + /// + /// # Returns + /// + /// An infinite `SpillPoolStream` that never ends until dropped + pub fn reader( + pool: Arc>, + spill_manager: Arc, + ) -> SpillPoolStream { + SpillPoolStream::new(pool, spill_manager) } /// Spills a batch to the pool, rotating files when necessary. @@ -272,74 +214,31 @@ impl SpillPool { self.current_write_size += batch_size; self.current_batch_count += 1; + // Wake any waiting readers + self.wake(); + Ok(()) } - /// Reads the next batch from the pool in FIFO order. - /// - /// Returns the oldest unread batch, or None if the pool is empty. - /// - /// # Errors - /// - /// Returns an error if disk I/O fails during read. - pub fn pop_batch(&mut self) -> Result> { - // Ensure any pending writes are flushed first - if self.current_write_file.is_some() { - self.flush()?; - } - - loop { - // Get the oldest file (front of queue) - let spill_file = match self.files.front_mut() { - Some(file) => file, - None => return Ok(None), // No files available - }; - - // Try to read next batch from this file - match spill_file.read_next_batch()? { - Some(batch) => { - // Check if file is now fully consumed after reading this batch - let is_consumed = spill_file.is_fully_consumed(); - if is_consumed { - // Remove the file from the queue - self.files.pop_front(); - } - return Ok(Some(batch)); - } - None => { - // File is fully consumed, remove it and try next file - self.files.pop_front(); - continue; - } - } + /// Registers a waker to be notified when new data is available + fn register_waker(&mut self, waker: Waker) { + // Only register if not already present (avoid duplicates) + if !self.wakers.iter().any(|w| w.will_wake(&waker)) { + self.wakers.push(waker); } } - /// Takes the next spill file from the pool for reading. - /// - /// Returns the oldest unread file, or None if the pool is empty. - /// The file is removed from the pool and should be read using - /// `SpillManager::read_spill_as_stream()`. - /// - /// This method flushes any pending writes before returning a file. - /// - /// # Errors - /// - /// Returns an error if flushing pending writes fails. - pub fn take_next_file(&mut self) -> Result> { - // Ensure any pending writes are flushed first - if self.current_write_file.is_some() { - self.flush()?; + /// Wakes all registered readers + fn wake(&mut self) { + for waker in self.wakers.drain(..) { + waker.wake(); } - - // Take the oldest file from the queue - Ok(self.files.pop_front().map(|spill_file| spill_file.file)) } /// Finalizes the current write file and adds it to the files queue. /// - /// Called automatically by `push_batch` when rotating files, but can - /// also be called explicitly to ensure all pending data is flushed. + /// This is called automatically when files reach the size limit, but can + /// also be called explicitly to ensure all pending data is available for reading. pub fn flush(&mut self) -> Result<()> { if self.current_write_file.is_some() { self.finish_current_file()?; @@ -368,6 +267,9 @@ impl SpillPool { // Reset write state self.current_write_size = 0; self.current_batch_count = 0; + + // Wake any waiting readers since a new complete file is available + self.wake(); } Ok(()) @@ -382,270 +284,125 @@ impl Drop for SpillPool { } } -#[cfg(test)] -mod tests { - use super::*; - use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; - use arrow::array::{ArrayRef, Int32Array}; - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_execution::runtime_env::RuntimeEnv; - - fn create_test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - ])) - } - - fn create_test_batch(start: i32, count: usize) -> RecordBatch { - let schema = create_test_schema(); - let a: ArrayRef = Arc::new(Int32Array::from( - (start..start + count as i32).collect::>(), - )); - let b: ArrayRef = Arc::new(Int32Array::from( - (start * 10..start * 10 + count as i32 * 10) - .step_by(10) - .collect::>(), - )); - RecordBatch::try_new(schema, vec![a, b]).unwrap() - } - - fn create_spill_pool(max_file_size: usize) -> SpillPool { - let env = Arc::new(RuntimeEnv::default()); - let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); - let schema = create_test_schema(); - let spill_manager = - Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); - - SpillPool::new(max_file_size, spill_manager, schema) - } - - #[test] - fn test_empty_pool() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - assert!(pool.is_empty()); - assert_eq!(pool.file_count(), 0); - assert_eq!(pool.batch_count(), 0); - assert!(pool.pop_batch()?.is_none()); - - Ok(()) - } - - #[test] - fn test_single_batch() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Push one batch - let batch1 = create_test_batch(0, 10); - pool.push_batch(&batch1)?; - pool.flush()?; - - assert!(!pool.is_empty()); - assert_eq!(pool.file_count(), 1); - assert_eq!(pool.batch_count(), 1); - - // Pop and verify - let result = pool.pop_batch()?.unwrap(); - assert_eq!(result.num_rows(), 10); - assert_eq!(result.num_columns(), 2); - - // Pool should be empty now - assert!(pool.is_empty()); - assert_eq!(pool.file_count(), 0); - - Ok(()) - } - - #[test] - fn test_multiple_batches_single_file() -> Result<()> { - let mut pool = create_spill_pool(10 * 1024 * 1024); // Large file size - - // Push multiple batches - for i in 0..5 { - let batch = create_test_batch(i * 10, 10); - pool.push_batch(&batch)?; - } - pool.flush()?; - - assert_eq!(pool.file_count(), 1); - assert_eq!(pool.batch_count(), 5); - - // Pop in FIFO order - for i in 0..5 { - let result = pool.pop_batch()?.unwrap(); - assert_eq!(result.num_rows(), 10); - - // Verify first value is correct (FIFO order) - let col_a = result - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), i * 10); - } - - assert!(pool.is_empty()); - Ok(()) - } +/// An infinite stream that reads from a SpillPool. +/// +/// The stream automatically handles file rotation and reads from completed files. +/// When no completed files are available, it returns `Poll::Pending` and waits +/// for the writer to complete more files. +/// +/// The stream never ends (`Poll::Ready(None)`) until it is dropped. +pub struct SpillPoolStream { + /// Shared reference to the spill pool + spill_pool: Arc>, + /// SpillManager for creating streams from spill files + spill_manager: Arc, + /// Current stream being read from + current_stream: Option, + /// Schema for the batches + schema: SchemaRef, +} - #[test] - fn test_file_rotation() -> Result<()> { - // Small file size to force rotation - let mut pool = create_spill_pool(500); // ~500 bytes +impl SpillPoolStream { + /// Creates a new infinite stream from a SpillPool. + /// + /// # Arguments + /// + /// * `spill_pool` - Shared reference to the pool to read from + /// * `spill_manager` - Manager for creating streams from spill files + pub fn new( + spill_pool: Arc>, + spill_manager: Arc, + ) -> Self { + let schema = { + let pool = spill_pool.lock(); + Arc::clone(&pool.schema) + }; - // Push multiple batches - should create multiple files - for i in 0..10 { - let batch = create_test_batch(i * 5, 5); - pool.push_batch(&batch)?; - } - pool.flush()?; - - // Should have multiple files due to size limit - assert!(pool.file_count() > 1, "Expected file rotation to occur"); - assert_eq!(pool.batch_count(), 10); - - // Pop all batches in FIFO order - for i in 0..10 { - let result = pool.pop_batch()?.unwrap(); - let col_a = result - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), i * 5); + Self { + spill_pool, + spill_manager, + current_stream: None, + schema, } - - assert!(pool.is_empty()); - Ok(()) } +} - #[test] - fn test_empty_batch_skipped() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - let batch1 = create_test_batch(0, 10); - let empty_batch = RecordBatch::new_empty(create_test_schema()); - let batch2 = create_test_batch(10, 10); - - pool.push_batch(&batch1)?; - pool.push_batch(&empty_batch)?; // Should be skipped - pool.push_batch(&batch2)?; - pool.flush()?; - - // Should only have 2 batches (empty one skipped) - assert_eq!(pool.batch_count(), 2); - - Ok(()) - } +impl futures::Stream for SpillPoolStream { + type Item = Result; - #[test] - fn test_interleaved_push_pop() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Push batch 0 - pool.push_batch(&create_test_batch(0, 5))?; - pool.flush()?; - - // Pop batch 0 - let result = pool.pop_batch()?.unwrap(); - assert_eq!(result.num_rows(), 5); - - // Push batches 1, 2 - pool.push_batch(&create_test_batch(10, 5))?; - pool.push_batch(&create_test_batch(20, 5))?; - pool.flush()?; - - // Pop batch 1 - let result = pool.pop_batch()?.unwrap(); - let col_a = result - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), 10); - - // Push batch 3 - pool.push_batch(&create_test_batch(30, 5))?; - pool.flush()?; - - // Pop remaining: should get 2, 3 - let result = pool.pop_batch()?.unwrap(); - let col_a = result - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), 20); - - let result = pool.pop_batch()?.unwrap(); - let col_a = result - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), 30); - - assert!(pool.is_empty()); - Ok(()) - } + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + use futures::StreamExt; + use std::task::Poll; - #[test] - fn test_auto_flush_on_pop() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); + loop { + // If we have a current stream, try to read from it + if let Some(stream) = &mut self.current_stream { + match stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Current stream exhausted (finished reading a completed file) + self.current_stream = None; + // Continue loop to try getting next file + } + Poll::Pending => { + // Stream not ready yet + return Poll::Pending; + } + } + } - // Push without explicit flush - pool.push_batch(&create_test_batch(0, 10))?; + // No current stream, try to get the next file to read + // Only read from completed files in the queue + let mut pool = self.spill_pool.lock(); - // Pop should auto-flush - let result = pool.pop_batch()?.unwrap(); - assert_eq!(result.num_rows(), 10); + if let Some(spill_file) = pool.files.pop_front() { + // We have a completed file to read + let file = spill_file.file; + drop(pool); // Release lock before creating stream - Ok(()) + match self.spill_manager.read_spill_as_stream(file, None) { + Ok(stream) => { + self.current_stream = Some(stream); + // Continue loop to poll the new stream + } + Err(e) => { + return Poll::Ready(Some(Err(e))); + } + } + } else { + // No completed files available + let is_finalized = pool.is_finalized(); + if is_finalized { + // Pool is finalized and no more files - we're done + return Poll::Ready(None); + } + // Register waker and wait for more files + pool.register_waker(cx.waker().clone()); + return Poll::Pending; + } + } } +} - #[test] - fn test_large_batches() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Create larger batches - let batch1 = create_test_batch(0, 1000); - let batch2 = create_test_batch(1000, 1000); - - pool.push_batch(&batch1)?; - pool.push_batch(&batch2)?; - pool.flush()?; - - let result1 = pool.pop_batch()?.unwrap(); - assert_eq!(result1.num_rows(), 1000); - - let result2 = pool.pop_batch()?.unwrap(); - assert_eq!(result2.num_rows(), 1000); - - Ok(()) +impl datafusion_execution::RecordBatchStream for SpillPoolStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) } +} - #[test] - fn test_file_cleanup() -> Result<()> { - let mut pool = create_spill_pool(500); - - // Create multiple files - for i in 0..10 { - pool.push_batch(&create_test_batch(i * 5, 5))?; - } - pool.flush()?; - - let initial_file_count = pool.file_count(); - assert!(initial_file_count > 1); - - // Pop some batches - should cleanup fully consumed files - for _ in 0..5 { - pool.pop_batch()?; - } - - // File count should decrease as old files are consumed - assert!(pool.file_count() <= initial_file_count); - - Ok(()) - } +#[cfg(test)] +mod tests { + // TODO: Update tests to use the new stream-based API + // The old tests tested pop_batch(), file_count(), batch_count(), is_empty() + // which have been removed in favor of the SpillPoolStream interface. + // + // The SpillPool is now tested through integration tests in repartition/mod.rs } From 6dd6b6d36b051e14340370c98b4a3baf0e40af42 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 21 Oct 2025 18:00:58 -0500 Subject: [PATCH 06/44] Add some tests --- .../physical-plan/src/spill/spill_pool.rs | 706 +++++++++++++++++- 1 file changed, 701 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index c481b2e36a7d3..f30e3baed295c 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -400,9 +400,705 @@ impl datafusion_execution::RecordBatchStream for SpillPoolStream { #[cfg(test)] mod tests { - // TODO: Update tests to use the new stream-based API - // The old tests tested pop_batch(), file_count(), batch_count(), is_empty() - // which have been removed in favor of the SpillPoolStream interface. - // - // The SpillPool is now tested through integration tests in repartition/mod.rs + use super::*; + use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; + use arrow::array::{ArrayRef, Int32Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_execution::runtime_env::RuntimeEnv; + use futures::StreamExt; + use std::task::Poll; + use tokio; + + // ============================================================================ + // Test Utilities + // ============================================================================ + + fn create_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ])) + } + + fn create_test_batch(start: i32, count: usize) -> RecordBatch { + let schema = create_test_schema(); + let a: ArrayRef = Arc::new(Int32Array::from( + (start..start + count as i32).collect::>(), + )); + let b: ArrayRef = Arc::new(Int32Array::from( + (start * 10..start * 10 + count as i32 * 10) + .step_by(10) + .collect::>(), + )); + RecordBatch::try_new(schema, vec![a, b]).unwrap() + } + + fn create_spill_pool(max_file_size: usize) -> SpillPool { + let env = Arc::new(RuntimeEnv::default()); + let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let schema = create_test_schema(); + let spill_manager = + Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); + + SpillPool::new(max_file_size, spill_manager, schema) + } + + /// Helper to collect all batches from a stream + async fn collect_batches(mut stream: SpillPoolStream) -> Result> { + let mut batches = Vec::new(); + while let Some(result) = stream.next().await { + batches.push(result?); + } + Ok(batches) + } + + // ============================================================================ + // Basic Functionality Tests + // ============================================================================ + + #[tokio::test] + async fn test_empty_pool_stream() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + pool.finalize(); // Mark as done with no data + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 0); + + Ok(()) + } + + #[tokio::test] + async fn test_single_batch() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Push one batch + let batch1 = create_test_batch(0, 10); + pool.push_batch(&batch1)?; + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 1); + assert_eq!(batches[0].num_rows(), 10); + assert_eq!(batches[0].num_columns(), 2); + + Ok(()) + } + + #[tokio::test] + async fn test_multiple_batches_single_file() -> Result<()> { + let mut pool = create_spill_pool(10 * 1024 * 1024); // Large file size + + // Push multiple batches + for i in 0..5 { + let batch = create_test_batch(i * 10, 10); + pool.push_batch(&batch)?; + } + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 5); + + // Verify FIFO order + for (i, batch) in batches.iter().enumerate() { + assert_eq!(batch.num_rows(), 10); + let col_a = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), (i as i32) * 10); + } + + Ok(()) + } + + #[tokio::test] + async fn test_file_rotation_on_size_limit() -> Result<()> { + // Small file size to force rotation + let mut pool = create_spill_pool(500); // ~500 bytes + + // Push multiple batches - should create multiple files + for i in 0..10 { + let batch = create_test_batch(i * 5, 5); + pool.push_batch(&batch)?; + } + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 10); + + // Verify all batches in FIFO order + for (i, batch) in batches.iter().enumerate() { + let col_a = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), (i as i32) * 5); + } + + Ok(()) + } + + #[tokio::test] + async fn test_empty_batches_skipped() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + let batch1 = create_test_batch(0, 10); + let empty_batch = RecordBatch::new_empty(create_test_schema()); + let batch2 = create_test_batch(10, 10); + + pool.push_batch(&batch1)?; + pool.push_batch(&empty_batch)?; // Should be skipped + pool.push_batch(&batch2)?; + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + // Should only have 2 batches (empty one skipped) + assert_eq!(batches.len(), 2); + + Ok(()) + } + + #[tokio::test] + async fn test_large_batches() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Create larger batches + let batch1 = create_test_batch(0, 1000); + let batch2 = create_test_batch(1000, 1000); + + pool.push_batch(&batch1)?; + pool.push_batch(&batch2)?; + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 1000); + assert_eq!(batches[1].num_rows(), 1000); + + Ok(()) + } + + // ============================================================================ + // Stream API Tests + // ============================================================================ + + #[tokio::test] + async fn test_stream_blocks_when_no_data() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let mut stream = SpillPool::reader(Arc::clone(&pool), spill_manager); + + // Poll should return Pending since no data and not finalized + let poll_result = futures::poll!(stream.next()); + assert!(matches!(poll_result, Poll::Pending)); + + Ok(()) + } + + #[tokio::test] + async fn test_stream_wakes_on_push() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + let pool_clone = Arc::clone(&pool_arc); + let stream = SpillPool::reader(pool_clone, spill_manager); + + // Spawn a task that will push data after a delay + let writer_pool = Arc::clone(&pool_arc); + tokio::spawn(async move { + tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; + let mut pool = writer_pool.lock(); + pool.push_batch(&create_test_batch(0, 10)).unwrap(); + pool.flush().unwrap(); + pool.finalize(); + }); + + // This should wait for data and then return it + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 1); + + Ok(()) + } + + #[tokio::test] + async fn test_stream_wakes_on_flush() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + let pool_clone = Arc::clone(&pool_arc); + let stream = SpillPool::reader(pool_clone, spill_manager); + + // Push without flush first + { + let mut pool = pool_arc.lock(); + pool.push_batch(&create_test_batch(0, 10)).unwrap(); + // Don't flush yet - data is in current_write_file + } + + // Spawn task to flush after delay + let writer_pool = Arc::clone(&pool_arc); + tokio::spawn(async move { + tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; + let mut pool = writer_pool.lock(); + pool.flush().unwrap(); + pool.finalize(); + }); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 1); + + Ok(()) + } + + #[tokio::test] + async fn test_stream_wakes_on_finalize() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + let pool_clone = Arc::clone(&pool_arc); + let mut stream = SpillPool::reader(pool_clone, spill_manager); + + // First poll should be pending + let poll_result = futures::poll!(stream.next()); + assert!(matches!(poll_result, Poll::Pending)); + + // Finalize after delay + let writer_pool = Arc::clone(&pool_arc); + tokio::spawn(async move { + tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; + writer_pool.lock().finalize(); + }); + + // Stream should eventually return None + let result = stream.next().await; + assert!(result.is_none()); + + Ok(()) + } + + #[tokio::test] + async fn test_finalize_before_flush() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Push data but DON'T flush + pool.push_batch(&create_test_batch(0, 10))?; + pool.finalize(); // Finalize without flush + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + // Data in current_write_file should still be lost since not flushed + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 0); + + Ok(()) + } + + // ============================================================================ + // Concurrent Reader/Writer Tests + // ============================================================================ + + #[tokio::test] + async fn test_concurrent_push_and_read() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + let writer_pool = Arc::clone(&pool_arc); + let writer = tokio::spawn(async move { + for i in 0..10 { + tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; + let mut pool = writer_pool.lock(); + pool.push_batch(&create_test_batch(i * 10, 10)).unwrap(); + pool.flush().unwrap(); + } + writer_pool.lock().finalize(); + }); + + let reader_pool = Arc::clone(&pool_arc); + let stream = SpillPool::reader(reader_pool, spill_manager); + let reader = tokio::spawn(async move { collect_batches(stream).await }); + + // Wait for both tasks + writer.await.unwrap(); + let batches = reader.await.unwrap()?; + + assert_eq!(batches.len(), 10); + for (i, batch) in batches.iter().enumerate() { + let col_a = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col_a.value(0), (i as i32) * 10); + } + + Ok(()) + } + + #[tokio::test] + async fn test_reader_catches_up_to_writer() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + // Start reader before any data is written + let reader_pool = Arc::clone(&pool_arc); + let mut stream = SpillPool::reader(reader_pool, spill_manager); + + // Should return pending + let poll_result = futures::poll!(stream.next()); + assert!(matches!(poll_result, Poll::Pending)); + + // Now add data + { + let mut pool = pool_arc.lock(); + pool.push_batch(&create_test_batch(0, 10))?; + pool.flush()?; + pool.finalize(); + } + + // Now stream should have data + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 1); + + Ok(()) + } + + #[tokio::test] + async fn test_multiple_readers_same_pool() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Push some batches + for i in 0..5 { + pool.push_batch(&create_test_batch(i * 10, 10))?; + } + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + // Create two readers + let stream1 = + SpillPool::reader(Arc::clone(&pool_arc), Arc::clone(&spill_manager)); + let stream2 = SpillPool::reader(Arc::clone(&pool_arc), spill_manager); + + // Read from both concurrently + let reader1 = tokio::spawn(async move { collect_batches(stream1).await }); + let reader2 = tokio::spawn(async move { collect_batches(stream2).await }); + + let batches1 = reader1.await.unwrap()?; + let batches2 = reader2.await.unwrap()?; + + // Each reader should consume different batches (pop_front removes from queue) + // The total number should be 5, but distributed between readers + let total = batches1.len() + batches2.len(); + assert_eq!(total, 5); + + Ok(()) + } + + #[tokio::test] + async fn test_file_cutover_during_read() -> Result<()> { + let pool = create_spill_pool(500); // Small size for rotation + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + let writer_pool = Arc::clone(&pool_arc); + let writer = tokio::spawn(async move { + // Write multiple batches that will cause rotation + for i in 0..8 { + { + let mut pool = writer_pool.lock(); + pool.push_batch(&create_test_batch(i * 5, 5)).unwrap(); + pool.flush().unwrap(); + } // Drop lock before sleep + tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; + } + writer_pool.lock().finalize(); + }); + + // Read concurrently + let reader_pool = Arc::clone(&pool_arc); + let stream = SpillPool::reader(reader_pool, spill_manager); + let reader = tokio::spawn(async move { collect_batches(stream).await }); + + writer.await.unwrap(); + let batches = reader.await.unwrap()?; + + // Should get all 8 batches despite file rotation + assert_eq!(batches.len(), 8); + + Ok(()) + } + + #[tokio::test] + async fn test_file_cutover_during_write() -> Result<()> { + let mut pool = create_spill_pool(300); // Very small to force frequent rotation + + // Push batches that will definitely cause rotation + for i in 0..5 { + let batch = create_test_batch(i * 10, 10); + pool.push_batch(&batch)?; + // Don't flush after each - let rotation happen naturally + } + pool.flush()?; + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool, spill_manager); + + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 5); + + Ok(()) + } + + // ============================================================================ + // Garbage Collection Tests + // ============================================================================ + + #[tokio::test] + async fn test_file_cleanup_after_read() -> Result<()> { + let mut pool = create_spill_pool(500); + + // Create multiple files + for i in 0..5 { + pool.push_batch(&create_test_batch(i * 10, 10))?; + pool.flush()?; // Each batch in its own file + } + + // Verify files exist before reading + let initial_file_count = pool.files.len(); + assert_eq!(initial_file_count, 5); + + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(Arc::clone(&pool_arc), spill_manager); + + // Read all batches + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 5); + + // All files should be consumed (dropped from queue) + let final_file_count = pool_arc.lock().files.len(); + assert_eq!(final_file_count, 0); + + Ok(()) + } + + #[tokio::test] + async fn test_cleanup_with_rotation() -> Result<()> { + let pool = create_spill_pool(400); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + // Write and read concurrently + let writer_pool = Arc::clone(&pool_arc); + let writer = tokio::spawn(async move { + for i in 0..10 { + { + let mut pool = writer_pool.lock(); + pool.push_batch(&create_test_batch(i * 10, 10)).unwrap(); + pool.flush().unwrap(); + } // Drop lock before sleep + tokio::time::sleep(tokio::time::Duration::from_millis(20)).await; + } + writer_pool.lock().finalize(); + }); + + let reader_pool = Arc::clone(&pool_arc); + let stream = SpillPool::reader(reader_pool, spill_manager); + let reader = tokio::spawn(async move { + let mut batches = Vec::new(); + let mut stream = stream; + while let Some(result) = stream.next().await { + batches.push(result.unwrap()); + // Small delay to let writer create more files + tokio::time::sleep(tokio::time::Duration::from_millis(15)).await; + } + batches + }); + + writer.await.unwrap(); + let batches = reader.await.unwrap(); + + assert_eq!(batches.len(), 10); + + // All files should be cleaned up + let final_file_count = pool_arc.lock().files.len(); + assert_eq!(final_file_count, 0); + + Ok(()) + } + + #[tokio::test] + async fn test_cleanup_with_unflushed_file() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Create some flushed files + for i in 0..3 { + pool.push_batch(&create_test_batch(i * 10, 10))?; + pool.flush()?; + } + + // Add unflushed data + pool.push_batch(&create_test_batch(30, 10))?; + // Don't flush! + + // current_write_file should have data + assert!(pool.current_write_file.is_some()); + assert_eq!(pool.files.len(), 3); + + pool.finalize(); + + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + let stream = SpillPool::reader(pool_arc, spill_manager); + + // Should only get the 3 flushed batches + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 3); + + Ok(()) + } + + // ============================================================================ + // Edge Cases & Error Handling Tests + // ============================================================================ + + #[tokio::test] + async fn test_interleaved_flush() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + // Push → flush + { + let mut pool = pool_arc.lock(); + pool.push_batch(&create_test_batch(0, 10))?; + pool.flush()?; + } + + // Read one batch + let stream = SpillPool::reader(Arc::clone(&pool_arc), Arc::clone(&spill_manager)); + let mut stream = stream; + let batch1 = stream.next().await.unwrap()?; + assert_eq!(batch1.num_rows(), 10); + + // Push → flush again + { + let mut pool = pool_arc.lock(); + pool.push_batch(&create_test_batch(10, 10))?; + pool.flush()?; + } + + // Read second batch from same stream + let batch2 = stream.next().await.unwrap()?; + assert_eq!(batch2.num_rows(), 10); + + // Finalize and verify stream ends + pool_arc.lock().finalize(); + let batch3 = stream.next().await; + assert!(batch3.is_none()); + + Ok(()) + } + + #[tokio::test] + async fn test_flush_empty_pool() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + // Flush with no data should be no-op + pool.flush()?; + pool.flush()?; // Multiple flushes + + assert_eq!(pool.files.len(), 0); + assert!(pool.current_write_file.is_none()); + + Ok(()) + } + + #[tokio::test] + async fn test_finalize_idempotent() -> Result<()> { + let mut pool = create_spill_pool(1024 * 1024); + + pool.push_batch(&create_test_batch(0, 10))?; + pool.flush()?; + + // Multiple finalize calls should be safe + pool.finalize(); + assert!(pool.is_finalized()); + pool.finalize(); + assert!(pool.is_finalized()); + pool.finalize(); + assert!(pool.is_finalized()); + + Ok(()) + } + + #[tokio::test] + async fn test_drop_flushes_current_file() -> Result<()> { + let pool = create_spill_pool(1024 * 1024); + let spill_manager = Arc::clone(&pool.spill_manager); + let pool_arc = Arc::new(Mutex::new(pool)); + + // Push without flush + { + let mut pool = pool_arc.lock(); + pool.push_batch(&create_test_batch(0, 10)).unwrap(); + pool.flush().unwrap(); + pool.finalize(); + } + + // Drop should trigger flush in Drop impl + // (though in this case we already flushed) + + let stream = SpillPool::reader(pool_arc, spill_manager); + let batches = collect_batches(stream).await?; + assert_eq!(batches.len(), 1); + + Ok(()) + } } From b25bdfa8df829ac94b86e5a7a6d9fb0b8f05c9f9 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 22 Oct 2025 14:58:48 -0500 Subject: [PATCH 07/44] fix lints --- .../physical-plan/src/repartition/mod.rs | 4 ++-- .../physical-plan/src/spill/spill_pool.rs | 23 ++++++++++--------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 0daa8a5d185bd..80c7cc6b19995 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1383,8 +1383,8 @@ impl Stream for PerPartitionStream { return Poll::Ready(Some(Err(e))); } Poll::Ready(None) => { - // Spill stream never ends, this shouldn't happen - unreachable!("SpillPoolStream should never end"); + // Spill stream ended - all spilled data has been read + return Poll::Ready(None); } Poll::Pending => { // No spilled data available diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index f30e3baed295c..4e12c476278f0 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -404,6 +404,7 @@ mod tests { use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; use arrow::array::{ArrayRef, Int32Array}; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common_runtime::SpawnedTask; use datafusion_execution::runtime_env::RuntimeEnv; use futures::StreamExt; use std::task::Poll; @@ -639,7 +640,7 @@ mod tests { // Spawn a task that will push data after a delay let writer_pool = Arc::clone(&pool_arc); - tokio::spawn(async move { + SpawnedTask::spawn(async move { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; let mut pool = writer_pool.lock(); pool.push_batch(&create_test_batch(0, 10)).unwrap(); @@ -672,7 +673,7 @@ mod tests { // Spawn task to flush after delay let writer_pool = Arc::clone(&pool_arc); - tokio::spawn(async move { + SpawnedTask::spawn(async move { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; let mut pool = writer_pool.lock(); pool.flush().unwrap(); @@ -700,7 +701,7 @@ mod tests { // Finalize after delay let writer_pool = Arc::clone(&pool_arc); - tokio::spawn(async move { + SpawnedTask::spawn(async move { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; writer_pool.lock().finalize(); }); @@ -742,7 +743,7 @@ mod tests { let pool_arc = Arc::new(Mutex::new(pool)); let writer_pool = Arc::clone(&pool_arc); - let writer = tokio::spawn(async move { + let writer = SpawnedTask::spawn(async move { for i in 0..10 { tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; let mut pool = writer_pool.lock(); @@ -754,7 +755,7 @@ mod tests { let reader_pool = Arc::clone(&pool_arc); let stream = SpillPool::reader(reader_pool, spill_manager); - let reader = tokio::spawn(async move { collect_batches(stream).await }); + let reader = SpawnedTask::spawn(async move { collect_batches(stream).await }); // Wait for both tasks writer.await.unwrap(); @@ -822,8 +823,8 @@ mod tests { let stream2 = SpillPool::reader(Arc::clone(&pool_arc), spill_manager); // Read from both concurrently - let reader1 = tokio::spawn(async move { collect_batches(stream1).await }); - let reader2 = tokio::spawn(async move { collect_batches(stream2).await }); + let reader1 = SpawnedTask::spawn(async move { collect_batches(stream1).await }); + let reader2 = SpawnedTask::spawn(async move { collect_batches(stream2).await }); let batches1 = reader1.await.unwrap()?; let batches2 = reader2.await.unwrap()?; @@ -843,7 +844,7 @@ mod tests { let pool_arc = Arc::new(Mutex::new(pool)); let writer_pool = Arc::clone(&pool_arc); - let writer = tokio::spawn(async move { + let writer = SpawnedTask::spawn(async move { // Write multiple batches that will cause rotation for i in 0..8 { { @@ -859,7 +860,7 @@ mod tests { // Read concurrently let reader_pool = Arc::clone(&pool_arc); let stream = SpillPool::reader(reader_pool, spill_manager); - let reader = tokio::spawn(async move { collect_batches(stream).await }); + let reader = SpawnedTask::spawn(async move { collect_batches(stream).await }); writer.await.unwrap(); let batches = reader.await.unwrap()?; @@ -936,7 +937,7 @@ mod tests { // Write and read concurrently let writer_pool = Arc::clone(&pool_arc); - let writer = tokio::spawn(async move { + let writer = SpawnedTask::spawn(async move { for i in 0..10 { { let mut pool = writer_pool.lock(); @@ -950,7 +951,7 @@ mod tests { let reader_pool = Arc::clone(&pool_arc); let stream = SpillPool::reader(reader_pool, spill_manager); - let reader = tokio::spawn(async move { + let reader = SpawnedTask::spawn(async move { let mut batches = Vec::new(); let mut stream = stream; while let Some(result) = stream.next().await { From 03ffc3c26df215d751deffd3e0009e5a0ad457fc Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 22 Oct 2025 16:44:04 -0500 Subject: [PATCH 08/44] bugfix --- .../physical-plan/src/repartition/mod.rs | 41 ++++++++++++------- 1 file changed, 27 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 80c7cc6b19995..7df51362f35c6 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -98,9 +98,9 @@ struct PartitionChannels { rx: InputPartitionsToCurrentPartitionReceiver, /// Memory reservation for this output partition reservation: SharedMemoryReservation, - /// SpillPool for batched spilling with file handle reuse (FIFO semantics) - /// Wrapped in Arc so it can be shared between input tasks and output streams - spill_pool: Arc>, + /// SpillPools for batched spilling - one per input partition (FIFO semantics) + /// Each (input, output) pair gets its own SpillPool to maintain proper ordering + spill_pools: Vec>>, /// SpillManager for creating streams from spill files spill_manager: Arc, } @@ -251,14 +251,23 @@ impl RepartitionExecState { input.schema(), )); - // Create SpillPool with configured max file size + // Create one SpillPool per input partition for this output partition + // This ensures proper FIFO ordering within each (input, output) pair let max_file_size = context .session_config() .options() .execution .max_spill_file_size_bytes; - let spill_pool = - SpillPool::new(max_file_size, Arc::clone(&spill_manager), input.schema()); + let spill_pools: Vec<_> = (0..num_input_partitions) + .map(|_| { + let spill_pool = SpillPool::new( + max_file_size, + Arc::clone(&spill_manager), + input.schema(), + ); + Arc::new(Mutex::new(spill_pool)) + }) + .collect(); channels.insert( partition, @@ -266,7 +275,7 @@ impl RepartitionExecState { tx, rx, reservation, - spill_pool: Arc::new(Mutex::new(spill_pool)), + spill_pools, spill_manager, }, ); @@ -285,7 +294,7 @@ impl RepartitionExecState { OutputChannel { sender: channels.tx[i].clone(), reservation: Arc::clone(&channels.reservation), - spill_pool: Arc::clone(&channels.spill_pool), + spill_pool: Arc::clone(&channels.spill_pools[i]), }, ) }) @@ -744,7 +753,7 @@ impl ExecutionPlan for RepartitionExec { let num_input_partitions = input.output_partitioning().partition_count(); // lock scope - let (mut rx, reservation, spill_pool, spill_manager, abort_helper) = { + let (mut rx, reservation, spill_pools, spill_manager, abort_helper) = { // lock mutexes let mut state = state.lock(); let state = state.consume_input_streams( @@ -761,7 +770,7 @@ impl ExecutionPlan for RepartitionExec { let PartitionChannels { rx, reservation, - spill_pool, + spill_pools, spill_manager, .. } = state @@ -772,7 +781,7 @@ impl ExecutionPlan for RepartitionExec { ( rx, reservation, - spill_pool, + spill_pools, spill_manager, Arc::clone(&state.abort_helper), ) @@ -784,10 +793,12 @@ impl ExecutionPlan for RepartitionExec { if preserve_order { // Store streams from all the input partitions: - // All streams share the same SpillPool from PartitionChannels + // Each input partition gets its own SpillPool to maintain proper FIFO ordering let input_streams = rx .into_iter() - .map(|receiver| { + .enumerate() + .map(|(idx, receiver)| { + let spill_pool = Arc::clone(&spill_pools[idx]); let spill_stream = SpillPool::reader( Arc::clone(&spill_pool), Arc::clone(&spill_manager), @@ -798,7 +809,7 @@ impl ExecutionPlan for RepartitionExec { receiver, _drop_helper: Arc::clone(&abort_helper), reservation: Arc::clone(&reservation), - spill_pool: Arc::clone(&spill_pool), + spill_pool, spill_stream, input_finished: false, }) as SendableRecordBatchStream @@ -822,6 +833,8 @@ impl ExecutionPlan for RepartitionExec { .with_reservation(merge_reservation) .build() } else { + // Non-preserve-order case: single input stream, so use the first SpillPool + let spill_pool = Arc::clone(&spill_pools[0]); let spill_stream = SpillPool::reader( Arc::clone(&spill_pool), Arc::clone(&spill_manager), From 966a2137c2f375100493da750f19d76faa34e6fb Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 22 Oct 2025 16:51:37 -0500 Subject: [PATCH 09/44] address pr feedback --- .../physical-plan/src/repartition/mod.rs | 21 +++++++++++++++---- .../physical-plan/src/spill/spill_pool.rs | 8 ++----- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 7df51362f35c6..57a96f0fa0c9a 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1278,7 +1278,16 @@ impl Stream for RepartitionStream { Poll::Pending => { // No spilled data available right now if self.all_inputs_finished { - // All inputs finished, wait for spill stream to have more data or finish + // All inputs finished, verify pool is finalized before waiting + // If not finalized, we may hang indefinitely + if !self.spill_pool.lock().is_finalized() { + return Poll::Ready(Some(Err( + datafusion_common::internal_err!( + "Spill pool not finalized despite all inputs finishing" + ), + ))); + } + // Pool is finalized, wait for spill stream to have more data or finish return Poll::Pending; } // Otherwise check the channel @@ -1325,7 +1334,9 @@ impl Stream for RepartitionStream { // Flush and finalize the SpillPool { let mut pool = self.spill_pool.lock(); - pool.flush().ok(); + if let Err(e) = pool.flush() { + return Poll::Ready(Some(Err(e))); + } pool.finalize(); } // Drop the lock before continuing self.all_inputs_finished = true; @@ -1411,7 +1422,7 @@ impl Stream for PerPartitionStream { // If input is finished, don't poll channel anymore if self.input_finished { - continue; + return Poll::Pending; } // Try to get next item from channel @@ -1446,7 +1457,9 @@ impl Stream for PerPartitionStream { // Flush and finalize the SpillPool { let mut pool = self.spill_pool.lock(); - pool.flush().ok(); + if let Err(e) = pool.flush() { + return Poll::Ready(Some(Err(e))); + } pool.finalize(); } // Drop the lock before continuing self.input_finished = true; diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 4e12c476278f0..c24292949f254 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -72,7 +72,7 @@ struct SpillFile { } impl SpillFile { - fn new(file: RefCountedTempFile, _total_batches: usize, _total_size: usize) -> Self { + fn new(file: RefCountedTempFile) -> Self { Self { file } } } @@ -255,12 +255,8 @@ impl SpillPool { let finished_file = file.finish()?; if let Some(temp_file) = finished_file { - // Get actual file size - let actual_size = temp_file.current_disk_usage() as usize; - // Create SpillFile and add to queue - let spill_file = - SpillFile::new(temp_file, self.current_batch_count, actual_size); + let spill_file = SpillFile::new(temp_file); self.files.push_back(spill_file); } From 1dbc730bc9ea680ab20869ea866ab9a4639eb813 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 22 Oct 2025 23:23:22 -0500 Subject: [PATCH 10/44] fix build --- datafusion/physical-plan/src/repartition/mod.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 57a96f0fa0c9a..48093c46239bc 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1281,11 +1281,10 @@ impl Stream for RepartitionStream { // All inputs finished, verify pool is finalized before waiting // If not finalized, we may hang indefinitely if !self.spill_pool.lock().is_finalized() { - return Poll::Ready(Some(Err( - datafusion_common::internal_err!( + return Poll::Ready(Some(Err(DataFusionError::Internal( "Spill pool not finalized despite all inputs finishing" - ), - ))); + .to_string(), + )))); } // Pool is finalized, wait for spill stream to have more data or finish return Poll::Pending; From e2ed52ddb0c501556dca71ae872b7dc6f71e41fb Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 22 Oct 2025 23:47:09 -0500 Subject: [PATCH 11/44] fix dropped task --- datafusion/physical-plan/src/spill/spill_pool.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index c24292949f254..0d452ab7c12e4 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -636,7 +636,7 @@ mod tests { // Spawn a task that will push data after a delay let writer_pool = Arc::clone(&pool_arc); - SpawnedTask::spawn(async move { + let _writer = SpawnedTask::spawn(async move { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; let mut pool = writer_pool.lock(); pool.push_batch(&create_test_batch(0, 10)).unwrap(); @@ -669,7 +669,7 @@ mod tests { // Spawn task to flush after delay let writer_pool = Arc::clone(&pool_arc); - SpawnedTask::spawn(async move { + let _writer = SpawnedTask::spawn(async move { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; let mut pool = writer_pool.lock(); pool.flush().unwrap(); @@ -697,7 +697,7 @@ mod tests { // Finalize after delay let writer_pool = Arc::clone(&pool_arc); - SpawnedTask::spawn(async move { + let _writer = SpawnedTask::spawn(async move { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; writer_pool.lock().finalize(); }); From c850240faa5671e04a17e17f7e70088640ebe26a Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 23 Oct 2025 00:18:18 -0500 Subject: [PATCH 12/44] updarte docstrings --- .../physical-plan/src/repartition/mod.rs | 8 ++--- .../physical-plan/src/spill/spill_pool.rs | 31 +++++++++++++------ 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 48093c46239bc..b1f1166902e95 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -74,7 +74,7 @@ enum RepartitionBatch { /// Batch held in memory (counts against memory reservation) Memory(RecordBatch), /// Marker indicating a batch was spilled to the partition's SpillPool - /// The actual batch can be retrieved via SpillPool::pop_batch() + /// The actual batch can be retrieved by reading from the SpillPoolStream Spilled, } @@ -110,7 +110,7 @@ struct ConsumingInputStreamsState { /// Key is the partition number. channels: HashMap, - /// Helper that ensures that that background job is killed once it is no longer needed. + /// Helper that ensures that background jobs are killed once they are no longer needed. abort_helper: Arc>>, } @@ -622,7 +622,7 @@ impl RepartitionExec { &self.cache.partitioning } - /// Get preserve_order flag of the RepartitionExecutor + /// Get preserve_order flag of the RepartitionExec /// `true` means `SortPreservingRepartitionExec`, `false` means `RepartitionExec` pub fn preserve_order(&self) -> bool { self.preserve_order @@ -1085,7 +1085,7 @@ impl RepartitionExec { /// Pulls data from the specified input plan, feeding it to the /// output partitions based on the desired partitioning /// - /// txs hold the output sending channels for each output partition + /// `output_channels` holds the output sending channels for each output partition async fn pull_from_input( mut stream: SendableRecordBatchStream, mut output_channels: HashMap, diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 0d452ab7c12e4..fdb4ce7e69f06 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -34,20 +34,30 @@ //! # Usage Example //! //! ```ignore +//! use std::sync::Arc; +//! use parking_lot::Mutex; +//! //! let pool = SpillPool::new( //! 100 * 1024 * 1024, // 100MB max per file //! spill_manager, //! schema, //! ); +//! let pool = Arc::new(Mutex::new(pool)); //! //! // Spill batches - automatically rotates files when size limit reached -//! pool.push_batch(batch1)?; -//! pool.push_batch(batch2)?; -//! pool.flush()?; // Finalize current file +//! { +//! let mut pool = pool.lock(); +//! pool.push_batch(batch1)?; +//! pool.push_batch(batch2)?; +//! pool.flush()?; // Finalize current file +//! pool.finalize(); // Signal no more writes +//! } //! -//! // Read back in FIFO order -//! let batch = pool.pop_batch()?.unwrap(); // Returns batch1 -//! let batch = pool.pop_batch()?.unwrap(); // Returns batch2 +//! // Read back in FIFO order using a stream +//! let mut stream = SpillPool::reader(pool, spill_manager); +//! let batch1 = stream.next().await.unwrap()?; // Returns batch1 +//! let batch2 = stream.next().await.unwrap()?; // Returns batch2 +//! // stream.next() returns None after finalize //! ``` use std::collections::VecDeque; @@ -147,7 +157,7 @@ impl SpillPool { self.finalized } - /// Creates an infinite stream reader for this pool. + /// Creates a stream reader for this pool. /// /// The stream automatically handles file rotation and can read concurrently /// while writes continue to the pool. When the stream catches up to the writer, @@ -160,7 +170,8 @@ impl SpillPool { /// /// # Returns /// - /// An infinite `SpillPoolStream` that never ends until dropped + /// A `SpillPoolStream` that returns batches in FIFO order and ends when the pool + /// is finalized and all data has been read pub fn reader( pool: Arc>, spill_manager: Arc, @@ -280,13 +291,13 @@ impl Drop for SpillPool { } } -/// An infinite stream that reads from a SpillPool. +/// A stream that reads from a SpillPool in FIFO order. /// /// The stream automatically handles file rotation and reads from completed files. /// When no completed files are available, it returns `Poll::Pending` and waits /// for the writer to complete more files. /// -/// The stream never ends (`Poll::Ready(None)`) until it is dropped. +/// The stream ends (`Poll::Ready(None)`) when the pool is finalized and all data has been read. pub struct SpillPoolStream { /// Shared reference to the spill pool spill_pool: Arc>, From 303258e15093f29b92ddf481092e5701cab32ab2 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 23 Oct 2025 00:20:22 -0500 Subject: [PATCH 13/44] remove wrapper struct --- .../physical-plan/src/spill/spill_pool.rs | 25 ++++--------------- 1 file changed, 5 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index fdb4ce7e69f06..f388e34a6ca18 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -75,18 +75,6 @@ use datafusion_execution::SendableRecordBatchStream; use super::in_progress_spill_file::InProgressSpillFile; use super::spill_manager::SpillManager; -/// A single spill file containing one or more record batches. -struct SpillFile { - /// The temp file handle (auto-deletes when dropped) - file: RefCountedTempFile, -} - -impl SpillFile { - fn new(file: RefCountedTempFile) -> Self { - Self { file } - } -} - /// A pool of spill files that manages batch-level spilling with FIFO semantics. /// /// Batches are written sequentially to files, with automatic rotation when the @@ -101,7 +89,7 @@ pub struct SpillPool { /// Maximum size in bytes before rotating to a new file max_file_size_bytes: usize, /// Queue of spill files (front = oldest, back = newest) - files: VecDeque, + files: VecDeque, /// Current file being written to (if any) current_write_file: Option, /// Size of current write file in bytes (estimated) @@ -110,8 +98,7 @@ pub struct SpillPool { current_batch_count: usize, /// SpillManager for creating files and tracking metrics spill_manager: Arc, - /// Schema for batches (kept for potential validation in debug builds) - #[allow(dead_code)] + /// Schema for batches (used by SpillPoolStream to implement RecordBatchStream) schema: SchemaRef, /// Wakers to notify when new data is available for readers wakers: Vec, @@ -266,9 +253,8 @@ impl SpillPool { let finished_file = file.finish()?; if let Some(temp_file) = finished_file { - // Create SpillFile and add to queue - let spill_file = SpillFile::new(temp_file); - self.files.push_back(spill_file); + // Add to queue + self.files.push_back(temp_file); } // Reset write state @@ -370,9 +356,8 @@ impl futures::Stream for SpillPoolStream { // Only read from completed files in the queue let mut pool = self.spill_pool.lock(); - if let Some(spill_file) = pool.files.pop_front() { + if let Some(file) = pool.files.pop_front() { // We have a completed file to read - let file = spill_file.file; drop(pool); // Release lock before creating stream match self.spill_manager.read_spill_as_stream(file, None) { From ab7f350220b801dac5fb2bc192471cdc68863e95 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 23 Oct 2025 00:25:16 -0500 Subject: [PATCH 14/44] hide stream behind a trait to avoid making more public stuff --- datafusion/physical-plan/src/repartition/mod.rs | 6 +++--- datafusion/physical-plan/src/spill/spill_pool.rs | 14 ++++++++------ 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index b1f1166902e95..9c5d52d45d1e8 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -39,7 +39,7 @@ use crate::repartition::distributor_channels::{ }; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; -use crate::spill::spill_pool::{SpillPool, SpillPoolStream}; +use crate::spill::spill_pool::SpillPool; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -1246,7 +1246,7 @@ struct RepartitionStream { spill_pool: Arc>, /// Infinite stream for reading from the spill pool - spill_stream: SpillPoolStream, + spill_stream: SendableRecordBatchStream, /// Flag indicating all inputs have finished all_inputs_finished: bool, @@ -1380,7 +1380,7 @@ struct PerPartitionStream { spill_pool: Arc>, /// Infinite stream for reading from the spill pool - spill_stream: SpillPoolStream, + spill_stream: SendableRecordBatchStream, /// Flag indicating input partition has finished input_finished: bool, diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index f388e34a6ca18..43ba05d67f15a 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -70,7 +70,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::disk_manager::RefCountedTempFile; -use datafusion_execution::SendableRecordBatchStream; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; use super::in_progress_spill_file::InProgressSpillFile; use super::spill_manager::SpillManager; @@ -162,8 +162,8 @@ impl SpillPool { pub fn reader( pool: Arc>, spill_manager: Arc, - ) -> SpillPoolStream { - SpillPoolStream::new(pool, spill_manager) + ) -> SendableRecordBatchStream { + Box::pin(SpillPoolStream::new(pool, spill_manager)) } /// Spills a batch to the pool, rotating files when necessary. @@ -284,7 +284,7 @@ impl Drop for SpillPool { /// for the writer to complete more files. /// /// The stream ends (`Poll::Ready(None)`) when the pool is finalized and all data has been read. -pub struct SpillPoolStream { +struct SpillPoolStream { /// Shared reference to the spill pool spill_pool: Arc>, /// SpillManager for creating streams from spill files @@ -384,7 +384,7 @@ impl futures::Stream for SpillPoolStream { } } -impl datafusion_execution::RecordBatchStream for SpillPoolStream { +impl RecordBatchStream for SpillPoolStream { fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) } @@ -437,7 +437,9 @@ mod tests { } /// Helper to collect all batches from a stream - async fn collect_batches(mut stream: SpillPoolStream) -> Result> { + async fn collect_batches( + mut stream: SendableRecordBatchStream, + ) -> Result> { let mut batches = Vec::new(); while let Some(result) = stream.next().await { batches.push(result?); From 9033e842dfd8a8894a23f7a2571e2cf1d43263e5 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 10:38:03 -0500 Subject: [PATCH 15/44] wip on cleanup --- datafusion/common/src/config.rs | 2 +- .../physical-plan/src/repartition/mod.rs | 113 +++++++++++++--- .../src/spill/in_progress_spill_file.rs | 20 +++ .../physical-plan/src/spill/spill_manager.rs | 5 + .../physical-plan/src/spill/spill_pool.rs | 122 +++++------------- 5 files changed, 153 insertions(+), 109 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 7953493d787e0..4389e515efe74 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -527,7 +527,7 @@ config_namespace! { /// /// A larger value reduces file creation overhead but may hold more disk space. /// A smaller value creates more files but allows finer-grained space reclamation - /// (especially in LIFO mode where files are truncated after reading). + /// as files can be deleted once fully consumed. /// /// Default: 100 MB pub max_spill_file_size_bytes: usize, default = 100 * 1024 * 1024 diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 9c5d52d45d1e8..a68f9057e1b1f 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -101,8 +101,6 @@ struct PartitionChannels { /// SpillPools for batched spilling - one per input partition (FIFO semantics) /// Each (input, output) pair gets its own SpillPool to maintain proper ordering spill_pools: Vec>>, - /// SpillManager for creating streams from spill files - spill_manager: Arc, } struct ConsumingInputStreamsState { @@ -260,11 +258,8 @@ impl RepartitionExecState { .max_spill_file_size_bytes; let spill_pools: Vec<_> = (0..num_input_partitions) .map(|_| { - let spill_pool = SpillPool::new( - max_file_size, - Arc::clone(&spill_manager), - input.schema(), - ); + let spill_pool = + SpillPool::new(max_file_size, Arc::clone(&spill_manager)); Arc::new(Mutex::new(spill_pool)) }) .collect(); @@ -276,7 +271,6 @@ impl RepartitionExecState { rx, reservation, spill_pools, - spill_manager, }, ); } @@ -753,7 +747,7 @@ impl ExecutionPlan for RepartitionExec { let num_input_partitions = input.output_partitioning().partition_count(); // lock scope - let (mut rx, reservation, spill_pools, spill_manager, abort_helper) = { + let (mut rx, reservation, spill_pools, abort_helper) = { // lock mutexes let mut state = state.lock(); let state = state.consume_input_streams( @@ -771,7 +765,6 @@ impl ExecutionPlan for RepartitionExec { rx, reservation, spill_pools, - spill_manager, .. } = state .channels @@ -782,7 +775,6 @@ impl ExecutionPlan for RepartitionExec { rx, reservation, spill_pools, - spill_manager, Arc::clone(&state.abort_helper), ) }; @@ -799,10 +791,7 @@ impl ExecutionPlan for RepartitionExec { .enumerate() .map(|(idx, receiver)| { let spill_pool = Arc::clone(&spill_pools[idx]); - let spill_stream = SpillPool::reader( - Arc::clone(&spill_pool), - Arc::clone(&spill_manager), - ); + let spill_stream = SpillPool::reader(Arc::clone(&spill_pool)); Box::pin(PerPartitionStream { schema: Arc::clone(&schema_captured), @@ -835,10 +824,7 @@ impl ExecutionPlan for RepartitionExec { } else { // Non-preserve-order case: single input stream, so use the first SpillPool let spill_pool = Arc::clone(&spill_pools[0]); - let spill_stream = SpillPool::reader( - Arc::clone(&spill_pool), - Arc::clone(&spill_manager), - ); + let spill_stream = SpillPool::reader(Arc::clone(&spill_pool)); Ok(Box::pin(RepartitionStream { num_input_partitions, @@ -2229,6 +2215,7 @@ mod tests { mod test { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::assert_batches_eq; use super::*; use crate::test::TestMemoryExec; @@ -2312,6 +2299,94 @@ mod test { Ok(()) } + #[tokio::test] + async fn test_preserve_order_with_spilling() -> Result<()> { + use arrow::array::UInt32Array; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_execution::TaskContext; + + // Test that order-preserving repartition successfully spills to disk + // when memory is constrained while maintaining correct order + let schema = test_schema(); + let sort_exprs = sort_exprs(&schema); + + // Create sorted input data across multiple partitions + // Each partition has sorted data: [1,2,3,4,5,6,7,8] + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))], + )?; + let partition1 = vec![batch.clone(); 25]; + let partition2 = vec![batch; 25]; + let input_partitions = vec![partition1, partition2]; + + // Set up context with very tight memory limit to force spilling + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; + + let task_ctx = TaskContext::default().with_runtime(runtime); + let task_ctx = Arc::new(task_ctx); + + // Create physical plan with order preservation + let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)? + .try_with_sort_information(vec![sort_exprs.clone(), sort_exprs])?; + let exec = Arc::new(TestMemoryExec::update_cache(Arc::new(exec))); + let exec = RepartitionExec::try_new(exec, Partitioning::RoundRobinBatch(4))? + .with_preserve_order(); + + let mut batches = vec![]; + + // Collect all partitions - should succeed by spilling to disk + let mut total_rows = 0; + for i in 0..exec.partitioning().partition_count() { + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; + while let Some(result) = stream.next().await { + let batch = result?; + total_rows += batch.num_rows(); + batches.push(batch); + } + } + + // Verify we got all the data (2 partitions * 25 batches * 8 rows each) + assert_eq!(total_rows, 2 * 25 * 8); + + // Verify spilling metrics to confirm spilling actually happened + let metrics = exec.metrics().unwrap(); + assert!( + metrics.spill_count().unwrap() > 0, + "Expected spill_count > 0 for order-preserving repartition, but got {:?}", + metrics.spill_count() + ); + assert!( + metrics.spilled_bytes().unwrap() > 0, + "Expected spilled_bytes > 0 for order-preserving repartition" + ); + assert!( + metrics.spilled_rows().unwrap() > 0, + "Expected spilled_rows > 0 for order-preserving repartition" + ); + + // Verify that the final output batches are in sorted order + #[rustfmt::skip] + let expected = [ + "+----+", + "| c0 |", + "+----+", + "| 1 |", + "| 2 |", + "| 3 |", + "| 4 |", + "| 5 |", + "| 6 |", + "| 7 |", + "| 8 |", + ]; + assert_batches_eq!(&expected, &batches); + + Ok(()) + } + #[tokio::test] async fn test_repartition() -> Result<()> { let schema = test_schema(); diff --git a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs index 14917e23b7921..3149c829c7262 100644 --- a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs +++ b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs @@ -35,6 +35,10 @@ pub struct InProgressSpillFile { writer: Option, /// Lazily initialized in-progress file, it will be moved out when the `finish` method is invoked in_progress_file: Option, + /// Number of batches written to this file + batch_count: usize, + /// Estimated size of data written to this file in bytes + estimated_size: usize, } impl InProgressSpillFile { @@ -46,6 +50,8 @@ impl InProgressSpillFile { spill_writer, in_progress_file: Some(in_progress_file), writer: None, + batch_count: 0, + estimated_size: 0, } } @@ -84,6 +90,10 @@ impl InProgressSpillFile { // Update metrics self.spill_writer.metrics.spilled_rows.add(spilled_rows); + + // Update stats + self.batch_count += 1; + self.estimated_size += batch.get_array_memory_size(); } Ok(()) } @@ -107,4 +117,14 @@ impl InProgressSpillFile { Ok(self.in_progress_file.take()) } + + /// Returns the number of batches written to this file + pub fn batch_count(&self) -> usize { + self.batch_count + } + + /// Returns the estimated size of data written to this file in bytes + pub fn estimated_size(&self) -> usize { + self.estimated_size + } } diff --git a/datafusion/physical-plan/src/spill/spill_manager.rs b/datafusion/physical-plan/src/spill/spill_manager.rs index cc39102d89819..788746fe2e353 100644 --- a/datafusion/physical-plan/src/spill/spill_manager.rs +++ b/datafusion/physical-plan/src/spill/spill_manager.rs @@ -72,6 +72,11 @@ impl SpillManager { self } + /// Returns the schema for batches managed by this SpillManager + pub fn schema(&self) -> SchemaRef { + self.schema.clone() + } + /// Creates a temporary file for in-progress operations, returning an error /// message if file creation fails. The file can be used to append batches /// incrementally and then finish the file when done. diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 43ba05d67f15a..7636f2a51c9e2 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -40,7 +40,6 @@ //! let pool = SpillPool::new( //! 100 * 1024 * 1024, // 100MB max per file //! spill_manager, -//! schema, //! ); //! let pool = Arc::new(Mutex::new(pool)); //! @@ -54,7 +53,7 @@ //! } //! //! // Read back in FIFO order using a stream -//! let mut stream = SpillPool::reader(pool, spill_manager); +//! let mut stream = SpillPool::reader(pool); //! let batch1 = stream.next().await.unwrap()?; // Returns batch1 //! let batch2 = stream.next().await.unwrap()?; // Returns batch2 //! // stream.next() returns None after finalize @@ -92,14 +91,8 @@ pub struct SpillPool { files: VecDeque, /// Current file being written to (if any) current_write_file: Option, - /// Size of current write file in bytes (estimated) - current_write_size: usize, - /// Number of batches written to current file - current_batch_count: usize, /// SpillManager for creating files and tracking metrics spill_manager: Arc, - /// Schema for batches (used by SpillPoolStream to implement RecordBatchStream) - schema: SchemaRef, /// Wakers to notify when new data is available for readers wakers: Vec, /// Flag indicating no more writes will occur @@ -113,20 +106,12 @@ impl SpillPool { /// /// * `max_file_size_bytes` - Maximum size per file before rotation (e.g., 100MB) /// * `spill_manager` - Manager for file creation and metrics - /// * `schema` - Schema for record batches - pub fn new( - max_file_size_bytes: usize, - spill_manager: Arc, - schema: SchemaRef, - ) -> Self { + pub fn new(max_file_size_bytes: usize, spill_manager: Arc) -> Self { Self { max_file_size_bytes, files: VecDeque::new(), current_write_file: None, - current_write_size: 0, - current_batch_count: 0, spill_manager, - schema, wakers: Vec::new(), finalized: false, } @@ -153,17 +138,13 @@ impl SpillPool { /// # Arguments /// /// * `pool` - Shared reference to the SpillPool - /// * `spill_manager` - Manager for creating streams from spill files /// /// # Returns /// /// A `SpillPoolStream` that returns batches in FIFO order and ends when the pool /// is finalized and all data has been read - pub fn reader( - pool: Arc>, - spill_manager: Arc, - ) -> SendableRecordBatchStream { - Box::pin(SpillPoolStream::new(pool, spill_manager)) + pub fn reader(pool: Arc>) -> SendableRecordBatchStream { + Box::pin(SpillPoolStream::new(pool)) } /// Spills a batch to the pool, rotating files when necessary. @@ -183,9 +164,9 @@ impl SpillPool { let batch_size = batch.get_array_memory_size(); // Check if we need to rotate to a new file - let needs_rotation = if self.current_write_file.is_some() { + let needs_rotation = if let Some(ref file) = self.current_write_file { // Rotate if adding this batch would exceed the max file size - self.current_write_size + batch_size > self.max_file_size_bytes + file.estimated_size() + batch_size > self.max_file_size_bytes } else { // No current file, need to create one true @@ -200,8 +181,6 @@ impl SpillPool { if self.current_write_file.is_none() { self.current_write_file = Some(self.spill_manager.create_in_progress_file("SpillPool")?); - self.current_write_size = 0; - self.current_batch_count = 0; } // Append batch to current file @@ -209,9 +188,6 @@ impl SpillPool { file.append_batch(batch)?; } - self.current_write_size += batch_size; - self.current_batch_count += 1; - // Wake any waiting readers self.wake(); @@ -257,10 +233,6 @@ impl SpillPool { self.files.push_back(temp_file); } - // Reset write state - self.current_write_size = 0; - self.current_batch_count = 0; - // Wake any waiting readers since a new complete file is available self.wake(); } @@ -287,8 +259,6 @@ impl Drop for SpillPool { struct SpillPoolStream { /// Shared reference to the spill pool spill_pool: Arc>, - /// SpillManager for creating streams from spill files - spill_manager: Arc, /// Current stream being read from current_stream: Option, /// Schema for the batches @@ -301,19 +271,14 @@ impl SpillPoolStream { /// # Arguments /// /// * `spill_pool` - Shared reference to the pool to read from - /// * `spill_manager` - Manager for creating streams from spill files - pub fn new( - spill_pool: Arc>, - spill_manager: Arc, - ) -> Self { + pub fn new(spill_pool: Arc>) -> Self { let schema = { let pool = spill_pool.lock(); - Arc::clone(&pool.schema) + pool.spill_manager.schema() }; Self { spill_pool, - spill_manager, current_stream: None, schema, } @@ -358,9 +323,10 @@ impl futures::Stream for SpillPoolStream { if let Some(file) = pool.files.pop_front() { // We have a completed file to read + let spill_manager = Arc::clone(&pool.spill_manager); drop(pool); // Release lock before creating stream - match self.spill_manager.read_spill_as_stream(file, None) { + match spill_manager.read_spill_as_stream(file, None) { Ok(stream) => { self.current_stream = Some(stream); // Continue loop to poll the new stream @@ -433,7 +399,7 @@ mod tests { let spill_manager = Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); - SpillPool::new(max_file_size, spill_manager, schema) + SpillPool::new(max_file_size, spill_manager) } /// Helper to collect all batches from a stream @@ -456,9 +422,8 @@ mod tests { let mut pool = create_spill_pool(1024 * 1024); pool.finalize(); // Mark as done with no data - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 0); @@ -476,9 +441,8 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 1); @@ -500,9 +464,8 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 5); @@ -534,9 +497,8 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 10); @@ -568,9 +530,8 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; // Should only have 2 batches (empty one skipped) @@ -592,9 +553,8 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 2); @@ -612,9 +572,8 @@ mod tests { async fn test_stream_blocks_when_no_data() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let mut stream = SpillPool::reader(Arc::clone(&pool), spill_manager); + let mut stream = SpillPool::reader(Arc::clone(&pool)); // Poll should return Pending since no data and not finalized let poll_result = futures::poll!(stream.next()); @@ -626,11 +585,10 @@ mod tests { #[tokio::test] async fn test_stream_wakes_on_push() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); let pool_clone = Arc::clone(&pool_arc); - let stream = SpillPool::reader(pool_clone, spill_manager); + let stream = SpillPool::reader(pool_clone); // Spawn a task that will push data after a delay let writer_pool = Arc::clone(&pool_arc); @@ -652,11 +610,10 @@ mod tests { #[tokio::test] async fn test_stream_wakes_on_flush() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); let pool_clone = Arc::clone(&pool_arc); - let stream = SpillPool::reader(pool_clone, spill_manager); + let stream = SpillPool::reader(pool_clone); // Push without flush first { @@ -683,11 +640,10 @@ mod tests { #[tokio::test] async fn test_stream_wakes_on_finalize() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); let pool_clone = Arc::clone(&pool_arc); - let mut stream = SpillPool::reader(pool_clone, spill_manager); + let mut stream = SpillPool::reader(pool_clone); // First poll should be pending let poll_result = futures::poll!(stream.next()); @@ -715,9 +671,8 @@ mod tests { pool.push_batch(&create_test_batch(0, 10))?; pool.finalize(); // Finalize without flush - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); // Data in current_write_file should still be lost since not flushed let batches = collect_batches(stream).await?; @@ -733,7 +688,6 @@ mod tests { #[tokio::test] async fn test_concurrent_push_and_read() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); let writer_pool = Arc::clone(&pool_arc); @@ -748,7 +702,7 @@ mod tests { }); let reader_pool = Arc::clone(&pool_arc); - let stream = SpillPool::reader(reader_pool, spill_manager); + let stream = SpillPool::reader(reader_pool); let reader = SpawnedTask::spawn(async move { collect_batches(stream).await }); // Wait for both tasks @@ -771,12 +725,11 @@ mod tests { #[tokio::test] async fn test_reader_catches_up_to_writer() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); // Start reader before any data is written let reader_pool = Arc::clone(&pool_arc); - let mut stream = SpillPool::reader(reader_pool, spill_manager); + let mut stream = SpillPool::reader(reader_pool); // Should return pending let poll_result = futures::poll!(stream.next()); @@ -808,13 +761,11 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); // Create two readers - let stream1 = - SpillPool::reader(Arc::clone(&pool_arc), Arc::clone(&spill_manager)); - let stream2 = SpillPool::reader(Arc::clone(&pool_arc), spill_manager); + let stream1 = SpillPool::reader(Arc::clone(&pool_arc)); + let stream2 = SpillPool::reader(Arc::clone(&pool_arc)); // Read from both concurrently let reader1 = SpawnedTask::spawn(async move { collect_batches(stream1).await }); @@ -834,7 +785,6 @@ mod tests { #[tokio::test] async fn test_file_cutover_during_read() -> Result<()> { let pool = create_spill_pool(500); // Small size for rotation - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); let writer_pool = Arc::clone(&pool_arc); @@ -853,7 +803,7 @@ mod tests { // Read concurrently let reader_pool = Arc::clone(&pool_arc); - let stream = SpillPool::reader(reader_pool, spill_manager); + let stream = SpillPool::reader(reader_pool); let reader = SpawnedTask::spawn(async move { collect_batches(stream).await }); writer.await.unwrap(); @@ -878,9 +828,8 @@ mod tests { pool.flush()?; pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool, spill_manager); + let stream = SpillPool::reader(pool); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 5); @@ -908,9 +857,8 @@ mod tests { pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(Arc::clone(&pool_arc), spill_manager); + let stream = SpillPool::reader(Arc::clone(&pool_arc)); // Read all batches let batches = collect_batches(stream).await?; @@ -926,7 +874,6 @@ mod tests { #[tokio::test] async fn test_cleanup_with_rotation() -> Result<()> { let pool = create_spill_pool(400); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); // Write and read concurrently @@ -944,7 +891,7 @@ mod tests { }); let reader_pool = Arc::clone(&pool_arc); - let stream = SpillPool::reader(reader_pool, spill_manager); + let stream = SpillPool::reader(reader_pool); let reader = SpawnedTask::spawn(async move { let mut batches = Vec::new(); let mut stream = stream; @@ -988,9 +935,8 @@ mod tests { pool.finalize(); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool_arc, spill_manager); + let stream = SpillPool::reader(pool_arc); // Should only get the 3 flushed batches let batches = collect_batches(stream).await?; @@ -1006,7 +952,6 @@ mod tests { #[tokio::test] async fn test_interleaved_flush() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); // Push → flush @@ -1017,7 +962,7 @@ mod tests { } // Read one batch - let stream = SpillPool::reader(Arc::clone(&pool_arc), Arc::clone(&spill_manager)); + let stream = SpillPool::reader(Arc::clone(&pool_arc)); let mut stream = stream; let batch1 = stream.next().await.unwrap()?; assert_eq!(batch1.num_rows(), 10); @@ -1076,7 +1021,6 @@ mod tests { #[tokio::test] async fn test_drop_flushes_current_file() -> Result<()> { let pool = create_spill_pool(1024 * 1024); - let spill_manager = Arc::clone(&pool.spill_manager); let pool_arc = Arc::new(Mutex::new(pool)); // Push without flush @@ -1090,7 +1034,7 @@ mod tests { // Drop should trigger flush in Drop impl // (though in this case we already flushed) - let stream = SpillPool::reader(pool_arc, spill_manager); + let stream = SpillPool::reader(pool_arc); let batches = collect_batches(stream).await?; assert_eq!(batches.len(), 1); From cccbd73070fe9c3d58c62d20a87d79d96abdef70 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 11:46:59 -0500 Subject: [PATCH 16/44] add a detailed test --- .../physical-plan/src/repartition/mod.rs | 173 ++++++++++++------ 1 file changed, 116 insertions(+), 57 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index a68f9057e1b1f..08f37e4921c07 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -191,6 +191,7 @@ impl RepartitionExecState { preserve_order: bool, name: String, context: Arc, + spill_manager: SpillManager, ) -> Result<&mut ConsumingInputStreamsState> { let streams_and_metrics = match self { RepartitionExecState::NotInitialized => { @@ -214,6 +215,8 @@ impl RepartitionExecState { let num_input_partitions = streams_and_metrics.len(); let num_output_partitions = partitioning.partition_count(); + let spill_manager = Arc::new(spill_manager); + let (txs, rxs) = if preserve_order { let (txs, rxs) = partition_aware_channels(num_input_partitions, num_output_partitions); @@ -242,12 +245,6 @@ impl RepartitionExecState { .with_can_spill(true) .register(context.memory_pool()), )); - let spill_metrics = SpillMetrics::new(&metrics, partition); - let spill_manager = Arc::new(SpillManager::new( - Arc::clone(&context.runtime_env()), - spill_metrics, - input.schema(), - )); // Create one SpillPool per input partition for this output partition // This ensures proper FIFO ordering within each (input, output) pair @@ -722,6 +719,8 @@ impl ExecutionPlan for RepartitionExec { partition ); + let spill_metrics = SpillMetrics::new(&self.metrics, partition); + let input = Arc::clone(&self.input); let partitioning = self.partitioning().clone(); let metrics = self.metrics.clone(); @@ -730,6 +729,12 @@ impl ExecutionPlan for RepartitionExec { let schema = self.schema(); let schema_captured = Arc::clone(&schema); + let spill_manager = SpillManager::new( + Arc::clone(&context.runtime_env()), + spill_metrics, + input.schema(), + ); + // Get existing ordering to use for merging let sort_exprs = self.sort_exprs().cloned(); @@ -757,6 +762,7 @@ impl ExecutionPlan for RepartitionExec { preserve_order, name.clone(), Arc::clone(&context), + spill_manager.clone(), )?; // now return stream for the specified *output* partition which will @@ -820,6 +826,7 @@ impl ExecutionPlan for RepartitionExec { .with_batch_size(context.session_config().batch_size()) .with_fetch(fetch) .with_reservation(merge_reservation) + .with_spill_manager(spill_manager) .build() } else { // Non-preserve-order case: single input stream, so use the first SpillPool @@ -1116,7 +1123,14 @@ impl RepartitionExec { Err(_) => { // We're memory limited - spill to SpillPool // SpillPool handles file handle reuse and rotation - channel.spill_pool.lock().push_batch(&batch)?; + { + let mut pool = channel.spill_pool.lock(); + pool.push_batch(&batch)?; + // Flush immediately to make the batch available for reading + // This is necessary for order-preserving repartition where + // the reader needs immediate access to spilled data + pool.flush()?; + } // Send marker indicating batch was spilled (RepartitionBatch::Spilled, false) @@ -1393,19 +1407,25 @@ impl Stream for PerPartitionStream { } Poll::Ready(None) => { // Spill stream ended - all spilled data has been read - return Poll::Ready(None); + // Only end the stream if input is also finished + if self.input_finished { + return Poll::Ready(None); + } + // Otherwise, continue to check the channel for new data } Poll::Pending => { - // No spilled data available + // No spilled data available yet (async I/O in progress) if self.input_finished { - // Input finished and no more spilled data - we're done - return Poll::Ready(None); + // Input finished, but spill stream might have data being read + // Wait for the async I/O to complete + return Poll::Pending; } - // Otherwise check the channel + // Spill stream is pending but not finished + // Fall through to check the channel } } - // If input is finished, don't poll channel anymore + // If input is finished, don't poll channel anymore - just wait for spill stream if self.input_finished { return Poll::Pending; } @@ -2213,6 +2233,7 @@ mod tests { #[cfg(test)] mod test { + use arrow::array::record_batch; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::assert_batches_eq; @@ -2301,28 +2322,32 @@ mod test { #[tokio::test] async fn test_preserve_order_with_spilling() -> Result<()> { - use arrow::array::UInt32Array; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_execution::TaskContext; - // Test that order-preserving repartition successfully spills to disk - // when memory is constrained while maintaining correct order - let schema = test_schema(); - let sort_exprs = sort_exprs(&schema); - // Create sorted input data across multiple partitions - // Each partition has sorted data: [1,2,3,4,5,6,7,8] - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))], - )?; - let partition1 = vec![batch.clone(); 25]; - let partition2 = vec![batch; 25]; + // Partition1: [1,3], [5,7], [9,11] + // Partition2: [2,4], [6,8], [10,12] + let batch1 = record_batch!(("c0", UInt32, [1, 3])).unwrap(); + let batch2 = record_batch!(("c0", UInt32, [2, 4])).unwrap(); + let batch3 = record_batch!(("c0", UInt32, [5, 7])).unwrap(); + let batch4 = record_batch!(("c0", UInt32, [6, 8])).unwrap(); + let batch5 = record_batch!(("c0", UInt32, [9, 11])).unwrap(); + let batch6 = record_batch!(("c0", UInt32, [10, 12])).unwrap(); + let schema = batch1.schema(); + let sort_exprs = LexOrdering::new([PhysicalSortExpr { + expr: col("c0", &schema).unwrap(), + options: SortOptions::default().asc(), + }]) + .unwrap(); + let partition1 = vec![batch1.clone(), batch3.clone(), batch5.clone()]; + let partition2 = vec![batch2.clone(), batch4.clone(), batch6.clone()]; let input_partitions = vec![partition1, partition2]; - // Set up context with very tight memory limit to force spilling + // Set up context with tight memory limit to force spilling + // Sorting needs some non-spillable memory, so 64 bytes should force spilling while still allowing the query to complete let runtime = RuntimeEnvBuilder::default() - .with_memory_limit(1, 1.0) + .with_memory_limit(64, 1.0) .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); @@ -2332,58 +2357,92 @@ mod test { let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)? .try_with_sort_information(vec![sort_exprs.clone(), sort_exprs])?; let exec = Arc::new(TestMemoryExec::update_cache(Arc::new(exec))); - let exec = RepartitionExec::try_new(exec, Partitioning::RoundRobinBatch(4))? + // Repartition into 3 partitions with order preservation + // We expect 1 batch per output partition after repartitioning + let exec = RepartitionExec::try_new(exec, Partitioning::RoundRobinBatch(3))? .with_preserve_order(); let mut batches = vec![]; // Collect all partitions - should succeed by spilling to disk - let mut total_rows = 0; for i in 0..exec.partitioning().partition_count() { let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; while let Some(result) = stream.next().await { let batch = result?; - total_rows += batch.num_rows(); batches.push(batch); } } - // Verify we got all the data (2 partitions * 25 batches * 8 rows each) - assert_eq!(total_rows, 2 * 25 * 8); + #[rustfmt::skip] + let expected = [ + [ + "+----+", + "| c0 |", + "+----+", + "| 1 |", + "| 2 |", + "| 3 |", + "| 4 |", + "+----+", + ], + [ + "+----+", + "| c0 |", + "+----+", + "| 5 |", + "| 6 |", + "| 7 |", + "| 8 |", + "+----+", + ], + [ + "+----+", + "| c0 |", + "+----+", + "| 9 |", + "| 10 |", + "| 11 |", + "| 12 |", + "+----+", + ], + ]; + + for (batch, expected) in batches.iter().zip(expected.iter()) { + assert_batches_eq!(expected, &[batch.clone()]); + } - // Verify spilling metrics to confirm spilling actually happened + // We should have spilled ~ all of the data. + // - We spill data during the repartitioning phase + // - We may also spill during the final merge sort + let all_batches = [batch1, batch2, batch3, batch4, batch5, batch6]; let metrics = exec.metrics().unwrap(); assert!( - metrics.spill_count().unwrap() > 0, - "Expected spill_count > 0 for order-preserving repartition, but got {:?}", + metrics.spill_count().unwrap() > input_partitions.len(), + "Expected spill_count > {} for order-preserving repartition, but got {:?}", + input_partitions.len(), metrics.spill_count() ); assert!( - metrics.spilled_bytes().unwrap() > 0, - "Expected spilled_bytes > 0 for order-preserving repartition" + metrics.spilled_bytes().unwrap() + > all_batches + .iter() + .map(|b| b.get_array_memory_size()) + .sum::(), + "Expected spilled_bytes > {} for order-preserving repartition, got {}", + all_batches + .iter() + .map(|b| b.get_array_memory_size()) + .sum::(), + metrics.spilled_bytes().unwrap() ); assert!( - metrics.spilled_rows().unwrap() > 0, - "Expected spilled_rows > 0 for order-preserving repartition" + metrics.spilled_rows().unwrap() + >= all_batches.iter().map(|b| b.num_rows()).sum::(), + "Expected spilled_rows > {} for order-preserving repartition, got {}", + all_batches.iter().map(|b| b.num_rows()).sum::(), + metrics.spilled_rows().unwrap() ); - // Verify that the final output batches are in sorted order - #[rustfmt::skip] - let expected = [ - "+----+", - "| c0 |", - "+----+", - "| 1 |", - "| 2 |", - "| 3 |", - "| 4 |", - "| 5 |", - "| 6 |", - "| 7 |", - "| 8 |", - ]; - assert_batches_eq!(&expected, &batches); - Ok(()) } From f43bbb6313c98279bdbdc9b060056bd0268877c7 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 12:13:41 -0500 Subject: [PATCH 17/44] Add slt test --- .../sqllogictest/test_files/repartition.slt | 132 ++++++++++++++++++ 1 file changed, 132 insertions(+) diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 29d20d10b6715..67b3f92a6d99b 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -146,3 +146,135 @@ statement ok DROP TABLE t1; # End repartition on empty columns test + +# Start spilling tests +# Spilling is hard to reproduce with real data / queries +# The memory limit was tuned to this specific datset (which was already used in `window.slt`) +# by hand to trigger spilling without being so low that the query would not succeed. +# Before we introduced spilling to `RepartitionExec` this query could not complete. + +statement ok +CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH ORDER (a ASC, b ASC, c ASC) +LOCATION '../../datafusion/core/tests/data/window_2.csv' +OPTIONS ('format.has_header' 'true'); + +statement ok +SET datafusion.runtime.memory_limit = '12K'; + +query IIII +SELECT SUM(a) OVER(partition by a, b order by c) as sum1, + SUM(a) OVER(partition by b, a order by c) as sum2, + SUM(a) OVER(partition by a, d order by b) as sum3, + SUM(a) OVER(partition by d order by a) as sum4 +FROM annotated_data_infinite2; +---- +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +2 2 2 4 +14 14 4 4 +12 12 2 4 +25 25 4 4 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +1 1 3 8 +2 2 8 8 +16 16 3 8 +7 7 8 8 +21 21 3 8 +18 18 8 8 +19 19 8 8 +21 21 8 8 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +3 3 10 15 +1 1 11 11 +4 4 10 15 +3 3 11 11 +5 5 6 12 +4 4 15 15 +6 6 4 11 +5 5 12 12 +7 7 10 15 +6 6 11 11 +8 8 10 15 +8 8 12 12 +9 9 10 15 +9 9 11 11 +10 10 4 11 +10 10 15 15 +11 11 6 12 +11 11 15 15 +13 13 10 15 +12 12 15 15 +14 14 6 12 +13 13 12 12 +15 15 6 12 +15 15 12 12 +17 17 4 11 +16 16 15 15 +18 18 6 12 +17 17 11 11 +19 19 10 15 +20 20 11 11 +20 20 10 15 +22 22 12 12 +22 22 4 11 +23 23 11 11 +23 23 10 15 +24 24 12 12 +24 24 10 15 +25 25 6 12 + +# End spilling tests From 5bf14ec43713c7af7293331a15afb3d846aa23de Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 12:25:02 -0500 Subject: [PATCH 18/44] use 128MB as the default spill file size --- datafusion/common/src/config.rs | 8 +++++--- datafusion/sqllogictest/test_files/information_schema.slt | 4 ++-- docs/source/user-guide/configs.md | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 4389e515efe74..aa0331917553a 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -520,7 +520,7 @@ config_namespace! { /// Maximum size in bytes for individual spill files before rotating to a new file. /// - /// When operators spill data to disk (e.g., RepartitionExec, SortExec), they write + /// When operators spill data to disk (e.g., RepartitionExec), they write /// multiple batches to the same file until this size limit is reached, then rotate /// to a new file. This reduces syscall overhead compared to one-file-per-batch /// while preventing files from growing too large. @@ -528,9 +528,11 @@ config_namespace! { /// A larger value reduces file creation overhead but may hold more disk space. /// A smaller value creates more files but allows finer-grained space reclamation /// as files can be deleted once fully consumed. + /// + /// Not all operators support this feature, some may create spill files larger than the limit. /// - /// Default: 100 MB - pub max_spill_file_size_bytes: usize, default = 100 * 1024 * 1024 + /// Default: 128 MB + pub max_spill_file_size_bytes: usize, default = 128 * 1024 * 1024 /// Number of files to read in parallel when inferring schema and statistics pub meta_fetch_concurrency: usize, default = 32 diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 8aeac175620a5..b7cf918e2b257 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -223,7 +223,7 @@ datafusion.execution.keep_partition_by_columns false datafusion.execution.listing_table_factory_infer_partitions true datafusion.execution.listing_table_ignore_subdirectory true datafusion.execution.max_buffered_batches_per_output_file 2 -datafusion.execution.max_spill_file_size_bytes 104857600 +datafusion.execution.max_spill_file_size_bytes 134217728 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 datafusion.execution.objectstore_writer_buffer_size 10485760 @@ -344,7 +344,7 @@ datafusion.execution.keep_partition_by_columns false Should DataFusion keep the datafusion.execution.listing_table_factory_infer_partitions true Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). datafusion.execution.listing_table_ignore_subdirectory true Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption -datafusion.execution.max_spill_file_size_bytes 104857600 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec, SortExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation (especially in LIFO mode where files are truncated after reading). Default: 100 MB +datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Not all operators support this feature, some may create spill files larger than the limit. Default: 128 MB datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.objectstore_writer_buffer_size 10485760 Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 949bcb201749a..1c1c901262a0b 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -114,7 +114,7 @@ The following configuration settings are available: | datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.max_spill_file_size_bytes | 104857600 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec, SortExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation (especially in LIFO mode where files are truncated after reading). Default: 100 MB | +| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Not all operators support this feature, some may create spill files larger than the limit. Default: 128 MB | | datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | | datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | From 39577f0ff1b090f22be4789c1a2afc1a93e099cb Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 12:26:10 -0500 Subject: [PATCH 19/44] make SpillPool pub(crate) --- datafusion/physical-plan/src/spill/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/spill/mod.rs b/datafusion/physical-plan/src/spill/mod.rs index a1a7bbee05c8a..395c98ca5f8d6 100644 --- a/datafusion/physical-plan/src/spill/mod.rs +++ b/datafusion/physical-plan/src/spill/mod.rs @@ -19,7 +19,7 @@ pub(crate) mod in_progress_spill_file; pub(crate) mod spill_manager; -pub mod spill_pool; +pub(crate) mod spill_pool; use std::fs::File; use std::io::BufReader; From 468ad57235035df7076e3c79f6b5f70f68025173 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 12:31:19 -0500 Subject: [PATCH 20/44] fmt --- datafusion/common/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index aa0331917553a..c74e03b1728ed 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -528,7 +528,7 @@ config_namespace! { /// A larger value reduces file creation overhead but may hold more disk space. /// A smaller value creates more files but allows finer-grained space reclamation /// as files can be deleted once fully consumed. - /// + /// /// Not all operators support this feature, some may create spill files larger than the limit. /// /// Default: 128 MB From 89dc51c78e0a81b57258ca640c2787e0a69d5b4e Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 12:48:48 -0500 Subject: [PATCH 21/44] lint --- datafusion/physical-plan/src/repartition/mod.rs | 1 + datafusion/physical-plan/src/spill/spill_manager.rs | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 08f37e4921c07..17040d2ac4a8c 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -183,6 +183,7 @@ impl RepartitionExecState { Ok(()) } + #[expect(clippy::too_many_arguments)] fn consume_input_streams( &mut self, input: Arc, diff --git a/datafusion/physical-plan/src/spill/spill_manager.rs b/datafusion/physical-plan/src/spill/spill_manager.rs index 788746fe2e353..b43323e623994 100644 --- a/datafusion/physical-plan/src/spill/spill_manager.rs +++ b/datafusion/physical-plan/src/spill/spill_manager.rs @@ -74,7 +74,7 @@ impl SpillManager { /// Returns the schema for batches managed by this SpillManager pub fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } /// Creates a temporary file for in-progress operations, returning an error From 54652ef594c6ecf7a9c4e37bdd40d2a34434969c Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 24 Oct 2025 12:52:05 -0500 Subject: [PATCH 22/44] update slt --- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index b7cf918e2b257..3f869a19c675b 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -344,7 +344,7 @@ datafusion.execution.keep_partition_by_columns false Should DataFusion keep the datafusion.execution.listing_table_factory_infer_partitions true Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). datafusion.execution.listing_table_ignore_subdirectory true Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption -datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Not all operators support this feature, some may create spill files larger than the limit. Default: 128 MB +datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Not all operators support this feature, some may create spill files larger than the limit. Default: 128 MB datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.objectstore_writer_buffer_size 10485760 Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. From d7af8efb06bae772f71e7ffe4f06cb5867ec94b2 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 30 Oct 2025 15:50:34 -0500 Subject: [PATCH 23/44] remove test --- .../sqllogictest/test_files/repartition.slt | 132 ------------------ 1 file changed, 132 deletions(-) diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 67b3f92a6d99b..29d20d10b6715 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -146,135 +146,3 @@ statement ok DROP TABLE t1; # End repartition on empty columns test - -# Start spilling tests -# Spilling is hard to reproduce with real data / queries -# The memory limit was tuned to this specific datset (which was already used in `window.slt`) -# by hand to trigger spilling without being so low that the query would not succeed. -# Before we introduced spilling to `RepartitionExec` this query could not complete. - -statement ok -CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER -) -STORED AS CSV -WITH ORDER (a ASC, b ASC, c ASC) -LOCATION '../../datafusion/core/tests/data/window_2.csv' -OPTIONS ('format.has_header' 'true'); - -statement ok -SET datafusion.runtime.memory_limit = '12K'; - -query IIII -SELECT SUM(a) OVER(partition by a, b order by c) as sum1, - SUM(a) OVER(partition by b, a order by c) as sum2, - SUM(a) OVER(partition by a, d order by b) as sum3, - SUM(a) OVER(partition by d order by a) as sum4 -FROM annotated_data_infinite2; ----- -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -2 2 2 4 -14 14 4 4 -12 12 2 4 -25 25 4 4 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -1 1 3 8 -2 2 8 8 -16 16 3 8 -7 7 8 8 -21 21 3 8 -18 18 8 8 -19 19 8 8 -21 21 8 8 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -3 3 10 15 -1 1 11 11 -4 4 10 15 -3 3 11 11 -5 5 6 12 -4 4 15 15 -6 6 4 11 -5 5 12 12 -7 7 10 15 -6 6 11 11 -8 8 10 15 -8 8 12 12 -9 9 10 15 -9 9 11 11 -10 10 4 11 -10 10 15 15 -11 11 6 12 -11 11 15 15 -13 13 10 15 -12 12 15 15 -14 14 6 12 -13 13 12 12 -15 15 6 12 -15 15 12 12 -17 17 4 11 -16 16 15 15 -18 18 6 12 -17 17 11 11 -19 19 10 15 -20 20 11 11 -20 20 10 15 -22 22 12 12 -22 22 4 11 -23 23 11 11 -23 23 10 15 -24 24 12 12 -24 24 10 15 -25 25 6 12 - -# End spilling tests From 0f6c83a68ccf66f47c3ff1ddfa26c5b251102d7e Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 30 Oct 2025 16:20:52 -0500 Subject: [PATCH 24/44] clippy --- datafusion/physical-plan/src/repartition/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 17040d2ac4a8c..42c828e486564 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -2409,7 +2409,7 @@ mod test { ]; for (batch, expected) in batches.iter().zip(expected.iter()) { - assert_batches_eq!(expected, &[batch.clone()]); + assert_batches_eq!(expected, std::slice::from_ref(batch)); } // We should have spilled ~ all of the data. From 93779c02bdb1194731df92e442194376ba2d7c22 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sun, 2 Nov 2025 11:32:35 -0600 Subject: [PATCH 25/44] Update datafusion/physical-plan/src/spill/spill_pool.rs Co-authored-by: Yongting You <2010youy01@gmail.com> --- datafusion/physical-plan/src/spill/spill_pool.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 7636f2a51c9e2..d64e68646bdc2 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -85,7 +85,9 @@ use super::spill_manager::SpillManager; /// `SpillPool` is not thread-safe and should be used from a single thread or /// protected with appropriate synchronization (e.g., `Arc>`). pub struct SpillPool { - /// Maximum size in bytes before rotating to a new file + /// Maximum size in bytes before rotating to a new file. + /// Typically initialized from the configuration option + /// `datafusion.execution.max_spill_file_size_bytes`. max_file_size_bytes: usize, /// Queue of spill files (front = oldest, back = newest) files: VecDeque, From 192f3ef37108c552cd488c319c022a8345640abe Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sun, 2 Nov 2025 11:52:09 -0600 Subject: [PATCH 26/44] update docs --- datafusion/physical-plan/src/spill/mod.rs | 2 +- .../physical-plan/src/spill/spill_pool.rs | 151 ++++++++++++------ 2 files changed, 106 insertions(+), 47 deletions(-) diff --git a/datafusion/physical-plan/src/spill/mod.rs b/datafusion/physical-plan/src/spill/mod.rs index 395c98ca5f8d6..a1a7bbee05c8a 100644 --- a/datafusion/physical-plan/src/spill/mod.rs +++ b/datafusion/physical-plan/src/spill/mod.rs @@ -19,7 +19,7 @@ pub(crate) mod in_progress_spill_file; pub(crate) mod spill_manager; -pub(crate) mod spill_pool; +pub mod spill_pool; use std::fs::File; use std::io::BufReader; diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index d64e68646bdc2..9d4eae0cc1592 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -15,49 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! SpillPool: A reusable abstraction for managing spill files with FIFO semantics. -//! -//! # Overview -//! -//! The `SpillPool` provides a centralized mechanism for spilling record batches to disk -//! when memory is constrained. It manages a collection of spill files, each containing -//! multiple batches, with configurable maximum file sizes. -//! -//! # Design -//! -//! - **FIFO (Queue) semantics**: Batches are read in the order they were spilled -//! - **File handle reuse**: Multiple batches are written to the same file to minimize syscalls -//! - **Automatic file rotation**: When a file exceeds `max_file_size_bytes`, rotate to a new file -//! - **Sequential reading**: Uses IPC Stream format's natural sequential access pattern -//! - **Automatic cleanup**: Files are deleted once fully consumed -//! -//! # Usage Example -//! -//! ```ignore -//! use std::sync::Arc; -//! use parking_lot::Mutex; -//! -//! let pool = SpillPool::new( -//! 100 * 1024 * 1024, // 100MB max per file -//! spill_manager, -//! ); -//! let pool = Arc::new(Mutex::new(pool)); -//! -//! // Spill batches - automatically rotates files when size limit reached -//! { -//! let mut pool = pool.lock(); -//! pool.push_batch(batch1)?; -//! pool.push_batch(batch2)?; -//! pool.flush()?; // Finalize current file -//! pool.finalize(); // Signal no more writes -//! } -//! -//! // Read back in FIFO order using a stream -//! let mut stream = SpillPool::reader(pool); -//! let batch1 = stream.next().await.unwrap()?; // Returns batch1 -//! let batch2 = stream.next().await.unwrap()?; // Returns batch2 -//! // stream.next() returns None after finalize -//! ``` +//! Spill pool for managing spill files with FIFO semantics. use std::collections::VecDeque; use std::sync::Arc; @@ -76,9 +34,110 @@ use super::spill_manager::SpillManager; /// A pool of spill files that manages batch-level spilling with FIFO semantics. /// -/// Batches are written sequentially to files, with automatic rotation when the -/// configured size limit is reached. Reading is done via an infinite stream -/// that can read concurrently while writes continue. +/// # Overview +/// +/// The `SpillPool` provides a centralized mechanism for spilling record batches to disk +/// when memory is constrained. It manages a collection of spill files, each containing +/// multiple batches, with configurable maximum file sizes. +/// +/// # Design +/// +/// - **FIFO (Queue) semantics**: Batches are read in the order they were spilled +/// - **File handle reuse**: Multiple batches are written to the same file to minimize syscalls +/// - **Automatic file rotation**: When a file exceeds `max_file_size_bytes`, rotate to a new file +/// - **Sequential reading**: Uses IPC Stream format's natural sequential access pattern +/// - **Automatic cleanup**: Files are deleted once fully consumed +/// - **Concurrent reading/writing**: Readers can poll and consume batches while writers continue +/// to spill more data, enabling pipelined processing +/// +/// # Usage Example +/// +/// This example demonstrates concurrent reading and writing. Note that the reader can start +/// polling for batches before the writer has finished, enabling pipelined processing: +/// +/// ``` +/// # use std::sync::Arc; +/// # use parking_lot::Mutex; +/// # use arrow::array::{ArrayRef, Int32Array}; +/// # use arrow::datatypes::{DataType, Field, Schema}; +/// # use arrow::record_batch::RecordBatch; +/// # use datafusion_common::Result; +/// # use datafusion_execution::runtime_env::RuntimeEnv; +/// # use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; +/// # use datafusion_physical_plan::SpillManager; +/// # use datafusion_physical_plan::spill::spill_pool::SpillPool; +/// # use datafusion_common_runtime::SpawnedTask; +/// # use futures::StreamExt; +/// # +/// # #[tokio::main] +/// # async fn main() -> Result<()> { +/// # // Create test schema and batches +/// # let schema = Arc::new(Schema::new(vec![ +/// # Field::new("a", DataType::Int32, false), +/// # ])); +/// # let batch1 = RecordBatch::try_new( +/// # Arc::clone(&schema), +/// # vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef], +/// # )?; +/// # let batch2 = RecordBatch::try_new( +/// # Arc::clone(&schema), +/// # vec![Arc::new(Int32Array::from(vec![4, 5, 6])) as ArrayRef], +/// # )?; +/// # let batch3 = RecordBatch::try_new( +/// # Arc::clone(&schema), +/// # vec![Arc::new(Int32Array::from(vec![7, 8, 9])) as ArrayRef], +/// # )?; +/// # +/// # // Set up spill manager +/// # let env = Arc::new(RuntimeEnv::default()); +/// # let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); +/// # let spill_manager = Arc::new(SpillManager::new(env, metrics, schema)); +/// # +/// // Create a spill pool +/// let pool = SpillPool::new( +/// 100 * 1024 * 1024, // 100MB max per file +/// spill_manager, +/// ); +/// let pool = Arc::new(Mutex::new(pool)); +/// +/// // Create a reader that will consume batches as they become available +/// let stream = SpillPool::reader(pool.clone()); +/// +/// // Spawn a task to write batches concurrently +/// let writer = SpawnedTask::spawn({ +/// let pool = pool.clone(); +/// async move { +/// let mut pool = pool.lock(); +/// pool.push_batch(&batch1).unwrap(); +/// pool.push_batch(&batch2).unwrap(); +/// pool.flush().unwrap(); // Finalize current file +/// pool.push_batch(&batch3).unwrap(); +/// pool.flush().unwrap(); // Flush the last batch +/// pool.finalize(); // Signal no more writes +/// } +/// }); +/// +/// // Reader can start consuming immediately, even while writer is still working +/// let reader = SpawnedTask::spawn(async move { +/// let mut batches = vec![]; +/// let mut stream = stream; +/// while let Some(result) = stream.next().await { +/// batches.push(result.unwrap()); +/// } +/// batches +/// }); +/// +/// // Wait for both tasks to complete +/// writer.join().await.unwrap(); +/// let batches = reader.join().await.unwrap(); +/// +/// assert_eq!(batches.len(), 3); +/// assert_eq!(batches[0].num_rows(), 3); +/// assert_eq!(batches[1].num_rows(), 3); +/// assert_eq!(batches[2].num_rows(), 3); +/// # Ok(()) +/// # } +/// ``` /// /// # Thread Safety /// From 374319b4f5c7a0086e1c750897fbfdb954cf80ce Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sun, 2 Nov 2025 13:32:48 -0600 Subject: [PATCH 27/44] Apply suggestion from @2010YOUY01 Co-authored-by: Yongting You <2010youy01@gmail.com> --- datafusion/common/src/config.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index c74e03b1728ed..71feb5555d3d1 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -529,7 +529,8 @@ config_namespace! { /// A smaller value creates more files but allows finer-grained space reclamation /// as files can be deleted once fully consumed. /// - /// Not all operators support this feature, some may create spill files larger than the limit. + /// Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators + /// may create spill files larger than the limit. /// /// Default: 128 MB pub max_spill_file_size_bytes: usize, default = 128 * 1024 * 1024 From b091574a0836cdd7dce49dcae60b6f2807e900a8 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 4 Nov 2025 11:12:31 -0600 Subject: [PATCH 28/44] lints --- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 3f869a19c675b..35e5dc5745996 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -344,7 +344,7 @@ datafusion.execution.keep_partition_by_columns false Should DataFusion keep the datafusion.execution.listing_table_factory_infer_partitions true Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). datafusion.execution.listing_table_ignore_subdirectory true Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption -datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Not all operators support this feature, some may create spill files larger than the limit. Default: 128 MB +datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators may create spill files larger than the limit. Default: 128 MB datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.objectstore_writer_buffer_size 10485760 Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1c1c901262a0b..403546873d912 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -114,7 +114,7 @@ The following configuration settings are available: | datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Not all operators support this feature, some may create spill files larger than the limit. Default: 128 MB | +| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators may create spill files larger than the limit. Default: 128 MB | | datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | | datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | From c34a5dfeae45d1ea80bd58802207cad288023490 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 4 Nov 2025 12:17:54 -0600 Subject: [PATCH 29/44] Add state machine for stream polling --- .../physical-plan/src/repartition/mod.rs | 265 +++++++++++++----- 1 file changed, 193 insertions(+), 72 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 42c828e486564..4684075c4b18d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -844,6 +844,7 @@ impl ExecutionPlan for RepartitionExec { spill_pool, spill_stream, all_inputs_finished: false, + state: StreamState::ReadingMemory, }) as SendableRecordBatchStream) } }) @@ -1224,6 +1225,16 @@ impl RepartitionExec { } } +/// State for tracking whether we're reading from memory channel or spill stream +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +enum StreamState { + /// Reading from the memory channel (normal operation) + ReadingMemory, + /// Waiting for a spilled batch from the spill stream + /// Must not poll channel until spilled batch is received + ReadingSpilled, +} + struct RepartitionStream { /// Number of input partitions that will be sending batches to this output channel num_input_partitions: usize, @@ -1251,6 +1262,9 @@ struct RepartitionStream { /// Flag indicating all inputs have finished all_inputs_finished: bool, + + /// Current state of the stream (reading from memory or spill) + state: StreamState, } impl Stream for RepartitionStream { @@ -1263,92 +1277,108 @@ impl Stream for RepartitionStream { use futures::StreamExt; loop { - // First, check if there's a spilled batch available - match self.spill_stream.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - // Got a spilled batch - return Poll::Ready(Some(Ok(batch))); - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(e))); - } - Poll::Ready(None) => { - // Spill stream ended - all spilled data has been read - return Poll::Ready(None); - } - Poll::Pending => { - // No spilled data available right now + match self.state { + StreamState::ReadingMemory => { + // If all inputs are finished, switch to draining spill stream only if self.all_inputs_finished { - // All inputs finished, verify pool is finalized before waiting - // If not finalized, we may hang indefinitely + // Verify pool is finalized before draining if !self.spill_pool.lock().is_finalized() { return Poll::Ready(Some(Err(DataFusionError::Internal( "Spill pool not finalized despite all inputs finishing" .to_string(), )))); } - // Pool is finalized, wait for spill stream to have more data or finish - return Poll::Pending; + // Transition to reading spilled to drain remaining data + self.state = StreamState::ReadingSpilled; + continue; } - // Otherwise check the channel - } - } - - // If all inputs are finished, don't poll channel anymore, just wait for spill_stream - if self.all_inputs_finished { - return Poll::Pending; - } - // Try to get next item from channel - let value = match self.input.recv().poll_unpin(cx) { - Poll::Ready(v) => v, - Poll::Pending => { - // Nothing from channel either, wait - return Poll::Pending; - } - }; + // Poll the memory channel for next message + let value = match self.input.recv().poll_unpin(cx) { + Poll::Ready(v) => v, + Poll::Pending => { + // Nothing from channel, wait + return Poll::Pending; + } + }; - match value { - Some(Some(v)) => match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Batch was spilled, it's available in spill_stream - // Loop back to poll spill_stream again - continue; - } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - }, - Some(None) => { - self.num_input_partitions_processed += 1; - - if self.num_input_partitions == self.num_input_partitions_processed { - // All input partitions have finished sending batches - // Flush and finalize the SpillPool - { - let mut pool = self.spill_pool.lock(); - if let Err(e) = pool.flush() { + match value { + Some(Some(v)) => match v { + Ok(RepartitionBatch::Memory(batch)) => { + // Release memory and return batch + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + return Poll::Ready(Some(Ok(batch))); + } + Ok(RepartitionBatch::Spilled) => { + // Batch was spilled, transition to reading from spill stream + // We must block on spill stream until we get the batch + // to preserve ordering + self.state = StreamState::ReadingSpilled; + continue; + } + Err(e) => { return Poll::Ready(Some(Err(e))); } - pool.finalize(); - } // Drop the lock before continuing - self.all_inputs_finished = true; - // Continue to drain any remaining spilled batches - continue; - } else { - // other partitions still have data to send - continue; + }, + Some(None) => { + // One input partition finished + self.num_input_partitions_processed += 1; + + if self.num_input_partitions + == self.num_input_partitions_processed + { + // All input partitions have finished sending batches + // Flush and finalize the SpillPool + { + let mut pool = self.spill_pool.lock(); + if let Err(e) = pool.flush() { + return Poll::Ready(Some(Err(e))); + } + pool.finalize(); + } // Drop the lock before continuing + self.all_inputs_finished = true; + // Transition to draining spilled batches + self.state = StreamState::ReadingSpilled; + continue; + } else { + // Other partitions still have data to send + continue; + } + } + None => { + // Channel closed unexpectedly + return Poll::Ready(None); + } } } - None => { - return Poll::Ready(None); + StreamState::ReadingSpilled => { + // Poll spill stream for the spilled batch + // Do NOT poll channel in this state - we must wait for the spilled batch + match self.spill_stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + // Got the spilled batch, return it and transition back to reading memory + // unless all inputs are finished (in which case stay in ReadingSpilled + // to drain remaining spilled batches) + if !self.all_inputs_finished { + self.state = StreamState::ReadingMemory; + } + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Spill stream ended - all spilled data has been read + return Poll::Ready(None); + } + Poll::Pending => { + // Spilled batch not ready yet, must wait + // This preserves ordering by blocking until spill data arrives + return Poll::Pending; + } + } } } } @@ -2230,6 +2260,97 @@ mod tests { ) .unwrap() } + + /// Create batches with sequential values for ordering tests + fn create_ordered_batches(num_batches: usize) -> Vec { + let schema = test_schema(); + (0..num_batches) + .map(|i| { + let start = (i * 8) as u32; + RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(UInt32Array::from( + (start..start + 8).collect::>(), + ))], + ) + .unwrap() + }) + .collect() + } + + #[tokio::test] + async fn test_repartition_ordering_with_spilling() -> Result<()> { + // Test that repartition preserves ordering when spilling occurs + // This tests the state machine fix where we must block on spill_stream + // when a Spilled marker is received, rather than continuing to poll the channel + + let schema = test_schema(); + // Create batches with sequential values: batch 0 has [0,1,2,3,4,5,6,7], + // batch 1 has [8,9,10,11,12,13,14,15], etc. + let partition = create_ordered_batches(20); + let input_partitions = vec![partition]; + + // Use RoundRobinBatch to ensure predictable ordering + let partitioning = Partitioning::RoundRobinBatch(2); + + // Set up context with very tight memory limit to force spilling + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; + + let task_ctx = TaskContext::default().with_runtime(runtime); + let task_ctx = Arc::new(task_ctx); + + // create physical plan + let exec = + TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?; + let exec = RepartitionExec::try_new(exec, partitioning)?; + + // Collect all output partitions + let mut all_batches = Vec::new(); + for i in 0..exec.partitioning().partition_count() { + let mut partition_batches = Vec::new(); + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; + while let Some(result) = stream.next().await { + let batch = result?; + partition_batches.push(batch); + } + all_batches.push(partition_batches); + } + + // Verify spilling occurred + let metrics = exec.metrics().unwrap(); + assert!( + metrics.spill_count().unwrap() > 0, + "Expected spilling to occur, but spill_count = 0" + ); + + // Verify ordering is preserved within each partition + // With RoundRobinBatch, even batches go to partition 0, odd batches to partition 1 + for (partition_idx, batches) in all_batches.iter().enumerate() { + let mut last_value = None; + for batch in batches { + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + + for i in 0..array.len() { + let value = array.value(i); + if let Some(last) = last_value { + assert!( + value > last, + "Ordering violated in partition {partition_idx}: {value} is not greater than {last}" + ); + } + last_value = Some(value); + } + } + } + + Ok(()) + } } #[cfg(test)] From e9aedf6350fcca155d6e4ba323a57264f1dbf799 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 4 Nov 2025 20:46:37 -0600 Subject: [PATCH 30/44] wip --- datafusion/execution/src/disk_manager.rs | 12 +- .../physical-plan/src/repartition/mod.rs | 458 ++++---- .../src/spill/in_progress_spill_file.rs | 4 + .../physical-plan/src/spill/spill_pool.rs | 975 +++--------------- 4 files changed, 413 insertions(+), 1036 deletions(-) diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs index 82f2d75ac1b57..0d0ce2219ef60 100644 --- a/datafusion/execution/src/disk_manager.rs +++ b/datafusion/execution/src/disk_manager.rs @@ -294,9 +294,11 @@ impl DiskManager { let dir_index = rng().random_range(0..local_dirs.len()); Ok(RefCountedTempFile { _parent_temp_dir: Arc::clone(&local_dirs[dir_index]), - tempfile: Builder::new() - .tempfile_in(local_dirs[dir_index].as_ref()) - .map_err(DataFusionError::IoError)?, + tempfile: Arc::new( + Builder::new() + .tempfile_in(local_dirs[dir_index].as_ref()) + .map_err(DataFusionError::IoError)?, + ), current_file_disk_usage: 0, disk_manager: Arc::clone(self), }) @@ -311,12 +313,12 @@ impl DiskManager { /// must invoke [`Self::update_disk_usage`] to update the global disk usage counter. /// This ensures the disk manager can properly enforce usage limits configured by /// [`DiskManager::with_max_temp_directory_size`]. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct RefCountedTempFile { /// The reference to the directory in which temporary files are created to ensure /// it is not cleaned up prior to the NamedTempFile _parent_temp_dir: Arc, - tempfile: NamedTempFile, + tempfile: Arc, /// Tracks the current disk usage of this temporary file. See /// [`Self::update_disk_usage`] for more details. current_file_disk_usage: u64, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 4684075c4b18d..77a150f3cfdf9 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -34,9 +34,6 @@ use crate::execution_plan::{CardinalityEffect, EvaluationType, SchedulingType}; use crate::hash_utils::create_hashes; use crate::metrics::{BaselineMetrics, SpillMetrics}; use crate::projection::{all_columns, make_with_child, update_expr, ProjectionExec}; -use crate::repartition::distributor_channels::{ - channels, partition_aware_channels, DistributionReceiver, DistributionSender, -}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; use crate::spill::spill_pool::SpillPool; @@ -52,7 +49,7 @@ use datafusion_common::utils::transpose; use datafusion_common::{internal_err, ColumnStatistics, HashMap}; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; -use datafusion_execution::memory_pool::MemoryConsumer; +use datafusion_execution::memory_pool::{FairSpillPool, GreedyMemoryPool, MemoryConsumer}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -65,6 +62,7 @@ use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; +use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; mod distributor_channels; @@ -79,13 +77,88 @@ enum RepartitionBatch { } type MaybeBatch = Option>; -type InputPartitionsToCurrentPartitionSender = Vec>; -type InputPartitionsToCurrentPartitionReceiver = Vec>; +type InputPartitionsToCurrentPartitionSender = + Vec>; +type InputPartitionsToCurrentPartitionReceiver = + Vec>; + +/// Wrapper around tokio UnboundedSender to match DistributionSender interface +struct UnboundedDistributionSender { + inner: UnboundedSender, +} + +// Implement Clone manually without requiring T: Clone +impl Clone for UnboundedDistributionSender { + fn clone(&self) -> Self { + Self { + inner: self.inner.clone(), + } + } +} + +impl UnboundedDistributionSender { + fn send(&self, value: T) -> UnboundedSendFuture { + UnboundedSendFuture { + is_err: self.inner.send(value).is_err(), + } + } +} + +struct UnboundedSendFuture { + is_err: bool, +} + +impl std::future::Future for UnboundedSendFuture { + type Output = Result<(), ()>; + + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { + // Unbounded send is immediate, so always ready + if self.is_err { + Poll::Ready(Err(())) + } else { + Poll::Ready(Ok(())) + } + } +} + +/// Wrapper around tokio UnboundedReceiver to match DistributionReceiver interface +struct UnboundedDistributionReceiver { + inner: UnboundedReceiver, +} + +impl UnboundedDistributionReceiver { + fn recv( + &mut self, + ) -> Pin> + Send + '_>> { + Box::pin(self.inner.recv()) + } +} + +/// Create unbounded channels without backpressure +fn unbounded_channels( + n: usize, +) -> ( + Vec>, + Vec>, +) { + let channels: Vec<_> = (0..n) + .map(|_| { + let (tx, rx) = unbounded_channel(); + ( + UnboundedDistributionSender { inner: tx }, + UnboundedDistributionReceiver { inner: rx }, + ) + }) + .collect(); + + let (senders, receivers): (Vec<_>, Vec<_>) = channels.into_iter().unzip(); + (senders, receivers) +} /// Output channel with its associated memory reservation and spill pool #[derive(Clone)] struct OutputChannel { - sender: DistributionSender, + sender: UnboundedDistributionSender, reservation: SharedMemoryReservation, spill_pool: Arc>, } @@ -219,17 +292,22 @@ impl RepartitionExecState { let spill_manager = Arc::new(spill_manager); let (txs, rxs) = if preserve_order { - let (txs, rxs) = - partition_aware_channels(num_input_partitions, num_output_partitions); + // Create unbounded channels for each input->output pair + let mut txs_all = vec![]; + let mut rxs_all = vec![]; + for _ in 0..num_input_partitions { + let (txs_row, rxs_row) = unbounded_channels(num_output_partitions); + txs_all.push(txs_row); + rxs_all.push(rxs_row); + } // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition - let txs = transpose(txs); - let rxs = transpose(rxs); + let txs = transpose(txs_all); + let rxs = transpose(rxs_all); (txs, rxs) } else { // create one channel per *output* partition - // note we use a custom channel that ensures there is always data for each receiver - // but limits the amount of buffering if required. - let (txs, rxs) = channels(num_output_partitions); + // Using unbounded channels to avoid backpressure/deadlock with spilling + let (txs, rxs) = unbounded_channels(num_output_partitions); // Clone sender for each input partitions let txs = txs .into_iter() @@ -244,7 +322,7 @@ impl RepartitionExecState { let reservation = Arc::new(Mutex::new( MemoryConsumer::new(format!("{name}[{partition}]")) .with_can_spill(true) - .register(context.memory_pool()), + .register(&(Arc::new(GreedyMemoryPool::new(1)) as Arc<_>)), )); // Create one SpillPool per input partition for this output partition @@ -799,16 +877,13 @@ impl ExecutionPlan for RepartitionExec { .map(|(idx, receiver)| { let spill_pool = Arc::clone(&spill_pools[idx]); let spill_stream = SpillPool::reader(Arc::clone(&spill_pool)); - - Box::pin(PerPartitionStream { - schema: Arc::clone(&schema_captured), + Box::pin(PerPartitionStream::new( + Arc::clone(&schema_captured), receiver, - _drop_helper: Arc::clone(&abort_helper), - reservation: Arc::clone(&reservation), - spill_pool, + Arc::clone(&abort_helper), + Arc::clone(&reservation), spill_stream, - input_finished: false, - }) as SendableRecordBatchStream + )) as SendableRecordBatchStream }) .collect::>(); // Note that receiver size (`rx.len()`) and `num_input_partitions` are same. @@ -834,18 +909,14 @@ impl ExecutionPlan for RepartitionExec { let spill_pool = Arc::clone(&spill_pools[0]); let spill_stream = SpillPool::reader(Arc::clone(&spill_pool)); - Ok(Box::pin(RepartitionStream { - num_input_partitions, - num_input_partitions_processed: 0, - schema: input.schema(), - input: rx.swap_remove(0), - _drop_helper: abort_helper, + Ok(Box::pin(RepartitionStream::new( + input.schema(), + rx.swap_remove(0), + abort_helper, reservation, - spill_pool, spill_stream, - all_inputs_finished: false, - state: StreamState::ReadingMemory, - }) as SendableRecordBatchStream) + num_input_partitions, + )) as SendableRecordBatchStream) } }) .try_flatten(); @@ -1120,32 +1191,34 @@ impl RepartitionExec { match channel.reservation.lock().try_grow(size) { Ok(_) => { // Memory available - send in-memory batch + eprintln!("[SEND] Partition {}: Sending Memory batch with {} rows", partition, batch.num_rows()); (RepartitionBatch::Memory(batch), true) } Err(_) => { // We're memory limited - spill to SpillPool // SpillPool handles file handle reuse and rotation + eprintln!("[SEND] Partition {}: Spilling batch with {} rows", partition, batch.num_rows()); { let mut pool = channel.spill_pool.lock(); pool.push_batch(&batch)?; - // Flush immediately to make the batch available for reading - // This is necessary for order-preserving repartition where - // the reader needs immediate access to spilled data - pool.flush()?; } - // Send marker indicating batch was spilled (RepartitionBatch::Spilled, false) } }; + eprintln!("[SEND] Partition {}: About to send {:?} to channel", partition, + if is_memory_batch { "Memory" } else { "Spilled" }); if channel.sender.send(Some(Ok(batch_to_send))).await.is_err() { + eprintln!("[SEND] Partition {}: Send failed, receiver hung up", partition); // If the other end has hung up, it was an early shutdown (e.g. LIMIT) // Only shrink memory if it was a memory batch if is_memory_batch { channel.reservation.lock().shrink(size); } output_channels.remove(&partition); + } else { + eprintln!("[SEND] Partition {}: Send completed successfully", partition); } } timer.done(); @@ -1175,6 +1248,14 @@ impl RepartitionExec { } } + // Flush all spill pools now that input is done + eprintln!("[SEND] Flushing all spill pools for {} output channels", output_channels.len()); + for (partition, channel) in output_channels.iter() { + let mut pool = channel.spill_pool.lock(); + eprintln!("[SEND] Flushing spill pool for partition {}", partition); + pool.flush()?; + } + Ok(()) } @@ -1185,7 +1266,7 @@ impl RepartitionExec { /// channels. async fn wait_for_task( input_task: SpawnedTask>, - txs: HashMap>, + txs: HashMap>, ) { // wait for completion, and propagate error // note we ignore errors on send (.ok) as that means the receiver has already shutdown. @@ -1217,7 +1298,9 @@ impl RepartitionExec { // Input task completed successfully Ok(Ok(())) => { // notify each output partition that this input partition has no more data - for (_, tx) in txs { + eprintln!("[SEND] Input task completed, sending None to {} output partitions", txs.len()); + for (partition, tx) in txs { + eprintln!("[SEND] Sending None to partition {}", partition); tx.send(None).await.ok(); } } @@ -1236,17 +1319,11 @@ enum StreamState { } struct RepartitionStream { - /// Number of input partitions that will be sending batches to this output channel - num_input_partitions: usize, - - /// Number of input partitions that have finished sending batches to this output channel - num_input_partitions_processed: usize, - /// Schema wrapped by Arc schema: SchemaRef, /// channel containing the repartitioned batches - input: DistributionReceiver, + input: UnboundedDistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -1254,17 +1331,35 @@ struct RepartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, - /// SpillPool for batched spilling with FIFO semantics (shared for writing) - spill_pool: Arc>, - /// Infinite stream for reading from the spill pool spill_stream: SendableRecordBatchStream, - /// Flag indicating all inputs have finished - all_inputs_finished: bool, - /// Current state of the stream (reading from memory or spill) state: StreamState, + + /// Number of input partitions that have not yet finished + remaining_partitions: usize, +} + +impl RepartitionStream { + fn new( + schema: SchemaRef, + input: UnboundedDistributionReceiver, + drop_helper: Arc>>, + reservation: SharedMemoryReservation, + spill_stream: SendableRecordBatchStream, + num_input_partitions: usize, + ) -> Self { + Self { + schema, + input, + _drop_helper: drop_helper, + reservation, + spill_stream, + state: StreamState::ReadingMemory, + remaining_partitions: num_input_partitions, + } + } } impl Stream for RepartitionStream { @@ -1277,26 +1372,25 @@ impl Stream for RepartitionStream { use futures::StreamExt; loop { + eprintln!("[RepartitionStream] State: {:?}, remaining_partitions: {}", + self.state, self.remaining_partitions); match self.state { StreamState::ReadingMemory => { - // If all inputs are finished, switch to draining spill stream only - if self.all_inputs_finished { - // Verify pool is finalized before draining - if !self.spill_pool.lock().is_finalized() { - return Poll::Ready(Some(Err(DataFusionError::Internal( - "Spill pool not finalized despite all inputs finishing" - .to_string(), - )))); - } - // Transition to reading spilled to drain remaining data - self.state = StreamState::ReadingSpilled; - continue; - } - // Poll the memory channel for next message let value = match self.input.recv().poll_unpin(cx) { - Poll::Ready(v) => v, + Poll::Ready(v) => { + eprintln!("[RepartitionStream] Memory channel poll returned: {:?}", + match &v { + Some(Some(Ok(RepartitionBatch::Memory(_)))) => "Some(Some(Ok(Memory)))", + Some(Some(Ok(RepartitionBatch::Spilled))) => "Some(Some(Ok(Spilled)))", + Some(Some(Err(_))) => "Some(Some(Err))", + Some(None) => "Some(None)", + None => "None", + }); + v + } Poll::Pending => { + eprintln!("[RepartitionStream] Memory channel poll returned: Pending"); // Nothing from channel, wait return Poll::Pending; } @@ -1305,6 +1399,7 @@ impl Stream for RepartitionStream { match value { Some(Some(v)) => match v { Ok(RepartitionBatch::Memory(batch)) => { + eprintln!("[RepartitionStream] Returning memory batch with {} rows", batch.num_rows()); // Release memory and return batch self.reservation .lock() @@ -1312,6 +1407,7 @@ impl Stream for RepartitionStream { return Poll::Ready(Some(Ok(batch))); } Ok(RepartitionBatch::Spilled) => { + eprintln!("[RepartitionStream] Received Spilled marker, transitioning to ReadingSpilled"); // Batch was spilled, transition to reading from spill stream // We must block on spill stream until we get the batch // to preserve ordering @@ -1319,35 +1415,25 @@ impl Stream for RepartitionStream { continue; } Err(e) => { + eprintln!("[RepartitionStream] Returning error"); return Poll::Ready(Some(Err(e))); } }, Some(None) => { // One input partition finished - self.num_input_partitions_processed += 1; - - if self.num_input_partitions - == self.num_input_partitions_processed - { - // All input partitions have finished sending batches - // Flush and finalize the SpillPool - { - let mut pool = self.spill_pool.lock(); - if let Err(e) = pool.flush() { - return Poll::Ready(Some(Err(e))); - } - pool.finalize(); - } // Drop the lock before continuing - self.all_inputs_finished = true; - // Transition to draining spilled batches - self.state = StreamState::ReadingSpilled; - continue; - } else { - // Other partitions still have data to send - continue; + self.remaining_partitions -= 1; + eprintln!("[RepartitionStream] Input partition finished, remaining: {}", + self.remaining_partitions); + if self.remaining_partitions == 0 { + eprintln!("[RepartitionStream] All input partitions finished, ending stream"); + // All input partitions finished + return Poll::Ready(None); } + // Continue to poll for more data from other partitions + continue; } None => { + eprintln!("[RepartitionStream] Channel closed unexpectedly"); // Channel closed unexpectedly return Poll::Ready(None); } @@ -1355,25 +1441,24 @@ impl Stream for RepartitionStream { } StreamState::ReadingSpilled => { // Poll spill stream for the spilled batch - // Do NOT poll channel in this state - we must wait for the spilled batch match self.spill_stream.poll_next_unpin(cx) { Poll::Ready(Some(Ok(batch))) => { - // Got the spilled batch, return it and transition back to reading memory - // unless all inputs are finished (in which case stay in ReadingSpilled - // to drain remaining spilled batches) - if !self.all_inputs_finished { - self.state = StreamState::ReadingMemory; - } + eprintln!("[RepartitionStream] Got spilled batch with {} rows, transitioning to ReadingMemory", + batch.num_rows()); + self.state = StreamState::ReadingMemory; return Poll::Ready(Some(Ok(batch))); } Poll::Ready(Some(Err(e))) => { + eprintln!("[RepartitionStream] Spill stream returned error"); return Poll::Ready(Some(Err(e))); } Poll::Ready(None) => { - // Spill stream ended - all spilled data has been read - return Poll::Ready(None); + eprintln!("[RepartitionStream] Spill stream ended, transitioning to ReadingMemory"); + // Spill stream ended keep draining the memory channel + self.state = StreamState::ReadingMemory; } Poll::Pending => { + eprintln!("[RepartitionStream] Spill stream poll returned: Pending"); // Spilled batch not ready yet, must wait // This preserves ordering by blocking until spill data arrives return Poll::Pending; @@ -1399,7 +1484,7 @@ struct PerPartitionStream { schema: SchemaRef, /// channel containing the repartitioned batches - receiver: DistributionReceiver, + receiver: UnboundedDistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -1407,14 +1492,30 @@ struct PerPartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, - /// SpillPool for batched spilling with FIFO semantics (shared for writing) - spill_pool: Arc>, - /// Infinite stream for reading from the spill pool spill_stream: SendableRecordBatchStream, - /// Flag indicating input partition has finished - input_finished: bool, + /// Internal state indicating if we are reading from memory or spill stream + state: StreamState, +} + +impl PerPartitionStream { + fn new( + schema: SchemaRef, + receiver: UnboundedDistributionReceiver, + drop_helper: Arc>>, + reservation: SharedMemoryReservation, + spill_stream: SendableRecordBatchStream, + ) -> Self { + Self { + schema, + receiver, + _drop_helper: drop_helper, + reservation, + spill_stream, + state: StreamState::ReadingMemory, + } + } } impl Stream for PerPartitionStream { @@ -1427,82 +1528,89 @@ impl Stream for PerPartitionStream { use futures::StreamExt; loop { - // First, check if there's a spilled batch available - match self.spill_stream.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - // Got a spilled batch - return Poll::Ready(Some(Ok(batch))); - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(e))); - } - Poll::Ready(None) => { - // Spill stream ended - all spilled data has been read - // Only end the stream if input is also finished - if self.input_finished { - return Poll::Ready(None); - } - // Otherwise, continue to check the channel for new data - } - Poll::Pending => { - // No spilled data available yet (async I/O in progress) - if self.input_finished { - // Input finished, but spill stream might have data being read - // Wait for the async I/O to complete - return Poll::Pending; - } - // Spill stream is pending but not finished - // Fall through to check the channel - } - } - - // If input is finished, don't poll channel anymore - just wait for spill stream - if self.input_finished { - return Poll::Pending; - } - - // Try to get next item from channel - let value = match self.receiver.recv().poll_unpin(cx) { - Poll::Ready(v) => v, - Poll::Pending => { - // Nothing from channel either, wait - return Poll::Pending; - } - }; + eprintln!("[PerPartitionStream] State: {:?}", self.state); + match self.state { + StreamState::ReadingMemory => { + // Poll the memory channel for next message + let value = match self.receiver.recv().poll_unpin(cx) { + Poll::Ready(v) => { + eprintln!("[PerPartitionStream] Memory channel poll returned: {:?}", + match &v { + Some(Some(Ok(RepartitionBatch::Memory(_)))) => "Some(Some(Ok(Memory)))", + Some(Some(Ok(RepartitionBatch::Spilled))) => "Some(Some(Ok(Spilled)))", + Some(Some(Err(_))) => "Some(Some(Err))", + Some(None) => "Some(None)", + None => "None", + }); + v + } + Poll::Pending => { + eprintln!("[PerPartitionStream] Memory channel poll returned: Pending"); + // Nothing from channel, wait + return Poll::Pending; + } + }; - match value { - Some(Some(v)) => match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Batch was spilled, it's available in spill_stream - // Loop back to poll spill_stream again - continue; - } - Err(e) => { - return Poll::Ready(Some(Err(e))); + match value { + Some(Some(v)) => match v { + Ok(RepartitionBatch::Memory(batch)) => { + eprintln!("[PerPartitionStream] Returning memory batch with {} rows", batch.num_rows()); + // Release memory and return batch + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + return Poll::Ready(Some(Ok(batch))); + } + Ok(RepartitionBatch::Spilled) => { + eprintln!("[PerPartitionStream] Received Spilled marker, transitioning to ReadingSpilled"); + // Batch was spilled, transition to reading from spill stream + // We must block on spill stream until we get the batch + // to preserve ordering + self.state = StreamState::ReadingSpilled; + continue; + } + Err(e) => { + eprintln!("[PerPartitionStream] Returning error"); + return Poll::Ready(Some(Err(e))); + } + }, + Some(None) => { + eprintln!("[PerPartitionStream] Received None, ending stream"); + return Poll::Ready(None); + } + None => { + eprintln!("[PerPartitionStream] Channel closed unexpectedly"); + // Channel closed unexpectedly + return Poll::Ready(None); + } } - }, - Some(None) => { - // Input partition has finished sending batches - // Flush and finalize the SpillPool - { - let mut pool = self.spill_pool.lock(); - if let Err(e) = pool.flush() { + } + StreamState::ReadingSpilled => { + // Poll spill stream for the spilled batch + match self.spill_stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + eprintln!("[PerPartitionStream] Got spilled batch with {} rows, transitioning to ReadingMemory", + batch.num_rows()); + self.state = StreamState::ReadingMemory; + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + eprintln!("[PerPartitionStream] Spill stream returned error"); return Poll::Ready(Some(Err(e))); } - pool.finalize(); - } // Drop the lock before continuing - self.input_finished = true; - // Continue to drain any remaining spilled batches - continue; + Poll::Ready(None) => { + eprintln!("[PerPartitionStream] Spill stream ended, transitioning to ReadingMemory"); + // Spill stream ended keep draining the memory channel + self.state = StreamState::ReadingMemory; + } + Poll::Pending => { + eprintln!("[PerPartitionStream] Spill stream poll returned: Pending"); + // Spilled batch not ready yet, must wait + // This preserves ordering by blocking until spill data arrives + return Poll::Pending; + } + } } - None => return Poll::Ready(None), } } } diff --git a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs index 3149c829c7262..5042354da3f9a 100644 --- a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs +++ b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs @@ -118,6 +118,10 @@ impl InProgressSpillFile { Ok(self.in_progress_file.take()) } + pub fn file(&self) -> Option<&RefCountedTempFile> { + self.in_progress_file.as_ref() + } + /// Returns the number of batches written to this file pub fn batch_count(&self) -> usize { self.batch_count diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 9d4eae0cc1592..ce8a870a2b249 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -20,6 +20,7 @@ use std::collections::VecDeque; use std::sync::Arc; use std::task::Waker; +use futures::{Stream, StreamExt}; use parking_lot::Mutex; @@ -148,16 +149,121 @@ pub struct SpillPool { /// Typically initialized from the configuration option /// `datafusion.execution.max_spill_file_size_bytes`. max_file_size_bytes: usize, - /// Queue of spill files (front = oldest, back = newest) - files: VecDeque, - /// Current file being written to (if any) - current_write_file: Option, /// SpillManager for creating files and tracking metrics spill_manager: Arc, /// Wakers to notify when new data is available for readers wakers: Vec, - /// Flag indicating no more writes will occur - finalized: bool, +} + +/// Shared state between writer and readers for an active spill file. +/// Protected by a Mutex to coordinate between concurrent readers and the writer. +struct ActiveSpillFileShared { + /// Writer handle - taken (set to None) when finish() is called + writer: Option, + /// Path to the spill file for creating readers + file_path: RefCountedTempFile, + /// Schema for creating readers + schema: SchemaRef, + /// Total number of batches written to this file + batches_written: usize, + /// Whether the writer has finished writing to this file + writer_finished: bool, +} + +/// Reader state for a SpillFile (owned by individual SpillFile instances). +/// This is kept separate from the shared state to avoid holding locks during I/O. +struct SpillFileReader { + /// The actual stream reading from disk + stream: SendableRecordBatchStream, + /// Number of batches this reader has consumed + batches_read: usize, +} + +// impl SpillFile { +// fn poll_next(&mut self) -> Option> { +// match self { +// SpillFile::InProgress(active) => { +// // If there are no unread batches, we cannot read yet +// if active.unread_batches == 0 { +// return None; +// } +// }, +// SpillFile::Completed(stream) => stream.next().await, +// } +// } +// } + +struct SpillFile { + /// Shared coordination state (contains writer and batch counts) + shared: Arc>, + /// Reader state (lazy-initialized, owned by this SpillFile) + reader: Option, + /// Spill manager for creating readers + spill_manager: Arc, +} + +impl Stream for SpillFile { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + use std::task::Poll; + + // Step 1: Lock shared state and check coordination + let (should_read, file_path, schema, batches_written) = { + let shared = self.shared.lock(); + + // Determine if we can read + let batches_read = self.reader.as_ref().map_or(0, |r| r.batches_read); + + if batches_read < shared.batches_written { + // More data available to read + (true, shared.file_path.clone(), Arc::clone(&shared.schema), shared.batches_written) + } else if shared.writer_finished { + // No more data and writer is done - EOF + return Poll::Ready(None); + } else { + // Caught up to writer, but writer still active - wait + return Poll::Pending; + } + }; // Lock released here + + // Step 2: Lazy-create reader stream if needed + if self.reader.is_none() && should_read { + match self.spill_manager.read_spill_as_stream(file_path, None) { + Ok(stream) => { + self.reader = Some(SpillFileReader { + stream, + batches_read: 0, + }); + } + Err(e) => return Poll::Ready(Some(Err(e))), + } + } + + // Step 3: Poll the reader stream (no lock held) + if let Some(reader) = &mut self.reader { + match reader.stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + // Successfully read a batch - increment counter + reader.batches_read += 1; + Poll::Ready(Some(Ok(batch))) + } + Poll::Ready(Some(Err(e))) => Poll::Ready(Some(Err(e))), + Poll::Ready(None) => { + // Stream exhausted unexpectedly + // This shouldn't happen if coordination is correct, but handle gracefully + Poll::Ready(None) + } + Poll::Pending => Poll::Pending, + } + } else { + // Should not reach here, but handle gracefully + Poll::Ready(None) + } + } } impl SpillPool { @@ -168,26 +274,7 @@ impl SpillPool { /// * `max_file_size_bytes` - Maximum size per file before rotation (e.g., 100MB) /// * `spill_manager` - Manager for file creation and metrics pub fn new(max_file_size_bytes: usize, spill_manager: Arc) -> Self { - Self { - max_file_size_bytes, - files: VecDeque::new(), - current_write_file: None, - spill_manager, - wakers: Vec::new(), - finalized: false, - } - } - - /// Marks the pool as finalized, indicating no more writes will occur. - /// This allows readers to know when to stop waiting for more data. - pub fn finalize(&mut self) { - self.finalized = true; - self.wake(); // Wake readers to check finalized status - } - - /// Returns true if the pool has been finalized - pub fn is_finalized(&self) -> bool { - self.finalized + todo!() } /// Creates a stream reader for this pool. @@ -217,42 +304,7 @@ impl SpillPool { /// /// Returns an error if disk I/O fails or disk quota is exceeded. pub fn push_batch(&mut self, batch: &RecordBatch) -> Result<()> { - if batch.num_rows() == 0 { - // Skip empty batches - return Ok(()); - } - - let batch_size = batch.get_array_memory_size(); - - // Check if we need to rotate to a new file - let needs_rotation = if let Some(ref file) = self.current_write_file { - // Rotate if adding this batch would exceed the max file size - file.estimated_size() + batch_size > self.max_file_size_bytes - } else { - // No current file, need to create one - true - }; - - if needs_rotation && self.current_write_file.is_some() { - // Finish current file and add to queue - self.finish_current_file()?; - } - - // Create new file if needed - if self.current_write_file.is_none() { - self.current_write_file = - Some(self.spill_manager.create_in_progress_file("SpillPool")?); - } - - // Append batch to current file - if let Some(ref mut file) = self.current_write_file { - file.append_batch(batch)?; - } - - // Wake any waiting readers - self.wake(); - - Ok(()) + todo!() } /// Registers a waker to be notified when new data is available @@ -269,45 +321,6 @@ impl SpillPool { waker.wake(); } } - - /// Finalizes the current write file and adds it to the files queue. - /// - /// This is called automatically when files reach the size limit, but can - /// also be called explicitly to ensure all pending data is available for reading. - pub fn flush(&mut self) -> Result<()> { - if self.current_write_file.is_some() { - self.finish_current_file()?; - } - Ok(()) - } - - // Private helper methods - - /// Finishes the current write file and moves it to the files queue. - fn finish_current_file(&mut self) -> Result<()> { - if let Some(mut file) = self.current_write_file.take() { - // Finish writing to get the final file - let finished_file = file.finish()?; - - if let Some(temp_file) = finished_file { - // Add to queue - self.files.push_back(temp_file); - } - - // Wake any waiting readers since a new complete file is available - self.wake(); - } - - Ok(()) - } -} - -impl Drop for SpillPool { - fn drop(&mut self) { - // Flush any pending writes to ensure metrics are accurate - // We ignore errors here since Drop doesn't allow returning errors - let _ = self.flush(); - } } /// A stream that reads from a SpillPool in FIFO order. @@ -320,10 +333,9 @@ impl Drop for SpillPool { struct SpillPoolStream { /// Shared reference to the spill pool spill_pool: Arc>, - /// Current stream being read from - current_stream: Option, - /// Schema for the batches + /// Schema of the spilled data schema: SchemaRef, + // Other state? } impl SpillPoolStream { @@ -333,81 +345,18 @@ impl SpillPoolStream { /// /// * `spill_pool` - Shared reference to the pool to read from pub fn new(spill_pool: Arc>) -> Self { - let schema = { - let pool = spill_pool.lock(); - pool.spill_manager.schema() - }; - - Self { - spill_pool, - current_stream: None, - schema, - } + todo!() } } -impl futures::Stream for SpillPoolStream { +impl Stream for SpillPoolStream { type Item = Result; fn poll_next( mut self: std::pin::Pin<&mut Self>, cx: &mut std::task::Context<'_>, ) -> std::task::Poll> { - use futures::StreamExt; - use std::task::Poll; - - loop { - // If we have a current stream, try to read from it - if let Some(stream) = &mut self.current_stream { - match stream.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - return Poll::Ready(Some(Ok(batch))); - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(e))); - } - Poll::Ready(None) => { - // Current stream exhausted (finished reading a completed file) - self.current_stream = None; - // Continue loop to try getting next file - } - Poll::Pending => { - // Stream not ready yet - return Poll::Pending; - } - } - } - - // No current stream, try to get the next file to read - // Only read from completed files in the queue - let mut pool = self.spill_pool.lock(); - - if let Some(file) = pool.files.pop_front() { - // We have a completed file to read - let spill_manager = Arc::clone(&pool.spill_manager); - drop(pool); // Release lock before creating stream - - match spill_manager.read_spill_as_stream(file, None) { - Ok(stream) => { - self.current_stream = Some(stream); - // Continue loop to poll the new stream - } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - } - } else { - // No completed files available - let is_finalized = pool.is_finalized(); - if is_finalized { - // Pool is finalized and no more files - we're done - return Poll::Ready(None); - } - // Register waker and wait for more files - pool.register_waker(cx.waker().clone()); - return Poll::Pending; - } - } + todo!(); } } @@ -416,689 +365,3 @@ impl RecordBatchStream for SpillPoolStream { Arc::clone(&self.schema) } } - -#[cfg(test)] -mod tests { - use super::*; - use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; - use arrow::array::{ArrayRef, Int32Array}; - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common_runtime::SpawnedTask; - use datafusion_execution::runtime_env::RuntimeEnv; - use futures::StreamExt; - use std::task::Poll; - use tokio; - - // ============================================================================ - // Test Utilities - // ============================================================================ - - fn create_test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - ])) - } - - fn create_test_batch(start: i32, count: usize) -> RecordBatch { - let schema = create_test_schema(); - let a: ArrayRef = Arc::new(Int32Array::from( - (start..start + count as i32).collect::>(), - )); - let b: ArrayRef = Arc::new(Int32Array::from( - (start * 10..start * 10 + count as i32 * 10) - .step_by(10) - .collect::>(), - )); - RecordBatch::try_new(schema, vec![a, b]).unwrap() - } - - fn create_spill_pool(max_file_size: usize) -> SpillPool { - let env = Arc::new(RuntimeEnv::default()); - let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); - let schema = create_test_schema(); - let spill_manager = - Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema))); - - SpillPool::new(max_file_size, spill_manager) - } - - /// Helper to collect all batches from a stream - async fn collect_batches( - mut stream: SendableRecordBatchStream, - ) -> Result> { - let mut batches = Vec::new(); - while let Some(result) = stream.next().await { - batches.push(result?); - } - Ok(batches) - } - - // ============================================================================ - // Basic Functionality Tests - // ============================================================================ - - #[tokio::test] - async fn test_empty_pool_stream() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - pool.finalize(); // Mark as done with no data - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 0); - - Ok(()) - } - - #[tokio::test] - async fn test_single_batch() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Push one batch - let batch1 = create_test_batch(0, 10); - pool.push_batch(&batch1)?; - pool.flush()?; - pool.finalize(); - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 1); - assert_eq!(batches[0].num_rows(), 10); - assert_eq!(batches[0].num_columns(), 2); - - Ok(()) - } - - #[tokio::test] - async fn test_multiple_batches_single_file() -> Result<()> { - let mut pool = create_spill_pool(10 * 1024 * 1024); // Large file size - - // Push multiple batches - for i in 0..5 { - let batch = create_test_batch(i * 10, 10); - pool.push_batch(&batch)?; - } - pool.flush()?; - pool.finalize(); - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 5); - - // Verify FIFO order - for (i, batch) in batches.iter().enumerate() { - assert_eq!(batch.num_rows(), 10); - let col_a = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), (i as i32) * 10); - } - - Ok(()) - } - - #[tokio::test] - async fn test_file_rotation_on_size_limit() -> Result<()> { - // Small file size to force rotation - let mut pool = create_spill_pool(500); // ~500 bytes - - // Push multiple batches - should create multiple files - for i in 0..10 { - let batch = create_test_batch(i * 5, 5); - pool.push_batch(&batch)?; - } - pool.flush()?; - pool.finalize(); - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 10); - - // Verify all batches in FIFO order - for (i, batch) in batches.iter().enumerate() { - let col_a = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), (i as i32) * 5); - } - - Ok(()) - } - - #[tokio::test] - async fn test_empty_batches_skipped() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - let batch1 = create_test_batch(0, 10); - let empty_batch = RecordBatch::new_empty(create_test_schema()); - let batch2 = create_test_batch(10, 10); - - pool.push_batch(&batch1)?; - pool.push_batch(&empty_batch)?; // Should be skipped - pool.push_batch(&batch2)?; - pool.flush()?; - pool.finalize(); - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - // Should only have 2 batches (empty one skipped) - assert_eq!(batches.len(), 2); - - Ok(()) - } - - #[tokio::test] - async fn test_large_batches() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Create larger batches - let batch1 = create_test_batch(0, 1000); - let batch2 = create_test_batch(1000, 1000); - - pool.push_batch(&batch1)?; - pool.push_batch(&batch2)?; - pool.flush()?; - pool.finalize(); - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 2); - assert_eq!(batches[0].num_rows(), 1000); - assert_eq!(batches[1].num_rows(), 1000); - - Ok(()) - } - - // ============================================================================ - // Stream API Tests - // ============================================================================ - - #[tokio::test] - async fn test_stream_blocks_when_no_data() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - - let pool = Arc::new(Mutex::new(pool)); - let mut stream = SpillPool::reader(Arc::clone(&pool)); - - // Poll should return Pending since no data and not finalized - let poll_result = futures::poll!(stream.next()); - assert!(matches!(poll_result, Poll::Pending)); - - Ok(()) - } - - #[tokio::test] - async fn test_stream_wakes_on_push() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - let pool_clone = Arc::clone(&pool_arc); - let stream = SpillPool::reader(pool_clone); - - // Spawn a task that will push data after a delay - let writer_pool = Arc::clone(&pool_arc); - let _writer = SpawnedTask::spawn(async move { - tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; - let mut pool = writer_pool.lock(); - pool.push_batch(&create_test_batch(0, 10)).unwrap(); - pool.flush().unwrap(); - pool.finalize(); - }); - - // This should wait for data and then return it - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 1); - - Ok(()) - } - - #[tokio::test] - async fn test_stream_wakes_on_flush() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - let pool_clone = Arc::clone(&pool_arc); - let stream = SpillPool::reader(pool_clone); - - // Push without flush first - { - let mut pool = pool_arc.lock(); - pool.push_batch(&create_test_batch(0, 10)).unwrap(); - // Don't flush yet - data is in current_write_file - } - - // Spawn task to flush after delay - let writer_pool = Arc::clone(&pool_arc); - let _writer = SpawnedTask::spawn(async move { - tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; - let mut pool = writer_pool.lock(); - pool.flush().unwrap(); - pool.finalize(); - }); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 1); - - Ok(()) - } - - #[tokio::test] - async fn test_stream_wakes_on_finalize() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - let pool_clone = Arc::clone(&pool_arc); - let mut stream = SpillPool::reader(pool_clone); - - // First poll should be pending - let poll_result = futures::poll!(stream.next()); - assert!(matches!(poll_result, Poll::Pending)); - - // Finalize after delay - let writer_pool = Arc::clone(&pool_arc); - let _writer = SpawnedTask::spawn(async move { - tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; - writer_pool.lock().finalize(); - }); - - // Stream should eventually return None - let result = stream.next().await; - assert!(result.is_none()); - - Ok(()) - } - - #[tokio::test] - async fn test_finalize_before_flush() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Push data but DON'T flush - pool.push_batch(&create_test_batch(0, 10))?; - pool.finalize(); // Finalize without flush - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - // Data in current_write_file should still be lost since not flushed - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 0); - - Ok(()) - } - - // ============================================================================ - // Concurrent Reader/Writer Tests - // ============================================================================ - - #[tokio::test] - async fn test_concurrent_push_and_read() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - let writer_pool = Arc::clone(&pool_arc); - let writer = SpawnedTask::spawn(async move { - for i in 0..10 { - tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; - let mut pool = writer_pool.lock(); - pool.push_batch(&create_test_batch(i * 10, 10)).unwrap(); - pool.flush().unwrap(); - } - writer_pool.lock().finalize(); - }); - - let reader_pool = Arc::clone(&pool_arc); - let stream = SpillPool::reader(reader_pool); - let reader = SpawnedTask::spawn(async move { collect_batches(stream).await }); - - // Wait for both tasks - writer.await.unwrap(); - let batches = reader.await.unwrap()?; - - assert_eq!(batches.len(), 10); - for (i, batch) in batches.iter().enumerate() { - let col_a = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(col_a.value(0), (i as i32) * 10); - } - - Ok(()) - } - - #[tokio::test] - async fn test_reader_catches_up_to_writer() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - // Start reader before any data is written - let reader_pool = Arc::clone(&pool_arc); - let mut stream = SpillPool::reader(reader_pool); - - // Should return pending - let poll_result = futures::poll!(stream.next()); - assert!(matches!(poll_result, Poll::Pending)); - - // Now add data - { - let mut pool = pool_arc.lock(); - pool.push_batch(&create_test_batch(0, 10))?; - pool.flush()?; - pool.finalize(); - } - - // Now stream should have data - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 1); - - Ok(()) - } - - #[tokio::test] - async fn test_multiple_readers_same_pool() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Push some batches - for i in 0..5 { - pool.push_batch(&create_test_batch(i * 10, 10))?; - } - pool.flush()?; - pool.finalize(); - - let pool_arc = Arc::new(Mutex::new(pool)); - - // Create two readers - let stream1 = SpillPool::reader(Arc::clone(&pool_arc)); - let stream2 = SpillPool::reader(Arc::clone(&pool_arc)); - - // Read from both concurrently - let reader1 = SpawnedTask::spawn(async move { collect_batches(stream1).await }); - let reader2 = SpawnedTask::spawn(async move { collect_batches(stream2).await }); - - let batches1 = reader1.await.unwrap()?; - let batches2 = reader2.await.unwrap()?; - - // Each reader should consume different batches (pop_front removes from queue) - // The total number should be 5, but distributed between readers - let total = batches1.len() + batches2.len(); - assert_eq!(total, 5); - - Ok(()) - } - - #[tokio::test] - async fn test_file_cutover_during_read() -> Result<()> { - let pool = create_spill_pool(500); // Small size for rotation - let pool_arc = Arc::new(Mutex::new(pool)); - - let writer_pool = Arc::clone(&pool_arc); - let writer = SpawnedTask::spawn(async move { - // Write multiple batches that will cause rotation - for i in 0..8 { - { - let mut pool = writer_pool.lock(); - pool.push_batch(&create_test_batch(i * 5, 5)).unwrap(); - pool.flush().unwrap(); - } // Drop lock before sleep - tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; - } - writer_pool.lock().finalize(); - }); - - // Read concurrently - let reader_pool = Arc::clone(&pool_arc); - let stream = SpillPool::reader(reader_pool); - let reader = SpawnedTask::spawn(async move { collect_batches(stream).await }); - - writer.await.unwrap(); - let batches = reader.await.unwrap()?; - - // Should get all 8 batches despite file rotation - assert_eq!(batches.len(), 8); - - Ok(()) - } - - #[tokio::test] - async fn test_file_cutover_during_write() -> Result<()> { - let mut pool = create_spill_pool(300); // Very small to force frequent rotation - - // Push batches that will definitely cause rotation - for i in 0..5 { - let batch = create_test_batch(i * 10, 10); - pool.push_batch(&batch)?; - // Don't flush after each - let rotation happen naturally - } - pool.flush()?; - pool.finalize(); - - let pool = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool); - - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 5); - - Ok(()) - } - - // ============================================================================ - // Garbage Collection Tests - // ============================================================================ - - #[tokio::test] - async fn test_file_cleanup_after_read() -> Result<()> { - let mut pool = create_spill_pool(500); - - // Create multiple files - for i in 0..5 { - pool.push_batch(&create_test_batch(i * 10, 10))?; - pool.flush()?; // Each batch in its own file - } - - // Verify files exist before reading - let initial_file_count = pool.files.len(); - assert_eq!(initial_file_count, 5); - - pool.finalize(); - - let pool_arc = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(Arc::clone(&pool_arc)); - - // Read all batches - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 5); - - // All files should be consumed (dropped from queue) - let final_file_count = pool_arc.lock().files.len(); - assert_eq!(final_file_count, 0); - - Ok(()) - } - - #[tokio::test] - async fn test_cleanup_with_rotation() -> Result<()> { - let pool = create_spill_pool(400); - let pool_arc = Arc::new(Mutex::new(pool)); - - // Write and read concurrently - let writer_pool = Arc::clone(&pool_arc); - let writer = SpawnedTask::spawn(async move { - for i in 0..10 { - { - let mut pool = writer_pool.lock(); - pool.push_batch(&create_test_batch(i * 10, 10)).unwrap(); - pool.flush().unwrap(); - } // Drop lock before sleep - tokio::time::sleep(tokio::time::Duration::from_millis(20)).await; - } - writer_pool.lock().finalize(); - }); - - let reader_pool = Arc::clone(&pool_arc); - let stream = SpillPool::reader(reader_pool); - let reader = SpawnedTask::spawn(async move { - let mut batches = Vec::new(); - let mut stream = stream; - while let Some(result) = stream.next().await { - batches.push(result.unwrap()); - // Small delay to let writer create more files - tokio::time::sleep(tokio::time::Duration::from_millis(15)).await; - } - batches - }); - - writer.await.unwrap(); - let batches = reader.await.unwrap(); - - assert_eq!(batches.len(), 10); - - // All files should be cleaned up - let final_file_count = pool_arc.lock().files.len(); - assert_eq!(final_file_count, 0); - - Ok(()) - } - - #[tokio::test] - async fn test_cleanup_with_unflushed_file() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Create some flushed files - for i in 0..3 { - pool.push_batch(&create_test_batch(i * 10, 10))?; - pool.flush()?; - } - - // Add unflushed data - pool.push_batch(&create_test_batch(30, 10))?; - // Don't flush! - - // current_write_file should have data - assert!(pool.current_write_file.is_some()); - assert_eq!(pool.files.len(), 3); - - pool.finalize(); - - let pool_arc = Arc::new(Mutex::new(pool)); - let stream = SpillPool::reader(pool_arc); - - // Should only get the 3 flushed batches - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 3); - - Ok(()) - } - - // ============================================================================ - // Edge Cases & Error Handling Tests - // ============================================================================ - - #[tokio::test] - async fn test_interleaved_flush() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - // Push → flush - { - let mut pool = pool_arc.lock(); - pool.push_batch(&create_test_batch(0, 10))?; - pool.flush()?; - } - - // Read one batch - let stream = SpillPool::reader(Arc::clone(&pool_arc)); - let mut stream = stream; - let batch1 = stream.next().await.unwrap()?; - assert_eq!(batch1.num_rows(), 10); - - // Push → flush again - { - let mut pool = pool_arc.lock(); - pool.push_batch(&create_test_batch(10, 10))?; - pool.flush()?; - } - - // Read second batch from same stream - let batch2 = stream.next().await.unwrap()?; - assert_eq!(batch2.num_rows(), 10); - - // Finalize and verify stream ends - pool_arc.lock().finalize(); - let batch3 = stream.next().await; - assert!(batch3.is_none()); - - Ok(()) - } - - #[tokio::test] - async fn test_flush_empty_pool() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - // Flush with no data should be no-op - pool.flush()?; - pool.flush()?; // Multiple flushes - - assert_eq!(pool.files.len(), 0); - assert!(pool.current_write_file.is_none()); - - Ok(()) - } - - #[tokio::test] - async fn test_finalize_idempotent() -> Result<()> { - let mut pool = create_spill_pool(1024 * 1024); - - pool.push_batch(&create_test_batch(0, 10))?; - pool.flush()?; - - // Multiple finalize calls should be safe - pool.finalize(); - assert!(pool.is_finalized()); - pool.finalize(); - assert!(pool.is_finalized()); - pool.finalize(); - assert!(pool.is_finalized()); - - Ok(()) - } - - #[tokio::test] - async fn test_drop_flushes_current_file() -> Result<()> { - let pool = create_spill_pool(1024 * 1024); - let pool_arc = Arc::new(Mutex::new(pool)); - - // Push without flush - { - let mut pool = pool_arc.lock(); - pool.push_batch(&create_test_batch(0, 10)).unwrap(); - pool.flush().unwrap(); - pool.finalize(); - } - - // Drop should trigger flush in Drop impl - // (though in this case we already flushed) - - let stream = SpillPool::reader(pool_arc); - let batches = collect_batches(stream).await?; - assert_eq!(batches.len(), 1); - - Ok(()) - } -} From 30bcfc946d91aa630d4692aee2f80d0c64bedc9a Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 00:22:17 -0600 Subject: [PATCH 31/44] rewrite spilling pool, incorporate new architecture --- datafusion/execution/src/disk_manager.rs | 25 +- .../physical-plan/src/repartition/mod.rs | 212 ++-- .../src/spill/in_progress_spill_file.rs | 30 +- .../physical-plan/src/spill/spill_manager.rs | 4 +- .../physical-plan/src/spill/spill_pool.rs | 991 ++++++++++++++---- 5 files changed, 950 insertions(+), 312 deletions(-) diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs index 0d0ce2219ef60..327c50fa03e4f 100644 --- a/datafusion/execution/src/disk_manager.rs +++ b/datafusion/execution/src/disk_manager.rs @@ -313,11 +313,21 @@ impl DiskManager { /// must invoke [`Self::update_disk_usage`] to update the global disk usage counter. /// This ensures the disk manager can properly enforce usage limits configured by /// [`DiskManager::with_max_temp_directory_size`]. -#[derive(Debug, Clone)] +/// +/// This type is Clone-able, allowing multiple references to the same underlying file. +/// The file is deleted only when the last reference is dropped. +/// +/// The parent temporary directory is also kept alive as long as any reference to +/// this file exists, preventing premature cleanup of the directory. +/// +/// Once all references to this file are dropped, the file is deleted, and the +/// disk usage is subtracted from the disk manager's total. +#[derive(Debug)] pub struct RefCountedTempFile { /// The reference to the directory in which temporary files are created to ensure /// it is not cleaned up prior to the NamedTempFile _parent_temp_dir: Arc, + /// The underlying temporary file, wrapped in Arc to allow cloning tempfile: Arc, /// Tracks the current disk usage of this temporary file. See /// [`Self::update_disk_usage`] for more details. @@ -326,13 +336,24 @@ pub struct RefCountedTempFile { disk_manager: Arc, } +impl Clone for RefCountedTempFile { + fn clone(&self) -> Self { + Self { + _parent_temp_dir: Arc::clone(&self._parent_temp_dir), + tempfile: Arc::clone(&self.tempfile), + current_file_disk_usage: self.current_file_disk_usage, + disk_manager: Arc::clone(&self.disk_manager), + } + } +} + impl RefCountedTempFile { pub fn path(&self) -> &Path { self.tempfile.path() } pub fn inner(&self) -> &NamedTempFile { - &self.tempfile + self.tempfile.as_ref() } /// Updates the global disk usage counter after modifications to the underlying file. diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 77a150f3cfdf9..a335fe741bd88 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -36,7 +36,7 @@ use crate::metrics::{BaselineMetrics, SpillMetrics}; use crate::projection::{all_columns, make_with_child, update_expr, ProjectionExec}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; -use crate::spill::spill_pool::SpillPool; +use crate::spill::spill_pool::{self, SpillPoolWriter}; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -49,7 +49,7 @@ use datafusion_common::utils::transpose; use datafusion_common::{internal_err, ColumnStatistics, HashMap}; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; -use datafusion_execution::memory_pool::{FairSpillPool, GreedyMemoryPool, MemoryConsumer}; +use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -155,12 +155,11 @@ fn unbounded_channels( (senders, receivers) } -/// Output channel with its associated memory reservation and spill pool -#[derive(Clone)] +/// Output channel with its associated memory reservation and spill writer struct OutputChannel { sender: UnboundedDistributionSender, reservation: SharedMemoryReservation, - spill_pool: Arc>, + spill_writer: SpillPoolWriter, } /// Channels and resources for a single output partition @@ -171,9 +170,12 @@ struct PartitionChannels { rx: InputPartitionsToCurrentPartitionReceiver, /// Memory reservation for this output partition reservation: SharedMemoryReservation, - /// SpillPools for batched spilling - one per input partition (FIFO semantics) - /// Each (input, output) pair gets its own SpillPool to maintain proper ordering - spill_pools: Vec>>, + /// Spill writers for writing spilled data - one per input partition (FIFO semantics) + /// Wrapped in Option so they can be moved out when creating OutputChannels + spill_writers: Vec>, + /// Spill readers for reading spilled data - one per input partition (FIFO semantics) + /// Each (input, output) pair gets its own reader to maintain proper ordering + spill_readers: Vec, } struct ConsumingInputStreamsState { @@ -322,23 +324,20 @@ impl RepartitionExecState { let reservation = Arc::new(Mutex::new( MemoryConsumer::new(format!("{name}[{partition}]")) .with_can_spill(true) - .register(&(Arc::new(GreedyMemoryPool::new(1)) as Arc<_>)), + .register(context.memory_pool()), )); - // Create one SpillPool per input partition for this output partition + // Create one spill channel per input partition for this output partition // This ensures proper FIFO ordering within each (input, output) pair let max_file_size = context .session_config() .options() .execution .max_spill_file_size_bytes; - let spill_pools: Vec<_> = (0..num_input_partitions) - .map(|_| { - let spill_pool = - SpillPool::new(max_file_size, Arc::clone(&spill_manager)); - Arc::new(Mutex::new(spill_pool)) - }) - .collect(); + let (spill_writers, spill_readers): (Vec<_>, Vec<_>) = (0 + ..num_input_partitions) + .map(|_| spill_pool::channel(max_file_size, Arc::clone(&spill_manager))) + .unzip(); channels.insert( partition, @@ -346,7 +345,8 @@ impl RepartitionExecState { tx, rx, reservation, - spill_pools, + spill_readers, + spill_writers: spill_writers.into_iter().map(Some).collect(), }, ); } @@ -357,34 +357,38 @@ impl RepartitionExecState { std::mem::take(streams_and_metrics).into_iter().enumerate() { let txs: HashMap<_, _> = channels - .iter() + .iter_mut() .map(|(partition, channels)| { ( *partition, OutputChannel { sender: channels.tx[i].clone(), reservation: Arc::clone(&channels.reservation), - spill_pool: Arc::clone(&channels.spill_pools[i]), + spill_writer: channels.spill_writers[i] + .take() + .expect("spill_writer should not be taken yet"), }, ) }) .collect(); + // Extract senders for wait_for_task before moving txs + let senders: HashMap<_, _> = txs + .iter() + .map(|(partition, channel)| (*partition, channel.sender.clone())) + .collect(); + let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input( stream, - txs.clone(), + txs, partitioning.clone(), metrics, )); // In a separate task, wait for each input to be done // (and pass along any errors, including panic!s) - let wait_for_task = SpawnedTask::spawn(RepartitionExec::wait_for_task( - input_task, - txs.into_iter() - .map(|(partition, channel)| (partition, channel.sender)) - .collect(), - )); + let wait_for_task = + SpawnedTask::spawn(RepartitionExec::wait_for_task(input_task, senders)); spawned_tasks.push(wait_for_task); } *self = Self::ConsumingInputStreams(ConsumingInputStreamsState { @@ -831,7 +835,7 @@ impl ExecutionPlan for RepartitionExec { let num_input_partitions = input.output_partitioning().partition_count(); // lock scope - let (mut rx, reservation, spill_pools, abort_helper) = { + let (mut rx, reservation, spill_readers, abort_helper) = { // lock mutexes let mut state = state.lock(); let state = state.consume_input_streams( @@ -849,7 +853,7 @@ impl ExecutionPlan for RepartitionExec { let PartitionChannels { rx, reservation, - spill_pools, + spill_readers, .. } = state .channels @@ -859,7 +863,7 @@ impl ExecutionPlan for RepartitionExec { ( rx, reservation, - spill_pools, + spill_readers, Arc::clone(&state.abort_helper), ) }; @@ -870,13 +874,11 @@ impl ExecutionPlan for RepartitionExec { if preserve_order { // Store streams from all the input partitions: - // Each input partition gets its own SpillPool to maintain proper FIFO ordering + // Each input partition gets its own spill reader to maintain proper FIFO ordering let input_streams = rx .into_iter() - .enumerate() - .map(|(idx, receiver)| { - let spill_pool = Arc::clone(&spill_pools[idx]); - let spill_stream = SpillPool::reader(Arc::clone(&spill_pool)); + .zip(spill_readers) + .map(|(receiver, spill_stream)| { Box::pin(PerPartitionStream::new( Arc::clone(&schema_captured), receiver, @@ -905,9 +907,11 @@ impl ExecutionPlan for RepartitionExec { .with_spill_manager(spill_manager) .build() } else { - // Non-preserve-order case: single input stream, so use the first SpillPool - let spill_pool = Arc::clone(&spill_pools[0]); - let spill_stream = SpillPool::reader(Arc::clone(&spill_pool)); + // Non-preserve-order case: single input stream, so use the first spill reader + let spill_stream = spill_readers + .into_iter() + .next() + .expect("at least one spill reader should exist"); Ok(Box::pin(RepartitionStream::new( input.schema(), @@ -1187,30 +1191,44 @@ impl RepartitionExec { let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it if let Some(channel) = output_channels.get_mut(&partition) { - let (batch_to_send, is_memory_batch) = - match channel.reservation.lock().try_grow(size) { - Ok(_) => { - // Memory available - send in-memory batch - eprintln!("[SEND] Partition {}: Sending Memory batch with {} rows", partition, batch.num_rows()); - (RepartitionBatch::Memory(batch), true) - } - Err(_) => { - // We're memory limited - spill to SpillPool - // SpillPool handles file handle reuse and rotation - eprintln!("[SEND] Partition {}: Spilling batch with {} rows", partition, batch.num_rows()); - { - let mut pool = channel.spill_pool.lock(); - pool.push_batch(&batch)?; - } - // Send marker indicating batch was spilled - (RepartitionBatch::Spilled, false) - } - }; + let (batch_to_send, is_memory_batch) = match channel + .reservation + .lock() + .try_grow(size) + { + Ok(_) => { + // Memory available - send in-memory batch + eprintln!( + "[SEND] Partition {}: Sending Memory batch with {} rows", + partition, + batch.num_rows() + ); + (RepartitionBatch::Memory(batch), true) + } + Err(_) => { + // We're memory limited - spill to SpillPool + // SpillPool handles file handle reuse and rotation + eprintln!( + "[SEND] Partition {}: Spilling batch with {} rows", + partition, + batch.num_rows() + ); + channel.spill_writer.push_batch(&batch)?; + // Send marker indicating batch was spilled + (RepartitionBatch::Spilled, false) + } + }; - eprintln!("[SEND] Partition {}: About to send {:?} to channel", partition, - if is_memory_batch { "Memory" } else { "Spilled" }); + eprintln!( + "[SEND] Partition {}: About to send {:?} to channel", + partition, + if is_memory_batch { "Memory" } else { "Spilled" } + ); if channel.sender.send(Some(Ok(batch_to_send))).await.is_err() { - eprintln!("[SEND] Partition {}: Send failed, receiver hung up", partition); + eprintln!( + "[SEND] Partition {}: Send failed, receiver hung up", + partition + ); // If the other end has hung up, it was an early shutdown (e.g. LIMIT) // Only shrink memory if it was a memory batch if is_memory_batch { @@ -1218,7 +1236,10 @@ impl RepartitionExec { } output_channels.remove(&partition); } else { - eprintln!("[SEND] Partition {}: Send completed successfully", partition); + eprintln!( + "[SEND] Partition {}: Send completed successfully", + partition + ); } } timer.done(); @@ -1248,14 +1269,8 @@ impl RepartitionExec { } } - // Flush all spill pools now that input is done - eprintln!("[SEND] Flushing all spill pools for {} output channels", output_channels.len()); - for (partition, channel) in output_channels.iter() { - let mut pool = channel.spill_pool.lock(); - eprintln!("[SEND] Flushing spill pool for partition {}", partition); - pool.flush()?; - } - + // Spill writers will auto-finalize when dropped + // No need for explicit flush Ok(()) } @@ -1298,7 +1313,10 @@ impl RepartitionExec { // Input task completed successfully Ok(Ok(())) => { // notify each output partition that this input partition has no more data - eprintln!("[SEND] Input task completed, sending None to {} output partitions", txs.len()); + eprintln!( + "[SEND] Input task completed, sending None to {} output partitions", + txs.len() + ); for (partition, tx) in txs { eprintln!("[SEND] Sending None to partition {}", partition); tx.send(None).await.ok(); @@ -1372,21 +1390,27 @@ impl Stream for RepartitionStream { use futures::StreamExt; loop { - eprintln!("[RepartitionStream] State: {:?}, remaining_partitions: {}", - self.state, self.remaining_partitions); + eprintln!( + "[RepartitionStream] State: {:?}, remaining_partitions: {}", + self.state, self.remaining_partitions + ); match self.state { StreamState::ReadingMemory => { // Poll the memory channel for next message let value = match self.input.recv().poll_unpin(cx) { Poll::Ready(v) => { - eprintln!("[RepartitionStream] Memory channel poll returned: {:?}", - match &v { - Some(Some(Ok(RepartitionBatch::Memory(_)))) => "Some(Some(Ok(Memory)))", - Some(Some(Ok(RepartitionBatch::Spilled))) => "Some(Some(Ok(Spilled)))", - Some(Some(Err(_))) => "Some(Some(Err))", - Some(None) => "Some(None)", - None => "None", - }); + eprintln!( + "[RepartitionStream] Memory channel poll returned: {:?}", + match &v { + Some(Some(Ok(RepartitionBatch::Memory(_)))) => + "Some(Some(Ok(Memory)))", + Some(Some(Ok(RepartitionBatch::Spilled))) => + "Some(Some(Ok(Spilled)))", + Some(Some(Err(_))) => "Some(Some(Err))", + Some(None) => "Some(None)", + None => "None", + } + ); v } Poll::Pending => { @@ -1458,7 +1482,9 @@ impl Stream for RepartitionStream { self.state = StreamState::ReadingMemory; } Poll::Pending => { - eprintln!("[RepartitionStream] Spill stream poll returned: Pending"); + eprintln!( + "[RepartitionStream] Spill stream poll returned: Pending" + ); // Spilled batch not ready yet, must wait // This preserves ordering by blocking until spill data arrives return Poll::Pending; @@ -1534,14 +1560,18 @@ impl Stream for PerPartitionStream { // Poll the memory channel for next message let value = match self.receiver.recv().poll_unpin(cx) { Poll::Ready(v) => { - eprintln!("[PerPartitionStream] Memory channel poll returned: {:?}", - match &v { - Some(Some(Ok(RepartitionBatch::Memory(_)))) => "Some(Some(Ok(Memory)))", - Some(Some(Ok(RepartitionBatch::Spilled))) => "Some(Some(Ok(Spilled)))", - Some(Some(Err(_))) => "Some(Some(Err))", - Some(None) => "Some(None)", - None => "None", - }); + eprintln!( + "[PerPartitionStream] Memory channel poll returned: {:?}", + match &v { + Some(Some(Ok(RepartitionBatch::Memory(_)))) => + "Some(Some(Ok(Memory)))", + Some(Some(Ok(RepartitionBatch::Spilled))) => + "Some(Some(Ok(Spilled)))", + Some(Some(Err(_))) => "Some(Some(Err))", + Some(None) => "Some(None)", + None => "None", + } + ); v } Poll::Pending => { @@ -1575,7 +1605,9 @@ impl Stream for PerPartitionStream { } }, Some(None) => { - eprintln!("[PerPartitionStream] Received None, ending stream"); + eprintln!( + "[PerPartitionStream] Received None, ending stream" + ); return Poll::Ready(None); } None => { diff --git a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs index 5042354da3f9a..e7f354a73b4cd 100644 --- a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs +++ b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs @@ -35,10 +35,6 @@ pub struct InProgressSpillFile { writer: Option, /// Lazily initialized in-progress file, it will be moved out when the `finish` method is invoked in_progress_file: Option, - /// Number of batches written to this file - batch_count: usize, - /// Estimated size of data written to this file in bytes - estimated_size: usize, } impl InProgressSpillFile { @@ -50,8 +46,6 @@ impl InProgressSpillFile { spill_writer, in_progress_file: Some(in_progress_file), writer: None, - batch_count: 0, - estimated_size: 0, } } @@ -90,14 +84,16 @@ impl InProgressSpillFile { // Update metrics self.spill_writer.metrics.spilled_rows.add(spilled_rows); - - // Update stats - self.batch_count += 1; - self.estimated_size += batch.get_array_memory_size(); } Ok(()) } + /// Returns a reference to the in-progress file, if it exists. + /// This can be used to get the file path for creating readers before the file is finished. + pub fn file(&self) -> Option<&RefCountedTempFile> { + self.in_progress_file.as_ref() + } + /// Finalizes the file, returning the completed file reference. /// If there are no batches spilled before, it returns `None`. pub fn finish(&mut self) -> Result> { @@ -117,18 +113,4 @@ impl InProgressSpillFile { Ok(self.in_progress_file.take()) } - - pub fn file(&self) -> Option<&RefCountedTempFile> { - self.in_progress_file.as_ref() - } - - /// Returns the number of batches written to this file - pub fn batch_count(&self) -> usize { - self.batch_count - } - - /// Returns the estimated size of data written to this file in bytes - pub fn estimated_size(&self) -> usize { - self.estimated_size - } } diff --git a/datafusion/physical-plan/src/spill/spill_manager.rs b/datafusion/physical-plan/src/spill/spill_manager.rs index b43323e623994..6fd97a8e2e6a0 100644 --- a/datafusion/physical-plan/src/spill/spill_manager.rs +++ b/datafusion/physical-plan/src/spill/spill_manager.rs @@ -73,8 +73,8 @@ impl SpillManager { } /// Returns the schema for batches managed by this SpillManager - pub fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + pub fn schema(&self) -> &SchemaRef { + &self.schema } /// Creates a temporary file for in-progress operations, returning an error diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index ce8a870a2b249..d8d2b5f0d4858 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -17,10 +17,10 @@ //! Spill pool for managing spill files with FIFO semantics. +use futures::{Stream, StreamExt}; use std::collections::VecDeque; use std::sync::Arc; use std::task::Waker; -use futures::{Stream, StreamExt}; use parking_lot::Mutex; @@ -33,126 +33,206 @@ use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; use super::in_progress_spill_file::InProgressSpillFile; use super::spill_manager::SpillManager; -/// A pool of spill files that manages batch-level spilling with FIFO semantics. -/// -/// # Overview -/// -/// The `SpillPool` provides a centralized mechanism for spilling record batches to disk -/// when memory is constrained. It manages a collection of spill files, each containing -/// multiple batches, with configurable maximum file sizes. -/// -/// # Design +/// Shared state between the writer and readers of a spill pool. +/// This contains the queue of files and coordination state. +struct SpillPoolShared { + /// Queue of ALL files (including the current write file if it exists). + /// Readers always read from the front of this queue (FIFO). + files: VecDeque>>, + /// SpillManager for creating files and tracking metrics + spill_manager: Arc, + /// Pool-level wakers to notify when new files are available + wakers: Vec, +} + +impl SpillPoolShared { + /// Creates a new shared pool state + fn new(spill_manager: Arc) -> Self { + Self { + files: VecDeque::new(), + spill_manager, + wakers: Vec::new(), + } + } + + /// Registers a waker to be notified when new data is available (pool-level) + fn register_waker(&mut self, waker: Waker) { + // Only register if not already present (avoid duplicates) + if !self.wakers.iter().any(|w| w.will_wake(&waker)) { + self.wakers.push(waker); + } + } + + /// Wakes all pool-level readers + fn wake(&mut self) { + for waker in self.wakers.drain(..) { + waker.wake(); + } + } +} + +/// Writer for a spill pool. Provides exclusive write access with FIFO semantics. +pub struct SpillPoolWriter { + /// Maximum size in bytes before rotating to a new file + max_file_size_bytes: usize, + /// Writer's reference to the current file (also in the shared files queue) + current_write_file: Option>>, + /// Shared state with readers + shared: Arc>, +} + +impl SpillPoolWriter { + /// Spills a batch to the pool, rotating files when necessary. + /// + /// If the current file would exceed `max_file_size_bytes` after adding + /// this batch, the file is finalized and a new one is started. + /// + /// # Errors + /// + /// Returns an error if disk I/O fails or disk quota is exceeded. + pub fn push_batch(&mut self, batch: &RecordBatch) -> Result<()> { + if batch.num_rows() == 0 { + // Skip empty batches + return Ok(()); + } + + let batch_size = batch.get_array_memory_size(); + + // Create new file if we don't have one yet + if self.current_write_file.is_none() { + let spill_manager = { + let shared = self.shared.lock(); + Arc::clone(&shared.spill_manager) + }; + + let writer = spill_manager.create_in_progress_file("SpillPool")?; + // Clone the file so readers can access it immediately + let file = writer.file().unwrap().clone(); + + let file_shared = Arc::new(Mutex::new(ActiveSpillFileShared { + writer: Some(writer), + file: Some(file), // Set immediately so readers can access it + batches_written: 0, + estimated_size: 0, + writer_finished: false, + wakers: Vec::new(), + })); + + // Push to shared queue and keep reference for writing + { + let mut shared = self.shared.lock(); + shared.files.push_back(Arc::clone(&file_shared)); + shared.wake(); // Wake readers waiting for new files + } + self.current_write_file = Some(file_shared); + } + + let current_write_file = self.current_write_file.take(); + + // Write batch to current file + if let Some(current_file) = current_write_file { + let mut file_shared = current_file.lock(); + + // Append the batch + if let Some(ref mut writer) = file_shared.writer { + writer.append_batch(batch)?; + file_shared.batches_written += 1; + file_shared.estimated_size += batch_size; + } + + // Wake readers waiting on this specific file + file_shared.wake_all(); + + // Check if we need to rotate + let needs_rotation = file_shared.estimated_size > self.max_file_size_bytes; + + if needs_rotation { + // Finish the IPC writer + if let Some(mut writer) = file_shared.writer.take() { + writer.finish()?; + } + // Mark as finished so readers know not to wait for more data + file_shared.writer_finished = true; + // Wake readers waiting on this file (it's now finished) + file_shared.wake_all(); + } else { + // Release lock + drop(file_shared); + // Put back the current file for further writing + self.current_write_file = Some(current_file); + } + } + + Ok(()) + } +} + +impl Drop for SpillPoolWriter { + fn drop(&mut self) { + // Finalize the current file when writer is dropped + if let Some(current_file) = self.current_write_file.take() { + let mut file_shared = current_file.lock(); + + // Finish the current writer if it exists + if let Some(mut writer) = file_shared.writer.take() { + // Ignore errors on drop - we're in destructor + let _ = writer.finish(); + } + + // Mark as finished so readers know not to wait for more data + file_shared.writer_finished = true; + + // Wake readers waiting on this file (it's now finished) + file_shared.wake_all(); + } + } +} + +/// Creates a paired writer and reader for a spill pool with channel-like semantics. /// -/// - **FIFO (Queue) semantics**: Batches are read in the order they were spilled -/// - **File handle reuse**: Multiple batches are written to the same file to minimize syscalls -/// - **Automatic file rotation**: When a file exceeds `max_file_size_bytes`, rotate to a new file -/// - **Sequential reading**: Uses IPC Stream format's natural sequential access pattern -/// - **Automatic cleanup**: Files are deleted once fully consumed -/// - **Concurrent reading/writing**: Readers can poll and consume batches while writers continue -/// to spill more data, enabling pipelined processing +/// This is the recommended way to create a spill pool. The writer has exclusive +/// write access, and the reader can consume batches in FIFO order. The reader +/// can start reading immediately while the writer continues to write more data. /// -/// # Usage Example +/// # Arguments /// -/// This example demonstrates concurrent reading and writing. Note that the reader can start -/// polling for batches before the writer has finished, enabling pipelined processing: +/// * `max_file_size_bytes` - Maximum size per file before rotation +/// * `spill_manager` - Manager for file creation and metrics /// -/// ``` -/// # use std::sync::Arc; -/// # use parking_lot::Mutex; -/// # use arrow::array::{ArrayRef, Int32Array}; -/// # use arrow::datatypes::{DataType, Field, Schema}; -/// # use arrow::record_batch::RecordBatch; -/// # use datafusion_common::Result; -/// # use datafusion_execution::runtime_env::RuntimeEnv; -/// # use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; -/// # use datafusion_physical_plan::SpillManager; -/// # use datafusion_physical_plan::spill::spill_pool::SpillPool; -/// # use datafusion_common_runtime::SpawnedTask; -/// # use futures::StreamExt; -/// # -/// # #[tokio::main] -/// # async fn main() -> Result<()> { -/// # // Create test schema and batches -/// # let schema = Arc::new(Schema::new(vec![ -/// # Field::new("a", DataType::Int32, false), -/// # ])); -/// # let batch1 = RecordBatch::try_new( -/// # Arc::clone(&schema), -/// # vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef], -/// # )?; -/// # let batch2 = RecordBatch::try_new( -/// # Arc::clone(&schema), -/// # vec![Arc::new(Int32Array::from(vec![4, 5, 6])) as ArrayRef], -/// # )?; -/// # let batch3 = RecordBatch::try_new( -/// # Arc::clone(&schema), -/// # vec![Arc::new(Int32Array::from(vec![7, 8, 9])) as ArrayRef], -/// # )?; -/// # -/// # // Set up spill manager -/// # let env = Arc::new(RuntimeEnv::default()); -/// # let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); -/// # let spill_manager = Arc::new(SpillManager::new(env, metrics, schema)); -/// # -/// // Create a spill pool -/// let pool = SpillPool::new( -/// 100 * 1024 * 1024, // 100MB max per file -/// spill_manager, -/// ); -/// let pool = Arc::new(Mutex::new(pool)); +/// # Returns /// -/// // Create a reader that will consume batches as they become available -/// let stream = SpillPool::reader(pool.clone()); +/// A tuple of `(SpillPoolWriter, SpillPoolReader)` that share the same underlying pool /// -/// // Spawn a task to write batches concurrently -/// let writer = SpawnedTask::spawn({ -/// let pool = pool.clone(); -/// async move { -/// let mut pool = pool.lock(); -/// pool.push_batch(&batch1).unwrap(); -/// pool.push_batch(&batch2).unwrap(); -/// pool.flush().unwrap(); // Finalize current file -/// pool.push_batch(&batch3).unwrap(); -/// pool.flush().unwrap(); // Flush the last batch -/// pool.finalize(); // Signal no more writes -/// } -/// }); +/// # Example /// -/// // Reader can start consuming immediately, even while writer is still working -/// let reader = SpawnedTask::spawn(async move { -/// let mut batches = vec![]; -/// let mut stream = stream; -/// while let Some(result) = stream.next().await { -/// batches.push(result.unwrap()); -/// } -/// batches -/// }); +/// ```ignore +/// let (mut writer, mut reader) = spill_pool::channel(1024 * 1024, spill_manager); /// -/// // Wait for both tasks to complete -/// writer.join().await.unwrap(); -/// let batches = reader.join().await.unwrap(); +/// // Writer writes batches +/// writer.push_batch(&batch)?; /// -/// assert_eq!(batches.len(), 3); -/// assert_eq!(batches[0].num_rows(), 3); -/// assert_eq!(batches[1].num_rows(), 3); -/// assert_eq!(batches[2].num_rows(), 3); -/// # Ok(()) -/// # } +/// // Reader consumes batches (can happen concurrently) +/// while let Some(result) = reader.next().await { +/// let batch = result?; +/// // Process batch... +/// } /// ``` -/// -/// # Thread Safety -/// -/// `SpillPool` is not thread-safe and should be used from a single thread or -/// protected with appropriate synchronization (e.g., `Arc>`). -pub struct SpillPool { - /// Maximum size in bytes before rotating to a new file. - /// Typically initialized from the configuration option - /// `datafusion.execution.max_spill_file_size_bytes`. +pub fn channel( max_file_size_bytes: usize, - /// SpillManager for creating files and tracking metrics spill_manager: Arc, - /// Wakers to notify when new data is available for readers - wakers: Vec, +) -> (SpillPoolWriter, SendableRecordBatchStream) { + let schema = Arc::clone(spill_manager.schema()); + let shared = Arc::new(Mutex::new(SpillPoolShared::new(spill_manager))); + + let writer = SpillPoolWriter { + max_file_size_bytes, + current_write_file: None, + shared: Arc::clone(&shared), + }; + + let reader = SpillPoolReader::new(shared, schema); + + (writer, Box::pin(reader)) } /// Shared state between writer and readers for an active spill file. @@ -160,14 +240,34 @@ pub struct SpillPool { struct ActiveSpillFileShared { /// Writer handle - taken (set to None) when finish() is called writer: Option, - /// Path to the spill file for creating readers - file_path: RefCountedTempFile, - /// Schema for creating readers - schema: SchemaRef, + /// The spill file, set when the writer finishes. + /// Taken by the reader when creating a stream (the file stays open via file handles). + file: Option, /// Total number of batches written to this file batches_written: usize, + /// Estimated size in bytes of data written to this file + estimated_size: usize, /// Whether the writer has finished writing to this file writer_finished: bool, + /// Wakers for readers waiting on this specific file + wakers: Vec, +} + +impl ActiveSpillFileShared { + /// Registers a waker to be notified when new data is written to this file + fn register_waker(&mut self, waker: Waker) { + // Only register if not already present (avoid duplicates) + if !self.wakers.iter().any(|w| w.will_wake(&waker)) { + self.wakers.push(waker); + } + } + + /// Wakes all readers waiting on this file + fn wake_all(&mut self) { + for waker in self.wakers.drain(..) { + waker.wake(); + } + } } /// Reader state for a SpillFile (owned by individual SpillFile instances). @@ -212,34 +312,48 @@ impl Stream for SpillFile { use std::task::Poll; // Step 1: Lock shared state and check coordination - let (should_read, file_path, schema, batches_written) = { - let shared = self.shared.lock(); + let (should_read, file) = { + let mut shared = self.shared.lock(); // Determine if we can read let batches_read = self.reader.as_ref().map_or(0, |r| r.batches_read); if batches_read < shared.batches_written { - // More data available to read - (true, shared.file_path.clone(), Arc::clone(&shared.schema), shared.batches_written) + // More data available to read - take the file if we don't have a reader yet + let file = if self.reader.is_none() { + shared.file.take() + } else { + None + }; + (true, file) } else if shared.writer_finished { // No more data and writer is done - EOF return Poll::Ready(None); } else { - // Caught up to writer, but writer still active - wait + // Caught up to writer, but writer still active - register waker and wait + shared.register_waker(cx.waker().clone()); return Poll::Pending; } }; // Lock released here // Step 2: Lazy-create reader stream if needed if self.reader.is_none() && should_read { - match self.spill_manager.read_spill_as_stream(file_path, None) { - Ok(stream) => { - self.reader = Some(SpillFileReader { - stream, - batches_read: 0, - }); + if let Some(file) = file { + match self.spill_manager.read_spill_as_stream(file, None) { + Ok(stream) => { + self.reader = Some(SpillFileReader { + stream, + batches_read: 0, + }); + } + Err(e) => return Poll::Ready(Some(Err(e))), } - Err(e) => return Poll::Ready(Some(Err(e))), + } else { + // File not available yet (writer hasn't finished or already taken) + // Register waker and wait for file to be ready + let mut shared = self.shared.lock(); + shared.register_waker(cx.waker().clone()); + return Poll::Pending; } } @@ -266,102 +380,591 @@ impl Stream for SpillFile { } } -impl SpillPool { - /// Creates a new SpillPool with FIFO semantics. - /// - /// # Arguments - /// - /// * `max_file_size_bytes` - Maximum size per file before rotation (e.g., 100MB) - /// * `spill_manager` - Manager for file creation and metrics - pub fn new(max_file_size_bytes: usize, spill_manager: Arc) -> Self { - todo!() - } - - /// Creates a stream reader for this pool. - /// - /// The stream automatically handles file rotation and can read concurrently - /// while writes continue to the pool. When the stream catches up to the writer, - /// it will return `Poll::Pending` and wait for more data. - /// - /// # Arguments - /// - /// * `pool` - Shared reference to the SpillPool - /// - /// # Returns - /// - /// A `SpillPoolStream` that returns batches in FIFO order and ends when the pool - /// is finalized and all data has been read - pub fn reader(pool: Arc>) -> SendableRecordBatchStream { - Box::pin(SpillPoolStream::new(pool)) - } - - /// Spills a batch to the pool, rotating files when necessary. - /// - /// If the current file would exceed `max_file_size_bytes` after adding - /// this batch, the file is finalized and a new one is started. - /// - /// # Errors - /// - /// Returns an error if disk I/O fails or disk quota is exceeded. - pub fn push_batch(&mut self, batch: &RecordBatch) -> Result<()> { - todo!() - } - - /// Registers a waker to be notified when new data is available - fn register_waker(&mut self, waker: Waker) { - // Only register if not already present (avoid duplicates) - if !self.wakers.iter().any(|w| w.will_wake(&waker)) { - self.wakers.push(waker); - } - } - - /// Wakes all registered readers - fn wake(&mut self) { - for waker in self.wakers.drain(..) { - waker.wake(); - } - } -} - /// A stream that reads from a SpillPool in FIFO order. /// /// The stream automatically handles file rotation and reads from completed files. /// When no completed files are available, it returns `Poll::Pending` and waits /// for the writer to complete more files. /// -/// The stream ends (`Poll::Ready(None)`) when the pool is finalized and all data has been read. -struct SpillPoolStream { +/// The stream will never end, it is an infinite stream. +pub struct SpillPoolReader { /// Shared reference to the spill pool - spill_pool: Arc>, + shared: Arc>, + /// Current SpillFile we're reading from + current_file: Option, /// Schema of the spilled data schema: SchemaRef, - // Other state? } -impl SpillPoolStream { - /// Creates a new infinite stream from a SpillPool. +impl SpillPoolReader { + /// Creates a new reader from shared pool state. + /// + /// This is private - use the `channel()` function to create a reader/writer pair. /// /// # Arguments /// - /// * `spill_pool` - Shared reference to the pool to read from - pub fn new(spill_pool: Arc>) -> Self { - todo!() + /// * `shared` - Shared reference to the pool state + fn new(shared: Arc>, schema: SchemaRef) -> Self { + Self { + shared, + current_file: None, + schema, + } } } -impl Stream for SpillPoolStream { +impl Stream for SpillPoolReader { type Item = Result; fn poll_next( mut self: std::pin::Pin<&mut Self>, cx: &mut std::task::Context<'_>, ) -> std::task::Poll> { - todo!(); + use std::task::Poll; + + loop { + // If we have a current file, try to read from it + if let Some(ref mut file) = self.current_file { + match file.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + // Got a batch, return it + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + // Error reading batch + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Current file stream exhausted + // Check if this file is marked as writer_finished + let writer_finished = { file.shared.lock().writer_finished }; + + if writer_finished { + // File is complete, pop it from the queue and move to next + let mut shared = self.shared.lock(); + shared.files.pop_front(); + drop(shared); // Release lock + + // Clear current file and continue loop to get next file + self.current_file = None; + continue; + } else { + // Stream exhausted but writer not finished - unexpected + // This shouldn't happen with proper coordination + return Poll::Ready(None); + } + } + Poll::Pending => { + // File not ready yet (waiting for writer) + // Register waker so we get notified when writer adds more batches + let mut shared = self.shared.lock(); + shared.register_waker(cx.waker().clone()); + return Poll::Pending; + } + } + } + + // No current file, need to get the next one + let mut shared = self.shared.lock(); + + // Peek at the front of the queue (don't pop yet) + if let Some(file_shared) = shared.files.front() { + // Create a SpillFile from the shared state + let spill_manager = Arc::clone(&shared.spill_manager); + let file_shared = Arc::clone(file_shared); + drop(shared); // Release lock before creating SpillFile + + self.current_file = Some(SpillFile { + shared: file_shared, + reader: None, + spill_manager, + }); + + // Continue loop to poll the new file + continue; + } + + // Done with this file, no more files available + // Register waker that will get notified when new files are added + shared.register_waker(cx.waker().clone()); + return Poll::Pending; + } } } -impl RecordBatchStream for SpillPoolStream { +impl RecordBatchStream for SpillPoolReader { fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; + use arrow::array::{ArrayRef, Int32Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_execution::runtime_env::RuntimeEnv; + use futures::StreamExt; + + fn create_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])) + } + + fn create_test_batch(start: i32, count: usize) -> RecordBatch { + let schema = create_test_schema(); + let a: ArrayRef = Arc::new(Int32Array::from( + (start..start + count as i32).collect::>(), + )); + RecordBatch::try_new(schema, vec![a]).unwrap() + } + + fn create_spill_channel( + max_file_size: usize, + ) -> (SpillPoolWriter, SendableRecordBatchStream) { + let env = Arc::new(RuntimeEnv::default()); + let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let schema = create_test_schema(); + let spill_manager = Arc::new(SpillManager::new(env, metrics, schema)); + + channel(max_file_size, spill_manager) + } + + #[tokio::test] + async fn test_basic_write_and_read() -> Result<()> { + let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + + // Write one batch + let batch1 = create_test_batch(0, 10); + writer.push_batch(&batch1)?; + + // Read the batch + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 10); + + // Write another batch + let batch2 = create_test_batch(10, 5); + writer.push_batch(&batch2)?; + // Read the second batch + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 5); + + Ok(()) + } + + #[tokio::test] + async fn test_single_batch_write_read() -> Result<()> { + let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + + // Write one batch + let batch = create_test_batch(0, 5); + writer.push_batch(&batch)?; + + // Read it back + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 5); + + // Verify the actual data + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), 0); + assert_eq!(col.value(4), 4); + + Ok(()) + } + + #[tokio::test] + async fn test_multiple_batches_sequential() -> Result<()> { + let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + + // Write multiple batches + for i in 0..5 { + let batch = create_test_batch(i * 10, 10); + writer.push_batch(&batch)?; + } + + // Read all batches and verify FIFO order + for i in 0..5 { + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 10); + + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), i * 10, "Batch {} not in FIFO order", i); + } + + Ok(()) + } + + #[tokio::test] + async fn test_empty_writer() -> Result<()> { + let (_writer, reader) = create_spill_channel(1024 * 1024); + + // Reader should pend since no batches were written + let mut reader = reader; + let result = + tokio::time::timeout(std::time::Duration::from_millis(100), reader.next()) + .await; + + assert!(result.is_err(), "Reader should timeout on empty writer"); + + Ok(()) + } + + #[tokio::test] + async fn test_empty_batch_skipping() -> Result<()> { + let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + + // Write empty batch + let empty_batch = create_test_batch(0, 0); + writer.push_batch(&empty_batch)?; + + // Write non-empty batch + let batch = create_test_batch(0, 5); + writer.push_batch(&batch)?; + + // Should only read the non-empty batch + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 5); + + Ok(()) + } + + #[tokio::test] + async fn test_rotation_triggered_by_size() -> Result<()> { + // Set a small max_file_size to trigger rotation after one batch + let batch1 = create_test_batch(0, 10); + let batch_size = batch1.get_array_memory_size() + 1; + + let (mut writer, mut reader) = create_spill_channel(batch_size); + + // Write first batch (should fit in first file) + writer.push_batch(&batch1)?; + + // Write second batch (should trigger rotation to second file) + let batch2 = create_test_batch(10, 10); + assert!( + batch2.get_array_memory_size() <= batch_size, + "batch2 size {} exceeds limit {batch_size}", + batch2.get_array_memory_size(), + ); + assert!( + batch1.get_array_memory_size() + batch2.get_array_memory_size() > batch_size, + "Combined size {} does not exceed limit to trigger rotation", + batch1.get_array_memory_size() + batch2.get_array_memory_size() + ); + writer.push_batch(&batch2)?; + + // Read both batches + let result1 = reader.next().await.unwrap()?; + assert_eq!(result1.num_rows(), 10); + + let result2 = reader.next().await.unwrap()?; + assert_eq!(result2.num_rows(), 10); + + Ok(()) + } + + #[tokio::test] + async fn test_multiple_rotations() -> Result<()> { + let batches = (0..10) + .map(|i| create_test_batch(i * 10, 10)) + .collect::>(); + + let batch_size = batches[0].get_array_memory_size() * 2 + 1; + + // Very small max_file_size to force frequent rotations + let (mut writer, mut reader) = create_spill_channel(batch_size); + + // Write many batches to cause multiple rotations + for i in 0..10 { + let batch = create_test_batch(i * 10, 10); + writer.push_batch(&batch)?; + } + + // Read all batches and verify order + for i in 0..10 { + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 10); + + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + col.value(0), + i * 10, + "Batch {} not in correct order after rotations", + i + ); + } + + Ok(()) + } + + #[tokio::test] + async fn test_single_batch_larger_than_limit() -> Result<()> { + // Very small limit + let (mut writer, mut reader) = create_spill_channel(100); + + // Write a batch that exceeds the limit + let large_batch = create_test_batch(0, 100); + writer.push_batch(&large_batch)?; + + // Should still write and read successfully + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 100); + + // Next batch should go to a new file + let batch2 = create_test_batch(100, 10); + writer.push_batch(&batch2)?; + + let result2 = reader.next().await.unwrap()?; + assert_eq!(result2.num_rows(), 10); + + Ok(()) + } + + #[tokio::test] + async fn test_very_small_max_file_size() -> Result<()> { + // Test with just 1 byte max (extreme case) + let (mut writer, mut reader) = create_spill_channel(1); + + // Any batch will exceed this limit + let batch = create_test_batch(0, 5); + writer.push_batch(&batch)?; + + // Should still work + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 5); + + Ok(()) + } + + #[tokio::test] + async fn test_exact_size_boundary() -> Result<()> { + // Create a batch and measure its approximate size + let batch = create_test_batch(0, 10); + let batch_size = batch.get_array_memory_size(); + + // Set max_file_size to exactly the batch size + let (mut writer, mut reader) = create_spill_channel(batch_size); + + // Write two batches + writer.push_batch(&batch)?; + let batch2 = create_test_batch(10, 10); + writer.push_batch(&batch2)?; + + // Both should be readable + let result1 = reader.next().await.unwrap()?; + assert_eq!(result1.num_rows(), 10); + + let result2 = reader.next().await.unwrap()?; + assert_eq!(result2.num_rows(), 10); + + Ok(()) + } + + #[tokio::test] + async fn test_concurrent_reader_writer() -> Result<()> { + let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + + // Spawn writer task + let writer_handle = tokio::spawn(async move { + for i in 0..10 { + let batch = create_test_batch(i * 10, 10); + writer.push_batch(&batch).unwrap(); + // Small delay to simulate real concurrent work + tokio::time::sleep(std::time::Duration::from_millis(5)).await; + } + }); + + // Reader task (runs concurrently) + let reader_handle = tokio::spawn(async move { + let mut count = 0; + for i in 0..10 { + let result = reader.next().await.unwrap().unwrap(); + assert_eq!(result.num_rows(), 10); + + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), i * 10); + count += 1; + } + count + }); + + // Wait for both to complete + writer_handle.await.unwrap(); + let batches_read = reader_handle.await.unwrap(); + assert_eq!(batches_read, 10); + + Ok(()) + } + + #[tokio::test] + async fn test_reader_catches_up_to_writer() -> Result<()> { + let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + + #[derive(Clone, Copy, Debug, PartialEq, Eq)] + enum ReadWriteEvent { + ReadStart, + Read(usize), + Write(usize), + } + + let events = Arc::new(Mutex::new(vec![])); + // Start reader first (will pend) + let reader_events = Arc::clone(&events); + let reader_handle = tokio::spawn(async move { + reader_events.lock().push(ReadWriteEvent::ReadStart); + let result = reader.next().await.unwrap().unwrap(); + reader_events + .lock() + .push(ReadWriteEvent::Read(result.num_rows())); + let result = reader.next().await.unwrap().unwrap(); + reader_events + .lock() + .push(ReadWriteEvent::Read(result.num_rows())); + }); + + // Give reader time to start pending + tokio::time::sleep(std::time::Duration::from_millis(5)).await; + + // Now write a batch (should wake the reader) + let batch = create_test_batch(0, 5); + events.lock().push(ReadWriteEvent::Write(batch.num_rows())); + writer.push_batch(&batch)?; + + // Wait for the reader to process + let processed = async { + loop { + if events.lock().len() >= 3 { + break; + } + tokio::time::sleep(std::time::Duration::from_micros(500)).await; + } + }; + tokio::time::timeout(std::time::Duration::from_secs(1), processed) + .await + .unwrap(); + + // Write another batch + let batch = create_test_batch(5, 10); + events.lock().push(ReadWriteEvent::Write(batch.num_rows())); + writer.push_batch(&batch)?; + + // Reader should complete + reader_handle.await.unwrap(); + let events = events.lock().clone(); + assert_eq!( + events, + vec![ + ReadWriteEvent::ReadStart, + ReadWriteEvent::Write(5), + ReadWriteEvent::Read(5), + ReadWriteEvent::Write(10), + ReadWriteEvent::Read(10) + ] + ); + + Ok(()) + } + + #[tokio::test] + async fn test_reader_starts_after_writer_finishes() -> Result<()> { + let (mut writer, reader) = create_spill_channel(128); + + // Writer writes all data + for i in 0..5 { + let batch = create_test_batch(i * 10, 10); + writer.push_batch(&batch)?; + } + + drop(writer); + + // Now start reader + let mut reader = reader; + let mut count = 0; + for i in 0..5 { + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 10); + + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), i * 10); + count += 1; + } + + assert_eq!(count, 5, "Should read all batches after writer finishes"); + + Ok(()) + } + + #[tokio::test] + async fn test_writer_drop_finalizes_file() -> Result<()> { + let env = Arc::new(RuntimeEnv::default()); + let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let schema = create_test_schema(); + let spill_manager = + Arc::new(SpillManager::new(Arc::clone(&env), metrics.clone(), schema)); + + let (mut writer, mut reader) = channel(1024 * 1024, spill_manager); + + // Write some batches + for i in 0..5 { + let batch = create_test_batch(i * 10, 10); + writer.push_batch(&batch)?; + } + + // Check metrics before drop - spilled_bytes should be 0 since file isn't finalized yet + let spilled_bytes_before = metrics.spilled_bytes.value(); + assert_eq!( + spilled_bytes_before, 0, + "Spilled bytes should be 0 before writer is dropped" + ); + + // Explicitly drop the writer - this should finalize the current file + drop(writer); + + // Check metrics after drop - spilled_bytes should be > 0 now + let spilled_bytes_after = metrics.spilled_bytes.value(); + assert!( + spilled_bytes_after > 0, + "Spilled bytes should be > 0 after writer is dropped (got {})", + spilled_bytes_after + ); + + // Verify reader can still read all batches + let mut count = 0; + for i in 0..5 { + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 10); + + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), i * 10); + count += 1; + } + + assert_eq!(count, 5, "Should read all batches after writer is dropped"); + + Ok(()) + } +} From df58e8b93d5ac8f2a0b269cc4481d2b9e6405fb6 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 00:36:12 -0600 Subject: [PATCH 32/44] re-incorporate gated channels --- .../physical-plan/src/repartition/mod.rs | 239 +++--------------- 1 file changed, 34 insertions(+), 205 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index a335fe741bd88..a0480b565244c 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -62,9 +62,11 @@ use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; -use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; mod distributor_channels; +use distributor_channels::{ + channels, partition_aware_channels, DistributionReceiver, DistributionSender, +}; /// A batch in the repartition queue - either in memory or spilled to disk #[derive(Debug)] @@ -77,87 +79,12 @@ enum RepartitionBatch { } type MaybeBatch = Option>; -type InputPartitionsToCurrentPartitionSender = - Vec>; -type InputPartitionsToCurrentPartitionReceiver = - Vec>; - -/// Wrapper around tokio UnboundedSender to match DistributionSender interface -struct UnboundedDistributionSender { - inner: UnboundedSender, -} - -// Implement Clone manually without requiring T: Clone -impl Clone for UnboundedDistributionSender { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - } - } -} - -impl UnboundedDistributionSender { - fn send(&self, value: T) -> UnboundedSendFuture { - UnboundedSendFuture { - is_err: self.inner.send(value).is_err(), - } - } -} - -struct UnboundedSendFuture { - is_err: bool, -} - -impl std::future::Future for UnboundedSendFuture { - type Output = Result<(), ()>; - - fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { - // Unbounded send is immediate, so always ready - if self.is_err { - Poll::Ready(Err(())) - } else { - Poll::Ready(Ok(())) - } - } -} - -/// Wrapper around tokio UnboundedReceiver to match DistributionReceiver interface -struct UnboundedDistributionReceiver { - inner: UnboundedReceiver, -} - -impl UnboundedDistributionReceiver { - fn recv( - &mut self, - ) -> Pin> + Send + '_>> { - Box::pin(self.inner.recv()) - } -} - -/// Create unbounded channels without backpressure -fn unbounded_channels( - n: usize, -) -> ( - Vec>, - Vec>, -) { - let channels: Vec<_> = (0..n) - .map(|_| { - let (tx, rx) = unbounded_channel(); - ( - UnboundedDistributionSender { inner: tx }, - UnboundedDistributionReceiver { inner: rx }, - ) - }) - .collect(); - - let (senders, receivers): (Vec<_>, Vec<_>) = channels.into_iter().unzip(); - (senders, receivers) -} +type InputPartitionsToCurrentPartitionSender = Vec>; +type InputPartitionsToCurrentPartitionReceiver = Vec>; /// Output channel with its associated memory reservation and spill writer struct OutputChannel { - sender: UnboundedDistributionSender, + sender: DistributionSender, reservation: SharedMemoryReservation, spill_writer: SpillPoolWriter, } @@ -294,22 +221,17 @@ impl RepartitionExecState { let spill_manager = Arc::new(spill_manager); let (txs, rxs) = if preserve_order { - // Create unbounded channels for each input->output pair - let mut txs_all = vec![]; - let mut rxs_all = vec![]; - for _ in 0..num_input_partitions { - let (txs_row, rxs_row) = unbounded_channels(num_output_partitions); - txs_all.push(txs_row); - rxs_all.push(rxs_row); - } + // Create partition-aware channels with one channel per (input, output) pair + // This provides backpressure while maintaining proper ordering + let (txs_all, rxs_all) = + partition_aware_channels(num_input_partitions, num_output_partitions); // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition let txs = transpose(txs_all); let rxs = transpose(rxs_all); (txs, rxs) } else { - // create one channel per *output* partition - // Using unbounded channels to avoid backpressure/deadlock with spilling - let (txs, rxs) = unbounded_channels(num_output_partitions); + // Create one channel per *output* partition with backpressure + let (txs, rxs) = channels(num_output_partitions); // Clone sender for each input partitions let txs = txs .into_iter() @@ -1191,55 +1113,28 @@ impl RepartitionExec { let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it if let Some(channel) = output_channels.get_mut(&partition) { - let (batch_to_send, is_memory_batch) = match channel - .reservation - .lock() - .try_grow(size) - { - Ok(_) => { - // Memory available - send in-memory batch - eprintln!( - "[SEND] Partition {}: Sending Memory batch with {} rows", - partition, - batch.num_rows() - ); - (RepartitionBatch::Memory(batch), true) - } - Err(_) => { - // We're memory limited - spill to SpillPool - // SpillPool handles file handle reuse and rotation - eprintln!( - "[SEND] Partition {}: Spilling batch with {} rows", - partition, - batch.num_rows() - ); - channel.spill_writer.push_batch(&batch)?; - // Send marker indicating batch was spilled - (RepartitionBatch::Spilled, false) - } - }; + let (batch_to_send, is_memory_batch) = + match channel.reservation.lock().try_grow(size) { + Ok(_) => { + // Memory available - send in-memory batch + (RepartitionBatch::Memory(batch), true) + } + Err(_) => { + // We're memory limited - spill to SpillPool + // SpillPool handles file handle reuse and rotation + channel.spill_writer.push_batch(&batch)?; + // Send marker indicating batch was spilled + (RepartitionBatch::Spilled, false) + } + }; - eprintln!( - "[SEND] Partition {}: About to send {:?} to channel", - partition, - if is_memory_batch { "Memory" } else { "Spilled" } - ); if channel.sender.send(Some(Ok(batch_to_send))).await.is_err() { - eprintln!( - "[SEND] Partition {}: Send failed, receiver hung up", - partition - ); // If the other end has hung up, it was an early shutdown (e.g. LIMIT) // Only shrink memory if it was a memory batch if is_memory_batch { channel.reservation.lock().shrink(size); } output_channels.remove(&partition); - } else { - eprintln!( - "[SEND] Partition {}: Send completed successfully", - partition - ); } } timer.done(); @@ -1281,7 +1176,7 @@ impl RepartitionExec { /// channels. async fn wait_for_task( input_task: SpawnedTask>, - txs: HashMap>, + txs: HashMap>, ) { // wait for completion, and propagate error // note we ignore errors on send (.ok) as that means the receiver has already shutdown. @@ -1313,12 +1208,7 @@ impl RepartitionExec { // Input task completed successfully Ok(Ok(())) => { // notify each output partition that this input partition has no more data - eprintln!( - "[SEND] Input task completed, sending None to {} output partitions", - txs.len() - ); - for (partition, tx) in txs { - eprintln!("[SEND] Sending None to partition {}", partition); + for (_partition, tx) in txs { tx.send(None).await.ok(); } } @@ -1341,7 +1231,7 @@ struct RepartitionStream { schema: SchemaRef, /// channel containing the repartitioned batches - input: UnboundedDistributionReceiver, + input: DistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -1362,7 +1252,7 @@ struct RepartitionStream { impl RepartitionStream { fn new( schema: SchemaRef, - input: UnboundedDistributionReceiver, + input: DistributionReceiver, drop_helper: Arc>>, reservation: SharedMemoryReservation, spill_stream: SendableRecordBatchStream, @@ -1390,31 +1280,12 @@ impl Stream for RepartitionStream { use futures::StreamExt; loop { - eprintln!( - "[RepartitionStream] State: {:?}, remaining_partitions: {}", - self.state, self.remaining_partitions - ); match self.state { StreamState::ReadingMemory => { // Poll the memory channel for next message let value = match self.input.recv().poll_unpin(cx) { - Poll::Ready(v) => { - eprintln!( - "[RepartitionStream] Memory channel poll returned: {:?}", - match &v { - Some(Some(Ok(RepartitionBatch::Memory(_)))) => - "Some(Some(Ok(Memory)))", - Some(Some(Ok(RepartitionBatch::Spilled))) => - "Some(Some(Ok(Spilled)))", - Some(Some(Err(_))) => "Some(Some(Err))", - Some(None) => "Some(None)", - None => "None", - } - ); - v - } + Poll::Ready(v) => v, Poll::Pending => { - eprintln!("[RepartitionStream] Memory channel poll returned: Pending"); // Nothing from channel, wait return Poll::Pending; } @@ -1423,7 +1294,6 @@ impl Stream for RepartitionStream { match value { Some(Some(v)) => match v { Ok(RepartitionBatch::Memory(batch)) => { - eprintln!("[RepartitionStream] Returning memory batch with {} rows", batch.num_rows()); // Release memory and return batch self.reservation .lock() @@ -1431,7 +1301,6 @@ impl Stream for RepartitionStream { return Poll::Ready(Some(Ok(batch))); } Ok(RepartitionBatch::Spilled) => { - eprintln!("[RepartitionStream] Received Spilled marker, transitioning to ReadingSpilled"); // Batch was spilled, transition to reading from spill stream // We must block on spill stream until we get the batch // to preserve ordering @@ -1439,17 +1308,13 @@ impl Stream for RepartitionStream { continue; } Err(e) => { - eprintln!("[RepartitionStream] Returning error"); return Poll::Ready(Some(Err(e))); } }, Some(None) => { // One input partition finished self.remaining_partitions -= 1; - eprintln!("[RepartitionStream] Input partition finished, remaining: {}", - self.remaining_partitions); if self.remaining_partitions == 0 { - eprintln!("[RepartitionStream] All input partitions finished, ending stream"); // All input partitions finished return Poll::Ready(None); } @@ -1457,7 +1322,6 @@ impl Stream for RepartitionStream { continue; } None => { - eprintln!("[RepartitionStream] Channel closed unexpectedly"); // Channel closed unexpectedly return Poll::Ready(None); } @@ -1467,24 +1331,17 @@ impl Stream for RepartitionStream { // Poll spill stream for the spilled batch match self.spill_stream.poll_next_unpin(cx) { Poll::Ready(Some(Ok(batch))) => { - eprintln!("[RepartitionStream] Got spilled batch with {} rows, transitioning to ReadingMemory", - batch.num_rows()); self.state = StreamState::ReadingMemory; return Poll::Ready(Some(Ok(batch))); } Poll::Ready(Some(Err(e))) => { - eprintln!("[RepartitionStream] Spill stream returned error"); return Poll::Ready(Some(Err(e))); } Poll::Ready(None) => { - eprintln!("[RepartitionStream] Spill stream ended, transitioning to ReadingMemory"); // Spill stream ended keep draining the memory channel self.state = StreamState::ReadingMemory; } Poll::Pending => { - eprintln!( - "[RepartitionStream] Spill stream poll returned: Pending" - ); // Spilled batch not ready yet, must wait // This preserves ordering by blocking until spill data arrives return Poll::Pending; @@ -1510,7 +1367,7 @@ struct PerPartitionStream { schema: SchemaRef, /// channel containing the repartitioned batches - receiver: UnboundedDistributionReceiver, + receiver: DistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -1528,7 +1385,7 @@ struct PerPartitionStream { impl PerPartitionStream { fn new( schema: SchemaRef, - receiver: UnboundedDistributionReceiver, + receiver: DistributionReceiver, drop_helper: Arc>>, reservation: SharedMemoryReservation, spill_stream: SendableRecordBatchStream, @@ -1554,28 +1411,12 @@ impl Stream for PerPartitionStream { use futures::StreamExt; loop { - eprintln!("[PerPartitionStream] State: {:?}", self.state); match self.state { StreamState::ReadingMemory => { // Poll the memory channel for next message let value = match self.receiver.recv().poll_unpin(cx) { - Poll::Ready(v) => { - eprintln!( - "[PerPartitionStream] Memory channel poll returned: {:?}", - match &v { - Some(Some(Ok(RepartitionBatch::Memory(_)))) => - "Some(Some(Ok(Memory)))", - Some(Some(Ok(RepartitionBatch::Spilled))) => - "Some(Some(Ok(Spilled)))", - Some(Some(Err(_))) => "Some(Some(Err))", - Some(None) => "Some(None)", - None => "None", - } - ); - v - } + Poll::Ready(v) => v, Poll::Pending => { - eprintln!("[PerPartitionStream] Memory channel poll returned: Pending"); // Nothing from channel, wait return Poll::Pending; } @@ -1584,7 +1425,6 @@ impl Stream for PerPartitionStream { match value { Some(Some(v)) => match v { Ok(RepartitionBatch::Memory(batch)) => { - eprintln!("[PerPartitionStream] Returning memory batch with {} rows", batch.num_rows()); // Release memory and return batch self.reservation .lock() @@ -1592,7 +1432,6 @@ impl Stream for PerPartitionStream { return Poll::Ready(Some(Ok(batch))); } Ok(RepartitionBatch::Spilled) => { - eprintln!("[PerPartitionStream] Received Spilled marker, transitioning to ReadingSpilled"); // Batch was spilled, transition to reading from spill stream // We must block on spill stream until we get the batch // to preserve ordering @@ -1600,18 +1439,13 @@ impl Stream for PerPartitionStream { continue; } Err(e) => { - eprintln!("[PerPartitionStream] Returning error"); return Poll::Ready(Some(Err(e))); } }, Some(None) => { - eprintln!( - "[PerPartitionStream] Received None, ending stream" - ); return Poll::Ready(None); } None => { - eprintln!("[PerPartitionStream] Channel closed unexpectedly"); // Channel closed unexpectedly return Poll::Ready(None); } @@ -1621,22 +1455,17 @@ impl Stream for PerPartitionStream { // Poll spill stream for the spilled batch match self.spill_stream.poll_next_unpin(cx) { Poll::Ready(Some(Ok(batch))) => { - eprintln!("[PerPartitionStream] Got spilled batch with {} rows, transitioning to ReadingMemory", - batch.num_rows()); self.state = StreamState::ReadingMemory; return Poll::Ready(Some(Ok(batch))); } Poll::Ready(Some(Err(e))) => { - eprintln!("[PerPartitionStream] Spill stream returned error"); return Poll::Ready(Some(Err(e))); } Poll::Ready(None) => { - eprintln!("[PerPartitionStream] Spill stream ended, transitioning to ReadingMemory"); // Spill stream ended keep draining the memory channel self.state = StreamState::ReadingMemory; } Poll::Pending => { - eprintln!("[PerPartitionStream] Spill stream poll returned: Pending"); // Spilled batch not ready yet, must wait // This preserves ordering by blocking until spill data arrives return Poll::Pending; From 66790ece4e88ecfcd5363b57d0e47af051d66c07 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 08:18:17 -0600 Subject: [PATCH 33/44] add lots of docs --- .../physical-plan/src/repartition/mod.rs | 164 ++++++++++++++- datafusion/physical-plan/src/spill/mod.rs | 4 + .../physical-plan/src/spill/spill_pool.rs | 189 +++++++++++++++--- 3 files changed, 322 insertions(+), 35 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index a0480b565244c..f9f8c7023c5ae 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -68,13 +68,62 @@ use distributor_channels::{ channels, partition_aware_channels, DistributionReceiver, DistributionSender, }; -/// A batch in the repartition queue - either in memory or spilled to disk +/// A batch in the repartition queue - either in memory or spilled to disk. +/// +/// This enum represents the two states a batch can be in during repartitioning. +/// The decision to spill is made based on memory availability when sending a batch +/// to an output partition. +/// +/// # Batch Flow with Spilling +/// +/// ```text +/// Input Stream ──▶ Partition Logic ──▶ try_grow() +/// │ +/// ┌───────────────┴────────────────┐ +/// │ │ +/// ▼ ▼ +/// try_grow() succeeds try_grow() fails +/// (Memory Available) (Memory Pressure) +/// │ │ +/// ▼ ▼ +/// RepartitionBatch::Memory spill_writer.push_batch() +/// (batch held in memory) (batch written to disk) +/// │ │ +/// │ ▼ +/// │ RepartitionBatch::Spilled +/// │ (marker - no batch data) +/// │ │ +/// └────────┬───────────────────────┘ +/// │ +/// ▼ +/// Send to channel +/// │ +/// ▼ +/// Output Stream (poll) +/// │ +/// ┌──────────────┴─────────────┐ +/// │ │ +/// ▼ ▼ +/// RepartitionBatch::Memory RepartitionBatch::Spilled +/// Return batch immediately Poll spill_stream (blocks) +/// │ │ +/// └────────┬───────────────────┘ +/// │ +/// ▼ +/// Return batch +/// (FIFO order preserved) +/// ``` +/// +/// See [`RepartitionExec`] for overall architecture and [`StreamState`] for +/// the state machine that handles reading these batches. #[derive(Debug)] enum RepartitionBatch { /// Batch held in memory (counts against memory reservation) Memory(RecordBatch), - /// Marker indicating a batch was spilled to the partition's SpillPool - /// The actual batch can be retrieved by reading from the SpillPoolStream + /// Marker indicating a batch was spilled to the partition's SpillPool. + /// The actual batch can be retrieved by reading from the SpillPoolStream. + /// This variant contains no data itself - it's just a signal to the reader + /// to fetch the next batch from the spill stream. Spilled, } @@ -89,7 +138,27 @@ struct OutputChannel { spill_writer: SpillPoolWriter, } -/// Channels and resources for a single output partition +/// Channels and resources for a single output partition. +/// +/// Each output partition has channels to receive data from all input partitions. +/// To handle memory pressure, each (input, output) pair gets its own +/// [`SpillPool`](crate::spill::spill_pool) channel via [`spill_pool::channel`]. +/// +/// # Structure +/// +/// For an output partition receiving from N input partitions: +/// - `tx`: N senders (one per input) for sending batches to this output +/// - `rx`: N receivers (one per input) for receiving batches at this output +/// - `spill_writers`: N spill writers (one per input) for writing spilled data +/// - `spill_readers`: N spill readers (one per input) for reading spilled data +/// +/// This 1:1 mapping between input partitions and spill channels ensures that +/// batches from each input are processed in FIFO order, even when some batches +/// are spilled to disk and others remain in memory. +/// +/// See [`RepartitionExec`] for the overall N×M architecture. +/// +/// [`spill_pool::channel`]: crate::spill::spill_pool::channel struct PartitionChannels { /// Senders for each input partition to send data to this output partition tx: InputPartitionsToCurrentPartitionSender, @@ -97,11 +166,11 @@ struct PartitionChannels { rx: InputPartitionsToCurrentPartitionReceiver, /// Memory reservation for this output partition reservation: SharedMemoryReservation, - /// Spill writers for writing spilled data - one per input partition (FIFO semantics) - /// Wrapped in Option so they can be moved out when creating OutputChannels + /// Spill writers for writing spilled data - one per input partition (FIFO semantics). + /// Wrapped in Option so they can be moved out when creating OutputChannels. spill_writers: Vec>, - /// Spill readers for reading spilled data - one per input partition (FIFO semantics) - /// Each (input, output) pair gets its own reader to maintain proper ordering + /// Spill readers for reading spilled data - one per input partition (FIFO semantics). + /// Each (input, output) pair gets its own reader to maintain proper ordering. spill_readers: Vec, } @@ -539,6 +608,38 @@ impl BatchPartitioner { /// arbitrary interleaving (and thus unordered) unless /// [`Self::with_preserve_order`] specifies otherwise. /// +/// # Spilling Architecture +/// +/// RepartitionExec uses [`SpillPool`](crate::spill::spill_pool) channels to handle +/// memory pressure during repartitioning. Each (input partition, output partition) +/// pair gets its own SpillPool channel for FIFO ordering. +/// +/// ```text +/// Input Partitions (N) Output Partitions (M) +/// ──────────────────── ───────────────────── +/// +/// Input 0 ──┐ ┌──▶ Output 0 +/// │ ┌──────────────┐ │ +/// ├─▶│ SpillPool │────┤ +/// │ │ [In0→Out0] │ │ +/// Input 1 ──┤ └──────────────┘ ├──▶ Output 1 +/// │ │ +/// │ ┌──────────────┐ │ +/// ├─▶│ SpillPool │────┤ +/// │ │ [In1→Out0] │ │ +/// Input 2 ──┤ └──────────────┘ ├──▶ Output 2 +/// │ │ +/// │ ... (N×M SpillPools total) +/// │ │ +/// │ ┌──────────────┐ │ +/// └─▶│ SpillPool │────┘ +/// │ [InN→OutM] │ +/// └──────────────┘ +/// +/// Each SpillPool maintains FIFO order for its (input, output) pair. +/// See `RepartitionBatch` for details on the memory/spill decision logic. +/// ``` +/// /// # Footnote /// /// The "Exchange Operator" was first described in the 1989 paper @@ -1216,13 +1317,54 @@ impl RepartitionExec { } } -/// State for tracking whether we're reading from memory channel or spill stream +/// State for tracking whether we're reading from memory channel or spill stream. +/// +/// This state machine ensures proper ordering when batches are mixed between memory +/// and spilled storage. When a [`RepartitionBatch::Spilled`] marker is received, +/// the stream must block on the spill stream until the corresponding batch arrives. +/// +/// # State Machine +/// +/// ```text +/// ┌─────────────────┐ +/// ┌───▶│ ReadingMemory │◀───┐ +/// │ └────────┬────────┘ │ +/// │ │ │ +/// │ Poll channel │ +/// │ │ │ +/// │ ┌──────────┼─────────────┐ +/// │ │ │ │ +/// │ ▼ ▼ │ +/// │ Memory Spilled │ +/// Got batch │ batch marker │ +/// from spill │ │ │ │ +/// │ │ ▼ │ +/// │ │ ┌──────────────────┐ │ +/// │ │ │ ReadingSpilled │ │ +/// │ │ └────────┬─────────┘ │ +/// │ │ │ │ +/// │ │ Poll spill_stream │ +/// │ │ │ │ +/// │ │ ▼ │ +/// │ │ Get batch │ +/// │ │ │ │ +/// └──┴───────────┴────────────┘ +/// │ +/// ▼ +/// Return batch +/// (Order preserved within +/// (input, output) pair) +/// ``` +/// +/// The transition to `ReadingSpilled` blocks further channel polling to maintain +/// FIFO ordering - we cannot read the next item from the channel until the spill +/// stream provides the current batch. #[derive(Debug, Clone, Copy, PartialEq, Eq)] enum StreamState { /// Reading from the memory channel (normal operation) ReadingMemory, - /// Waiting for a spilled batch from the spill stream - /// Must not poll channel until spilled batch is received + /// Waiting for a spilled batch from the spill stream. + /// Must not poll channel until spilled batch is received to preserve ordering. ReadingSpilled, } diff --git a/datafusion/physical-plan/src/spill/mod.rs b/datafusion/physical-plan/src/spill/mod.rs index a1a7bbee05c8a..58fd016a63dd7 100644 --- a/datafusion/physical-plan/src/spill/mod.rs +++ b/datafusion/physical-plan/src/spill/mod.rs @@ -21,6 +21,10 @@ pub(crate) mod in_progress_spill_file; pub(crate) mod spill_manager; pub mod spill_pool; +// Re-export SpillManager for doctests only (hidden from public docs) +#[doc(hidden)] +pub use spill_manager::SpillManager; + use std::fs::File; use std::io::BufReader; use std::path::{Path, PathBuf}; diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index d8d2b5f0d4858..1233d2a538705 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -72,6 +72,12 @@ impl SpillPoolShared { } /// Writer for a spill pool. Provides exclusive write access with FIFO semantics. +/// +/// Created by [`channel`]. See that function for architecture diagrams and usage examples. +/// +/// The writer automatically manages file rotation based on the `max_file_size_bytes` +/// configured in [`channel`]. When dropped, it finalizes the current file so readers +/// can access all written data. pub struct SpillPoolWriter { /// Maximum size in bytes before rotating to a new file max_file_size_bytes: usize, @@ -87,6 +93,35 @@ impl SpillPoolWriter { /// If the current file would exceed `max_file_size_bytes` after adding /// this batch, the file is finalized and a new one is started. /// + /// See [`channel`] for overall architecture and examples. + /// + /// # File Rotation Logic + /// + /// ```text + /// push_batch() + /// │ + /// ▼ + /// Current file exists? + /// │ + /// ├─ No ──▶ Create new file ──▶ Add to shared queue + /// │ Wake readers + /// ▼ + /// Write batch to current file + /// │ + /// ▼ + /// estimated_size > max_file_size_bytes? + /// │ + /// ├─ No ──▶ Keep current file for next batch + /// │ + /// ▼ + /// Yes: finish() current file + /// Mark writer_finished = true + /// Wake readers + /// │ + /// ▼ + /// Next push_batch() creates new file + /// ``` + /// /// # Errors /// /// Returns an error if disk I/O fails or disk quota is exceeded. @@ -194,29 +229,139 @@ impl Drop for SpillPoolWriter { /// write access, and the reader can consume batches in FIFO order. The reader /// can start reading immediately while the writer continues to write more data. /// +/// # Architecture +/// +/// ```text +/// ┌─────────────────────────────────────────────────────────────────────────┐ +/// │ SpillPool │ +/// │ │ +/// │ Writer Side Shared State Reader Side │ +/// │ ─────────── ──────────── ─────────── │ +/// │ │ +/// │ SpillPoolWriter ┌────────────────────┐ SpillPoolReader │ +/// │ │ │ VecDeque │ │ │ +/// │ │ │ ┌────┐┌────┐ │ │ │ +/// │ push_batch() │ │ F1 ││ F2 │ ... │ next().await │ +/// │ │ │ └────┘└────┘ │ │ │ +/// │ ▼ │ (FIFO order) │ ▼ │ +/// │ ┌─────────┐ │ │ ┌──────────┐ │ +/// │ │Current │───────▶│ Coordination: │◀───│ Current │ │ +/// │ │Write │ │ - Wakers │ │ Read │ │ +/// │ │File │ │ - Batch counts │ │ File │ │ +/// │ └─────────┘ │ - Writer status │ └──────────┘ │ +/// │ │ └────────────────────┘ │ │ +/// │ │ │ │ +/// │ Size > limit? Read all batches? │ +/// │ │ │ │ +/// │ ▼ ▼ │ +/// │ Rotate to new file Pop from queue │ +/// └─────────────────────────────────────────────────────────────────────────┘ +/// +/// Writer produces → Shared FIFO queue → Reader consumes +/// ``` +/// +/// # File State Machine +/// +/// Each file in the pool coordinates between writer and reader: +/// +/// ```text +/// Writer View Reader View +/// ─────────── ─────────── +/// +/// Created writer: Some(..) batches_read: 0 +/// batches_written: 0 (waiting for data) +/// │ +/// ▼ +/// Writing append_batch() Can read if: +/// batches_written++ batches_read < batches_written +/// wake readers +/// │ │ +/// │ ▼ +/// ┌──────┴──────┐ poll_next() → batch +/// │ │ batches_read++ +/// ▼ ▼ +/// Size > limit? More data? +/// │ │ +/// │ └─▶ Yes ──▶ Continue writing +/// ▼ +/// finish() Reader catches up: +/// writer_finished = true batches_read == batches_written +/// wake readers │ +/// │ ▼ +/// └─────────────────────▶ Returns Poll::Ready(None) +/// File complete, pop from queue +/// ``` +/// /// # Arguments /// -/// * `max_file_size_bytes` - Maximum size per file before rotation -/// * `spill_manager` - Manager for file creation and metrics +/// * `max_file_size_bytes` - Maximum size per file before rotation. When a file +/// exceeds this size, the writer automatically rotates to a new file. +/// * `spill_manager` - Manager for file creation and metrics tracking /// /// # Returns /// -/// A tuple of `(SpillPoolWriter, SpillPoolReader)` that share the same underlying pool +/// A tuple of `(SpillPoolWriter, SendableRecordBatchStream)` that share the same +/// underlying pool. The reader is returned as a stream for immediate use with +/// async stream combinators. /// /// # Example /// -/// ```ignore +/// ``` +/// use std::sync::Arc; +/// use arrow::array::{ArrayRef, Int32Array}; +/// use arrow::datatypes::{DataType, Field, Schema}; +/// use arrow::record_batch::RecordBatch; +/// use datafusion_execution::runtime_env::RuntimeEnv; +/// use futures::StreamExt; +/// +/// # use datafusion_physical_plan::spill::spill_pool; +/// # use datafusion_physical_plan::spill::SpillManager; // Re-exported for doctests +/// # use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; +/// # +/// # #[tokio::main] +/// # async fn main() -> datafusion_common::Result<()> { +/// # // Setup for the example (typically comes from TaskContext in production) +/// # let env = Arc::new(RuntimeEnv::default()); +/// # let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); +/// # let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); +/// # let spill_manager = Arc::new(SpillManager::new(env, metrics, schema.clone())); +/// # +/// // Create channel with 1MB file size limit /// let (mut writer, mut reader) = spill_pool::channel(1024 * 1024, spill_manager); /// -/// // Writer writes batches -/// writer.push_batch(&batch)?; +/// // Spawn writer task to produce batches +/// let write_handle = tokio::spawn(async move { +/// for i in 0..5 { +/// let array: ArrayRef = Arc::new(Int32Array::from(vec![i; 100])); +/// let batch = RecordBatch::try_new(schema.clone(), vec![array]).unwrap(); +/// writer.push_batch(&batch).unwrap(); +/// } +/// // Writer dropped here, finalizing current file +/// }); /// -/// // Reader consumes batches (can happen concurrently) +/// // Reader consumes batches in FIFO order (can run concurrently with writer) +/// let mut batches_read = 0; /// while let Some(result) = reader.next().await { /// let batch = result?; +/// batches_read += 1; /// // Process batch... +/// if batches_read == 5 { +/// break; // Got all expected batches +/// } /// } +/// +/// write_handle.await.unwrap(); +/// assert_eq!(batches_read, 5); +/// # Ok(()) +/// # } /// ``` +/// +/// # Use Cases +/// +/// - **Backpressure handling**: Writer can continue producing while reader is slow +/// - **Memory management**: Files automatically rotate based on size limits +/// - **Concurrent I/O**: Reader and writer operate independently with async coordination +/// - **FIFO semantics**: Batches are consumed in the exact order they were written pub fn channel( max_file_size_bytes: usize, spill_manager: Arc, @@ -279,20 +424,6 @@ struct SpillFileReader { batches_read: usize, } -// impl SpillFile { -// fn poll_next(&mut self) -> Option> { -// match self { -// SpillFile::InProgress(active) => { -// // If there are no unread batches, we cannot read yet -// if active.unread_batches == 0 { -// return None; -// } -// }, -// SpillFile::Completed(stream) => stream.next().await, -// } -// } -// } - struct SpillFile { /// Shared coordination state (contains writer and batch counts) shared: Arc>, @@ -382,11 +513,21 @@ impl Stream for SpillFile { /// A stream that reads from a SpillPool in FIFO order. /// +/// Created by [`channel`]. See that function for architecture diagrams and usage examples. +/// /// The stream automatically handles file rotation and reads from completed files. -/// When no completed files are available, it returns `Poll::Pending` and waits -/// for the writer to complete more files. +/// When no data is available, it returns `Poll::Pending` and registers a waker to +/// be notified when the writer produces more data. +/// +/// # Infinite Stream Semantics +/// +/// This stream never returns `None` (`Poll::Ready(None)`) on its own - it will keep +/// waiting for the writer to produce more data. The stream ends only when: +/// - The reader is dropped +/// - The writer is dropped AND all queued data has been consumed /// -/// The stream will never end, it is an infinite stream. +/// This makes it suitable for continuous streaming scenarios where the writer may +/// produce data intermittently. pub struct SpillPoolReader { /// Shared reference to the spill pool shared: Arc>, From 3f40fdb4b01036d2d18719ac3423492d127072b7 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 08:30:00 -0600 Subject: [PATCH 34/44] add metrics for file and byte counts to tests --- .../physical-plan/src/spill/spill_pool.rs | 171 +++++++++++++++++- 1 file changed, 163 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 1233d2a538705..ff8eb15adf821 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -142,7 +142,7 @@ impl SpillPoolWriter { let writer = spill_manager.create_in_progress_file("SpillPool")?; // Clone the file so readers can access it immediately - let file = writer.file().unwrap().clone(); + let file = writer.file().expect("InProgressSpillFile should always have a file when it is first created").clone(); let file_shared = Arc::new(Mutex::new(ActiveSpillFileShared { writer: Some(writer), @@ -671,6 +671,18 @@ mod tests { channel(max_file_size, spill_manager) } + fn create_spill_channel_with_metrics( + max_file_size: usize, + ) -> (SpillPoolWriter, SendableRecordBatchStream, SpillMetrics) { + let env = Arc::new(RuntimeEnv::default()); + let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let schema = create_test_schema(); + let spill_manager = Arc::new(SpillManager::new(env, metrics.clone(), schema)); + + let (writer, reader) = channel(max_file_size, spill_manager); + (writer, reader, metrics) + } + #[tokio::test] async fn test_basic_write_and_read() -> Result<()> { let (mut writer, mut reader) = create_spill_channel(1024 * 1024); @@ -783,12 +795,30 @@ mod tests { let batch1 = create_test_batch(0, 10); let batch_size = batch1.get_array_memory_size() + 1; - let (mut writer, mut reader) = create_spill_channel(batch_size); + let (mut writer, mut reader, metrics) = + create_spill_channel_with_metrics(batch_size); // Write first batch (should fit in first file) writer.push_batch(&batch1)?; - // Write second batch (should trigger rotation to second file) + // Check metrics after first batch - file created but not finalized yet + assert_eq!( + metrics.spill_file_count.value(), + 1, + "Should have created 1 file after first batch" + ); + assert_eq!( + metrics.spilled_bytes.value(), + 0, + "Spilled bytes should be 0 before file finalization" + ); + assert_eq!( + metrics.spilled_rows.value(), + 10, + "Should have spilled 10 rows from first batch" + ); + + // Write second batch (should trigger rotation - finalize first file) let batch2 = create_test_batch(10, 10); assert!( batch2.get_array_memory_size() <= batch_size, @@ -802,13 +832,50 @@ mod tests { ); writer.push_batch(&batch2)?; - // Read both batches + // Check metrics after rotation - first file finalized, but second file not created yet + // (new file created lazily on next push_batch call) + assert_eq!( + metrics.spill_file_count.value(), + 1, + "Should still have 1 file (second file not created until next write)" + ); + assert!( + metrics.spilled_bytes.value() > 0, + "Spilled bytes should be > 0 after first file finalized (got {})", + metrics.spilled_bytes.value() + ); + assert_eq!( + metrics.spilled_rows.value(), + 20, + "Should have spilled 20 total rows (10 + 10)" + ); + + // Write a third batch to confirm rotation occurred (creates second file) + let batch3 = create_test_batch(20, 5); + writer.push_batch(&batch3)?; + + // Now check that second file was created + assert_eq!( + metrics.spill_file_count.value(), + 2, + "Should have created 2 files after writing to new file" + ); + assert_eq!( + metrics.spilled_rows.value(), + 25, + "Should have spilled 25 total rows (10 + 10 + 5)" + ); + + // Read all three batches let result1 = reader.next().await.unwrap()?; assert_eq!(result1.num_rows(), 10); let result2 = reader.next().await.unwrap()?; assert_eq!(result2.num_rows(), 10); + let result3 = reader.next().await.unwrap()?; + assert_eq!(result3.num_rows(), 5); + Ok(()) } @@ -821,7 +888,8 @@ mod tests { let batch_size = batches[0].get_array_memory_size() * 2 + 1; // Very small max_file_size to force frequent rotations - let (mut writer, mut reader) = create_spill_channel(batch_size); + let (mut writer, mut reader, metrics) = + create_spill_channel_with_metrics(batch_size); // Write many batches to cause multiple rotations for i in 0..10 { @@ -829,6 +897,26 @@ mod tests { writer.push_batch(&batch)?; } + // Check metrics after all writes - should have multiple files due to rotations + // With batch_size = 2 * one_batch + 1, each file fits ~2 batches before rotating + // 10 batches should create multiple files (exact count depends on rotation timing) + let file_count = metrics.spill_file_count.value(); + assert!( + file_count >= 4, + "Should have created at least 4 files with multiple rotations (got {})", + file_count + ); + assert!( + metrics.spilled_bytes.value() > 0, + "Spilled bytes should be > 0 after rotations (got {})", + metrics.spilled_bytes.value() + ); + assert_eq!( + metrics.spilled_rows.value(), + 100, + "Should have spilled 100 total rows (10 batches * 10 rows)" + ); + // Read all batches and verify order for i in 0..10 { let result = reader.next().await.unwrap()?; @@ -853,12 +941,24 @@ mod tests { #[tokio::test] async fn test_single_batch_larger_than_limit() -> Result<()> { // Very small limit - let (mut writer, mut reader) = create_spill_channel(100); + let (mut writer, mut reader, metrics) = create_spill_channel_with_metrics(100); // Write a batch that exceeds the limit let large_batch = create_test_batch(0, 100); writer.push_batch(&large_batch)?; + // Check metrics after large batch - should trigger rotation immediately + assert_eq!( + metrics.spill_file_count.value(), + 1, + "Should have created 1 file for large batch" + ); + assert_eq!( + metrics.spilled_rows.value(), + 100, + "Should have spilled 100 rows from large batch" + ); + // Should still write and read successfully let result = reader.next().await.unwrap()?; assert_eq!(result.num_rows(), 100); @@ -867,6 +967,18 @@ mod tests { let batch2 = create_test_batch(100, 10); writer.push_batch(&batch2)?; + // Check metrics after second batch - should have rotated to a new file + assert_eq!( + metrics.spill_file_count.value(), + 2, + "Should have created 2 files after rotation" + ); + assert_eq!( + metrics.spilled_rows.value(), + 110, + "Should have spilled 110 total rows (100 + 10)" + ); + let result2 = reader.next().await.unwrap()?; assert_eq!(result2.num_rows(), 10); @@ -896,13 +1008,47 @@ mod tests { let batch_size = batch.get_array_memory_size(); // Set max_file_size to exactly the batch size - let (mut writer, mut reader) = create_spill_channel(batch_size); + let (mut writer, mut reader, metrics) = + create_spill_channel_with_metrics(batch_size); - // Write two batches + // Write first batch (exactly at the size limit) writer.push_batch(&batch)?; + + // Check metrics after first batch - should NOT rotate yet (size == limit, not >) + assert_eq!( + metrics.spill_file_count.value(), + 1, + "Should have created 1 file after first batch at exact boundary" + ); + assert_eq!( + metrics.spilled_rows.value(), + 10, + "Should have spilled 10 rows from first batch" + ); + + // Write second batch (exceeds the limit, should trigger rotation) let batch2 = create_test_batch(10, 10); writer.push_batch(&batch2)?; + // Check metrics after second batch - rotation triggered, first file finalized + // Note: second file not created yet (lazy creation on next write) + assert_eq!( + metrics.spill_file_count.value(), + 1, + "Should still have 1 file after rotation (second file created lazily)" + ); + assert_eq!( + metrics.spilled_rows.value(), + 20, + "Should have spilled 20 total rows (10 + 10)" + ); + // Verify first file was finalized by checking spilled_bytes + assert!( + metrics.spilled_bytes.value() > 0, + "Spilled bytes should be > 0 after file finalization (got {})", + metrics.spilled_bytes.value() + ); + // Both should be readable let result1 = reader.next().await.unwrap()?; assert_eq!(result1.num_rows(), 10); @@ -910,6 +1056,15 @@ mod tests { let result2 = reader.next().await.unwrap()?; assert_eq!(result2.num_rows(), 10); + // Spill another batch, now we should see the second file created + let batch3 = create_test_batch(20, 5); + writer.push_batch(&batch3)?; + assert_eq!( + metrics.spill_file_count.value(), + 2, + "Should have created 2 files after writing to new file" + ); + Ok(()) } From f61148af04ec9a1161316ce564d58c285af29627 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 09:30:14 -0600 Subject: [PATCH 35/44] Update datafusion/physical-plan/src/repartition/mod.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- datafusion/physical-plan/src/repartition/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f9f8c7023c5ae..468a80e6e4db5 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1480,7 +1480,7 @@ impl Stream for RepartitionStream { return Poll::Ready(Some(Err(e))); } Poll::Ready(None) => { - // Spill stream ended keep draining the memory channel + // Spill stream ended, keep draining the memory channel self.state = StreamState::ReadingMemory; } Poll::Pending => { From 2dcf3f5ca88705409d8a570ea707613e8c4b1361 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 09:30:42 -0600 Subject: [PATCH 36/44] Update datafusion/physical-plan/src/repartition/mod.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- datafusion/physical-plan/src/repartition/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 468a80e6e4db5..c367d945612d0 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1604,7 +1604,7 @@ impl Stream for PerPartitionStream { return Poll::Ready(Some(Err(e))); } Poll::Ready(None) => { - // Spill stream ended keep draining the memory channel + // Spill stream ended, keep draining the memory channel self.state = StreamState::ReadingMemory; } Poll::Pending => { From e31f8277b96a2432124d04acd1ad0f47b63a66ad Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 5 Nov 2025 09:45:10 -0600 Subject: [PATCH 37/44] clippy --- datafusion/execution/src/disk_manager.rs | 6 +++--- .../physical-plan/src/spill/spill_pool.rs | 18 ++++++++---------- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs index 327c50fa03e4f..e450d77db6ada 100644 --- a/datafusion/execution/src/disk_manager.rs +++ b/datafusion/execution/src/disk_manager.rs @@ -293,7 +293,7 @@ impl DiskManager { let dir_index = rng().random_range(0..local_dirs.len()); Ok(RefCountedTempFile { - _parent_temp_dir: Arc::clone(&local_dirs[dir_index]), + parent_temp_dir: Arc::clone(&local_dirs[dir_index]), tempfile: Arc::new( Builder::new() .tempfile_in(local_dirs[dir_index].as_ref()) @@ -326,7 +326,7 @@ impl DiskManager { pub struct RefCountedTempFile { /// The reference to the directory in which temporary files are created to ensure /// it is not cleaned up prior to the NamedTempFile - _parent_temp_dir: Arc, + parent_temp_dir: Arc, /// The underlying temporary file, wrapped in Arc to allow cloning tempfile: Arc, /// Tracks the current disk usage of this temporary file. See @@ -339,7 +339,7 @@ pub struct RefCountedTempFile { impl Clone for RefCountedTempFile { fn clone(&self) -> Self { Self { - _parent_temp_dir: Arc::clone(&self._parent_temp_dir), + parent_temp_dir: Arc::clone(&self.parent_temp_dir), tempfile: Arc::clone(&self.tempfile), current_file_disk_usage: self.current_file_disk_usage, disk_manager: Arc::clone(&self.disk_manager), diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index ff8eb15adf821..ddefc5917a943 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -645,6 +645,7 @@ mod tests { use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics}; use arrow::array::{ArrayRef, Int32Array}; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common_runtime::SpawnedTask; use datafusion_execution::runtime_env::RuntimeEnv; use futures::StreamExt; @@ -749,7 +750,7 @@ mod tests { .as_any() .downcast_ref::() .unwrap(); - assert_eq!(col.value(0), i * 10, "Batch {} not in FIFO order", i); + assert_eq!(col.value(0), i * 10, "Batch {i} not in FIFO order"); } Ok(()) @@ -903,8 +904,7 @@ mod tests { let file_count = metrics.spill_file_count.value(); assert!( file_count >= 4, - "Should have created at least 4 files with multiple rotations (got {})", - file_count + "Should have created at least 4 files with multiple rotations (got {file_count})" ); assert!( metrics.spilled_bytes.value() > 0, @@ -930,8 +930,7 @@ mod tests { assert_eq!( col.value(0), i * 10, - "Batch {} not in correct order after rotations", - i + "Batch {i} not in correct order after rotations" ); } @@ -1073,7 +1072,7 @@ mod tests { let (mut writer, mut reader) = create_spill_channel(1024 * 1024); // Spawn writer task - let writer_handle = tokio::spawn(async move { + let writer_handle = SpawnedTask::spawn(async move { for i in 0..10 { let batch = create_test_batch(i * 10, 10); writer.push_batch(&batch).unwrap(); @@ -1083,7 +1082,7 @@ mod tests { }); // Reader task (runs concurrently) - let reader_handle = tokio::spawn(async move { + let reader_handle = SpawnedTask::spawn(async move { let mut count = 0; for i in 0..10 { let result = reader.next().await.unwrap().unwrap(); @@ -1122,7 +1121,7 @@ mod tests { let events = Arc::new(Mutex::new(vec![])); // Start reader first (will pend) let reader_events = Arc::clone(&events); - let reader_handle = tokio::spawn(async move { + let reader_handle = SpawnedTask::spawn(async move { reader_events.lock().push(ReadWriteEvent::ReadStart); let result = reader.next().await.unwrap().unwrap(); reader_events @@ -1240,8 +1239,7 @@ mod tests { let spilled_bytes_after = metrics.spilled_bytes.value(); assert!( spilled_bytes_after > 0, - "Spilled bytes should be > 0 after writer is dropped (got {})", - spilled_bytes_after + "Spilled bytes should be > 0 after writer is dropped (got {spilled_bytes_after})" ); // Verify reader can still read all batches From 8cba32ec1674688c2e6c9856da82e5417de77fc1 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 6 Nov 2025 06:23:08 -0600 Subject: [PATCH 38/44] tweak docstrings --- .../physical-plan/src/spill/spill_pool.rs | 41 +++++++++++++++---- 1 file changed, 32 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index ddefc5917a943..164adbe1e0f7d 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -//! Spill pool for managing spill files with FIFO semantics. - use futures::{Stream, StreamExt}; use std::collections::VecDeque; use std::sync::Arc; @@ -223,12 +221,28 @@ impl Drop for SpillPoolWriter { } } -/// Creates a paired writer and reader for a spill pool with channel-like semantics. +/// Creates a paired writer and reader for a spill pool with SPSC (single-producer, single-consumer) +/// semantics. /// /// This is the recommended way to create a spill pool. The writer has exclusive /// write access, and the reader can consume batches in FIFO order. The reader /// can start reading immediately while the writer continues to write more data. /// +/// Internally this coordinates rotating spill files based on size limits, and +/// handles asynchronous notification between the writer and reader using wakers. +/// This ensures that we manage disk usage efficiently while allowing concurrent +/// I/O between the writer and reader. +/// +/// # Data Flow Overview +/// +/// 1. Writer write batch `B0` to F1 +/// 2. Writer write batch `B1` to F1, notices the size limit exceeded, finishes F1. +/// 3. Reader read `B0` from F1 +/// 4. Reader read `B1`, no more batch to read -> wait on the waker +/// 5. Writer write batch `B2` to a new file `F2`, wake up the waiting reader. +/// 6. Reader read `B2` from F2. +/// 7. Repeat until writer is dropped. +/// /// # Architecture /// /// ```text @@ -356,12 +370,21 @@ impl Drop for SpillPoolWriter { /// # } /// ``` /// -/// # Use Cases -/// -/// - **Backpressure handling**: Writer can continue producing while reader is slow -/// - **Memory management**: Files automatically rotate based on size limits -/// - **Concurrent I/O**: Reader and writer operate independently with async coordination -/// - **FIFO semantics**: Batches are consumed in the exact order they were written +/// # Why rotate files? +/// +/// File rotation ensures we don't end up with unreferenced disk usage. +/// If we used a single file for all spilled data, we would end up with +/// unreferenced data at the beginning of the file that has already been read +/// by readers but we can't delete because you can't truncate from the start of a file. +/// +/// Consider the case of a query like `SELECT * FROM large_table WHERE false`. +/// Obviously this query produces no output rows, but if we had a spilling operator +/// in the middle of this query between the scan and the filter it would see the entire +/// `large_table` flow through it and thus would spill all of that data to disk. +/// So we'd end up using up to `size(large_table)` bytes of disk space. +/// If instead we use file rotation, and as long as the readers can keep up with the writer, +/// then we can ensure that once a file is fully read by all readers it can be deleted, +/// thus bounding the maximum disk usage to roughly `max_file_size_bytes`. pub fn channel( max_file_size_bytes: usize, spill_manager: Arc, From f1de9da2237061991a3d10267113f5d4f1c0729c Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 6 Nov 2025 13:41:41 -0600 Subject: [PATCH 39/44] fmt --- datafusion/physical-plan/src/spill/spill_pool.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 164adbe1e0f7d..b5da50acbfec7 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -371,12 +371,12 @@ impl Drop for SpillPoolWriter { /// ``` /// /// # Why rotate files? -/// +/// /// File rotation ensures we don't end up with unreferenced disk usage. /// If we used a single file for all spilled data, we would end up with /// unreferenced data at the beginning of the file that has already been read /// by readers but we can't delete because you can't truncate from the start of a file. -/// +/// /// Consider the case of a query like `SELECT * FROM large_table WHERE false`. /// Obviously this query produces no output rows, but if we had a spilling operator /// in the middle of this query between the scan and the filter it would see the entire From d05b0cf0c5785de8b2316beaf07852c680fbe7d0 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 7 Nov 2025 12:10:28 -0600 Subject: [PATCH 40/44] fix and track disk usage --- datafusion/execution/src/disk_manager.rs | 218 ++++++++++++++- .../physical-plan/src/repartition/mod.rs | 258 ++++++++---------- .../physical-plan/src/spill/spill_pool.rs | 115 +++++++- 3 files changed, 430 insertions(+), 161 deletions(-) diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs index e450d77db6ada..924095075cc3a 100644 --- a/datafusion/execution/src/disk_manager.rs +++ b/datafusion/execution/src/disk_manager.rs @@ -299,7 +299,7 @@ impl DiskManager { .tempfile_in(local_dirs[dir_index].as_ref()) .map_err(DataFusionError::IoError)?, ), - current_file_disk_usage: 0, + current_file_disk_usage: Arc::new(AtomicU64::new(0)), disk_manager: Arc::clone(self), }) } @@ -331,7 +331,10 @@ pub struct RefCountedTempFile { tempfile: Arc, /// Tracks the current disk usage of this temporary file. See /// [`Self::update_disk_usage`] for more details. - current_file_disk_usage: u64, + /// + /// This is wrapped in Arc so that all clones share the same + /// disk usage tracking, preventing incorrect accounting when clones are dropped. + current_file_disk_usage: Arc, /// The disk manager that created and manages this temporary file disk_manager: Arc, } @@ -341,7 +344,7 @@ impl Clone for RefCountedTempFile { Self { parent_temp_dir: Arc::clone(&self.parent_temp_dir), tempfile: Arc::clone(&self.tempfile), - current_file_disk_usage: self.current_file_disk_usage, + current_file_disk_usage: Arc::clone(&self.current_file_disk_usage), disk_manager: Arc::clone(&self.disk_manager), } } @@ -365,11 +368,14 @@ impl RefCountedTempFile { let metadata = self.tempfile.as_file().metadata()?; let new_disk_usage = metadata.len(); + // Get the old disk usage + let old_disk_usage = self.current_file_disk_usage.load(Ordering::Relaxed); + // Update the global disk usage by: // 1. Subtracting the old file size from the global counter self.disk_manager .used_disk_space - .fetch_sub(self.current_file_disk_usage, Ordering::Relaxed); + .fetch_sub(old_disk_usage, Ordering::Relaxed); // 2. Adding the new file size to the global counter self.disk_manager .used_disk_space @@ -385,23 +391,29 @@ impl RefCountedTempFile { } // 4. Update the local file size tracking - self.current_file_disk_usage = new_disk_usage; + self.current_file_disk_usage + .store(new_disk_usage, Ordering::Relaxed); Ok(()) } pub fn current_disk_usage(&self) -> u64 { - self.current_file_disk_usage + self.current_file_disk_usage.load(Ordering::Relaxed) } } /// When the temporary file is dropped, subtract its disk usage from the disk manager's total impl Drop for RefCountedTempFile { fn drop(&mut self) { - // Subtract the current file's disk usage from the global counter - self.disk_manager - .used_disk_space - .fetch_sub(self.current_file_disk_usage, Ordering::Relaxed); + // Only subtract disk usage when this is the last reference to the file + // Check if we're the last one by seeing if there's only one strong reference + // left to the underlying tempfile (the one we're holding) + if Arc::strong_count(&self.tempfile) == 1 { + let current_usage = self.current_file_disk_usage.load(Ordering::Relaxed); + self.disk_manager + .used_disk_space + .fetch_sub(current_usage, Ordering::Relaxed); + } } } @@ -556,4 +568,190 @@ mod tests { Ok(()) } + + #[test] + fn test_disk_usage_basic() -> Result<()> { + use std::io::Write; + + let dm = Arc::new(DiskManagerBuilder::default().build()?); + let mut temp_file = dm.create_tmp_file("Testing")?; + + // Initially, disk usage should be 0 + assert_eq!(dm.used_disk_space(), 0); + assert_eq!(temp_file.current_disk_usage(), 0); + + // Write some data to the file + temp_file.inner().as_file().write_all(b"hello world")?; + temp_file.update_disk_usage()?; + + // Disk usage should now reflect the written data + let expected_usage = temp_file.current_disk_usage(); + assert!(expected_usage > 0); + assert_eq!(dm.used_disk_space(), expected_usage); + + // Write more data + temp_file.inner().as_file().write_all(b" more data")?; + temp_file.update_disk_usage()?; + + // Disk usage should increase + let new_usage = temp_file.current_disk_usage(); + assert!(new_usage > expected_usage); + assert_eq!(dm.used_disk_space(), new_usage); + + // Drop the file + drop(temp_file); + + // Disk usage should return to 0 + assert_eq!(dm.used_disk_space(), 0); + + Ok(()) + } + + #[test] + fn test_disk_usage_with_clones() -> Result<()> { + use std::io::Write; + + let dm = Arc::new(DiskManagerBuilder::default().build()?); + let mut temp_file = dm.create_tmp_file("Testing")?; + + // Write some data + temp_file.inner().as_file().write_all(b"test data")?; + temp_file.update_disk_usage()?; + + let usage_after_write = temp_file.current_disk_usage(); + assert!(usage_after_write > 0); + assert_eq!(dm.used_disk_space(), usage_after_write); + + // Clone the file + let clone1 = temp_file.clone(); + let clone2 = temp_file.clone(); + + // All clones should see the same disk usage + assert_eq!(clone1.current_disk_usage(), usage_after_write); + assert_eq!(clone2.current_disk_usage(), usage_after_write); + + // Global disk usage should still be the same (not multiplied by number of clones) + assert_eq!(dm.used_disk_space(), usage_after_write); + + // Write more data through one clone + clone1.inner().as_file().write_all(b" more data")?; + let mut mutable_clone1 = clone1; + mutable_clone1.update_disk_usage()?; + + let new_usage = mutable_clone1.current_disk_usage(); + assert!(new_usage > usage_after_write); + + // All clones should see the updated disk usage + assert_eq!(temp_file.current_disk_usage(), new_usage); + assert_eq!(clone2.current_disk_usage(), new_usage); + assert_eq!(mutable_clone1.current_disk_usage(), new_usage); + + // Global disk usage should reflect the new size (not multiplied) + assert_eq!(dm.used_disk_space(), new_usage); + + // Drop one clone + drop(mutable_clone1); + + // Disk usage should NOT change (other clones still exist) + assert_eq!(dm.used_disk_space(), new_usage); + assert_eq!(temp_file.current_disk_usage(), new_usage); + assert_eq!(clone2.current_disk_usage(), new_usage); + + // Drop another clone + drop(clone2); + + // Disk usage should still NOT change (original still exists) + assert_eq!(dm.used_disk_space(), new_usage); + assert_eq!(temp_file.current_disk_usage(), new_usage); + + // Drop the original + drop(temp_file); + + // Now disk usage should return to 0 (last reference dropped) + assert_eq!(dm.used_disk_space(), 0); + + Ok(()) + } + + #[test] + fn test_disk_usage_clones_dropped_out_of_order() -> Result<()> { + use std::io::Write; + + let dm = Arc::new(DiskManagerBuilder::default().build()?); + let mut temp_file = dm.create_tmp_file("Testing")?; + + // Write data + temp_file.inner().as_file().write_all(b"test")?; + temp_file.update_disk_usage()?; + + let usage = temp_file.current_disk_usage(); + assert_eq!(dm.used_disk_space(), usage); + + // Create multiple clones + let clone1 = temp_file.clone(); + let clone2 = temp_file.clone(); + let clone3 = temp_file.clone(); + + // Drop the original first (out of order) + drop(temp_file); + + // Disk usage should still be tracked (clones exist) + assert_eq!(dm.used_disk_space(), usage); + assert_eq!(clone1.current_disk_usage(), usage); + + // Drop clones in different order + drop(clone2); + assert_eq!(dm.used_disk_space(), usage); + + drop(clone1); + assert_eq!(dm.used_disk_space(), usage); + + // Drop the last clone + drop(clone3); + + // Now disk usage should be 0 + assert_eq!(dm.used_disk_space(), 0); + + Ok(()) + } + + #[test] + fn test_disk_usage_multiple_files() -> Result<()> { + use std::io::Write; + + let dm = Arc::new(DiskManagerBuilder::default().build()?); + + // Create multiple temp files + let mut file1 = dm.create_tmp_file("Testing1")?; + let mut file2 = dm.create_tmp_file("Testing2")?; + + // Write to first file + file1.inner().as_file().write_all(b"file1")?; + file1.update_disk_usage()?; + let usage1 = file1.current_disk_usage(); + + assert_eq!(dm.used_disk_space(), usage1); + + // Write to second file + file2.inner().as_file().write_all(b"file2 data")?; + file2.update_disk_usage()?; + let usage2 = file2.current_disk_usage(); + + // Global usage should be sum of both files + assert_eq!(dm.used_disk_space(), usage1 + usage2); + + // Drop first file + drop(file1); + + // Usage should only reflect second file + assert_eq!(dm.used_disk_space(), usage2); + + // Drop second file + drop(file2); + + // Usage should be 0 + assert_eq!(dm.used_disk_space(), 0); + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c367d945612d0..327c5f66e7e0b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -854,11 +854,11 @@ impl ExecutionPlan for RepartitionExec { )?; } - let stream = futures::stream::once(async move { - let num_input_partitions = input.output_partitioning().partition_count(); + let num_input_partitions = input.output_partitioning().partition_count(); + let stream = futures::stream::once(async move { // lock scope - let (mut rx, reservation, spill_readers, abort_helper) = { + let (rx, reservation, spill_readers, abort_helper) = { // lock mutexes let mut state = state.lock(); let state = state.consume_input_streams( @@ -902,12 +902,14 @@ impl ExecutionPlan for RepartitionExec { .into_iter() .zip(spill_readers) .map(|(receiver, spill_stream)| { + // In preserve_order mode, each receiver corresponds to exactly one input partition Box::pin(PerPartitionStream::new( Arc::clone(&schema_captured), receiver, Arc::clone(&abort_helper), Arc::clone(&reservation), spill_stream, + 1, // Each receiver handles one input partition )) as SendableRecordBatchStream }) .collect::>(); @@ -930,19 +932,29 @@ impl ExecutionPlan for RepartitionExec { .with_spill_manager(spill_manager) .build() } else { - // Non-preserve-order case: single input stream, so use the first spill reader - let spill_stream = spill_readers + // Non-preserve-order case: need to merge streams from all input partitions + // Each input partition gets its own spill reader to maintain proper FIFO ordering + let input_streams = rx .into_iter() - .next() - .expect("at least one spill reader should exist"); + .zip(spill_readers) + .map(|(receiver, spill_stream)| { + // In non-preserve-order mode, all input partitions send to the same receiver + Box::pin(PerPartitionStream::new( + Arc::clone(&schema_captured), + receiver, + Arc::clone(&abort_helper), + Arc::clone(&reservation), + spill_stream, + num_input_partitions, // Must wait for all input partitions to finish + )) as SendableRecordBatchStream + }) + .collect::>(); - Ok(Box::pin(RepartitionStream::new( - input.schema(), - rx.swap_remove(0), - abort_helper, - reservation, - spill_stream, - num_input_partitions, + // Merge all input partition streams without sorting (arrival order) + let merged_stream = futures::stream::select_all(input_streams); + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema_captured, + merged_stream, )) as SendableRecordBatchStream) } }) @@ -1368,12 +1380,14 @@ enum StreamState { ReadingSpilled, } -struct RepartitionStream { +/// This struct converts a receiver to a stream. +/// Receiver receives data on an SPSC channel. +struct PerPartitionStream { /// Schema wrapped by Arc schema: SchemaRef, /// channel containing the repartitioned batches - input: DistributionReceiver, + receiver: DistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -1384,17 +1398,19 @@ struct RepartitionStream { /// Infinite stream for reading from the spill pool spill_stream: SendableRecordBatchStream, - /// Current state of the stream (reading from memory or spill) + /// Internal state indicating if we are reading from memory or spill stream state: StreamState, - /// Number of input partitions that have not yet finished + /// Number of input partitions that have not yet finished. + /// In non-preserve-order mode, multiple input partitions send to the same channel, + /// each sending None when complete. We must wait for all of them. remaining_partitions: usize, } -impl RepartitionStream { +impl PerPartitionStream { fn new( schema: SchemaRef, - input: DistributionReceiver, + receiver: DistributionReceiver, drop_helper: Arc>>, reservation: SharedMemoryReservation, spill_stream: SendableRecordBatchStream, @@ -1402,7 +1418,7 @@ impl RepartitionStream { ) -> Self { Self { schema, - input, + receiver, _drop_helper: drop_helper, reservation, spill_stream, @@ -1412,7 +1428,7 @@ impl RepartitionStream { } } -impl Stream for RepartitionStream { +impl Stream for PerPartitionStream { type Item = Result; fn poll_next( @@ -1425,7 +1441,7 @@ impl Stream for RepartitionStream { match self.state { StreamState::ReadingMemory => { // Poll the memory channel for next message - let value = match self.input.recv().poll_unpin(cx) { + let value = match self.receiver.recv().poll_unpin(cx) { Poll::Ready(v) => v, Poll::Pending => { // Nothing from channel, wait @@ -1495,130 +1511,6 @@ impl Stream for RepartitionStream { } } -impl RecordBatchStream for RepartitionStream { - /// Get the schema - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} - -/// This struct converts a receiver to a stream. -/// Receiver receives data on an SPSC channel. -struct PerPartitionStream { - /// Schema wrapped by Arc - schema: SchemaRef, - - /// channel containing the repartitioned batches - receiver: DistributionReceiver, - - /// Handle to ensure background tasks are killed when no longer needed. - _drop_helper: Arc>>, - - /// Memory reservation. - reservation: SharedMemoryReservation, - - /// Infinite stream for reading from the spill pool - spill_stream: SendableRecordBatchStream, - - /// Internal state indicating if we are reading from memory or spill stream - state: StreamState, -} - -impl PerPartitionStream { - fn new( - schema: SchemaRef, - receiver: DistributionReceiver, - drop_helper: Arc>>, - reservation: SharedMemoryReservation, - spill_stream: SendableRecordBatchStream, - ) -> Self { - Self { - schema, - receiver, - _drop_helper: drop_helper, - reservation, - spill_stream, - state: StreamState::ReadingMemory, - } - } -} - -impl Stream for PerPartitionStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - use futures::StreamExt; - - loop { - match self.state { - StreamState::ReadingMemory => { - // Poll the memory channel for next message - let value = match self.receiver.recv().poll_unpin(cx) { - Poll::Ready(v) => v, - Poll::Pending => { - // Nothing from channel, wait - return Poll::Pending; - } - }; - - match value { - Some(Some(v)) => match v { - Ok(RepartitionBatch::Memory(batch)) => { - // Release memory and return batch - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); - return Poll::Ready(Some(Ok(batch))); - } - Ok(RepartitionBatch::Spilled) => { - // Batch was spilled, transition to reading from spill stream - // We must block on spill stream until we get the batch - // to preserve ordering - self.state = StreamState::ReadingSpilled; - continue; - } - Err(e) => { - return Poll::Ready(Some(Err(e))); - } - }, - Some(None) => { - return Poll::Ready(None); - } - None => { - // Channel closed unexpectedly - return Poll::Ready(None); - } - } - } - StreamState::ReadingSpilled => { - // Poll spill stream for the spilled batch - match self.spill_stream.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - self.state = StreamState::ReadingMemory; - return Poll::Ready(Some(Ok(batch))); - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(e))); - } - Poll::Ready(None) => { - // Spill stream ended, keep draining the memory channel - self.state = StreamState::ReadingMemory; - } - Poll::Pending => { - // Spilled batch not ready yet, must wait - // This preserves ordering by blocking until spill data arrives - return Poll::Pending; - } - } - } - } - } - } -} - impl RecordBatchStream for PerPartitionStream { /// Get the schema fn schema(&self) -> SchemaRef { @@ -2679,6 +2571,78 @@ mod test { Ok(()) } + #[tokio::test] + async fn test_hash_partitioning_with_spilling() -> Result<()> { + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_execution::TaskContext; + + // Create input data similar to the round-robin test + let batch1 = record_batch!(("c0", UInt32, [1, 3])).unwrap(); + let batch2 = record_batch!(("c0", UInt32, [2, 4])).unwrap(); + let batch3 = record_batch!(("c0", UInt32, [5, 7])).unwrap(); + let batch4 = record_batch!(("c0", UInt32, [6, 8])).unwrap(); + let schema = batch1.schema(); + + let partition1 = vec![batch1.clone(), batch3.clone()]; + let partition2 = vec![batch2.clone(), batch4.clone()]; + let input_partitions = vec![partition1, partition2]; + + // Set up context with memory limit to test hash partitioning with spilling infrastructure + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; + + let task_ctx = TaskContext::default().with_runtime(runtime); + let task_ctx = Arc::new(task_ctx); + + // Create physical plan with hash partitioning + let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; + let exec = Arc::new(TestMemoryExec::update_cache(Arc::new(exec))); + // Hash partition into 2 partitions by column c0 + let hash_expr = col("c0", &schema)?; + let exec = + RepartitionExec::try_new(exec, Partitioning::Hash(vec![hash_expr], 2))?; + + // Collect all partitions concurrently using JoinSet - this prevents deadlock + // where the distribution channel gate closes when all output channels are full + let mut join_set = tokio::task::JoinSet::new(); + for i in 0..exec.partitioning().partition_count() { + let stream = exec.execute(i, Arc::clone(&task_ctx))?; + join_set.spawn(async move { + let mut count = 0; + futures::pin_mut!(stream); + while let Some(result) = stream.next().await { + let batch = result?; + count += batch.num_rows(); + } + Ok::(count) + }); + } + + // Wait for all partitions and sum the rows + let mut total_rows = 0; + while let Some(result) = join_set.join_next().await { + total_rows += result.unwrap()?; + } + + // Verify we got all rows back + let all_batches = [batch1, batch2, batch3, batch4]; + let expected_rows: usize = all_batches.iter().map(|b| b.num_rows()).sum(); + assert_eq!(total_rows, expected_rows); + + // Verify metrics are available + let metrics = exec.metrics().unwrap(); + // Just verify the metrics can be retrieved (spilling may or may not occur) + let spill_count = metrics.spill_count().unwrap_or(0); + assert!(spill_count > 0); + let spilled_bytes = metrics.spilled_bytes().unwrap_or(0); + assert!(spilled_bytes > 0); + let spilled_rows = metrics.spilled_rows().unwrap_or(0); + assert!(spilled_rows > 0); + + Ok(()) + } + #[tokio::test] async fn test_repartition() -> Result<()> { let schema = test_schema(); diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index b5da50acbfec7..09385e0967e13 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -41,6 +41,8 @@ struct SpillPoolShared { spill_manager: Arc, /// Pool-level wakers to notify when new files are available wakers: Vec, + /// Whether the writer has been dropped (no more files will be added) + writer_dropped: bool, } impl SpillPoolShared { @@ -50,6 +52,7 @@ impl SpillPoolShared { files: VecDeque::new(), spill_manager, wakers: Vec::new(), + writer_dropped: false, } } @@ -77,7 +80,8 @@ impl SpillPoolShared { /// configured in [`channel`]. When dropped, it finalizes the current file so readers /// can access all written data. pub struct SpillPoolWriter { - /// Maximum size in bytes before rotating to a new file + /// Maximum size in bytes before rotating to a new file. + /// Typically set from configuration `datafusion.execution.max_spill_file_size_bytes`. max_file_size_bytes: usize, /// Writer's reference to the current file (also in the shared files queue) current_write_file: Option>>, @@ -218,6 +222,11 @@ impl Drop for SpillPoolWriter { // Wake readers waiting on this file (it's now finished) file_shared.wake_all(); } + + // Mark writer as dropped and wake pool-level readers + let mut shared = self.shared.lock(); + shared.writer_dropped = true; + shared.wake(); } } @@ -226,7 +235,8 @@ impl Drop for SpillPoolWriter { /// /// This is the recommended way to create a spill pool. The writer has exclusive /// write access, and the reader can consume batches in FIFO order. The reader -/// can start reading immediately while the writer continues to write more data. +/// can start reading immediately after the writer appends a batch to the spill file, +/// without waiting for the file to be sealed, while the writer continues to write more data. /// /// Internally this coordinates rotating spill files based on size limits, and /// handles asynchronous notification between the writer and reader using wakers. @@ -648,8 +658,13 @@ impl Stream for SpillPoolReader { continue; } - // Done with this file, no more files available - // Register waker that will get notified when new files are added + // No files in queue - check if writer is done + if shared.writer_dropped { + // Writer is done and no more files will be added - EOF + return Poll::Ready(None); + } + + // Writer still active, register waker that will get notified when new files are added shared.register_waker(cx.waker().clone()); return Poll::Pending; } @@ -1284,4 +1299,96 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_disk_usage_decreases_as_files_consumed() -> Result<()> { + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + + // Step 1: Create a test batch and measure its size + let batch = create_test_batch(0, 100); + let batch_size = batch.get_array_memory_size(); + + // Step 2: Configure file rotation to approximately 1 batch per file + // Create a custom RuntimeEnv so we can access the DiskManager + let runtime = Arc::new(RuntimeEnvBuilder::default().build()?); + let disk_manager = Arc::clone(&runtime.disk_manager); + + let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let schema = create_test_schema(); + let spill_manager = Arc::new(SpillManager::new(runtime, metrics.clone(), schema)); + + let (mut writer, mut reader) = channel(batch_size, spill_manager); + + // Step 3: Write 25 batches to create approximately 25 files + let num_batches = 25; + for i in 0..num_batches { + writer.push_batch(&create_test_batch(i * 100, 100))?; + } + + // Check how many files were created (should be at least a few due to file rotation) + let file_count = metrics.spill_file_count.value(); + assert!( + file_count >= 10, + "Expected at least 10 files with rotation, got {file_count}" + ); + + // Step 4: Verify initial disk usage reflects all files + let initial_disk_usage = disk_manager.used_disk_space(); + assert!( + initial_disk_usage > 0, + "Expected disk usage > 0 after writing batches, got {initial_disk_usage}" + ); + + // Step 5: Read 24 batches (all but 1) + // As each file is fully consumed, it should be dropped and disk usage should decrease + for i in 0..(num_batches - 1) { + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 100); + + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), i * 100); + } + + // Step 6: Verify disk usage decreased but is not zero (at least 1 batch remains) + let partial_disk_usage = disk_manager.used_disk_space(); + assert!( + partial_disk_usage > 0, + "Disk usage should be > 0 with remaining batches" + ); + assert!( + partial_disk_usage < initial_disk_usage, + "Disk usage should have decreased after reading most batches: initial={initial_disk_usage}, partial={partial_disk_usage}" + ); + + // Step 7: Read the final batch + let result = reader.next().await.unwrap()?; + assert_eq!(result.num_rows(), 100); + + // Step 8: Drop writer first to signal no more data will be written + // The reader has infinite stream semantics and will wait for the writer + // to be dropped before returning None + drop(writer); + + // Verify we've read all batches - now the reader should return None + assert!( + reader.next().await.is_none(), + "Should have no more batches to read" + ); + + // Step 9: Drop reader to release all references + drop(reader); + + // Step 10: Verify complete cleanup - disk usage should be 0 + let final_disk_usage = disk_manager.used_disk_space(); + assert_eq!( + final_disk_usage, 0, + "Disk usage should be 0 after all files dropped, got {final_disk_usage}" + ); + + Ok(()) + } } From 23ebd30590344ecf61a95fc08c9c94b643eba67d Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 7 Nov 2025 12:34:05 -0600 Subject: [PATCH 41/44] clean up test --- .../physical-plan/src/spill/spill_pool.rs | 35 +++++++++++-------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 09385e0967e13..492729946f982 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -1304,8 +1304,12 @@ mod tests { async fn test_disk_usage_decreases_as_files_consumed() -> Result<()> { use datafusion_execution::runtime_env::RuntimeEnvBuilder; + // Test configuration + const NUM_BATCHES: usize = 3; + const ROWS_PER_BATCH: usize = 100; + // Step 1: Create a test batch and measure its size - let batch = create_test_batch(0, 100); + let batch = create_test_batch(0, ROWS_PER_BATCH); let batch_size = batch.get_array_memory_size(); // Step 2: Configure file rotation to approximately 1 batch per file @@ -1319,17 +1323,19 @@ mod tests { let (mut writer, mut reader) = channel(batch_size, spill_manager); - // Step 3: Write 25 batches to create approximately 25 files - let num_batches = 25; - for i in 0..num_batches { - writer.push_batch(&create_test_batch(i * 100, 100))?; + // Step 3: Write NUM_BATCHES batches to create approximately NUM_BATCHES files + for i in 0..NUM_BATCHES { + let start = (i * ROWS_PER_BATCH) as i32; + writer.push_batch(&create_test_batch(start, ROWS_PER_BATCH))?; } // Check how many files were created (should be at least a few due to file rotation) let file_count = metrics.spill_file_count.value(); - assert!( - file_count >= 10, - "Expected at least 10 files with rotation, got {file_count}" + assert_eq!( + file_count, + NUM_BATCHES - 1, + "Expected at {} files with rotation, got {file_count}", + NUM_BATCHES - 1 ); // Step 4: Verify initial disk usage reflects all files @@ -1339,24 +1345,25 @@ mod tests { "Expected disk usage > 0 after writing batches, got {initial_disk_usage}" ); - // Step 5: Read 24 batches (all but 1) + // Step 5: Read NUM_BATCHES - 1 batches (all but 1) // As each file is fully consumed, it should be dropped and disk usage should decrease - for i in 0..(num_batches - 1) { + for i in 0..(NUM_BATCHES - 1) { let result = reader.next().await.unwrap()?; - assert_eq!(result.num_rows(), 100); + assert_eq!(result.num_rows(), ROWS_PER_BATCH); let col = result .column(0) .as_any() .downcast_ref::() .unwrap(); - assert_eq!(col.value(0), i * 100); + assert_eq!(col.value(0), (i * ROWS_PER_BATCH) as i32); } // Step 6: Verify disk usage decreased but is not zero (at least 1 batch remains) let partial_disk_usage = disk_manager.used_disk_space(); assert!( - partial_disk_usage > 0, + partial_disk_usage > 0 + && partial_disk_usage < (batch_size * NUM_BATCHES * 2) as u64, "Disk usage should be > 0 with remaining batches" ); assert!( @@ -1366,7 +1373,7 @@ mod tests { // Step 7: Read the final batch let result = reader.next().await.unwrap()?; - assert_eq!(result.num_rows(), 100); + assert_eq!(result.num_rows(), ROWS_PER_BATCH); // Step 8: Drop writer first to signal no more data will be written // The reader has infinite stream semantics and will wait for the writer From 34eb5648b85a06020203226b4608811ea34caad8 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 7 Nov 2025 12:47:46 -0600 Subject: [PATCH 42/44] fix docs --- datafusion/execution/src/disk_manager.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs index 924095075cc3a..f1c768be5b48c 100644 --- a/datafusion/execution/src/disk_manager.rs +++ b/datafusion/execution/src/disk_manager.rs @@ -332,7 +332,7 @@ pub struct RefCountedTempFile { /// Tracks the current disk usage of this temporary file. See /// [`Self::update_disk_usage`] for more details. /// - /// This is wrapped in Arc so that all clones share the same + /// This is wrapped in `Arc` so that all clones share the same /// disk usage tracking, preventing incorrect accounting when clones are dropped. current_file_disk_usage: Arc, /// The disk manager that created and manages this temporary file From 195f72f7804b6290f5f4bebeb868b43166620128 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 7 Nov 2025 13:53:16 -0600 Subject: [PATCH 43/44] use a single waker --- .../physical-plan/src/spill/spill_pool.rs | 44 ++++++++----------- 1 file changed, 19 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index 492729946f982..e8bfc74a33cc4 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -39,8 +39,8 @@ struct SpillPoolShared { files: VecDeque>>, /// SpillManager for creating files and tracking metrics spill_manager: Arc, - /// Pool-level wakers to notify when new files are available - wakers: Vec, + /// Pool-level waker to notify when new files are available (SPSC: only one reader) + waker: Option, /// Whether the writer has been dropped (no more files will be added) writer_dropped: bool, } @@ -51,22 +51,19 @@ impl SpillPoolShared { Self { files: VecDeque::new(), spill_manager, - wakers: Vec::new(), + waker: None, writer_dropped: false, } } /// Registers a waker to be notified when new data is available (pool-level) fn register_waker(&mut self, waker: Waker) { - // Only register if not already present (avoid duplicates) - if !self.wakers.iter().any(|w| w.will_wake(&waker)) { - self.wakers.push(waker); - } + self.waker = Some(waker); } - /// Wakes all pool-level readers + /// Wakes the pool-level reader fn wake(&mut self) { - for waker in self.wakers.drain(..) { + if let Some(waker) = self.waker.take() { waker.wake(); } } @@ -152,7 +149,7 @@ impl SpillPoolWriter { batches_written: 0, estimated_size: 0, writer_finished: false, - wakers: Vec::new(), + waker: None, })); // Push to shared queue and keep reference for writing @@ -177,8 +174,8 @@ impl SpillPoolWriter { file_shared.estimated_size += batch_size; } - // Wake readers waiting on this specific file - file_shared.wake_all(); + // Wake reader waiting on this specific file + file_shared.wake(); // Check if we need to rotate let needs_rotation = file_shared.estimated_size > self.max_file_size_bytes; @@ -190,8 +187,8 @@ impl SpillPoolWriter { } // Mark as finished so readers know not to wait for more data file_shared.writer_finished = true; - // Wake readers waiting on this file (it's now finished) - file_shared.wake_all(); + // Wake reader waiting on this file (it's now finished) + file_shared.wake(); } else { // Release lock drop(file_shared); @@ -219,8 +216,8 @@ impl Drop for SpillPoolWriter { // Mark as finished so readers know not to wait for more data file_shared.writer_finished = true; - // Wake readers waiting on this file (it's now finished) - file_shared.wake_all(); + // Wake reader waiting on this file (it's now finished) + file_shared.wake(); } // Mark writer as dropped and wake pool-level readers @@ -427,22 +424,19 @@ struct ActiveSpillFileShared { estimated_size: usize, /// Whether the writer has finished writing to this file writer_finished: bool, - /// Wakers for readers waiting on this specific file - wakers: Vec, + /// Waker for reader waiting on this specific file (SPSC: only one reader) + waker: Option, } impl ActiveSpillFileShared { /// Registers a waker to be notified when new data is written to this file fn register_waker(&mut self, waker: Waker) { - // Only register if not already present (avoid duplicates) - if !self.wakers.iter().any(|w| w.will_wake(&waker)) { - self.wakers.push(waker); - } + self.waker = Some(waker); } - /// Wakes all readers waiting on this file - fn wake_all(&mut self) { - for waker in self.wakers.drain(..) { + /// Wakes the reader waiting on this file + fn wake(&mut self) { + if let Some(waker) = self.waker.take() { waker.wake(); } } From 86364b2cc45595e2cc2aa0f35a0d680d91ebdbfc Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 7 Nov 2025 14:20:21 -0600 Subject: [PATCH 44/44] fix test, make MPSC, docs, cleanup --- .../physical-plan/src/repartition/mod.rs | 63 +++++---- .../physical-plan/src/spill/spill_pool.rs | 132 +++++++++++------- 2 files changed, 113 insertions(+), 82 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 327c5f66e7e0b..890113de3d3dd 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -166,9 +166,9 @@ struct PartitionChannels { rx: InputPartitionsToCurrentPartitionReceiver, /// Memory reservation for this output partition reservation: SharedMemoryReservation, - /// Spill writers for writing spilled data - one per input partition (FIFO semantics). - /// Wrapped in Option so they can be moved out when creating OutputChannels. - spill_writers: Vec>, + /// Spill writers for writing spilled data. + /// SpillPoolWriter is Clone, so multiple writers can share state in non-preserve-order mode. + spill_writers: Vec, /// Spill readers for reading spilled data - one per input partition (FIFO semantics). /// Each (input, output) pair gets its own reader to maintain proper ordering. spill_readers: Vec, @@ -318,15 +318,22 @@ impl RepartitionExecState { .register(context.memory_pool()), )); - // Create one spill channel per input partition for this output partition - // This ensures proper FIFO ordering within each (input, output) pair + // Create spill channels based on mode: + // - preserve_order: one spill channel per (input, output) pair for proper FIFO ordering + // - non-preserve-order: one shared spill channel per output partition since all inputs + // share the same receiver let max_file_size = context .session_config() .options() .execution .max_spill_file_size_bytes; + let num_spill_channels = if preserve_order { + num_input_partitions + } else { + 1 + }; let (spill_writers, spill_readers): (Vec<_>, Vec<_>) = (0 - ..num_input_partitions) + ..num_spill_channels) .map(|_| spill_pool::channel(max_file_size, Arc::clone(&spill_manager))) .unzip(); @@ -337,7 +344,7 @@ impl RepartitionExecState { rx, reservation, spill_readers, - spill_writers: spill_writers.into_iter().map(Some).collect(), + spill_writers, }, ); } @@ -348,16 +355,18 @@ impl RepartitionExecState { std::mem::take(streams_and_metrics).into_iter().enumerate() { let txs: HashMap<_, _> = channels - .iter_mut() + .iter() .map(|(partition, channels)| { + // In preserve_order mode: each input gets its own spill writer (index i) + // In non-preserve-order mode: all inputs share spill writer 0 via clone + let spill_writer_idx = if preserve_order { i } else { 0 }; ( *partition, OutputChannel { sender: channels.tx[i].clone(), reservation: Arc::clone(&channels.reservation), - spill_writer: channels.spill_writers[i] - .take() - .expect("spill_writer should not be taken yet"), + spill_writer: channels.spill_writers[spill_writer_idx] + .clone(), }, ) }) @@ -932,29 +941,21 @@ impl ExecutionPlan for RepartitionExec { .with_spill_manager(spill_manager) .build() } else { - // Non-preserve-order case: need to merge streams from all input partitions - // Each input partition gets its own spill reader to maintain proper FIFO ordering - let input_streams = rx + // Non-preserve-order case: single input stream, so use the first spill reader + let spill_stream = spill_readers .into_iter() - .zip(spill_readers) - .map(|(receiver, spill_stream)| { - // In non-preserve-order mode, all input partitions send to the same receiver - Box::pin(PerPartitionStream::new( - Arc::clone(&schema_captured), - receiver, - Arc::clone(&abort_helper), - Arc::clone(&reservation), - spill_stream, - num_input_partitions, // Must wait for all input partitions to finish - )) as SendableRecordBatchStream - }) - .collect::>(); + .next() + .expect("at least one spill reader should exist"); - // Merge all input partition streams without sorting (arrival order) - let merged_stream = futures::stream::select_all(input_streams); - Ok(Box::pin(RecordBatchStreamAdapter::new( + Ok(Box::pin(PerPartitionStream::new( schema_captured, - merged_stream, + rx.into_iter() + .next() + .expect("at least one receiver should exist"), + abort_helper, + reservation, + spill_stream, + num_input_partitions, )) as SendableRecordBatchStream) } }) diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index e8bfc74a33cc4..bbe54ca45caa3 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -33,16 +33,34 @@ use super::spill_manager::SpillManager; /// Shared state between the writer and readers of a spill pool. /// This contains the queue of files and coordination state. +/// +/// # Locking Design +/// +/// This struct uses **fine-grained locking** with nested `Arc>`: +/// - `SpillPoolShared` is wrapped in `Arc>` (outer lock) +/// - Each `ActiveSpillFileShared` is wrapped in `Arc>` (inner lock) +/// +/// This enables: +/// 1. **Short critical sections**: The outer lock is held only for queue operations +/// 2. **I/O outside locks**: Disk I/O happens while holding only the file-specific lock +/// 3. **Concurrent operations**: Reader can access the queue while writer does I/O +/// +/// **Lock ordering discipline**: Never hold both locks simultaneously to prevent deadlock. +/// Always: acquire outer lock → release outer lock → acquire inner lock (if needed). struct SpillPoolShared { /// Queue of ALL files (including the current write file if it exists). /// Readers always read from the front of this queue (FIFO). + /// Each file has its own lock to enable concurrent reader/writer access. files: VecDeque>>, /// SpillManager for creating files and tracking metrics spill_manager: Arc, - /// Pool-level waker to notify when new files are available (SPSC: only one reader) + /// Pool-level waker to notify when new files are available (single reader) waker: Option, /// Whether the writer has been dropped (no more files will be added) writer_dropped: bool, + /// Writer's reference to the current file (shared by all cloned writers). + /// Has its own lock to allow I/O without blocking queue access. + current_write_file: Option>>, } impl SpillPoolShared { @@ -53,6 +71,7 @@ impl SpillPoolShared { spill_manager, waker: None, writer_dropped: false, + current_write_file: None, } } @@ -69,20 +88,21 @@ impl SpillPoolShared { } } -/// Writer for a spill pool. Provides exclusive write access with FIFO semantics. +/// Writer for a spill pool. Provides coordinated write access with FIFO semantics. /// /// Created by [`channel`]. See that function for architecture diagrams and usage examples. /// +/// The writer is `Clone`, allowing multiple writers to coordinate on the same pool. +/// All clones share the same current write file and coordinate file rotation. /// The writer automatically manages file rotation based on the `max_file_size_bytes` -/// configured in [`channel`]. When dropped, it finalizes the current file so readers -/// can access all written data. +/// configured in [`channel`]. When the last writer clone is dropped, it finalizes the +/// current file so readers can access all written data. +#[derive(Clone)] pub struct SpillPoolWriter { /// Maximum size in bytes before rotating to a new file. /// Typically set from configuration `datafusion.execution.max_spill_file_size_bytes`. max_file_size_bytes: usize, - /// Writer's reference to the current file (also in the shared files queue) - current_write_file: Option>>, - /// Shared state with readers + /// Shared state with readers (includes current_write_file for coordination) shared: Arc>, } @@ -124,7 +144,7 @@ impl SpillPoolWriter { /// # Errors /// /// Returns an error if disk I/O fails or disk quota is exceeded. - pub fn push_batch(&mut self, batch: &RecordBatch) -> Result<()> { + pub fn push_batch(&self, batch: &RecordBatch) -> Result<()> { if batch.num_rows() == 0 { // Skip empty batches return Ok(()); @@ -132,12 +152,14 @@ impl SpillPoolWriter { let batch_size = batch.get_array_memory_size(); + // Fine-grained locking: Lock shared state briefly for queue access + let mut shared = self.shared.lock(); + // Create new file if we don't have one yet - if self.current_write_file.is_none() { - let spill_manager = { - let shared = self.shared.lock(); - Arc::clone(&shared.spill_manager) - }; + if shared.current_write_file.is_none() { + let spill_manager = Arc::clone(&shared.spill_manager); + // Release shared lock before disk I/O (fine-grained locking) + drop(shared); let writer = spill_manager.create_in_progress_file("SpillPool")?; // Clone the file so readers can access it immediately @@ -152,19 +174,21 @@ impl SpillPoolWriter { waker: None, })); - // Push to shared queue and keep reference for writing - { - let mut shared = self.shared.lock(); - shared.files.push_back(Arc::clone(&file_shared)); - shared.wake(); // Wake readers waiting for new files - } - self.current_write_file = Some(file_shared); + // Re-acquire lock and push to shared queue + shared = self.shared.lock(); + shared.files.push_back(Arc::clone(&file_shared)); + shared.current_write_file = Some(file_shared); + shared.wake(); // Wake readers waiting for new files } - let current_write_file = self.current_write_file.take(); + let current_write_file = shared.current_write_file.take(); + // Release shared lock before file I/O (fine-grained locking) + // This allows readers to access the queue while we do disk I/O + drop(shared); - // Write batch to current file + // Write batch to current file - lock only the specific file if let Some(current_file) = current_write_file { + // Now lock just this file for I/O (separate from shared lock) let mut file_shared = current_file.lock(); // Append the batch @@ -189,11 +213,13 @@ impl SpillPoolWriter { file_shared.writer_finished = true; // Wake reader waiting on this file (it's now finished) file_shared.wake(); + // Don't put back current_write_file - let it rotate } else { - // Release lock + // Release file lock drop(file_shared); // Put back the current file for further writing - self.current_write_file = Some(current_file); + let mut shared = self.shared.lock(); + shared.current_write_file = Some(current_file); } } @@ -203,8 +229,13 @@ impl SpillPoolWriter { impl Drop for SpillPoolWriter { fn drop(&mut self) { - // Finalize the current file when writer is dropped - if let Some(current_file) = self.current_write_file.take() { + let mut shared = self.shared.lock(); + + // Finalize the current file when the last writer is dropped + if let Some(current_file) = shared.current_write_file.take() { + // Release shared lock before locking file + drop(shared); + let mut file_shared = current_file.lock(); // Finish the current writer if it exists @@ -218,22 +249,25 @@ impl Drop for SpillPoolWriter { // Wake reader waiting on this file (it's now finished) file_shared.wake(); + + drop(file_shared); + shared = self.shared.lock(); } // Mark writer as dropped and wake pool-level readers - let mut shared = self.shared.lock(); shared.writer_dropped = true; shared.wake(); } } -/// Creates a paired writer and reader for a spill pool with SPSC (single-producer, single-consumer) +/// Creates a paired writer and reader for a spill pool with MPSC (multi-producer, single-consumer) /// semantics. /// -/// This is the recommended way to create a spill pool. The writer has exclusive -/// write access, and the reader can consume batches in FIFO order. The reader -/// can start reading immediately after the writer appends a batch to the spill file, -/// without waiting for the file to be sealed, while the writer continues to write more data. +/// This is the recommended way to create a spill pool. The writer is `Clone`, allowing +/// multiple producers to coordinate writes to the same pool. The reader can consume batches +/// in FIFO order. The reader can start reading immediately after a writer appends a batch +/// to the spill file, without waiting for the file to be sealed, while writers continue to +/// write more data. /// /// Internally this coordinates rotating spill files based on size limits, and /// handles asynchronous notification between the writer and reader using wakers. @@ -348,7 +382,7 @@ impl Drop for SpillPoolWriter { /// # let spill_manager = Arc::new(SpillManager::new(env, metrics, schema.clone())); /// # /// // Create channel with 1MB file size limit -/// let (mut writer, mut reader) = spill_pool::channel(1024 * 1024, spill_manager); +/// let (writer, mut reader) = spill_pool::channel(1024 * 1024, spill_manager); /// /// // Spawn writer task to produce batches /// let write_handle = tokio::spawn(async move { @@ -401,7 +435,6 @@ pub fn channel( let writer = SpillPoolWriter { max_file_size_bytes, - current_write_file: None, shared: Arc::clone(&shared), }; @@ -718,7 +751,7 @@ mod tests { #[tokio::test] async fn test_basic_write_and_read() -> Result<()> { - let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + let (writer, mut reader) = create_spill_channel(1024 * 1024); // Write one batch let batch1 = create_test_batch(0, 10); @@ -740,7 +773,7 @@ mod tests { #[tokio::test] async fn test_single_batch_write_read() -> Result<()> { - let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + let (writer, mut reader) = create_spill_channel(1024 * 1024); // Write one batch let batch = create_test_batch(0, 5); @@ -764,7 +797,7 @@ mod tests { #[tokio::test] async fn test_multiple_batches_sequential() -> Result<()> { - let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + let (writer, mut reader) = create_spill_channel(1024 * 1024); // Write multiple batches for i in 0..5 { @@ -805,7 +838,7 @@ mod tests { #[tokio::test] async fn test_empty_batch_skipping() -> Result<()> { - let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + let (writer, mut reader) = create_spill_channel(1024 * 1024); // Write empty batch let empty_batch = create_test_batch(0, 0); @@ -828,8 +861,7 @@ mod tests { let batch1 = create_test_batch(0, 10); let batch_size = batch1.get_array_memory_size() + 1; - let (mut writer, mut reader, metrics) = - create_spill_channel_with_metrics(batch_size); + let (writer, mut reader, metrics) = create_spill_channel_with_metrics(batch_size); // Write first batch (should fit in first file) writer.push_batch(&batch1)?; @@ -921,8 +953,7 @@ mod tests { let batch_size = batches[0].get_array_memory_size() * 2 + 1; // Very small max_file_size to force frequent rotations - let (mut writer, mut reader, metrics) = - create_spill_channel_with_metrics(batch_size); + let (writer, mut reader, metrics) = create_spill_channel_with_metrics(batch_size); // Write many batches to cause multiple rotations for i in 0..10 { @@ -972,7 +1003,7 @@ mod tests { #[tokio::test] async fn test_single_batch_larger_than_limit() -> Result<()> { // Very small limit - let (mut writer, mut reader, metrics) = create_spill_channel_with_metrics(100); + let (writer, mut reader, metrics) = create_spill_channel_with_metrics(100); // Write a batch that exceeds the limit let large_batch = create_test_batch(0, 100); @@ -1019,7 +1050,7 @@ mod tests { #[tokio::test] async fn test_very_small_max_file_size() -> Result<()> { // Test with just 1 byte max (extreme case) - let (mut writer, mut reader) = create_spill_channel(1); + let (writer, mut reader) = create_spill_channel(1); // Any batch will exceed this limit let batch = create_test_batch(0, 5); @@ -1039,8 +1070,7 @@ mod tests { let batch_size = batch.get_array_memory_size(); // Set max_file_size to exactly the batch size - let (mut writer, mut reader, metrics) = - create_spill_channel_with_metrics(batch_size); + let (writer, mut reader, metrics) = create_spill_channel_with_metrics(batch_size); // Write first batch (exactly at the size limit) writer.push_batch(&batch)?; @@ -1101,7 +1131,7 @@ mod tests { #[tokio::test] async fn test_concurrent_reader_writer() -> Result<()> { - let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + let (writer, mut reader) = create_spill_channel(1024 * 1024); // Spawn writer task let writer_handle = SpawnedTask::spawn(async move { @@ -1141,7 +1171,7 @@ mod tests { #[tokio::test] async fn test_reader_catches_up_to_writer() -> Result<()> { - let (mut writer, mut reader) = create_spill_channel(1024 * 1024); + let (writer, mut reader) = create_spill_channel(1024 * 1024); #[derive(Clone, Copy, Debug, PartialEq, Eq)] enum ReadWriteEvent { @@ -1210,7 +1240,7 @@ mod tests { #[tokio::test] async fn test_reader_starts_after_writer_finishes() -> Result<()> { - let (mut writer, reader) = create_spill_channel(128); + let (writer, reader) = create_spill_channel(128); // Writer writes all data for i in 0..5 { @@ -1249,7 +1279,7 @@ mod tests { let spill_manager = Arc::new(SpillManager::new(Arc::clone(&env), metrics.clone(), schema)); - let (mut writer, mut reader) = channel(1024 * 1024, spill_manager); + let (writer, mut reader) = channel(1024 * 1024, spill_manager); // Write some batches for i in 0..5 { @@ -1315,7 +1345,7 @@ mod tests { let schema = create_test_schema(); let spill_manager = Arc::new(SpillManager::new(runtime, metrics.clone(), schema)); - let (mut writer, mut reader) = channel(batch_size, spill_manager); + let (writer, mut reader) = channel(batch_size, spill_manager); // Step 3: Write NUM_BATCHES batches to create approximately NUM_BATCHES files for i in 0..NUM_BATCHES {