Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

External sorting not working for (maybe only for string columns??) #12136

Open
Tracked by #14077
alamb opened this issue Aug 23, 2024 · 19 comments
Open
Tracked by #14077

External sorting not working for (maybe only for string columns??) #12136

alamb opened this issue Aug 23, 2024 · 19 comments
Labels
bug Something isn't working help wanted Extra attention is needed

Comments

@alamb
Copy link
Contributor

alamb commented Aug 23, 2024

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

image

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.

image

@alamb alamb added the bug Something isn't working label Aug 23, 2024
@alamb alamb changed the title External sorting not working for string columns External sorting not working for (maybe only for string columns??) Aug 23, 2024
@alamb alamb added the help wanted Extra attention is needed label Aug 23, 2024
@cjolowicz
Copy link

cjolowicz commented Aug 30, 2024

This is probably unrelated, but there's something that puzzles me about the fair spill pool logic. The try_grow function checks that the consumer isn't allocating more than its fair share. So it needs to determine how much memory the consumer has already allocated. But it only looks at the reservation that's passed, not at all reservations of that consumer.

Consider a single spillable consumer that allocates the entire pool. It shouldn't be able to grow the resulting reservation. But if it splits the reservation into two, it can now grow it - even when the other reservation remains allocated.

Here's a failing test showing this:
https://github.com/apache/datafusion/pull/12170/files

---- memory_pool::pool::tests::test_fair_with_split_reservation stdout ----
thread 'memory_pool::pool::tests::test_fair_with_split_reservation' panicked at datafusion/execution/src/memory_pool/pool.rs:417:34:
called `Result::unwrap_err()` on an `Ok` value: ()

Let me know if I'm misunderstanding something here.

@cjolowicz
Copy link

I couldn't get the sort to complete because the pod froze when filesystem usage grew to 1TB.

I could reproduce the pod freezing up at 1TB disk usage. This happened within a minute of spawning 500 threads. All threads were blocked on a mutex (futex_wait) except one in epoll with an inode fd. Maybe deadlock at the start of the merge phase?

1TB seems a bit much for sorting a 100GiB partition.

It actually makes sense because spill files use Apache IPC format without compression, while the partition uses Parquet files with Snappy compression.

cjolowicz added a commit to cjolowicz/datafusion that referenced this issue Sep 2, 2024
This change replaces `try_resize` with `resize` in three sites, allowing memory
to overshoot the configured pool size. These are sites where we don't fall back
to spilling to disk when the allocation fails.

Fixes: apache#12136
cjolowicz added a commit to cjolowicz/datafusion that referenced this issue Sep 2, 2024
This change replaces `try_resize` with `resize` in three sites, allowing memory
to overshoot the configured pool size. These are sites where we don't fall back
to spilling to disk when the allocation fails.

Fixes: apache#12136
cjolowicz added a commit to cjolowicz/datafusion that referenced this issue Sep 2, 2024
This change replaces `try_resize` with `resize` in three sites, allowing memory
to overshoot the configured pool size. These are sites where we don't fall back
to spilling to disk when the allocation fails.

Fixes: apache#12136
cjolowicz added a commit to cjolowicz/datafusion that referenced this issue Sep 9, 2024
This change replaces `try_resize` with `resize` in three sites, allowing memory
to overshoot the configured pool size. These are sites where we don't fall back
to spilling to disk when the allocation fails.

Fixes: apache#12136
@16pierre
Copy link

16pierre commented Jan 5, 2025

Also hitting this bug, is there any update on a fix ?

@alamb
Copy link
Contributor Author

alamb commented Jan 5, 2025

Also hitting this bug, is there any update on a fix ?

I don't know of anyone working explicitly working to make external sorting better. Some recent work maybe would make it better

But realistically I think there is significant additional room for improvement. Any help welcome

@Omega359
Copy link
Contributor

Omega359 commented Jan 6, 2025

I don't believe that adding an option for compression in DiskManager would be all that difficult - the IPCWriter and reader already has support for LZ4. It would of course incur the cost of compression/decompression but would likely result in 50+% space savings on disk. It's a nice to have though - I think of it more as a band-aid than a solution.

@2010YOUY01
Copy link
Contributor

2010YOUY01 commented Jan 9, 2025

For the given reproducer, I got the error

Error: ResourcesExhausted("Failed to allocate additional 117568 bytes for ExternalSorter[0] with 0 bytes already allocated for this reservation - 0 bytes remain available for the total pool

Configure datafusion.execution.sort_spill_reservation_bytes to 1MB can let it run successfully. ( I don't know whether the parquet related error message is caused by the same issue)

// Reproducer: place in datafusion/core/tests/memory_limit/mod.rs
#[tokio::test]
async fn test_sort_with_memory_limit() -> Result<()> {
    // initialize logging to see DataFusion's internal logging
    let _ = env_logger::try_init();

    // how much data to sort
    let row_limit = 10 * 1000;
    let mem_limit = 10 * 1024 * 1024; // 10 MB
    let sort_spill_reservation_bytes = 1024 * 1024; // 1 MB

    let generator = AccessLogGenerator::new()
        .with_row_limit(row_limit)
        .with_max_batch_size(100); // 100 rows per batch

    let pool = Arc::new(GreedyMemoryPool::new(mem_limit));
    let runtime = RuntimeEnvBuilder::new()
        .with_memory_pool(pool)
        .with_disk_manager(DiskManagerConfig::new())
        .build()?;
    let session_config = SessionConfig::new()
        .with_sort_spill_reservation_bytes(sort_spill_reservation_bytes);
    let state = SessionStateBuilder::new()
        .with_config(session_config)
        .with_runtime_env(Arc::new(runtime))
        .build();

    let ctx = SessionContext::new_with_state(state);

    // 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?;

    Ok(())
}

Reasons:

  • How sort_spill_reservation_bytes works: For example there is a sort query with memory limit 10MB, and sort_spill_reservation_bytes is 1M, it will accumulate batches in memory until it reaches 9M, then do in-memory sort. The sort will do in-place sort for all individual batches, and finally do a sort-preserving merge to get the final one sorted run. 1MB is reserved for internal data structures of the merging phase.
  • The query in reproducer only has 10 MB memory budget, and all 10MB is reserved for later merge usage, then the execution can fail

Thoughts:
Note I'm not 100% sure about the implementation detail, if that's the case, I think we can get rid of this configuration option, and figure out sort_spill_reservation_bytes according to the available memory budget, to prevent similar failures.

@westonpace
Copy link
Member

@2010YOUY01 Your solution does not work for me. I did play around with sort_spill_reservation_bytes when I originally encountered this but couldn't find a way to get it to work. Here is a reproducer that uses a 100MB memory budget and fails for all values of sort_spill_reservation_bytes between 1 and 100 MB. It might be that I am configuring things or using something improperly however.

@xuchen-plus
Copy link
Contributor

xuchen-plus commented Jan 23, 2025

We have also encountered this issue. After some debugging (by adding debug logs before every call to try_grow), it seems that the memory counting of batches returned by SortPreservingMergeStream is wrong.

I set sort_spill_reservation_bytes to 8MB, and memory_limit to 256MB.

After insert some batches, the pool is full and start to in-memory sort, at this moment the pool usage is: GreedyMemoryPool { pool_size: 268435456, used: 267618380 }.
And the sort and collect at

self.in_mem_batches = self
.in_mem_sort_stream(self.metrics.baseline.intermediate())?
.try_collect()
.await?;

finished without error. After sort, all memory are released.

Then the following code count the sorted batches' memory:

let size: usize = self
.in_mem_batches
.iter()
.map(get_record_batch_memory_size)
.sum();

And this size is: 698840964, which exceeds the memory limit and the following self.reservation.try_resize(size)?; failed:

Error: ResourcesExhausted("Additional allocation failed with top memory consumers (across reservations) as: ExternalSorterMerge[0] consumed 33554432 bytes, ExternalSorterMerge[18] consumed 0 bytes, ExternalSorter[7] consumed 0 bytes, ExternalSorter[14] consumed 0 bytes, ExternalSorter[1] consumed 0 bytes. Error: Failed to allocate additional 698840964 bytes for ExternalSorter[0] with 0 bytes already allocated for this reservation - 234881024 bytes remain available for the total pool")

Not sure why the sorted batches' memory is over 2.6x than the batches before sort.

@xuchen-plus
Copy link
Contributor

xuchen-plus commented Feb 7, 2025

Not sure why the sorted batches' memory is over 2.6x than the batches before sort.

Some findings so far:

  1. My test was reading a parquet file with mostly string columns and sort by one column. It seems that enabling string view would cause get_record_batch_memory_size produce overlarge values since multiple string view columns from multiple batches may share the same buffer.
    a) The memory counting at insert_batch may be larger than actual physical memory usage:

    async fn insert_batch(&mut self, input: RecordBatch) -> Result<()> {
    if input.num_rows() == 0 {
    return Ok(());
    }
    self.reserve_memory_for_merge()?;
    let size = get_record_batch_memory_size(&input);

    b) The memory counting is also too large after in_mem_sort:
    let size: usize = self
    .in_mem_batches
    .iter()
    .map(get_record_batch_memory_size)
    .sum();

  2. I disabled string view for parquet reader, and increase the value of sort_spill_reservation_bytes, my test can be finished successfully, during which the disk spill works correctly. Some observations
    a) sort_spill_reservation_bytes should be set to a big enough value to hold:

    • Extra memory required during sorting each in memory batch. The memory consumption for the sorted batch could be larger than the original.
    • Extra memory for allocating Rows for SortPreservingMergeStream.

    b) The memory consumption in

    self.merge_reservation.free();
    self.in_mem_batches = self
    .in_mem_sort_stream(self.metrics.baseline.intermediate())?
    .try_collect()
    .await?;

    is not counted in the memory limit. During the collect, the sorted batches will be interleaved to produce merged batches by BatchBuilder. At this time both the sorted batches and the interleaved batches exist in memory and the memory consumption may be doubled.

@xuchen-plus
Copy link
Contributor

@2010YOUY01 Your solution does not work for me. I did play around with sort_spill_reservation_bytes when I originally encountered this but couldn't find a way to get it to work. Here is a reproducer that uses a 100MB memory budget and fails for all values of sort_spill_reservation_bytes between 1 and 100 MB. It might be that I am configuring things or using something improperly however.

This reproducer can also run successfully by disabling string view via session_config.options_mut().execution.parquet.schema_force_view_types = false; and set sort_spill_reservation_bytes to 16MB.

@zhuqi-lucas
Copy link
Contributor

zhuqi-lucas commented Feb 8, 2025

Thank you @xuchen-plus , so we disable the stringView it will succeed when we increase the sort_spill_reservation_bytes. If so we need to investigate how to fix stringView case?

@zhuqi-lucas
Copy link
Contributor

zhuqi-lucas commented Feb 8, 2025

@2010YOUY01 Your solution does not work for me. I did play around with sort_spill_reservation_bytes when I originally encountered this but couldn't find a way to get it to work. Here is a reproducer that uses a 100MB memory budget and fails for all values of sort_spill_reservation_bytes between 1 and 100 MB. It might be that I am configuring things or using something improperly however.

This reproducer can also run successfully by disabling string view via session_config.options_mut().execution.parquet.schema_force_view_types = false; and set sort_spill_reservation_bytes to 16MB.

It still failed for me for the setting, the reproduce code:

use arrow::array::{RecordBatch, StringBuilder};
use arrow_schema::{DataType, Field, Schema};
use datafusion::execution::disk_manager::DiskManagerConfig;
use datafusion::execution::memory_pool::FairSpillPool;
use datafusion::execution::runtime_env::RuntimeEnvBuilder;
use datafusion::parquet::arrow::ArrowWriter;
use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext};
use futures::TryStreamExt;
use std::sync::Arc;

#[tokio::main(flavor = "multi_thread", worker_threads = 1)]
pub async fn main() {
    build_parquet();

    let env = RuntimeEnvBuilder::new()
        .with_disk_manager(DiskManagerConfig::default())
        .with_memory_pool(Arc::new(FairSpillPool::new(100 * 1024 * 1024)))
        .build_arc()
        .unwrap();

    let mut config = SessionConfig::new().with_sort_spill_reservation_bytes(16 * 1024 * 1024);
    config.options_mut().execution.parquet.schema_force_view_types = false;

    let ctx = SessionContext::new_with_config_rt(config, env);

    ctx.register_parquet(
        "big_strings",
        "/tmp/big_strings.parquet",
        ParquetReadOptions::default(),
    )
    .await
    .unwrap();

    let sql = "SELECT * FROM big_strings ORDER BY strings";
    println!("Sorting strings");
    ctx.sql(sql)
        .await
        .unwrap()
        .execute_stream()
        .await
        .unwrap()
        .try_for_each(|_| std::future::ready(Ok(())))
        .await
        .unwrap();
}

fn build_parquet() {
    if std::fs::File::open("/tmp/big_strings.parquet").is_ok() {
        println!("Using existing file at /tmp/big_strings.parquet");
        return;
    }
    println!("Generating test file at /tmp/big_strings.parquet");
    let file = std::fs::File::create("/tmp/big_strings.parquet").unwrap();
    let schema = Arc::new(Schema::new(vec![Field::new(
        "strings",
        DataType::Utf8,
        false,
    )]));
    let mut writer = ArrowWriter::try_new(file, schema.clone(), None).unwrap();

    for batch_idx in 0..100 {
        println!("Generating batch {} of 100", batch_idx);
        let mut string_array_builder =
            StringBuilder::with_capacity(1024 * 1024, 1024 * 1024 * 3 * 14);
        for i in 0..(1024 * 1024) {
            string_array_builder
                .append_value(format!("string-{}string-{}string-{}", i, i, i));
        }
        let array = Arc::new(string_array_builder.finish());
        let batch = RecordBatch::try_new(schema.clone(), vec![array]).unwrap();
        writer.write(&batch).unwrap();
    }
    writer.close().unwrap();
}

And when disable stringView, a new finding is the sorted data is smaller than the original data:

Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data

Failed with error:

Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
thread 'main' panicked at datafusion-examples/examples/large_string_sort.rs:62:10:
called `Result::unwrap()` on an `Err` value: ResourcesExhausted("Failed to allocate additional 353536 bytes for ExternalSorter[1] with 7138944 bytes already allocated for this reservation - 7489828 bytes remain available for the total pool")

@xuchen-plus
Copy link
Contributor

Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Failed with error:

Before sorting we have 6233344 bytes of unsorted data
After sorting we now have 4932480 bytes of sorted data
Spilling
Before sorting we have 6233344 bytes of unsorted data
thread 'main' panicked at datafusion-examples/examples/large_string_sort.rs:62:10:
called Result::unwrap() on an Err value: ResourcesExhausted("Failed to allocate additional 353536 bytes for ExternalSorter[1] with 7138944 bytes already allocated for this reservation - 7489828 bytes remain available for the total pool")

@zhuqi-lucas Sorry. I actually adjusted sort_spill_reservation_bytes to 32MB and the above test passed. When sort_spill_reservation_bytes is 16MB the test would failed to reserve extra memory during sort.

@zhuqi-lucas
Copy link
Contributor

zhuqi-lucas commented Feb 13, 2025

Thanks @xuchen-plus , unluckily after change to 32MB for sort_spill_reservation_bytes, it still failed for me, i am not sure which i am missing.

use arrow::array::{RecordBatch, StringBuilder};
use arrow_schema::{DataType, Field, Schema};
use datafusion::execution::disk_manager::DiskManagerConfig;
use datafusion::execution::memory_pool::FairSpillPool;
use datafusion::execution::runtime_env::RuntimeEnvBuilder;
use datafusion::parquet::arrow::ArrowWriter;
use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext};
use futures::TryStreamExt;
use std::sync::Arc;

#[tokio::main(flavor = "multi_thread", worker_threads = 1)]
pub async fn main() {
    build_parquet();

    let env = RuntimeEnvBuilder::new()
        .with_disk_manager(DiskManagerConfig::default())
        .with_memory_pool(Arc::new(FairSpillPool::new(100 * 1024 * 1024)))
        .build_arc()
        .unwrap();

    let mut config = SessionConfig::new().with_sort_spill_reservation_bytes(32 * 1024 * 1024);
    config.options_mut().execution.parquet.schema_force_view_types = false;

    let ctx = SessionContext::new_with_config_rt(config, env);

    ctx.register_parquet(
        "big_strings",
        "/tmp/big_strings.parquet",
        ParquetReadOptions::default(),
    )
        .await
        .unwrap();

    let sql = "SELECT * FROM big_strings ORDER BY strings";
    println!("Sorting strings");
    ctx.sql(sql)
        .await
        .unwrap()
        .execute_stream()
        .await
        .unwrap()
        .try_for_each(|_| std::future::ready(Ok(())))
        .await
        .unwrap();
}

fn build_parquet() {
    if std::fs::File::open("/tmp/big_strings.parquet").is_ok() {
        println!("Using existing file at /tmp/big_strings.parquet");
        return;
    }
    println!("Generating test file at /tmp/big_strings.parquet");
    let file = std::fs::File::create("/tmp/big_strings.parquet").unwrap();
    let schema = Arc::new(Schema::new(vec![Field::new(
        "strings",
        DataType::Utf8,
        false,
    )]));
    let mut writer = ArrowWriter::try_new(file, schema.clone(), None).unwrap();

    for batch_idx in 0..100 {
        println!("Generating batch {} of 100", batch_idx);
        let mut string_array_builder =
            StringBuilder::with_capacity(1024 * 1024, 1024 * 1024 * 3 * 14);
        for i in 0..(1024 * 1024) {
            string_array_builder
                .append_value(format!("string-{}string-{}string-{}", i, i, i));
        }
        let array = Arc::new(string_array_builder.finish());
        let batch = RecordBatch::try_new(schema.clone(), vec![array]).unwrap();
        writer.write(&batch).unwrap();
    }
    writer.close().unwrap();
}
called `Result::unwrap()` on an `Err` value: ResourcesExhausted("Failed to allocate additional 3875520 bytes for ExternalSorter[2] with 0 bytes already allocated for this reservation - 2696338 bytes remain available for the total pool")

@xuchen-plus
Copy link
Contributor

xuchen-plus commented Feb 13, 2025

Thanks @xuchen-plus , unluckily after change to 32MB for sort_spill_reservation_bytes, it still failed for me, i am not sure which i am missing.

@zhuqi-lucas I was using main branch at commit id 55730dcecebe5475af40b9e8a5c4805a73c31b11. I tested released version 45.0.0 and also failed with the same error like yours.

@2010YOUY01
Copy link
Contributor

I haven't followed the discussion yet, but to make sure the external execution results are reproducible, datafusion.execution.target_partitions should also be set to a constant number. The default value is the number of CPU cores on the running machine, which can vary

@Kontinuation
Copy link
Member

Kontinuation commented Feb 13, 2025

I have also encountered the same problem with string views.

DataFusion uses interleave function to produce merged batches, and interleave tends to produce batches that has super large size due to apache/arrow-rs#6779. Although it simply references to the data buffers of interleaved arrays so it does not actually take extra memory space, but it makes the result of get_record_batch_memory_size(batch) or batch.get_array_memory_size() super large, increasing the chance of getting memory reservation failures.

When spilling happens, these interleaved arrays will be serialized using Arrow IPC and produces very large binaries. When we read them back in spill-read phase, we have to allocate super large buffers for these arrays, which makes things much worse.

@alamb
Copy link
Contributor Author

alamb commented Feb 13, 2025

I have also encountered the same problem with string views.

DataFusion uses interleave function to produce merged batches, and interleave tends to produce batches that has super large size due to apache/arrow-rs#6779. Although it simply references to the data buffers of interleaved arrays so it does not actually take extra memory space, but it makes the result of get_record_batch_memory_size(batch) or batch.get_array_memory_size() super large, increasing the chance of getting memory reservation failures.

When spilling happens, these interleaved arrays will be serialized using Arrow IPC and produces very large binaries. When we read them back in spill-read phase, we have to allocate super large buffers for these arrays, which makes things much worse.

I think the fix for apache/arrow-rs#6779 is in DataFusion 45 -- does this still happen?

@Kontinuation
Copy link
Member

Kontinuation commented Feb 13, 2025

I think the fix for apache/arrow-rs#6779 is in DataFusion 45 -- does this still happen?

I am using the latest main branch of DataFusion (commit 28856e1) and the problem still happens.

Edit: this can be easily reproduced using sort-tpch Q3. The following error message is obtained from a dfbench sort-tpch run with 200MB memory limit and 100MB sort_spill_reservation_bytes:

ExternalSorterMerge[0] consumed 104857600 bytes (100MB), ExternalSorter[0] consumed 0 bytes.
Error: Failed to allocate additional 2262379458 bytes (2.1GB) for ExternalSorter[0] with 0 bytes already allocated for this reservation - 104857600 bytes remain available for the total pool

We see 100MB in-memory batches got sorted and became 2.1GB.

Backtrace:

backtrace:    0: std::backtrace_rs::backtrace::libunwind::trace
             at /rustc/eeb90cda1969383f56a2637cbd3037bdf598841c/library/std/src/../../backtrace/src/backtrace/libunwind.rs:116:5
   1: std::backtrace_rs::backtrace::trace_unsynchronized
             at /rustc/eeb90cda1969383f56a2637cbd3037bdf598841c/library/std/src/../../backtrace/src/backtrace/mod.rs:66:5
   2: std::backtrace::Backtrace::create
             at /rustc/eeb90cda1969383f56a2637cbd3037bdf598841c/library/std/src/backtrace.rs:331:13
   3: datafusion_common::error::DataFusionError::get_back_trace
             at ./datafusion/common/src/error.rs:473:30
   4: datafusion_execution::memory_pool::pool::insufficient_capacity_err
             at ./datafusion/execution/src/memory_pool/pool.rs:249:5
   5: <datafusion_execution::memory_pool::pool::FairSpillPool as datafusion_execution::memory_pool::MemoryPool>::try_grow
             at ./datafusion/execution/src/memory_pool/pool.rs:206:32
   6: <datafusion_execution::memory_pool::pool::TrackConsumersPool<I> as datafusion_execution::memory_pool::MemoryPool>::try_grow
             at ./datafusion/execution/src/memory_pool/pool.rs:362:9
   7: datafusion_execution::memory_pool::MemoryReservation::try_grow
             at ./datafusion/execution/src/memory_pool/mod.rs:298:9
   8: datafusion_execution::memory_pool::MemoryReservation::try_resize
             at ./datafusion/execution/src/memory_pool/mod.rs:281:34
   9: datafusion_physical_plan::sorts::sort::ExternalSorter::in_mem_sort::{{closure}}
             at ./datafusion/physical-plan/src/sorts/sort.rs:453:9
  10: datafusion_physical_plan::sorts::sort::ExternalSorter::insert_batch::{{closure}}
             at ./datafusion/physical-plan/src/sorts/sort.rs:309:32
  11: <datafusion_physical_plan::sorts::sort::SortExec as datafusion_physical_plan::execution_plan::ExecutionPlan>::execute::{{closure}}
...

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working help wanted Extra attention is needed
Projects
None yet
Development

Successfully merging a pull request may close this issue.

9 participants