Skip to content

Conversation

@adriangb
Copy link
Contributor

@adriangb adriangb commented Oct 21, 2025

Addresses #18014 (comment), potentially paves the path to solve #18011 for other operators as well

@adriangb adriangb requested a review from crepererum October 21, 2025 18:09
@github-actions github-actions bot added sqllogictest SQL Logic Tests (.slt) common Related to common crate physical-plan Changes to the physical-plan crate labels Oct 21, 2025
@adriangb adriangb marked this pull request as draft October 21, 2025 18:15
@adriangb adriangb removed the request for review from crepererum October 21, 2025 18:15
@adriangb
Copy link
Contributor Author

Marking as draft for now. Open to input but needs a bit more work. I'm still familiarizing myself with the spilling infrastructure.

@github-actions github-actions bot added the documentation Improvements or additions to documentation label Oct 21, 2025
@2010YOUY01
Copy link
Contributor

This PR is setting size limit to spill files, when the size exceeds threshold, the spiller rotates to new file. I'm wondering why this design? Now the spill writer and reader is able to do streaming read/write, so a large spill file usually won't be the issue, unless it needs more parallelism somewhere.

@adriangb
Copy link
Contributor Author

This PR is setting size limit to spill files, when the size exceeds threshold, the spiller rotates to new file. I'm wondering why this design? Now the spill writer and reader is able to do streaming read/write, so a large spill file usually won't be the issue, unless it needs more parallelism somewhere.

The issue with using a single FIFO file is that you accumulate dead data, bloating disk usage considerably. The idea is to cap that at say 100MB and then start a new file so that once all of the original file has been consumed we can garbage collect it.

@adriangb adriangb force-pushed the spilling-pool branch 2 times, most recently from f7c84fe to c5b40ee Compare October 22, 2025 21:35
@adriangb
Copy link
Contributor Author

@2010YOUY01 let me know if that makes sense, there's an example of this issue in #18011

@adriangb adriangb requested a review from Copilot October 22, 2025 21:44
Copy link
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull Request Overview

This PR introduces a SpillPool abstraction to centralize the management of spill files with FIFO semantics. The pool handles file rotation, batching multiple record batches into single files up to a configurable size limit, and provides streaming read access to spilled data.

Key changes:

  • Adds a new SpillPool module with FIFO queue semantics for managing spill files
  • Integrates SpillPool into RepartitionExec to replace the previous one-file-per-batch approach
  • Adds a new configuration option max_spill_file_size_bytes (default 100MB) to control when spill files rotate

Reviewed Changes

Copilot reviewed 6 out of 6 changed files in this pull request and generated 5 comments.

Show a summary per file
File Description
datafusion/physical-plan/src/spill/spill_pool.rs New module implementing SpillPool and SpillPoolStream with comprehensive tests
datafusion/physical-plan/src/spill/mod.rs Exports the new spill_pool module
datafusion/physical-plan/src/repartition/mod.rs Refactored to use SpillPool instead of one-file-per-batch spilling
datafusion/common/src/config.rs Adds max_spill_file_size_bytes configuration option
docs/source/user-guide/configs.md Documents the new max_spill_file_size_bytes configuration
datafusion/sqllogictest/test_files/information_schema.slt Updates test expectations to include new configuration option

Tip: Customize your code reviews with copilot-instructions.md. Create the file or learn how to get started.

@2010YOUY01
Copy link
Contributor

This PR is setting size limit to spill files, when the size exceeds threshold, the spiller rotates to new file. I'm wondering why this design? Now the spill writer and reader is able to do streaming read/write, so a large spill file usually won't be the issue, unless it needs more parallelism somewhere.

The issue with using a single FIFO file is that you accumulate dead data, bloating disk usage considerably. The idea is to cap that at say 100MB and then start a new file so that once all of the original file has been consumed we can garbage collect it.

This makes a lot of sense, operators should release disk usage sooner if possible.

I will to review it soon.

@adriangb adriangb marked this pull request as ready for review October 23, 2025 05:35
@adriangb adriangb requested a review from Copilot October 23, 2025 12:50
Copy link
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull Request Overview

Copilot reviewed 6 out of 6 changed files in this pull request and generated no new comments.


Tip: Customize your code reviews with copilot-instructions.md. Create the file or learn how to get started.

Copy link
Contributor

@2010YOUY01 2010YOUY01 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you, looks good in general.

I think it needs several additional test coverage:

  1. e2e tests, potentially a query that triggers spilling in RepartitionExec. I think we can also do a quick benchmark on it to see how things work.
  2. #18207 (comment)

I also left some suggestions to simplify the implementation, but they're optional.

///
/// 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).
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now we're reclaiming disk space in the 'chunked file' granularity, perhaps this truncating way don't have to be mentioned, since it don't have a real usage yet.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yep was leftover from a previous implementation

Comment on lines 95 to 98
/// Size of current write file in bytes (estimated)
current_write_size: usize,
/// Number of batches written to current file
current_batch_count: usize,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can track them inside InProgressSpillFile, and expose an API. This approach can simplify SpillPool a bit.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

/// SpillManager for creating files and tracking metrics
spill_manager: Arc<SpillManager>,
/// Schema for batches (used by SpillPoolStream to implement RecordBatchStream)
schema: SchemaRef,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To avoid duplication, the schema inside spill_manager can be used instead.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍🏻 done

/// Shared reference to the spill pool
spill_pool: Arc<Mutex<SpillPool>>,
/// SpillManager for creating streams from spill files
spill_manager: Arc<SpillManager>,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to use the spill_manager inside spill_pool, and eliminate this field?

// Input finished and no more spilled data - we're done
return Poll::Ready(None);
}
// Otherwise check the channel
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PerPartitionStream is for the order-preserving case of RepartitionExec, it seems a bit tricky to get the order correct, I recommend to find the existing tests for order-preserving repartition, and include spilling to it.


// Append batch to current file
if let Some(ref mut file) = self.current_write_file {
file.append_batch(batch)?;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A potential follow-up to do: #18261


# End repartition on empty columns test

# Start spilling tests
Copy link
Contributor Author

@adriangb adriangb Oct 24, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test passes on main but fails on datafusion-cli v50:

❯ datafusion-cli           
DataFusion CLI v50.0.0
> 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');
SET datafusion.runtime.memory_limit = '12K';
EXPLAIN ANALYZE
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 row(s) fetched. 
Elapsed 0.026 seconds.

0 row(s) fetched. 
Elapsed 0.001 seconds.

Resources exhausted: Additional allocation failed with top memory consumers (across reservations) as:
  RepartitionExec[Merge 11]#85(can spill: false) consumed 1896.0 B, peak 1896.0 B,
  RepartitionExec[Merge 5]#73(can spill: false) consumed 1448.0 B, peak 1448.0 B,
  RepartitionExec[Merge 3]#59(can spill: false) consumed 1384.0 B, peak 1384.0 B,
  RepartitionExec[Merge 1]#56(can spill: false) consumed 1304.0 B, peak 1304.0 B,
  RepartitionExec[8]#48(can spill: false) consumed 1216.0 B, peak 1856.0 B.
Error: Failed to allocate additional 240.0 B for RepartitionExec[Merge 6] with 0.0 B already allocated for this reservation - 8.0 B remain available for the total pool

It's not this PR that enabled it to pass, it was #18014, but worth adding anyway.

@adriangb
Copy link
Contributor Author

adriangb commented Nov 5, 2025

@2010YOUY01 I haven't merged this because I dove in to do another pass and found various issues / bugs. I'm working on a reworked version which has nicer APIs, fixes the bugs and manages to enable reading of a spill file as it's being written (making sure there is no latency hit for spilling other than the IO cost). I'll ping back once I think this is ready for another review (sadly I think that will be warranted). Thank you so much for your review and patience 🙏🏻

@github-actions github-actions bot added the execution Related to the execution crate label Nov 5, 2025
@adriangb adriangb requested a review from Copilot November 5, 2025 14:18
@adriangb
Copy link
Contributor Author

adriangb commented Nov 5, 2025

Okay @2010YOUY01 this is ready for review 😄

Copy link
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull Request Overview

Copilot reviewed 9 out of 9 changed files in this pull request and generated 2 comments.


💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

adriangb and others added 4 commits November 5, 2025 08:30
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
/// - **Memory management**: Files automatically rotate based on size limits
/// - **Concurrent I/O**: Reader and writer operate independently with async coordination
/// - **FIFO semantics**: Batches are consumed in the exact order they were written
pub fn channel(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Correct me if I'm not understand it correctly: this is a SPSC channel, and the writer and the reader can operator concurrently inside the same in progress file, like the following timeline:

  1. writer write batch B0 to F1
  2. writer write batch B1 to F1
  3. reader read B0
  4. reader read B1, no more batch to read -> wait on the waker
  5. writer write batch B2 and finish F1 then continue writing to a new file, wake up the waiting reader, then write B3 to F2
  6. reader wake up and read B2, then drop F1 and release the resources.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes SPSC channel is the right terminology and given the new API that is very channel like I think I can update the docs to reflect that

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

/// # Architecture
///
/// ```text
/// ┌─────────────────────────────────────────────────────────────────────────┐
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This figure looks 👍🏼 , are they AI-generated? I’ve tried generating ASCII arts with ChatGPT several times, but all attempts failed, so I still draw them manually.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A mix. Sonnet 4.5 always gets the formatting slightly wrong but I still find it easier t have it generate them and review / fix them up than to draw by hand in some cases. Sometimes it fails miserably and you have to do it yourself...

@adriangb
Copy link
Contributor Author

adriangb commented Nov 6, 2025

@2010YOUY01 just wanted to check, did you want me to leave this up for review and approve again or is the review in #18207 (review) plus the existing approval a sign to merge it?

Copy link
Contributor

@2010YOUY01 2010YOUY01 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have went through another review pass, I think it's good to go after the disk usage counting issue is fixed. Thanks again!

Self {
parent_temp_dir: Arc::clone(&self.parent_temp_dir),
tempfile: Arc::clone(&self.tempfile),
current_file_disk_usage: self.current_file_disk_usage,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we allow clone, the disk usage tracking seems to be a issue. See below update_disk_usage() and Drop

let input_partitions = vec![partition];

// Use RoundRobinBatch to ensure predictable ordering
let partitioning = Partitioning::RoundRobinBatch(2);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also test Partitioning::Hash

/// configured in [`channel`]. When dropped, it finalizes the current file so readers
/// can access all written data.
pub struct SpillPoolWriter {
/// Maximum size in bytes before rotating to a new file
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/// Maximum size in bytes before rotating to a new file
/// Maximum size in bytes before rotating to a new file.
/// Typically set from configuration `datafusion.execution.max_spill_file_size_bytes`.

///
/// This is the recommended way to create a spill pool. The writer has exclusive
/// write access, and the reader can consume batches in FIFO order. The reader
/// can start reading immediately while the writer continues to write more data.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/// can start reading immediately while the writer continues to write more data.
/// can start reading immediately after the writer appends a batch to the spill file
/// , without waiting for the file to be sealed, while the writer continues to write
/// additional data.

/// Whether the writer has finished writing to this file
writer_finished: bool,
/// Wakers for readers waiting on this specific file
wakers: Vec<Waker>,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If it is only intended to have a single reader, do we still need a Vec?

}

#[tokio::test]
async fn test_reader_catches_up_to_writer() -> Result<()> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I just want to point out an 'unsafe' implementation here, other reviewers might want to double-check it.

Tailing (concurrent reader and writer on the same file) within the same IPC Stream file should be a risky operation, I think it's not allowed from the spec https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format

But we have external coordination to ensure

  • Batch level mutex: reader won't see half-written records
  • Reader won't call reader.next() if it has already reached the last batch, and the writer plans to append more batches later, here is a example:
# 1 reader and 1 writer operating concurrently on the same file
T0: writer append batch `B0` to spill file `F0`
T1: reader calls `.next()` to read `B0`, and we have coordination to ensure it won't call `.next()` again before the writer appends more batches, otherwise I think it can trigger some error.
T2: writer append `B1`
...

Also this test has triggered the edge case, so I think the implementation is good.

}
}

#[cfg(test)]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Love the test coverage 💯

@adriangb adriangb added this pull request to the merge queue Nov 7, 2025
@adriangb
Copy link
Contributor Author

adriangb commented Nov 7, 2025

Thanks so much @2010YOUY01 !

Merged via the queue into apache:main with commit d02642e Nov 7, 2025
33 checks passed
@adriangb adriangb deleted the spilling-pool branch November 7, 2025 21:40
codetyri0n pushed a commit to codetyri0n/datafusion that referenced this pull request Nov 11, 2025
Addresses
apache#18014 (comment),
potentially paves the path to solve
apache#18011 for other operators as
well

---------

Co-authored-by: Yongting You <2010youy01@gmail.com>
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

common Related to common crate documentation Improvements or additions to documentation execution Related to the execution crate physical-plan Changes to the physical-plan crate sqllogictest SQL Logic Tests (.slt)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants