Skip to content

Conversation

@adriangb
Copy link
Contributor

@adriangb adriangb commented Oct 10, 2025

Addresses #17334 (comment)

I ran into this using datafusion-distributed which I think makes the issue of partition execution time skew even more likely to happen. As per that issue it can also happen with non-distributed queries, e.g. if one partition's sort spills and others don't.

Due to the nature of ReparitionExec I don't think we can bound the channels, that could lead to deadlocks. So what I did was at least make queries that would have previously fail continue forward with disk spilling. I did not account for memory usage when reading batches back from disk since DataFusion in general does not generally account for "in-flight" batches.

Written with help from Claude

@github-actions github-actions bot added the physical-plan Changes to the physical-plan crate label Oct 10, 2025
@adriangb adriangb changed the title Spilling repartition Add spilling to RepartitionExec Oct 10, 2025
Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Thanks @adriangb -- can you please add some "end to end" tests (aka that run a real query / LogicalPlan with a memory limit) and show that it still works?

@adriangb
Copy link
Contributor Author

Good call. I'll have to think about how we can force RepartitionExec specifically to spill. I've only experienced it with large datasets, lopsided hash keys, etc

@comphead
Copy link
Contributor

Should we get this PR as draft for now before spilling tests added?

(RepartitionBatch::Memory(batch), true)
}
Err(_) => {
// We're memory limited - spill this single batch to its own file
Copy link
Contributor

Choose a reason for hiding this comment

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

would we avoid out of memory error but risk "to many open files" with this approach, or I'm missing something?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Do you mean because of spilling in general or the choice to spill each batch to its own file? In general DataFusion doesn't track number of files. I'm not sure if we keep the files open between writing and reading, I'd guess not. If we close them we only need as many file descriptors as files we spill concurrently - which should not be many (~ number of partitions)

Copy link
Contributor

Choose a reason for hiding this comment

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

yes, having a file per batch can get process to get killed (by the os) due to too many open files, also there are like 4 system calls per batch involved.

closing and then opening files for will bring at least 6 system calls per batch.

Ideally if we can keep file open and then share offsets after write, would save lot of syscals, plus we could keep a file per partition. 🤔

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, having a file per batch can get process to get killed (by the os) due to too many open files

I checked and indeed we close the files after writing so this is not going to happen. the only thing we accumulate is PathBufs, not open file descriptors.

closing and then opening files for will bring at least 6 system calls per batch
Ideally if we can keep file open and then share offsets after write, would save lot of syscals

I think that would be nice but I can't think of a scheme that would make that work: reading from the end of the file isn't compatible with our use case because we need it to be FIFO but using a single file only really works if you do LIFO. Do you have any ideas on how we could do this in a relatively simple way?

Given that any query that would spill here would have previously errored out I'm not too worried about performance. And frankly I think if batch sizes are reasonable a couple extra sys calls won't be measurable.

That said since how the spilling happens is all very internal / private there's no reason we can't merge this and then come back and improve it later.

Copy link
Contributor

Choose a reason for hiding this comment

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

Why would file be LIFO if you send offset, I'm not sure I understand. Writing single batch to a file,opening closing still seams excessive to me

Copy link
Contributor

@milenkovicm milenkovicm Oct 11, 2025

Choose a reason for hiding this comment

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

Shuffle exchange in spark or ballista will spill whole repartition to disk if I'm not mistaken I guess spilling in this case would not be different, apart from fact that this is triggered due to memory constraints.

Maybe we could learn something from new real-time mode in spark https://docs.google.com/document/d/1CvJvtlTGP6TwQIT4kW6GFT1JbdziAYOBvt60ybb7Dw8/edit?usp=sharing

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

One alternative could be to do a spill file per-channel and have some sort of gc process where we say "if the spill file exceeds XGB and/or we have more than YGB of junk space we pay the price of copying over into a new spill file to keep disk usage from blowing up". That might be a general approach to handle cases like #18011 as well. But I'm not sure the complexity is worth it. If that happens even once I feel that it will vastly exceed the cost of the extra sys calls to create and delete files.

Copy link
Contributor

Choose a reason for hiding this comment

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

Just one idea, would it be possible to spill before actual repartition? Try to acquire memory equal to received batch, if it fails do not do repartition, spill it. That would produce just N file (where N is number of partition)

Copy link
Contributor

Choose a reason for hiding this comment

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

One alternative could be to do a spill file per-channel and have some sort of gc process where we say "if the spill file exceeds XGB and/or we have more than YGB of junk space we pay the price of copying over into a new spill file to keep disk usage from blowing up".

You could avoid a copying gc process maybe via:

  1. Keep a LIFO list of spill files per channel
  2. write new batches to the end of the most recent file until its size exceeds some threshold (e.g, 100MB). When the threshold is exceed make a new file
  3. Read from the oldest file until all batches have been read. Once all batches have been read from a file it can be deleted.

@adriangb
Copy link
Contributor Author

Should we get this PR as draft for now before spilling tests added?

Sure thing. Note that there are tests, just not e2e SLT tests.

@adriangb
Copy link
Contributor Author

Should we get this PR as draft for now before spilling tests added?

Sure thing. Note that there are tests, just not e2e SLT tests.

Not sure how to do it from mobile. Will do next time I'm at my computer.

@adriangb adriangb marked this pull request as draft October 10, 2025 22:10
@github-actions github-actions bot added the core Core DataFusion crate label Oct 11, 2025
@adriangb
Copy link
Contributor Author

@adriangb adriangb marked this pull request as ready for review October 13, 2025 15:34
@adriangb adriangb force-pushed the spilling-repartition branch 2 times, most recently from 4091979 to bcfd64a Compare October 15, 2025 11:20
@2010YOUY01
Copy link
Contributor

I have a question: are we assuming that it's not possible to make RepartitionExec memory-constant (i.e., O(n_partitions * batch_size) memory)? Is this due to an engineering limitation, or is it theoretically impossible?
If we can make RepartitionExec only use constant memory, is this spilling RepartiitonExec still needed for datafusion-distributed usage?

@adriangb
Copy link
Contributor Author

I have a question: are we assuming that it's not possible to make RepartitionExec memory-constant (i.e., O(n_partitions * batch_size) memory)? Is this due to an engineering limitation, or is it theoretically impossible?

If we can make RepartitionExec only use constant memory, is this spilling RepartiitonExec still needed for datafusion-distributed usage?

I think it's theoretically impossible. If one partition is faster than another the data flowing into the slow partition has to accumulate somewhere. I consider the possibility of making the channels bounded, but I'm afraid that would result in deadlocks.

@gabotechs
Copy link
Contributor

Not really related to this PR specifically, but here are my two cents about some things that could be happening:

I ran into this using datafusion-distributed which I think makes the issue of partition execution time skew even more likely to happen

Not completely sure, but I'd say it should not be making things much worst. One thing that datafusion-distributed does is artificially scaling up the output partitions of RepartitionExec nodes, as it's a core piece for performing network shuffles. This means that you can end up in situations with a RepartitionExec with just 8 input partitions but 1000+ output partitions. Not sure how that affects this problem, but leaving the info here in case someone finds it relevant.

So what I did was at least make queries that would have previously fail continue forward with disk spilling

Note that currently DataFusion is capable of severely over-accounting memory under certain situations (more info in #16841). So even if this PR has value itself, It'd be worth to double check you are not hitting one of those situations.

@adriangb
Copy link
Contributor Author

Not completely sure, but I'd say it should not be making things much worst

Yeah I'm not sure either, I was just mentioning that in passing, but since there are other examples (#17334 (comment)) of this happening in plain DataFusion it made sense to me to implement this, especially since the only cases that will hit the spilling are cases that would have failed previously.

@alamb
Copy link
Contributor

alamb commented Oct 21, 2025

🤖 ./gh_compare_branch.sh Benchmark Script Running
Linux aal-dev 6.14.0-1017-gcp #18~24.04.1-Ubuntu SMP Tue Sep 23 17:51:44 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Comparing spilling-repartition (bcfd64a) to 057583d diff using: tpch_mem clickbench_partitioned clickbench_extended
Results will be posted here when complete

Copy link
Contributor

@alamb alamb 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 @adriangb for this contribution

I agree this is better than what is on main as some queries that would have straightup failed will now have a chance of running.

As long as the benchmark runs don't show any performance regressions I think we could proceed with merging this PR in.

The only thing I think needs to be done is to restore the oom test (though I do think we could do it as a follow on PR)

That being said, I also agree with @milenkovicm, @gabotechs and @Omega359 that using a single file for each spilled RecordBatch is likely going to be wildly inefficient (and really slow).

What I suggest is:

  1. Merge this PR
  2. File a follow on ticket describing a more sophisticated algorithm for reading/writing spill files to reduce the syscall overhead, (e.g. writing multiple batches to the same file when possible, etc) leaving a ticket reference in the comments
  3. Maybe add a logging message or some other hint when the spilling happens to help debugging

Then if/when we see queries fail / have bad performance due to this per-batch spilling, we can base the follow on performance enhancement work on real-world experiences / actual usecases.

cc @crepererum as you are familar with this work

.create_logical_plan("SELECT c1, count(*) as c FROM t GROUP BY c1;")
.await
.unwrap();
let plan = ctx.state().create_physical_plan(&plan).await.unwrap();
Copy link
Contributor

Choose a reason for hiding this comment

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

I recommend also verifying here that there are exactly two partitions in the final plan

Suggested change
let plan = ctx.state().create_physical_plan(&plan).await.unwrap();
let plan = ctx.state().create_physical_plan(&plan).await.unwrap();
assert_eq!(plan.output_partitioning().partition_count(), 2);

/// Just as some examples of real world scenarios where this can happen consider
/// lopsided groups in a group by especially if one partitions spills and others don't,
/// or in distributed systems if one upstream node is slower than others.
#[tokio::test]
Copy link
Contributor

Choose a reason for hiding this comment

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

This is a neat test. However, for some reason it takes over 5 seconds to run on laptop.

andrewlamb@Andrews-MacBook-Pro-3:~/Software/datafusion$ cargo test --test core_integration -- repartition_memory
    Finished `test` profile [unoptimized + debuginfo] target(s) in 0.21s
     Running tests/core_integration.rs (target/debug/deps/core_integration-97767bbb2ea8803a)

running 1 test
test memory_limit::repartition_mem_limit::test_repartition_memory_limit ... ok

test result: ok. 1 passed; 0 failed; 0 ignored; 0 measured; 704 filtered out; finished in 4.99s. <--- 5 seconds!!!

I suspect it is because it is creating something like 1M / 32 = 31,250 files

Is there any way to we can reduce the runtime (like maybe only use 1000 rows, or 10,000 rows?)

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, takes 0.62s on my laptop now

SharedMemoryReservation,
),
>,
channels: HashMap<usize, PartitionChannels>,
Copy link
Contributor

Choose a reason for hiding this comment

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

this is much nicer

(RepartitionBatch::Memory(batch), true)
}
Err(_) => {
// We're memory limited - spill this single batch to its own file
Copy link
Contributor

Choose a reason for hiding this comment

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

One alternative could be to do a spill file per-channel and have some sort of gc process where we say "if the spill file exceeds XGB and/or we have more than YGB of junk space we pay the price of copying over into a new spill file to keep disk usage from blowing up".

You could avoid a copying gc process maybe via:

  1. Keep a LIFO list of spill files per channel
  2. write new batches to the end of the most recent file until its size exceeds some threshold (e.g, 100MB). When the threshold is exceed make a new file
  3. Read from the oldest file until all batches have been read. Once all batches have been read from a file it can be deleted.

match &mut self.state {
RepartitionStreamState::ReceivingFromChannel => {
match self.input.recv().poll_unpin(cx) {
Poll::Ready(Some(Some(v))) => {
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 you could reduce the level of indenting here using the ready! macro which would make it more readable

match &mut self.state {
RepartitionStreamState::ReceivingFromChannel => {
match self.receiver.recv().poll_unpin(cx) {
Poll::Ready(Some(Some(v))) => {
Copy link
Contributor

Choose a reason for hiding this comment

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

same comment here about using ready! macro to simpify this loop

}

#[tokio::test]
async fn oom() -> 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 think it is important to leave the oom test here to verify that the system will still get an OOM error when no disk manager is enabled

@alamb
Copy link
Contributor

alamb commented Oct 21, 2025

🤖: Benchmark completed

Details

Comparing HEAD and spilling-repartition
--------------------
Benchmark clickbench_extended.json
--------------------
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓
┃ Query        ┃        HEAD ┃ spilling-repartition ┃       Change ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━┩
│ QQuery 0     │  2642.35 ms │           2669.29 ms │    no change │
│ QQuery 1     │  1224.23 ms │           1326.10 ms │ 1.08x slower │
│ QQuery 2     │  2327.52 ms │           2469.25 ms │ 1.06x slower │
│ QQuery 3     │  1200.37 ms │           1163.42 ms │    no change │
│ QQuery 4     │  2246.59 ms │           2211.98 ms │    no change │
│ QQuery 5     │ 27237.88 ms │          27700.81 ms │    no change │
│ QQuery 6     │  4215.97 ms │           4242.02 ms │    no change │
│ QQuery 7     │  3615.59 ms │           3551.18 ms │    no change │
└──────────────┴─────────────┴──────────────────────┴──────────────┘
┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━┓
┃ Benchmark Summary                   ┃            ┃
┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━┩
│ Total Time (HEAD)                   │ 44710.49ms │
│ Total Time (spilling-repartition)   │ 45334.05ms │
│ Average Time (HEAD)                 │  5588.81ms │
│ Average Time (spilling-repartition) │  5666.76ms │
│ Queries Faster                      │          0 │
│ Queries Slower                      │          2 │
│ Queries with No Change              │          6 │
│ Queries with Failure                │          0 │
└─────────────────────────────────────┴────────────┘
--------------------
Benchmark clickbench_partitioned.json
--------------------
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓
┃ Query        ┃        HEAD ┃ spilling-repartition ┃        Change ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩
│ QQuery 0     │     2.18 ms │              2.24 ms │     no change │
│ QQuery 1     │    49.15 ms │             47.54 ms │     no change │
│ QQuery 2     │   136.09 ms │            137.36 ms │     no change │
│ QQuery 3     │   160.09 ms │            163.56 ms │     no change │
│ QQuery 4     │  1079.58 ms │           1077.78 ms │     no change │
│ QQuery 5     │  1456.99 ms │           1522.66 ms │     no change │
│ QQuery 6     │     2.11 ms │              2.18 ms │     no change │
│ QQuery 7     │    51.67 ms │             53.35 ms │     no change │
│ QQuery 8     │  1401.62 ms │           1475.13 ms │  1.05x slower │
│ QQuery 9     │  1827.62 ms │           1836.21 ms │     no change │
│ QQuery 10    │   382.33 ms │            391.52 ms │     no change │
│ QQuery 11    │   422.77 ms │            443.64 ms │     no change │
│ QQuery 12    │  1319.84 ms │           1378.26 ms │     no change │
│ QQuery 13    │  2058.83 ms │           2135.95 ms │     no change │
│ QQuery 14    │  1231.47 ms │           1291.21 ms │     no change │
│ QQuery 15    │  1204.01 ms │           1231.39 ms │     no change │
│ QQuery 16    │  2613.88 ms │           2669.77 ms │     no change │
│ QQuery 17    │  2586.36 ms │           2680.52 ms │     no change │
│ QQuery 18    │  5629.22 ms │           5023.00 ms │ +1.12x faster │
│ QQuery 19    │   127.49 ms │            127.07 ms │     no change │
│ QQuery 20    │  2038.41 ms │           1977.80 ms │     no change │
│ QQuery 21    │  2362.49 ms │           2317.31 ms │     no change │
│ QQuery 22    │  7898.73 ms │           3955.36 ms │ +2.00x faster │
│ QQuery 23    │ 26312.62 ms │          12865.24 ms │ +2.05x faster │
│ QQuery 24    │   243.73 ms │            213.34 ms │ +1.14x faster │
│ QQuery 25    │   540.00 ms │            516.48 ms │     no change │
│ QQuery 26    │   220.13 ms │            216.07 ms │     no change │
│ QQuery 27    │  2840.38 ms │           2773.85 ms │     no change │
│ QQuery 28    │ 23282.25 ms │          22925.11 ms │     no change │
│ QQuery 29    │   983.09 ms │            982.21 ms │     no change │
│ QQuery 30    │  1330.95 ms │           1332.26 ms │     no change │
│ QQuery 31    │  1343.99 ms │           1322.08 ms │     no change │
│ QQuery 32    │  4639.67 ms │           4464.29 ms │     no change │
│ QQuery 33    │  5829.41 ms │           5633.71 ms │     no change │
│ QQuery 34    │  6075.39 ms │           5870.28 ms │     no change │
│ QQuery 35    │  2047.39 ms │           2015.47 ms │     no change │
│ QQuery 36    │   123.25 ms │            120.97 ms │     no change │
│ QQuery 37    │    53.49 ms │             53.30 ms │     no change │
│ QQuery 38    │   122.15 ms │            122.14 ms │     no change │
│ QQuery 39    │   196.89 ms │            194.44 ms │     no change │
│ QQuery 40    │    43.77 ms │             44.78 ms │     no change │
│ QQuery 41    │    38.72 ms │             41.19 ms │  1.06x slower │
│ QQuery 42    │    33.55 ms │             33.82 ms │     no change │
└──────────────┴─────────────┴──────────────────────┴───────────────┘
┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓
┃ Benchmark Summary                   ┃             ┃
┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩
│ Total Time (HEAD)                   │ 112343.76ms │
│ Total Time (spilling-repartition)   │  93681.84ms │
│ Average Time (HEAD)                 │   2612.65ms │
│ Average Time (spilling-repartition) │   2178.65ms │
│ Queries Faster                      │           4 │
│ Queries Slower                      │           2 │
│ Queries with No Change              │          37 │
│ Queries with Failure                │           0 │
└─────────────────────────────────────┴─────────────┘
--------------------
Benchmark tpch_mem_sf1.json
--------------------
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓
┃ Query        ┃      HEAD ┃ spilling-repartition ┃        Change ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩
│ QQuery 1     │ 177.84 ms │            166.09 ms │ +1.07x faster │
│ QQuery 2     │  26.78 ms │             24.65 ms │ +1.09x faster │
│ QQuery 3     │  39.64 ms │             36.47 ms │ +1.09x faster │
│ QQuery 4     │  28.28 ms │             28.14 ms │     no change │
│ QQuery 5     │  76.71 ms │             76.88 ms │     no change │
│ QQuery 6     │  19.70 ms │             19.64 ms │     no change │
│ QQuery 7     │ 206.92 ms │            210.25 ms │     no change │
│ QQuery 8     │  33.79 ms │             33.77 ms │     no change │
│ QQuery 9     │  99.44 ms │            100.81 ms │     no change │
│ QQuery 10    │  57.68 ms │             58.81 ms │     no change │
│ QQuery 11    │  17.27 ms │             17.21 ms │     no change │
│ QQuery 12    │  50.13 ms │             50.67 ms │     no change │
│ QQuery 13    │  45.89 ms │             46.56 ms │     no change │
│ QQuery 14    │  13.83 ms │             13.46 ms │     no change │
│ QQuery 15    │  24.79 ms │             24.28 ms │     no change │
│ QQuery 16    │  24.12 ms │             24.89 ms │     no change │
│ QQuery 17    │ 142.91 ms │            145.70 ms │     no change │
│ QQuery 18    │ 322.15 ms │            320.72 ms │     no change │
│ QQuery 19    │  36.21 ms │             36.28 ms │     no change │
│ QQuery 20    │  47.35 ms │             47.16 ms │     no change │
│ QQuery 21    │ 295.10 ms │            312.34 ms │  1.06x slower │
│ QQuery 22    │  20.80 ms │             20.50 ms │     no change │
└──────────────┴───────────┴──────────────────────┴───────────────┘
┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┓
┃ Benchmark Summary                   ┃           ┃
┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━┩
│ Total Time (HEAD)                   │ 1807.34ms │
│ Total Time (spilling-repartition)   │ 1815.29ms │
│ Average Time (HEAD)                 │   82.15ms │
│ Average Time (spilling-repartition) │   82.51ms │
│ Queries Faster                      │         3 │
│ Queries Slower                      │         1 │
│ Queries with No Change              │        18 │
│ Queries with Failure                │         0 │
└─────────────────────────────────────┴───────────┘

@adriangb adriangb force-pushed the spilling-repartition branch from bcfd64a to 8f81a36 Compare October 21, 2025 14:48
@adriangb
Copy link
Contributor Author

QQuery 23 │ 26312.62 ms │ 12865.24 ms │ +2.05x faster

Suspicious 🤔

@adriangb
Copy link
Contributor Author

adriangb commented Oct 21, 2025

What I suggest is:

  1. Merge this PR
  2. File a follow on ticket describing a more sophisticated algorithm for reading/writing spill files to reduce the syscall overhead, (e.g. writing multiple batches to the same file when possible, etc) leaving a ticket reference in the comments
  3. Maybe add a logging message or some other hint when the spilling happens to help debugging

I have a followup ready using the approach suggested in #18014 (comment): pydantic#40

I'll go ahead and merge this PR to avoid needing re-review and we can continue in that one.

@adriangb adriangb added this pull request to the merge queue Oct 21, 2025
Merged via the queue into apache:main with commit 1e30aed Oct 21, 2025
32 checks passed
@adriangb adriangb deleted the spilling-repartition branch October 21, 2025 18:16
Ok(())
}

#[tokio::test]
Copy link
Contributor

Choose a reason for hiding this comment

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

❤️

adriangb added a commit to pydantic/datafusion that referenced this pull request Oct 27, 2025
Addresses
apache#17334 (comment)

I ran into this using `datafusion-distributed` which I think makes the
issue of partition execution time skew even more likely to happen. As
per that issue it can also happen with non-distributed queries, e.g. if
one partition's sort spills and others don't.

Due to the nature of `ReparitionExec` I don't think we can bound the
channels, that could lead to deadlocks. So what I did was at least make
queries that would have previously fail continue forward with disk
spilling. I did not account for memory usage when reading batches back
from disk since DataFusion in general does not generally account for
"in-flight" batches.

Written with help from Claude

---------

Co-authored-by: Bruce Ritchie <bruce.ritchie@veeva.com>
tobixdev pushed a commit to tobixdev/datafusion that referenced this pull request Nov 2, 2025
Addresses
apache#17334 (comment)

I ran into this using `datafusion-distributed` which I think makes the
issue of partition execution time skew even more likely to happen. As
per that issue it can also happen with non-distributed queries, e.g. if
one partition's sort spills and others don't.

Due to the nature of `ReparitionExec` I don't think we can bound the
channels, that could lead to deadlocks. So what I did was at least make
queries that would have previously fail continue forward with disk
spilling. I did not account for memory usage when reading batches back
from disk since DataFusion in general does not generally account for
"in-flight" batches.

Written with help from Claude

---------

Co-authored-by: Bruce Ritchie <bruce.ritchie@veeva.com>
github-merge-queue bot pushed a commit that referenced this pull request Nov 7, 2025
Addresses
#18014 (comment),
potentially paves the path to solve
#18011 for other operators as
well

---------

Co-authored-by: Yongting You <2010youy01@gmail.com>
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
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>
EeshanBembi pushed a commit to EeshanBembi/datafusion that referenced this pull request Nov 24, 2025
Addresses
apache#17334 (comment)

I ran into this using `datafusion-distributed` which I think makes the
issue of partition execution time skew even more likely to happen. As
per that issue it can also happen with non-distributed queries, e.g. if
one partition's sort spills and others don't.

Due to the nature of `ReparitionExec` I don't think we can bound the
channels, that could lead to deadlocks. So what I did was at least make
queries that would have previously fail continue forward with disk
spilling. I did not account for memory usage when reading batches back
from disk since DataFusion in general does not generally account for
"in-flight" batches.

Written with help from Claude

---------

Co-authored-by: Bruce Ritchie <bruce.ritchie@veeva.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Core DataFusion crate physical-plan Changes to the physical-plan crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants