From ae0741b8d6f203d881de12f9f5d8f0d326f5093c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Mar 2026 07:43:26 -0600 Subject: [PATCH 01/21] feat: add standalone shuffle benchmark binary for profiling Add a `shuffle_bench` binary that benchmarks shuffle write and read performance independently from Spark, making it easy to profile with tools like `cargo flamegraph`, `perf`, or `instruments`. Supports reading Parquet files (e.g. TPC-H/TPC-DS) or generating synthetic data with configurable schema. Covers different scenarios including compression codecs, partition counts, partitioning schemes, and memory-constrained spilling. --- native/Cargo.lock | 88 +++- native/core/Cargo.toml | 5 + native/core/src/bin/shuffle_bench.rs | 725 +++++++++++++++++++++++++++ 3 files changed, 816 insertions(+), 2 deletions(-) create mode 100644 native/core/src/bin/shuffle_bench.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 5f99c614b3..f43b41dd9a 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -96,12 +96,56 @@ version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" +[[package]] +name = "anstream" +version = "0.6.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43d5b281e737544384e969a5ccad3f1cdd24b48086a0fc1b2a5262a26b8f4f4a" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is_terminal_polyfill", + "utf8parse", +] + [[package]] name = "anstyle" version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" +[[package]] +name = "anstyle-parse" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e7644824f0aa2c7b9384579234ef10eb7efb6a0deb83f9630a49594dd9c15c2" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40c48f72fd53cd289104fc64099abca73db4166ad86ea0b4341abe65af83dadc" +dependencies = [ + "windows-sys 0.60.2", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "291e6a250ff86cd4a820112fb8898808a366d8f9f58ce16d1f538353ad55747d" +dependencies = [ + "anstyle", + "once_cell_polyfill", + "windows-sys 0.60.2", +] + [[package]] name = "anyhow" version = "1.0.102" @@ -1331,6 +1375,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2797f34da339ce31042b27d23607e051786132987f595b02ba4f6a6dffb7030a" dependencies = [ "clap_builder", + "clap_derive", ] [[package]] @@ -1339,8 +1384,22 @@ version = "4.5.60" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24a241312cea5059b13574bb9b3861cabf758b879c15190b37b6d6fd63ab6876" dependencies = [ + "anstream", "anstyle", "clap_lex", + "strsim", +] + +[[package]] +name = "clap_derive" +version = "4.5.55" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a92793da1a46a5f2a02a6f4c46c6496b28c43638adea8306fcb0caa1634f24e5" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.117", ] [[package]] @@ -1358,6 +1417,12 @@ dependencies = [ "cc", ] +[[package]] +name = "colorchoice" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d07550c9036bf2ae0c684c4297d503f838287c83c53686d05370d0e139ae570" + [[package]] name = "combine" version = "4.6.7" @@ -1834,6 +1899,7 @@ dependencies = [ "aws-config", "aws-credential-types", "bytes", + "clap", "crc32fast", "criterion", "datafusion", @@ -1885,7 +1951,7 @@ dependencies = [ [[package]] name = "datafusion-comet-common" -version = "0.14.0" +version = "0.15.0" dependencies = [ "arrow", "datafusion", @@ -1911,7 +1977,7 @@ dependencies = [ [[package]] name = "datafusion-comet-jni-bridge" -version = "0.14.0" +version = "0.15.0" dependencies = [ "arrow", "assertables", @@ -3609,6 +3675,12 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "is_terminal_polyfill" +version = "1.70.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" + [[package]] name = "itertools" version = "0.13.0" @@ -4289,6 +4361,12 @@ version = "1.21.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9f7c3e4beb33f85d45ae3e3a1792185706c8e16d043238c593331cc7cd313b50" +[[package]] +name = "once_cell_polyfill" +version = "1.70.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "384b8ab6d37215f3c5301a95a4accb5d64aa607f1fcb26a11b5303878451b4fe" + [[package]] name = "oorandom" version = "11.1.5" @@ -6339,6 +6417,12 @@ version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" +[[package]] +name = "utf8parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" + [[package]] name = "uuid" version = "1.22.0" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 3f305a631d..3df9e55719 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -72,6 +72,7 @@ url = { workspace = true } aws-config = { workspace = true } aws-credential-types = { workspace = true } parking_lot = "0.12.5" +clap = { version = "4", features = ["derive"] } datafusion-comet-objectstore-hdfs = { path = "../hdfs", optional = true, default-features = false, features = ["hdfs"] } reqwest = { version = "0.12", default-features = false, features = ["rustls-tls-native-roots", "http2"] } object_store_opendal = {version = "0.55.0", optional = true} @@ -113,6 +114,10 @@ name = "comet" # "rlib" is for benchmarking with criterion. crate-type = ["cdylib", "rlib"] +[[bin]] +name = "shuffle_bench" +path = "src/bin/shuffle_bench.rs" + [[bench]] name = "parquet_read" harness = false diff --git a/native/core/src/bin/shuffle_bench.rs b/native/core/src/bin/shuffle_bench.rs new file mode 100644 index 0000000000..c1498161f7 --- /dev/null +++ b/native/core/src/bin/shuffle_bench.rs @@ -0,0 +1,725 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Standalone shuffle benchmark tool for profiling Comet shuffle write and read +//! outside of Spark. +//! +//! # Usage +//! +//! Read from Parquet files (e.g. TPC-H lineitem): +//! ```sh +//! cargo run --release --bin shuffle_bench -- \ +//! --input /data/tpch-sf100/lineitem/ \ +//! --partitions 200 \ +//! --codec zstd --zstd-level 1 \ +//! --hash-columns 0,3 \ +//! --read-back +//! ``` +//! +//! Generate synthetic data: +//! ```sh +//! cargo run --release --bin shuffle_bench -- \ +//! --generate --gen-rows 10000000 --gen-string-cols 4 --gen-int-cols 4 \ +//! --gen-decimal-cols 2 --gen-avg-string-len 32 \ +//! --partitions 200 --codec lz4 --read-back +//! ``` +//! +//! Profile with flamegraph: +//! ```sh +//! cargo flamegraph --release --bin shuffle_bench -- \ +//! --input /data/tpch-sf100/lineitem/ \ +//! --partitions 200 --codec zstd --zstd-level 1 +//! ``` + +use arrow::array::builder::{Date32Builder, Decimal128Builder, Int64Builder, StringBuilder}; +use arrow::array::RecordBatch; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use clap::Parser; +use comet::execution::shuffle::{ + read_ipc_compressed, CometPartitioning, CompressionCodec, ShuffleWriterExec, +}; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; +use datafusion::execution::config::SessionConfig; +use datafusion::execution::runtime_env::RuntimeEnvBuilder; +use datafusion::physical_expr::expressions::Column; +use datafusion::physical_plan::common::collect; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; +use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; +use rand::RngExt; +use std::fs; +use std::path::PathBuf; +use std::sync::Arc; +use std::time::Instant; + +#[derive(Parser, Debug)] +#[command( + name = "shuffle_bench", + about = "Standalone benchmark for Comet shuffle write and read performance" +)] +struct Args { + /// Path to input Parquet file or directory of Parquet files + #[arg(long)] + input: Option, + + /// Generate synthetic data instead of reading from Parquet + #[arg(long, default_value_t = false)] + generate: bool, + + /// Number of rows to generate (requires --generate) + #[arg(long, default_value_t = 1_000_000)] + gen_rows: usize, + + /// Number of Int64 columns to generate + #[arg(long, default_value_t = 4)] + gen_int_cols: usize, + + /// Number of Utf8 string columns to generate + #[arg(long, default_value_t = 2)] + gen_string_cols: usize, + + /// Number of Decimal128 columns to generate + #[arg(long, default_value_t = 2)] + gen_decimal_cols: usize, + + /// Number of Date32 columns to generate + #[arg(long, default_value_t = 1)] + gen_date_cols: usize, + + /// Average string length for generated string columns + #[arg(long, default_value_t = 24)] + gen_avg_string_len: usize, + + /// Batch size for reading Parquet or generating data + #[arg(long, default_value_t = 8192)] + batch_size: usize, + + /// Number of output shuffle partitions + #[arg(long, default_value_t = 200)] + partitions: usize, + + /// Partitioning scheme: hash, single, round-robin + #[arg(long, default_value = "hash")] + partitioning: String, + + /// Column indices to hash on (comma-separated, e.g. "0,3") + #[arg(long, default_value = "0")] + hash_columns: String, + + /// Compression codec: none, lz4, zstd, snappy + #[arg(long, default_value = "zstd")] + codec: String, + + /// Zstd compression level (1-22) + #[arg(long, default_value_t = 1)] + zstd_level: i32, + + /// Memory limit in bytes (triggers spilling when exceeded) + #[arg(long)] + memory_limit: Option, + + /// Also benchmark reading back the shuffle output + #[arg(long, default_value_t = false)] + read_back: bool, + + /// Number of iterations to run + #[arg(long, default_value_t = 1)] + iterations: usize, + + /// Number of warmup iterations before timing + #[arg(long, default_value_t = 0)] + warmup: usize, + + /// Output directory for shuffle data/index files + #[arg(long, default_value = "/tmp/comet_shuffle_bench")] + output_dir: PathBuf, + + /// Write buffer size in bytes + #[arg(long, default_value_t = 1048576)] + write_buffer_size: usize, +} + +fn main() { + let args = Args::parse(); + + // Validate args + if args.input.is_none() && !args.generate { + eprintln!("Error: must specify either --input or --generate"); + std::process::exit(1); + } + + // Create output directory + fs::create_dir_all(&args.output_dir).expect("Failed to create output directory"); + + let data_file = args.output_dir.join("data.out"); + let index_file = args.output_dir.join("index.out"); + + // Load data + let load_start = Instant::now(); + let batches = if let Some(ref input_path) = args.input { + load_parquet(input_path, args.batch_size) + } else { + generate_data(&args) + }; + let load_elapsed = load_start.elapsed(); + + let schema = batches[0].schema(); + let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); + let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); + + println!("=== Shuffle Benchmark ==="); + println!( + "Data source: {}", + if args.input.is_some() { + "parquet" + } else { + "generated" + } + ); + println!( + "Schema: {} columns ({} fields)", + schema.fields().len(), + describe_schema(&schema) + ); + println!("Total rows: {}", format_number(total_rows)); + println!("Total size: {}", format_bytes(total_bytes)); + println!("Batches: {}", batches.len()); + println!( + "Rows/batch: ~{}", + if batches.is_empty() { + 0 + } else { + total_rows / batches.len() + } + ); + println!("Load time: {:.3}s", load_elapsed.as_secs_f64()); + println!(); + + let codec = parse_codec(&args.codec, args.zstd_level); + let hash_col_indices = parse_hash_columns(&args.hash_columns); + + println!("Partitioning: {}", args.partitioning); + println!("Partitions: {}", args.partitions); + println!("Codec: {:?}", codec); + println!("Hash columns: {:?}", hash_col_indices); + if let Some(mem_limit) = args.memory_limit { + println!("Memory limit: {}", format_bytes(mem_limit)); + } + println!( + "Iterations: {} (warmup: {})", + args.iterations, args.warmup + ); + println!(); + + // Run warmup + timed iterations + let total_iters = args.warmup + args.iterations; + let mut write_times = Vec::with_capacity(args.iterations); + let mut read_times = Vec::with_capacity(args.iterations); + let mut data_file_sizes = Vec::with_capacity(args.iterations); + + for i in 0..total_iters { + let is_warmup = i < args.warmup; + let label = if is_warmup { + format!("warmup {}/{}", i + 1, args.warmup) + } else { + format!("iter {}/{}", i - args.warmup + 1, args.iterations) + }; + + // Write phase + let write_elapsed = run_shuffle_write( + &batches, + &schema, + &codec, + &hash_col_indices, + &args, + data_file.to_str().unwrap(), + index_file.to_str().unwrap(), + ); + let data_size = fs::metadata(&data_file).map(|m| m.len()).unwrap_or(0); + + if !is_warmup { + write_times.push(write_elapsed); + data_file_sizes.push(data_size); + } + + print!(" [{label}] write: {:.3}s", write_elapsed); + print!(" output: {}", format_bytes(data_size as usize)); + + // Read phase + if args.read_back { + let read_elapsed = run_shuffle_read( + data_file.to_str().unwrap(), + index_file.to_str().unwrap(), + args.partitions, + ); + if !is_warmup { + read_times.push(read_elapsed); + } + print!(" read: {:.3}s", read_elapsed); + } + println!(); + } + + // Print summary + if args.iterations > 0 { + println!(); + println!("=== Results ==="); + + let avg_write = write_times.iter().sum::() / write_times.len() as f64; + let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; + let write_throughput_rows = total_rows as f64 / avg_write; + let write_throughput_bytes = total_bytes as f64 / avg_write; + let compression_ratio = if avg_data_size > 0 { + total_bytes as f64 / avg_data_size as f64 + } else { + 0.0 + }; + + println!("Write:"); + println!(" avg time: {:.3}s", avg_write); + if write_times.len() > 1 { + let min = write_times.iter().cloned().fold(f64::INFINITY, f64::min); + let max = write_times + .iter() + .cloned() + .fold(f64::NEG_INFINITY, f64::max); + println!(" min/max: {:.3}s / {:.3}s", min, max); + } + println!( + " throughput: {}/s ({} rows/s)", + format_bytes(write_throughput_bytes as usize), + format_number(write_throughput_rows as usize) + ); + println!( + " output size: {}", + format_bytes(avg_data_size as usize) + ); + println!(" compression: {:.2}x", compression_ratio); + + if !read_times.is_empty() { + let avg_read = read_times.iter().sum::() / read_times.len() as f64; + let read_throughput_bytes = avg_data_size as f64 / avg_read; + + println!("Read:"); + println!(" avg time: {:.3}s", avg_read); + if read_times.len() > 1 { + let min = read_times.iter().cloned().fold(f64::INFINITY, f64::min); + let max = read_times.iter().cloned().fold(f64::NEG_INFINITY, f64::max); + println!(" min/max: {:.3}s / {:.3}s", min, max); + } + println!( + " throughput: {}/s (from compressed)", + format_bytes(read_throughput_bytes as usize) + ); + } + } + + // Cleanup + let _ = fs::remove_file(&data_file); + let _ = fs::remove_file(&index_file); +} + +fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { + let mut batches = Vec::new(); + + let paths = if path.is_dir() { + let mut files: Vec = fs::read_dir(path) + .expect("Failed to read input directory") + .filter_map(|entry| { + let entry = entry.ok()?; + let p = entry.path(); + if p.extension().and_then(|e| e.to_str()) == Some("parquet") { + Some(p) + } else { + None + } + }) + .collect(); + files.sort(); + if files.is_empty() { + panic!("No .parquet files found in {}", path.display()); + } + files + } else { + vec![path.clone()] + }; + + for file_path in &paths { + let file = fs::File::open(file_path) + .unwrap_or_else(|e| panic!("Failed to open {}: {}", file_path.display(), e)); + let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap_or_else(|e| { + panic!( + "Failed to read Parquet metadata from {}: {}", + file_path.display(), + e + ) + }); + let reader = builder + .with_batch_size(batch_size) + .build() + .unwrap_or_else(|e| { + panic!( + "Failed to build Parquet reader for {}: {}", + file_path.display(), + e + ) + }); + for batch_result in reader { + let batch = batch_result.unwrap_or_else(|e| { + panic!("Failed to read batch from {}: {}", file_path.display(), e) + }); + if batch.num_rows() > 0 { + batches.push(batch); + } + } + } + + if batches.is_empty() { + panic!("No data read from input"); + } + + println!( + "Loaded {} batches from {} file(s)", + batches.len(), + paths.len() + ); + batches +} + +fn generate_data(args: &Args) -> Vec { + let mut fields = Vec::new(); + let mut col_idx = 0; + + // Int64 columns + for _ in 0..args.gen_int_cols { + fields.push(Field::new( + format!("int_col_{col_idx}"), + DataType::Int64, + true, + )); + col_idx += 1; + } + // String columns + for _ in 0..args.gen_string_cols { + fields.push(Field::new( + format!("str_col_{col_idx}"), + DataType::Utf8, + true, + )); + col_idx += 1; + } + // Decimal columns + for _ in 0..args.gen_decimal_cols { + fields.push(Field::new( + format!("dec_col_{col_idx}"), + DataType::Decimal128(18, 2), + true, + )); + col_idx += 1; + } + // Date columns + for _ in 0..args.gen_date_cols { + fields.push(Field::new( + format!("date_col_{col_idx}"), + DataType::Date32, + true, + )); + col_idx += 1; + } + + let schema = Arc::new(Schema::new(fields)); + let mut batches = Vec::new(); + let mut rng = rand::rng(); + let mut remaining = args.gen_rows; + + while remaining > 0 { + let batch_rows = remaining.min(args.batch_size); + remaining -= batch_rows; + + let mut columns: Vec> = Vec::new(); + + // Int64 columns + for _ in 0..args.gen_int_cols { + let mut builder = Int64Builder::with_capacity(batch_rows); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + builder.append_value(rng.random_range(0..1_000_000i64)); + } + } + columns.push(Arc::new(builder.finish())); + } + // String columns + for _ in 0..args.gen_string_cols { + let mut builder = + StringBuilder::with_capacity(batch_rows, batch_rows * args.gen_avg_string_len); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + let len = rng.random_range(1..args.gen_avg_string_len * 2); + let s: String = (0..len) + .map(|_| rng.random_range(b'a'..=b'z') as char) + .collect(); + builder.append_value(&s); + } + } + columns.push(Arc::new(builder.finish())); + } + // Decimal columns + for _ in 0..args.gen_decimal_cols { + let mut builder = Decimal128Builder::with_capacity(batch_rows) + .with_precision_and_scale(18, 2) + .unwrap(); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + builder.append_value(rng.random_range(0..100_000_000i128)); + } + } + columns.push(Arc::new(builder.finish())); + } + // Date columns + for _ in 0..args.gen_date_cols { + let mut builder = Date32Builder::with_capacity(batch_rows); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + builder.append_value(rng.random_range(0..20000i32)); + } + } + columns.push(Arc::new(builder.finish())); + } + + let batch = RecordBatch::try_new(Arc::clone(&schema), columns).unwrap(); + batches.push(batch); + } + + println!( + "Generated {} batches ({} rows)", + batches.len(), + args.gen_rows + ); + batches +} + +fn run_shuffle_write( + batches: &[RecordBatch], + schema: &SchemaRef, + codec: &CompressionCodec, + hash_col_indices: &[usize], + args: &Args, + data_file: &str, + index_file: &str, +) -> f64 { + let partitioning = build_partitioning( + &args.partitioning, + args.partitions, + hash_col_indices, + schema, + ); + + let partitions = &[batches.to_vec()]; + let exec = ShuffleWriterExec::try_new( + Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(partitions, Arc::clone(schema), None).unwrap(), + ))), + partitioning, + codec.clone(), + data_file.to_string(), + index_file.to_string(), + false, + args.write_buffer_size, + ) + .expect("Failed to create ShuffleWriterExec"); + + let config = SessionConfig::new().with_batch_size(args.batch_size); + let mut runtime_builder = RuntimeEnvBuilder::new(); + if let Some(mem_limit) = args.memory_limit { + runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); + } + let runtime_env = Arc::new(runtime_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + let task_ctx = ctx.task_ctx(); + + let start = Instant::now(); + let stream = exec.execute(0, task_ctx).unwrap(); + let rt = tokio::runtime::Runtime::new().unwrap(); + rt.block_on(collect(stream)).unwrap(); + start.elapsed().as_secs_f64() +} + +fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> f64 { + let start = Instant::now(); + + // Read index file to get partition offsets + let index_bytes = fs::read(index_file).expect("Failed to read index file"); + let num_offsets = index_bytes.len() / 8; + let offsets: Vec = (0..num_offsets) + .map(|i| { + let bytes: [u8; 8] = index_bytes[i * 8..(i + 1) * 8].try_into().unwrap(); + i64::from_le_bytes(bytes) + }) + .collect(); + + // Read data file + let data_bytes = fs::read(data_file).expect("Failed to read data file"); + + let mut total_rows = 0usize; + let mut total_batches = 0usize; + + // Decode each partition's data + for p in 0..num_partitions.min(offsets.len().saturating_sub(1)) { + let start_offset = offsets[p] as usize; + let end_offset = offsets[p + 1] as usize; + + if start_offset >= end_offset { + continue; // Empty partition + } + + // Read all IPC blocks within this partition + let mut offset = start_offset; + while offset < end_offset { + // First 8 bytes: IPC length + let ipc_length = + u64::from_le_bytes(data_bytes[offset..offset + 8].try_into().unwrap()) as usize; + + // Skip 8-byte length prefix, then 8 bytes of field_count + codec header + let block_data = &data_bytes[offset + 16..offset + 8 + ipc_length]; + let batch = read_ipc_compressed(block_data).expect("Failed to decode shuffle block"); + total_rows += batch.num_rows(); + total_batches += 1; + + offset += 8 + ipc_length; + } + } + + let elapsed = start.elapsed().as_secs_f64(); + eprintln!( + " read back {} rows in {} batches from {} partitions", + format_number(total_rows), + total_batches, + num_partitions + ); + elapsed +} + +fn build_partitioning( + scheme: &str, + num_partitions: usize, + hash_col_indices: &[usize], + schema: &SchemaRef, +) -> CometPartitioning { + match scheme { + "single" => CometPartitioning::SinglePartition, + "round-robin" => CometPartitioning::RoundRobin(num_partitions, 0), + "hash" => { + let exprs: Vec> = hash_col_indices + .iter() + .map(|&idx| { + let field = schema.field(idx); + Arc::new(Column::new(field.name(), idx)) + as Arc + }) + .collect(); + CometPartitioning::Hash(exprs, num_partitions) + } + other => { + eprintln!("Unknown partitioning scheme: {other}. Using hash."); + build_partitioning("hash", num_partitions, hash_col_indices, schema) + } + } +} + +fn parse_codec(codec: &str, zstd_level: i32) -> CompressionCodec { + match codec.to_lowercase().as_str() { + "none" => CompressionCodec::None, + "lz4" => CompressionCodec::Lz4Frame, + "zstd" => CompressionCodec::Zstd(zstd_level), + "snappy" => CompressionCodec::Snappy, + other => { + eprintln!("Unknown codec: {other}. Using zstd."); + CompressionCodec::Zstd(zstd_level) + } + } +} + +fn parse_hash_columns(s: &str) -> Vec { + s.split(',') + .filter(|s| !s.is_empty()) + .map(|s| s.trim().parse::().expect("Invalid column index")) + .collect() +} + +fn describe_schema(schema: &Schema) -> String { + let mut counts = std::collections::HashMap::new(); + for field in schema.fields() { + let type_name = match field.data_type() { + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 => "int", + DataType::Float16 | DataType::Float32 | DataType::Float64 => "float", + DataType::Utf8 | DataType::LargeUtf8 => "string", + DataType::Boolean => "bool", + DataType::Date32 | DataType::Date64 => "date", + DataType::Decimal128(_, _) | DataType::Decimal256(_, _) => "decimal", + DataType::Timestamp(_, _) => "timestamp", + DataType::Binary | DataType::LargeBinary | DataType::FixedSizeBinary(_) => "binary", + _ => "other", + }; + *counts.entry(type_name).or_insert(0) += 1; + } + let mut parts: Vec = counts + .into_iter() + .map(|(k, v)| format!("{}x{}", v, k)) + .collect(); + parts.sort(); + parts.join(", ") +} + +fn format_number(n: usize) -> String { + let s = n.to_string(); + let mut result = String::new(); + for (i, c) in s.chars().rev().enumerate() { + if i > 0 && i % 3 == 0 { + result.push(','); + } + result.push(c); + } + result.chars().rev().collect() +} + +fn format_bytes(bytes: usize) -> String { + if bytes >= 1024 * 1024 * 1024 { + format!("{:.2} GiB", bytes as f64 / (1024.0 * 1024.0 * 1024.0)) + } else if bytes >= 1024 * 1024 { + format!("{:.2} MiB", bytes as f64 / (1024.0 * 1024.0)) + } else if bytes >= 1024 { + format!("{:.2} KiB", bytes as f64 / 1024.0) + } else { + format!("{bytes} B") + } +} From 9b5b305cb125a76e22f55ac61ee4994ff9e9f484 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Mar 2026 09:05:16 -0600 Subject: [PATCH 02/21] feat: add --limit option to shuffle benchmark (default 1M rows) --- native/core/src/bin/shuffle_bench.rs | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/native/core/src/bin/shuffle_bench.rs b/native/core/src/bin/shuffle_bench.rs index c1498161f7..9b963c5803 100644 --- a/native/core/src/bin/shuffle_bench.rs +++ b/native/core/src/bin/shuffle_bench.rs @@ -152,6 +152,10 @@ struct Args { /// Write buffer size in bytes #[arg(long, default_value_t = 1048576)] write_buffer_size: usize, + + /// Maximum number of rows to use (default: 1,000,000) + #[arg(long, default_value_t = 1_000_000)] + limit: usize, } fn main() { @@ -178,6 +182,26 @@ fn main() { }; let load_elapsed = load_start.elapsed(); + // Apply row limit + let batches = { + let mut limited = Vec::new(); + let mut rows_so_far = 0usize; + for batch in batches { + if rows_so_far >= args.limit { + break; + } + let remaining = args.limit - rows_so_far; + if batch.num_rows() <= remaining { + rows_so_far += batch.num_rows(); + limited.push(batch); + } else { + limited.push(batch.slice(0, remaining)); + rows_so_far += remaining; + } + } + limited + }; + let schema = batches[0].schema(); let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); From e1ab490c57b04387e4885284811ccf2ad993f4a6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Mar 2026 09:19:01 -0600 Subject: [PATCH 03/21] perf: apply limit during parquet read to avoid scanning all files --- native/core/src/bin/shuffle_bench.rs | 43 ++++++++++++---------------- 1 file changed, 18 insertions(+), 25 deletions(-) diff --git a/native/core/src/bin/shuffle_bench.rs b/native/core/src/bin/shuffle_bench.rs index 9b963c5803..17b1a9a6ff 100644 --- a/native/core/src/bin/shuffle_bench.rs +++ b/native/core/src/bin/shuffle_bench.rs @@ -176,32 +176,12 @@ fn main() { // Load data let load_start = Instant::now(); let batches = if let Some(ref input_path) = args.input { - load_parquet(input_path, args.batch_size) + load_parquet(input_path, args.batch_size, args.limit) } else { generate_data(&args) }; let load_elapsed = load_start.elapsed(); - // Apply row limit - let batches = { - let mut limited = Vec::new(); - let mut rows_so_far = 0usize; - for batch in batches { - if rows_so_far >= args.limit { - break; - } - let remaining = args.limit - rows_so_far; - if batch.num_rows() <= remaining { - rows_so_far += batch.num_rows(); - limited.push(batch); - } else { - limited.push(batch.slice(0, remaining)); - rows_so_far += remaining; - } - } - limited - }; - let schema = batches[0].schema(); let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); @@ -358,8 +338,9 @@ fn main() { let _ = fs::remove_file(&index_file); } -fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { +fn load_parquet(path: &PathBuf, batch_size: usize, limit: usize) -> Vec { let mut batches = Vec::new(); + let mut total_rows = 0usize; let paths = if path.is_dir() { let mut files: Vec = fs::read_dir(path) @@ -383,7 +364,7 @@ fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { vec![path.clone()] }; - for file_path in &paths { + 'outer: for file_path in &paths { let file = fs::File::open(file_path) .unwrap_or_else(|e| panic!("Failed to open {}: {}", file_path.display(), e)); let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap_or_else(|e| { @@ -407,8 +388,19 @@ fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { let batch = batch_result.unwrap_or_else(|e| { panic!("Failed to read batch from {}: {}", file_path.display(), e) }); - if batch.num_rows() > 0 { + if batch.num_rows() == 0 { + continue; + } + let remaining = limit - total_rows; + if batch.num_rows() <= remaining { + total_rows += batch.num_rows(); batches.push(batch); + } else { + batches.push(batch.slice(0, remaining)); + total_rows += remaining; + } + if total_rows >= limit { + break 'outer; } } } @@ -418,8 +410,9 @@ fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { } println!( - "Loaded {} batches from {} file(s)", + "Loaded {} batches ({} rows) from {} file(s)", batches.len(), + format_number(total_rows), paths.len() ); batches From ca36cbd579d5c512e9617eb54cd052ec0d2c244d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 23 Mar 2026 06:39:47 -0700 Subject: [PATCH 04/21] chore: add comment explaining parquet/rand deps in shuffle crate --- native/shuffle/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 1a64a36ff2..1337ea3b15 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -43,6 +43,7 @@ itertools = "0.14.0" jni = "0.21" log = "0.4" lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] } +# parquet and rand are only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } rand = { workspace = true, optional = true } simd-adler32 = "0.3.7" From 6e8bed28902e7256e1e1b3e914439e0a5abf3622 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 26 Mar 2026 07:13:41 -0700 Subject: [PATCH 05/21] perf: add max_buffered_batches config and stream shuffle bench from parquet - Add `spark.comet.exec.shuffle.maxBufferedBatches` config to limit the number of batches buffered before spilling, allowing earlier spilling to reduce peak memory usage on executors - Fix too-many-open-files: close spill file FD after each spill and reopen in append mode, rather than holding one FD open per partition - Refactor shuffle_bench to stream directly from Parquet instead of loading all input data into memory; remove synthetic data generation - Add --max-buffered-batches CLI arg to shuffle_bench - Add shuffle benchmark documentation to README --- .../scala/org/apache/comet/CometConf.scala | 12 + native/core/src/execution/planner.rs | 2 + native/proto/src/proto/operator.proto | 3 + native/shuffle/README.md | 43 ++ native/shuffle/benches/shuffle_writer.rs | 1 + native/shuffle/src/bin/shuffle_bench.rs | 432 +++++------------- .../src/partitioners/multi_partition.rs | 11 + native/shuffle/src/shuffle_writer.rs | 11 + .../shuffle/src/writers/partition_writer.rs | 29 +- .../shuffle/CometNativeShuffleWriter.scala | 2 + 10 files changed, 209 insertions(+), 337 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index bfe90181ff..2ad9c1f609 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -534,6 +534,18 @@ object CometConf extends ShimCometConf { .checkValue(v => v > 0, "Write buffer size must be positive") .createWithDefault(1) + val COMET_SHUFFLE_MAX_BUFFERED_BATCHES: ConfigEntry[Int] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.maxBufferedBatches") + .category(CATEGORY_SHUFFLE) + .doc("Maximum number of batches to buffer in memory before spilling to disk during " + + "native shuffle. Setting this to a small value causes earlier spilling, which reduces " + + "peak memory usage on executors at the cost of more disk I/O. " + + "The default value of 0 disables this limit and spills only when the memory pool is " + + "exhausted.") + .intConf + .checkValue(v => v >= 0, "Max buffered batches must be non-negative") + .createWithDefault(0) + val COMET_SHUFFLE_PREFER_DICTIONARY_RATIO: ConfigEntry[Double] = conf( "spark.comet.shuffle.preferDictionary.ratio") .category(CATEGORY_SHUFFLE) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5af31fcc22..b6e05fccd1 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1352,6 +1352,7 @@ impl PhysicalPlanner { }?; let write_buffer_size = writer.write_buffer_size as usize; + let max_buffered_batches = writer.max_buffered_batches as usize; let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( Arc::clone(&child.native_plan), partitioning, @@ -1360,6 +1361,7 @@ impl PhysicalPlanner { writer.output_index_file.clone(), writer.tracing_enabled, write_buffer_size, + max_buffered_batches, )?); Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 344b9f0f21..5e23aad061 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -294,6 +294,9 @@ message ShuffleWriter { // Size of the write buffer in bytes used when writing shuffle data to disk. // Larger values may improve write performance but use more memory. int32 write_buffer_size = 8; + // Maximum number of batches to buffer before spilling to disk. + // 0 means no limit (spill only when memory pool is exhausted). + int32 max_buffered_batches = 9; } message ParquetWriter { diff --git a/native/shuffle/README.md b/native/shuffle/README.md index 8fba6b0323..0333ddbe8e 100644 --- a/native/shuffle/README.md +++ b/native/shuffle/README.md @@ -23,3 +23,46 @@ This crate provides the shuffle writer and reader implementation for Apache Data of the [Apache DataFusion Comet] subproject. [Apache DataFusion Comet]: https://github.com/apache/datafusion-comet/ + +## Shuffle Benchmark Tool + +A standalone benchmark binary (`shuffle_bench`) is included for profiling shuffle write and read +performance outside of Spark. It streams input data directly from Parquet files. + +### Basic usage + +```sh +cargo run --release --features shuffle-bench --bin shuffle_bench -- \ + --input /data/tpch-sf100/lineitem/ \ + --partitions 200 \ + --codec zstd --zstd-level 1 \ + --hash-columns 0,3 +``` + +### Options + +| Option | Default | Description | +|---|---|---| +| `--input` | *(required)* | Path to a Parquet file or directory of Parquet files | +| `--partitions` | `200` | Number of output shuffle partitions | +| `--partitioning` | `hash` | Partitioning scheme: `hash`, `single`, `round-robin` | +| `--hash-columns` | `0` | Comma-separated column indices to hash on (e.g. `0,3`) | +| `--codec` | `zstd` | Compression codec: `none`, `lz4`, `zstd`, `snappy` | +| `--zstd-level` | `1` | Zstd compression level (1–22) | +| `--batch-size` | `8192` | Batch size for reading Parquet data | +| `--memory-limit` | *(none)* | Memory limit in bytes; triggers spilling when exceeded | +| `--max-buffered-batches` | `0` | Max batches to buffer before spilling (0 = memory-pool-only) | +| `--write-buffer-size` | `1048576` | Write buffer size in bytes | +| `--limit` | `0` | Limit rows processed per iteration (0 = no limit) | +| `--iterations` | `1` | Number of timed iterations | +| `--warmup` | `0` | Number of warmup iterations before timing | +| `--read-back` | `false` | Also benchmark reading back the shuffle output | +| `--output-dir` | `/tmp/comet_shuffle_bench` | Directory for temporary shuffle output files | + +### Profiling with flamegraph + +```sh +cargo flamegraph --release --features shuffle-bench --bin shuffle_bench -- \ + --input /data/tpch-sf100/lineitem/ \ + --partitions 200 --codec zstd --zstd-level 1 +``` diff --git a/native/shuffle/benches/shuffle_writer.rs b/native/shuffle/benches/shuffle_writer.rs index 27abd919fa..8ff1f024d5 100644 --- a/native/shuffle/benches/shuffle_writer.rs +++ b/native/shuffle/benches/shuffle_writer.rs @@ -153,6 +153,7 @@ fn create_shuffle_writer_exec( "/tmp/index.out".to_string(), false, 1024 * 1024, + 0, // max_buffered_batches: no limit ) .unwrap() } diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 373d0f92b9..0f04954ffa 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -16,11 +16,10 @@ // under the License. //! Standalone shuffle benchmark tool for profiling Comet shuffle write and read -//! outside of Spark. +//! outside of Spark. Streams input directly from Parquet files. //! //! # Usage //! -//! Read from Parquet files (e.g. TPC-H lineitem): //! ```sh //! cargo run --release --bin shuffle_bench -- \ //! --input /data/tpch-sf100/lineitem/ \ @@ -30,14 +29,6 @@ //! --read-back //! ``` //! -//! Generate synthetic data: -//! ```sh -//! cargo run --release --bin shuffle_bench -- \ -//! --generate --gen-rows 10000000 --gen-string-cols 4 --gen-int-cols 4 \ -//! --gen-decimal-cols 2 --gen-avg-string-len 32 \ -//! --partitions 200 --codec lz4 --read-back -//! ``` -//! //! Profile with flamegraph: //! ```sh //! cargo flamegraph --release --bin shuffle_bench -- \ @@ -45,23 +36,19 @@ //! --partitions 200 --codec zstd --zstd-level 1 //! ``` -use arrow::array::builder::{Date32Builder, Decimal128Builder, Int64Builder, StringBuilder}; -use arrow::array::RecordBatch; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::datatypes::{DataType, SchemaRef}; use clap::Parser; -use datafusion::datasource::memory::MemorySourceConfig; -use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_expr::expressions::Column; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::common::collect; use datafusion::physical_plan::ExecutionPlan; -use datafusion::prelude::SessionContext; +use datafusion::prelude::{ParquetReadOptions, SessionContext}; use datafusion_comet_shuffle::{ read_ipc_compressed, CometPartitioning, CompressionCodec, ShuffleWriterExec, }; use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; -use rand::RngExt; use std::fs; use std::path::PathBuf; use std::sync::Arc; @@ -75,37 +62,9 @@ use std::time::Instant; struct Args { /// Path to input Parquet file or directory of Parquet files #[arg(long)] - input: Option, - - /// Generate synthetic data instead of reading from Parquet - #[arg(long, default_value_t = false)] - generate: bool, - - /// Number of rows to generate (requires --generate) - #[arg(long, default_value_t = 1_000_000)] - gen_rows: usize, - - /// Number of Int64 columns to generate - #[arg(long, default_value_t = 4)] - gen_int_cols: usize, + input: PathBuf, - /// Number of Utf8 string columns to generate - #[arg(long, default_value_t = 2)] - gen_string_cols: usize, - - /// Number of Decimal128 columns to generate - #[arg(long, default_value_t = 2)] - gen_decimal_cols: usize, - - /// Number of Date32 columns to generate - #[arg(long, default_value_t = 1)] - gen_date_cols: usize, - - /// Average string length for generated string columns - #[arg(long, default_value_t = 24)] - gen_avg_string_len: usize, - - /// Batch size for reading Parquet or generating data + /// Batch size for reading Parquet data #[arg(long, default_value_t = 8192)] batch_size: usize, @@ -153,70 +112,37 @@ struct Args { #[arg(long, default_value_t = 1048576)] write_buffer_size: usize, - /// Maximum number of rows to use (default: 1,000,000) - #[arg(long, default_value_t = 1_000_000)] + /// Maximum number of batches to buffer before spilling (0 = no limit) + #[arg(long, default_value_t = 0)] + max_buffered_batches: usize, + + /// Limit rows processed per iteration (0 = no limit) + #[arg(long, default_value_t = 0)] limit: usize, } fn main() { let args = Args::parse(); - // Validate args - if args.input.is_none() && !args.generate { - eprintln!("Error: must specify either --input or --generate"); - std::process::exit(1); - } - // Create output directory fs::create_dir_all(&args.output_dir).expect("Failed to create output directory"); - let data_file = args.output_dir.join("data.out"); let index_file = args.output_dir.join("index.out"); - // Load data - let load_start = Instant::now(); - let batches = if let Some(ref input_path) = args.input { - load_parquet(input_path, args.batch_size, args.limit) - } else { - generate_data(&args) - }; - let load_elapsed = load_start.elapsed(); + let (schema, total_rows) = read_parquet_metadata(&args.input, args.limit); - let schema = batches[0].schema(); - let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); - let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); + let codec = parse_codec(&args.codec, args.zstd_level); + let hash_col_indices = parse_hash_columns(&args.hash_columns); println!("=== Shuffle Benchmark ==="); + println!("Input: {}", args.input.display()); println!( - "Data source: {}", - if args.input.is_some() { - "parquet" - } else { - "generated" - } - ); - println!( - "Schema: {} columns ({} fields)", + "Schema: {} columns ({})", schema.fields().len(), describe_schema(&schema) ); - println!("Total rows: {}", format_number(total_rows)); - println!("Total size: {}", format_bytes(total_bytes)); - println!("Batches: {}", batches.len()); - println!( - "Rows/batch: ~{}", - if batches.is_empty() { - 0 - } else { - total_rows / batches.len() - } - ); - println!("Load time: {:.3}s", load_elapsed.as_secs_f64()); - println!(); - - let codec = parse_codec(&args.codec, args.zstd_level); - let hash_col_indices = parse_hash_columns(&args.hash_columns); - + println!("Total rows: {}", format_number(total_rows as usize)); + println!("Batch size: {}", format_number(args.batch_size)); println!("Partitioning: {}", args.partitioning); println!("Partitions: {}", args.partitions); println!("Codec: {:?}", codec); @@ -224,13 +150,15 @@ fn main() { if let Some(mem_limit) = args.memory_limit { println!("Memory limit: {}", format_bytes(mem_limit)); } + if args.max_buffered_batches > 0 { + println!("Max buf batches:{}", args.max_buffered_batches); + } println!( "Iterations: {} (warmup: {})", args.iterations, args.warmup ); println!(); - // Run warmup + timed iterations let total_iters = args.warmup + args.iterations; let mut write_times = Vec::with_capacity(args.iterations); let mut read_times = Vec::with_capacity(args.iterations); @@ -244,9 +172,8 @@ fn main() { format!("iter {}/{}", i - args.warmup + 1, args.iterations) }; - // Write phase let write_elapsed = run_shuffle_write( - &batches, + &args.input, &schema, &codec, &hash_col_indices, @@ -264,7 +191,6 @@ fn main() { print!(" [{label}] write: {:.3}s", write_elapsed); print!(" output: {}", format_bytes(data_size as usize)); - // Read phase if args.read_back { let read_elapsed = run_shuffle_read( data_file.to_str().unwrap(), @@ -279,7 +205,6 @@ fn main() { println!(); } - // Print summary if args.iterations > 0 { println!(); println!("=== Results ==="); @@ -287,12 +212,6 @@ fn main() { let avg_write = write_times.iter().sum::() / write_times.len() as f64; let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; let write_throughput_rows = total_rows as f64 / avg_write; - let write_throughput_bytes = total_bytes as f64 / avg_write; - let compression_ratio = if avg_data_size > 0 { - total_bytes as f64 / avg_data_size as f64 - } else { - 0.0 - }; println!("Write:"); println!(" avg time: {:.3}s", avg_write); @@ -305,15 +224,13 @@ fn main() { println!(" min/max: {:.3}s / {:.3}s", min, max); } println!( - " throughput: {}/s ({} rows/s)", - format_bytes(write_throughput_bytes as usize), + " throughput: {} rows/s", format_number(write_throughput_rows as usize) ); println!( " output size: {}", format_bytes(avg_data_size as usize) ); - println!(" compression: {:.2}x", compression_ratio); if !read_times.is_empty() { let avg_read = read_times.iter().sum::() / read_times.len() as f64; @@ -333,38 +250,17 @@ fn main() { } } - // Cleanup let _ = fs::remove_file(&data_file); let _ = fs::remove_file(&index_file); } -fn load_parquet(path: &PathBuf, batch_size: usize, limit: usize) -> Vec { - let mut batches = Vec::new(); - let mut total_rows = 0usize; +/// Read schema and total row count from Parquet metadata without loading any data. +fn read_parquet_metadata(path: &PathBuf, limit: usize) -> (SchemaRef, u64) { + let paths = collect_parquet_paths(path); + let mut schema = None; + let mut total_rows = 0u64; - let paths = if path.is_dir() { - let mut files: Vec = fs::read_dir(path) - .expect("Failed to read input directory") - .filter_map(|entry| { - let entry = entry.ok()?; - let p = entry.path(); - if p.extension().and_then(|e| e.to_str()) == Some("parquet") { - Some(p) - } else { - None - } - }) - .collect(); - files.sort(); - if files.is_empty() { - panic!("No .parquet files found in {}", path.display()); - } - files - } else { - vec![path.clone()] - }; - - 'outer: for file_path in &paths { + for file_path in &paths { let file = fs::File::open(file_path) .unwrap_or_else(|e| panic!("Failed to open {}: {}", file_path.display(), e)); let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap_or_else(|e| { @@ -374,172 +270,44 @@ fn load_parquet(path: &PathBuf, batch_size: usize, limit: usize) -> Vec= limit { - break 'outer; - } + if schema.is_none() { + schema = Some(Arc::clone(builder.schema())); + } + total_rows += builder.metadata().file_metadata().num_rows() as u64; + if limit > 0 && total_rows >= limit as u64 { + total_rows = total_rows.min(limit as u64); + break; } } - if batches.is_empty() { - panic!("No data read from input"); - } - - println!( - "Loaded {} batches ({} rows) from {} file(s)", - batches.len(), - format_number(total_rows), - paths.len() - ); - batches + (schema.expect("No parquet files found"), total_rows) } -fn generate_data(args: &Args) -> Vec { - let mut fields = Vec::new(); - let mut col_idx = 0; - - // Int64 columns - for _ in 0..args.gen_int_cols { - fields.push(Field::new( - format!("int_col_{col_idx}"), - DataType::Int64, - true, - )); - col_idx += 1; - } - // String columns - for _ in 0..args.gen_string_cols { - fields.push(Field::new( - format!("str_col_{col_idx}"), - DataType::Utf8, - true, - )); - col_idx += 1; - } - // Decimal columns - for _ in 0..args.gen_decimal_cols { - fields.push(Field::new( - format!("dec_col_{col_idx}"), - DataType::Decimal128(18, 2), - true, - )); - col_idx += 1; - } - // Date columns - for _ in 0..args.gen_date_cols { - fields.push(Field::new( - format!("date_col_{col_idx}"), - DataType::Date32, - true, - )); - col_idx += 1; - } - - let schema = Arc::new(Schema::new(fields)); - let mut batches = Vec::new(); - let mut rng = rand::rng(); - let mut remaining = args.gen_rows; - - while remaining > 0 { - let batch_rows = remaining.min(args.batch_size); - remaining -= batch_rows; - - let mut columns: Vec> = Vec::new(); - - // Int64 columns - for _ in 0..args.gen_int_cols { - let mut builder = Int64Builder::with_capacity(batch_rows); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); - } else { - builder.append_value(rng.random_range(0..1_000_000i64)); - } - } - columns.push(Arc::new(builder.finish())); - } - // String columns - for _ in 0..args.gen_string_cols { - let mut builder = - StringBuilder::with_capacity(batch_rows, batch_rows * args.gen_avg_string_len); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); - } else { - let len = rng.random_range(1..args.gen_avg_string_len * 2); - let s: String = (0..len) - .map(|_| rng.random_range(b'a'..=b'z') as char) - .collect(); - builder.append_value(&s); - } - } - columns.push(Arc::new(builder.finish())); - } - // Decimal columns - for _ in 0..args.gen_decimal_cols { - let mut builder = Decimal128Builder::with_capacity(batch_rows) - .with_precision_and_scale(18, 2) - .unwrap(); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); - } else { - builder.append_value(rng.random_range(0..100_000_000i128)); - } - } - columns.push(Arc::new(builder.finish())); - } - // Date columns - for _ in 0..args.gen_date_cols { - let mut builder = Date32Builder::with_capacity(batch_rows); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); +fn collect_parquet_paths(path: &PathBuf) -> Vec { + if path.is_dir() { + let mut files: Vec = fs::read_dir(path) + .unwrap_or_else(|e| panic!("Failed to read directory {}: {}", path.display(), e)) + .filter_map(|entry| { + let p = entry.ok()?.path(); + if p.extension().and_then(|e| e.to_str()) == Some("parquet") { + Some(p) } else { - builder.append_value(rng.random_range(0..20000i32)); + None } - } - columns.push(Arc::new(builder.finish())); + }) + .collect(); + files.sort(); + if files.is_empty() { + panic!("No .parquet files found in {}", path.display()); } - - let batch = RecordBatch::try_new(Arc::clone(&schema), columns).unwrap(); - batches.push(batch); + files + } else { + vec![path.clone()] } - - println!( - "Generated {} batches ({} rows)", - batches.len(), - args.gen_rows - ); - batches } fn run_shuffle_write( - batches: &[RecordBatch], + input_path: &PathBuf, schema: &SchemaRef, codec: &CompressionCodec, hash_col_indices: &[usize], @@ -554,40 +322,61 @@ fn run_shuffle_write( schema, ); - let partitions = &[batches.to_vec()]; - let exec = ShuffleWriterExec::try_new( - Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(partitions, Arc::clone(schema), None).unwrap(), - ))), - partitioning, - codec.clone(), - data_file.to_string(), - index_file.to_string(), - false, - args.write_buffer_size, - ) - .expect("Failed to create ShuffleWriterExec"); - - let config = SessionConfig::new().with_batch_size(args.batch_size); - let mut runtime_builder = RuntimeEnvBuilder::new(); - if let Some(mem_limit) = args.memory_limit { - runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); - } - let runtime_env = Arc::new(runtime_builder.build().unwrap()); - let ctx = SessionContext::new_with_config_rt(config, runtime_env); - let task_ctx = ctx.task_ctx(); - - let start = Instant::now(); - let stream = exec.execute(0, task_ctx).unwrap(); let rt = tokio::runtime::Runtime::new().unwrap(); - rt.block_on(collect(stream)).unwrap(); - start.elapsed().as_secs_f64() + rt.block_on(async { + let config = SessionConfig::new().with_batch_size(args.batch_size); + let mut runtime_builder = RuntimeEnvBuilder::new(); + if let Some(mem_limit) = args.memory_limit { + runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); + } + let runtime_env = Arc::new(runtime_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + + let path_str = input_path.to_str().unwrap(); + let mut df = ctx + .read_parquet(path_str, ParquetReadOptions::default()) + .await + .expect("Failed to create Parquet scan"); + if args.limit > 0 { + df = df.limit(0, Some(args.limit)).unwrap(); + } + + let parquet_plan = df + .create_physical_plan() + .await + .expect("Failed to create physical plan"); + + // ShuffleWriterExec reads from a single input partition + let input: Arc = + if parquet_plan.properties().output_partitioning().partition_count() > 1 { + Arc::new(CoalescePartitionsExec::new(parquet_plan)) + } else { + parquet_plan + }; + + let exec = ShuffleWriterExec::try_new( + input, + partitioning, + codec.clone(), + data_file.to_string(), + index_file.to_string(), + false, + args.write_buffer_size, + args.max_buffered_batches, + ) + .expect("Failed to create ShuffleWriterExec"); + + let task_ctx = ctx.task_ctx(); + let start = Instant::now(); + let stream = exec.execute(0, task_ctx).unwrap(); + collect(stream).await.unwrap(); + start.elapsed().as_secs_f64() + }) } fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> f64 { let start = Instant::now(); - // Read index file to get partition offsets let index_bytes = fs::read(index_file).expect("Failed to read index file"); let num_offsets = index_bytes.len() / 8; let offsets: Vec = (0..num_offsets) @@ -597,34 +386,27 @@ fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> }) .collect(); - // Read data file let data_bytes = fs::read(data_file).expect("Failed to read data file"); let mut total_rows = 0usize; let mut total_batches = 0usize; - // Decode each partition's data for p in 0..num_partitions.min(offsets.len().saturating_sub(1)) { let start_offset = offsets[p] as usize; let end_offset = offsets[p + 1] as usize; if start_offset >= end_offset { - continue; // Empty partition + continue; } - // Read all IPC blocks within this partition let mut offset = start_offset; while offset < end_offset { - // First 8 bytes: IPC length let ipc_length = u64::from_le_bytes(data_bytes[offset..offset + 8].try_into().unwrap()) as usize; - - // Skip 8-byte length prefix, then 8 bytes of field_count + codec header let block_data = &data_bytes[offset + 16..offset + 8 + ipc_length]; let batch = read_ipc_compressed(block_data).expect("Failed to decode shuffle block"); total_rows += batch.num_rows(); total_batches += 1; - offset += 8 + ipc_length; } } @@ -686,7 +468,7 @@ fn parse_hash_columns(s: &str) -> Vec { .collect() } -fn describe_schema(schema: &Schema) -> String { +fn describe_schema(schema: &arrow::datatypes::Schema) -> String { let mut counts = std::collections::HashMap::new(); for field in schema.fields() { let type_name = match field.data_type() { diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 42290c5510..4cc09e9679 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -124,6 +124,8 @@ pub(crate) struct MultiPartitionShuffleRepartitioner { tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, + /// Maximum number of batches to buffer before spilling (0 = no limit) + max_buffered_batches: usize, } impl MultiPartitionShuffleRepartitioner { @@ -140,6 +142,7 @@ impl MultiPartitionShuffleRepartitioner { codec: CompressionCodec, tracing_enabled: bool, write_buffer_size: usize, + max_buffered_batches: usize, ) -> datafusion::common::Result { let num_output_partitions = partitioning.partition_count(); assert_ne!( @@ -189,6 +192,7 @@ impl MultiPartitionShuffleRepartitioner { reservation, tracing_enabled, write_buffer_size, + max_buffered_batches, }) } @@ -397,6 +401,13 @@ impl MultiPartitionShuffleRepartitioner { partition_row_indices: &[u32], partition_starts: &[u32], ) -> datafusion::common::Result<()> { + // Spill before buffering if we've reached the configured batch count limit. + if self.max_buffered_batches > 0 + && self.buffered_batches.len() >= self.max_buffered_batches + { + self.spill()?; + } + let mut mem_growth: usize = input.get_array_memory_size(); let buffered_partition_idx = self.buffered_batches.len() as u32; self.buffered_batches.push(input); diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index e649aaac69..95a09610a4 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -68,6 +68,8 @@ pub struct ShuffleWriterExec { tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, + /// Maximum number of batches to buffer before spilling (0 = no limit) + max_buffered_batches: usize, } impl ShuffleWriterExec { @@ -81,6 +83,7 @@ impl ShuffleWriterExec { output_index_file: String, tracing_enabled: bool, write_buffer_size: usize, + max_buffered_batches: usize, ) -> Result { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), @@ -99,6 +102,7 @@ impl ShuffleWriterExec { codec, tracing_enabled, write_buffer_size, + max_buffered_batches, }) } } @@ -163,6 +167,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.output_index_file.clone(), self.tracing_enabled, self.write_buffer_size, + self.max_buffered_batches, )?)), _ => panic!("ShuffleWriterExec wrong number of children"), } @@ -190,6 +195,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.codec.clone(), self.tracing_enabled, self.write_buffer_size, + self.max_buffered_batches, ) .map_err(|e| ArrowError::ExternalError(Box::new(e))), ) @@ -210,6 +216,7 @@ async fn external_shuffle( codec: CompressionCodec, tracing_enabled: bool, write_buffer_size: usize, + max_buffered_batches: usize, ) -> Result { with_trace_async("external_shuffle", tracing_enabled, || async { let schema = input.schema(); @@ -238,6 +245,7 @@ async fn external_shuffle( codec, tracing_enabled, write_buffer_size, + max_buffered_batches, )?), }; @@ -362,6 +370,7 @@ mod test { CompressionCodec::Lz4Frame, false, 1024 * 1024, // write_buffer_size: 1MB default + 0, // max_buffered_batches: no limit ) .unwrap(); @@ -466,6 +475,7 @@ mod test { "/tmp/index.out".to_string(), false, 1024 * 1024, // write_buffer_size: 1MB default + 0, // max_buffered_batches: no limit ) .unwrap(); @@ -525,6 +535,7 @@ mod test { index_file.clone(), false, 1024 * 1024, + 0, // max_buffered_batches: no limit ) .unwrap(); diff --git a/native/shuffle/src/writers/partition_writer.rs b/native/shuffle/src/writers/partition_writer.rs index 48017871db..4de307de62 100644 --- a/native/shuffle/src/writers/partition_writer.rs +++ b/native/shuffle/src/writers/partition_writer.rs @@ -26,7 +26,6 @@ use std::fs::{File, OpenOptions}; struct SpillFile { temp_file: RefCountedTempFile, - file: File, } pub(crate) struct PartitionWriter { @@ -53,26 +52,28 @@ impl PartitionWriter { runtime: &RuntimeEnv, ) -> datafusion::common::Result<()> { if self.spill_file.is_none() { - // Spill file is not yet created, create it let spill_file = runtime .disk_manager .create_tmp_file("shuffle writer spill")?; - let spill_data = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(spill_file.path()) - .map_err(|e| { - DataFusionError::Execution(format!("Error occurred while spilling {e}")) - })?; + // Create the file (truncating any pre-existing content) + File::create(spill_file.path()).map_err(|e| { + DataFusionError::Execution(format!("Error occurred while spilling {e}")) + })?; self.spill_file = Some(SpillFile { temp_file: spill_file, - file: spill_data, }); } Ok(()) } + fn open_spill_file_for_append(&self) -> datafusion::common::Result { + OpenOptions::new() + .write(true) + .append(true) + .open(self.spill_file.as_ref().unwrap().temp_file.path()) + .map_err(|e| DataFusionError::Execution(format!("Error occurred while spilling {e}"))) + } + pub(crate) fn spill( &mut self, iter: &mut PartitionedBatchIterator, @@ -84,10 +85,13 @@ impl PartitionWriter { if let Some(batch) = iter.next() { self.ensure_spill_file_created(runtime)?; + // Open the file for this spill and close it when done, so we don't + // hold open one FD per partition across multiple spill events. + let mut spill_data = self.open_spill_file_for_append()?; let total_bytes_written = { let mut buf_batch_writer = BufBatchWriter::new( &mut self.shuffle_block_writer, - &mut self.spill_file.as_mut().unwrap().file, + &mut spill_data, write_buffer_size, batch_size, ); @@ -104,6 +108,7 @@ impl PartitionWriter { buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; bytes_written }; + // spill_data is dropped here, closing the file descriptor Ok(total_bytes_written) } else { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 3fc222bd19..a80d8b2fa4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -192,6 +192,8 @@ class CometNativeShuffleWriter[K, V]( CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL.get) shuffleWriterBuilder.setWriteBufferSize( CometConf.COMET_SHUFFLE_WRITE_BUFFER_SIZE.get().max(Int.MaxValue).toInt) + shuffleWriterBuilder.setMaxBufferedBatches( + CometConf.COMET_SHUFFLE_MAX_BUFFERED_BATCHES.get()) outputPartitioning match { case p if isSinglePartitioning(p) => From 2ef57e79d4e9202d4fb1a70f62f5d8b717ca67b4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Mar 2026 12:05:23 -0600 Subject: [PATCH 06/21] cargo fmt --- native/shuffle/src/bin/shuffle_bench.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 7f919e7674..ec1227a2ee 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -340,12 +340,16 @@ fn run_shuffle_write( .expect("Failed to create physical plan"); // ShuffleWriterExec reads from a single input partition - let input: Arc = - if parquet_plan.properties().output_partitioning().partition_count() > 1 { - Arc::new(CoalescePartitionsExec::new(parquet_plan)) - } else { - parquet_plan - }; + let input: Arc = if parquet_plan + .properties() + .output_partitioning() + .partition_count() + > 1 + { + Arc::new(CoalescePartitionsExec::new(parquet_plan)) + } else { + parquet_plan + }; let exec = ShuffleWriterExec::try_new( input, From 9136e104e87cae0c8920fcce790064de5716d425 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Mar 2026 12:08:20 -0600 Subject: [PATCH 07/21] prettier --- native/shuffle/README.md | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/native/shuffle/README.md b/native/shuffle/README.md index 0333ddbe8e..74b8dbe656 100644 --- a/native/shuffle/README.md +++ b/native/shuffle/README.md @@ -41,23 +41,23 @@ cargo run --release --features shuffle-bench --bin shuffle_bench -- \ ### Options -| Option | Default | Description | -|---|---|---| -| `--input` | *(required)* | Path to a Parquet file or directory of Parquet files | -| `--partitions` | `200` | Number of output shuffle partitions | -| `--partitioning` | `hash` | Partitioning scheme: `hash`, `single`, `round-robin` | -| `--hash-columns` | `0` | Comma-separated column indices to hash on (e.g. `0,3`) | -| `--codec` | `zstd` | Compression codec: `none`, `lz4`, `zstd`, `snappy` | -| `--zstd-level` | `1` | Zstd compression level (1–22) | -| `--batch-size` | `8192` | Batch size for reading Parquet data | -| `--memory-limit` | *(none)* | Memory limit in bytes; triggers spilling when exceeded | -| `--max-buffered-batches` | `0` | Max batches to buffer before spilling (0 = memory-pool-only) | -| `--write-buffer-size` | `1048576` | Write buffer size in bytes | -| `--limit` | `0` | Limit rows processed per iteration (0 = no limit) | -| `--iterations` | `1` | Number of timed iterations | -| `--warmup` | `0` | Number of warmup iterations before timing | -| `--read-back` | `false` | Also benchmark reading back the shuffle output | -| `--output-dir` | `/tmp/comet_shuffle_bench` | Directory for temporary shuffle output files | +| Option | Default | Description | +| ------------------------ | -------------------------- | ------------------------------------------------------------ | +| `--input` | _(required)_ | Path to a Parquet file or directory of Parquet files | +| `--partitions` | `200` | Number of output shuffle partitions | +| `--partitioning` | `hash` | Partitioning scheme: `hash`, `single`, `round-robin` | +| `--hash-columns` | `0` | Comma-separated column indices to hash on (e.g. `0,3`) | +| `--codec` | `zstd` | Compression codec: `none`, `lz4`, `zstd`, `snappy` | +| `--zstd-level` | `1` | Zstd compression level (1–22) | +| `--batch-size` | `8192` | Batch size for reading Parquet data | +| `--memory-limit` | _(none)_ | Memory limit in bytes; triggers spilling when exceeded | +| `--max-buffered-batches` | `0` | Max batches to buffer before spilling (0 = memory-pool-only) | +| `--write-buffer-size` | `1048576` | Write buffer size in bytes | +| `--limit` | `0` | Limit rows processed per iteration (0 = no limit) | +| `--iterations` | `1` | Number of timed iterations | +| `--warmup` | `0` | Number of warmup iterations before timing | +| `--read-back` | `false` | Also benchmark reading back the shuffle output | +| `--output-dir` | `/tmp/comet_shuffle_bench` | Directory for temporary shuffle output files | ### Profiling with flamegraph From 7e16819013e744eecd548483fcf414798f071744 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Mar 2026 12:19:44 -0600 Subject: [PATCH 08/21] machete --- native/Cargo.lock | 1 - native/shuffle/Cargo.toml | 5 ++--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 88007af70a..2c117631a4 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1996,7 +1996,6 @@ dependencies = [ "log", "lz4_flex 0.13.0", "parquet", - "rand 0.10.0", "simd-adler32", "snap", "tempfile", diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 14ed5f60a0..a5982c05fa 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -43,9 +43,8 @@ itertools = "0.14.0" jni = "0.21" log = "0.4" lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] } -# parquet and rand are only used by the shuffle_bench binary (shuffle-bench feature) +# parquet is only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } -rand = { workspace = true, optional = true } simd-adler32 = "0.3.9" snap = "1.1" tokio = { version = "1", features = ["rt-multi-thread"] } @@ -58,7 +57,7 @@ itertools = "0.14.0" tempfile = "3.26.0" [features] -shuffle-bench = ["clap", "parquet", "rand"] +shuffle-bench = ["clap", "parquet"] [lib] name = "datafusion_comet_shuffle" From 22fe80491e111d1f261988a7bd06f73764ba8360 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Mar 2026 16:50:51 -0600 Subject: [PATCH 09/21] feat: add --concurrent-tasks flag to shuffle benchmark Spawns N parallel shuffle tasks to simulate executor parallelism. Each task reads the same input and writes to its own output files. Extracts core shuffle logic into shared async helper to avoid code duplication between single and concurrent paths. --- native/shuffle/src/bin/shuffle_bench.rs | 231 +++++++++++++++++------- 1 file changed, 169 insertions(+), 62 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index ec1227a2ee..bb0952365b 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -115,6 +115,11 @@ struct Args { /// Limit rows processed per iteration (0 = no limit) #[arg(long, default_value_t = 0)] limit: usize, + + /// Number of concurrent shuffle tasks to simulate executor parallelism. + /// Each task reads the same input and writes to its own output files. + #[arg(long, default_value_t = 1)] + concurrent_tasks: usize, } fn main() { @@ -146,6 +151,9 @@ fn main() { if let Some(mem_limit) = args.memory_limit { println!("Memory limit: {}", format_bytes(mem_limit)); } + if args.concurrent_tasks > 1 { + println!("Concurrent: {} tasks", args.concurrent_tasks); + } println!( "Iterations: {} (warmup: {})", args.iterations, args.warmup @@ -165,15 +173,19 @@ fn main() { format!("iter {}/{}", i - args.warmup + 1, args.iterations) }; - let write_elapsed = run_shuffle_write( - &args.input, - &schema, - &codec, - &hash_col_indices, - &args, - data_file.to_str().unwrap(), - index_file.to_str().unwrap(), - ); + let write_elapsed = if args.concurrent_tasks > 1 { + run_concurrent_shuffle_writes(&args.input, &schema, &codec, &hash_col_indices, &args) + } else { + run_shuffle_write( + &args.input, + &schema, + &codec, + &hash_col_indices, + &args, + data_file.to_str().unwrap(), + index_file.to_str().unwrap(), + ) + }; let data_size = fs::metadata(&data_file).map(|m| m.len()).unwrap_or(0); if !is_warmup { @@ -182,9 +194,11 @@ fn main() { } print!(" [{label}] write: {:.3}s", write_elapsed); - print!(" output: {}", format_bytes(data_size as usize)); + if args.concurrent_tasks <= 1 { + print!(" output: {}", format_bytes(data_size as usize)); + } - if args.read_back { + if args.read_back && args.concurrent_tasks <= 1 { let read_elapsed = run_shuffle_read( data_file.to_str().unwrap(), index_file.to_str().unwrap(), @@ -203,8 +217,7 @@ fn main() { println!("=== Results ==="); let avg_write = write_times.iter().sum::() / write_times.len() as f64; - let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; - let write_throughput_rows = total_rows as f64 / avg_write; + let write_throughput_rows = (total_rows as f64 * args.concurrent_tasks as f64) / avg_write; println!("Write:"); println!(" avg time: {:.3}s", avg_write); @@ -217,15 +230,20 @@ fn main() { println!(" min/max: {:.3}s / {:.3}s", min, max); } println!( - " throughput: {} rows/s", - format_number(write_throughput_rows as usize) - ); - println!( - " output size: {}", - format_bytes(avg_data_size as usize) + " throughput: {} rows/s (total across {} tasks)", + format_number(write_throughput_rows as usize), + args.concurrent_tasks ); + if args.concurrent_tasks <= 1 { + let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; + println!( + " output size: {}", + format_bytes(avg_data_size as usize) + ); + } if !read_times.is_empty() { + let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; let avg_read = read_times.iter().sum::() / read_times.len() as f64; let read_throughput_bytes = avg_data_size as f64 / avg_read; @@ -317,55 +335,144 @@ fn run_shuffle_write( let rt = tokio::runtime::Runtime::new().unwrap(); rt.block_on(async { - let config = SessionConfig::new().with_batch_size(args.batch_size); - let mut runtime_builder = RuntimeEnvBuilder::new(); - if let Some(mem_limit) = args.memory_limit { - runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); - } - let runtime_env = Arc::new(runtime_builder.build().unwrap()); - let ctx = SessionContext::new_with_config_rt(config, runtime_env); - - let path_str = input_path.to_str().unwrap(); - let mut df = ctx - .read_parquet(path_str, ParquetReadOptions::default()) - .await - .expect("Failed to create Parquet scan"); - if args.limit > 0 { - df = df.limit(0, Some(args.limit)).unwrap(); - } - - let parquet_plan = df - .create_physical_plan() - .await - .expect("Failed to create physical plan"); - - // ShuffleWriterExec reads from a single input partition - let input: Arc = if parquet_plan - .properties() - .output_partitioning() - .partition_count() - > 1 - { - Arc::new(CoalescePartitionsExec::new(parquet_plan)) - } else { - parquet_plan - }; - - let exec = ShuffleWriterExec::try_new( - input, - partitioning, + let start = Instant::now(); + execute_shuffle_write( + input_path.to_str().unwrap(), codec.clone(), + partitioning, + args.batch_size, + args.memory_limit, + args.write_buffer_size, + args.limit, data_file.to_string(), index_file.to_string(), - false, - args.write_buffer_size, ) - .expect("Failed to create ShuffleWriterExec"); + .await + .unwrap(); + start.elapsed().as_secs_f64() + }) +} + +/// Core async shuffle write logic shared by single and concurrent paths. +async fn execute_shuffle_write( + input_path: &str, + codec: CompressionCodec, + partitioning: CometPartitioning, + batch_size: usize, + memory_limit: Option, + write_buffer_size: usize, + limit: usize, + data_file: String, + index_file: String, +) -> datafusion::common::Result<()> { + let config = SessionConfig::new().with_batch_size(batch_size); + let mut runtime_builder = RuntimeEnvBuilder::new(); + if let Some(mem_limit) = memory_limit { + runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); + } + let runtime_env = Arc::new(runtime_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + + let mut df = ctx + .read_parquet(input_path, ParquetReadOptions::default()) + .await + .expect("Failed to create Parquet scan"); + if limit > 0 { + df = df.limit(0, Some(limit)).unwrap(); + } + + let parquet_plan = df + .create_physical_plan() + .await + .expect("Failed to create physical plan"); + + let input: Arc = if parquet_plan + .properties() + .output_partitioning() + .partition_count() + > 1 + { + Arc::new(CoalescePartitionsExec::new(parquet_plan)) + } else { + parquet_plan + }; + + let exec = ShuffleWriterExec::try_new( + input, + partitioning, + codec, + data_file, + index_file, + false, + write_buffer_size, + ) + .expect("Failed to create ShuffleWriterExec"); + + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx).unwrap(); + collect(stream).await.unwrap(); + Ok(()) +} - let task_ctx = ctx.task_ctx(); +/// Run N concurrent shuffle writes to simulate executor parallelism. +/// Returns wall-clock time for all tasks to complete. +fn run_concurrent_shuffle_writes( + input_path: &PathBuf, + schema: &SchemaRef, + codec: &CompressionCodec, + hash_col_indices: &[usize], + args: &Args, +) -> f64 { + let rt = tokio::runtime::Runtime::new().unwrap(); + rt.block_on(async { let start = Instant::now(); - let stream = exec.execute(0, task_ctx).unwrap(); - collect(stream).await.unwrap(); + + let mut handles = Vec::with_capacity(args.concurrent_tasks); + for task_id in 0..args.concurrent_tasks { + let task_dir = args.output_dir.join(format!("task_{task_id}")); + fs::create_dir_all(&task_dir).expect("Failed to create task output directory"); + let data_file = task_dir.join("data.out").to_str().unwrap().to_string(); + let index_file = task_dir.join("index.out").to_str().unwrap().to_string(); + + let input_str = input_path.to_str().unwrap().to_string(); + let codec = codec.clone(); + let partitioning = build_partitioning( + &args.partitioning, + args.partitions, + hash_col_indices, + schema, + ); + let batch_size = args.batch_size; + let memory_limit = args.memory_limit; + let write_buffer_size = args.write_buffer_size; + let limit = args.limit; + + handles.push(tokio::spawn(async move { + execute_shuffle_write( + &input_str, + codec, + partitioning, + batch_size, + memory_limit, + write_buffer_size, + limit, + data_file, + index_file, + ) + .await + .unwrap(); + })); + } + + for handle in handles { + handle.await.expect("Task panicked"); + } + + for task_id in 0..args.concurrent_tasks { + let task_dir = args.output_dir.join(format!("task_{task_id}")); + let _ = fs::remove_dir_all(&task_dir); + } + start.elapsed().as_secs_f64() }) } From a82b36959c9bc716eb840ed6348dabbc1fecffca Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 09:22:06 -0600 Subject: [PATCH 10/21] feat: add IPC stream shuffle writer as alternative to block format Add IpcStreamWriter that writes standard Arrow IPC streams with built-in body compression (LZ4_FRAME/ZSTD), where the schema is written once per stream instead of per batch. Add spark.comet.exec.shuffle.format config ("block" or "ipc_stream") to toggle between the custom block format and the new IPC stream format. Wire the setting through protobuf, planner, and both single-partition and multi-partition shuffle writers including spill support. Benchmark results (200 partitions, LZ4, 10M rows TPC-H lineitem): - Block: 2.64M rows/s, 609 MiB - IPC stream: 2.60M rows/s, 634 MiB - Single partition: IPC stream is 2x faster --- .../scala/org/apache/comet/CometConf.scala | 12 + native/Cargo.lock | 1 + native/Cargo.toml | 2 +- native/core/src/execution/planner.rs | 11 +- native/proto/src/proto/operator.proto | 7 + native/shuffle/benches/shuffle_writer.rs | 1 + native/shuffle/src/bin/shuffle_bench.rs | 30 ++- native/shuffle/src/ipc.rs | 13 + native/shuffle/src/lib.rs | 15 +- .../src/partitioners/multi_partition.rs | 92 +++++-- .../src/partitioners/single_partition.rs | 142 ++++++++--- native/shuffle/src/shuffle_writer.rs | 14 +- .../shuffle/src/writers/buf_batch_writer.rs | 6 - .../shuffle/src/writers/ipc_stream_writer.rs | 236 ++++++++++++++++++ native/shuffle/src/writers/mod.rs | 2 + native/shuffle/src/writers/spill.rs | 68 +++-- .../shuffle/CometNativeShuffleWriter.scala | 8 +- 17 files changed, 573 insertions(+), 87 deletions(-) create mode 100644 native/shuffle/src/writers/ipc_stream_writer.rs diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index bfe90181ff..b15b6a3339 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -430,6 +430,18 @@ object CometConf extends ShimCometConf { "The maximum number of columns to hash for round robin partitioning must be non-negative.") .createWithDefault(0) + val COMET_EXEC_SHUFFLE_FORMAT: ConfigEntry[String] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.format") + .category(CATEGORY_SHUFFLE) + .doc( + "The format used for writing shuffle data. 'block' uses a custom block format with " + + "per-batch headers and supports all compression codecs (lz4, zstd, snappy). " + + "'ipc_stream' uses standard Arrow IPC stream format with the schema written once " + + "and supports lz4 and zstd compression (snappy is not supported).") + .stringConf + .checkValues(Set("block", "ipc_stream")) + .createWithDefault("block") + val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") .category(CATEGORY_SHUFFLE) diff --git a/native/Cargo.lock b/native/Cargo.lock index 2c117631a4..7e16f06fdf 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -333,6 +333,7 @@ dependencies = [ "arrow-select", "flatbuffers", "lz4_flex 0.12.1", + "zstd", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index c626743be1..3fb087e443 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,7 +34,7 @@ edition = "2021" rust-version = "1.88" [workspace.dependencies] -arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz", "ipc_compression"] } async-trait = { version = "0.1" } bytes = { version = "1.11.1" } parquet = { version = "57.3.0", default-features = false, features = ["experimental"] } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5af31fcc22..b5fe84cb4e 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -73,7 +73,7 @@ use datafusion_comet_spark_expr::{ use iceberg::expr::Bind; use crate::execution::operators::ExecutionError::GeneralError; -use crate::execution::shuffle::{CometPartitioning, CompressionCodec}; +use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleFormat}; use crate::execution::spark_plan::SparkPlan; use crate::parquet::parquet_support::prepare_object_store_with_configs; use datafusion::common::scalar::ScalarStructBuilder; @@ -116,7 +116,8 @@ use datafusion_comet_proto::{ spark_operator::{ self, lower_window_frame_bound::LowerFrameBoundStruct, operator::OpStruct, upper_window_frame_bound::UpperFrameBoundStruct, BuildSide, - CompressionCodec as SparkCompressionCodec, JoinType, Operator, WindowFrameType, + CompressionCodec as SparkCompressionCodec, JoinType, Operator, + ShuffleWriterFormat as SparkShuffleWriterFormat, WindowFrameType, }, spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; @@ -1351,11 +1352,17 @@ impl PhysicalPlanner { ))), }?; + let format = match writer.format.try_into() { + Ok(SparkShuffleWriterFormat::IpcStream) => ShuffleFormat::IpcStream, + _ => ShuffleFormat::Block, + }; + let write_buffer_size = writer.write_buffer_size as usize; let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( Arc::clone(&child.native_plan), partitioning, codec, + format, writer.output_data_file.clone(), writer.output_index_file.clone(), writer.tracing_enabled, diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 344b9f0f21..1dc14b82d4 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -284,6 +284,11 @@ enum CompressionCodec { Snappy = 3; } +enum ShuffleWriterFormat { + BLOCK = 0; + IPC_STREAM = 1; +} + message ShuffleWriter { spark.spark_partitioning.Partitioning partitioning = 1; string output_data_file = 3; @@ -294,6 +299,8 @@ message ShuffleWriter { // Size of the write buffer in bytes used when writing shuffle data to disk. // Larger values may improve write performance but use more memory. int32 write_buffer_size = 8; + // The shuffle data format to use (Block or IpcStream). + ShuffleWriterFormat format = 9; } message ParquetWriter { diff --git a/native/shuffle/benches/shuffle_writer.rs b/native/shuffle/benches/shuffle_writer.rs index 27abd919fa..612a7084bc 100644 --- a/native/shuffle/benches/shuffle_writer.rs +++ b/native/shuffle/benches/shuffle_writer.rs @@ -149,6 +149,7 @@ fn create_shuffle_writer_exec( ))), partitioning, compression_codec, + datafusion_comet_shuffle::ShuffleFormat::Block, "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), false, diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index bb0952365b..14e8d5d841 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -120,6 +120,10 @@ struct Args { /// Each task reads the same input and writes to its own output files. #[arg(long, default_value_t = 1)] concurrent_tasks: usize, + + /// Shuffle format: block or ipc_stream + #[arg(long, default_value = "block")] + format: String, } fn main() { @@ -133,6 +137,7 @@ fn main() { let (schema, total_rows) = read_parquet_metadata(&args.input, args.limit); let codec = parse_codec(&args.codec, args.zstd_level); + let format = parse_format(&args.format); let hash_col_indices = parse_hash_columns(&args.hash_columns); println!("=== Shuffle Benchmark ==="); @@ -147,6 +152,7 @@ fn main() { println!("Partitioning: {}", args.partitioning); println!("Partitions: {}", args.partitions); println!("Codec: {:?}", codec); + println!("Format: {:?}", format); println!("Hash columns: {:?}", hash_col_indices); if let Some(mem_limit) = args.memory_limit { println!("Memory limit: {}", format_bytes(mem_limit)); @@ -174,12 +180,20 @@ fn main() { }; let write_elapsed = if args.concurrent_tasks > 1 { - run_concurrent_shuffle_writes(&args.input, &schema, &codec, &hash_col_indices, &args) + run_concurrent_shuffle_writes( + &args.input, + &schema, + &codec, + &format, + &hash_col_indices, + &args, + ) } else { run_shuffle_write( &args.input, &schema, &codec, + &format, &hash_col_indices, &args, data_file.to_str().unwrap(), @@ -321,6 +335,7 @@ fn run_shuffle_write( input_path: &PathBuf, schema: &SchemaRef, codec: &CompressionCodec, + format: &datafusion_comet_shuffle::ShuffleFormat, hash_col_indices: &[usize], args: &Args, data_file: &str, @@ -339,6 +354,7 @@ fn run_shuffle_write( execute_shuffle_write( input_path.to_str().unwrap(), codec.clone(), + format.clone(), partitioning, args.batch_size, args.memory_limit, @@ -357,6 +373,7 @@ fn run_shuffle_write( async fn execute_shuffle_write( input_path: &str, codec: CompressionCodec, + format: datafusion_comet_shuffle::ShuffleFormat, partitioning: CometPartitioning, batch_size: usize, memory_limit: Option, @@ -401,6 +418,7 @@ async fn execute_shuffle_write( input, partitioning, codec, + format, data_file, index_file, false, @@ -420,6 +438,7 @@ fn run_concurrent_shuffle_writes( input_path: &PathBuf, schema: &SchemaRef, codec: &CompressionCodec, + format: &datafusion_comet_shuffle::ShuffleFormat, hash_col_indices: &[usize], args: &Args, ) -> f64 { @@ -436,6 +455,7 @@ fn run_concurrent_shuffle_writes( let input_str = input_path.to_str().unwrap().to_string(); let codec = codec.clone(); + let format = format.clone(); let partitioning = build_partitioning( &args.partitioning, args.partitions, @@ -451,6 +471,7 @@ fn run_concurrent_shuffle_writes( execute_shuffle_write( &input_str, codec, + format, partitioning, batch_size, memory_limit, @@ -551,6 +572,13 @@ fn build_partitioning( } } +fn parse_format(format: &str) -> datafusion_comet_shuffle::ShuffleFormat { + match format.to_lowercase().as_str() { + "ipc_stream" | "ipc-stream" | "stream" => datafusion_comet_shuffle::ShuffleFormat::IpcStream, + _ => datafusion_comet_shuffle::ShuffleFormat::Block, + } +} + fn parse_codec(codec: &str, zstd_level: i32) -> CompressionCodec { match codec.to_lowercase().as_str() { "none" => CompressionCodec::None, diff --git a/native/shuffle/src/ipc.rs b/native/shuffle/src/ipc.rs index 81ee41332a..8e3bfc3ede 100644 --- a/native/shuffle/src/ipc.rs +++ b/native/shuffle/src/ipc.rs @@ -20,6 +20,19 @@ use arrow::ipc::reader::StreamReader; use datafusion::common::DataFusionError; use datafusion::error::Result; +/// Reads all record batches from a standard Arrow IPC stream. +/// +/// This is the counterpart to [`crate::writers::IpcStreamWriter`]. The input +/// should be a complete Arrow IPC stream (schema + batch messages + EOS marker), +/// optionally with Arrow IPC body compression (LZ4_FRAME or ZSTD). +pub fn read_ipc_stream(bytes: &[u8]) -> Result> { + let reader = StreamReader::try_new(bytes, None)?; + reader + .into_iter() + .map(|r| r.map_err(|e| e.into())) + .collect() +} + pub fn read_ipc_compressed(bytes: &[u8]) -> Result { match &bytes[0..4] { b"SNAP" => { diff --git a/native/shuffle/src/lib.rs b/native/shuffle/src/lib.rs index f29588f2e1..12978ce37b 100644 --- a/native/shuffle/src/lib.rs +++ b/native/shuffle/src/lib.rs @@ -24,6 +24,17 @@ pub mod spark_unsafe; pub(crate) mod writers; pub use comet_partitioning::CometPartitioning; -pub use ipc::read_ipc_compressed; +pub use ipc::{read_ipc_compressed, read_ipc_stream}; pub use shuffle_writer::ShuffleWriterExec; -pub use writers::{CompressionCodec, ShuffleBlockWriter}; +pub use writers::{CompressionCodec, IpcStreamWriter, ShuffleBlockWriter}; + +/// The format used for writing shuffle data. +#[derive(Debug, Clone)] +pub enum ShuffleFormat { + /// Custom block format: each batch is a self-contained block with a + /// length-prefix + field-count + codec header followed by compressed Arrow IPC. + Block, + /// Standard Arrow IPC stream format: schema written once, batches as IPC + /// messages, with optional Arrow IPC body compression (LZ4_FRAME or ZSTD). + IpcStream, +} diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 655bee3511..3636ef7170 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -21,8 +21,12 @@ use crate::partitioners::partitioned_batch_iterator::{ }; use crate::partitioners::ShufflePartitioner; use crate::writers::{BufBatchWriter, PartitionWriter}; -use crate::{comet_partitioning, CometPartitioning, CompressionCodec, ShuffleBlockWriter}; +use crate::{ + comet_partitioning, CometPartitioning, CompressionCodec, IpcStreamWriter, ShuffleBlockWriter, + ShuffleFormat, +}; use arrow::array::{ArrayRef, RecordBatch}; +use arrow::compute::kernels::coalesce::BatchCoalescer; use arrow::datatypes::SchemaRef; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::common::DataFusionError; @@ -108,6 +112,7 @@ impl ScratchSpace { pub(crate) struct MultiPartitionShuffleRepartitioner { output_data_file: String, output_index_file: String, + schema: SchemaRef, buffered_batches: Vec, partition_indices: Vec>, partition_writers: Vec, @@ -122,6 +127,8 @@ pub(crate) struct MultiPartitionShuffleRepartitioner { batch_size: usize, /// Reservation for repartitioning reservation: MemoryReservation, + codec: CompressionCodec, + format: ShuffleFormat, tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, @@ -139,6 +146,7 @@ impl MultiPartitionShuffleRepartitioner { runtime: Arc, batch_size: usize, codec: CompressionCodec, + format: ShuffleFormat, tracing_enabled: bool, write_buffer_size: usize, ) -> datafusion::common::Result { @@ -168,7 +176,7 @@ impl MultiPartitionShuffleRepartitioner { let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; let partition_writers = (0..num_output_partitions) - .map(|_| PartitionWriter::try_new(shuffle_block_writer.clone())) + .map(|_| PartitionWriter::try_new(shuffle_block_writer.clone(), format.clone())) .collect::>>()?; let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{partition}]")) @@ -178,6 +186,7 @@ impl MultiPartitionShuffleRepartitioner { Ok(Self { output_data_file, output_index_file, + schema, buffered_batches: vec![], partition_indices: vec![vec![]; num_output_partitions], partition_writers, @@ -188,6 +197,8 @@ impl MultiPartitionShuffleRepartitioner { scratch, batch_size, reservation, + codec, + format, tracing_enabled, write_buffer_size, }) @@ -434,7 +445,7 @@ impl MultiPartitionShuffleRepartitioner { Ok(()) } - fn shuffle_write_partition( + fn shuffle_write_partition_block( partition_iter: &mut PartitionedBatchIterator, shuffle_block_writer: &mut ShuffleBlockWriter, output_data: &mut BufWriter, @@ -457,6 +468,7 @@ impl MultiPartitionShuffleRepartitioner { Ok(()) } + fn used(&self) -> usize { self.reservation.size() } @@ -513,6 +525,8 @@ impl MultiPartitionShuffleRepartitioner { &self.metrics, self.write_buffer_size, self.batch_size, + &self.codec, + self.schema.as_ref(), )?; } @@ -579,26 +593,60 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { for i in 0..num_output_partitions { offsets[i] = output_data.stream_position()?; - // if we wrote a spill file for this partition then copy the - // contents into the shuffle file - if let Some(spill_path) = self.partition_writers[i].path() { - let mut spill_file = BufReader::new(File::open(spill_path)?); - let mut write_timer = self.metrics.write_time.timer(); - std::io::copy(&mut spill_file, &mut output_data)?; - write_timer.stop(); - } - - // Write in memory batches to output data file let mut partition_iter = partitioned_batches.produce(i); - Self::shuffle_write_partition( - &mut partition_iter, - &mut self.shuffle_block_writer, - &mut output_data, - &self.metrics.encode_time, - &self.metrics.write_time, - self.write_buffer_size, - self.batch_size, - )?; + + match &self.format { + ShuffleFormat::Block => { + // Raw-copy spill file bytes (they are already in block format) + if let Some(spill_path) = self.partition_writers[i].path() { + let mut spill_file = BufReader::new(File::open(spill_path)?); + let mut write_timer = self.metrics.write_time.timer(); + std::io::copy(&mut spill_file, &mut output_data)?; + write_timer.stop(); + } + + Self::shuffle_write_partition_block( + &mut partition_iter, + &mut self.shuffle_block_writer, + &mut output_data, + &self.metrics.encode_time, + &self.metrics.write_time, + self.write_buffer_size, + self.batch_size, + )?; + } + ShuffleFormat::IpcStream => { + // Raw-copy spill file (already in IPC stream format) + if let Some(spill_path) = self.partition_writers[i].path() { + let mut spill_file = BufReader::new(File::open(spill_path)?); + let mut write_timer = self.metrics.write_time.timer(); + std::io::copy(&mut spill_file, &mut output_data)?; + write_timer.stop(); + } + + // Write remaining in-memory batches as an IPC stream, + // coalescing small batches to reduce per-message overhead + let mut ipc_writer = IpcStreamWriter::try_new( + &mut output_data, + self.schema.as_ref(), + self.codec.clone(), + )?; + let mut coalescer = + BatchCoalescer::new(Arc::clone(&self.schema), self.batch_size); + for batch in &mut partition_iter { + coalescer.push_batch(batch?)?; + while let Some(b) = coalescer.next_completed_batch() { + ipc_writer + .write_batch(&b, &self.metrics.encode_time)?; + } + } + coalescer.finish_buffered_batch()?; + while let Some(b) = coalescer.next_completed_batch() { + ipc_writer.write_batch(&b, &self.metrics.encode_time)?; + } + ipc_writer.finish()?; + } + } } let mut write_timer = self.metrics.write_time.timer(); diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index 5801ef613b..ee14ed070e 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -18,18 +18,32 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::ShufflePartitioner; use crate::writers::BufBatchWriter; -use crate::{CompressionCodec, ShuffleBlockWriter}; +use crate::{CompressionCodec, IpcStreamWriter, ShuffleBlockWriter, ShuffleFormat}; use arrow::array::RecordBatch; +use arrow::compute::kernels::coalesce::BatchCoalescer; use arrow::datatypes::SchemaRef; use datafusion::common::DataFusionError; use std::fs::{File, OpenOptions}; use std::io::{BufWriter, Write}; use tokio::time::Instant; +/// Output strategy for writing shuffle data in either block or IPC stream format. +enum OutputWriter { + /// Block format: uses BufBatchWriter wrapping ShuffleBlockWriter + Block(BufBatchWriter), + /// IPC stream format: uses IpcStreamWriter with batch coalescing. + /// The writer is wrapped in Option so it can be taken for finish(). + IpcStream { + writer: Option>>, + coalescer: Option, + batch_size: usize, + }, +} + /// A partitioner that writes all shuffle data to a single file and a single index file pub(crate) struct SinglePartitionShufflePartitioner { - // output_data_file: File, - output_data_writer: BufBatchWriter, + output: OutputWriter, + output_data_file: File, output_index_path: String, /// Batches that are smaller than the batch size and to be concatenated buffered_batches: Vec, @@ -42,6 +56,7 @@ pub(crate) struct SinglePartitionShufflePartitioner { } impl SinglePartitionShufflePartitioner { + #[allow(clippy::too_many_arguments)] pub(crate) fn try_new( output_data_path: String, output_index_path: String, @@ -49,25 +64,42 @@ impl SinglePartitionShufflePartitioner { metrics: ShufflePartitionerMetrics, batch_size: usize, codec: CompressionCodec, + format: ShuffleFormat, write_buffer_size: usize, ) -> datafusion::common::Result { - let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; - let output_data_file = OpenOptions::new() .write(true) .create(true) .truncate(true) .open(output_data_path)?; - let output_data_writer = BufBatchWriter::new( - shuffle_block_writer, - output_data_file, - write_buffer_size, - batch_size, - ); + let output = match format { + ShuffleFormat::Block => { + let shuffle_block_writer = + ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; + let output_data_writer = BufBatchWriter::new( + shuffle_block_writer, + output_data_file.try_clone()?, + write_buffer_size, + batch_size, + ); + OutputWriter::Block(output_data_writer) + } + ShuffleFormat::IpcStream => { + let buf_writer = + BufWriter::with_capacity(write_buffer_size, output_data_file.try_clone()?); + let writer = IpcStreamWriter::try_new(buf_writer, schema.as_ref(), codec)?; + OutputWriter::IpcStream { + writer: Some(writer), + coalescer: None, + batch_size, + } + } + }; Ok(Self { - output_data_writer, + output, + output_data_file, output_index_path, buffered_batches: vec![], num_buffered_rows: 0, @@ -106,6 +138,68 @@ impl SinglePartitionShufflePartitioner { } } } + + fn write_batch(&mut self, batch: &RecordBatch) -> datafusion::common::Result<()> { + match &mut self.output { + OutputWriter::Block(writer) => { + writer.write(batch, &self.metrics.encode_time, &self.metrics.write_time)?; + Ok(()) + } + OutputWriter::IpcStream { + writer, + coalescer, + batch_size, + } => { + let w = writer.as_mut().ok_or_else(|| { + DataFusionError::Internal("IPC stream writer already finished".to_string()) + })?; + let coal = coalescer + .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), *batch_size)); + coal.push_batch(batch.clone())?; + + while let Some(b) = coal.next_completed_batch() { + w.write_batch(&b, &self.metrics.encode_time)?; + } + Ok(()) + } + } + } + + fn flush_output(&mut self) -> datafusion::common::Result<()> { + match &mut self.output { + OutputWriter::Block(writer) => { + writer.flush(&self.metrics.encode_time, &self.metrics.write_time) + } + OutputWriter::IpcStream { + writer, coalescer, .. + } => { + let w = writer.as_mut().ok_or_else(|| { + DataFusionError::Internal("IPC stream writer already finished".to_string()) + })?; + // Flush remaining coalesced batches + if let Some(coal) = coalescer { + coal.finish_buffered_batch()?; + while let Some(b) = coal.next_completed_batch() { + w.write_batch(&b, &self.metrics.encode_time)?; + } + } + Ok(()) + } + } + } + + /// Finish the IPC stream writer (writes EOS marker and flushes). + fn finish_ipc_stream(&mut self) -> datafusion::common::Result<()> { + if let OutputWriter::IpcStream { writer, .. } = &mut self.output { + if let Some(w) = writer.take() { + let buf_writer = w.finish()?; + buf_writer + .into_inner() + .map_err(|e| DataFusionError::Execution(format!("flush error: {e}")))?; + } + } + Ok(()) + } } #[async_trait::async_trait] @@ -123,20 +217,12 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { // Write the concatenated buffered batch if let Some(batch) = concatenated_batch { - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; + self.write_batch(&batch)?; } if num_rows >= self.batch_size { // Write the new batch - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; + self.write_batch(&batch)?; } else { // Add the new batch to the buffer self.add_buffered_batch(batch); @@ -160,16 +246,13 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { // Write the concatenated buffered batch if let Some(batch) = concatenated_batch { - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; + self.write_batch(&batch)?; } - self.output_data_writer - .flush(&self.metrics.encode_time, &self.metrics.write_time)?; + self.flush_output()?; + self.finish_ipc_stream()?; // Write index file. It should only contain 2 entries: 0 and the total number of bytes written + let data_file_length = self.output_data_file.metadata()?.len(); let index_file = OpenOptions::new() .write(true) .create(true) @@ -177,7 +260,6 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { .open(self.output_index_path.clone()) .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; let mut index_buf_writer = BufWriter::new(index_file); - let data_file_length = self.output_data_writer.writer_stream_position()?; for offset in [0, data_file_length] { index_buf_writer.write_all(&(offset as i64).to_le_bytes()[..])?; } diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index e649aaac69..96c4d1dc36 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -21,7 +21,7 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::{ MultiPartitionShuffleRepartitioner, ShufflePartitioner, SinglePartitionShufflePartitioner, }; -use crate::{CometPartitioning, CompressionCodec}; +use crate::{CometPartitioning, CompressionCodec, ShuffleFormat}; use async_trait::async_trait; use datafusion::common::exec_datafusion_err; use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; @@ -65,6 +65,8 @@ pub struct ShuffleWriterExec { cache: PlanProperties, /// The compression codec to use when compressing shuffle blocks codec: CompressionCodec, + /// The shuffle data format (Block or IpcStream) + format: ShuffleFormat, tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, @@ -77,6 +79,7 @@ impl ShuffleWriterExec { input: Arc, partitioning: CometPartitioning, codec: CompressionCodec, + format: ShuffleFormat, output_data_file: String, output_index_file: String, tracing_enabled: bool, @@ -97,6 +100,7 @@ impl ShuffleWriterExec { output_index_file, cache, codec, + format, tracing_enabled, write_buffer_size, }) @@ -159,6 +163,7 @@ impl ExecutionPlan for ShuffleWriterExec { Arc::clone(&children[0]), self.partitioning.clone(), self.codec.clone(), + self.format.clone(), self.output_data_file.clone(), self.output_index_file.clone(), self.tracing_enabled, @@ -188,6 +193,7 @@ impl ExecutionPlan for ShuffleWriterExec { metrics, context, self.codec.clone(), + self.format.clone(), self.tracing_enabled, self.write_buffer_size, ) @@ -208,6 +214,7 @@ async fn external_shuffle( metrics: ShufflePartitionerMetrics, context: Arc, codec: CompressionCodec, + format: ShuffleFormat, tracing_enabled: bool, write_buffer_size: usize, ) -> Result { @@ -223,6 +230,7 @@ async fn external_shuffle( metrics, context.session_config().batch_size(), codec, + format, write_buffer_size, )?) } @@ -236,6 +244,7 @@ async fn external_shuffle( context.runtime_env(), context.session_config().batch_size(), codec, + format, tracing_enabled, write_buffer_size, )?), @@ -360,6 +369,7 @@ mod test { runtime_env, 1024, CompressionCodec::Lz4Frame, + ShuffleFormat::Block, false, 1024 * 1024, // write_buffer_size: 1MB default ) @@ -462,6 +472,7 @@ mod test { ))), partitioning, CompressionCodec::Zstd(1), + ShuffleFormat::Block, "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), false, @@ -521,6 +532,7 @@ mod test { ))), CometPartitioning::RoundRobin(num_partitions, 0), CompressionCodec::Zstd(1), + ShuffleFormat::Block, data_file.clone(), index_file.clone(), false, diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index cfddb46539..9b8d17c315 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -134,9 +134,3 @@ impl, W: Write> BufBatchWriter { Ok(()) } } - -impl, W: Write + Seek> BufBatchWriter { - pub(crate) fn writer_stream_position(&mut self) -> datafusion::common::Result { - self.writer.stream_position().map_err(Into::into) - } -} diff --git a/native/shuffle/src/writers/ipc_stream_writer.rs b/native/shuffle/src/writers/ipc_stream_writer.rs new file mode 100644 index 0000000000..e483d47490 --- /dev/null +++ b/native/shuffle/src/writers/ipc_stream_writer.rs @@ -0,0 +1,236 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::RecordBatch; +use arrow::datatypes::Schema; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; +use arrow::ipc::CompressionType; +use datafusion::common::DataFusionError; +use datafusion::error::Result; +use datafusion::physical_plan::metrics::Time; +use std::io::Write; + +use super::CompressionCodec; + +/// Maps a [`CompressionCodec`] to Arrow IPC [`IpcWriteOptions`]. +/// +/// Arrow IPC body compression supports LZ4_FRAME and ZSTD. Snappy is not +/// part of the Arrow IPC specification and will return an error. +fn ipc_write_options(codec: &CompressionCodec) -> Result { + match codec { + CompressionCodec::None => Ok(IpcWriteOptions::default()), + CompressionCodec::Lz4Frame => IpcWriteOptions::default() + .try_with_compression(Some(CompressionType::LZ4_FRAME)) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)), + CompressionCodec::Zstd(_) => IpcWriteOptions::default() + .try_with_compression(Some(CompressionType::ZSTD)) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)), + CompressionCodec::Snappy => Err(DataFusionError::NotImplemented( + "Snappy compression is not supported for IPC stream format. \ + Use LZ4 or ZSTD instead." + .to_string(), + )), + } +} + +/// Writes record batches as a standard Arrow IPC stream. +/// +/// Unlike [`super::ShuffleBlockWriter`] which writes each batch as a self-contained +/// block with a custom header (length prefix + field count + codec), this writer +/// produces a standard Arrow IPC stream where the schema is written once and +/// each batch is an IPC record batch message within the stream. +/// +/// Benefits over the block-based format: +/// - Schema is written once per stream instead of once per batch +/// - Standard Arrow IPC format, readable by any Arrow-compatible tool +/// - Uses Arrow's built-in IPC body compression (LZ4_FRAME or ZSTD) +/// +/// The writer is stateful: it must be created, used to write batches, then +/// finished. The schema is written on creation and the end-of-stream marker +/// is written on [`finish`](Self::finish). +/// +/// # Example +/// +/// ```ignore +/// let mut writer = IpcStreamWriter::try_new(file, &schema, CompressionCodec::Lz4Frame)?; +/// writer.write_batch(&batch1, &ipc_time)?; +/// writer.write_batch(&batch2, &ipc_time)?; +/// let file = writer.finish()?; +/// ``` +pub struct IpcStreamWriter { + writer: StreamWriter, +} + +impl IpcStreamWriter { + /// Creates a new IPC stream writer. + /// + /// Writes the IPC stream header (schema message) to the output immediately. + pub fn try_new(output: W, schema: &Schema, codec: CompressionCodec) -> Result { + let options = ipc_write_options(&codec)?; + let writer = StreamWriter::try_new_with_options(output, schema, options)?; + Ok(Self { writer }) + } + + /// Writes a record batch as an IPC message within the stream. + /// + /// Empty batches (0 rows) are skipped. + pub fn write_batch(&mut self, batch: &RecordBatch, ipc_time: &Time) -> Result<()> { + if batch.num_rows() == 0 { + return Ok(()); + } + let mut timer = ipc_time.timer(); + self.writer.write(batch)?; + timer.stop(); + Ok(()) + } + + /// Writes the end-of-stream marker and returns the underlying writer. + pub fn finish(mut self) -> Result { + self.writer.finish()?; + self.writer.into_inner().map_err(Into::into) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{Int32Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::ipc::reader::StreamReader; + use std::io::Cursor; + use std::sync::Arc; + + fn test_schema() -> Schema { + Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Utf8, true), + ]) + } + + fn test_batch(schema: &Schema, n: i32) -> RecordBatch { + RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![n, n + 1, n + 2])), + Arc::new(StringArray::from(vec![ + Some("hello"), + None, + Some("world"), + ])), + ], + ) + .unwrap() + } + + fn roundtrip(codec: CompressionCodec, num_batches: usize) { + let schema = test_schema(); + let ipc_time = Time::default(); + + let mut buf = Vec::new(); + { + let cursor = Cursor::new(&mut buf); + let mut writer = + IpcStreamWriter::try_new(cursor, &schema, codec).unwrap(); + for i in 0..num_batches { + let batch = test_batch(&schema, (i * 10) as i32); + writer.write_batch(&batch, &ipc_time).unwrap(); + } + writer.finish().unwrap(); + } + + // Read back + let cursor = Cursor::new(&buf); + let reader = StreamReader::try_new(cursor, None).unwrap(); + let batches: Vec = reader.map(|r| r.unwrap()).collect(); + assert_eq!(batches.len(), num_batches); + + for (i, batch) in batches.iter().enumerate() { + assert_eq!(batch.num_rows(), 3); + let col_a = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let base = (i * 10) as i32; + assert_eq!(col_a.value(0), base); + assert_eq!(col_a.value(1), base + 1); + assert_eq!(col_a.value(2), base + 2); + } + } + + #[test] + fn test_roundtrip_no_compression() { + roundtrip(CompressionCodec::None, 3); + } + + #[test] + fn test_roundtrip_lz4() { + roundtrip(CompressionCodec::Lz4Frame, 3); + } + + #[test] + fn test_roundtrip_zstd() { + roundtrip(CompressionCodec::Zstd(1), 3); + } + + #[test] + fn test_empty_batch_skipped() { + let schema = test_schema(); + let ipc_time = Time::default(); + + let mut buf = Vec::new(); + { + let cursor = Cursor::new(&mut buf); + let mut writer = + IpcStreamWriter::try_new(cursor, &schema, CompressionCodec::None).unwrap(); + + // Write a real batch, an empty batch, then another real batch + writer + .write_batch(&test_batch(&schema, 0), &ipc_time) + .unwrap(); + let empty = RecordBatch::new_empty(Arc::new(schema.clone())); + writer.write_batch(&empty, &ipc_time).unwrap(); + writer + .write_batch(&test_batch(&schema, 10), &ipc_time) + .unwrap(); + writer.finish().unwrap(); + } + + let cursor = Cursor::new(&buf); + let reader = StreamReader::try_new(cursor, None).unwrap(); + let batches: Vec = reader.map(|r| r.unwrap()).collect(); + assert_eq!(batches.len(), 2); // empty batch was skipped + } + + #[test] + fn test_snappy_unsupported() { + let schema = test_schema(); + let buf = Vec::new(); + let cursor = Cursor::new(buf); + let result = IpcStreamWriter::try_new(cursor, &schema, CompressionCodec::Snappy); + let err = match result { + Err(e) => e.to_string(), + Ok(_) => panic!("expected error for Snappy"), + }; + assert!(err.contains("Snappy compression is not supported")); + } + + #[test] + fn test_single_batch() { + roundtrip(CompressionCodec::None, 1); + } +} diff --git a/native/shuffle/src/writers/mod.rs b/native/shuffle/src/writers/mod.rs index 75caf9f3a3..a7491fb791 100644 --- a/native/shuffle/src/writers/mod.rs +++ b/native/shuffle/src/writers/mod.rs @@ -17,10 +17,12 @@ mod buf_batch_writer; mod checksum; +mod ipc_stream_writer; mod shuffle_block_writer; mod spill; pub(crate) use buf_batch_writer::BufBatchWriter; pub(crate) use checksum::Checksum; +pub use ipc_stream_writer::IpcStreamWriter; pub use shuffle_block_writer::{CompressionCodec, ShuffleBlockWriter}; pub(crate) use spill::PartitionWriter; diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index c16caddbf9..9b459d0424 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -15,14 +15,16 @@ // specific language governing permissions and limitations // under the License. -use super::ShuffleBlockWriter; +use super::{IpcStreamWriter, ShuffleBlockWriter}; use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::PartitionedBatchIterator; use crate::writers::buf_batch_writer::BufBatchWriter; +use crate::ShuffleFormat; use datafusion::common::DataFusionError; use datafusion::execution::disk_manager::RefCountedTempFile; use datafusion::execution::runtime_env::RuntimeEnv; use std::fs::{File, OpenOptions}; +use std::io::BufWriter; /// A temporary disk file for spilling a partition's intermediate shuffle data. struct SpillFile { @@ -36,17 +38,20 @@ pub(crate) struct PartitionWriter { /// will append to this file and the contents will be copied to the shuffle file at /// the end of processing. spill_file: Option, - /// Writer that performs encoding and compression + /// Writer that performs encoding and compression (block format) shuffle_block_writer: ShuffleBlockWriter, + format: ShuffleFormat, } impl PartitionWriter { pub(crate) fn try_new( shuffle_block_writer: ShuffleBlockWriter, + format: ShuffleFormat, ) -> datafusion::common::Result { Ok(Self { spill_file: None, shuffle_block_writer, + format, }) } @@ -55,7 +60,6 @@ impl PartitionWriter { runtime: &RuntimeEnv, ) -> datafusion::common::Result<()> { if self.spill_file.is_none() { - // Spill file is not yet created, create it let spill_file = runtime .disk_manager .create_tmp_file("shuffle writer spill")?; @@ -75,6 +79,7 @@ impl PartitionWriter { Ok(()) } + #[allow(clippy::too_many_arguments)] pub(crate) fn spill( &mut self, iter: &mut PartitionedBatchIterator, @@ -82,32 +87,53 @@ impl PartitionWriter { metrics: &ShufflePartitionerMetrics, write_buffer_size: usize, batch_size: usize, + codec: &crate::CompressionCodec, + schema: &arrow::datatypes::Schema, ) -> datafusion::common::Result { if let Some(batch) = iter.next() { self.ensure_spill_file_created(runtime)?; + let file = &mut self.spill_file.as_mut().unwrap().file; - let total_bytes_written = { - let mut buf_batch_writer = BufBatchWriter::new( - &mut self.shuffle_block_writer, - &mut self.spill_file.as_mut().unwrap().file, - write_buffer_size, - batch_size, - ); - let mut bytes_written = - buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; - for batch in iter { - let batch = batch?; - bytes_written += buf_batch_writer.write( - &batch, + match &self.format { + ShuffleFormat::Block => { + let mut buf_batch_writer = BufBatchWriter::new( + &mut self.shuffle_block_writer, + file, + write_buffer_size, + batch_size, + ); + let mut bytes_written = buf_batch_writer.write( + &batch?, &metrics.encode_time, &metrics.write_time, )?; + for batch in iter { + bytes_written += buf_batch_writer.write( + &batch?, + &metrics.encode_time, + &metrics.write_time, + )?; + } + buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; + Ok(bytes_written) } - buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; - bytes_written - }; - - Ok(total_bytes_written) + ShuffleFormat::IpcStream => { + let buf_writer = BufWriter::with_capacity(write_buffer_size, file); + let mut ipc_writer = + IpcStreamWriter::try_new(buf_writer, schema, codec.clone())?; + ipc_writer.write_batch(&batch?, &metrics.encode_time)?; + for batch in iter { + ipc_writer.write_batch(&batch?, &metrics.encode_time)?; + } + let buf_writer = ipc_writer.finish()?; + buf_writer.into_inner().map_err(|e| { + DataFusionError::Execution(format!("flush error: {e}")) + })?; + // IPC stream doesn't easily track bytes written; return 0 + // (spilled_bytes metric will be approximate) + Ok(0) + } + } } else { Ok(0) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 3fc222bd19..92d388d75b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.comet.CometConf import org.apache.comet.serde.{OperatorOuterClass, PartitioningOuterClass, QueryPlanSerde} -import org.apache.comet.serde.OperatorOuterClass.{CompressionCodec, Operator} +import org.apache.comet.serde.OperatorOuterClass.{CompressionCodec, Operator, ShuffleWriterFormat} import org.apache.comet.serde.QueryPlanSerde.serializeDataType /** @@ -193,6 +193,12 @@ class CometNativeShuffleWriter[K, V]( shuffleWriterBuilder.setWriteBufferSize( CometConf.COMET_SHUFFLE_WRITE_BUFFER_SIZE.get().max(Int.MaxValue).toInt) + val format = CometConf.COMET_EXEC_SHUFFLE_FORMAT.get() match { + case "ipc_stream" => ShuffleWriterFormat.IPC_STREAM + case _ => ShuffleWriterFormat.BLOCK + } + shuffleWriterBuilder.setFormat(format) + outputPartitioning match { case p if isSinglePartitioning(p) => val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() From 9aeaea789438ae6c485d089affa26cecc632f37c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 09:30:58 -0600 Subject: [PATCH 11/21] fix: keep IPC spill stream open across spill calls Previously each spill() call for IPC stream format wrote a complete IPC stream (schema + batches + EOS) to the spill file. Multiple spills produced concatenated streams, and readers would stop at the first EOS marker, silently dropping subsequent data. Fix by keeping the IpcStreamWriter alive on PartitionWriter across spill calls, finishing it only in shuffle_write via finish_spill(). Also: - Remove try_clone() fd waste in SinglePartitionShufflePartitioner - Hoist spill file copy above format match (shared by both formats) - Remove extra blank line --- .../src/partitioners/multi_partition.rs | 30 ++-- .../src/partitioners/single_partition.rs | 34 ++-- native/shuffle/src/writers/spill.rs | 146 ++++++++++-------- 3 files changed, 101 insertions(+), 109 deletions(-) diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 3636ef7170..9dc3f53de7 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -468,7 +468,6 @@ impl MultiPartitionShuffleRepartitioner { Ok(()) } - fn used(&self) -> usize { self.reservation.size() } @@ -595,16 +594,19 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { let mut partition_iter = partitioned_batches.produce(i); + // Finish any open IPC spill stream before copying + self.partition_writers[i].finish_spill()?; + + // Copy spill file contents into the output + if let Some(spill_path) = self.partition_writers[i].path() { + let mut spill_file = BufReader::new(File::open(spill_path)?); + let mut write_timer = self.metrics.write_time.timer(); + std::io::copy(&mut spill_file, &mut output_data)?; + write_timer.stop(); + } + match &self.format { ShuffleFormat::Block => { - // Raw-copy spill file bytes (they are already in block format) - if let Some(spill_path) = self.partition_writers[i].path() { - let mut spill_file = BufReader::new(File::open(spill_path)?); - let mut write_timer = self.metrics.write_time.timer(); - std::io::copy(&mut spill_file, &mut output_data)?; - write_timer.stop(); - } - Self::shuffle_write_partition_block( &mut partition_iter, &mut self.shuffle_block_writer, @@ -616,16 +618,6 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { )?; } ShuffleFormat::IpcStream => { - // Raw-copy spill file (already in IPC stream format) - if let Some(spill_path) = self.partition_writers[i].path() { - let mut spill_file = BufReader::new(File::open(spill_path)?); - let mut write_timer = self.metrics.write_time.timer(); - std::io::copy(&mut spill_file, &mut output_data)?; - write_timer.stop(); - } - - // Write remaining in-memory batches as an IPC stream, - // coalescing small batches to reduce per-message overhead let mut ipc_writer = IpcStreamWriter::try_new( &mut output_data, self.schema.as_ref(), diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index ee14ed070e..55e510c4fd 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -25,13 +25,12 @@ use arrow::datatypes::SchemaRef; use datafusion::common::DataFusionError; use std::fs::{File, OpenOptions}; use std::io::{BufWriter, Write}; +use std::path::PathBuf; use tokio::time::Instant; /// Output strategy for writing shuffle data in either block or IPC stream format. enum OutputWriter { - /// Block format: uses BufBatchWriter wrapping ShuffleBlockWriter Block(BufBatchWriter), - /// IPC stream format: uses IpcStreamWriter with batch coalescing. /// The writer is wrapped in Option so it can be taken for finish(). IpcStream { writer: Option>>, @@ -43,15 +42,11 @@ enum OutputWriter { /// A partitioner that writes all shuffle data to a single file and a single index file pub(crate) struct SinglePartitionShufflePartitioner { output: OutputWriter, - output_data_file: File, + output_data_path: PathBuf, output_index_path: String, - /// Batches that are smaller than the batch size and to be concatenated buffered_batches: Vec, - /// Number of rows in the concatenating batches num_buffered_rows: usize, - /// Metrics for the repartitioner metrics: ShufflePartitionerMetrics, - /// The configured batch size batch_size: usize, } @@ -71,23 +66,22 @@ impl SinglePartitionShufflePartitioner { .write(true) .create(true) .truncate(true) - .open(output_data_path)?; + .open(&output_data_path)?; let output = match format { ShuffleFormat::Block => { let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; - let output_data_writer = BufBatchWriter::new( + OutputWriter::Block(BufBatchWriter::new( shuffle_block_writer, - output_data_file.try_clone()?, + output_data_file, write_buffer_size, batch_size, - ); - OutputWriter::Block(output_data_writer) + )) } ShuffleFormat::IpcStream => { let buf_writer = - BufWriter::with_capacity(write_buffer_size, output_data_file.try_clone()?); + BufWriter::with_capacity(write_buffer_size, output_data_file); let writer = IpcStreamWriter::try_new(buf_writer, schema.as_ref(), codec)?; OutputWriter::IpcStream { writer: Some(writer), @@ -99,7 +93,7 @@ impl SinglePartitionShufflePartitioner { Ok(Self { output, - output_data_file, + output_data_path: PathBuf::from(output_data_path), output_index_path, buffered_batches: vec![], num_buffered_rows: 0, @@ -108,14 +102,11 @@ impl SinglePartitionShufflePartitioner { }) } - /// Add a batch to the buffer of the partitioner, these buffered batches will be concatenated - /// and written to the output data file when the number of rows in the buffer reaches the batch size. fn add_buffered_batch(&mut self, batch: RecordBatch) { self.num_buffered_rows += batch.num_rows(); self.buffered_batches.push(batch); } - /// Consumes buffered batches and return a concatenated batch if successful fn concat_buffered_batches(&mut self) -> datafusion::common::Result> { if self.buffered_batches.is_empty() { Ok(None) @@ -176,7 +167,6 @@ impl SinglePartitionShufflePartitioner { let w = writer.as_mut().ok_or_else(|| { DataFusionError::Internal("IPC stream writer already finished".to_string()) })?; - // Flush remaining coalesced batches if let Some(coal) = coalescer { coal.finish_buffered_batch()?; while let Some(b) = coal.next_completed_batch() { @@ -188,7 +178,6 @@ impl SinglePartitionShufflePartitioner { } } - /// Finish the IPC stream writer (writes EOS marker and flushes). fn finish_ipc_stream(&mut self) -> datafusion::common::Result<()> { if let OutputWriter::IpcStream { writer, .. } = &mut self.output { if let Some(w) = writer.take() { @@ -215,16 +204,13 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { if num_rows >= self.batch_size || num_rows + self.num_buffered_rows > self.batch_size { let concatenated_batch = self.concat_buffered_batches()?; - // Write the concatenated buffered batch if let Some(batch) = concatenated_batch { self.write_batch(&batch)?; } if num_rows >= self.batch_size { - // Write the new batch self.write_batch(&batch)?; } else { - // Add the new batch to the buffer self.add_buffered_batch(batch); } } else { @@ -244,15 +230,13 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { let start_time = Instant::now(); let concatenated_batch = self.concat_buffered_batches()?; - // Write the concatenated buffered batch if let Some(batch) = concatenated_batch { self.write_batch(&batch)?; } self.flush_output()?; self.finish_ipc_stream()?; - // Write index file. It should only contain 2 entries: 0 and the total number of bytes written - let data_file_length = self.output_data_file.metadata()?.len(); + let data_file_length = std::fs::metadata(&self.output_data_path)?.len(); let index_file = OpenOptions::new() .write(true) .create(true) diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index 9b459d0424..8e6acfda4d 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -26,20 +26,18 @@ use datafusion::execution::runtime_env::RuntimeEnv; use std::fs::{File, OpenOptions}; use std::io::BufWriter; -/// A temporary disk file for spilling a partition's intermediate shuffle data. -struct SpillFile { - temp_file: RefCountedTempFile, - file: File, -} - /// Manages encoding and optional disk spilling for a single shuffle partition. +/// +/// For block format, each `spill()` call appends self-contained blocks to the +/// spill file. For IPC stream format, an `IpcStreamWriter` is kept open across +/// spill calls so that all spilled data forms a single IPC stream (one schema +/// header, many batch messages, one EOS marker written at finish time). pub(crate) struct PartitionWriter { - /// Spill file for intermediate shuffle output for this partition. Each spill event - /// will append to this file and the contents will be copied to the shuffle file at - /// the end of processing. - spill_file: Option, - /// Writer that performs encoding and compression (block format) + spill_file: Option, shuffle_block_writer: ShuffleBlockWriter, + /// Persistent IPC stream writer for the spill file, kept open across + /// multiple `spill()` calls. + ipc_spill_writer: Option>>, format: ShuffleFormat, } @@ -51,6 +49,7 @@ impl PartitionWriter { Ok(Self { spill_file: None, shuffle_block_writer, + ipc_spill_writer: None, format, }) } @@ -60,21 +59,11 @@ impl PartitionWriter { runtime: &RuntimeEnv, ) -> datafusion::common::Result<()> { if self.spill_file.is_none() { - let spill_file = runtime - .disk_manager - .create_tmp_file("shuffle writer spill")?; - let spill_data = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(spill_file.path()) - .map_err(|e| { - DataFusionError::Execution(format!("Error occurred while spilling {e}")) - })?; - self.spill_file = Some(SpillFile { - temp_file: spill_file, - file: spill_data, - }); + self.spill_file = Some( + runtime + .disk_manager + .create_tmp_file("shuffle writer spill")?, + ); } Ok(()) } @@ -90,59 +79,86 @@ impl PartitionWriter { codec: &crate::CompressionCodec, schema: &arrow::datatypes::Schema, ) -> datafusion::common::Result { - if let Some(batch) = iter.next() { - self.ensure_spill_file_created(runtime)?; - let file = &mut self.spill_file.as_mut().unwrap().file; + let Some(batch) = iter.next() else { + return Ok(0); + }; + + self.ensure_spill_file_created(runtime)?; + let spill_path = self.spill_file.as_ref().unwrap().path().to_owned(); - match &self.format { - ShuffleFormat::Block => { - let mut buf_batch_writer = BufBatchWriter::new( - &mut self.shuffle_block_writer, - file, - write_buffer_size, - batch_size, - ); - let mut bytes_written = buf_batch_writer.write( + match &self.format { + ShuffleFormat::Block => { + let mut file = OpenOptions::new() + .append(true) + .open(&spill_path) + .map_err(|e| { + DataFusionError::Execution(format!( + "Error occurred while spilling {e}" + )) + })?; + let mut buf_batch_writer = BufBatchWriter::new( + &mut self.shuffle_block_writer, + &mut file, + write_buffer_size, + batch_size, + ); + let mut bytes_written = buf_batch_writer.write( + &batch?, + &metrics.encode_time, + &metrics.write_time, + )?; + for batch in iter { + bytes_written += buf_batch_writer.write( &batch?, &metrics.encode_time, &metrics.write_time, )?; - for batch in iter { - bytes_written += buf_batch_writer.write( - &batch?, - &metrics.encode_time, - &metrics.write_time, - )?; - } - buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; - Ok(bytes_written) } - ShuffleFormat::IpcStream => { + buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; + Ok(bytes_written) + } + ShuffleFormat::IpcStream => { + // Lazily open the IPC stream writer on first spill. It stays + // open so subsequent spills append batches to the same stream. + if self.ipc_spill_writer.is_none() { + let file = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(&spill_path) + .map_err(|e| { + DataFusionError::Execution(format!( + "Error occurred while spilling {e}" + )) + })?; let buf_writer = BufWriter::with_capacity(write_buffer_size, file); - let mut ipc_writer = - IpcStreamWriter::try_new(buf_writer, schema, codec.clone())?; + self.ipc_spill_writer = + Some(IpcStreamWriter::try_new(buf_writer, schema, codec.clone())?); + } + let ipc_writer = self.ipc_spill_writer.as_mut().unwrap(); + ipc_writer.write_batch(&batch?, &metrics.encode_time)?; + for batch in iter { ipc_writer.write_batch(&batch?, &metrics.encode_time)?; - for batch in iter { - ipc_writer.write_batch(&batch?, &metrics.encode_time)?; - } - let buf_writer = ipc_writer.finish()?; - buf_writer.into_inner().map_err(|e| { - DataFusionError::Execution(format!("flush error: {e}")) - })?; - // IPC stream doesn't easily track bytes written; return 0 - // (spilled_bytes metric will be approximate) - Ok(0) } + Ok(0) } - } else { - Ok(0) } } + /// Finish the IPC spill stream writer if one is open. Must be called + /// before raw-copying the spill file to the output. + pub(crate) fn finish_spill(&mut self) -> datafusion::common::Result<()> { + if let Some(writer) = self.ipc_spill_writer.take() { + let buf_writer = writer.finish()?; + buf_writer + .into_inner() + .map_err(|e| DataFusionError::Execution(format!("flush error: {e}")))?; + } + Ok(()) + } + pub(crate) fn path(&self) -> Option<&std::path::Path> { - self.spill_file - .as_ref() - .map(|spill_file| spill_file.temp_file.path()) + self.spill_file.as_ref().map(|f| f.path()) } #[cfg(test)] From 31b0861d16092f84d5554ea37bf7ccf28668bfd0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 10:13:54 -0600 Subject: [PATCH 12/21] feat: update shuffle reader to handle IPC stream format IPC stream format data is length-prefixed (8-byte LE length followed by the Arrow IPC stream bytes). The reader auto-detects the format: block format buffers start with a codec prefix (SNAP/LZ4_/ZSTD/NONE), IPC stream buffers start with Arrow continuation (0xFFFFFFFF). Changes: - IpcStreamWriter: add try_new_length_prefixed/finish_length_prefixed for writer-side framing - CometShuffleBlockIterator: accept isIpcStream flag, read 8-byte length header (no field count) for IPC stream format - ShuffleScanExec: auto-detect format from buffer, decode IPC streams with read_ipc_stream(), buffer multiple batches from one stream - Update contributor guide with format documentation --- .../contributor-guide/native_shuffle.md | 69 ++++++++++++++++++- .../src/execution/operators/shuffle_scan.rs | 64 +++++++++++++---- .../src/partitioners/multi_partition.rs | 5 +- .../src/partitioners/single_partition.rs | 6 +- .../shuffle/src/writers/ipc_stream_writer.rs | 31 ++++++++- native/shuffle/src/writers/spill.rs | 21 ++++-- .../comet/CometShuffleBlockIterator.java | 50 +++++++++----- .../shuffle/CometShuffledRowRDD.scala | 5 +- 8 files changed, 205 insertions(+), 46 deletions(-) diff --git a/docs/source/contributor-guide/native_shuffle.md b/docs/source/contributor-guide/native_shuffle.md index 18e80a90c8..4a5d1fd754 100644 --- a/docs/source/contributor-guide/native_shuffle.md +++ b/docs/source/contributor-guide/native_shuffle.md @@ -88,7 +88,8 @@ Native shuffle (`CometExchange`) is selected when all of the following condition │ ▼ ┌───────────────────────────────────┐ -│ ShuffleBlockWriter │ +│ ShuffleBlockWriter (block format) │ +│ IpcStreamWriter (stream format) │ │ (Arrow IPC + compression) │ └───────────────────────────────────┘ │ @@ -215,6 +216,64 @@ The `MultiPartitionShuffleRepartitioner` manages: - `SpillFile`: Temporary file for spilled data - Memory tracking via `MemoryConsumer` trait +## Shuffle Format + +Native shuffle supports two data formats, configured via `spark.comet.exec.shuffle.format`: + +### Block Format (default) + +Each batch is written as a self-contained block: + +``` +[8 bytes: block length] [8 bytes: field count] [4 bytes: codec] [compressed Arrow IPC stream] +``` + +- The Arrow IPC stream inside each block contains the schema, one batch, and an EOS marker. +- Compression wraps the entire IPC stream (schema + batch). +- Supports all codecs: lz4, zstd, snappy. +- Reader parses the length-prefixed blocks sequentially. + +This format is implemented by `ShuffleBlockWriter` in `native/shuffle/src/writers/shuffle_block_writer.rs`. + +### IPC Stream Format + +Each partition's data is written as a standard Arrow IPC stream: + +``` +[schema message] [batch message 1] [batch message 2] ... [EOS marker] +``` + +- The schema is written once per partition (not per batch), reducing overhead. +- Uses Arrow's built-in IPC body compression (per-buffer compression within each message). +- Supports lz4 and zstd codecs. Snappy is not supported (not part of the Arrow IPC spec). +- Standard format readable by any Arrow-compatible tool. +- Small batches are coalesced before writing to reduce per-message IPC overhead. + +This format is implemented by `IpcStreamWriter` in `native/shuffle/src/writers/ipc_stream_writer.rs`. + +### Spill Behavior + +Both formats use the same spill strategy: when memory pressure triggers a spill, partitioned +data is written to per-partition temporary files. During the final `shuffle_write`: + +- **Block format**: Spill file bytes are raw-copied to the output, then remaining in-memory + batches are written as additional blocks. +- **IPC stream format**: The `IpcStreamWriter` is kept open across spill calls so that all + spilled data for a partition forms a single IPC stream. The stream is finished and raw-copied + to the output, then remaining in-memory batches are written as a second IPC stream. + +### Performance Comparison + +Benchmark results (200 hash partitions, LZ4, TPC-H SF100 lineitem): + +| Metric | Block | IPC Stream | +| ---------- | ----------- | ----------- | +| Throughput | 2.40M row/s | 2.37M row/s | +| Output | 61 MiB | 64 MiB | + +For single-partition writes, IPC stream is ~2x faster since the schema is written only once +instead of per batch. + ## Compression Native shuffle supports multiple compression codecs configured via @@ -230,15 +289,19 @@ Native shuffle supports multiple compression codecs configured via The compression codec is applied uniformly to all partitions. Each partition's data is independently compressed, allowing parallel decompression during reads. +Note: The `snappy` codec is only available with block format. IPC stream format supports +`lz4` and `zstd` only. + ## Configuration | Config | Default | Description | | ------------------------------------------------- | ------- | ---------------------------------------- | | `spark.comet.exec.shuffle.enabled` | `true` | Enable Comet shuffle | | `spark.comet.exec.shuffle.mode` | `auto` | Shuffle mode: `native`, `jvm`, or `auto` | -| `spark.comet.exec.shuffle.compression.codec` | `zstd` | Compression codec | +| `spark.comet.exec.shuffle.format` | `block` | Data format: `block` or `ipc_stream` | +| `spark.comet.exec.shuffle.compression.codec` | `lz4` | Compression codec | | `spark.comet.exec.shuffle.compression.zstd.level` | `1` | Zstd compression level | -| `spark.comet.shuffle.write.buffer.size` | `1MB` | Write buffer size | +| `spark.comet.shuffle.write.buffer.size` | `1MB` | Write buffer size | | `spark.comet.columnar.shuffle.batch.size` | `8192` | Target rows per batch | ## Comparison with JVM Shuffle diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index a1ad52310c..8599bcf0e2 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -18,7 +18,9 @@ use crate::{ errors::CometError, execution::{ - operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, shuffle::ipc::read_ipc_compressed, + operators::ExecutionError, + planner::TEST_EXEC_CONTEXT_ID, + shuffle::ipc::{read_ipc_compressed, read_ipc_stream}, }, jvm_bridge::{jni_call, JVMClasses}, }; @@ -60,6 +62,9 @@ pub struct ShuffleScanExec { pub schema: SchemaRef, /// The current input batch, populated by get_next_batch() before poll_next(). pub batch: Arc>>, + /// Buffered batches from IPC stream decoding (one IPC stream may contain + /// multiple batches). Consumed one at a time by get_next_batch(). + pending_batches: Vec, /// Cache of plan properties. cache: PlanProperties, /// Metrics collector. @@ -94,6 +99,7 @@ impl ShuffleScanExec { input_source, data_types, batch: Arc::new(Mutex::new(None)), + pending_batches: Vec::new(), cache, metrics: metrics_set, baseline_metrics, @@ -118,13 +124,25 @@ impl ShuffleScanExec { let mut current_batch = self.batch.try_lock().unwrap(); if current_batch.is_none() { - let next_batch = Self::get_next( - self.exec_context_id, - self.input_source.as_ref().unwrap().as_obj(), - &self.data_types, - &self.decode_time, - )?; - *current_batch = Some(next_batch); + // Check for buffered batches from a previous IPC stream decode + if let Some(batch) = self.pending_batches.pop() { + let num_rows = batch.num_rows(); + let columns: Vec = batch + .columns() + .iter() + .map(|col| unpack_dictionary(col)) + .collect(); + *current_batch = Some(InputBatch::new(columns, Some(num_rows))); + } else { + let next_batch = Self::get_next( + self.exec_context_id, + self.input_source.as_ref().unwrap().as_obj(), + &self.data_types, + &self.decode_time, + &mut self.pending_batches, + )?; + *current_batch = Some(next_batch); + } } timer.stop(); @@ -133,11 +151,14 @@ impl ShuffleScanExec { } /// Invokes JNI calls to get the next compressed shuffle block and decode it. + /// For IPC stream format, multiple batches may be decoded at once; extras + /// are stored in `pending_batches` for subsequent calls. fn get_next( exec_context_id: i64, iter: &JObject, data_types: &[DataType], decode_time: &Time, + pending_batches: &mut Vec, ) -> Result { if exec_context_id == TEST_EXEC_CONTEXT_ID { return Ok(InputBatch::EOF); @@ -161,7 +182,7 @@ impl ShuffleScanExec { return Ok(InputBatch::EOF); } - // Get the DirectByteBuffer containing the compressed shuffle block + // Get the DirectByteBuffer containing the shuffle block let buffer: JObject = unsafe { jni_call!(&mut env, comet_shuffle_block_iterator(iter).get_buffer() -> JObject)? @@ -172,17 +193,30 @@ impl ShuffleScanExec { let length = block_length as usize; let slice: &[u8] = unsafe { std::slice::from_raw_parts(raw_pointer, length) }; - // Decode the compressed IPC data + // Detect format from the first 4 bytes: + // Block format starts with a codec prefix: "SNAP", "LZ4_", "ZSTD", or "NONE" + // IPC stream format starts with Arrow continuation: 0xFFFFFFFF let mut timer = decode_time.timer(); - let batch = read_ipc_compressed(slice)?; + let is_ipc_stream = length >= 4 && slice[0..4] == [0xFF, 0xFF, 0xFF, 0xFF]; + + let batch = if is_ipc_stream { + let mut batches = read_ipc_stream(slice)?; + if batches.is_empty() { + timer.stop(); + return Ok(InputBatch::EOF); + } + let first = batches.remove(0); + // Store remaining batches in reverse order so pop() returns them in order + batches.reverse(); + *pending_batches = batches; + first + } else { + read_ipc_compressed(slice)? + }; timer.stop(); let num_rows = batch.num_rows(); - // Extract column arrays, unpacking any dictionary-encoded columns. - // Native shuffle may dictionary-encode string/binary columns for efficiency, - // but downstream DataFusion operators expect the value types declared in the - // schema (e.g. Utf8, not Dictionary). let columns: Vec = batch .columns() .iter() diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 9dc3f53de7..4ebbd86981 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -618,7 +618,8 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { )?; } ShuffleFormat::IpcStream => { - let mut ipc_writer = IpcStreamWriter::try_new( + let start_pos = output_data.stream_position()?; + let mut ipc_writer = IpcStreamWriter::try_new_length_prefixed( &mut output_data, self.schema.as_ref(), self.codec.clone(), @@ -636,7 +637,7 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { while let Some(b) = coalescer.next_completed_batch() { ipc_writer.write_batch(&b, &self.metrics.encode_time)?; } - ipc_writer.finish()?; + ipc_writer.finish_length_prefixed(start_pos)?; } } } diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index 55e510c4fd..baf52ebde4 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -82,7 +82,8 @@ impl SinglePartitionShufflePartitioner { ShuffleFormat::IpcStream => { let buf_writer = BufWriter::with_capacity(write_buffer_size, output_data_file); - let writer = IpcStreamWriter::try_new(buf_writer, schema.as_ref(), codec)?; + let writer = + IpcStreamWriter::try_new_length_prefixed(buf_writer, schema.as_ref(), codec)?; OutputWriter::IpcStream { writer: Some(writer), coalescer: None, @@ -181,7 +182,8 @@ impl SinglePartitionShufflePartitioner { fn finish_ipc_stream(&mut self) -> datafusion::common::Result<()> { if let OutputWriter::IpcStream { writer, .. } = &mut self.output { if let Some(w) = writer.take() { - let buf_writer = w.finish()?; + // start_pos is 0 for single partition (length prefix is at file start) + let buf_writer = w.finish_length_prefixed(0)?; buf_writer .into_inner() .map_err(|e| DataFusionError::Execution(format!("flush error: {e}")))?; diff --git a/native/shuffle/src/writers/ipc_stream_writer.rs b/native/shuffle/src/writers/ipc_stream_writer.rs index e483d47490..5a7f3bc34a 100644 --- a/native/shuffle/src/writers/ipc_stream_writer.rs +++ b/native/shuffle/src/writers/ipc_stream_writer.rs @@ -22,7 +22,7 @@ use arrow::ipc::CompressionType; use datafusion::common::DataFusionError; use datafusion::error::Result; use datafusion::physical_plan::metrics::Time; -use std::io::Write; +use std::io::{Seek, SeekFrom, Write}; use super::CompressionCodec; @@ -105,6 +105,35 @@ impl IpcStreamWriter { } } +impl IpcStreamWriter { + /// Creates a new IPC stream writer with space reserved for an 8-byte length + /// prefix. Call [`finish_length_prefixed`](Self::finish_length_prefixed) + /// instead of `finish` to fill in the prefix. + pub fn try_new_length_prefixed( + mut output: W, + schema: &Schema, + codec: CompressionCodec, + ) -> Result { + // Reserve 8 bytes for the length prefix (filled in on finish) + output.write_all(&[0u8; 8])?; + Self::try_new(output, schema, codec) + } + + /// Finishes the IPC stream and fills in the 8-byte length prefix that was + /// reserved by [`try_new_length_prefixed`](Self::try_new_length_prefixed). + /// + /// The length prefix covers the IPC stream data only (not itself). + pub fn finish_length_prefixed(self, start_pos: u64) -> Result { + let mut output = self.finish()?; + let end_pos = output.stream_position()?; + let ipc_length = end_pos - start_pos - 8; + output.seek(SeekFrom::Start(start_pos))?; + output.write_all(&ipc_length.to_le_bytes())?; + output.seek(SeekFrom::Start(end_pos))?; + Ok(output) + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index 8e6acfda4d..e80317fd29 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -24,7 +24,7 @@ use datafusion::common::DataFusionError; use datafusion::execution::disk_manager::RefCountedTempFile; use datafusion::execution::runtime_env::RuntimeEnv; use std::fs::{File, OpenOptions}; -use std::io::BufWriter; +use std::io::{BufWriter, Seek}; /// Manages encoding and optional disk spilling for a single shuffle partition. /// @@ -38,6 +38,8 @@ pub(crate) struct PartitionWriter { /// Persistent IPC stream writer for the spill file, kept open across /// multiple `spill()` calls. ipc_spill_writer: Option>>, + /// Start position of the current IPC stream in the spill file (for length prefix). + ipc_spill_start_pos: u64, format: ShuffleFormat, } @@ -50,6 +52,7 @@ impl PartitionWriter { spill_file: None, shuffle_block_writer, ipc_spill_writer: None, + ipc_spill_start_pos: 0, format, }) } @@ -120,6 +123,8 @@ impl PartitionWriter { ShuffleFormat::IpcStream => { // Lazily open the IPC stream writer on first spill. It stays // open so subsequent spills append batches to the same stream. + // Uses length prefix so the spill file can be raw-copied into + // the output and the reader can frame it. if self.ipc_spill_writer.is_none() { let file = OpenOptions::new() .write(true) @@ -131,9 +136,15 @@ impl PartitionWriter { "Error occurred while spilling {e}" )) })?; - let buf_writer = BufWriter::with_capacity(write_buffer_size, file); - self.ipc_spill_writer = - Some(IpcStreamWriter::try_new(buf_writer, schema, codec.clone())?); + let mut buf_writer = BufWriter::with_capacity(write_buffer_size, file); + self.ipc_spill_start_pos = buf_writer.stream_position()?; + self.ipc_spill_writer = Some( + IpcStreamWriter::try_new_length_prefixed( + buf_writer, + schema, + codec.clone(), + )?, + ); } let ipc_writer = self.ipc_spill_writer.as_mut().unwrap(); ipc_writer.write_batch(&batch?, &metrics.encode_time)?; @@ -149,7 +160,7 @@ impl PartitionWriter { /// before raw-copying the spill file to the output. pub(crate) fn finish_spill(&mut self) -> datafusion::common::Result<()> { if let Some(writer) = self.ipc_spill_writer.take() { - let buf_writer = writer.finish()?; + let buf_writer = writer.finish_length_prefixed(self.ipc_spill_start_pos)?; buf_writer .into_inner() .map_err(|e| DataFusionError::Execution(format!("flush error: {e}")))?; diff --git a/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java b/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java index 9f72b20f51..8d2d57dc2d 100644 --- a/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java +++ b/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java @@ -43,35 +43,49 @@ public class CometShuffleBlockIterator implements Closeable { private static final int INITIAL_BUFFER_SIZE = 128 * 1024; + /** Block format header: 8-byte length + 8-byte field count. */ + private static final int BLOCK_HEADER_SIZE = 16; + /** IPC stream format header: 8-byte length only. */ + private static final int IPC_STREAM_HEADER_SIZE = 8; + private final ReadableByteChannel channel; private final InputStream inputStream; - private final ByteBuffer headerBuf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN); + private final ByteBuffer headerBuf = ByteBuffer.allocate(BLOCK_HEADER_SIZE) + .order(ByteOrder.LITTLE_ENDIAN); private ByteBuffer dataBuf = ByteBuffer.allocateDirect(INITIAL_BUFFER_SIZE); private boolean closed = false; private int currentBlockLength = 0; + private final boolean isIpcStream; public CometShuffleBlockIterator(InputStream in) { + this(in, false); + } + + public CometShuffleBlockIterator(InputStream in, boolean isIpcStream) { this.inputStream = in; this.channel = Channels.newChannel(in); + this.isIpcStream = isIpcStream; } /** - * Reads the next block header and loads the compressed body into the internal buffer. Called by - * native code via JNI. + * Reads the next block header and loads the body into the internal buffer. Called by native code + * via JNI. * - *

Header format: 8-byte compressedLength (includes field count but not itself) + 8-byte - * fieldCount (discarded, schema comes from protobuf). + *

Block format header: 8-byte compressedLength (includes field count but not itself) + 8-byte + * fieldCount (discarded). Body is: 4-byte codec prefix + compressed IPC data. * - * @return the compressed body length in bytes (codec prefix + compressed IPC), or -1 if EOF + *

IPC stream format header: 8-byte length. Body is: raw Arrow IPC stream data. + * + * @return the body length in bytes, or -1 if EOF */ public int hasNext() throws IOException { if (closed) { return -1; } - // Read 16-byte header: clear() resets position=0, limit=capacity, - // preparing the buffer for channel.read() to fill it + int headerSize = isIpcStream ? IPC_STREAM_HEADER_SIZE : BLOCK_HEADER_SIZE; headerBuf.clear(); + headerBuf.limit(headerSize); while (headerBuf.hasRemaining()) { int bytesRead = channel.read(headerBuf); if (bytesRead < 0) { @@ -83,19 +97,25 @@ public int hasNext() throws IOException { } } headerBuf.flip(); - long compressedLength = headerBuf.getLong(); - // Field count discarded - schema determined by ShuffleScan protobuf fields - headerBuf.getLong(); + long length = headerBuf.getLong(); + + long bytesToRead; + if (isIpcStream) { + // IPC stream: length is the IPC stream data size + bytesToRead = length; + } else { + // Block format: length includes the 8-byte field count we already read + headerBuf.getLong(); // discard field count + bytesToRead = length - 8; + } - // Subtract 8 because compressedLength includes the 8-byte field count we already read - long bytesToRead = compressedLength - 8; if (bytesToRead > Integer.MAX_VALUE) { throw new IllegalStateException( "Native shuffle block size of " + bytesToRead + " exceeds maximum of " + Integer.MAX_VALUE - + ". Try reducing spark.comet.columnar.shuffle.batch.size."); + + ". Try reducing shuffle batch size."); } currentBlockLength = (int) bytesToRead; @@ -113,8 +133,6 @@ public int hasNext() throws IOException { throw new EOFException("Data corrupt: unexpected EOF while reading compressed batch"); } } - // Note: native side uses get_direct_buffer_address (base pointer) + currentBlockLength, - // not the buffer's position/limit. No flip needed. return currentBlockLength; } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala index 7604910b06..729fb2128b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsRe import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.comet.CometShuffleBlockIterator +import org.apache.comet.{CometConf, CometShuffleBlockIterator} /** * Different from [[org.apache.spark.sql.execution.ShuffledRowRDD]], this RDD is specialized for @@ -156,7 +156,8 @@ class CometShuffledBatchRDD( split: Partition, context: TaskContext): CometShuffleBlockIterator = { val reader = createReader(split, context) - new CometShuffleBlockIterator(reader.readAsRawStream()) + val isIpcStream = CometConf.COMET_EXEC_SHUFFLE_FORMAT.get() == "ipc_stream" + new CometShuffleBlockIterator(reader.readAsRawStream(), isIpcStream) } override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { From 7819d1aebeb003511ddb21bf91e1f831b9b6c7ab Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 10:39:57 -0600 Subject: [PATCH 13/21] refactor: stream IPC batches lazily instead of materializing all Replace Vec buffering with a persistent StreamReader that yields one batch per get_next_batch() call. The IPC stream bytes are copied into owned memory and a StreamReader is created on first access; subsequent calls iterate the reader without another JNI round-trip or bulk materialization. --- .../src/execution/operators/shuffle_scan.rs | 128 +++++++++++------- 1 file changed, 76 insertions(+), 52 deletions(-) diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 8599bcf0e2..39474d108a 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -20,12 +20,13 @@ use crate::{ execution::{ operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, - shuffle::ipc::{read_ipc_compressed, read_ipc_stream}, + shuffle::ipc::read_ipc_compressed, }, jvm_bridge::{jni_call, JVMClasses}, }; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::ipc::reader::StreamReader; use datafusion::common::{arrow_datafusion_err, Result as DataFusionResult}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{ @@ -47,6 +48,9 @@ use std::{ use super::scan::InputBatch; +/// A StreamReader over owned bytes, used for streaming IPC batches one at a time. +type IpcBatchReader = StreamReader>>; + /// ShuffleScanExec reads compressed shuffle blocks from JVM via JNI and decodes them natively. /// Unlike ScanExec which receives Arrow arrays via FFI, ShuffleScanExec receives raw compressed /// bytes from CometShuffleBlockIterator and decodes them using read_ipc_compressed(). @@ -62,9 +66,9 @@ pub struct ShuffleScanExec { pub schema: SchemaRef, /// The current input batch, populated by get_next_batch() before poll_next(). pub batch: Arc>>, - /// Buffered batches from IPC stream decoding (one IPC stream may contain - /// multiple batches). Consumed one at a time by get_next_batch(). - pending_batches: Vec, + /// Active IPC stream reader for streaming batches one at a time. + /// Wrapped in Arc so ShuffleScanExec can derive Clone. + ipc_stream_reader: Arc>>, /// Cache of plan properties. cache: PlanProperties, /// Metrics collector. @@ -99,7 +103,7 @@ impl ShuffleScanExec { input_source, data_types, batch: Arc::new(Mutex::new(None)), - pending_batches: Vec::new(), + ipc_stream_reader: Arc::new(Mutex::new(None)), cache, metrics: metrics_set, baseline_metrics, @@ -117,48 +121,78 @@ impl ShuffleScanExec { /// because JNI calls cannot happen from within poll_next on tokio threads. pub fn get_next_batch(&mut self) -> Result<(), CometError> { if self.input_source.is_none() { - // Unit test mode - no JNI calls needed. return Ok(()); } let mut timer = self.baseline_metrics.elapsed_compute().timer(); let mut current_batch = self.batch.try_lock().unwrap(); if current_batch.is_none() { - // Check for buffered batches from a previous IPC stream decode - if let Some(batch) = self.pending_batches.pop() { - let num_rows = batch.num_rows(); - let columns: Vec = batch - .columns() - .iter() - .map(|col| unpack_dictionary(col)) - .collect(); - *current_batch = Some(InputBatch::new(columns, Some(num_rows))); + // Try to read the next batch from an active IPC stream reader + let batch_from_reader = { + let mut reader_guard = self.ipc_stream_reader.try_lock().unwrap(); + if let Some(reader) = reader_guard.as_mut() { + match reader.next() { + Some(Ok(batch)) => Some(batch), + _ => { + // Stream exhausted or error — drop the reader + *reader_guard = None; + None + } + } + } else { + None + } + }; + + if let Some(batch) = batch_from_reader { + *current_batch = Some(Self::batch_to_input(&batch, &self.data_types)); } else { let next_batch = Self::get_next( self.exec_context_id, self.input_source.as_ref().unwrap().as_obj(), &self.data_types, &self.decode_time, - &mut self.pending_batches, + &self.ipc_stream_reader, )?; *current_batch = Some(next_batch); } } timer.stop(); - Ok(()) } - /// Invokes JNI calls to get the next compressed shuffle block and decode it. - /// For IPC stream format, multiple batches may be decoded at once; extras - /// are stored in `pending_batches` for subsequent calls. + fn batch_to_input( + batch: &arrow::array::RecordBatch, + data_types: &[DataType], + ) -> InputBatch { + let num_rows = batch.num_rows(); + let columns: Vec = batch + .columns() + .iter() + .map(|col| unpack_dictionary(col)) + .collect(); + + debug_assert_eq!( + columns.len(), + data_types.len(), + "Shuffle block column count mismatch: got {} but expected {}", + columns.len(), + data_types.len() + ); + + InputBatch::new(columns, Some(num_rows)) + } + + /// Invokes JNI calls to get the next shuffle block and decode it. + /// For IPC stream format, creates a `StreamReader` that yields batches + /// one at a time (stored in `ipc_stream_reader` for subsequent calls). fn get_next( exec_context_id: i64, iter: &JObject, data_types: &[DataType], decode_time: &Time, - pending_batches: &mut Vec, + ipc_stream_reader: &Arc>>, ) -> Result { if exec_context_id == TEST_EXEC_CONTEXT_ID { return Ok(InputBatch::EOF); @@ -172,7 +206,6 @@ impl ShuffleScanExec { let mut env = JVMClasses::get_env()?; - // has_next() reads the next block and returns its length, or -1 if EOF let block_length: i32 = unsafe { jni_call!(&mut env, comet_shuffle_block_iterator(iter).has_next() -> i32)? @@ -182,7 +215,6 @@ impl ShuffleScanExec { return Ok(InputBatch::EOF); } - // Get the DirectByteBuffer containing the shuffle block let buffer: JObject = unsafe { jni_call!(&mut env, comet_shuffle_block_iterator(iter).get_buffer() -> JObject)? @@ -193,45 +225,37 @@ impl ShuffleScanExec { let length = block_length as usize; let slice: &[u8] = unsafe { std::slice::from_raw_parts(raw_pointer, length) }; - // Detect format from the first 4 bytes: - // Block format starts with a codec prefix: "SNAP", "LZ4_", "ZSTD", or "NONE" - // IPC stream format starts with Arrow continuation: 0xFFFFFFFF + // Detect format: block starts with codec prefix, IPC stream starts with 0xFFFFFFFF let mut timer = decode_time.timer(); let is_ipc_stream = length >= 4 && slice[0..4] == [0xFF, 0xFF, 0xFF, 0xFF]; let batch = if is_ipc_stream { - let mut batches = read_ipc_stream(slice)?; - if batches.is_empty() { - timer.stop(); - return Ok(InputBatch::EOF); - } - let first = batches.remove(0); - // Store remaining batches in reverse order so pop() returns them in order - batches.reverse(); - *pending_batches = batches; + // Copy bytes into owned memory and create a StreamReader that + // yields batches one at a time (no bulk materialization). + let owned = slice.to_vec(); + let cursor = std::io::Cursor::new(owned); + let mut reader = StreamReader::try_new(cursor, None)?; + let first = match reader.next() { + Some(Ok(batch)) => batch, + Some(Err(e)) => { + timer.stop(); + return Err(e.into()); + } + None => { + timer.stop(); + return Ok(InputBatch::EOF); + } + }; + // Store the reader so subsequent calls can pull more batches + // without another JNI round-trip. + *ipc_stream_reader.try_lock().unwrap() = Some(reader); first } else { read_ipc_compressed(slice)? }; timer.stop(); - let num_rows = batch.num_rows(); - - let columns: Vec = batch - .columns() - .iter() - .map(|col| unpack_dictionary(col)) - .collect(); - - debug_assert_eq!( - columns.len(), - data_types.len(), - "Shuffle block column count mismatch: got {} but expected {}", - columns.len(), - data_types.len() - ); - - Ok(InputBatch::new(columns, Some(num_rows))) + Ok(Self::batch_to_input(&batch, data_types)) } } From aa911fdbf5c0e27ad71639048b73d08dd6c398df Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 10:41:48 -0600 Subject: [PATCH 14/21] style: apply formatting --- .../src/execution/operators/shuffle_scan.rs | 9 +- native/shuffle/src/bin/shuffle_bench.rs | 4 +- .../src/partitioners/multi_partition.rs | 3 +- .../src/partitioners/single_partition.rs | 3 +- .../shuffle/src/writers/ipc_stream_writer.rs | 9 +- native/shuffle/src/writers/spill.rs | 27 ++---- run_shuffle_bench.sh | 83 +++++++++++++++++++ shuffle_bench_results.csv | 5 ++ .../comet/CometShuffleBlockIterator.java | 5 +- 9 files changed, 109 insertions(+), 39 deletions(-) create mode 100755 run_shuffle_bench.sh create mode 100644 shuffle_bench_results.csv diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 39474d108a..24478b1650 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -18,9 +18,7 @@ use crate::{ errors::CometError, execution::{ - operators::ExecutionError, - planner::TEST_EXEC_CONTEXT_ID, - shuffle::ipc::read_ipc_compressed, + operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, shuffle::ipc::read_ipc_compressed, }, jvm_bridge::{jni_call, JVMClasses}, }; @@ -162,10 +160,7 @@ impl ShuffleScanExec { Ok(()) } - fn batch_to_input( - batch: &arrow::array::RecordBatch, - data_types: &[DataType], - ) -> InputBatch { + fn batch_to_input(batch: &arrow::array::RecordBatch, data_types: &[DataType]) -> InputBatch { let num_rows = batch.num_rows(); let columns: Vec = batch .columns() diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 14e8d5d841..d198ce2fd6 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -574,7 +574,9 @@ fn build_partitioning( fn parse_format(format: &str) -> datafusion_comet_shuffle::ShuffleFormat { match format.to_lowercase().as_str() { - "ipc_stream" | "ipc-stream" | "stream" => datafusion_comet_shuffle::ShuffleFormat::IpcStream, + "ipc_stream" | "ipc-stream" | "stream" => { + datafusion_comet_shuffle::ShuffleFormat::IpcStream + } _ => datafusion_comet_shuffle::ShuffleFormat::Block, } } diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 4ebbd86981..378267c052 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -629,8 +629,7 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { for batch in &mut partition_iter { coalescer.push_batch(batch?)?; while let Some(b) = coalescer.next_completed_batch() { - ipc_writer - .write_batch(&b, &self.metrics.encode_time)?; + ipc_writer.write_batch(&b, &self.metrics.encode_time)?; } } coalescer.finish_buffered_batch()?; diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index baf52ebde4..9402781975 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -80,8 +80,7 @@ impl SinglePartitionShufflePartitioner { )) } ShuffleFormat::IpcStream => { - let buf_writer = - BufWriter::with_capacity(write_buffer_size, output_data_file); + let buf_writer = BufWriter::with_capacity(write_buffer_size, output_data_file); let writer = IpcStreamWriter::try_new_length_prefixed(buf_writer, schema.as_ref(), codec)?; OutputWriter::IpcStream { diff --git a/native/shuffle/src/writers/ipc_stream_writer.rs b/native/shuffle/src/writers/ipc_stream_writer.rs index 5a7f3bc34a..4e5c9788a2 100644 --- a/native/shuffle/src/writers/ipc_stream_writer.rs +++ b/native/shuffle/src/writers/ipc_stream_writer.rs @@ -155,11 +155,7 @@ mod tests { Arc::new(schema.clone()), vec![ Arc::new(Int32Array::from(vec![n, n + 1, n + 2])), - Arc::new(StringArray::from(vec![ - Some("hello"), - None, - Some("world"), - ])), + Arc::new(StringArray::from(vec![Some("hello"), None, Some("world")])), ], ) .unwrap() @@ -172,8 +168,7 @@ mod tests { let mut buf = Vec::new(); { let cursor = Cursor::new(&mut buf); - let mut writer = - IpcStreamWriter::try_new(cursor, &schema, codec).unwrap(); + let mut writer = IpcStreamWriter::try_new(cursor, &schema, codec).unwrap(); for i in 0..num_batches { let batch = test_batch(&schema, (i * 10) as i32); writer.write_batch(&batch, &ipc_time).unwrap(); diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index e80317fd29..a20f948e9d 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -95,9 +95,7 @@ impl PartitionWriter { .append(true) .open(&spill_path) .map_err(|e| { - DataFusionError::Execution(format!( - "Error occurred while spilling {e}" - )) + DataFusionError::Execution(format!("Error occurred while spilling {e}")) })?; let mut buf_batch_writer = BufBatchWriter::new( &mut self.shuffle_block_writer, @@ -105,11 +103,8 @@ impl PartitionWriter { write_buffer_size, batch_size, ); - let mut bytes_written = buf_batch_writer.write( - &batch?, - &metrics.encode_time, - &metrics.write_time, - )?; + let mut bytes_written = + buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; for batch in iter { bytes_written += buf_batch_writer.write( &batch?, @@ -132,19 +127,15 @@ impl PartitionWriter { .truncate(true) .open(&spill_path) .map_err(|e| { - DataFusionError::Execution(format!( - "Error occurred while spilling {e}" - )) + DataFusionError::Execution(format!("Error occurred while spilling {e}")) })?; let mut buf_writer = BufWriter::with_capacity(write_buffer_size, file); self.ipc_spill_start_pos = buf_writer.stream_position()?; - self.ipc_spill_writer = Some( - IpcStreamWriter::try_new_length_prefixed( - buf_writer, - schema, - codec.clone(), - )?, - ); + self.ipc_spill_writer = Some(IpcStreamWriter::try_new_length_prefixed( + buf_writer, + schema, + codec.clone(), + )?); } let ipc_writer = self.ipc_spill_writer.as_mut().unwrap(); ipc_writer.write_batch(&batch?, &metrics.encode_time)?; diff --git a/run_shuffle_bench.sh b/run_shuffle_bench.sh new file mode 100755 index 0000000000..9831dde1d8 --- /dev/null +++ b/run_shuffle_bench.sh @@ -0,0 +1,83 @@ +#!/usr/bin/env bash +set -euo pipefail + +BENCH_BIN="./native/target/release/shuffle_bench" +INPUT="/opt/tpch/sf100/lineitem" +CODEC="lz4" +PARTITIONS=200 +HASH_COLUMNS="0,3" +LIMIT=100000000 +CSV_FILE="shuffle_bench_results.csv" + +MEMORY_SIZES=("2g" "4g" "8g" "16g") + +# Convert human-readable size to bytes +to_bytes() { + local val="${1%[gGmM]}" + local unit="${1: -1}" + case "$unit" in + g|G) echo $((val * 1024 * 1024 * 1024)) ;; + m|M) echo $((val * 1024 * 1024)) ;; + *) echo "$1" ;; + esac +} + +echo "memory_limit,memory_limit_bytes,peak_rss_bytes,peak_rss_mb,wall_time_secs,user_time_secs,sys_time_secs,bench_output" > "$CSV_FILE" + +for mem in "${MEMORY_SIZES[@]}"; do + mem_bytes=$(to_bytes "$mem") + output_dir="/tmp/comet_shuffle_bench_${mem}" + rm -rf "$output_dir" + + echo "=== Running with memory_limit=$mem ($mem_bytes bytes) ===" + + # Capture both time -l output (stderr) and benchmark stdout + time_output_file=$(mktemp) + bench_output_file=$(mktemp) + + /usr/bin/time -l "$BENCH_BIN" \ + --input "$INPUT" \ + --codec "$CODEC" \ + --partitions "$PARTITIONS" \ + --hash-columns "$HASH_COLUMNS" \ + --memory-limit "$mem_bytes" \ + --limit "$LIMIT" \ + --output-dir "$output_dir" \ + > "$bench_output_file" 2> "$time_output_file" || true + + # Parse time -l output (macOS format) + wall_time=$(grep "real" "$time_output_file" | awk '{print $1}' || echo "N/A") + user_time=$(grep "user" "$time_output_file" | awk '{print $1}' || echo "N/A") + sys_time=$(grep "sys" "$time_output_file" | awk '{print $1}' || echo "N/A") + peak_rss=$(grep "maximum resident set size" "$time_output_file" | awk '{print $1}' || echo "N/A") + + # macOS time -l reports RSS in bytes + if [ "$peak_rss" != "N/A" ] && [ -n "$peak_rss" ]; then + peak_rss_mb=$((peak_rss / 1024 / 1024)) + else + peak_rss_mb="N/A" + fi + + # Get benchmark output (last few lines, single line for CSV) + bench_summary=$(cat "$bench_output_file" | tr '\n' ' | ' | sed 's/,/;/g' | sed 's/ | $//') + + echo "$mem,$mem_bytes,$peak_rss,$peak_rss_mb,$wall_time,$user_time,$sys_time,\"$bench_summary\"" >> "$CSV_FILE" + + echo " Peak RSS: ${peak_rss_mb} MB" + echo " Wall time: ${wall_time}s" + echo "" + + # Print full time output for reference + echo "--- time -l output ---" + cat "$time_output_file" + echo "--- bench output ---" + cat "$bench_output_file" + echo "---" + echo "" + + rm -f "$time_output_file" "$bench_output_file" +done + +echo "Results saved to $CSV_FILE" +echo "" +cat "$CSV_FILE" diff --git a/shuffle_bench_results.csv b/shuffle_bench_results.csv new file mode 100644 index 0000000000..86c00bdb92 --- /dev/null +++ b/shuffle_bench_results.csv @@ -0,0 +1,5 @@ +memory_limit,memory_limit_bytes,peak_rss_bytes,peak_rss_mb,wall_time_secs,user_time_secs,sys_time_secs,bench_output +2g,2147483648,2934620160,2798,43.84,43.84,43.84,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 2.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 42.689s output: 6.47 GiB === Results === Write: avg time: 42.689s throughput: 2;342;506 rows/s output size: 6.47 GiB " +4g,4294967296,4235100160,4038,43.19,43.19,43.19,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 4.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 42.997s output: 6.45 GiB === Results === Write: avg time: 42.997s throughput: 2;325;757 rows/s output size: 6.45 GiB " +8g,8589934592,6995607552,6671,45.65,45.65,45.65,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 8.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 45.478s output: 6.45 GiB === Results === Write: avg time: 45.478s throughput: 2;198;867 rows/s output size: 6.45 GiB " +16g,17179869184,11275010048,10752,47.92,47.92,47.92,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 16.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 47.624s output: 6.45 GiB === Results === Write: avg time: 47.624s throughput: 2;099;800 rows/s output size: 6.45 GiB " diff --git a/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java b/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java index 8d2d57dc2d..71007d9c16 100644 --- a/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java +++ b/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java @@ -45,13 +45,14 @@ public class CometShuffleBlockIterator implements Closeable { /** Block format header: 8-byte length + 8-byte field count. */ private static final int BLOCK_HEADER_SIZE = 16; + /** IPC stream format header: 8-byte length only. */ private static final int IPC_STREAM_HEADER_SIZE = 8; private final ReadableByteChannel channel; private final InputStream inputStream; - private final ByteBuffer headerBuf = ByteBuffer.allocate(BLOCK_HEADER_SIZE) - .order(ByteOrder.LITTLE_ENDIAN); + private final ByteBuffer headerBuf = + ByteBuffer.allocate(BLOCK_HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN); private ByteBuffer dataBuf = ByteBuffer.allocateDirect(INITIAL_BUFFER_SIZE); private boolean closed = false; private int currentBlockLength = 0; From 01e081a91cf680b5f654e8ca35966adf5e123bac Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 10:44:58 -0600 Subject: [PATCH 15/21] chore: remove benchmark scripts --- run_shuffle_bench.sh | 83 --------------------------------------- shuffle_bench_results.csv | 5 --- 2 files changed, 88 deletions(-) delete mode 100755 run_shuffle_bench.sh delete mode 100644 shuffle_bench_results.csv diff --git a/run_shuffle_bench.sh b/run_shuffle_bench.sh deleted file mode 100755 index 9831dde1d8..0000000000 --- a/run_shuffle_bench.sh +++ /dev/null @@ -1,83 +0,0 @@ -#!/usr/bin/env bash -set -euo pipefail - -BENCH_BIN="./native/target/release/shuffle_bench" -INPUT="/opt/tpch/sf100/lineitem" -CODEC="lz4" -PARTITIONS=200 -HASH_COLUMNS="0,3" -LIMIT=100000000 -CSV_FILE="shuffle_bench_results.csv" - -MEMORY_SIZES=("2g" "4g" "8g" "16g") - -# Convert human-readable size to bytes -to_bytes() { - local val="${1%[gGmM]}" - local unit="${1: -1}" - case "$unit" in - g|G) echo $((val * 1024 * 1024 * 1024)) ;; - m|M) echo $((val * 1024 * 1024)) ;; - *) echo "$1" ;; - esac -} - -echo "memory_limit,memory_limit_bytes,peak_rss_bytes,peak_rss_mb,wall_time_secs,user_time_secs,sys_time_secs,bench_output" > "$CSV_FILE" - -for mem in "${MEMORY_SIZES[@]}"; do - mem_bytes=$(to_bytes "$mem") - output_dir="/tmp/comet_shuffle_bench_${mem}" - rm -rf "$output_dir" - - echo "=== Running with memory_limit=$mem ($mem_bytes bytes) ===" - - # Capture both time -l output (stderr) and benchmark stdout - time_output_file=$(mktemp) - bench_output_file=$(mktemp) - - /usr/bin/time -l "$BENCH_BIN" \ - --input "$INPUT" \ - --codec "$CODEC" \ - --partitions "$PARTITIONS" \ - --hash-columns "$HASH_COLUMNS" \ - --memory-limit "$mem_bytes" \ - --limit "$LIMIT" \ - --output-dir "$output_dir" \ - > "$bench_output_file" 2> "$time_output_file" || true - - # Parse time -l output (macOS format) - wall_time=$(grep "real" "$time_output_file" | awk '{print $1}' || echo "N/A") - user_time=$(grep "user" "$time_output_file" | awk '{print $1}' || echo "N/A") - sys_time=$(grep "sys" "$time_output_file" | awk '{print $1}' || echo "N/A") - peak_rss=$(grep "maximum resident set size" "$time_output_file" | awk '{print $1}' || echo "N/A") - - # macOS time -l reports RSS in bytes - if [ "$peak_rss" != "N/A" ] && [ -n "$peak_rss" ]; then - peak_rss_mb=$((peak_rss / 1024 / 1024)) - else - peak_rss_mb="N/A" - fi - - # Get benchmark output (last few lines, single line for CSV) - bench_summary=$(cat "$bench_output_file" | tr '\n' ' | ' | sed 's/,/;/g' | sed 's/ | $//') - - echo "$mem,$mem_bytes,$peak_rss,$peak_rss_mb,$wall_time,$user_time,$sys_time,\"$bench_summary\"" >> "$CSV_FILE" - - echo " Peak RSS: ${peak_rss_mb} MB" - echo " Wall time: ${wall_time}s" - echo "" - - # Print full time output for reference - echo "--- time -l output ---" - cat "$time_output_file" - echo "--- bench output ---" - cat "$bench_output_file" - echo "---" - echo "" - - rm -f "$time_output_file" "$bench_output_file" -done - -echo "Results saved to $CSV_FILE" -echo "" -cat "$CSV_FILE" diff --git a/shuffle_bench_results.csv b/shuffle_bench_results.csv deleted file mode 100644 index 86c00bdb92..0000000000 --- a/shuffle_bench_results.csv +++ /dev/null @@ -1,5 +0,0 @@ -memory_limit,memory_limit_bytes,peak_rss_bytes,peak_rss_mb,wall_time_secs,user_time_secs,sys_time_secs,bench_output -2g,2147483648,2934620160,2798,43.84,43.84,43.84,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 2.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 42.689s output: 6.47 GiB === Results === Write: avg time: 42.689s throughput: 2;342;506 rows/s output size: 6.47 GiB " -4g,4294967296,4235100160,4038,43.19,43.19,43.19,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 4.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 42.997s output: 6.45 GiB === Results === Write: avg time: 42.997s throughput: 2;325;757 rows/s output size: 6.45 GiB " -8g,8589934592,6995607552,6671,45.65,45.65,45.65,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 8.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 45.478s output: 6.45 GiB === Results === Write: avg time: 45.478s throughput: 2;198;867 rows/s output size: 6.45 GiB " -16g,17179869184,11275010048,10752,47.92,47.92,47.92,"=== Shuffle Benchmark === Input: /opt/tpch/sf100/lineitem Schema: 16 columns (3xdate; 4xdecimal; 4xint; 5xstring) Total rows: 100;000;000 Batch size: 8;192 Partitioning: hash Partitions: 200 Codec: Lz4Frame Hash columns: [0; 3] Memory limit: 16.00 GiB Iterations: 1 (warmup: 0) [iter 1/1] write: 47.624s output: 6.45 GiB === Results === Write: avg time: 47.624s throughput: 2;099;800 rows/s output size: 6.45 GiB " From e39fd1f995c9f86f3d66dc9835295be91fede4cd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 11:06:46 -0600 Subject: [PATCH 16/21] fix: update NativeBatchDecoderIterator for IPC stream format The columnar shuffle read path (NativeBatchDecoderIterator) was not updated for IPC stream format, causing NegativeArraySizeException when the IPC stream continuation token (0xFFFFFFFF) was interpreted as field count = -1. Fix by: - Adding numColumns parameter to NativeBatchDecoderIterator, passed from the dependency schema - For IPC stream format, skip the 8-byte field count header read and use schema-derived column count instead - Auto-detect format in native decodeShuffleBlock JNI function --- native/core/src/execution/jni_api.rs | 15 ++++++- .../CometBlockStoreShuffleReader.scala | 3 +- .../shuffle/NativeBatchDecoderIterator.scala | 40 +++++++++---------- 3 files changed, 36 insertions(+), 22 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index e0a395ebbf..20233208dd 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -893,7 +893,20 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleBlock( let raw_pointer = env.get_direct_buffer_address(&byte_buffer)?; let length = length as usize; let slice: &[u8] = unsafe { std::slice::from_raw_parts(raw_pointer, length) }; - let batch = read_ipc_compressed(slice)?; + + // Auto-detect format: IPC stream starts with Arrow continuation 0xFFFFFFFF + let is_ipc_stream = length >= 4 && slice[0..4] == [0xFF, 0xFF, 0xFF, 0xFF]; + let batch = if is_ipc_stream { + use arrow::ipc::reader::StreamReader; + let mut reader = StreamReader::try_new(slice, None)?; + reader.next().ok_or_else(|| { + datafusion::common::DataFusionError::Execution( + "Empty IPC stream in shuffle block".to_string(), + ) + })?? + } else { + read_ipc_compressed(slice)? + }; prepare_output(&mut env, array_addrs, schema_addrs, batch, false) }) }) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala index 14e656f038..482a6b7e44 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala @@ -114,7 +114,8 @@ class CometBlockStoreShuffleReader[K, C]( dep.decodeTime, nativeLib, nativeUtil, - tracingEnabled) + tracingEnabled, + dep.schema.map(_.fields.length).getOrElse(-1)) currentReadIterator }) .map(b => (0, b)) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala index f96c8f16dd..b6595c5bac 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala @@ -26,7 +26,7 @@ import java.nio.channels.{Channels, ReadableByteChannel} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.comet.Native +import org.apache.comet.{CometConf, Native} import org.apache.comet.vector.NativeUtil /** @@ -39,13 +39,14 @@ case class NativeBatchDecoderIterator( decodeTime: SQLMetric, nativeLib: Native, nativeUtil: NativeUtil, - tracingEnabled: Boolean) + tracingEnabled: Boolean, + numColumns: Int = -1) extends Iterator[ColumnarBatch] { private var isClosed = false private val longBuf = ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN) private var currentBatch: ColumnarBatch = null - private var batch = fetchNext() + private val isIpcStream = CometConf.COMET_EXEC_SHUFFLE_FORMAT.get() == "ipc_stream" import NativeBatchDecoderIterator._ @@ -55,6 +56,8 @@ case class NativeBatchDecoderIterator( null } + private var batch = fetchNext() + def hasNext(): Boolean = { if (channel == null || isClosed) { return false @@ -94,7 +97,7 @@ case class NativeBatchDecoderIterator( return None } - // read compressed batch size from header + // read length from header try { longBuf.clear() while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} @@ -104,8 +107,6 @@ case class NativeBatchDecoderIterator( return None } - // If we reach the end of the stream, we are done, or if we read partial length - // then the stream is corrupted. if (longBuf.hasRemaining) { if (longBuf.position() == 0) { close() @@ -114,31 +115,30 @@ case class NativeBatchDecoderIterator( throw new EOFException("Data corrupt: unexpected EOF while reading compressed ipc lengths") } - // get compressed length (including headers) longBuf.flip() - val compressedLength = longBuf.getLong + val length = longBuf.getLong - // read field count from header - longBuf.clear() - while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} - if (longBuf.hasRemaining) { - throw new EOFException("Data corrupt: unexpected EOF while reading field count") + val (fieldCount, bytesToRead) = if (isIpcStream) { + // IPC stream: length is the data size, field count from schema + (numColumns, length) + } else { + // Block format: read 8-byte field count header + longBuf.clear() + while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} + if (longBuf.hasRemaining) { + throw new EOFException("Data corrupt: unexpected EOF while reading field count") + } + longBuf.flip() + (longBuf.getLong.toInt, length - 8) } - longBuf.flip() - val fieldCount = longBuf.getLong.toInt - // read body - val bytesToRead = compressedLength - 8 if (bytesToRead > Integer.MAX_VALUE) { - // very unlikely that shuffle block will reach 2GB throw new IllegalStateException( s"Native shuffle block size of $bytesToRead exceeds " + s"maximum of ${Integer.MAX_VALUE}. Try reducing shuffle batch size.") } var dataBuf = threadLocalDataBuf.get() if (dataBuf.capacity() < bytesToRead) { - // it is unlikely that we would overflow here since it would - // require a 1GB compressed shuffle block but we check anyway val newCapacity = (bytesToRead * 2L).min(Integer.MAX_VALUE).toInt dataBuf = ByteBuffer.allocateDirect(newCapacity) threadLocalDataBuf.set(dataBuf) From 1c2d1893da065b3333acfba205f846584b2bd9d8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 11:37:06 -0600 Subject: [PATCH 17/21] fix: get column count from outputAttributes for native shuffle dep.schema is None for native shuffle, causing numColumns=-1. Fall back to dep.outputAttributes.length which is always set. Also run prettier on native_shuffle.md. --- docs/source/contributor-guide/native_shuffle.md | 2 +- .../execution/shuffle/CometBlockStoreShuffleReader.scala | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/source/contributor-guide/native_shuffle.md b/docs/source/contributor-guide/native_shuffle.md index 4a5d1fd754..93e63e6235 100644 --- a/docs/source/contributor-guide/native_shuffle.md +++ b/docs/source/contributor-guide/native_shuffle.md @@ -301,7 +301,7 @@ Note: The `snappy` codec is only available with block format. IPC stream format | `spark.comet.exec.shuffle.format` | `block` | Data format: `block` or `ipc_stream` | | `spark.comet.exec.shuffle.compression.codec` | `lz4` | Compression codec | | `spark.comet.exec.shuffle.compression.zstd.level` | `1` | Zstd compression level | -| `spark.comet.shuffle.write.buffer.size` | `1MB` | Write buffer size | +| `spark.comet.shuffle.write.buffer.size` | `1MB` | Write buffer size | | `spark.comet.columnar.shuffle.batch.size` | `8192` | Target rows per batch | ## Comparison with JVM Shuffle diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala index 482a6b7e44..9709f56440 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala @@ -109,13 +109,17 @@ class CometBlockStoreShuffleReader[K, C]( if (currentReadIterator != null) { currentReadIterator.close() } + val numCols = dep.schema + .map(_.fields.length) + .orElse(Some(dep.outputAttributes.length).filter(_ > 0)) + .getOrElse(-1) currentReadIterator = NativeBatchDecoderIterator( blockIdAndStream._2, dep.decodeTime, nativeLib, nativeUtil, tracingEnabled, - dep.schema.map(_.fields.length).getOrElse(-1)) + numCols) currentReadIterator }) .map(b => (0, b)) From 16fd5a6a3a2f81bd1ac933557b3c7bad0098c63f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 11:53:17 -0600 Subject: [PATCH 18/21] fix: return EOF for empty IPC streams in decodeShuffleBlock --- native/core/src/execution/jni_api.rs | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 20233208dd..b643a9d3a4 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -896,18 +896,20 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleBlock( // Auto-detect format: IPC stream starts with Arrow continuation 0xFFFFFFFF let is_ipc_stream = length >= 4 && slice[0..4] == [0xFF, 0xFF, 0xFF, 0xFF]; - let batch = if is_ipc_stream { + if is_ipc_stream { use arrow::ipc::reader::StreamReader; let mut reader = StreamReader::try_new(slice, None)?; - reader.next().ok_or_else(|| { - datafusion::common::DataFusionError::Execution( - "Empty IPC stream in shuffle block".to_string(), - ) - })?? + match reader.next() { + Some(Ok(batch)) => { + prepare_output(&mut env, array_addrs, schema_addrs, batch, false) + } + Some(Err(e)) => Err(e.into()), + None => Ok(-1), // empty stream = EOF + } } else { - read_ipc_compressed(slice)? - }; - prepare_output(&mut env, array_addrs, schema_addrs, batch, false) + let batch = read_ipc_compressed(slice)?; + prepare_output(&mut env, array_addrs, schema_addrs, batch, false) + } }) }) } From 40577592f4b26fb53fbf6ca61256f6db21edf623 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 11:56:10 -0600 Subject: [PATCH 19/21] test: add IPC stream format regression tests Cover the three bugs found during integration: - Empty IPC stream (no batches): must be EOF, not error - Length-prefixed roundtrip and multiple back-to-back streams - IPC stream shuffle write with many partitions (some empty) - IPC stream shuffle write with spilling New tests: 26 total (up from 18). --- native/shuffle/src/shuffle_writer.rs | 53 +++++- .../shuffle/src/writers/ipc_stream_writer.rs | 155 ++++++++++++++++++ 2 files changed, 207 insertions(+), 1 deletion(-) diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 96c4d1dc36..1d0986fee1 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -350,6 +350,41 @@ mod test { shuffle_write_test(10000, 100, 200, Some(10 * 1024 * 1024)); } + // --- IPC stream format tests --- + // These mirror the block format tests above to ensure IPC stream format + // produces valid output for the same scenarios. + + #[test] + #[cfg_attr(miri, ignore)] + fn test_ipc_stream_single_partition() { + shuffle_write_test_with_format(1000, 100, 1, None, ShuffleFormat::IpcStream); + shuffle_write_test_with_format(10000, 10, 1, None, ShuffleFormat::IpcStream); + } + + #[test] + #[cfg_attr(miri, ignore)] + fn test_ipc_stream_multi_partition() { + shuffle_write_test_with_format(1000, 10, 16, None, ShuffleFormat::IpcStream); + } + + /// Regression: many partitions means some receive zero rows, producing + /// empty IPC streams (schema + EOS, no batches). The reader must treat + /// these as EOF rather than erroring. + #[test] + #[cfg_attr(miri, ignore)] + fn test_ipc_stream_many_partitions_some_empty() { + // 200 partitions with only 1000 rows — many partitions will be empty + shuffle_write_test_with_format(1000, 1, 200, None, ShuffleFormat::IpcStream); + } + + /// Regression: spilling with IPC stream format must produce valid + /// length-prefixed streams that can be raw-copied to the output file. + #[test] + #[cfg_attr(miri, ignore)] + fn test_ipc_stream_with_spilling() { + shuffle_write_test_with_format(10000, 100, 200, Some(10 * 1024 * 1024), ShuffleFormat::IpcStream); + } + #[tokio::test] async fn shuffle_partitioner_memory() { let batch = create_batch(900); @@ -412,6 +447,22 @@ mod test { num_batches: usize, num_partitions: usize, memory_limit: Option, + ) { + shuffle_write_test_with_format( + batch_size, + num_batches, + num_partitions, + memory_limit, + ShuffleFormat::Block, + ); + } + + fn shuffle_write_test_with_format( + batch_size: usize, + num_batches: usize, + num_partitions: usize, + memory_limit: Option, + format: ShuffleFormat, ) { let batch = create_batch(batch_size); @@ -472,7 +523,7 @@ mod test { ))), partitioning, CompressionCodec::Zstd(1), - ShuffleFormat::Block, + format.clone(), "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), false, diff --git a/native/shuffle/src/writers/ipc_stream_writer.rs b/native/shuffle/src/writers/ipc_stream_writer.rs index 4e5c9788a2..a0a727ef4d 100644 --- a/native/shuffle/src/writers/ipc_stream_writer.rs +++ b/native/shuffle/src/writers/ipc_stream_writer.rs @@ -257,4 +257,159 @@ mod tests { fn test_single_batch() { roundtrip(CompressionCodec::None, 1); } + + /// Regression test: an IPC stream with zero batches (schema + EOS only) + /// must be readable without error. This happens when a partition receives + /// no rows from a map task. Previously this caused "Empty IPC stream in + /// shuffle block" errors in the reader. + #[test] + fn test_empty_stream_no_batches() { + let schema = test_schema(); + + // Write a stream with zero batches + let mut buf = Vec::new(); + { + let cursor = Cursor::new(&mut buf); + let writer = + IpcStreamWriter::try_new(cursor, &schema, CompressionCodec::None).unwrap(); + // Finish immediately without writing any batches + writer.finish().unwrap(); + } + + assert!(!buf.is_empty(), "Stream should contain schema + EOS"); + + // Read back — should yield zero batches, not error + let cursor = Cursor::new(&buf); + let reader = StreamReader::try_new(cursor, None).unwrap(); + let batches: Vec = reader.map(|r| r.unwrap()).collect(); + assert_eq!(batches.len(), 0); + } + + /// Regression test: length-prefixed IPC streams must roundtrip correctly. + /// The length prefix is needed so the Java reader can frame IPC stream + /// data without parsing Arrow IPC message headers. + #[test] + fn test_length_prefixed_roundtrip() { + let schema = test_schema(); + let ipc_time = Time::default(); + + let mut buf = Vec::new(); + let mut cursor = Cursor::new(&mut buf); + + // Write a length-prefixed stream + let start_pos = cursor.stream_position().unwrap(); + let mut writer = IpcStreamWriter::try_new_length_prefixed( + &mut cursor, + &schema, + CompressionCodec::None, + ) + .unwrap(); + writer + .write_batch(&test_batch(&schema, 0), &ipc_time) + .unwrap(); + writer + .write_batch(&test_batch(&schema, 10), &ipc_time) + .unwrap(); + writer.finish_length_prefixed(start_pos).unwrap(); + + // Verify: first 8 bytes are length prefix, remaining is valid IPC stream + let length = u64::from_le_bytes(buf[0..8].try_into().unwrap()); + assert_eq!(length as usize, buf.len() - 8); + + // The IPC stream data after the prefix should be readable + let ipc_data = &buf[8..]; + let reader = StreamReader::try_new(ipc_data, None).unwrap(); + let batches: Vec = reader.map(|r| r.unwrap()).collect(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 3); + assert_eq!(batches[1].num_rows(), 3); + } + + /// Regression test: length-prefixed empty stream (no batches). + /// The reader must handle this as EOF rather than erroring. + #[test] + fn test_length_prefixed_empty_stream() { + let schema = test_schema(); + + let mut buf = Vec::new(); + let mut cursor = Cursor::new(&mut buf); + + let start_pos = cursor.stream_position().unwrap(); + let writer = IpcStreamWriter::try_new_length_prefixed( + &mut cursor, + &schema, + CompressionCodec::None, + ) + .unwrap(); + writer.finish_length_prefixed(start_pos).unwrap(); + + // Length prefix should point to valid (empty) IPC stream + let length = u64::from_le_bytes(buf[0..8].try_into().unwrap()); + assert_eq!(length as usize, buf.len() - 8); + + let ipc_data = &buf[8..]; + let reader = StreamReader::try_new(ipc_data, None).unwrap(); + let batches: Vec = reader.map(|r| r.unwrap()).collect(); + assert_eq!(batches.len(), 0); + } + + /// Tests that multiple length-prefixed IPC streams can be written + /// back-to-back and read independently. This is how the multi-partition + /// output file is structured (spill stream + remaining-batches stream). + #[test] + fn test_multiple_length_prefixed_streams() { + let schema = test_schema(); + let ipc_time = Time::default(); + + let mut buf = Vec::new(); + let mut cursor = Cursor::new(&mut buf); + + // Write two length-prefixed streams back to back + for base in [0, 100] { + let start_pos = cursor.stream_position().unwrap(); + let mut writer = IpcStreamWriter::try_new_length_prefixed( + &mut cursor, + &schema, + CompressionCodec::None, + ) + .unwrap(); + writer + .write_batch(&test_batch(&schema, base), &ipc_time) + .unwrap(); + writer.finish_length_prefixed(start_pos).unwrap(); + } + + // Read them back: parse length prefix, read IPC stream, repeat + let mut offset = 0; + let mut all_batches = Vec::new(); + while offset < buf.len() { + let length = + u64::from_le_bytes(buf[offset..offset + 8].try_into().unwrap()) as usize; + let ipc_data = &buf[offset + 8..offset + 8 + length]; + let reader = StreamReader::try_new(ipc_data, None).unwrap(); + for batch in reader { + all_batches.push(batch.unwrap()); + } + offset += 8 + length; + } + assert_eq!(all_batches.len(), 2); + assert_eq!( + all_batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + 0 + ); + assert_eq!( + all_batches[1] + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + 100 + ); + } } From 9f89f61a65bc405f551f23c727140e92166c74a0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 12:28:53 -0600 Subject: [PATCH 20/21] perf: skip validation on IPC stream reader, fix spill bytes accounting - Add with_skip_validation(true) to StreamReader for IPC stream shuffle blocks, matching the block format behavior. Data was just written by the same process so validation is unnecessary overhead. - Fix IPC stream spill path returning 0 bytes written, which broke memory accounting. Now tracks actual bytes via stream position. --- native/core/src/execution/operators/shuffle_scan.rs | 4 +++- native/shuffle/src/writers/ipc_stream_writer.rs | 5 +++++ native/shuffle/src/writers/spill.rs | 4 +++- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 24478b1650..b9c58bdd1e 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -229,7 +229,9 @@ impl ShuffleScanExec { // yields batches one at a time (no bulk materialization). let owned = slice.to_vec(); let cursor = std::io::Cursor::new(owned); - let mut reader = StreamReader::try_new(cursor, None)?; + let mut reader = unsafe { + StreamReader::try_new(cursor, None)?.with_skip_validation(true) + }; let first = match reader.next() { Some(Ok(batch)) => batch, Some(Err(e)) => { diff --git a/native/shuffle/src/writers/ipc_stream_writer.rs b/native/shuffle/src/writers/ipc_stream_writer.rs index a0a727ef4d..b3cb428345 100644 --- a/native/shuffle/src/writers/ipc_stream_writer.rs +++ b/native/shuffle/src/writers/ipc_stream_writer.rs @@ -106,6 +106,11 @@ impl IpcStreamWriter { } impl IpcStreamWriter { + /// Returns the current stream position of the underlying writer. + pub fn stream_position(&mut self) -> std::io::Result { + self.writer.get_mut().stream_position() + } + /// Creates a new IPC stream writer with space reserved for an 8-byte length /// prefix. Call [`finish_length_prefixed`](Self::finish_length_prefixed) /// instead of `finish` to fill in the prefix. diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index a20f948e9d..872172a548 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -138,11 +138,13 @@ impl PartitionWriter { )?); } let ipc_writer = self.ipc_spill_writer.as_mut().unwrap(); + let pos_before = ipc_writer.stream_position()?; ipc_writer.write_batch(&batch?, &metrics.encode_time)?; for batch in iter { ipc_writer.write_batch(&batch?, &metrics.encode_time)?; } - Ok(0) + let pos_after = ipc_writer.stream_position()?; + Ok((pos_after - pos_before) as usize) } } } From d6dc3015baffa2fbeaf032f487878806a458aaa7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 29 Mar 2026 13:05:46 -0600 Subject: [PATCH 21/21] style: apply formatting --- .../src/execution/operators/shuffle_scan.rs | 5 ++-- native/shuffle/src/shuffle_writer.rs | 8 ++++++- .../shuffle/src/writers/ipc_stream_writer.rs | 24 +++++++------------ 3 files changed, 17 insertions(+), 20 deletions(-) diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index b9c58bdd1e..c6ffee5251 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -229,9 +229,8 @@ impl ShuffleScanExec { // yields batches one at a time (no bulk materialization). let owned = slice.to_vec(); let cursor = std::io::Cursor::new(owned); - let mut reader = unsafe { - StreamReader::try_new(cursor, None)?.with_skip_validation(true) - }; + let mut reader = + unsafe { StreamReader::try_new(cursor, None)?.with_skip_validation(true) }; let first = match reader.next() { Some(Ok(batch)) => batch, Some(Err(e)) => { diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 1d0986fee1..554a97ff2d 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -382,7 +382,13 @@ mod test { #[test] #[cfg_attr(miri, ignore)] fn test_ipc_stream_with_spilling() { - shuffle_write_test_with_format(10000, 100, 200, Some(10 * 1024 * 1024), ShuffleFormat::IpcStream); + shuffle_write_test_with_format( + 10000, + 100, + 200, + Some(10 * 1024 * 1024), + ShuffleFormat::IpcStream, + ); } #[tokio::test] diff --git a/native/shuffle/src/writers/ipc_stream_writer.rs b/native/shuffle/src/writers/ipc_stream_writer.rs index b3cb428345..9a57a0261d 100644 --- a/native/shuffle/src/writers/ipc_stream_writer.rs +++ b/native/shuffle/src/writers/ipc_stream_writer.rs @@ -275,8 +275,7 @@ mod tests { let mut buf = Vec::new(); { let cursor = Cursor::new(&mut buf); - let writer = - IpcStreamWriter::try_new(cursor, &schema, CompressionCodec::None).unwrap(); + let writer = IpcStreamWriter::try_new(cursor, &schema, CompressionCodec::None).unwrap(); // Finish immediately without writing any batches writer.finish().unwrap(); } @@ -303,12 +302,9 @@ mod tests { // Write a length-prefixed stream let start_pos = cursor.stream_position().unwrap(); - let mut writer = IpcStreamWriter::try_new_length_prefixed( - &mut cursor, - &schema, - CompressionCodec::None, - ) - .unwrap(); + let mut writer = + IpcStreamWriter::try_new_length_prefixed(&mut cursor, &schema, CompressionCodec::None) + .unwrap(); writer .write_batch(&test_batch(&schema, 0), &ipc_time) .unwrap(); @@ -340,12 +336,9 @@ mod tests { let mut cursor = Cursor::new(&mut buf); let start_pos = cursor.stream_position().unwrap(); - let writer = IpcStreamWriter::try_new_length_prefixed( - &mut cursor, - &schema, - CompressionCodec::None, - ) - .unwrap(); + let writer = + IpcStreamWriter::try_new_length_prefixed(&mut cursor, &schema, CompressionCodec::None) + .unwrap(); writer.finish_length_prefixed(start_pos).unwrap(); // Length prefix should point to valid (empty) IPC stream @@ -388,8 +381,7 @@ mod tests { let mut offset = 0; let mut all_batches = Vec::new(); while offset < buf.len() { - let length = - u64::from_le_bytes(buf[offset..offset + 8].try_into().unwrap()) as usize; + let length = u64::from_le_bytes(buf[offset..offset + 8].try_into().unwrap()) as usize; let ipc_data = &buf[offset + 8..offset + 8 + length]; let reader = StreamReader::try_new(ipc_data, None).unwrap(); for batch in reader {