apache / datafusion

Apache DataFusion SQL Query Engine
https://datafusion.apache.org/
Apache License 2.0
6k stars 1.14k forks source link

Data set which is much bigger than RAM #10897

Open Smotrov opened 3 months ago

Smotrov commented 3 months ago

I'm using Rust, meanwhile I'm new to DataFusion.

I need to repartition big dataset which is hundreds of GB. It is stored on S3 as multiple compressed packet files. It should be partitioned by the value of a column. Here is what I'm doing

    // Define the partitioned Listing Table
    let listing_options = ListingOptions::new(file_format)
        .with_table_partition_cols(part)
        .with_target_partitions(1)
        .with_file_extension(".ndjson.zst");

    ctx.register_listing_table(
        "data",
        format!("s3://{BUCKET_NAME}/data_lake/data_warehouse"),
        listing_options,
        Some(schema),
        None,
    )
    .await?;

    let df = ctx
        .sql(
            r#"
       SELECT 
          SUBSTRING("OriginalRequest", 9, 3) as dep, *
       FROM data 
          WHERE 
           /*partitions predicates here*/

       "#,
        )
        .await?;

  let s3 = AmazonS3Builder::new()
        .with_bucket_name(save_bucket_name)
        .with_region(REGION)
        .build()?;

    // Register the S3 store in DataFusion context
    let path = format!("s3://{save_bucket_name}");
    let s3_url = Url::parse(&path).unwrap();
    let arc_s3 = Arc::new(s3);
    ctx.runtime_env()
        .register_object_store(&s3_url, arc_s3.clone());

    // Write the data as JSON partitioned by `dep`
    let output_path = "s3://my_bucket/output/json/";
    //write as JSON to s3

    let options = DataFrameWriteOptions::new()
     .with_partition_by(vec!["dep".to_string()]);

    let mut json_options = JsonOptions::default();
    json_options.compression = CompressionTypeVariant::ZSTD;

    df
        .write_json(&output_path, options, Some(json_options))
        .await?;

Will it swallow all memory and fail or it will be running in a kind on streaming format? How could I limit the amount of memory which can be used to run the app inside of the Docker and make sure it would not run out.

alamb commented 3 months ago

Will it swallow all memory and fail or it will be running in a kind on streaming format?

Hi @Smotrov, given your description and code, I would expect this query to run incrementally and not buffer all the results to memory -- that is I would expect the query to stream

There are some operators that require potentially buffering all data (grouping, joins, sorts) but you don't seem to be doing that

I am not super familar with exactly how the json writing is implemented, but I believe that should be streaming as well

How could I limit the amount of memory

You can limit the amount of memory using https://docs.rs/datafusion/latest/datafusion/execution/memory_pool/trait.MemoryPool.html

However, as I mentioed I wouldn't expect your query to buffer large amounts of memory, so if it is maybe we need to adjust the writer seetings or there is some improvement to make to datafusion

Let us know how it goes!

Smotrov commented 3 months ago

Thank you @alamb

This is what I actually did.

const MEMORY_LIMIT: usize = 8 * 1024 * 1024 * 1024; // 8GB

fn create_context() -> Result<SessionContext> {
    // Create a memory pool with a limit
    let memory_pool = Arc::new(FairSpillPool::new(MEMORY_LIMIT));

    // Configure the runtime environment to use the memory pool
    let rt_config = RuntimeConfig::new()
      .with_memory_pool(memory_pool)
      .with_temp_file_path(PathBuf::from("./tmp"));
    let runtime_env = Arc::new(RuntimeEnv::new(rt_config)?);

    // Configure the session context to use the runtime environment
    let session_config = SessionConfig::new();
    let ctx = SessionContext::new_with_config_rt(session_config, runtime_env);
    Ok(ctx)
}

However it easily takes 20..30GB of RAM and what is interesting, the CPU load stays relatively small. Like 20...30%.

The memory consumption is that high when I set at lease 4 target partitions.

// Define the partitioned Listing Table
let listing_options = ListingOptions::new(file_format)
        .with_table_partition_cols(part)
        .with_target_partitions(4)
        .with_file_extension(".ndjson.zst");

Would be grate if it would be possible to set an actual limit for the memory otherwise I can't use it on docker :-( Full utilization of all CPU cores would also be cool if possible.

alamb commented 3 months ago

Hi @Smotrov -- I agree the use of 20-30 GB seems not good. Perhaps there is something in DataFusion that is not accounting for memory correctly (perhaps it is the decoding of the ndjson / zstd stream) 🤔

korowa commented 3 months ago

FWIW: I was able to reproduce it while writing single file into partitioned table with multiple partitions (~5kk rows, 4k partitions): json_zstd_insert The memory usage comes from having separate instance of ZSTD encoder per writing thread (per partition). Probably this feature could help with the memory usage issue (when implemented in zstd-rs -> async-compression).

UPD: for single file -> non-partitioned table case, DF works just fine (~75MB peak in total, ~15MB of them is memory for encoder), and it's also ok for writing multiple partitions without compression (~500MB in total due to buffering for 4k writes) so it's just an issue in case of dozens / hundreds of partitions + ZSTD.

alamb commented 2 months ago

Thanks @korowa -- this analysis makes sense (aka that there is some constant overhead per active partition)

@Smotrov does this match your dataset? As in how many partitions (aka files) are created by your query?

Some other ideas for improvements:

  1. Account for this overhead somehow in the memory manager. This would not reduce the memory required, but instead would cause the query to error rather than using too much memory
  2. Implement early flushing somehow if too much memory was used (force flush currently open files), though this might result in very many small files for a highly partitioned dataset