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/24] 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 bc321b227ee5..7953493d787e 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 8174f71c31af..d14ba4b488a8 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 5b9a91e781b1..a1a7bbee05c8 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 000000000000..9557535ef797 --- /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/24] 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 d14ba4b488a8..9e6e3d1d4292 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 9557535ef797..ded66bba84c1 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 c67405715149..8aeac175620a 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/24] 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 9e6e3d1d4292..2cf07dc1e548 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 ded66bba84c1..5a9d9717afaf 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/24] 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 2cf07dc1e548..5951abfabab1 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 7ca5eb8f7be4..949bcb201749 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/24] 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 5951abfabab1..0daa8a5d185b 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 5a9d9717afaf..c481b2e36a7d 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/24] 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 c481b2e36a7d..f30e3baed295 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/24] 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 0daa8a5d185b..80c7cc6b1999 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 f30e3baed295..4e12c476278f 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/24] 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 80c7cc6b1999..7df51362f35c 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/24] 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 7df51362f35c..57a96f0fa0c9 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 4e12c476278f..c24292949f25 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/24] 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 57a96f0fa0c9..48093c46239b 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/24] 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 c24292949f25..0d452ab7c12e 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/24] 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 48093c46239b..b1f1166902e9 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 0d452ab7c12e..fdb4ce7e69f0 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/24] 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 fdb4ce7e69f0..f388e34a6ca1 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/24] 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 b1f1166902e9..9c5d52d45d1e 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 f388e34a6ca1..43ba05d67f15 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/24] 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 7953493d787e..4389e515efe7 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 9c5d52d45d1e..a68f9057e1b1 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 14917e23b792..3149c829c726 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 cc39102d8981..788746fe2e35 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 43ba05d67f15..7636f2a51c9e 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/24] 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 a68f9057e1b1..08f37e4921c0 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/24] 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 29d20d10b671..67b3f92a6d99 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/24] 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 4389e515efe7..aa0331917553 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 8aeac175620a..b7cf918e2b25 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 949bcb201749..1c1c901262a0 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/24] 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 a1a7bbee05c8..395c98ca5f8d 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/24] 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 aa0331917553..c74e03b1728e 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/24] 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 08f37e4921c0..17040d2ac4a8 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 788746fe2e35..b43323e62399 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/24] 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 b7cf918e2b25..3f869a19c675 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/24] 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 67b3f92a6d99..29d20d10b671 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/24] 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 17040d2ac4a8..42c828e48656 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.