From e32dd525489175e19fbb1701e43fcfdf545a4e38 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 11:31:49 -0700 Subject: [PATCH 01/22] perf: optimize shuffle array element iteration with slice-based append MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Use bulk-append methods for primitive types in SparkUnsafeArray: - Non-nullable path uses append_slice() for optimal memcpy-style copy - Nullable path uses pointer iteration with efficient null bitset reading Supported types: i8, i16, i32, i64, f32, f64, date32, timestamp Benchmark results (10K elements): | Type | Baseline | Optimized | Speedup | |------|----------|-----------|---------| | i32/no_nulls | 6.08µs | 0.65µs | **9.3x** | | i32/with_nulls | 22.49µs | 16.21µs | **1.39x** | | i64/no_nulls | 6.15µs | 1.22µs | **5x** | | i64/with_nulls | 16.41µs | 16.41µs | 1x | | f64/no_nulls | 8.05µs | 1.22µs | **6.6x** | | f64/with_nulls | 16.52µs | 16.21µs | 1.02x | | date32/no_nulls | - | 0.66µs | ~9x | | timestamp/no_nulls | - | 1.21µs | ~5x | Co-Authored-By: Claude Opus 4.5 --- native/core/Cargo.toml | 4 + native/core/benches/array_conversion.rs | 263 +++++++++++++++ native/core/src/execution/shuffle/list.rs | 389 +++++++++++++++++++--- native/core/src/execution/shuffle/mod.rs | 2 +- 4 files changed, 602 insertions(+), 56 deletions(-) create mode 100644 native/core/benches/array_conversion.rs diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 5e30883e35..a4622eb932 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -131,3 +131,7 @@ harness = false [[bench]] name = "parquet_decode" harness = false + +[[bench]] +name = "array_conversion" +harness = false diff --git a/native/core/benches/array_conversion.rs b/native/core/benches/array_conversion.rs new file mode 100644 index 0000000000..4cb1f7d13b --- /dev/null +++ b/native/core/benches/array_conversion.rs @@ -0,0 +1,263 @@ +// 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. + +//! Benchmarks for SparkUnsafeArray to Arrow array conversion. +//! This specifically tests the append_to_builder function used in shuffle read path. + +use arrow::array::builder::{ArrayBuilder, Int32Builder, Int64Builder, Float64Builder, Date32Builder, TimestampMicrosecondBuilder}; +use arrow::datatypes::{DataType, TimeUnit}; +use comet::execution::shuffle::list::{append_to_builder, SparkUnsafeArray}; +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; + +const NUM_ELEMENTS: usize = 10000; + +/// Create a SparkUnsafeArray in memory with i32 elements. +/// Layout: +/// - 8 bytes: num_elements (i64) +/// - null bitset: 8 bytes per 64 elements +/// - element data: 4 bytes per element (i32) +fn create_spark_unsafe_array_i32(num_elements: usize, with_nulls: bool) -> Vec { + // Header size: 8 (num_elements) + ceil(num_elements/64) * 8 (null bitset) + let null_bitset_words = (num_elements + 63) / 64; + let header_size = 8 + null_bitset_words * 8; + let data_size = num_elements * 4; // i32 = 4 bytes + let total_size = header_size + data_size; + + let mut buffer = vec![0u8; total_size]; + + // Write num_elements + buffer[0..8].copy_from_slice(&(num_elements as i64).to_le_bytes()); + + // Write null bitset (set every 10th element as null if with_nulls) + if with_nulls { + for i in (0..num_elements).step_by(10) { + let word_idx = i / 64; + let bit_idx = i % 64; + let word_offset = 8 + word_idx * 8; + let current_word = i64::from_le_bytes(buffer[word_offset..word_offset + 8].try_into().unwrap()); + let new_word = current_word | (1i64 << bit_idx); + buffer[word_offset..word_offset + 8].copy_from_slice(&new_word.to_le_bytes()); + } + } + + // Write element data + for i in 0..num_elements { + let offset = header_size + i * 4; + buffer[offset..offset + 4].copy_from_slice(&(i as i32).to_le_bytes()); + } + + buffer +} + +/// Create a SparkUnsafeArray in memory with i64 elements. +fn create_spark_unsafe_array_i64(num_elements: usize, with_nulls: bool) -> Vec { + let null_bitset_words = (num_elements + 63) / 64; + let header_size = 8 + null_bitset_words * 8; + let data_size = num_elements * 8; // i64 = 8 bytes + let total_size = header_size + data_size; + + let mut buffer = vec![0u8; total_size]; + + // Write num_elements + buffer[0..8].copy_from_slice(&(num_elements as i64).to_le_bytes()); + + // Write null bitset + if with_nulls { + for i in (0..num_elements).step_by(10) { + let word_idx = i / 64; + let bit_idx = i % 64; + let word_offset = 8 + word_idx * 8; + let current_word = i64::from_le_bytes(buffer[word_offset..word_offset + 8].try_into().unwrap()); + let new_word = current_word | (1i64 << bit_idx); + buffer[word_offset..word_offset + 8].copy_from_slice(&new_word.to_le_bytes()); + } + } + + // Write element data + for i in 0..num_elements { + let offset = header_size + i * 8; + buffer[offset..offset + 8].copy_from_slice(&(i as i64).to_le_bytes()); + } + + buffer +} + +/// Create a SparkUnsafeArray in memory with f64 elements. +fn create_spark_unsafe_array_f64(num_elements: usize, with_nulls: bool) -> Vec { + let null_bitset_words = (num_elements + 63) / 64; + let header_size = 8 + null_bitset_words * 8; + let data_size = num_elements * 8; // f64 = 8 bytes + let total_size = header_size + data_size; + + let mut buffer = vec![0u8; total_size]; + + // Write num_elements + buffer[0..8].copy_from_slice(&(num_elements as i64).to_le_bytes()); + + // Write null bitset + if with_nulls { + for i in (0..num_elements).step_by(10) { + let word_idx = i / 64; + let bit_idx = i % 64; + let word_offset = 8 + word_idx * 8; + let current_word = i64::from_le_bytes(buffer[word_offset..word_offset + 8].try_into().unwrap()); + let new_word = current_word | (1i64 << bit_idx); + buffer[word_offset..word_offset + 8].copy_from_slice(&new_word.to_le_bytes()); + } + } + + // Write element data + for i in 0..num_elements { + let offset = header_size + i * 8; + buffer[offset..offset + 8].copy_from_slice(&(i as f64).to_le_bytes()); + } + + buffer +} + +fn benchmark_array_conversion(c: &mut Criterion) { + let mut group = c.benchmark_group("spark_unsafe_array_to_arrow"); + + // Benchmark i32 array conversion + for with_nulls in [false, true] { + let buffer = create_spark_unsafe_array_i32(NUM_ELEMENTS, with_nulls); + let array = SparkUnsafeArray::new(buffer.as_ptr() as i64); + let null_str = if with_nulls { "with_nulls" } else { "no_nulls" }; + + group.bench_with_input( + BenchmarkId::new("i32", null_str), + &(&array, &buffer), + |b, (array, _buffer)| { + b.iter(|| { + let mut builder = Int32Builder::with_capacity(NUM_ELEMENTS); + if with_nulls { + append_to_builder::(&DataType::Int32, &mut builder, array).unwrap(); + } else { + append_to_builder::(&DataType::Int32, &mut builder, array).unwrap(); + } + builder.finish() + }); + }, + ); + } + + // Benchmark i64 array conversion + for with_nulls in [false, true] { + let buffer = create_spark_unsafe_array_i64(NUM_ELEMENTS, with_nulls); + let array = SparkUnsafeArray::new(buffer.as_ptr() as i64); + let null_str = if with_nulls { "with_nulls" } else { "no_nulls" }; + + group.bench_with_input( + BenchmarkId::new("i64", null_str), + &(&array, &buffer), + |b, (array, _buffer)| { + b.iter(|| { + let mut builder = Int64Builder::with_capacity(NUM_ELEMENTS); + if with_nulls { + append_to_builder::(&DataType::Int64, &mut builder, array).unwrap(); + } else { + append_to_builder::(&DataType::Int64, &mut builder, array).unwrap(); + } + builder.finish() + }); + }, + ); + } + + // Benchmark f64 array conversion + for with_nulls in [false, true] { + let buffer = create_spark_unsafe_array_f64(NUM_ELEMENTS, with_nulls); + let array = SparkUnsafeArray::new(buffer.as_ptr() as i64); + let null_str = if with_nulls { "with_nulls" } else { "no_nulls" }; + + group.bench_with_input( + BenchmarkId::new("f64", null_str), + &(&array, &buffer), + |b, (array, _buffer)| { + b.iter(|| { + let mut builder = Float64Builder::with_capacity(NUM_ELEMENTS); + if with_nulls { + append_to_builder::(&DataType::Float64, &mut builder, array).unwrap(); + } else { + append_to_builder::(&DataType::Float64, &mut builder, array).unwrap(); + } + builder.finish() + }); + }, + ); + } + + // Benchmark date32 array conversion (same memory layout as i32) + for with_nulls in [false, true] { + let buffer = create_spark_unsafe_array_i32(NUM_ELEMENTS, with_nulls); + let array = SparkUnsafeArray::new(buffer.as_ptr() as i64); + let null_str = if with_nulls { "with_nulls" } else { "no_nulls" }; + + group.bench_with_input( + BenchmarkId::new("date32", null_str), + &(&array, &buffer), + |b, (array, _buffer)| { + b.iter(|| { + let mut builder = Date32Builder::with_capacity(NUM_ELEMENTS); + if with_nulls { + append_to_builder::(&DataType::Date32, &mut builder, array).unwrap(); + } else { + append_to_builder::(&DataType::Date32, &mut builder, array).unwrap(); + } + builder.finish() + }); + }, + ); + } + + // Benchmark timestamp array conversion (same memory layout as i64) + for with_nulls in [false, true] { + let buffer = create_spark_unsafe_array_i64(NUM_ELEMENTS, with_nulls); + let array = SparkUnsafeArray::new(buffer.as_ptr() as i64); + let null_str = if with_nulls { "with_nulls" } else { "no_nulls" }; + + group.bench_with_input( + BenchmarkId::new("timestamp", null_str), + &(&array, &buffer), + |b, (array, _buffer)| { + b.iter(|| { + let mut builder = TimestampMicrosecondBuilder::with_capacity(NUM_ELEMENTS); + let dt = DataType::Timestamp(TimeUnit::Microsecond, None); + if with_nulls { + append_to_builder::(&dt, &mut builder, array).unwrap(); + } else { + append_to_builder::(&dt, &mut builder, array).unwrap(); + } + builder.finish() + }); + }, + ); + } + + group.finish(); +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = benchmark_array_conversion +} +criterion_main!(benches); diff --git a/native/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs index c31244b87d..e4ee31618a 100644 --- a/native/core/src/execution/shuffle/list.rs +++ b/native/core/src/execution/shuffle/list.rs @@ -90,6 +90,322 @@ impl SparkUnsafeArray { (word & mask) != 0 } } + + /// Returns the null bitset pointer (starts at row_addr + 8). + #[inline] + pub(crate) fn null_bitset_ptr(&self) -> *const i64 { + (self.row_addr + 8) as *const i64 + } + + /// Bulk append i32 values to builder. + /// For non-nullable: uses slice append for optimal performance. + /// For nullable: uses pointer iteration with efficient null bitset reading. + #[inline] + pub(crate) fn append_ints_to_builder( + &self, + builder: &mut Int32Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const i32; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + // Use slice-based append for non-nullable path (much faster) + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const i32, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append i64 values to builder. + #[inline] + pub(crate) fn append_longs_to_builder( + &self, + builder: &mut Int64Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const i64; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const i64, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append i16 values to builder. + #[inline] + pub(crate) fn append_shorts_to_builder( + &self, + builder: &mut Int16Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const i16; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const i16, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append i8 values to builder. + #[inline] + pub(crate) fn append_bytes_to_builder( + &self, + builder: &mut Int8Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const i8; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const i8, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append f32 values to builder. + #[inline] + pub(crate) fn append_floats_to_builder( + &self, + builder: &mut Float32Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const f32; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const f32, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append f64 values to builder. + #[inline] + pub(crate) fn append_doubles_to_builder( + &self, + builder: &mut Float64Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const f64; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const f64, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append boolean values to builder using pointer iteration. + #[inline] + pub(crate) fn append_booleans_to_builder( + &self, + builder: &mut BooleanBuilder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + let mut ptr = self.element_offset as *const u8; + + if NULLABLE { + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr != 0 }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + for _ in 0..num_elements { + builder.append_value(unsafe { *ptr != 0 }); + ptr = unsafe { ptr.add(1) }; + } + } + } + + /// Bulk append timestamp values to builder. + #[inline] + pub(crate) fn append_timestamps_to_builder( + &self, + builder: &mut TimestampMicrosecondBuilder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const i64; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const i64, num_elements) + }; + builder.append_slice(slice); + } + } + + /// Bulk append date values to builder. + #[inline] + pub(crate) fn append_dates_to_builder( + &self, + builder: &mut Date32Builder, + ) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const i32; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + builder.append_value(unsafe { *ptr }); + } + ptr = unsafe { ptr.add(1) }; + } + } else { + let slice = unsafe { + std::slice::from_raw_parts(self.element_offset as *const i32, num_elements) + }; + builder.append_slice(slice); + } + } } pub fn append_to_builder( @@ -112,77 +428,40 @@ pub fn append_to_builder( match data_type { DataType::Boolean => { - add_values!( - BooleanBuilder, - |builder: &mut BooleanBuilder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_boolean(idx)), - |builder: &mut BooleanBuilder| builder.append_null() - ); + let builder = downcast_builder_ref!(BooleanBuilder, builder); + array.append_booleans_to_builder::(builder); } DataType::Int8 => { - add_values!( - Int8Builder, - |builder: &mut Int8Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_byte(idx)), - |builder: &mut Int8Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Int8Builder, builder); + array.append_bytes_to_builder::(builder); } DataType::Int16 => { - add_values!( - Int16Builder, - |builder: &mut Int16Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_short(idx)), - |builder: &mut Int16Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Int16Builder, builder); + array.append_shorts_to_builder::(builder); } DataType::Int32 => { - add_values!( - Int32Builder, - |builder: &mut Int32Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_int(idx)), - |builder: &mut Int32Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Int32Builder, builder); + array.append_ints_to_builder::(builder); } DataType::Int64 => { - add_values!( - Int64Builder, - |builder: &mut Int64Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_long(idx)), - |builder: &mut Int64Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Int64Builder, builder); + array.append_longs_to_builder::(builder); } DataType::Float32 => { - add_values!( - Float32Builder, - |builder: &mut Float32Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_float(idx)), - |builder: &mut Float32Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Float32Builder, builder); + array.append_floats_to_builder::(builder); } DataType::Float64 => { - add_values!( - Float64Builder, - |builder: &mut Float64Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_double(idx)), - |builder: &mut Float64Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Float64Builder, builder); + array.append_doubles_to_builder::(builder); } DataType::Timestamp(TimeUnit::Microsecond, _) => { - add_values!( - TimestampMicrosecondBuilder, - |builder: &mut TimestampMicrosecondBuilder, - values: &SparkUnsafeArray, - idx: usize| builder.append_value(values.get_timestamp(idx)), - |builder: &mut TimestampMicrosecondBuilder| builder.append_null() - ); + let builder = downcast_builder_ref!(TimestampMicrosecondBuilder, builder); + array.append_timestamps_to_builder::(builder); } DataType::Date32 => { - add_values!( - Date32Builder, - |builder: &mut Date32Builder, values: &SparkUnsafeArray, idx: usize| builder - .append_value(values.get_date(idx)), - |builder: &mut Date32Builder| builder.append_null() - ); + let builder = downcast_builder_ref!(Date32Builder, builder); + array.append_dates_to_builder::(builder); } DataType::Binary => { add_values!( diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index e2798df63e..172dc5f942 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -17,7 +17,7 @@ pub(crate) mod codec; mod comet_partitioning; -mod list; +pub mod list; mod map; pub mod row; mod shuffle_writer; From 2ea5631ffc6f0a6377bc6f76b7144cbb7087bcb8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 11:32:59 -0700 Subject: [PATCH 02/22] chore: format code Co-Authored-By: Claude Opus 4.5 --- native/core/benches/array_conversion.rs | 17 ++++++++++++----- native/core/src/execution/shuffle/list.rs | 15 +++------------ 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/native/core/benches/array_conversion.rs b/native/core/benches/array_conversion.rs index 4cb1f7d13b..ed6639a146 100644 --- a/native/core/benches/array_conversion.rs +++ b/native/core/benches/array_conversion.rs @@ -18,7 +18,10 @@ //! Benchmarks for SparkUnsafeArray to Arrow array conversion. //! This specifically tests the append_to_builder function used in shuffle read path. -use arrow::array::builder::{ArrayBuilder, Int32Builder, Int64Builder, Float64Builder, Date32Builder, TimestampMicrosecondBuilder}; +use arrow::array::builder::{ + ArrayBuilder, Date32Builder, Float64Builder, Int32Builder, Int64Builder, + TimestampMicrosecondBuilder, +}; use arrow::datatypes::{DataType, TimeUnit}; use comet::execution::shuffle::list::{append_to_builder, SparkUnsafeArray}; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; @@ -48,7 +51,8 @@ fn create_spark_unsafe_array_i32(num_elements: usize, with_nulls: bool) -> Vec Vec Vec(&DataType::Float64, &mut builder, array).unwrap(); } else { - append_to_builder::(&DataType::Float64, &mut builder, array).unwrap(); + append_to_builder::(&DataType::Float64, &mut builder, array) + .unwrap(); } builder.finish() }); diff --git a/native/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs index e4ee31618a..267f343ff5 100644 --- a/native/core/src/execution/shuffle/list.rs +++ b/native/core/src/execution/shuffle/list.rs @@ -101,10 +101,7 @@ impl SparkUnsafeArray { /// For non-nullable: uses slice append for optimal performance. /// For nullable: uses pointer iteration with efficient null bitset reading. #[inline] - pub(crate) fn append_ints_to_builder( - &self, - builder: &mut Int32Builder, - ) { + pub(crate) fn append_ints_to_builder(&self, builder: &mut Int32Builder) { let num_elements = self.num_elements; if num_elements == 0 { return; @@ -136,10 +133,7 @@ impl SparkUnsafeArray { /// Bulk append i64 values to builder. #[inline] - pub(crate) fn append_longs_to_builder( - &self, - builder: &mut Int64Builder, - ) { + pub(crate) fn append_longs_to_builder(&self, builder: &mut Int64Builder) { let num_elements = self.num_elements; if num_elements == 0 { return; @@ -204,10 +198,7 @@ impl SparkUnsafeArray { /// Bulk append i8 values to builder. #[inline] - pub(crate) fn append_bytes_to_builder( - &self, - builder: &mut Int8Builder, - ) { + pub(crate) fn append_bytes_to_builder(&self, builder: &mut Int8Builder) { let num_elements = self.num_elements; if num_elements == 0 { return; From a224e222f3926cca8b404288ad2b0d8145a62643 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 11:35:44 -0700 Subject: [PATCH 03/22] refactor: remove unnecessary #[inline] from large functions The #[inline] attribute on functions with loops iterating over thousands of elements provides no benefit - the function call overhead is negligible compared to loop body execution, and inlining large functions causes instruction cache pressure. Keep #[inline] only on small helper functions: - get_header_portion_in_bytes (tiny const fn) - is_null_at (small, hot path) - null_bitset_ptr (tiny accessor) Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/list.rs | 9 --------- 1 file changed, 9 deletions(-) diff --git a/native/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs index 267f343ff5..6236bdf5c1 100644 --- a/native/core/src/execution/shuffle/list.rs +++ b/native/core/src/execution/shuffle/list.rs @@ -100,7 +100,6 @@ impl SparkUnsafeArray { /// Bulk append i32 values to builder. /// For non-nullable: uses slice append for optimal performance. /// For nullable: uses pointer iteration with efficient null bitset reading. - #[inline] pub(crate) fn append_ints_to_builder(&self, builder: &mut Int32Builder) { let num_elements = self.num_elements; if num_elements == 0 { @@ -132,7 +131,6 @@ impl SparkUnsafeArray { } /// Bulk append i64 values to builder. - #[inline] pub(crate) fn append_longs_to_builder(&self, builder: &mut Int64Builder) { let num_elements = self.num_elements; if num_elements == 0 { @@ -163,7 +161,6 @@ impl SparkUnsafeArray { } /// Bulk append i16 values to builder. - #[inline] pub(crate) fn append_shorts_to_builder( &self, builder: &mut Int16Builder, @@ -197,7 +194,6 @@ impl SparkUnsafeArray { } /// Bulk append i8 values to builder. - #[inline] pub(crate) fn append_bytes_to_builder(&self, builder: &mut Int8Builder) { let num_elements = self.num_elements; if num_elements == 0 { @@ -228,7 +224,6 @@ impl SparkUnsafeArray { } /// Bulk append f32 values to builder. - #[inline] pub(crate) fn append_floats_to_builder( &self, builder: &mut Float32Builder, @@ -262,7 +257,6 @@ impl SparkUnsafeArray { } /// Bulk append f64 values to builder. - #[inline] pub(crate) fn append_doubles_to_builder( &self, builder: &mut Float64Builder, @@ -296,7 +290,6 @@ impl SparkUnsafeArray { } /// Bulk append boolean values to builder using pointer iteration. - #[inline] pub(crate) fn append_booleans_to_builder( &self, builder: &mut BooleanBuilder, @@ -331,7 +324,6 @@ impl SparkUnsafeArray { } /// Bulk append timestamp values to builder. - #[inline] pub(crate) fn append_timestamps_to_builder( &self, builder: &mut TimestampMicrosecondBuilder, @@ -365,7 +357,6 @@ impl SparkUnsafeArray { } /// Bulk append date values to builder. - #[inline] pub(crate) fn append_dates_to_builder( &self, builder: &mut Date32Builder, From fe5454877e8207e5c25e5134586bfd7fbb9533a3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 11:41:31 -0700 Subject: [PATCH 04/22] fix: address clippy warnings in benchmark - Remove unused ArrayBuilder import - Use div_ceil() instead of manual implementation Co-Authored-By: Claude Opus 4.5 --- native/core/benches/array_conversion.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/native/core/benches/array_conversion.rs b/native/core/benches/array_conversion.rs index ed6639a146..5007bff1ce 100644 --- a/native/core/benches/array_conversion.rs +++ b/native/core/benches/array_conversion.rs @@ -19,8 +19,7 @@ //! This specifically tests the append_to_builder function used in shuffle read path. use arrow::array::builder::{ - ArrayBuilder, Date32Builder, Float64Builder, Int32Builder, Int64Builder, - TimestampMicrosecondBuilder, + Date32Builder, Float64Builder, Int32Builder, Int64Builder, TimestampMicrosecondBuilder, }; use arrow::datatypes::{DataType, TimeUnit}; use comet::execution::shuffle::list::{append_to_builder, SparkUnsafeArray}; @@ -35,7 +34,7 @@ const NUM_ELEMENTS: usize = 10000; /// - element data: 4 bytes per element (i32) fn create_spark_unsafe_array_i32(num_elements: usize, with_nulls: bool) -> Vec { // Header size: 8 (num_elements) + ceil(num_elements/64) * 8 (null bitset) - let null_bitset_words = (num_elements + 63) / 64; + let null_bitset_words = num_elements.div_ceil(64); let header_size = 8 + null_bitset_words * 8; let data_size = num_elements * 4; // i32 = 4 bytes let total_size = header_size + data_size; @@ -69,7 +68,7 @@ fn create_spark_unsafe_array_i32(num_elements: usize, with_nulls: bool) -> Vec Vec { - let null_bitset_words = (num_elements + 63) / 64; + let null_bitset_words = num_elements.div_ceil(64); let header_size = 8 + null_bitset_words * 8; let data_size = num_elements * 8; // i64 = 8 bytes let total_size = header_size + data_size; @@ -103,7 +102,7 @@ fn create_spark_unsafe_array_i64(num_elements: usize, with_nulls: bool) -> Vec Vec { - let null_bitset_words = (num_elements + 63) / 64; + let null_bitset_words = num_elements.div_ceil(64); let header_size = 8 + null_bitset_words * 8; let data_size = num_elements * 8; // f64 = 8 bytes let total_size = header_size + data_size; From 471fb2ac143f1a6b8676e39a25534b1b001e8ca1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 12:19:56 -0700 Subject: [PATCH 05/22] perf: optimize struct field processing with field-major order MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Optimize struct field processing in native shuffle by using field-major instead of row-major order. This moves type dispatch from O(rows × fields) to O(fields), eliminating per-row type matching overhead. Previously, for each row we iterated over all fields and called `append_field()` which did a type match for EVERY field in EVERY row. For a struct with N fields and M rows, that's N×M type matches. The new approach: 1. First pass: Loop over rows, build struct validity 2. Second pass: For each field, get typed builder once, then process all rows for that field This keeps type dispatch at O(fields) instead of O(rows × fields). For complex nested types (struct, list, map), falls back to existing `append_field` since they have their own recursive processing logic. Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/row.rs | 231 ++++++++++++++++++++--- 1 file changed, 210 insertions(+), 21 deletions(-) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 821607ddb9..73a126ff67 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -439,6 +439,205 @@ pub(crate) fn append_field( Ok(()) } +/// Appends struct fields to the struct builder using field-major order. +/// This processes one field at a time across all rows, which moves type dispatch +/// outside the row loop (O(fields) dispatches instead of O(rows × fields)). +#[allow(clippy::redundant_closure_call, clippy::too_many_arguments)] +fn append_struct_fields_field_major( + row_addresses_ptr: *mut jlong, + row_sizes_ptr: *mut jint, + row_start: usize, + row_end: usize, + parent_row: &mut SparkUnsafeRow, + column_idx: usize, + struct_builder: &mut StructBuilder, + fields: &arrow::datatypes::Fields, +) -> Result<(), CometError> { + let num_rows = row_end - row_start; + let num_fields = fields.len(); + + // First pass: Build struct validity and collect which structs are null + // We use a Vec for simplicity; could use a bitset for better memory + let mut struct_is_null = Vec::with_capacity(num_rows); + + for i in row_start..row_end { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + + let is_null = parent_row.is_null_at(column_idx); + struct_is_null.push(is_null); + + if is_null { + struct_builder.append_null(); + } else { + struct_builder.append(true); + } + } + + // Helper macro for processing primitive fields + macro_rules! process_field { + ($builder_type:ty, $field_idx:expr, $get_value:expr) => {{ + let field_builder = struct_builder + .field_builder::<$builder_type>($field_idx) + .unwrap(); + + for (row_idx, i) in (row_start..row_end).enumerate() { + if struct_is_null[row_idx] { + // Struct is null, field is also null + field_builder.append_null(); + } else { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + let nested_row = parent_row.get_struct(column_idx, num_fields); + + if nested_row.is_null_at($field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value($get_value(&nested_row, $field_idx)); + } + } + } + }}; + } + + // Second pass: Process each field across all rows + for (field_idx, field) in fields.iter().enumerate() { + match field.data_type() { + DataType::Boolean => { + process_field!(BooleanBuilder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_boolean(idx)); + } + DataType::Int8 => { + process_field!(Int8Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_byte(idx)); + } + DataType::Int16 => { + process_field!(Int16Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_short(idx)); + } + DataType::Int32 => { + process_field!(Int32Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_int(idx)); + } + DataType::Int64 => { + process_field!(Int64Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_long(idx)); + } + DataType::Float32 => { + process_field!(Float32Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_float(idx)); + } + DataType::Float64 => { + process_field!(Float64Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_double(idx)); + } + DataType::Date32 => { + process_field!(Date32Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_date(idx)); + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + process_field!( + TimestampMicrosecondBuilder, + field_idx, + |row: &SparkUnsafeRow, idx| row.get_timestamp(idx) + ); + } + DataType::Binary => { + let field_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + for (row_idx, i) in (row_start..row_end).enumerate() { + if struct_is_null[row_idx] { + field_builder.append_null(); + } else { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + let nested_row = parent_row.get_struct(column_idx, num_fields); + + if nested_row.is_null_at(field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value(nested_row.get_binary(field_idx)); + } + } + } + } + DataType::Utf8 => { + let field_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + for (row_idx, i) in (row_start..row_end).enumerate() { + if struct_is_null[row_idx] { + field_builder.append_null(); + } else { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + let nested_row = parent_row.get_struct(column_idx, num_fields); + + if nested_row.is_null_at(field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value(nested_row.get_string(field_idx)); + } + } + } + } + DataType::Decimal128(p, _) => { + let p = *p; + let field_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + for (row_idx, i) in (row_start..row_end).enumerate() { + if struct_is_null[row_idx] { + field_builder.append_null(); + } else { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + let nested_row = parent_row.get_struct(column_idx, num_fields); + + if nested_row.is_null_at(field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value(nested_row.get_decimal(field_idx, p)); + } + } + } + } + // For complex types (struct, list, map), fall back to append_field + // since they have their own nested processing logic + dt @ (DataType::Struct(_) | DataType::List(_) | DataType::Map(_, _)) => { + for (row_idx, i) in (row_start..row_end).enumerate() { + let nested_row = if struct_is_null[row_idx] { + SparkUnsafeRow::default() + } else { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + parent_row.get_struct(column_idx, num_fields) + }; + append_field(dt, struct_builder, &nested_row, field_idx)?; + } + } + _ => { + unreachable!( + "Unsupported data type of struct field: {:?}", + field.data_type() + ) + } + } + } + + Ok(()) +} + /// Appends column of top rows to the given array builder. #[allow(clippy::redundant_closure_call, clippy::too_many_arguments)] pub(crate) fn append_columns( @@ -637,27 +836,17 @@ pub(crate) fn append_columns( .expect("StructBuilder"); let mut row = SparkUnsafeRow::new(schema); - for i in row_start..row_end { - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); - - let is_null = row.is_null_at(column_idx); - - let nested_row = if is_null { - // The struct is null. - // Append a null value to the struct builder and field builders. - struct_builder.append_null(); - SparkUnsafeRow::default() - } else { - struct_builder.append(true); - row.get_struct(column_idx, fields.len()) - }; - - for (idx, field) in fields.into_iter().enumerate() { - append_field(field.data_type(), struct_builder, &nested_row, idx)?; - } - } + // Use field-major processing to avoid per-row type dispatch + append_struct_fields_field_major( + row_addresses_ptr, + row_sizes_ptr, + row_start, + row_end, + &mut row, + column_idx, + struct_builder, + fields, + )?; } _ => { unreachable!("Unsupported data type of column: {:?}", dt) From bbe2da7c8c37a269e7ad861ed0fe44fd90da4ef6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 12:37:15 -0700 Subject: [PATCH 06/22] test: add benchmark for struct column processing Add a Criterion benchmark to measure the performance of struct column processing in native shuffle. Tests various struct sizes (5, 10, 20 fields) and row counts (1K, 10K rows). Co-Authored-By: Claude Opus 4.5 --- native/core/Cargo.toml | 4 + native/core/benches/struct_conversion.rs | 174 +++++++++++++++++++++++ 2 files changed, 178 insertions(+) create mode 100644 native/core/benches/struct_conversion.rs diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 5e30883e35..8799fcad4d 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -131,3 +131,7 @@ harness = false [[bench]] name = "parquet_decode" harness = false + +[[bench]] +name = "struct_conversion" +harness = false diff --git a/native/core/benches/struct_conversion.rs b/native/core/benches/struct_conversion.rs new file mode 100644 index 0000000000..e00c742a61 --- /dev/null +++ b/native/core/benches/struct_conversion.rs @@ -0,0 +1,174 @@ +// 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. + +//! Benchmark for struct column processing in native shuffle. +//! +//! This benchmark measures the performance of converting Spark UnsafeRow +//! with struct columns to Arrow arrays. + +use arrow::datatypes::{DataType, Field, Fields}; +use comet::execution::shuffle::row::{ + process_sorted_row_partition, SparkUnsafeObject, SparkUnsafeRow, +}; +use comet::execution::shuffle::CompressionCodec; +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; +use tempfile::Builder; + +const BATCH_SIZE: usize = 5000; + +/// Create a struct schema with the given number of int64 fields. +fn make_struct_schema(num_fields: usize) -> DataType { + let fields: Vec = (0..num_fields) + .map(|i| Field::new(format!("f{}", i), DataType::Int64, true)) + .collect(); + DataType::Struct(Fields::from(fields)) +} + +/// Calculate the row size for a struct with the given number of fields. +/// UnsafeRow layout: [null bits] [fixed-length values] +/// For struct: the struct value is stored as offset+size (8 bytes) pointing to nested row +fn get_row_size(num_struct_fields: usize) -> usize { + // Top-level row has 1 column (the struct) + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + // Struct pointer (offset + size) is 8 bytes + let struct_pointer_size = 8; + // Nested struct row + let nested_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_struct_fields); + let nested_data_size = num_struct_fields * 8; // int64 values + + top_level_bitset_width + struct_pointer_size + nested_bitset_width + nested_data_size +} + +struct RowData { + data: Vec, +} + +impl RowData { + fn new(num_struct_fields: usize) -> Self { + let row_size = get_row_size(num_struct_fields); + let mut data = vec![0u8; row_size]; + + // Top-level row layout: + // [null bits for 1 field] [struct pointer (offset, size)] + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + + // Nested struct starts after top-level row header + pointer + let nested_offset = top_level_bitset_width + 8; + let nested_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_struct_fields); + let nested_size = nested_bitset_width + num_struct_fields * 8; + + // Write struct pointer (offset in upper 32 bits, size in lower 32 bits) + let offset_and_size = ((nested_offset as i64) << 32) | (nested_size as i64); + data[top_level_bitset_width..top_level_bitset_width + 8] + .copy_from_slice(&offset_and_size.to_le_bytes()); + + // Fill nested struct with some data + for i in 0..num_struct_fields { + let value_offset = nested_offset + nested_bitset_width + i * 8; + let value = (i as i64) * 100; + data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + + RowData { data } + } + + fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { + spark_row.point_to_slice(&self.data); + } +} + +fn benchmark_struct_conversion(c: &mut Criterion) { + let mut group = c.benchmark_group("struct_conversion"); + + // Test with different struct sizes and row counts + for num_fields in [5, 10, 20] { + for num_rows in [1000, 10000] { + let schema = vec![make_struct_schema(num_fields)]; + + // Create row data + let rows: Vec = (0..num_rows).map(|_| RowData::new(num_fields)).collect(); + + let mut spark_rows: Vec = rows + .iter() + .map(|row_data| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); + row_data.to_spark_row(&mut spark_row); + // Mark the struct column as not null + spark_row.set_not_null_at(0); + spark_row + }) + .collect(); + + let mut row_addresses: Vec = spark_rows + .iter() + .map(|row| row.get_row_addr()) + .collect(); + let mut row_sizes: Vec = spark_rows + .iter() + .map(|row| row.get_row_size()) + .collect(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + + group.bench_with_input( + BenchmarkId::new( + format!("fields_{}", num_fields), + format!("rows_{}", num_rows), + ), + &(num_rows, &schema), + |b, (num_rows, schema)| { + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + *num_rows, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }, + ); + + // Keep spark_rows alive for the benchmark + std::mem::drop(spark_rows); + } + } + + group.finish(); +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = benchmark_struct_conversion +} +criterion_main!(benches); From f8fe7ba4d1bb99d488d7fe6af26ab74c5a4f12a9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 Jan 2026 12:39:57 -0700 Subject: [PATCH 07/22] fix: remove unused import and mut in benchmark --- native/core/benches/struct_conversion.rs | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/native/core/benches/struct_conversion.rs b/native/core/benches/struct_conversion.rs index e00c742a61..953978e263 100644 --- a/native/core/benches/struct_conversion.rs +++ b/native/core/benches/struct_conversion.rs @@ -26,7 +26,6 @@ use comet::execution::shuffle::row::{ }; use comet::execution::shuffle::CompressionCodec; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use std::sync::Arc; use tempfile::Builder; const BATCH_SIZE: usize = 5000; @@ -103,7 +102,7 @@ fn benchmark_struct_conversion(c: &mut Criterion) { // Create row data let rows: Vec = (0..num_rows).map(|_| RowData::new(num_fields)).collect(); - let mut spark_rows: Vec = rows + let spark_rows: Vec = rows .iter() .map(|row_data| { let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); @@ -114,14 +113,9 @@ fn benchmark_struct_conversion(c: &mut Criterion) { }) .collect(); - let mut row_addresses: Vec = spark_rows - .iter() - .map(|row| row.get_row_addr()) - .collect(); - let mut row_sizes: Vec = spark_rows - .iter() - .map(|row| row.get_row_size()) - .collect(); + let mut row_addresses: Vec = + spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); let row_address_ptr = row_addresses.as_mut_ptr(); let row_size_ptr = row_sizes.as_mut_ptr(); From f3da0dcdbe96792bcd2b047e5abe0ace7eb1b725 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 09:04:37 -0700 Subject: [PATCH 08/22] perf: extend field-major processing to nested struct fields This extends the field-major optimization (from commit 471fb2ac143) to recursively handle nested Struct fields. Previously, nested structs fell back to row-major processing via `append_field`, losing the benefit of field-major processing at each nesting level. Changes: - Add `append_nested_struct_fields_field_major` helper function that recursively processes nested struct fields using field-major order - Update `append_struct_fields_field_major` to use field-major processing for nested Struct fields instead of falling back to `append_field` - Add benchmarks for 2-level and 3-level nested structs The optimization: 1. Gets the nested StructBuilder once per field 2. Builds nested struct validity in one pass 3. Recursively applies field-major processing to nested struct fields List and Map fields continue to fall back to `append_field` since they have variable-length elements that are harder to optimize. Co-Authored-By: Claude Opus 4.5 --- native/core/benches/struct_conversion.rs | 305 ++++++++++++++++++++++- native/core/src/execution/shuffle/row.rs | 287 ++++++++++++++++++++- 2 files changed, 583 insertions(+), 9 deletions(-) diff --git a/native/core/benches/struct_conversion.rs b/native/core/benches/struct_conversion.rs index 953978e263..388b31912a 100644 --- a/native/core/benches/struct_conversion.rs +++ b/native/core/benches/struct_conversion.rs @@ -156,6 +156,309 @@ fn benchmark_struct_conversion(c: &mut Criterion) { group.finish(); } +/// Create a schema with nested structs: Struct> +fn make_nested_struct_schema(num_fields: usize) -> DataType { + let inner_fields: Vec = (0..num_fields) + .map(|i| Field::new(format!("inner_f{}", i), DataType::Int64, true)) + .collect(); + let inner_struct = DataType::Struct(Fields::from(inner_fields)); + let outer_fields = vec![Field::new("nested", inner_struct, true)]; + DataType::Struct(Fields::from(outer_fields)) +} + +/// Create a schema with deeply nested structs (3 levels): Struct>> +fn make_deeply_nested_struct_schema(num_fields: usize) -> DataType { + let inner_fields: Vec = (0..num_fields) + .map(|i| Field::new(format!("inner_f{}", i), DataType::Int64, true)) + .collect(); + let inner_struct = DataType::Struct(Fields::from(inner_fields)); + let middle_fields = vec![Field::new("level2", inner_struct, true)]; + let middle_struct = DataType::Struct(Fields::from(middle_fields)); + let outer_fields = vec![Field::new("level1", middle_struct, true)]; + DataType::Struct(Fields::from(outer_fields)) +} + +/// Calculate row size for nested struct: Struct> +fn get_nested_row_size(num_inner_fields: usize) -> usize { + // Top-level row has 1 column (the outer struct) + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let struct_pointer_size = 8; + + // Outer struct has 1 field (the inner struct) + let outer_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let outer_struct_size = outer_bitset_width + 8; // pointer to inner struct + + // Inner struct has num_inner_fields int64 fields + let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); + let inner_data_size = num_inner_fields * 8; + let inner_struct_size = inner_bitset_width + inner_data_size; + + top_level_bitset_width + struct_pointer_size + outer_struct_size + inner_struct_size +} + +/// Calculate row size for deeply nested struct: Struct>> +fn get_deeply_nested_row_size(num_inner_fields: usize) -> usize { + // Top-level row has 1 column (the level1 struct) + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let struct_pointer_size = 8; + + // Level 1 struct has 1 field (the level2 struct) + let level1_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let level1_struct_size = level1_bitset_width + 8; + + // Level 2 struct has 1 field (the inner struct) + let level2_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let level2_struct_size = level2_bitset_width + 8; + + // Inner struct has num_inner_fields int64 fields + let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); + let inner_data_size = num_inner_fields * 8; + let inner_struct_size = inner_bitset_width + inner_data_size; + + top_level_bitset_width + + struct_pointer_size + + level1_struct_size + + level2_struct_size + + inner_struct_size +} + +struct NestedRowData { + data: Vec, +} + +impl NestedRowData { + fn new(num_inner_fields: usize) -> Self { + let row_size = get_nested_row_size(num_inner_fields); + let mut data = vec![0u8; row_size]; + + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let outer_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); + + // Calculate offsets + let outer_struct_start = top_level_bitset_width + 8; + let outer_struct_size = outer_bitset_width + 8; + let inner_struct_start = outer_struct_start + outer_struct_size; + let inner_struct_size = inner_bitset_width + num_inner_fields * 8; + + // Write top-level struct pointer (points to outer struct) + let outer_offset_and_size = + ((outer_struct_start as i64) << 32) | (outer_struct_size as i64); + data[top_level_bitset_width..top_level_bitset_width + 8] + .copy_from_slice(&outer_offset_and_size.to_le_bytes()); + + // Write outer struct pointer (points to inner struct) + // Offset is relative to outer struct start + let inner_relative_offset = inner_struct_start - outer_struct_start; + let inner_offset_and_size = + ((inner_relative_offset as i64) << 32) | (inner_struct_size as i64); + data[outer_struct_start + outer_bitset_width..outer_struct_start + outer_bitset_width + 8] + .copy_from_slice(&inner_offset_and_size.to_le_bytes()); + + // Fill inner struct with some data + for i in 0..num_inner_fields { + let value_offset = inner_struct_start + inner_bitset_width + i * 8; + let value = (i as i64) * 100; + data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + + NestedRowData { data } + } + + fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { + spark_row.point_to_slice(&self.data); + } +} + +struct DeeplyNestedRowData { + data: Vec, +} + +impl DeeplyNestedRowData { + fn new(num_inner_fields: usize) -> Self { + let row_size = get_deeply_nested_row_size(num_inner_fields); + let mut data = vec![0u8; row_size]; + + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let level1_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let level2_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); + + // Calculate offsets + let level1_struct_start = top_level_bitset_width + 8; + let level1_struct_size = level1_bitset_width + 8; + let level2_struct_start = level1_struct_start + level1_struct_size; + let level2_struct_size = level2_bitset_width + 8; + let inner_struct_start = level2_struct_start + level2_struct_size; + let inner_struct_size = inner_bitset_width + num_inner_fields * 8; + + // Write top-level struct pointer (points to level1 struct) + let level1_offset_and_size = + ((level1_struct_start as i64) << 32) | (level1_struct_size as i64); + data[top_level_bitset_width..top_level_bitset_width + 8] + .copy_from_slice(&level1_offset_and_size.to_le_bytes()); + + // Write level1 struct pointer (points to level2 struct) + let level2_relative_offset = level2_struct_start - level1_struct_start; + let level2_offset_and_size = + ((level2_relative_offset as i64) << 32) | (level2_struct_size as i64); + data[level1_struct_start + level1_bitset_width + ..level1_struct_start + level1_bitset_width + 8] + .copy_from_slice(&level2_offset_and_size.to_le_bytes()); + + // Write level2 struct pointer (points to inner struct) + let inner_relative_offset = inner_struct_start - level2_struct_start; + let inner_offset_and_size = + ((inner_relative_offset as i64) << 32) | (inner_struct_size as i64); + data[level2_struct_start + level2_bitset_width + ..level2_struct_start + level2_bitset_width + 8] + .copy_from_slice(&inner_offset_and_size.to_le_bytes()); + + // Fill inner struct with some data + for i in 0..num_inner_fields { + let value_offset = inner_struct_start + inner_bitset_width + i * 8; + let value = (i as i64) * 100; + data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + + DeeplyNestedRowData { data } + } + + fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { + spark_row.point_to_slice(&self.data); + } +} + +fn benchmark_nested_struct_conversion(c: &mut Criterion) { + let mut group = c.benchmark_group("nested_struct_conversion"); + + // Test nested structs with different inner field counts + for num_fields in [5, 10, 20] { + for num_rows in [1000, 10000] { + let schema = vec![make_nested_struct_schema(num_fields)]; + + // Create row data + let rows: Vec = (0..num_rows) + .map(|_| NestedRowData::new(num_fields)) + .collect(); + + let spark_rows: Vec = rows + .iter() + .map(|row_data| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); + row_data.to_spark_row(&mut spark_row); + spark_row.set_not_null_at(0); + spark_row + }) + .collect(); + + let mut row_addresses: Vec = + spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + + group.bench_with_input( + BenchmarkId::new( + format!("inner_fields_{}", num_fields), + format!("rows_{}", num_rows), + ), + &(num_rows, &schema), + |b, (num_rows, schema)| { + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + *num_rows, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }, + ); + + std::mem::drop(spark_rows); + } + } + + group.finish(); +} + +fn benchmark_deeply_nested_struct_conversion(c: &mut Criterion) { + let mut group = c.benchmark_group("deeply_nested_struct_conversion"); + + // Test deeply nested structs (3 levels) with different inner field counts + for num_fields in [5, 10, 20] { + for num_rows in [1000, 10000] { + let schema = vec![make_deeply_nested_struct_schema(num_fields)]; + + // Create row data + let rows: Vec = (0..num_rows) + .map(|_| DeeplyNestedRowData::new(num_fields)) + .collect(); + + let spark_rows: Vec = rows + .iter() + .map(|row_data| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); + row_data.to_spark_row(&mut spark_row); + spark_row.set_not_null_at(0); + spark_row + }) + .collect(); + + let mut row_addresses: Vec = + spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + + group.bench_with_input( + BenchmarkId::new( + format!("inner_fields_{}", num_fields), + format!("rows_{}", num_rows), + ), + &(num_rows, &schema), + |b, (num_rows, schema)| { + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + *num_rows, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }, + ); + + std::mem::drop(spark_rows); + } + } + + group.finish(); +} + fn config() -> Criterion { Criterion::default() } @@ -163,6 +466,6 @@ fn config() -> Criterion { criterion_group! { name = benches; config = config(); - targets = benchmark_struct_conversion + targets = benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion } criterion_main!(benches); diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 73a126ff67..bdc8096876 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -439,6 +439,227 @@ pub(crate) fn append_field( Ok(()) } +/// Appends nested struct fields to the struct builder using field-major order. +/// This is a helper function for processing nested struct fields recursively. +/// +/// Unlike `append_struct_fields_field_major`, this function takes slices of row addresses, +/// sizes, and null flags directly, without needing to navigate from a parent row. +#[allow(clippy::redundant_closure_call)] +fn append_nested_struct_fields_field_major( + row_addresses: &[jlong], + row_sizes: &[jint], + struct_is_null: &[bool], + struct_builder: &mut StructBuilder, + fields: &arrow::datatypes::Fields, +) -> Result<(), CometError> { + let num_rows = row_addresses.len(); + let mut row = SparkUnsafeRow::new_with_num_fields(fields.len()); + + // Helper macro for processing primitive fields + macro_rules! process_field { + ($builder_type:ty, $field_idx:expr, $get_value:expr) => {{ + let field_builder = struct_builder + .field_builder::<$builder_type>($field_idx) + .unwrap(); + + for row_idx in 0..num_rows { + if struct_is_null[row_idx] { + // Struct is null, field is also null + field_builder.append_null(); + } else { + let row_addr = row_addresses[row_idx]; + let row_size = row_sizes[row_idx]; + row.point_to(row_addr, row_size); + + if row.is_null_at($field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value($get_value(&row, $field_idx)); + } + } + } + }}; + } + + // Process each field across all rows + for (field_idx, field) in fields.iter().enumerate() { + match field.data_type() { + DataType::Boolean => { + process_field!(BooleanBuilder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_boolean(idx)); + } + DataType::Int8 => { + process_field!(Int8Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_byte(idx)); + } + DataType::Int16 => { + process_field!(Int16Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_short(idx)); + } + DataType::Int32 => { + process_field!(Int32Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_int(idx)); + } + DataType::Int64 => { + process_field!(Int64Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_long(idx)); + } + DataType::Float32 => { + process_field!(Float32Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_float(idx)); + } + DataType::Float64 => { + process_field!(Float64Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_double(idx)); + } + DataType::Date32 => { + process_field!(Date32Builder, field_idx, |row: &SparkUnsafeRow, idx| row + .get_date(idx)); + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + process_field!( + TimestampMicrosecondBuilder, + field_idx, + |row: &SparkUnsafeRow, idx| row.get_timestamp(idx) + ); + } + DataType::Binary => { + let field_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + for row_idx in 0..num_rows { + if struct_is_null[row_idx] { + field_builder.append_null(); + } else { + let row_addr = row_addresses[row_idx]; + let row_size = row_sizes[row_idx]; + row.point_to(row_addr, row_size); + + if row.is_null_at(field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value(row.get_binary(field_idx)); + } + } + } + } + DataType::Utf8 => { + let field_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + for row_idx in 0..num_rows { + if struct_is_null[row_idx] { + field_builder.append_null(); + } else { + let row_addr = row_addresses[row_idx]; + let row_size = row_sizes[row_idx]; + row.point_to(row_addr, row_size); + + if row.is_null_at(field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value(row.get_string(field_idx)); + } + } + } + } + DataType::Decimal128(p, _) => { + let p = *p; + let field_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + for row_idx in 0..num_rows { + if struct_is_null[row_idx] { + field_builder.append_null(); + } else { + let row_addr = row_addresses[row_idx]; + let row_size = row_sizes[row_idx]; + row.point_to(row_addr, row_size); + + if row.is_null_at(field_idx) { + field_builder.append_null(); + } else { + field_builder.append_value(row.get_decimal(field_idx, p)); + } + } + } + } + DataType::Struct(nested_fields) => { + let nested_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + // Collect nested struct addresses and sizes in one pass, building validity + let mut nested_addresses: Vec = Vec::with_capacity(num_rows); + let mut nested_sizes: Vec = Vec::with_capacity(num_rows); + let mut nested_is_null: Vec = Vec::with_capacity(num_rows); + + for row_idx in 0..num_rows { + if struct_is_null[row_idx] { + // Parent struct is null, nested struct is also null + nested_builder.append_null(); + nested_is_null.push(true); + nested_addresses.push(0); + nested_sizes.push(0); + } else { + let row_addr = row_addresses[row_idx]; + let row_size = row_sizes[row_idx]; + row.point_to(row_addr, row_size); + + if row.is_null_at(field_idx) { + nested_builder.append_null(); + nested_is_null.push(true); + nested_addresses.push(0); + nested_sizes.push(0); + } else { + nested_builder.append(true); + nested_is_null.push(false); + // Get nested struct address and size + let nested_row = row.get_struct(field_idx, nested_fields.len()); + nested_addresses.push(nested_row.get_row_addr()); + nested_sizes.push(nested_row.get_row_size()); + } + } + } + + // Recursively process nested struct fields in field-major order + append_nested_struct_fields_field_major( + &nested_addresses, + &nested_sizes, + &nested_is_null, + nested_builder, + nested_fields, + )?; + } + // For list and map, fall back to append_field since they have variable-length elements + dt @ (DataType::List(_) | DataType::Map(_, _)) => { + for row_idx in 0..num_rows { + if struct_is_null[row_idx] { + let null_row = SparkUnsafeRow::default(); + append_field(dt, struct_builder, &null_row, field_idx)?; + } else { + let row_addr = row_addresses[row_idx]; + let row_size = row_sizes[row_idx]; + row.point_to(row_addr, row_size); + append_field(dt, struct_builder, &row, field_idx)?; + } + } + } + _ => { + unreachable!( + "Unsupported data type of struct field: {:?}", + field.data_type() + ) + } + } + } + + Ok(()) +} + /// Appends struct fields to the struct builder using field-major order. /// This processes one field at a time across all rows, which moves type dispatch /// outside the row loop (O(fields) dispatches instead of O(rows × fields)). @@ -611,19 +832,69 @@ fn append_struct_fields_field_major( } } } - // For complex types (struct, list, map), fall back to append_field - // since they have their own nested processing logic - dt @ (DataType::Struct(_) | DataType::List(_) | DataType::Map(_, _)) => { + // For nested structs, apply field-major processing recursively + DataType::Struct(nested_fields) => { + let nested_builder = struct_builder + .field_builder::(field_idx) + .unwrap(); + + // Collect nested struct addresses and sizes in one pass, building validity + let mut nested_addresses: Vec = Vec::with_capacity(num_rows); + let mut nested_sizes: Vec = Vec::with_capacity(num_rows); + let mut nested_is_null: Vec = Vec::with_capacity(num_rows); + for (row_idx, i) in (row_start..row_end).enumerate() { - let nested_row = if struct_is_null[row_idx] { - SparkUnsafeRow::default() + if struct_is_null[row_idx] { + // Parent struct is null, nested struct is also null + nested_builder.append_null(); + nested_is_null.push(true); + nested_addresses.push(0); + nested_sizes.push(0); } else { let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); - parent_row.get_struct(column_idx, num_fields) - }; - append_field(dt, struct_builder, &nested_row, field_idx)?; + let parent_struct = parent_row.get_struct(column_idx, num_fields); + + if parent_struct.is_null_at(field_idx) { + nested_builder.append_null(); + nested_is_null.push(true); + nested_addresses.push(0); + nested_sizes.push(0); + } else { + nested_builder.append(true); + nested_is_null.push(false); + // Get nested struct address and size + let nested_row = + parent_struct.get_struct(field_idx, nested_fields.len()); + nested_addresses.push(nested_row.get_row_addr()); + nested_sizes.push(nested_row.get_row_size()); + } + } + } + + // Recursively process nested struct fields in field-major order + append_nested_struct_fields_field_major( + &nested_addresses, + &nested_sizes, + &nested_is_null, + nested_builder, + nested_fields, + )?; + } + // For list and map, fall back to append_field since they have variable-length elements + dt @ (DataType::List(_) | DataType::Map(_, _)) => { + for (row_idx, i) in (row_start..row_end).enumerate() { + if struct_is_null[row_idx] { + let null_row = SparkUnsafeRow::default(); + append_field(dt, struct_builder, &null_row, field_idx)?; + } else { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + parent_row.point_to(row_addr, row_size); + let nested_row = parent_row.get_struct(column_idx, num_fields); + append_field(dt, struct_builder, &nested_row, field_idx)?; + } } } _ => { From 3e8f5b56e42e81ee4f3cc99880aa86fbbd546da6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 10:19:16 -0700 Subject: [PATCH 09/22] perf: batch processing for List and Map columns MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add batched processing for List and Map columns that moves type dispatch outside the row loop, similar to the struct field-major optimization. Changes: - Add `append_list_column_batch` that dispatches on element type once, then processes all rows with the typed builder - Add `append_map_column_batch` that dispatches on key/value types once, with optimized paths for common combinations (Int64/Int64, Int32/Int32, etc.) - Update `append_columns` to use the new batch functions - Add benchmark for List column conversion The optimization: - List columns: Type dispatch goes from O(rows) to O(1) for primitive elements - Map columns: Type dispatch goes from O(rows × 2) to O(2) for primitive key/values - Complex element types fall back to per-row dispatch Co-Authored-By: Claude Opus 4.5 --- native/core/benches/struct_conversion.rs | 129 ++++++++++- native/core/src/execution/shuffle/row.rs | 261 +++++++++++++++++++---- 2 files changed, 351 insertions(+), 39 deletions(-) diff --git a/native/core/benches/struct_conversion.rs b/native/core/benches/struct_conversion.rs index 388b31912a..a89245ea4d 100644 --- a/native/core/benches/struct_conversion.rs +++ b/native/core/benches/struct_conversion.rs @@ -26,6 +26,7 @@ use comet::execution::shuffle::row::{ }; use comet::execution::shuffle::CompressionCodec; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; use tempfile::Builder; const BATCH_SIZE: usize = 5000; @@ -459,6 +460,132 @@ fn benchmark_deeply_nested_struct_conversion(c: &mut Criterion) { group.finish(); } +/// Create a schema with a list column: List +fn make_list_schema(element_type: DataType) -> DataType { + DataType::List(Arc::new(Field::new("item", element_type, true))) +} + +/// Calculate row size for a list with the given number of elements. +/// UnsafeRow layout for list: [null bits] [list pointer (offset, size)] +/// List data: [num_elements (8 bytes)] [null bits] [element data] +fn get_list_row_size(num_elements: usize, element_size: usize) -> usize { + // Top-level row has 1 column (the list) + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let list_pointer_size = 8; + + // List header: num_elements (8 bytes) + null bitset + let list_null_bitset = ((num_elements + 63) / 64) * 8; + let list_header = 8 + list_null_bitset; + let list_data_size = num_elements * element_size; + + top_level_bitset_width + list_pointer_size + list_header + list_data_size +} + +struct ListRowData { + data: Vec, +} + +impl ListRowData { + fn new_int64_list(num_elements: usize) -> Self { + let row_size = get_list_row_size(num_elements, 8); + let mut data = vec![0u8; row_size]; + + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let list_null_bitset = ((num_elements + 63) / 64) * 8; + + // List starts after top-level header + pointer + let list_offset = top_level_bitset_width + 8; + let list_size = 8 + list_null_bitset + num_elements * 8; + + // Write list pointer (offset in upper 32 bits, size in lower 32 bits) + let offset_and_size = ((list_offset as i64) << 32) | (list_size as i64); + data[top_level_bitset_width..top_level_bitset_width + 8] + .copy_from_slice(&offset_and_size.to_le_bytes()); + + // Write number of elements at list start + data[list_offset..list_offset + 8].copy_from_slice(&(num_elements as i64).to_le_bytes()); + + // Fill list with data (after header) + let data_start = list_offset + 8 + list_null_bitset; + for i in 0..num_elements { + let value_offset = data_start + i * 8; + let value = (i as i64) * 100; + data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + + ListRowData { data } + } + + fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { + spark_row.point_to_slice(&self.data); + } +} + +fn benchmark_list_conversion(c: &mut Criterion) { + let mut group = c.benchmark_group("list_conversion"); + + // Test with different list sizes and row counts + for num_elements in [10, 100] { + for num_rows in [1000, 10000] { + let schema = vec![make_list_schema(DataType::Int64)]; + + // Create row data - each row has a list with num_elements items + let rows: Vec = (0..num_rows) + .map(|_| ListRowData::new_int64_list(num_elements)) + .collect(); + + let spark_rows: Vec = rows + .iter() + .map(|row_data| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); + row_data.to_spark_row(&mut spark_row); + spark_row.set_not_null_at(0); + spark_row + }) + .collect(); + + let mut row_addresses: Vec = + spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + + group.bench_with_input( + BenchmarkId::new( + format!("elements_{}", num_elements), + format!("rows_{}", num_rows), + ), + &(num_rows, &schema), + |b, (num_rows, schema)| { + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + *num_rows, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }, + ); + + std::mem::drop(spark_rows); + } + } + + group.finish(); +} + fn config() -> Criterion { Criterion::default() } @@ -466,6 +593,6 @@ fn config() -> Criterion { criterion_group! { name = benches; config = config(); - targets = benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion + targets = benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion, benchmark_list_conversion } criterion_main!(benches); diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index bdc8096876..fbd48120b3 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -660,6 +660,207 @@ fn append_nested_struct_fields_field_major( Ok(()) } +/// Appends a batch of list values to the list builder with a single type dispatch. +/// This moves type dispatch from O(rows) to O(1), significantly improving performance +/// for large batches. +#[allow(clippy::too_many_arguments)] +fn append_list_column_batch( + row_addresses_ptr: *mut jlong, + row_sizes_ptr: *mut jint, + row_start: usize, + row_end: usize, + schema: &[DataType], + column_idx: usize, + element_type: &DataType, + list_builder: &mut ListBuilder>, +) -> Result<(), CometError> { + let mut row = SparkUnsafeRow::new(schema); + + // Helper macro for primitive element types - gets builder fresh each iteration + // to avoid borrow conflicts with list_builder.append() + macro_rules! process_primitive_lists { + ($builder_type:ty, $append_fn:ident) => {{ + for i in row_start..row_end { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + row.point_to(row_addr, row_size); + + if row.is_null_at(column_idx) { + list_builder.append_null(); + } else { + let array = row.get_array(column_idx); + // Get values builder fresh each iteration to avoid borrow conflict + let values_builder = list_builder + .values() + .as_any_mut() + .downcast_mut::<$builder_type>() + .unwrap(); + array.$append_fn::(values_builder); + list_builder.append(true); + } + } + }}; + } + + match element_type { + DataType::Boolean => { + process_primitive_lists!(BooleanBuilder, append_booleans_to_builder); + } + DataType::Int8 => { + process_primitive_lists!(Int8Builder, append_bytes_to_builder); + } + DataType::Int16 => { + process_primitive_lists!(Int16Builder, append_shorts_to_builder); + } + DataType::Int32 => { + process_primitive_lists!(Int32Builder, append_ints_to_builder); + } + DataType::Int64 => { + process_primitive_lists!(Int64Builder, append_longs_to_builder); + } + DataType::Float32 => { + process_primitive_lists!(Float32Builder, append_floats_to_builder); + } + DataType::Float64 => { + process_primitive_lists!(Float64Builder, append_doubles_to_builder); + } + DataType::Date32 => { + process_primitive_lists!(Date32Builder, append_dates_to_builder); + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + process_primitive_lists!(TimestampMicrosecondBuilder, append_timestamps_to_builder); + } + // For complex element types, fall back to per-row dispatch + _ => { + for i in row_start..row_end { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + row.point_to(row_addr, row_size); + + if row.is_null_at(column_idx) { + list_builder.append_null(); + } else { + append_list_element(element_type, list_builder, &row.get_array(column_idx))?; + } + } + } + } + + Ok(()) +} + +/// Appends a batch of map values to the map builder with a single type dispatch. +/// This moves type dispatch from O(rows × 2) to O(2), improving performance for maps. +#[allow(clippy::too_many_arguments)] +fn append_map_column_batch( + row_addresses_ptr: *mut jlong, + row_sizes_ptr: *mut jint, + row_start: usize, + row_end: usize, + schema: &[DataType], + column_idx: usize, + field: &arrow::datatypes::FieldRef, + map_builder: &mut MapBuilder, Box>, +) -> Result<(), CometError> { + let mut row = SparkUnsafeRow::new(schema); + let (key_field, value_field, _) = get_map_key_value_fields(field)?; + let key_type = key_field.data_type(); + let value_type = value_field.data_type(); + + // Helper macro for processing maps with primitive key/value types + // Uses scoped borrows to avoid borrow checker conflicts + macro_rules! process_primitive_maps { + ($key_builder:ty, $key_append:ident, $val_builder:ty, $val_append:ident) => {{ + for i in row_start..row_end { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + row.point_to(row_addr, row_size); + + if row.is_null_at(column_idx) { + map_builder.append(false)?; + } else { + let map = row.get_map(column_idx); + // Process keys in a scope so borrow ends + { + let keys_builder = map_builder + .keys() + .as_any_mut() + .downcast_mut::<$key_builder>() + .unwrap(); + map.keys.$key_append::(keys_builder); + } + // Process values in a scope so borrow ends + { + let values_builder = map_builder + .values() + .as_any_mut() + .downcast_mut::<$val_builder>() + .unwrap(); + map.values.$val_append::(values_builder); + } + map_builder.append(true)?; + } + } + }}; + } + + // Optimize common map type combinations + match (key_type, value_type) { + // Map + (DataType::Int64, DataType::Int64) => { + process_primitive_maps!( + Int64Builder, + append_longs_to_builder, + Int64Builder, + append_longs_to_builder + ); + } + // Map + (DataType::Int64, DataType::Float64) => { + process_primitive_maps!( + Int64Builder, + append_longs_to_builder, + Float64Builder, + append_doubles_to_builder + ); + } + // Map + (DataType::Int32, DataType::Int32) => { + process_primitive_maps!( + Int32Builder, + append_ints_to_builder, + Int32Builder, + append_ints_to_builder + ); + } + // Map + (DataType::Int32, DataType::Int64) => { + process_primitive_maps!( + Int32Builder, + append_ints_to_builder, + Int64Builder, + append_longs_to_builder + ); + } + // For other types, fall back to per-row dispatch + _ => { + for i in row_start..row_end { + let row_addr = unsafe { *row_addresses_ptr.add(i) }; + let row_size = unsafe { *row_sizes_ptr.add(i) }; + row.point_to(row_addr, row_size); + + if row.is_null_at(column_idx) { + map_builder.append(false)?; + } else { + append_map_elements(field, map_builder, &row.get_map(column_idx))?; + } + } + } + } + + Ok(()) +} + /// Appends struct fields to the struct builder using field-major order. /// This processes one field at a time across all rows, which moves type dispatch /// outside the row loop (O(fields) dispatches instead of O(rows × fields)). @@ -1058,47 +1259,31 @@ pub(crate) fn append_columns( MapBuilder, Box>, builder ); - let mut row = SparkUnsafeRow::new(schema); - - for i in row_start..row_end { - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); - - let is_null = row.is_null_at(column_idx); - - if is_null { - // The map is null. - // Append a null value to the map builder. - map_builder.append(false)?; - } else { - append_map_elements(field, map_builder, &row.get_map(column_idx))? - } - } + // Use batched processing for better performance + append_map_column_batch( + row_addresses_ptr, + row_sizes_ptr, + row_start, + row_end, + schema, + column_idx, + field, + map_builder, + )?; } DataType::List(field) => { let list_builder = downcast_builder_ref!(ListBuilder>, builder); - let mut row = SparkUnsafeRow::new(schema); - - for i in row_start..row_end { - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); - - let is_null = row.is_null_at(column_idx); - - if is_null { - // The list is null. - // Append a null value to the list builder. - list_builder.append_null(); - } else { - append_list_element( - field.data_type(), - list_builder, - &row.get_array(column_idx), - )? - } - } + // Use batched processing for better performance + append_list_column_batch( + row_addresses_ptr, + row_sizes_ptr, + row_start, + row_end, + schema, + column_idx, + field.data_type(), + list_builder, + )?; } DataType::Struct(fields) => { let struct_builder = builder From 11a7bca8973977a310f9337b7fae400cbf4839e9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 10:26:54 -0700 Subject: [PATCH 10/22] test: add benchmark for map column processing Adds criterion benchmark for Map conversion to ensure the batched map column processing optimization is covered by benchmarks. Co-Authored-By: Claude Opus 4.5 --- native/core/benches/struct_conversion.rs | 165 ++++++++++++++++++++++- 1 file changed, 164 insertions(+), 1 deletion(-) diff --git a/native/core/benches/struct_conversion.rs b/native/core/benches/struct_conversion.rs index a89245ea4d..6ace15c9e4 100644 --- a/native/core/benches/struct_conversion.rs +++ b/native/core/benches/struct_conversion.rs @@ -586,6 +586,169 @@ fn benchmark_list_conversion(c: &mut Criterion) { group.finish(); } +/// Create a schema with a map column: Map +fn make_map_schema() -> DataType { + // Map is represented as List> in Arrow + let key_field = Field::new("key", DataType::Int64, false); + let value_field = Field::new("value", DataType::Int64, true); + let entries_field = Field::new( + "entries", + DataType::Struct(Fields::from(vec![key_field, value_field])), + false, + ); + DataType::Map(Arc::new(entries_field), false) +} + +/// Calculate row size for a map with the given number of entries. +/// UnsafeRow layout for map: [null bits] [map pointer (offset, size)] +/// Map data: [key_array_size (8 bytes)] [key_array] [value_array] +/// Array format: [num_elements (8 bytes)] [null bits] [element data] +fn get_map_row_size(num_entries: usize) -> usize { + // Top-level row has 1 column (the map) + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let map_pointer_size = 8; + + // Key array: num_elements (8) + null bitset + data + let key_null_bitset = ((num_entries + 63) / 64) * 8; + let key_array_size = 8 + key_null_bitset + num_entries * 8; + + // Value array: num_elements (8) + null bitset + data + let value_null_bitset = ((num_entries + 63) / 64) * 8; + let value_array_size = 8 + value_null_bitset + num_entries * 8; + + // Map header (key array size) + key array + value array + let map_size = 8 + key_array_size + value_array_size; + + top_level_bitset_width + map_pointer_size + map_size +} + +struct MapRowData { + data: Vec, +} + +impl MapRowData { + fn new_int64_map(num_entries: usize) -> Self { + let row_size = get_map_row_size(num_entries); + let mut data = vec![0u8; row_size]; + + let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); + let key_null_bitset = ((num_entries + 63) / 64) * 8; + let value_null_bitset = ((num_entries + 63) / 64) * 8; + + let key_array_size = 8 + key_null_bitset + num_entries * 8; + let value_array_size = 8 + value_null_bitset + num_entries * 8; + let map_size = 8 + key_array_size + value_array_size; + + // Map starts after top-level header + pointer + let map_offset = top_level_bitset_width + 8; + + // Write map pointer (offset in upper 32 bits, size in lower 32 bits) + let offset_and_size = ((map_offset as i64) << 32) | (map_size as i64); + data[top_level_bitset_width..top_level_bitset_width + 8] + .copy_from_slice(&offset_and_size.to_le_bytes()); + + // Write key array size at map start + data[map_offset..map_offset + 8].copy_from_slice(&(key_array_size as i64).to_le_bytes()); + + // Key array starts after map header + let key_array_offset = map_offset + 8; + // Write number of elements + data[key_array_offset..key_array_offset + 8] + .copy_from_slice(&(num_entries as i64).to_le_bytes()); + // Fill key data (after header) + let key_data_start = key_array_offset + 8 + key_null_bitset; + for i in 0..num_entries { + let value_offset = key_data_start + i * 8; + let value = i as i64; + data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + + // Value array starts after key array + let value_array_offset = key_array_offset + key_array_size; + // Write number of elements + data[value_array_offset..value_array_offset + 8] + .copy_from_slice(&(num_entries as i64).to_le_bytes()); + // Fill value data (after header) + let value_data_start = value_array_offset + 8 + value_null_bitset; + for i in 0..num_entries { + let value_offset = value_data_start + i * 8; + let value = (i as i64) * 100; + data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + + MapRowData { data } + } + + fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { + spark_row.point_to_slice(&self.data); + } +} + +fn benchmark_map_conversion(c: &mut Criterion) { + let mut group = c.benchmark_group("map_conversion"); + + // Test with different map sizes and row counts + for num_entries in [10, 100] { + for num_rows in [1000, 10000] { + let schema = vec![make_map_schema()]; + + // Create row data - each row has a map with num_entries items + let rows: Vec = (0..num_rows) + .map(|_| MapRowData::new_int64_map(num_entries)) + .collect(); + + let spark_rows: Vec = rows + .iter() + .map(|row_data| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); + row_data.to_spark_row(&mut spark_row); + spark_row.set_not_null_at(0); + spark_row + }) + .collect(); + + let mut row_addresses: Vec = + spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + + group.bench_with_input( + BenchmarkId::new( + format!("entries_{}", num_entries), + format!("rows_{}", num_rows), + ), + &(num_rows, &schema), + |b, (num_rows, schema)| { + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + *num_rows, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }, + ); + + std::mem::drop(spark_rows); + } + } + + group.finish(); +} + fn config() -> Criterion { Criterion::default() } @@ -593,6 +756,6 @@ fn config() -> Criterion { criterion_group! { name = benches; config = config(); - targets = benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion, benchmark_list_conversion + targets = benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion, benchmark_list_conversion, benchmark_map_conversion } criterion_main!(benches); From 9b91cfd7e15bd1717b4e1a4c2f1f728e475bb46a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 10:34:30 -0700 Subject: [PATCH 11/22] refactor: rename struct_conversion benchmark to complex_type_conversion - Rename benchmark file to better reflect its scope (struct, list, map) - Fix incorrect comment: "native shuffle" -> "JVM shuffle" Co-Authored-By: Claude Opus 4.5 --- native/core/Cargo.toml | 2 +- .../{struct_conversion.rs => complex_type_conversion.rs} | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) rename native/core/benches/{struct_conversion.rs => complex_type_conversion.rs} (99%) diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 0783be1312..3cc1c1521b 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -133,7 +133,7 @@ name = "parquet_decode" harness = false [[bench]] -name = "struct_conversion" +name = "complex_type_conversion" harness = false [[bench]] diff --git a/native/core/benches/struct_conversion.rs b/native/core/benches/complex_type_conversion.rs similarity index 99% rename from native/core/benches/struct_conversion.rs rename to native/core/benches/complex_type_conversion.rs index 6ace15c9e4..ba0413beb8 100644 --- a/native/core/benches/struct_conversion.rs +++ b/native/core/benches/complex_type_conversion.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. -//! Benchmark for struct column processing in native shuffle. +//! Benchmarks for complex type processing in JVM shuffle row-to-columnar conversion. //! //! This benchmark measures the performance of converting Spark UnsafeRow -//! with struct columns to Arrow arrays. +//! with complex type columns (struct, list, map) to Arrow arrays via +//! `process_sorted_row_partition()`. use arrow::datatypes::{DataType, Field, Fields}; use comet::execution::shuffle::row::{ From 05d72a4c47dce55923e3039ae18a0e280a590d74 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 10:41:02 -0700 Subject: [PATCH 12/22] refactor: consolidate and rename shuffle benchmarks - Rename complex_type_conversion.rs to jvm_shuffle.rs - Rename array_conversion.rs to array_element_append.rs - Merge row_columnar.rs primitive benchmark into jvm_shuffle.rs - Delete redundant row_columnar.rs - Update comments to clarify these benchmark JVM shuffle path The jvm_shuffle benchmark now covers: - Primitive types (100 Int64 columns) - Struct (flat, nested, deeply nested) - List - Map The array_element_append benchmark is a micro-benchmark for the inner loop of array element iteration. Co-Authored-By: Claude Opus 4.5 --- native/core/Cargo.toml | 8 +- ..._conversion.rs => array_element_append.rs} | 7 +- ...plex_type_conversion.rs => jvm_shuffle.rs} | 87 +++++++++++++- native/core/benches/row_columnar.rs | 113 ------------------ 4 files changed, 90 insertions(+), 125 deletions(-) rename native/core/benches/{array_conversion.rs => array_element_append.rs} (97%) rename native/core/benches/{complex_type_conversion.rs => jvm_shuffle.rs} (90%) delete mode 100644 native/core/benches/row_columnar.rs diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 3cc1c1521b..4d28127c92 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -120,10 +120,6 @@ harness = false name = "bit_util" harness = false -[[bench]] -name = "row_columnar" -harness = false - [[bench]] name = "shuffle_writer" harness = false @@ -133,9 +129,9 @@ name = "parquet_decode" harness = false [[bench]] -name = "complex_type_conversion" +name = "jvm_shuffle" harness = false [[bench]] -name = "array_conversion" +name = "array_element_append" harness = false diff --git a/native/core/benches/array_conversion.rs b/native/core/benches/array_element_append.rs similarity index 97% rename from native/core/benches/array_conversion.rs rename to native/core/benches/array_element_append.rs index 5007bff1ce..75fc2bbf76 100644 --- a/native/core/benches/array_conversion.rs +++ b/native/core/benches/array_element_append.rs @@ -15,8 +15,11 @@ // specific language governing permissions and limitations // under the License. -//! Benchmarks for SparkUnsafeArray to Arrow array conversion. -//! This specifically tests the append_to_builder function used in shuffle read path. +//! Micro-benchmarks for SparkUnsafeArray element iteration. +//! +//! This tests the low-level `append_to_builder` function which converts +//! SparkUnsafeArray elements to Arrow array builders. This is the inner loop +//! used when processing List/Array columns in JVM shuffle. use arrow::array::builder::{ Date32Builder, Float64Builder, Int32Builder, Int64Builder, TimestampMicrosecondBuilder, diff --git a/native/core/benches/complex_type_conversion.rs b/native/core/benches/jvm_shuffle.rs similarity index 90% rename from native/core/benches/complex_type_conversion.rs rename to native/core/benches/jvm_shuffle.rs index ba0413beb8..f07151c36a 100644 --- a/native/core/benches/complex_type_conversion.rs +++ b/native/core/benches/jvm_shuffle.rs @@ -15,11 +15,17 @@ // specific language governing permissions and limitations // under the License. -//! Benchmarks for complex type processing in JVM shuffle row-to-columnar conversion. +//! Benchmarks for JVM shuffle row-to-columnar conversion. //! //! This benchmark measures the performance of converting Spark UnsafeRow -//! with complex type columns (struct, list, map) to Arrow arrays via -//! `process_sorted_row_partition()`. +//! to Arrow arrays via `process_sorted_row_partition()`, which is called +//! by JVM shuffle (CometColumnarShuffle) when writing shuffle data. +//! +//! Covers: +//! - Primitive types (Int64) +//! - Struct (flat, nested, deeply nested) +//! - List +//! - Map use arrow::datatypes::{DataType, Field, Fields}; use comet::execution::shuffle::row::{ @@ -750,6 +756,79 @@ fn benchmark_map_conversion(c: &mut Criterion) { group.finish(); } +/// Benchmark for primitive type columns (many Int64 columns). +/// This tests the baseline performance without complex type overhead. +fn benchmark_primitive_columns(c: &mut Criterion) { + let mut group = c.benchmark_group("primitive_columns"); + + const NUM_COLS: usize = 100; + let row_size: usize = SparkUnsafeRow::get_row_bitset_width(NUM_COLS) + NUM_COLS * 8; + + for num_rows in [1000, 10000] { + let schema = vec![DataType::Int64; NUM_COLS]; + + // Create row data + let row_data: Vec> = (0..num_rows) + .map(|_| { + let mut data = vec![0u8; row_size]; + // Fill with some data after the bitset + for i in SparkUnsafeRow::get_row_bitset_width(NUM_COLS)..row_size { + data[i] = i as u8; + } + data + }) + .collect(); + + let spark_rows: Vec = row_data + .iter() + .map(|data| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(NUM_COLS); + spark_row.point_to_slice(data); + for i in 0..NUM_COLS { + spark_row.set_not_null_at(i); + } + spark_row + }) + .collect(); + + let mut row_addresses: Vec = + spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + + group.bench_with_input( + BenchmarkId::new("cols_100", format!("rows_{}", num_rows)), + &(num_rows, &schema), + |b, (num_rows, schema)| { + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + *num_rows, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }, + ); + + std::mem::drop(spark_rows); + } + + group.finish(); +} + fn config() -> Criterion { Criterion::default() } @@ -757,6 +836,6 @@ fn config() -> Criterion { criterion_group! { name = benches; config = config(); - targets = benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion, benchmark_list_conversion, benchmark_map_conversion + targets = benchmark_primitive_columns, benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion, benchmark_list_conversion, benchmark_map_conversion } criterion_main!(benches); diff --git a/native/core/benches/row_columnar.rs b/native/core/benches/row_columnar.rs deleted file mode 100644 index a62574111b..0000000000 --- a/native/core/benches/row_columnar.rs +++ /dev/null @@ -1,113 +0,0 @@ -// 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::datatypes::DataType as ArrowDataType; -use comet::execution::shuffle::row::{ - process_sorted_row_partition, SparkUnsafeObject, SparkUnsafeRow, -}; -use comet::execution::shuffle::CompressionCodec; -use criterion::{criterion_group, criterion_main, Criterion}; -use tempfile::Builder; - -const NUM_ROWS: usize = 10000; -const BATCH_SIZE: usize = 5000; -const NUM_COLS: usize = 100; -const ROW_SIZE: usize = SparkUnsafeRow::get_row_bitset_width(NUM_COLS) + NUM_COLS * 8; - -fn benchmark(c: &mut Criterion) { - let mut group = c.benchmark_group("row_array_conversion"); - - group.bench_function("row_to_array", |b| { - let spark_rows = (0..NUM_ROWS) - .map(|_| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(NUM_COLS); - let mut row = Row::new(); - - for i in SparkUnsafeRow::get_row_bitset_width(NUM_COLS)..ROW_SIZE { - row.data[i] = i as u8; - } - - row.to_spark_row(&mut spark_row); - - for i in 0..NUM_COLS { - spark_row.set_not_null_at(i); - } - - spark_row - }) - .collect::>(); - - let mut row_addresses = spark_rows - .iter() - .map(|row| row.get_row_addr()) - .collect::>(); - let mut row_sizes = spark_rows - .iter() - .map(|row| row.get_row_size()) - .collect::>(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - let schema = vec![ArrowDataType::Int64; NUM_COLS]; - - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - NUM_ROWS, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - &schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }); -} - -struct Row { - data: Box<[u8; ROW_SIZE]>, -} - -impl Row { - pub fn new() -> Self { - Row { - data: Box::new([0u8; ROW_SIZE]), - } - } - - pub fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { - spark_row.point_to_slice(self.data.as_ref()); - } -} - -fn config() -> Criterion { - Criterion::default() -} - -criterion_group! { - name = benches; - config = config(); - targets = benchmark -} -criterion_main!(benches); From 0deac663e3fca8d5d4524d05960a0b8d27f8d1d7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 10:46:48 -0700 Subject: [PATCH 13/22] lint --- native/core/benches/jvm_shuffle.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/native/core/benches/jvm_shuffle.rs b/native/core/benches/jvm_shuffle.rs index f07151c36a..c8c13e9a52 100644 --- a/native/core/benches/jvm_shuffle.rs +++ b/native/core/benches/jvm_shuffle.rs @@ -791,8 +791,7 @@ fn benchmark_primitive_columns(c: &mut Criterion) { }) .collect(); - let mut row_addresses: Vec = - spark_rows.iter().map(|row| row.get_row_addr()).collect(); + let mut row_addresses: Vec = spark_rows.iter().map(|row| row.get_row_addr()).collect(); let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); let row_address_ptr = row_addresses.as_mut_ptr(); From a6123c016a284705904603fefa97fd89b8892400 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 11:20:22 -0700 Subject: [PATCH 14/22] refactor: add safety comments and reduce boilerplate in shuffle code - Add macro `impl_append_to_builder` to generate bulk append methods, reducing ~190 lines of duplicated unsafe code in list.rs - Add comprehensive safety documentation to SparkUnsafeObject trait explaining memory layout invariants and JVM ownership guarantees - Add safety documentation to append_columns function - Add #[inline] annotations to trait accessor methods for better optimization - Keep unsafe pointer iteration for performance (benchmarks show 7-14% regression with safe accessor approach for some types) Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/list.rs | 266 ++++++---------------- native/core/src/execution/shuffle/row.rs | 34 +++ 2 files changed, 105 insertions(+), 195 deletions(-) diff --git a/native/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs index 6236bdf5c1..ed6e253115 100644 --- a/native/core/src/execution/shuffle/list.rs +++ b/native/core/src/execution/shuffle/list.rs @@ -32,6 +32,52 @@ use arrow::array::{ }; use arrow::datatypes::{DataType, TimeUnit}; +/// Generates bulk append methods for primitive types in SparkUnsafeArray. +/// +/// # Safety invariants for all generated methods: +/// - `element_offset` points to contiguous element data of length `num_elements` +/// - `null_bitset_ptr()` returns a pointer to `ceil(num_elements/64)` i64 words +/// - These invariants are guaranteed by the SparkUnsafeArray layout from the JVM +macro_rules! impl_append_to_builder { + ($method_name:ident, $builder_type:ty, $element_type:ty) => { + pub(crate) fn $method_name(&self, builder: &mut $builder_type) { + let num_elements = self.num_elements; + if num_elements == 0 { + return; + } + + if NULLABLE { + let mut ptr = self.element_offset as *const $element_type; + let null_words = self.null_bitset_ptr(); + for idx in 0..num_elements { + let word_idx = idx >> 6; + let bit_idx = idx & 0x3f; + // SAFETY: word_idx < ceil(num_elements/64) since idx < num_elements + let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; + + if is_null { + builder.append_null(); + } else { + // SAFETY: ptr is within element data bounds + builder.append_value(unsafe { *ptr }); + } + // SAFETY: ptr stays within bounds, iterating num_elements times + ptr = unsafe { ptr.add(1) }; + } + } else { + // SAFETY: element_offset points to contiguous data of length num_elements + let slice = unsafe { + std::slice::from_raw_parts( + self.element_offset as *const $element_type, + num_elements, + ) + }; + builder.append_slice(slice); + } + } + }; +} + pub struct SparkUnsafeArray { row_addr: i64, num_elements: usize, @@ -39,10 +85,12 @@ pub struct SparkUnsafeArray { } impl SparkUnsafeObject for SparkUnsafeArray { + #[inline] fn get_row_addr(&self) -> i64 { self.row_addr } + #[inline] fn get_element_offset(&self, index: usize, element_size: usize) -> *const u8 { (self.element_offset + (index * element_size) as i64) as *const u8 } @@ -93,203 +141,19 @@ impl SparkUnsafeArray { /// Returns the null bitset pointer (starts at row_addr + 8). #[inline] - pub(crate) fn null_bitset_ptr(&self) -> *const i64 { + fn null_bitset_ptr(&self) -> *const i64 { (self.row_addr + 8) as *const i64 } - /// Bulk append i32 values to builder. - /// For non-nullable: uses slice append for optimal performance. - /// For nullable: uses pointer iteration with efficient null bitset reading. - pub(crate) fn append_ints_to_builder(&self, builder: &mut Int32Builder) { - let num_elements = self.num_elements; - if num_elements == 0 { - return; - } - - if NULLABLE { - let mut ptr = self.element_offset as *const i32; - let null_words = self.null_bitset_ptr(); - for idx in 0..num_elements { - let word_idx = idx >> 6; - let bit_idx = idx & 0x3f; - let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; - - if is_null { - builder.append_null(); - } else { - builder.append_value(unsafe { *ptr }); - } - ptr = unsafe { ptr.add(1) }; - } - } else { - // Use slice-based append for non-nullable path (much faster) - let slice = unsafe { - std::slice::from_raw_parts(self.element_offset as *const i32, num_elements) - }; - builder.append_slice(slice); - } - } - - /// Bulk append i64 values to builder. - pub(crate) fn append_longs_to_builder(&self, builder: &mut Int64Builder) { - let num_elements = self.num_elements; - if num_elements == 0 { - return; - } - - if NULLABLE { - let mut ptr = self.element_offset as *const i64; - let null_words = self.null_bitset_ptr(); - for idx in 0..num_elements { - let word_idx = idx >> 6; - let bit_idx = idx & 0x3f; - let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; - - if is_null { - builder.append_null(); - } else { - builder.append_value(unsafe { *ptr }); - } - ptr = unsafe { ptr.add(1) }; - } - } else { - let slice = unsafe { - std::slice::from_raw_parts(self.element_offset as *const i64, num_elements) - }; - builder.append_slice(slice); - } - } - - /// Bulk append i16 values to builder. - pub(crate) fn append_shorts_to_builder( - &self, - builder: &mut Int16Builder, - ) { - let num_elements = self.num_elements; - if num_elements == 0 { - return; - } - - if NULLABLE { - let mut ptr = self.element_offset as *const i16; - let null_words = self.null_bitset_ptr(); - for idx in 0..num_elements { - let word_idx = idx >> 6; - let bit_idx = idx & 0x3f; - let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; - - if is_null { - builder.append_null(); - } else { - builder.append_value(unsafe { *ptr }); - } - ptr = unsafe { ptr.add(1) }; - } - } else { - let slice = unsafe { - std::slice::from_raw_parts(self.element_offset as *const i16, num_elements) - }; - builder.append_slice(slice); - } - } - - /// Bulk append i8 values to builder. - pub(crate) fn append_bytes_to_builder(&self, builder: &mut Int8Builder) { - let num_elements = self.num_elements; - if num_elements == 0 { - return; - } - - if NULLABLE { - let mut ptr = self.element_offset as *const i8; - let null_words = self.null_bitset_ptr(); - for idx in 0..num_elements { - let word_idx = idx >> 6; - let bit_idx = idx & 0x3f; - let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; - - if is_null { - builder.append_null(); - } else { - builder.append_value(unsafe { *ptr }); - } - ptr = unsafe { ptr.add(1) }; - } - } else { - let slice = unsafe { - std::slice::from_raw_parts(self.element_offset as *const i8, num_elements) - }; - builder.append_slice(slice); - } - } - - /// Bulk append f32 values to builder. - pub(crate) fn append_floats_to_builder( - &self, - builder: &mut Float32Builder, - ) { - let num_elements = self.num_elements; - if num_elements == 0 { - return; - } - - if NULLABLE { - let mut ptr = self.element_offset as *const f32; - let null_words = self.null_bitset_ptr(); - for idx in 0..num_elements { - let word_idx = idx >> 6; - let bit_idx = idx & 0x3f; - let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; - - if is_null { - builder.append_null(); - } else { - builder.append_value(unsafe { *ptr }); - } - ptr = unsafe { ptr.add(1) }; - } - } else { - let slice = unsafe { - std::slice::from_raw_parts(self.element_offset as *const f32, num_elements) - }; - builder.append_slice(slice); - } - } - - /// Bulk append f64 values to builder. - pub(crate) fn append_doubles_to_builder( - &self, - builder: &mut Float64Builder, - ) { - let num_elements = self.num_elements; - if num_elements == 0 { - return; - } - - if NULLABLE { - let mut ptr = self.element_offset as *const f64; - let null_words = self.null_bitset_ptr(); - for idx in 0..num_elements { - let word_idx = idx >> 6; - let bit_idx = idx & 0x3f; - let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; - - if is_null { - builder.append_null(); - } else { - builder.append_value(unsafe { *ptr }); - } - ptr = unsafe { ptr.add(1) }; - } - } else { - let slice = unsafe { - std::slice::from_raw_parts(self.element_offset as *const f64, num_elements) - }; - builder.append_slice(slice); - } - } + impl_append_to_builder!(append_ints_to_builder, Int32Builder, i32); + impl_append_to_builder!(append_longs_to_builder, Int64Builder, i64); + impl_append_to_builder!(append_shorts_to_builder, Int16Builder, i16); + impl_append_to_builder!(append_bytes_to_builder, Int8Builder, i8); + impl_append_to_builder!(append_floats_to_builder, Float32Builder, f32); + impl_append_to_builder!(append_doubles_to_builder, Float64Builder, f64); - /// Bulk append boolean values to builder using pointer iteration. + /// Bulk append boolean values to builder. + /// Booleans are stored as 1 byte each in SparkUnsafeArray, requiring special handling. pub(crate) fn append_booleans_to_builder( &self, builder: &mut BooleanBuilder, @@ -306,24 +170,28 @@ impl SparkUnsafeArray { for idx in 0..num_elements { let word_idx = idx >> 6; let bit_idx = idx & 0x3f; + // SAFETY: word_idx < ceil(num_elements/64) since idx < num_elements let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; if is_null { builder.append_null(); } else { + // SAFETY: ptr is within element data bounds builder.append_value(unsafe { *ptr != 0 }); } + // SAFETY: ptr stays within bounds, iterating num_elements times ptr = unsafe { ptr.add(1) }; } } else { for _ in 0..num_elements { + // SAFETY: ptr is within element data bounds builder.append_value(unsafe { *ptr != 0 }); ptr = unsafe { ptr.add(1) }; } } } - /// Bulk append timestamp values to builder. + /// Bulk append timestamp values to builder (stored as i64 microseconds). pub(crate) fn append_timestamps_to_builder( &self, builder: &mut TimestampMicrosecondBuilder, @@ -339,16 +207,20 @@ impl SparkUnsafeArray { for idx in 0..num_elements { let word_idx = idx >> 6; let bit_idx = idx & 0x3f; + // SAFETY: word_idx < ceil(num_elements/64) since idx < num_elements let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; if is_null { builder.append_null(); } else { + // SAFETY: ptr is within element data bounds builder.append_value(unsafe { *ptr }); } + // SAFETY: ptr stays within bounds, iterating num_elements times ptr = unsafe { ptr.add(1) }; } } else { + // SAFETY: element_offset points to contiguous i64 data of length num_elements let slice = unsafe { std::slice::from_raw_parts(self.element_offset as *const i64, num_elements) }; @@ -356,7 +228,7 @@ impl SparkUnsafeArray { } } - /// Bulk append date values to builder. + /// Bulk append date values to builder (stored as i32 days since epoch). pub(crate) fn append_dates_to_builder( &self, builder: &mut Date32Builder, @@ -372,16 +244,20 @@ impl SparkUnsafeArray { for idx in 0..num_elements { let word_idx = idx >> 6; let bit_idx = idx & 0x3f; + // SAFETY: word_idx < ceil(num_elements/64) since idx < num_elements let is_null = unsafe { (*null_words.add(word_idx) & (1i64 << bit_idx)) != 0 }; if is_null { builder.append_null(); } else { + // SAFETY: ptr is within element data bounds builder.append_value(unsafe { *ptr }); } + // SAFETY: ptr stays within bounds, iterating num_elements times ptr = unsafe { ptr.add(1) }; } } else { + // SAFETY: element_offset points to contiguous i32 data of length num_elements let slice = unsafe { std::slice::from_raw_parts(self.element_offset as *const i32, num_elements) }; diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index fbd48120b3..04e6960901 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -56,6 +56,19 @@ const NESTED_TYPE_BUILDER_CAPACITY: usize = 100; /// A common trait for Spark Unsafe classes that can be used to access the underlying data, /// e.g., `UnsafeRow` and `UnsafeArray`. This defines a set of methods that can be used to /// access the underlying data with index. +/// +/// # Safety +/// +/// Implementations must ensure that: +/// - `get_row_addr()` returns a valid pointer to JVM-allocated memory +/// - `get_element_offset()` returns a valid pointer within the row/array data region +/// - The memory layout follows Spark's UnsafeRow/UnsafeArray format +/// - The memory remains valid for the lifetime of the object (guaranteed by JVM ownership) +/// +/// All accessor methods (get_boolean, get_int, etc.) use unsafe pointer operations but are +/// safe to call as long as: +/// - The index is within bounds (caller's responsibility) +/// - The object was constructed from valid Spark UnsafeRow/UnsafeArray data pub trait SparkUnsafeObject { /// Returns the address of the row. fn get_row_addr(&self) -> i64; @@ -73,12 +86,14 @@ pub trait SparkUnsafeObject { } /// Returns boolean value at the given index of the object. + #[inline] fn get_boolean(&self, index: usize) -> bool { let addr = self.get_element_offset(index, 1); unsafe { *addr != 0 } } /// Returns byte value at the given index of the object. + #[inline] fn get_byte(&self, index: usize) -> i8 { let addr = self.get_element_offset(index, 1); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 1) }; @@ -86,6 +101,7 @@ pub trait SparkUnsafeObject { } /// Returns short value at the given index of the object. + #[inline] fn get_short(&self, index: usize) -> i16 { let addr = self.get_element_offset(index, 2); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 2) }; @@ -93,6 +109,7 @@ pub trait SparkUnsafeObject { } /// Returns integer value at the given index of the object. + #[inline] fn get_int(&self, index: usize) -> i32 { let addr = self.get_element_offset(index, 4); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 4) }; @@ -100,6 +117,7 @@ pub trait SparkUnsafeObject { } /// Returns long value at the given index of the object. + #[inline] fn get_long(&self, index: usize) -> i64 { let addr = self.get_element_offset(index, 8); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 8) }; @@ -107,6 +125,7 @@ pub trait SparkUnsafeObject { } /// Returns float value at the given index of the object. + #[inline] fn get_float(&self, index: usize) -> f32 { let addr = self.get_element_offset(index, 4); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 4) }; @@ -114,6 +133,7 @@ pub trait SparkUnsafeObject { } /// Returns double value at the given index of the object. + #[inline] fn get_double(&self, index: usize) -> f64 { let addr = self.get_element_offset(index, 8); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 8) }; @@ -137,6 +157,7 @@ pub trait SparkUnsafeObject { } /// Returns date value at the given index of the object. + #[inline] fn get_date(&self, index: usize) -> i32 { let addr = self.get_element_offset(index, 4); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 4) }; @@ -144,6 +165,7 @@ pub trait SparkUnsafeObject { } /// Returns timestamp value at the given index of the object. + #[inline] fn get_timestamp(&self, index: usize) -> i64 { let addr = self.get_element_offset(index, 8); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr, 8) }; @@ -1111,6 +1133,16 @@ fn append_struct_fields_field_major( } /// Appends column of top rows to the given array builder. +/// +/// # Safety +/// +/// The caller must ensure: +/// - `row_addresses_ptr` points to an array of at least `row_end` jlong values +/// - `row_sizes_ptr` points to an array of at least `row_end` jint values +/// - Each address in `row_addresses_ptr[row_start..row_end]` points to valid Spark UnsafeRow data +/// - The memory remains valid for the duration of this function call +/// +/// These invariants are guaranteed when called from JNI with arrays provided by the JVM. #[allow(clippy::redundant_closure_call, clippy::too_many_arguments)] pub(crate) fn append_columns( row_addresses_ptr: *mut jlong, @@ -1132,6 +1164,8 @@ pub(crate) fn append_columns( let mut row = SparkUnsafeRow::new(schema); for i in row_start..row_end { + // SAFETY: row_addresses_ptr and row_sizes_ptr are valid for indices [row_start, row_end) + // as guaranteed by the caller (JVM provides these arrays with correct bounds) let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; row.point_to(row_addr, row_size); From 245c64742abee8ae44117e3f7b71bf428ab999ba Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 11:27:48 -0700 Subject: [PATCH 15/22] refactor: improve error handling for builder downcasts Replace .unwrap() and .expect() calls on builder downcasts with proper error handling that returns CometError::Internal with descriptive messages including: - The expected type - The actual type (via type_id for downcast_builder_ref) - The field index (for get_field_builder) Added two macros: - downcast_builder_ref!: returns Result with type mismatch details - get_field_builder!: returns Result with field index and expected type Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/row.rs | 90 +++++++++++++----------- 1 file changed, 50 insertions(+), 40 deletions(-) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 04e6960901..6d9c6f85f5 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -297,11 +297,32 @@ impl SparkUnsafeRow { } macro_rules! downcast_builder_ref { - ($builder_type:ty, $builder:expr) => { + ($builder_type:ty, $builder:expr) => {{ + let actual_type_id = $builder.as_any().type_id(); $builder .as_any_mut() .downcast_mut::<$builder_type>() - .expect(stringify!($builder_type)) + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast builder: expected {}, got {:?}", + stringify!($builder_type), + actual_type_id + )) + })? + }}; +} + +macro_rules! get_field_builder { + ($struct_builder:expr, $builder_type:ty, $idx:expr) => { + $struct_builder + .field_builder::<$builder_type>($idx) + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to get field builder at index {}: expected {}", + $idx, + stringify!($builder_type) + )) + })? }; } @@ -324,7 +345,7 @@ pub(crate) fn append_field( /// A macro for generating code of appending value into field builder of Arrow struct builder. macro_rules! append_field_to_builder { ($builder_type:ty, $accessor:expr) => {{ - let field_builder = struct_builder.field_builder::<$builder_type>(idx).unwrap(); + let field_builder = get_field_builder!(struct_builder, $builder_type, idx); if row.is_null_row() { // The row is null. @@ -397,7 +418,7 @@ pub(crate) fn append_field( } DataType::Struct(fields) => { // Appending value into struct field builder of Arrow struct builder. - let field_builder = struct_builder.field_builder::(idx).unwrap(); + let field_builder = get_field_builder!(struct_builder, StructBuilder, idx); let nested_row = if row.is_null_row() || row.is_null_at(idx) { // The row is null, or the field in the row is null, i.e., a null nested row. @@ -414,9 +435,11 @@ pub(crate) fn append_field( } } DataType::Map(field, _) => { - let field_builder = struct_builder - .field_builder::, Box>>(idx) - .unwrap(); + let field_builder = get_field_builder!( + struct_builder, + MapBuilder, Box>, + idx + ); if row.is_null_row() { // The row is null. @@ -434,9 +457,8 @@ pub(crate) fn append_field( } } DataType::List(field) => { - let field_builder = struct_builder - .field_builder::>>(idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, ListBuilder>, idx); if row.is_null_row() { // The row is null. @@ -480,9 +502,7 @@ fn append_nested_struct_fields_field_major( // Helper macro for processing primitive fields macro_rules! process_field { ($builder_type:ty, $field_idx:expr, $get_value:expr) => {{ - let field_builder = struct_builder - .field_builder::<$builder_type>($field_idx) - .unwrap(); + let field_builder = get_field_builder!(struct_builder, $builder_type, $field_idx); for row_idx in 0..num_rows { if struct_is_null[row_idx] { @@ -546,9 +566,8 @@ fn append_nested_struct_fields_field_major( ); } DataType::Binary => { - let field_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, BinaryBuilder, field_idx); for row_idx in 0..num_rows { if struct_is_null[row_idx] { @@ -567,9 +586,8 @@ fn append_nested_struct_fields_field_major( } } DataType::Utf8 => { - let field_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, StringBuilder, field_idx); for row_idx in 0..num_rows { if struct_is_null[row_idx] { @@ -589,9 +607,8 @@ fn append_nested_struct_fields_field_major( } DataType::Decimal128(p, _) => { let p = *p; - let field_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, Decimal128Builder, field_idx); for row_idx in 0..num_rows { if struct_is_null[row_idx] { @@ -610,9 +627,8 @@ fn append_nested_struct_fields_field_major( } } DataType::Struct(nested_fields) => { - let nested_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let nested_builder = + get_field_builder!(struct_builder, StructBuilder, field_idx); // Collect nested struct addresses and sizes in one pass, building validity let mut nested_addresses: Vec = Vec::with_capacity(num_rows); @@ -922,9 +938,7 @@ fn append_struct_fields_field_major( // Helper macro for processing primitive fields macro_rules! process_field { ($builder_type:ty, $field_idx:expr, $get_value:expr) => {{ - let field_builder = struct_builder - .field_builder::<$builder_type>($field_idx) - .unwrap(); + let field_builder = get_field_builder!(struct_builder, $builder_type, $field_idx); for (row_idx, i) in (row_start..row_end).enumerate() { if struct_is_null[row_idx] { @@ -989,9 +1003,8 @@ fn append_struct_fields_field_major( ); } DataType::Binary => { - let field_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, BinaryBuilder, field_idx); for (row_idx, i) in (row_start..row_end).enumerate() { if struct_is_null[row_idx] { @@ -1011,9 +1024,8 @@ fn append_struct_fields_field_major( } } DataType::Utf8 => { - let field_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, StringBuilder, field_idx); for (row_idx, i) in (row_start..row_end).enumerate() { if struct_is_null[row_idx] { @@ -1034,9 +1046,8 @@ fn append_struct_fields_field_major( } DataType::Decimal128(p, _) => { let p = *p; - let field_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let field_builder = + get_field_builder!(struct_builder, Decimal128Builder, field_idx); for (row_idx, i) in (row_start..row_end).enumerate() { if struct_is_null[row_idx] { @@ -1057,9 +1068,8 @@ fn append_struct_fields_field_major( } // For nested structs, apply field-major processing recursively DataType::Struct(nested_fields) => { - let nested_builder = struct_builder - .field_builder::(field_idx) - .unwrap(); + let nested_builder = + get_field_builder!(struct_builder, StructBuilder, field_idx); // Collect nested struct addresses and sizes in one pass, building validity let mut nested_addresses: Vec = Vec::with_capacity(num_rows); From ac8c18202a286508086b40d979b33394e67cb062 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 11:54:50 -0700 Subject: [PATCH 16/22] lint --- native/core/src/execution/shuffle/row.rs | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 6d9c6f85f5..604194b623 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -566,8 +566,7 @@ fn append_nested_struct_fields_field_major( ); } DataType::Binary => { - let field_builder = - get_field_builder!(struct_builder, BinaryBuilder, field_idx); + let field_builder = get_field_builder!(struct_builder, BinaryBuilder, field_idx); for row_idx in 0..num_rows { if struct_is_null[row_idx] { @@ -586,8 +585,7 @@ fn append_nested_struct_fields_field_major( } } DataType::Utf8 => { - let field_builder = - get_field_builder!(struct_builder, StringBuilder, field_idx); + let field_builder = get_field_builder!(struct_builder, StringBuilder, field_idx); for row_idx in 0..num_rows { if struct_is_null[row_idx] { @@ -627,8 +625,7 @@ fn append_nested_struct_fields_field_major( } } DataType::Struct(nested_fields) => { - let nested_builder = - get_field_builder!(struct_builder, StructBuilder, field_idx); + let nested_builder = get_field_builder!(struct_builder, StructBuilder, field_idx); // Collect nested struct addresses and sizes in one pass, building validity let mut nested_addresses: Vec = Vec::with_capacity(num_rows); @@ -1003,8 +1000,7 @@ fn append_struct_fields_field_major( ); } DataType::Binary => { - let field_builder = - get_field_builder!(struct_builder, BinaryBuilder, field_idx); + let field_builder = get_field_builder!(struct_builder, BinaryBuilder, field_idx); for (row_idx, i) in (row_start..row_end).enumerate() { if struct_is_null[row_idx] { @@ -1024,8 +1020,7 @@ fn append_struct_fields_field_major( } } DataType::Utf8 => { - let field_builder = - get_field_builder!(struct_builder, StringBuilder, field_idx); + let field_builder = get_field_builder!(struct_builder, StringBuilder, field_idx); for (row_idx, i) in (row_start..row_end).enumerate() { if struct_is_null[row_idx] { @@ -1068,8 +1063,7 @@ fn append_struct_fields_field_major( } // For nested structs, apply field-major processing recursively DataType::Struct(nested_fields) => { - let nested_builder = - get_field_builder!(struct_builder, StructBuilder, field_idx); + let nested_builder = get_field_builder!(struct_builder, StructBuilder, field_idx); // Collect nested struct addresses and sizes in one pass, building validity let mut nested_addresses: Vec = Vec::with_capacity(num_rows); From 83d1d3056911e88aab53f353670b150019871d55 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 12:01:01 -0700 Subject: [PATCH 17/22] refactor: add safety comments to remaining unsafe blocks Add SAFETY comments to: - SparkUnsafeRow::is_null_at and set_not_null_at - SparkUnsafeArray::new and is_null_at - Batch processing functions (append_list_column_batch, append_map_column_batch, append_struct_fields_field_major) Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/list.rs | 6 ++++- native/core/src/execution/shuffle/row.rs | 29 +++++++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs index ed6e253115..cb21cc3497 100644 --- a/native/core/src/execution/shuffle/list.rs +++ b/native/core/src/execution/shuffle/list.rs @@ -99,7 +99,8 @@ impl SparkUnsafeObject for SparkUnsafeArray { impl SparkUnsafeArray { /// Creates a `SparkUnsafeArray` which points to the given address and size in bytes. pub fn new(addr: i64) -> Self { - // Read the number of elements from the first 8 bytes. + // SAFETY: addr points to valid Spark UnsafeArray data from the JVM. + // The first 8 bytes contain the element count as a little-endian i64. let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr as *const u8, 8) }; let num_elements = i64::from_le_bytes(slice.try_into().unwrap()); @@ -131,6 +132,9 @@ impl SparkUnsafeArray { /// Returns true if the null bit at the given index of the array is set. #[inline] pub(crate) fn is_null_at(&self, index: usize) -> bool { + // SAFETY: row_addr points to valid Spark UnsafeArray data. The null bitset starts + // at offset 8 and contains ceil(num_elements/64) * 8 bytes. The caller ensures + // index < num_elements, so word_offset is within the bitset region. unsafe { let mask: i64 = 1i64 << (index & 0x3f); let word_offset = (self.row_addr + 8 + (((index >> 6) as i64) << 3)) as *const i64; diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 604194b623..b8bab8ad86 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -277,6 +277,9 @@ impl SparkUnsafeRow { /// Returns true if the null bit at the given index of the row is set. #[inline] pub(crate) fn is_null_at(&self, index: usize) -> bool { + // SAFETY: row_addr points to valid Spark UnsafeRow data with at least + // ceil(num_fields/64) * 8 bytes of null bitset. The caller ensures index < num_fields. + // word_offset is within the bitset region since (index >> 6) << 3 < bitset size. unsafe { let mask: i64 = 1i64 << (index & 0x3f); let word_offset = (self.row_addr + (((index >> 6) as i64) << 3)) as *const i64; @@ -287,6 +290,10 @@ impl SparkUnsafeRow { /// Unsets the null bit at the given index of the row, i.e., set the bit to 0 (not null). pub fn set_not_null_at(&mut self, index: usize) { + // SAFETY: row_addr points to valid Spark UnsafeRow data with at least + // ceil(num_fields/64) * 8 bytes of null bitset. The caller ensures index < num_fields. + // word_offset is within the bitset region since (index >> 6) << 3 < bitset size. + // Writing is safe because we have mutable access and the memory is owned by the JVM. unsafe { let mask: i64 = 1i64 << (index & 0x3f); let word_offset = (self.row_addr + (((index >> 6) as i64) << 3)) as *mut i64; @@ -716,6 +723,8 @@ fn append_list_column_batch( macro_rules! process_primitive_lists { ($builder_type:ty, $append_fn:ident) => {{ for i in row_start..row_end { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; row.point_to(row_addr, row_size); @@ -768,6 +777,8 @@ fn append_list_column_batch( // For complex element types, fall back to per-row dispatch _ => { for i in row_start..row_end { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; row.point_to(row_addr, row_size); @@ -807,6 +818,8 @@ fn append_map_column_batch( macro_rules! process_primitive_maps { ($key_builder:ty, $key_append:ident, $val_builder:ty, $val_append:ident) => {{ for i in row_start..row_end { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; row.point_to(row_addr, row_size); @@ -880,6 +893,8 @@ fn append_map_column_batch( // For other types, fall back to per-row dispatch _ => { for i in row_start..row_end { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; row.point_to(row_addr, row_size); @@ -918,6 +933,8 @@ fn append_struct_fields_field_major( let mut struct_is_null = Vec::with_capacity(num_rows); for i in row_start..row_end { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); @@ -942,6 +959,8 @@ fn append_struct_fields_field_major( // Struct is null, field is also null field_builder.append_null(); } else { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); @@ -1006,6 +1025,8 @@ fn append_struct_fields_field_major( if struct_is_null[row_idx] { field_builder.append_null(); } else { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); @@ -1026,6 +1047,8 @@ fn append_struct_fields_field_major( if struct_is_null[row_idx] { field_builder.append_null(); } else { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); @@ -1048,6 +1071,8 @@ fn append_struct_fields_field_major( if struct_is_null[row_idx] { field_builder.append_null(); } else { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); @@ -1078,6 +1103,8 @@ fn append_struct_fields_field_major( nested_addresses.push(0); nested_sizes.push(0); } else { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); @@ -1116,6 +1143,8 @@ fn append_struct_fields_field_major( let null_row = SparkUnsafeRow::default(); append_field(dt, struct_builder, &null_row, field_idx)?; } else { + // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr + // are valid for indices [row_start, row_end) as provided by the JVM let row_addr = unsafe { *row_addresses_ptr.add(i) }; let row_size = unsafe { *row_sizes_ptr.add(i) }; parent_row.point_to(row_addr, row_size); From 11fe509b8ca0befb222fc626ef040e59d2a31510 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 12:06:09 -0700 Subject: [PATCH 18/22] refactor: introduce read_row_at! macro to reduce boilerplate Replace repeated unsafe pointer dereference patterns with a macro that encapsulates the safety invariants. This reduces code duplication and centralizes the safety documentation. Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/row.rs | 86 +++++++----------------- 1 file changed, 26 insertions(+), 60 deletions(-) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index b8bab8ad86..bc6da10228 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -702,6 +702,20 @@ fn append_nested_struct_fields_field_major( Ok(()) } +/// Reads row address and size from JVM-provided pointer arrays and points the row to that data. +/// +/// # Safety +/// Caller must ensure row_addresses_ptr and row_sizes_ptr are valid for index i. +/// This is guaranteed when called from append_columns with indices in [row_start, row_end). +macro_rules! read_row_at { + ($row:expr, $row_addresses_ptr:expr, $row_sizes_ptr:expr, $i:expr) => {{ + // SAFETY: Caller guarantees pointers are valid for this index (see macro doc) + let row_addr = unsafe { *$row_addresses_ptr.add($i) }; + let row_size = unsafe { *$row_sizes_ptr.add($i) }; + $row.point_to(row_addr, row_size); + }}; +} + /// Appends a batch of list values to the list builder with a single type dispatch. /// This moves type dispatch from O(rows) to O(1), significantly improving performance /// for large batches. @@ -723,11 +737,7 @@ fn append_list_column_batch( macro_rules! process_primitive_lists { ($builder_type:ty, $append_fn:ident) => {{ for i in row_start..row_end { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); + read_row_at!(row, row_addresses_ptr, row_sizes_ptr, i); if row.is_null_at(column_idx) { list_builder.append_null(); @@ -777,11 +787,7 @@ fn append_list_column_batch( // For complex element types, fall back to per-row dispatch _ => { for i in row_start..row_end { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); + read_row_at!(row, row_addresses_ptr, row_sizes_ptr, i); if row.is_null_at(column_idx) { list_builder.append_null(); @@ -818,11 +824,7 @@ fn append_map_column_batch( macro_rules! process_primitive_maps { ($key_builder:ty, $key_append:ident, $val_builder:ty, $val_append:ident) => {{ for i in row_start..row_end { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); + read_row_at!(row, row_addresses_ptr, row_sizes_ptr, i); if row.is_null_at(column_idx) { map_builder.append(false)?; @@ -893,11 +895,7 @@ fn append_map_column_batch( // For other types, fall back to per-row dispatch _ => { for i in row_start..row_end { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); + read_row_at!(row, row_addresses_ptr, row_sizes_ptr, i); if row.is_null_at(column_idx) { map_builder.append(false)?; @@ -933,11 +931,7 @@ fn append_struct_fields_field_major( let mut struct_is_null = Vec::with_capacity(num_rows); for i in row_start..row_end { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let is_null = parent_row.is_null_at(column_idx); struct_is_null.push(is_null); @@ -959,11 +953,7 @@ fn append_struct_fields_field_major( // Struct is null, field is also null field_builder.append_null(); } else { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let nested_row = parent_row.get_struct(column_idx, num_fields); if nested_row.is_null_at($field_idx) { @@ -1025,11 +1015,7 @@ fn append_struct_fields_field_major( if struct_is_null[row_idx] { field_builder.append_null(); } else { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let nested_row = parent_row.get_struct(column_idx, num_fields); if nested_row.is_null_at(field_idx) { @@ -1047,11 +1033,7 @@ fn append_struct_fields_field_major( if struct_is_null[row_idx] { field_builder.append_null(); } else { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let nested_row = parent_row.get_struct(column_idx, num_fields); if nested_row.is_null_at(field_idx) { @@ -1071,11 +1053,7 @@ fn append_struct_fields_field_major( if struct_is_null[row_idx] { field_builder.append_null(); } else { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let nested_row = parent_row.get_struct(column_idx, num_fields); if nested_row.is_null_at(field_idx) { @@ -1103,11 +1081,7 @@ fn append_struct_fields_field_major( nested_addresses.push(0); nested_sizes.push(0); } else { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let parent_struct = parent_row.get_struct(column_idx, num_fields); if parent_struct.is_null_at(field_idx) { @@ -1143,11 +1117,7 @@ fn append_struct_fields_field_major( let null_row = SparkUnsafeRow::default(); append_field(dt, struct_builder, &null_row, field_idx)?; } else { - // SAFETY: Caller (append_columns) guarantees row_addresses_ptr and row_sizes_ptr - // are valid for indices [row_start, row_end) as provided by the JVM - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - parent_row.point_to(row_addr, row_size); + read_row_at!(parent_row, row_addresses_ptr, row_sizes_ptr, i); let nested_row = parent_row.get_struct(column_idx, num_fields); append_field(dt, struct_builder, &nested_row, field_idx)?; } @@ -1197,11 +1167,7 @@ pub(crate) fn append_columns( let mut row = SparkUnsafeRow::new(schema); for i in row_start..row_end { - // SAFETY: row_addresses_ptr and row_sizes_ptr are valid for indices [row_start, row_end) - // as guaranteed by the caller (JVM provides these arrays with correct bounds) - let row_addr = unsafe { *row_addresses_ptr.add(i) }; - let row_size = unsafe { *row_sizes_ptr.add(i) }; - row.point_to(row_addr, row_size); + read_row_at!(row, row_addresses_ptr, row_sizes_ptr, i); let is_null = row.is_null_at(column_idx); From ca702eeddc5d3548993a0608291874d214675411 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 12:08:43 -0700 Subject: [PATCH 19/22] refactor: replace unwrap with expect for builder downcasts Use expect with the builder type name to provide better error messages if a downcast fails. Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/shuffle/row.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index bc6da10228..7a23254256 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -748,7 +748,7 @@ fn append_list_column_batch( .values() .as_any_mut() .downcast_mut::<$builder_type>() - .unwrap(); + .expect(stringify!($builder_type)); array.$append_fn::(values_builder); list_builder.append(true); } @@ -836,7 +836,7 @@ fn append_map_column_batch( .keys() .as_any_mut() .downcast_mut::<$key_builder>() - .unwrap(); + .expect(stringify!($key_builder)); map.keys.$key_append::(keys_builder); } // Process values in a scope so borrow ends @@ -845,7 +845,7 @@ fn append_map_column_batch( .values() .as_any_mut() .downcast_mut::<$val_builder>() - .unwrap(); + .expect(stringify!($val_builder)); map.values.$val_append::(values_builder); } map_builder.append(true)?; From 970e30ae714a0ebb87e60e8cd13e24425e7be5c7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 12:10:00 -0700 Subject: [PATCH 20/22] smaller difff --- native/core/Cargo.toml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4d28127c92..9221e74b97 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -121,15 +121,15 @@ name = "bit_util" harness = false [[bench]] -name = "shuffle_writer" +name = "jvm_shuffle" harness = false [[bench]] -name = "parquet_decode" +name = "shuffle_writer" harness = false [[bench]] -name = "jvm_shuffle" +name = "parquet_decode" harness = false [[bench]] From 1b760540bb1c2c8aa2e127a7afeef1196ab4935d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 12:12:46 -0700 Subject: [PATCH 21/22] fix: resolve clippy errors in jni_api.rs - Use if-let instead of is_some() + unwrap() in update_metrics - Use local root_op variable directly instead of re-reading from exec_context Co-Authored-By: Claude Opus 4.5 --- native/core/src/execution/jni_api.rs | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 75c53198b8..1a11abee14 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -489,7 +489,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( let physical_plan_time = start.elapsed(); exec_context.plan_creation_time += physical_plan_time; - exec_context.root_op = Some(Arc::clone(&root_op)); exec_context.scans = scans; if exec_context.explain_native { @@ -501,13 +500,9 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( let task_ctx = exec_context.session_ctx.task_ctx(); // Each Comet native execution corresponds to a single Spark partition, // so we should always execute partition 0. - let stream = exec_context - .root_op - .as_ref() - .unwrap() - .native_plan - .execute(0, task_ctx)?; + let stream = root_op.native_plan.execute(0, task_ctx)?; exec_context.stream = Some(stream); + exec_context.root_op = Some(root_op); } else { // Pull input batches pull_input_batches(exec_context)?; @@ -617,8 +612,7 @@ pub extern "system" fn Java_org_apache_comet_Native_releasePlan( /// Updates the metrics of the query plan. fn update_metrics(env: &mut JNIEnv, exec_context: &mut ExecutionContext) -> CometResult<()> { - if exec_context.root_op.is_some() { - let native_query = exec_context.root_op.as_ref().unwrap(); + if let Some(native_query) = &exec_context.root_op { let metrics = exec_context.metrics.as_obj(); update_comet_metric(env, metrics, native_query) } else { From 4c5eb0bd56399623d1335fa75aaa01ad3a8bcf8e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 26 Jan 2026 13:37:23 -0700 Subject: [PATCH 22/22] revert: restore row_columnar.rs benchmark and remove jvm_shuffle.rs Co-Authored-By: Claude Opus 4.5 --- native/core/Cargo.toml | 2 +- native/core/benches/jvm_shuffle.rs | 840 ---------------------------- native/core/benches/row_columnar.rs | 113 ++++ 3 files changed, 114 insertions(+), 841 deletions(-) delete mode 100644 native/core/benches/jvm_shuffle.rs create mode 100644 native/core/benches/row_columnar.rs diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 73dbd619f9..ba5509583c 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -122,7 +122,7 @@ name = "bit_util" harness = false [[bench]] -name = "jvm_shuffle" +name = "row_columnar" harness = false [[bench]] diff --git a/native/core/benches/jvm_shuffle.rs b/native/core/benches/jvm_shuffle.rs deleted file mode 100644 index c8c13e9a52..0000000000 --- a/native/core/benches/jvm_shuffle.rs +++ /dev/null @@ -1,840 +0,0 @@ -// 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. - -//! Benchmarks for JVM shuffle row-to-columnar conversion. -//! -//! This benchmark measures the performance of converting Spark UnsafeRow -//! to Arrow arrays via `process_sorted_row_partition()`, which is called -//! by JVM shuffle (CometColumnarShuffle) when writing shuffle data. -//! -//! Covers: -//! - Primitive types (Int64) -//! - Struct (flat, nested, deeply nested) -//! - List -//! - Map - -use arrow::datatypes::{DataType, Field, Fields}; -use comet::execution::shuffle::row::{ - process_sorted_row_partition, SparkUnsafeObject, SparkUnsafeRow, -}; -use comet::execution::shuffle::CompressionCodec; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use std::sync::Arc; -use tempfile::Builder; - -const BATCH_SIZE: usize = 5000; - -/// Create a struct schema with the given number of int64 fields. -fn make_struct_schema(num_fields: usize) -> DataType { - let fields: Vec = (0..num_fields) - .map(|i| Field::new(format!("f{}", i), DataType::Int64, true)) - .collect(); - DataType::Struct(Fields::from(fields)) -} - -/// Calculate the row size for a struct with the given number of fields. -/// UnsafeRow layout: [null bits] [fixed-length values] -/// For struct: the struct value is stored as offset+size (8 bytes) pointing to nested row -fn get_row_size(num_struct_fields: usize) -> usize { - // Top-level row has 1 column (the struct) - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - // Struct pointer (offset + size) is 8 bytes - let struct_pointer_size = 8; - // Nested struct row - let nested_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_struct_fields); - let nested_data_size = num_struct_fields * 8; // int64 values - - top_level_bitset_width + struct_pointer_size + nested_bitset_width + nested_data_size -} - -struct RowData { - data: Vec, -} - -impl RowData { - fn new(num_struct_fields: usize) -> Self { - let row_size = get_row_size(num_struct_fields); - let mut data = vec![0u8; row_size]; - - // Top-level row layout: - // [null bits for 1 field] [struct pointer (offset, size)] - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - - // Nested struct starts after top-level row header + pointer - let nested_offset = top_level_bitset_width + 8; - let nested_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_struct_fields); - let nested_size = nested_bitset_width + num_struct_fields * 8; - - // Write struct pointer (offset in upper 32 bits, size in lower 32 bits) - let offset_and_size = ((nested_offset as i64) << 32) | (nested_size as i64); - data[top_level_bitset_width..top_level_bitset_width + 8] - .copy_from_slice(&offset_and_size.to_le_bytes()); - - // Fill nested struct with some data - for i in 0..num_struct_fields { - let value_offset = nested_offset + nested_bitset_width + i * 8; - let value = (i as i64) * 100; - data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); - } - - RowData { data } - } - - fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { - spark_row.point_to_slice(&self.data); - } -} - -fn benchmark_struct_conversion(c: &mut Criterion) { - let mut group = c.benchmark_group("struct_conversion"); - - // Test with different struct sizes and row counts - for num_fields in [5, 10, 20] { - for num_rows in [1000, 10000] { - let schema = vec![make_struct_schema(num_fields)]; - - // Create row data - let rows: Vec = (0..num_rows).map(|_| RowData::new(num_fields)).collect(); - - let spark_rows: Vec = rows - .iter() - .map(|row_data| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); - row_data.to_spark_row(&mut spark_row); - // Mark the struct column as not null - spark_row.set_not_null_at(0); - spark_row - }) - .collect(); - - let mut row_addresses: Vec = - spark_rows.iter().map(|row| row.get_row_addr()).collect(); - let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - - group.bench_with_input( - BenchmarkId::new( - format!("fields_{}", num_fields), - format!("rows_{}", num_rows), - ), - &(num_rows, &schema), - |b, (num_rows, schema)| { - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - *num_rows, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }, - ); - - // Keep spark_rows alive for the benchmark - std::mem::drop(spark_rows); - } - } - - group.finish(); -} - -/// Create a schema with nested structs: Struct> -fn make_nested_struct_schema(num_fields: usize) -> DataType { - let inner_fields: Vec = (0..num_fields) - .map(|i| Field::new(format!("inner_f{}", i), DataType::Int64, true)) - .collect(); - let inner_struct = DataType::Struct(Fields::from(inner_fields)); - let outer_fields = vec![Field::new("nested", inner_struct, true)]; - DataType::Struct(Fields::from(outer_fields)) -} - -/// Create a schema with deeply nested structs (3 levels): Struct>> -fn make_deeply_nested_struct_schema(num_fields: usize) -> DataType { - let inner_fields: Vec = (0..num_fields) - .map(|i| Field::new(format!("inner_f{}", i), DataType::Int64, true)) - .collect(); - let inner_struct = DataType::Struct(Fields::from(inner_fields)); - let middle_fields = vec![Field::new("level2", inner_struct, true)]; - let middle_struct = DataType::Struct(Fields::from(middle_fields)); - let outer_fields = vec![Field::new("level1", middle_struct, true)]; - DataType::Struct(Fields::from(outer_fields)) -} - -/// Calculate row size for nested struct: Struct> -fn get_nested_row_size(num_inner_fields: usize) -> usize { - // Top-level row has 1 column (the outer struct) - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let struct_pointer_size = 8; - - // Outer struct has 1 field (the inner struct) - let outer_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let outer_struct_size = outer_bitset_width + 8; // pointer to inner struct - - // Inner struct has num_inner_fields int64 fields - let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); - let inner_data_size = num_inner_fields * 8; - let inner_struct_size = inner_bitset_width + inner_data_size; - - top_level_bitset_width + struct_pointer_size + outer_struct_size + inner_struct_size -} - -/// Calculate row size for deeply nested struct: Struct>> -fn get_deeply_nested_row_size(num_inner_fields: usize) -> usize { - // Top-level row has 1 column (the level1 struct) - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let struct_pointer_size = 8; - - // Level 1 struct has 1 field (the level2 struct) - let level1_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let level1_struct_size = level1_bitset_width + 8; - - // Level 2 struct has 1 field (the inner struct) - let level2_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let level2_struct_size = level2_bitset_width + 8; - - // Inner struct has num_inner_fields int64 fields - let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); - let inner_data_size = num_inner_fields * 8; - let inner_struct_size = inner_bitset_width + inner_data_size; - - top_level_bitset_width - + struct_pointer_size - + level1_struct_size - + level2_struct_size - + inner_struct_size -} - -struct NestedRowData { - data: Vec, -} - -impl NestedRowData { - fn new(num_inner_fields: usize) -> Self { - let row_size = get_nested_row_size(num_inner_fields); - let mut data = vec![0u8; row_size]; - - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let outer_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); - - // Calculate offsets - let outer_struct_start = top_level_bitset_width + 8; - let outer_struct_size = outer_bitset_width + 8; - let inner_struct_start = outer_struct_start + outer_struct_size; - let inner_struct_size = inner_bitset_width + num_inner_fields * 8; - - // Write top-level struct pointer (points to outer struct) - let outer_offset_and_size = - ((outer_struct_start as i64) << 32) | (outer_struct_size as i64); - data[top_level_bitset_width..top_level_bitset_width + 8] - .copy_from_slice(&outer_offset_and_size.to_le_bytes()); - - // Write outer struct pointer (points to inner struct) - // Offset is relative to outer struct start - let inner_relative_offset = inner_struct_start - outer_struct_start; - let inner_offset_and_size = - ((inner_relative_offset as i64) << 32) | (inner_struct_size as i64); - data[outer_struct_start + outer_bitset_width..outer_struct_start + outer_bitset_width + 8] - .copy_from_slice(&inner_offset_and_size.to_le_bytes()); - - // Fill inner struct with some data - for i in 0..num_inner_fields { - let value_offset = inner_struct_start + inner_bitset_width + i * 8; - let value = (i as i64) * 100; - data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); - } - - NestedRowData { data } - } - - fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { - spark_row.point_to_slice(&self.data); - } -} - -struct DeeplyNestedRowData { - data: Vec, -} - -impl DeeplyNestedRowData { - fn new(num_inner_fields: usize) -> Self { - let row_size = get_deeply_nested_row_size(num_inner_fields); - let mut data = vec![0u8; row_size]; - - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let level1_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let level2_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let inner_bitset_width = SparkUnsafeRow::get_row_bitset_width(num_inner_fields); - - // Calculate offsets - let level1_struct_start = top_level_bitset_width + 8; - let level1_struct_size = level1_bitset_width + 8; - let level2_struct_start = level1_struct_start + level1_struct_size; - let level2_struct_size = level2_bitset_width + 8; - let inner_struct_start = level2_struct_start + level2_struct_size; - let inner_struct_size = inner_bitset_width + num_inner_fields * 8; - - // Write top-level struct pointer (points to level1 struct) - let level1_offset_and_size = - ((level1_struct_start as i64) << 32) | (level1_struct_size as i64); - data[top_level_bitset_width..top_level_bitset_width + 8] - .copy_from_slice(&level1_offset_and_size.to_le_bytes()); - - // Write level1 struct pointer (points to level2 struct) - let level2_relative_offset = level2_struct_start - level1_struct_start; - let level2_offset_and_size = - ((level2_relative_offset as i64) << 32) | (level2_struct_size as i64); - data[level1_struct_start + level1_bitset_width - ..level1_struct_start + level1_bitset_width + 8] - .copy_from_slice(&level2_offset_and_size.to_le_bytes()); - - // Write level2 struct pointer (points to inner struct) - let inner_relative_offset = inner_struct_start - level2_struct_start; - let inner_offset_and_size = - ((inner_relative_offset as i64) << 32) | (inner_struct_size as i64); - data[level2_struct_start + level2_bitset_width - ..level2_struct_start + level2_bitset_width + 8] - .copy_from_slice(&inner_offset_and_size.to_le_bytes()); - - // Fill inner struct with some data - for i in 0..num_inner_fields { - let value_offset = inner_struct_start + inner_bitset_width + i * 8; - let value = (i as i64) * 100; - data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); - } - - DeeplyNestedRowData { data } - } - - fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { - spark_row.point_to_slice(&self.data); - } -} - -fn benchmark_nested_struct_conversion(c: &mut Criterion) { - let mut group = c.benchmark_group("nested_struct_conversion"); - - // Test nested structs with different inner field counts - for num_fields in [5, 10, 20] { - for num_rows in [1000, 10000] { - let schema = vec![make_nested_struct_schema(num_fields)]; - - // Create row data - let rows: Vec = (0..num_rows) - .map(|_| NestedRowData::new(num_fields)) - .collect(); - - let spark_rows: Vec = rows - .iter() - .map(|row_data| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); - row_data.to_spark_row(&mut spark_row); - spark_row.set_not_null_at(0); - spark_row - }) - .collect(); - - let mut row_addresses: Vec = - spark_rows.iter().map(|row| row.get_row_addr()).collect(); - let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - - group.bench_with_input( - BenchmarkId::new( - format!("inner_fields_{}", num_fields), - format!("rows_{}", num_rows), - ), - &(num_rows, &schema), - |b, (num_rows, schema)| { - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - *num_rows, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }, - ); - - std::mem::drop(spark_rows); - } - } - - group.finish(); -} - -fn benchmark_deeply_nested_struct_conversion(c: &mut Criterion) { - let mut group = c.benchmark_group("deeply_nested_struct_conversion"); - - // Test deeply nested structs (3 levels) with different inner field counts - for num_fields in [5, 10, 20] { - for num_rows in [1000, 10000] { - let schema = vec![make_deeply_nested_struct_schema(num_fields)]; - - // Create row data - let rows: Vec = (0..num_rows) - .map(|_| DeeplyNestedRowData::new(num_fields)) - .collect(); - - let spark_rows: Vec = rows - .iter() - .map(|row_data| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); - row_data.to_spark_row(&mut spark_row); - spark_row.set_not_null_at(0); - spark_row - }) - .collect(); - - let mut row_addresses: Vec = - spark_rows.iter().map(|row| row.get_row_addr()).collect(); - let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - - group.bench_with_input( - BenchmarkId::new( - format!("inner_fields_{}", num_fields), - format!("rows_{}", num_rows), - ), - &(num_rows, &schema), - |b, (num_rows, schema)| { - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - *num_rows, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }, - ); - - std::mem::drop(spark_rows); - } - } - - group.finish(); -} - -/// Create a schema with a list column: List -fn make_list_schema(element_type: DataType) -> DataType { - DataType::List(Arc::new(Field::new("item", element_type, true))) -} - -/// Calculate row size for a list with the given number of elements. -/// UnsafeRow layout for list: [null bits] [list pointer (offset, size)] -/// List data: [num_elements (8 bytes)] [null bits] [element data] -fn get_list_row_size(num_elements: usize, element_size: usize) -> usize { - // Top-level row has 1 column (the list) - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let list_pointer_size = 8; - - // List header: num_elements (8 bytes) + null bitset - let list_null_bitset = ((num_elements + 63) / 64) * 8; - let list_header = 8 + list_null_bitset; - let list_data_size = num_elements * element_size; - - top_level_bitset_width + list_pointer_size + list_header + list_data_size -} - -struct ListRowData { - data: Vec, -} - -impl ListRowData { - fn new_int64_list(num_elements: usize) -> Self { - let row_size = get_list_row_size(num_elements, 8); - let mut data = vec![0u8; row_size]; - - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let list_null_bitset = ((num_elements + 63) / 64) * 8; - - // List starts after top-level header + pointer - let list_offset = top_level_bitset_width + 8; - let list_size = 8 + list_null_bitset + num_elements * 8; - - // Write list pointer (offset in upper 32 bits, size in lower 32 bits) - let offset_and_size = ((list_offset as i64) << 32) | (list_size as i64); - data[top_level_bitset_width..top_level_bitset_width + 8] - .copy_from_slice(&offset_and_size.to_le_bytes()); - - // Write number of elements at list start - data[list_offset..list_offset + 8].copy_from_slice(&(num_elements as i64).to_le_bytes()); - - // Fill list with data (after header) - let data_start = list_offset + 8 + list_null_bitset; - for i in 0..num_elements { - let value_offset = data_start + i * 8; - let value = (i as i64) * 100; - data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); - } - - ListRowData { data } - } - - fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { - spark_row.point_to_slice(&self.data); - } -} - -fn benchmark_list_conversion(c: &mut Criterion) { - let mut group = c.benchmark_group("list_conversion"); - - // Test with different list sizes and row counts - for num_elements in [10, 100] { - for num_rows in [1000, 10000] { - let schema = vec![make_list_schema(DataType::Int64)]; - - // Create row data - each row has a list with num_elements items - let rows: Vec = (0..num_rows) - .map(|_| ListRowData::new_int64_list(num_elements)) - .collect(); - - let spark_rows: Vec = rows - .iter() - .map(|row_data| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); - row_data.to_spark_row(&mut spark_row); - spark_row.set_not_null_at(0); - spark_row - }) - .collect(); - - let mut row_addresses: Vec = - spark_rows.iter().map(|row| row.get_row_addr()).collect(); - let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - - group.bench_with_input( - BenchmarkId::new( - format!("elements_{}", num_elements), - format!("rows_{}", num_rows), - ), - &(num_rows, &schema), - |b, (num_rows, schema)| { - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - *num_rows, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }, - ); - - std::mem::drop(spark_rows); - } - } - - group.finish(); -} - -/// Create a schema with a map column: Map -fn make_map_schema() -> DataType { - // Map is represented as List> in Arrow - let key_field = Field::new("key", DataType::Int64, false); - let value_field = Field::new("value", DataType::Int64, true); - let entries_field = Field::new( - "entries", - DataType::Struct(Fields::from(vec![key_field, value_field])), - false, - ); - DataType::Map(Arc::new(entries_field), false) -} - -/// Calculate row size for a map with the given number of entries. -/// UnsafeRow layout for map: [null bits] [map pointer (offset, size)] -/// Map data: [key_array_size (8 bytes)] [key_array] [value_array] -/// Array format: [num_elements (8 bytes)] [null bits] [element data] -fn get_map_row_size(num_entries: usize) -> usize { - // Top-level row has 1 column (the map) - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let map_pointer_size = 8; - - // Key array: num_elements (8) + null bitset + data - let key_null_bitset = ((num_entries + 63) / 64) * 8; - let key_array_size = 8 + key_null_bitset + num_entries * 8; - - // Value array: num_elements (8) + null bitset + data - let value_null_bitset = ((num_entries + 63) / 64) * 8; - let value_array_size = 8 + value_null_bitset + num_entries * 8; - - // Map header (key array size) + key array + value array - let map_size = 8 + key_array_size + value_array_size; - - top_level_bitset_width + map_pointer_size + map_size -} - -struct MapRowData { - data: Vec, -} - -impl MapRowData { - fn new_int64_map(num_entries: usize) -> Self { - let row_size = get_map_row_size(num_entries); - let mut data = vec![0u8; row_size]; - - let top_level_bitset_width = SparkUnsafeRow::get_row_bitset_width(1); - let key_null_bitset = ((num_entries + 63) / 64) * 8; - let value_null_bitset = ((num_entries + 63) / 64) * 8; - - let key_array_size = 8 + key_null_bitset + num_entries * 8; - let value_array_size = 8 + value_null_bitset + num_entries * 8; - let map_size = 8 + key_array_size + value_array_size; - - // Map starts after top-level header + pointer - let map_offset = top_level_bitset_width + 8; - - // Write map pointer (offset in upper 32 bits, size in lower 32 bits) - let offset_and_size = ((map_offset as i64) << 32) | (map_size as i64); - data[top_level_bitset_width..top_level_bitset_width + 8] - .copy_from_slice(&offset_and_size.to_le_bytes()); - - // Write key array size at map start - data[map_offset..map_offset + 8].copy_from_slice(&(key_array_size as i64).to_le_bytes()); - - // Key array starts after map header - let key_array_offset = map_offset + 8; - // Write number of elements - data[key_array_offset..key_array_offset + 8] - .copy_from_slice(&(num_entries as i64).to_le_bytes()); - // Fill key data (after header) - let key_data_start = key_array_offset + 8 + key_null_bitset; - for i in 0..num_entries { - let value_offset = key_data_start + i * 8; - let value = i as i64; - data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); - } - - // Value array starts after key array - let value_array_offset = key_array_offset + key_array_size; - // Write number of elements - data[value_array_offset..value_array_offset + 8] - .copy_from_slice(&(num_entries as i64).to_le_bytes()); - // Fill value data (after header) - let value_data_start = value_array_offset + 8 + value_null_bitset; - for i in 0..num_entries { - let value_offset = value_data_start + i * 8; - let value = (i as i64) * 100; - data[value_offset..value_offset + 8].copy_from_slice(&value.to_le_bytes()); - } - - MapRowData { data } - } - - fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { - spark_row.point_to_slice(&self.data); - } -} - -fn benchmark_map_conversion(c: &mut Criterion) { - let mut group = c.benchmark_group("map_conversion"); - - // Test with different map sizes and row counts - for num_entries in [10, 100] { - for num_rows in [1000, 10000] { - let schema = vec![make_map_schema()]; - - // Create row data - each row has a map with num_entries items - let rows: Vec = (0..num_rows) - .map(|_| MapRowData::new_int64_map(num_entries)) - .collect(); - - let spark_rows: Vec = rows - .iter() - .map(|row_data| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(1); - row_data.to_spark_row(&mut spark_row); - spark_row.set_not_null_at(0); - spark_row - }) - .collect(); - - let mut row_addresses: Vec = - spark_rows.iter().map(|row| row.get_row_addr()).collect(); - let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - - group.bench_with_input( - BenchmarkId::new( - format!("entries_{}", num_entries), - format!("rows_{}", num_rows), - ), - &(num_rows, &schema), - |b, (num_rows, schema)| { - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - *num_rows, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }, - ); - - std::mem::drop(spark_rows); - } - } - - group.finish(); -} - -/// Benchmark for primitive type columns (many Int64 columns). -/// This tests the baseline performance without complex type overhead. -fn benchmark_primitive_columns(c: &mut Criterion) { - let mut group = c.benchmark_group("primitive_columns"); - - const NUM_COLS: usize = 100; - let row_size: usize = SparkUnsafeRow::get_row_bitset_width(NUM_COLS) + NUM_COLS * 8; - - for num_rows in [1000, 10000] { - let schema = vec![DataType::Int64; NUM_COLS]; - - // Create row data - let row_data: Vec> = (0..num_rows) - .map(|_| { - let mut data = vec![0u8; row_size]; - // Fill with some data after the bitset - for i in SparkUnsafeRow::get_row_bitset_width(NUM_COLS)..row_size { - data[i] = i as u8; - } - data - }) - .collect(); - - let spark_rows: Vec = row_data - .iter() - .map(|data| { - let mut spark_row = SparkUnsafeRow::new_with_num_fields(NUM_COLS); - spark_row.point_to_slice(data); - for i in 0..NUM_COLS { - spark_row.set_not_null_at(i); - } - spark_row - }) - .collect(); - - let mut row_addresses: Vec = spark_rows.iter().map(|row| row.get_row_addr()).collect(); - let mut row_sizes: Vec = spark_rows.iter().map(|row| row.get_row_size()).collect(); - - let row_address_ptr = row_addresses.as_mut_ptr(); - let row_size_ptr = row_sizes.as_mut_ptr(); - - group.bench_with_input( - BenchmarkId::new("cols_100", format!("rows_{}", num_rows)), - &(num_rows, &schema), - |b, (num_rows, schema)| { - b.iter(|| { - let tempfile = Builder::new().tempfile().unwrap(); - - process_sorted_row_partition( - *num_rows, - BATCH_SIZE, - row_address_ptr, - row_size_ptr, - schema, - tempfile.path().to_str().unwrap().to_string(), - 1.0, - false, - 0, - None, - &CompressionCodec::Zstd(1), - ) - .unwrap(); - }); - }, - ); - - std::mem::drop(spark_rows); - } - - group.finish(); -} - -fn config() -> Criterion { - Criterion::default() -} - -criterion_group! { - name = benches; - config = config(); - targets = benchmark_primitive_columns, benchmark_struct_conversion, benchmark_nested_struct_conversion, benchmark_deeply_nested_struct_conversion, benchmark_list_conversion, benchmark_map_conversion -} -criterion_main!(benches); diff --git a/native/core/benches/row_columnar.rs b/native/core/benches/row_columnar.rs new file mode 100644 index 0000000000..a62574111b --- /dev/null +++ b/native/core/benches/row_columnar.rs @@ -0,0 +1,113 @@ +// 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::datatypes::DataType as ArrowDataType; +use comet::execution::shuffle::row::{ + process_sorted_row_partition, SparkUnsafeObject, SparkUnsafeRow, +}; +use comet::execution::shuffle::CompressionCodec; +use criterion::{criterion_group, criterion_main, Criterion}; +use tempfile::Builder; + +const NUM_ROWS: usize = 10000; +const BATCH_SIZE: usize = 5000; +const NUM_COLS: usize = 100; +const ROW_SIZE: usize = SparkUnsafeRow::get_row_bitset_width(NUM_COLS) + NUM_COLS * 8; + +fn benchmark(c: &mut Criterion) { + let mut group = c.benchmark_group("row_array_conversion"); + + group.bench_function("row_to_array", |b| { + let spark_rows = (0..NUM_ROWS) + .map(|_| { + let mut spark_row = SparkUnsafeRow::new_with_num_fields(NUM_COLS); + let mut row = Row::new(); + + for i in SparkUnsafeRow::get_row_bitset_width(NUM_COLS)..ROW_SIZE { + row.data[i] = i as u8; + } + + row.to_spark_row(&mut spark_row); + + for i in 0..NUM_COLS { + spark_row.set_not_null_at(i); + } + + spark_row + }) + .collect::>(); + + let mut row_addresses = spark_rows + .iter() + .map(|row| row.get_row_addr()) + .collect::>(); + let mut row_sizes = spark_rows + .iter() + .map(|row| row.get_row_size()) + .collect::>(); + + let row_address_ptr = row_addresses.as_mut_ptr(); + let row_size_ptr = row_sizes.as_mut_ptr(); + let schema = vec![ArrowDataType::Int64; NUM_COLS]; + + b.iter(|| { + let tempfile = Builder::new().tempfile().unwrap(); + + process_sorted_row_partition( + NUM_ROWS, + BATCH_SIZE, + row_address_ptr, + row_size_ptr, + &schema, + tempfile.path().to_str().unwrap().to_string(), + 1.0, + false, + 0, + None, + &CompressionCodec::Zstd(1), + ) + .unwrap(); + }); + }); +} + +struct Row { + data: Box<[u8; ROW_SIZE]>, +} + +impl Row { + pub fn new() -> Self { + Row { + data: Box::new([0u8; ROW_SIZE]), + } + } + + pub fn to_spark_row(&self, spark_row: &mut SparkUnsafeRow) { + spark_row.point_to_slice(self.data.as_ref()); + } +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = benchmark +} +criterion_main!(benches);