-
Notifications
You must be signed in to change notification settings - Fork 1.7k
Description
Describe the bug
Filing a ticket based on a conversation in discord: https://discord.com/channels/885562378132000778/1166447479609376850/1275728622224932959
Basically, I expect that when properly configured, DataFusion would be able to sort data that doesn't fit in RAM, but instead it results in an error like
failed to optimize table: Parquet { source: General("Z-order failed while scanning data: ResourcesExhausted(\"Failed to allocate additional 2348322032 bytes for ExternalSorter[0] with 0 bytes already allocated for this reservation - 1501294808 bytes remain available for the total pool\")") }
To Reproduce
Here is a reproducer: rust_playground.tar.gz
tar xf rust_playground.tar.gz
cd rust_playground/
cargo run
The code looks like this
// how much data to sort
let row_limit = 10 * 1000;
let mem_limit = 10 * 1024 * 1024;
let print_results = false;
let pool = FairSpillPool::new(mem_limit);
let config = RuntimeConfig::new()
.with_memory_pool(Arc::new(pool))
.with_disk_manager(DiskManagerConfig::new());
let runtime_env = RuntimeEnv::new(config)?;
let builder = SessionStateBuilder::new().with_runtime_env(Arc::new(runtime_env));
let ctx = SessionContext::new_with_state(builder.build());
let generator = AccessLogGenerator::new()
.with_row_limit(row_limit)
.with_max_batch_size(100); // 100 rows per batch
// create a plan that simply sorts on the hostname
let df = ctx
.read_batches(generator)?
.sort(vec![col("host").sort(true, true)])?;
// execute the plan (it should succeed)
let results: Vec<RecordBatch> = df.collect().await?;
Expected behavior
I expect the query to succeed (by spilling data to disk, etc)
Additional context
@westonpace notes #10073 may be related
Here is some of the commentary from discord:
I see four sites where an allocation failure doesn't lead to spilling:
self.reservation.try_grow(size)? |
self.reservation.try_resize(size)?; |
self.reservation.try_resize(batch.get_array_memory_size())?; |
self.reservation.try_grow(batch.get_array_memory_size())?; |
My main suspect is the fourth which is used by SortPreservingMergeStream when appending a batch, during the streaming merge of in-memory batches and spill files.
Guessed wrong. It's the second call site from the list above. Here's the relevant part of the backtrace:
0x559ea91a9cef - core::result::Result<T,E>::unwrap::h6b8b66539b551ac9
at /build/rust/tmp/build/rustc-1.80.0-src/library/core/src/result.rs:1102:23
13: 0x559ea91a9cef - datafusion_execution::memory_pool::MemoryReservation::try_grow::h6d435672ae31c4ca
at /home/claudio/Code/github.com/apache/datafusion/datafusion/execution/src/memory_pool/mod.rs:270:12
14: 0x559ea91a9cef - datafusion_execution::memory_pool::MemoryReservation::try_resize::h80744f5d7b3aa70f
at /home/claudio/Code/github.com/apache/datafusion/datafusion/execution/src/memory_pool/mod.rs:252:34
15: 0x559ea8b95ba9 - datafusion_physical_plan::sorts::sort::ExternalSorter::in_mem_sort::{{closure}}::h4686d81a505f0439
at /home/claudio/Code/github.com/apache/datafusion/datafusion/physical-plan/src/sorts/sort.rs:434:9
16: 0x559ea8b6fda0 - datafusion_physical_plan::sorts::sort::ExternalSorter::insert_batch::{{closure}}::hdd00ee915381b03a
at /home/claudio/Code/github.com/apache/datafusion/datafusion/physical-plan/src/sorts/sort.rs:289:32
17: 0x559ea8b6fda0 - <datafusion_physical_plan::sorts::sort::SortExec as datafusion_physical_plan::execution_plan::ExecutionPlan>::execute::{{closure}}::h256a01c17fe43713
at /home/claudio/Code/github.com/apache/datafusion/datafusion/physical-plan/src/sorts/sort.rs:910:52
I got this same backtrace in three worker threads at once:
thread 'tokio-runtime-worker' panicked at /home/claudio/Code/github.com/apache/datafusion/datafusion/execution/src/memory_pool/mod.rs:270:37:
called `Result::unwrap()` on an `Err` value: ResourcesExhausted("Failed to allocate additional 811511472 bytes for ExternalSorter[6] with 0 bytes already allocated for this reservation - 612881930 bytes remain available for the total pool")
stack backtrace:
thread 'tokio-runtime-worker' panicked at /home/claudio/Code/github.com/apache/datafusion/datafusion/execution/src/memory_pool/mod.rs:270:37:
called `Result::unwrap()` on an `Err` value: ResourcesExhausted("Failed to allocate additional 767287128 bytes for ExternalSorter[0] with 0 bytes already allocated for this reservation - 597992699 bytes remain available for the total pool")
thread 'tokio-runtime-worker' panicked at /home/claudio/Code/github.com/apache/datafusion/datafusion/execution/src/memory_pool/mod.rs:270:37:
called `Result::unwrap()` on an `Err` value: ResourcesExhausted("Failed to allocate additional 734693152 bytes for ExternalSorter[3] with 0 bytes already allocated for this reservation - 627555485 bytes remain available for the total pool")
Memory usage steadily grows to 1GiB until the crash but the pool size is 20 GiB
I've been able to avoid the allocation failures during sort with spill-to-disk by replacing try_grow with grow when we can't fall back to spilling. Specifically, I made that change in these places (all of which triggered allocation failures during earlier attempts):
self.reservation.try_resize(size)?; |
self.merge_reservation.try_resize(size)?; |
self.reservation.try_grow(batch.get_array_memory_size())?; |
The first two happen during in-memory sorting, for the spillable and unspillable reservations, respectively. The second one happens during the merge phase.
Memory slightly overshot the 20GiB pool size but stayed close initially. Eventually it grew to 30GiB (during the merge phase, maybe?).
I couldn't get the sort to complete because the pod froze when filesystem usage grew to 1TB. That seems a bit much for sorting a 100GiB partition, I'll be looking into that next.