From 27100825c65ce6e53f2b15e2411229ef6689d727 Mon Sep 17 00:00:00 2001 From: hall-alex Date: Mon, 6 Apr 2026 16:33:01 +0200 Subject: [PATCH 01/13] Make sure we hand up ordered batches per file --- iceberg_rust_ffi/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iceberg_rust_ffi/src/lib.rs b/iceberg_rust_ffi/src/lib.rs index dee2aad..7eede7c 100644 --- a/iceberg_rust_ffi/src/lib.rs +++ b/iceberg_rust_ffi/src/lib.rs @@ -174,7 +174,7 @@ pub(crate) fn transform_stream_with_parallel_serialization( )), } }) - .buffer_unordered(concurrency) + .buffered(concurrency) .boxed() } From e49a4da9c1b31163a012848f7c721a9aee630646 Mon Sep 17 00:00:00 2001 From: hall-alex Date: Mon, 6 Apr 2026 18:29:38 +0200 Subject: [PATCH 02/13] File-parallel pipeline with strict ordering and backpressure Replace the to_arrow() code path with a custom file-parallel pipeline that processes N parquet files concurrently while preserving strict file-then-row ordering for Julia consumption. Architecture: - plan_files() provides an ordered list of FileScanTasks - Each file task runs as a background tokio task with its own mpsc channel and per-file Semaphore(100MB) for backpressure - FuturesOrdered yields per-file receivers in file order; the consumer drains file 0 batch-by-batch, then file 1, etc. - Each file uses the same iceberg-rs ArrowReader code path as before Changes: - New: ordered_file_pipeline.rs (~420 lines incl. profiling stats) - full.rs: IcebergScan gains file_io, batch_size, file_concurrency fields; iceberg_arrow_stream uses plan_files() + pipeline - scan_common.rs / incremental.rs: pass through new fields in macros - Temporary PipelineStats with FFI print_summary for benchmarking Co-Authored-By: Claude Opus 4.6 (1M context) --- iceberg_rust_ffi/src/full.rs | 116 +++- iceberg_rust_ffi/src/incremental.rs | 7 + iceberg_rust_ffi/src/lib.rs | 3 + iceberg_rust_ffi/src/ordered_file_pipeline.rs | 499 ++++++++++++++++++ iceberg_rust_ffi/src/scan_common.rs | 12 + 5 files changed, 608 insertions(+), 29 deletions(-) create mode 100644 iceberg_rust_ffi/src/ordered_file_pipeline.rs diff --git a/iceberg_rust_ffi/src/full.rs b/iceberg_rust_ffi/src/full.rs index 1cb7285..0324610 100644 --- a/iceberg_rust_ffi/src/full.rs +++ b/iceberg_rust_ffi/src/full.rs @@ -1,50 +1,91 @@ use std::ffi::{c_char, c_void, CStr}; use std::ptr; +use iceberg::io::FileIO; use iceberg::scan::{TableScan, TableScanBuilder}; use object_store_ffi::{ export_runtime_op, with_cancellation, CResult, NotifyGuard, ResponseGuard, RT, }; -use tokio::sync::Mutex as AsyncMutex; - use crate::scan_common::*; use crate::{IcebergArrowStream, IcebergArrowStreamResponse, IcebergTable}; -/// Struct for regular (full) scan builder and scan +/// Holds state for a full table scan across its lifecycle: +/// 1. Construction: `builder` is set, everything else is None/0. +/// 2. Configuration: Julia calls with_* methods that transform `builder`. +/// 3. Build: `builder` is consumed → `scan` is populated. +/// 4. Stream: `scan` + `file_io` + `batch_size` are consumed by +/// `iceberg_arrow_stream` to create the file-parallel pipeline. #[repr(C)] pub struct IcebergScan { pub builder: Option>, pub scan: Option, - /// 0 = auto-detect (num_cpus) + /// Serialization thread count (0 = auto-detect). Currently unused by the + /// pipeline (kept for future use / backward compat with incremental scan). pub serialization_concurrency: usize, + /// Cloned from the Table at construction time. Passed to the pipeline so + /// each per-file ArrowReader can open its own parquet file. + pub file_io: Option, + /// Captured when Julia calls with_batch_size. Forwarded to each per-file + /// ArrowReaderBuilder inside the pipeline. + pub batch_size: Option, + /// How many parquet files to process concurrently in the pipeline. + /// Set by with_data_file_concurrency_limit (0 = auto-detect). + pub file_concurrency: usize, } unsafe impl Send for IcebergScan {} -/// Create a new scan builder +/// Create a new scan builder from an opened table. +/// Captures `file_io` from the table for later use by the pipeline. #[no_mangle] pub extern "C" fn iceberg_new_scan(table: *mut IcebergTable) -> *mut IcebergScan { if table.is_null() { return ptr::null_mut(); } let table_ref = unsafe { &*table }; + let file_io = table_ref.table.file_io().clone(); let scan_builder = table_ref.table.scan(); Box::into_raw(Box::new(IcebergScan { builder: Some(scan_builder), scan: None, serialization_concurrency: 0, + file_io: Some(file_io), + batch_size: None, + file_concurrency: 0, })) } -// Use macros from scan_common for shared functionality +// ── Scan builder configuration (via macros from scan_common.rs) ───────── + impl_select_columns!(iceberg_select_columns, IcebergScan); -impl_scan_builder_method!( - iceberg_scan_with_data_file_concurrency_limit, - IcebergScan, - with_data_file_concurrency_limit, - n: usize -); +/// Set file concurrency. Hand-written (not macro-generated) because we need +/// to both (a) forward the value to the iceberg-rs scan builder and (b) +/// capture it in `file_concurrency` for our pipeline. +#[no_mangle] +pub extern "C" fn iceberg_scan_with_data_file_concurrency_limit( + scan: &mut *mut IcebergScan, + n: usize, +) -> CResult { + if scan.is_null() || (*scan).is_null() { + return CResult::Error; + } + let scan_ref = unsafe { Box::from_raw(*scan) }; + if scan_ref.builder.is_none() { + return CResult::Error; + } + *scan = Box::into_raw(Box::new(IcebergScan { + builder: scan_ref + .builder + .map(|b| b.with_data_file_concurrency_limit(n)), + scan: scan_ref.scan, + serialization_concurrency: scan_ref.serialization_concurrency, + file_io: scan_ref.file_io, + batch_size: scan_ref.batch_size, + file_concurrency: n, + })); + CResult::Ok +} impl_scan_builder_method!( iceberg_scan_with_manifest_file_concurrency_limit, @@ -80,7 +121,16 @@ impl_scan_builder_method!( snapshot_id: i64 ); -// Async function to initialize stream from a table scan +// ── Stream creation ───────────────────────────────────────────────────── +// +// Instead of calling iceberg-rs's `scan.to_arrow()` (which uses +// `try_for_each_concurrent` internally and interleaves batches across +// files in arbitrary order), we: +// 1. Call `scan.plan_files()` to get an ordered list of FileScanTasks. +// 2. Feed them into our own file-parallel pipeline +// (ordered_file_pipeline.rs) which processes N files concurrently +// but yields batches in strict file-then-row order. + export_runtime_op!( iceberg_arrow_stream, IcebergArrowStreamResponse, @@ -88,39 +138,47 @@ export_runtime_op!( if scan.is_null() { return Err(anyhow::anyhow!("Null scan pointer provided")); } - let scan_ptr = unsafe { &*scan }; + let scan_ptr = unsafe { &mut *scan }; let scan_ref = &scan_ptr.scan; if scan_ref.is_none() { return Err(anyhow::anyhow!("Scan not initialized")); } - // Determine concurrency (0 = auto-detect) - let serialization_concurrency = scan_ptr.serialization_concurrency; - let serialization_concurrency = if serialization_concurrency == 0 { + // File pipeline concurrency (0 = auto-detect from available CPUs) + let concurrency = scan_ptr.file_concurrency; + let concurrency = if concurrency == 0 { std::thread::available_parallelism() .map(|n| n.get()) .unwrap_or(1) } else { - serialization_concurrency + concurrency }; - Ok((scan_ref.as_ref().unwrap(), serialization_concurrency)) + // Take file_io — it's moved into the pipeline (one clone per file task). + let file_io = scan_ptr + .file_io + .take() + .ok_or_else(|| anyhow::anyhow!("file_io not available"))?; + let batch_size = scan_ptr.batch_size; + + Ok((scan_ref.as_ref().unwrap(), concurrency, file_io, batch_size)) }, result_tuple, async { - let (scan_ref, serialization_concurrency) = result_tuple; + let (scan_ref, concurrency, file_io, batch_size) = result_tuple; - let stream = scan_ref.to_arrow().await?; + // Collect the ordered file task list from iceberg-rs. + use futures::TryStreamExt; + let tasks: Vec = + scan_ref.plan_files().await?.try_collect().await?; - // Transform stream: RecordBatch -> ArrowBatch with parallel serialization - let serialized_stream = crate::transform_stream_with_parallel_serialization( - stream, - serialization_concurrency - ); + // Hand off to the file-parallel pipeline. + let stream = crate::ordered_file_pipeline::create_pipeline( + tasks, file_io, batch_size, concurrency, + ) + .await?; - Ok::(IcebergArrowStream { - stream: AsyncMutex::new(serialized_stream), - }) + Ok::(stream) }, scan: *mut IcebergScan ); diff --git a/iceberg_rust_ffi/src/incremental.rs b/iceberg_rust_ffi/src/incremental.rs index 9e1d1fe..8a281c2 100644 --- a/iceberg_rust_ffi/src/incremental.rs +++ b/iceberg_rust_ffi/src/incremental.rs @@ -22,6 +22,10 @@ pub struct IcebergIncrementalScan { pub scan: Option, /// 0 = auto-detect (num_cpus) pub serialization_concurrency: usize, + // Present for macro compatibility with IcebergScan; unused for incremental. + pub file_io: Option, + pub batch_size: Option, + pub file_concurrency: usize, } unsafe impl Send for IcebergIncrementalScan {} @@ -102,6 +106,9 @@ pub extern "C" fn iceberg_new_incremental_scan( builder: Some(scan_builder), scan: None, serialization_concurrency: 0, + file_io: None, + batch_size: None, + file_concurrency: 0, })) } diff --git a/iceberg_rust_ffi/src/lib.rs b/iceberg_rust_ffi/src/lib.rs index 7eede7c..be2dd10 100644 --- a/iceberg_rust_ffi/src/lib.rs +++ b/iceberg_rust_ffi/src/lib.rs @@ -31,6 +31,9 @@ mod writer; // Column-based writer module (zero-copy from Julia) mod writer_columns; +// Ordered file-parallel pipeline +mod ordered_file_pipeline; + // Response types module mod response; diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs new file mode 100644 index 0000000..d091b63 --- /dev/null +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -0,0 +1,499 @@ +//! File-parallel pipeline for reading Iceberg tables with strict ordering. +//! +//! # Problem +//! iceberg-rs's `to_arrow()` uses `try_for_each_concurrent` which interleaves +//! batches from different files in arbitrary order. We need strict +//! file-then-row ordering. +//! +//! # Approach +//! We call `plan_files()` to get an ordered list of FileScanTasks, then +//! process N files concurrently while yielding batches in strict file order. +//! +//! Each file task is a background tokio task that: +//! 1. Builds a per-file ArrowReader (same iceberg-rs code path as to_arrow) +//! 2. Reads row groups sequentially → RecordBatch stream +//! 3. Serializes each batch to Arrow IPC (via spawn_blocking) +//! 4. Sends serialized batches into a per-file mpsc channel +//! +//! A consumer (`run`) uses FuturesOrdered to drain files in order: +//! - FuturesOrdered yields per-file receivers in the order files were pushed +//! - The consumer drains file 0's channel batch-by-batch, then file 1's, etc. +//! - Each drained batch is forwarded to the outer channel (read by Julia) +//! +//! # Memory bounding +//! Each file task has its own Semaphore(MAX_BUFFERED_BYTES_PER_TASK). After +//! serializing a batch, the task acquires byte_len permits. If the budget is +//! exhausted, the task yields (async, not blocking) until the consumer drains +//! batches and releases permits. This caps each file's buffered output to +//! ~100MB independently. + +use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering}; +use std::sync::Arc; +use std::time::Instant; + +use futures::StreamExt; +use iceberg::arrow::ArrowReaderBuilder; +use iceberg::io::FileIO; +use iceberg::scan::FileScanTask; +use tokio::sync::{mpsc, Mutex as AsyncMutex, Semaphore}; + +use crate::table::{ArrowBatch, IcebergArrowStream}; + +/// Per-file cap on serialized bytes buffered ahead of the consumer. +const MAX_BUFFERED_BYTES_PER_TASK: usize = 100 * 1024 * 1024; + +/// Hard cap on file-level concurrency to keep total memory bounded. +const MAX_FILE_CONCURRENCY: usize = 16; + +// =========================================================================== +// Temporary profiling — will be removed before merging to production. +// +// Global atomic counters accumulate timing/size data across all file tasks. +// Called from Julia via `@ccall iceberg_print_pipeline_stats()`. +// =========================================================================== + +/// Accumulates profiling data across all file tasks in a pipeline run. +/// All fields are atomics so concurrent file tasks can update without locks. +struct PipelineStats { + // ── Overall ── + pipeline_wall_ns: AtomicU64, + files_completed: AtomicUsize, + batches_produced: AtomicUsize, + bytes_produced: AtomicU64, + + // ── Concurrency ── + peak_concurrency: AtomicUsize, + active_tasks: AtomicUsize, + + // ── Per-phase cumulative time (ns), summed across all file tasks ── + /// Time to build ArrowReader and call reader.read() (opens parquet, + /// loads metadata, resolves schema, loads delete files, builds filters). + reader_setup_ns: AtomicU64, + /// Time inside batch_stream.next() — fetches compressed pages from + /// storage, decompresses (ZSTD), decodes pages, assembles columns. + fetch_decode_ns: AtomicU64, + /// Time in spawn_blocking(serialize_record_batch) — writes RecordBatch + /// to Arrow IPC wire format for transfer to Julia. + serialize_ns: AtomicU64, + /// Time blocked on per-file semaphore (backpressure from consumer). + semaphore_wait_ns: AtomicU64, + + // ── Consumer ── + /// Time the consumer spends waiting on FuturesOrdered for the next + /// file's receiver (ordering stall — head file not ready yet). + consumer_wait_ns: AtomicU64, + + // ── Memory ── + /// Live counter of serialized bytes buffered across all file tasks. + buffered_bytes: AtomicU64, + /// High-water mark of buffered_bytes. + peak_buffered_bytes: AtomicU64, +} + +impl PipelineStats { + const fn new() -> Self { + Self { + pipeline_wall_ns: AtomicU64::new(0), + files_completed: AtomicUsize::new(0), + batches_produced: AtomicUsize::new(0), + bytes_produced: AtomicU64::new(0), + peak_concurrency: AtomicUsize::new(0), + active_tasks: AtomicUsize::new(0), + reader_setup_ns: AtomicU64::new(0), + fetch_decode_ns: AtomicU64::new(0), + serialize_ns: AtomicU64::new(0), + semaphore_wait_ns: AtomicU64::new(0), + consumer_wait_ns: AtomicU64::new(0), + buffered_bytes: AtomicU64::new(0), + peak_buffered_bytes: AtomicU64::new(0), + } + } + + fn reset(&self) { + self.pipeline_wall_ns.store(0, Ordering::Relaxed); + self.files_completed.store(0, Ordering::Relaxed); + self.batches_produced.store(0, Ordering::Relaxed); + self.bytes_produced.store(0, Ordering::Relaxed); + self.peak_concurrency.store(0, Ordering::Relaxed); + self.active_tasks.store(0, Ordering::Relaxed); + self.reader_setup_ns.store(0, Ordering::Relaxed); + self.fetch_decode_ns.store(0, Ordering::Relaxed); + self.serialize_ns.store(0, Ordering::Relaxed); + self.semaphore_wait_ns.store(0, Ordering::Relaxed); + self.consumer_wait_ns.store(0, Ordering::Relaxed); + self.buffered_bytes.store(0, Ordering::Relaxed); + self.peak_buffered_bytes.store(0, Ordering::Relaxed); + } + + fn track_task_start(&self) { + let prev = self.active_tasks.fetch_add(1, Ordering::Relaxed); + self.peak_concurrency.fetch_max(prev + 1, Ordering::Relaxed); + } + + fn track_task_end(&self) { + self.active_tasks.fetch_sub(1, Ordering::Relaxed); + } + + fn track_buffer_add(&self, bytes: u64) { + let prev = self.buffered_bytes.fetch_add(bytes, Ordering::Relaxed); + self.peak_buffered_bytes + .fetch_max(prev + bytes, Ordering::Relaxed); + } + + fn track_buffer_release(&self, bytes: u64) { + self.buffered_bytes.fetch_sub(bytes, Ordering::Relaxed); + } + + fn print_summary(&self) { + let wall_ms = self.pipeline_wall_ns.load(Ordering::Relaxed) as f64 / 1e6; + let files = self.files_completed.load(Ordering::Relaxed); + let batches = self.batches_produced.load(Ordering::Relaxed); + let bytes = self.bytes_produced.load(Ordering::Relaxed); + let peak = self.peak_concurrency.load(Ordering::Relaxed); + let setup_ms = self.reader_setup_ns.load(Ordering::Relaxed) as f64 / 1e6; + let fd_ms = self.fetch_decode_ns.load(Ordering::Relaxed) as f64 / 1e6; + let ser_ms = self.serialize_ns.load(Ordering::Relaxed) as f64 / 1e6; + let sem_ms = self.semaphore_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; + let con_ms = self.consumer_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; + let peak_buf = self.peak_buffered_bytes.load(Ordering::Relaxed) as f64 / (1024.0 * 1024.0); + + let bytes_mb = bytes as f64 / (1024.0 * 1024.0); + let throughput = if wall_ms > 0.0 { + bytes_mb / (wall_ms / 1000.0) + } else { + 0.0 + }; + let file_wall_ms = setup_ms + fd_ms + ser_ms + sem_ms; + let parallelism = if wall_ms > 0.0 { + file_wall_ms / wall_ms + } else { + 0.0 + }; + let limit_mb = MAX_BUFFERED_BYTES_PER_TASK / (1024 * 1024); + + // Box layout: "│ " + content + padding + " │", total = BOX chars. + // All content uses ASCII only so byte len = display width. + const BOX: usize = 68; // total width including borders + + let row = |content: &str| { + // "│ " = 3, " │" = 3 => content area = BOX - 6 + let pad = (BOX - 6).saturating_sub(content.len()); + println!("│ {}{:pad$} │", content, "", pad = pad); + }; + let dashes = |n: usize| -> String { "─".repeat(n) }; + let sep = |label: &str| { + // "│ ── label ──── │" — label is ASCII, dashes are multi-byte. + // Display width: 3 + 3 + label.len() + 1 + fill + 1 = BOX + // => fill = BOX - 3 - 3 - label.len() - 3 - 1 = BOX - 10 - label.len() + let fill = (BOX - 10).saturating_sub(label.len()); + println!("│ {} {} {} │", dashes(2), label, dashes(fill)); + }; + let border = |left: char, right: char| { + println!("{}{}{}", left, dashes(BOX - 2), right); + }; + + border('┌', '┐'); + row(&format!("Pipeline Stats")); + row(""); + row(&format!("wall time: {:>9.1} ms", wall_ms)); + row(&format!("files: {:>9} peak concurrency: {}", files, peak)); + row(&format!("batches: {:>9} serialized: {:.1} MB", batches, bytes_mb)); + row(&format!("throughput: {:>9.1} MB/s parallelism: {:.1}x", throughput, parallelism)); + sep("time across all file tasks (sum)"); + row(&format!("reader setup: {:>9.1} ms (open, metadata, deletes)", setup_ms)); + row(&format!("fetch+decode: {:>9.1} ms (I/O + ZSTD + decode)", fd_ms)); + row(&format!("serialize IPC: {:>9.1} ms (RecordBatch -> Arrow IPC)", ser_ms)); + row(&format!("semaphore wait: {:>9.1} ms (backpressure)", sem_ms)); + sep("consumer"); + row(&format!("ordering stall: {:>9.1} ms (waiting for head file)", con_ms)); + sep("memory"); + row(&format!("peak buffered: {:>9.1} MB (limit: {} MB/task)", peak_buf, limit_mb)); + border('└', '┘'); + } +} + +static STATS: PipelineStats = PipelineStats::new(); + +// ── FFI exports for profiling (called from Julia benchmark teardown) ───── + +#[no_mangle] +pub extern "C" fn iceberg_print_pipeline_stats() { + STATS.print_summary(); +} + +#[no_mangle] +pub extern "C" fn iceberg_reset_pipeline_stats() { + STATS.reset(); +} + +// =========================================================================== +// Pipeline implementation +// =========================================================================== + +/// A serialized Arrow IPC batch bundled with its byte size and a reference +/// to the originating file task's semaphore. The consumer releases permits +/// after forwarding the batch to Julia, unblocking the producer. +struct BufferedBatch { + batch: ArrowBatch, + byte_len: usize, + semaphore: Arc, +} + +/// Create the file-parallel pipeline and return it as an IcebergArrowStream. +/// +/// Spawns a background `run` task that processes files concurrently and +/// feeds serialized batches into an mpsc channel. The returned stream +/// wraps the receiving end of that channel. +pub async fn create_pipeline( + tasks: Vec, + file_io: FileIO, + batch_size: Option, + concurrency: usize, +) -> anyhow::Result { + assert!( + concurrency <= MAX_FILE_CONCURRENCY, + "file concurrency {concurrency} exceeds hard cap {MAX_FILE_CONCURRENCY}" + ); + + STATS.reset(); + + // Outer channel: run() → Julia (via IcebergArrowStream). + let (tx, rx) = mpsc::channel(concurrency * 2); + + tokio::spawn(run(tasks, file_io, batch_size, concurrency, tx)); + + // Wrap the mpsc receiver as a BoxStream for IcebergArrowStream. + let stream = futures::stream::unfold(rx, |mut rx| async move { + rx.recv().await.map(|item| (item, rx)) + }) + .boxed(); + + Ok(IcebergArrowStream { + stream: AsyncMutex::new(stream), + }) +} + +/// Spawn a single file task. Returns a future that resolves immediately +/// to the receiving end of the file's batch channel. +/// +/// The actual work (parquet I/O, decode, serialize) happens in the +/// background tokio task. The future resolves to the Receiver so that +/// FuturesOrdered can yield receivers in file order. +fn spawn_file_task( + task: FileScanTask, + file_io: FileIO, + batch_size: Option, +) -> impl std::future::Future< + Output = Result>, iceberg::Error>, +> { + // Each file gets its own semaphore for independent backpressure. + let sem = Arc::new(Semaphore::new(MAX_BUFFERED_BYTES_PER_TASK)); + let (file_tx, file_rx) = mpsc::channel(8); + + tokio::spawn(process_file(task, file_io, batch_size, sem, file_tx)); + + async move { Ok(file_rx) } +} + +/// Main consumer loop. Orchestrates file-level parallelism while +/// maintaining strict file ordering. +/// +/// Uses FuturesOrdered to poll N file tasks concurrently but yield their +/// receivers in push order. For each file, drains its channel batch-by-batch, +/// forwarding to the outer channel and releasing semaphore permits. +async fn run( + tasks: Vec, + file_io: FileIO, + batch_size: Option, + concurrency: usize, + tx: mpsc::Sender>, +) { + use futures::stream::FuturesOrdered; + + let pipeline_start = Instant::now(); + let mut in_flight = FuturesOrdered::new(); + let mut task_iter = tasks.into_iter(); + + // Seed the first N file tasks. + for _ in 0..concurrency { + if let Some(task) = task_iter.next() { + in_flight.push_back(spawn_file_task(task, file_io.clone(), batch_size)); + } + } + + // FuturesOrdered::next() yields results in push order (file 0, 1, 2, ...). + while let Some(file_result) = in_flight.next().await { + // Eagerly start the next file to keep N tasks in flight. + if let Some(task) = task_iter.next() { + in_flight.push_back(spawn_file_task(task, file_io.clone(), batch_size)); + } + + let mut file_rx = match file_result { + Ok(rx) => rx, + Err(e) => { + let _ = tx.send(Err(e)).await; + return; + } + }; + + // Drain this file's batches in row order, forwarding to Julia. + while let Some(batch_result) = file_rx.recv().await { + match batch_result { + Ok(buf) => { + let sem = buf.semaphore.clone(); + let byte_len = buf.byte_len; + if tx.send(Ok(buf.batch)).await.is_err() { + return; // Julia side dropped the stream + } + // Release permits so the file task can produce more. + sem.add_permits(byte_len); + STATS.track_buffer_release(byte_len as u64); + } + Err(e) => { + let _ = tx.send(Err(e)).await; + return; + } + } + } + // file_rx exhausted → file task finished → move to next file + } + + STATS + .pipeline_wall_ns + .store(pipeline_start.elapsed().as_nanos() as u64, Ordering::Relaxed); +} + +/// Wrapper around process_file_inner that ensures errors are sent to the +/// channel and stats are updated even on failure. When this function +/// returns, `tx` is dropped, causing the consumer's `file_rx.recv()` to +/// return None — signaling that this file is done. +async fn process_file( + task: FileScanTask, + file_io: FileIO, + batch_size: Option, + semaphore: Arc, + tx: mpsc::Sender>, +) { + let result = process_file_inner(task, file_io, batch_size, semaphore, &tx).await; + if let Err(e) = result { + let _ = tx.send(Err(e)).await; + } + STATS.track_task_end(); +} + +/// Process a single parquet file through four timed phases: +/// 1. Reader setup — build ArrowReader, open file, resolve schema +/// 2. Fetch + decode — I/O, ZSTD decompression, column assembly +/// 3. Serialize — RecordBatch → Arrow IPC for transfer to Julia +/// 4. Backpressure — wait on semaphore if buffered too far ahead +/// +/// Each phase's cumulative time is recorded in STATS for the summary. +async fn process_file_inner( + task: FileScanTask, + file_io: FileIO, + batch_size: Option, + semaphore: Arc, + tx: &mpsc::Sender>, +) -> Result<(), iceberg::Error> { + STATS.track_task_start(); + + // ── Phase 1: Reader setup ─────────────────────────────────────────── + // Builds a per-file ArrowReader using the same iceberg-rs code path as + // to_arrow(): opens parquet metadata, resolves schema, loads delete + // files, builds row filters. with_data_file_concurrency_limit(1) means + // this reader processes one file (the one we give it). + let setup_start = Instant::now(); + let mut builder = ArrowReaderBuilder::new(file_io).with_data_file_concurrency_limit(1); + if let Some(bs) = batch_size { + builder = builder.with_batch_size(bs); + } + let reader = builder.build(); + let task_stream = Box::pin(futures::stream::once(async { Ok(task) })); + let batch_stream = reader + .read(task_stream) + .map_err(|e| iceberg::Error::new(iceberg::ErrorKind::Unexpected, e.to_string()))?; + STATS + .reader_setup_ns + .fetch_add(setup_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + + tokio::pin!(batch_stream); + + loop { + // ── Phase 2: Fetch + decode ───────────────────────────────────── + // Each .next() call fetches compressed parquet pages from storage, + // decompresses (ZSTD), decodes column encodings, and assembles a + // RecordBatch. These are inseparable without forking parquet-rs. + let fd_start = Instant::now(); + let batch_opt = batch_stream.next().await; + STATS + .fetch_decode_ns + .fetch_add(fd_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + + let batch = match batch_opt { + Some(Ok(b)) => b, + Some(Err(e)) => return Err(e), + None => break, // end of file + }; + + // ── Phase 3: Serialize to Arrow IPC ───────────────────────────── + // CPU-bound work, offloaded to the blocking thread pool to avoid + // starving the tokio executor. + let ser_start = Instant::now(); + let serialized = tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch)) + .await + .map_err(|e| { + iceberg::Error::new( + iceberg::ErrorKind::Unexpected, + format!("serialize panicked: {e}"), + ) + })? + .map_err(|e| iceberg::Error::new(iceberg::ErrorKind::Unexpected, e.to_string()))?; + STATS + .serialize_ns + .fetch_add(ser_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + + let byte_len = serialized.length; + STATS.batches_produced.fetch_add(1, Ordering::Relaxed); + STATS + .bytes_produced + .fetch_add(byte_len as u64, Ordering::Relaxed); + + // ── Phase 4: Backpressure ─────────────────────────────────────── + // Acquire permits equal to the serialized size. If this file task + // has produced > MAX_BUFFERED_BYTES_PER_TASK ahead of the consumer, + // this yields (async, not thread-blocking) until permits are freed. + let sem_start = Instant::now(); + let _permit = semaphore + .acquire_many(byte_len as u32) + .await + .map_err(|e| { + iceberg::Error::new( + iceberg::ErrorKind::Unexpected, + format!("semaphore: {e}"), + ) + })?; + STATS + .semaphore_wait_ns + .fetch_add(sem_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + // Detach the permit — the consumer releases it via add_permits(). + std::mem::forget(_permit); + + STATS.track_buffer_add(byte_len as u64); + + // Send the batch to this file's channel. + if tx + .send(Ok(BufferedBatch { + batch: serialized, + byte_len, + semaphore: semaphore.clone(), + })) + .await + .is_err() + { + return Ok(()); // consumer dropped the receiver + } + } + + STATS.files_completed.fetch_add(1, Ordering::Relaxed); + Ok(()) +} diff --git a/iceberg_rust_ffi/src/scan_common.rs b/iceberg_rust_ffi/src/scan_common.rs index daac382..f8c6737 100644 --- a/iceberg_rust_ffi/src/scan_common.rs +++ b/iceberg_rust_ffi/src/scan_common.rs @@ -38,6 +38,9 @@ macro_rules! impl_select_columns { builder: scan_ref.builder.map(|b| b.select(columns)), scan: scan_ref.scan, serialization_concurrency: scan_ref.serialization_concurrency, + file_io: scan_ref.file_io, + batch_size: scan_ref.batch_size, + file_concurrency: scan_ref.file_concurrency, })); CResult::Ok @@ -84,6 +87,9 @@ macro_rules! impl_scan_builder_method { builder: scan_ref.builder.map(|b| b.$builder_method($($param),*)), scan: scan_ref.scan, serialization_concurrency: scan_ref.serialization_concurrency, + file_io: scan_ref.file_io, + batch_size: scan_ref.batch_size, + file_concurrency: scan_ref.file_concurrency, })); CResult::Ok @@ -111,6 +117,9 @@ macro_rules! impl_with_batch_size { builder: scan_ref.builder.map(|b| b.with_batch_size(Some(n))), scan: None, serialization_concurrency: scan_ref.serialization_concurrency, + file_io: scan_ref.file_io, + batch_size: Some(n), + file_concurrency: scan_ref.file_concurrency, })); CResult::Ok @@ -138,6 +147,9 @@ macro_rules! impl_scan_build { builder: None, scan: Some(built_scan), serialization_concurrency: scan_ref.serialization_concurrency, + file_io: scan_ref.file_io, + batch_size: scan_ref.batch_size, + file_concurrency: scan_ref.file_concurrency, })); CResult::Ok } From 9d81d4217dbc242711889e02659f4f554abf6eb4 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 05:49:03 +0000 Subject: [PATCH 03/13] Add nested file-scan pipeline (stream of per-file streams) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Exposes a new iceberg_file_scan_stream / scan_nested! API that yields one IcebergFileScan per file, each carrying the filename, record count, and a prefetched inner IcebergArrowStream of Arrow IPC batches. The existing flat scan! path is kept by re-implementing it as a thin flattening wrapper over the same nested pipeline, so all ordering, backpressure, and stats logic lives in one place. Key changes: - ordered_file_pipeline: add FileScan (internal type), make_file_stream, spawn_file_task_with_meta, run_nested / create_nested_pipeline; rewrite create_pipeline to flatten the nested pipeline. Switch run_nested from FuturesOrdered to FuturesUnordered (spawn futures resolve immediately so ordering is irrelevant here). - table.rs: add IcebergFileScan (#[repr(C)]), IcebergFileScanStream, IcebergFileScanResponse; add iceberg_file_scan_free, iceberg_file_scan_stream_free, iceberg_file_scan_record_count, iceberg_file_scan_filename (sync getters), iceberg_next_file_scan (async). - full.rs: add iceberg_file_scan_stream export op; make IcebergScan.file_io non-optional (FileIO instead of Option) — always set at construction, use .clone() in stream ops instead of .take(). - Julia: scan_nested!, nested_arrow_stream, next_file_scan, file_scan_record_count, file_scan_filename, file_scan_arrow_stream, free_file_scan!, free_file_scan_stream! exported from RustyIceberg. Labels: dismiss-release-notes build:benchmark Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/full.rs | 67 +++++- iceberg_rust_ffi/src/lib.rs | 5 +- iceberg_rust_ffi/src/ordered_file_pipeline.rs | 218 ++++++++++++------ iceberg_rust_ffi/src/table.rs | 133 +++++++++++ src/RustyIceberg.jl | 3 + src/full.jl | 129 +++++++++++ 6 files changed, 473 insertions(+), 82 deletions(-) diff --git a/iceberg_rust_ffi/src/full.rs b/iceberg_rust_ffi/src/full.rs index 0324610..42b8707 100644 --- a/iceberg_rust_ffi/src/full.rs +++ b/iceberg_rust_ffi/src/full.rs @@ -7,7 +7,10 @@ use object_store_ffi::{ export_runtime_op, with_cancellation, CResult, NotifyGuard, ResponseGuard, RT, }; use crate::scan_common::*; -use crate::{IcebergArrowStream, IcebergArrowStreamResponse, IcebergTable}; +use crate::{ + IcebergArrowStream, IcebergArrowStreamResponse, IcebergFileScanStream, + IcebergFileScanStreamResponse, IcebergTable, +}; /// Holds state for a full table scan across its lifecycle: /// 1. Construction: `builder` is set, everything else is None/0. @@ -24,7 +27,7 @@ pub struct IcebergScan { pub serialization_concurrency: usize, /// Cloned from the Table at construction time. Passed to the pipeline so /// each per-file ArrowReader can open its own parquet file. - pub file_io: Option, + pub file_io: FileIO, /// Captured when Julia calls with_batch_size. Forwarded to each per-file /// ArrowReaderBuilder inside the pipeline. pub batch_size: Option, @@ -49,7 +52,7 @@ pub extern "C" fn iceberg_new_scan(table: *mut IcebergTable) -> *mut IcebergScan builder: Some(scan_builder), scan: None, serialization_concurrency: 0, - file_io: Some(file_io), + file_io, batch_size: None, file_concurrency: 0, })) @@ -154,11 +157,7 @@ export_runtime_op!( concurrency }; - // Take file_io — it's moved into the pipeline (one clone per file task). - let file_io = scan_ptr - .file_io - .take() - .ok_or_else(|| anyhow::anyhow!("file_io not available"))?; + let file_io = scan_ptr.file_io.clone(); let batch_size = scan_ptr.batch_size; Ok((scan_ref.as_ref().unwrap(), concurrency, file_io, batch_size)) @@ -184,3 +183,55 @@ export_runtime_op!( ); impl_scan_free!(iceberg_scan_free, IcebergScan); + +// ── Nested stream creation ─────────────────────────────────────────────── +// +// Returns an IcebergFileScanStream whose items are per-file (filename, +// record_count, inner-batch-stream) tuples, yielded in strict file order. +// The flat iceberg_arrow_stream is implemented as a flattening wrapper over +// this same nested pipeline. + +export_runtime_op!( + iceberg_file_scan_stream, + IcebergFileScanStreamResponse, + || { + if scan.is_null() { + return Err(anyhow::anyhow!("Null scan pointer provided")); + } + let scan_ptr = unsafe { &mut *scan }; + let scan_ref = &scan_ptr.scan; + if scan_ref.is_none() { + return Err(anyhow::anyhow!("Scan not initialized")); + } + + let concurrency = scan_ptr.file_concurrency; + let concurrency = if concurrency == 0 { + std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1) + } else { + concurrency + }; + + let file_io = scan_ptr.file_io.clone(); + let batch_size = scan_ptr.batch_size; + + Ok((scan_ref.as_ref().unwrap(), concurrency, file_io, batch_size)) + }, + result_tuple, + async { + let (scan_ref, concurrency, file_io, batch_size) = result_tuple; + + use futures::TryStreamExt; + let tasks: Vec = + scan_ref.plan_files().await?.try_collect().await?; + + let stream = crate::ordered_file_pipeline::create_nested_pipeline( + tasks, file_io, batch_size, concurrency, + ) + .await?; + + Ok::(stream) + }, + scan: *mut IcebergScan +); diff --git a/iceberg_rust_ffi/src/lib.rs b/iceberg_rust_ffi/src/lib.rs index be2dd10..45775a5 100644 --- a/iceberg_rust_ffi/src/lib.rs +++ b/iceberg_rust_ffi/src/lib.rs @@ -49,8 +49,9 @@ pub use response::{ IcebergStringListResponse, }; pub use table::{ - ArrowBatch, IcebergArrowStream, IcebergArrowStreamResponse, IcebergBatchResponse, IcebergTable, - IcebergTableResponse, + ArrowBatch, IcebergArrowStream, IcebergArrowStreamResponse, IcebergBatchResponse, + IcebergFileScan, IcebergFileScanResponse, IcebergFileScanStream, IcebergFileScanStreamResponse, + IcebergTable, IcebergTableResponse, }; pub use transaction::{IcebergDataFiles, IcebergTransaction, IcebergTransactionResponse}; pub use writer::{ diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index d091b63..364550a 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -15,10 +15,10 @@ //! 3. Serializes each batch to Arrow IPC (via spawn_blocking) //! 4. Sends serialized batches into a per-file mpsc channel //! -//! A consumer (`run`) uses FuturesOrdered to drain files in order: -//! - FuturesOrdered yields per-file receivers in the order files were pushed -//! - The consumer drains file 0's channel batch-by-batch, then file 1's, etc. -//! - Each drained batch is forwarded to the outer channel (read by Julia) +//! A consumer (`run_nested`) uses FuturesUnordered to maintain N tasks in +//! flight and yield per-file FileScan values as they become ready. Each +//! FileScan wraps the file's batch channel as an IcebergArrowStream with +//! integrated semaphore-permit release. //! //! # Memory bounding //! Each file task has its own Semaphore(MAX_BUFFERED_BYTES_PER_TASK). After @@ -37,7 +37,7 @@ use iceberg::io::FileIO; use iceberg::scan::FileScanTask; use tokio::sync::{mpsc, Mutex as AsyncMutex, Semaphore}; -use crate::table::{ArrowBatch, IcebergArrowStream}; +use crate::table::{ArrowBatch, IcebergArrowStream, IcebergFileScanStream}; /// Per-file cap on serialized bytes buffered ahead of the consumer. const MAX_BUFFERED_BYTES_PER_TASK: usize = 100 * 1024 * 1024; @@ -79,8 +79,7 @@ struct PipelineStats { semaphore_wait_ns: AtomicU64, // ── Consumer ── - /// Time the consumer spends waiting on FuturesOrdered for the next - /// file's receiver (ordering stall — head file not ready yet). + /// Time the flat consumer (`run_flat`) spends waiting for the next file. consumer_wait_ns: AtomicU64, // ── Memory ── @@ -239,17 +238,44 @@ struct BufferedBatch { semaphore: Arc, } -/// Create the file-parallel pipeline and return it as an IcebergArrowStream. -/// -/// Spawns a background `run` task that processes files concurrently and -/// feeds serialized batches into an mpsc channel. The returned stream -/// wraps the receiving end of that channel. -pub async fn create_pipeline( +/// Internal per-file scan result: filename, record count, and a prefetched +/// inner batch stream. Used as the item type of the nested pipeline channel. +pub struct FileScan { + pub filename: String, + pub record_count: i64, + pub stream: IcebergArrowStream, +} + +/// Convert a per-file mpsc receiver into an IcebergArrowStream. +/// Semaphore permits are released and STATS updated as each batch is yielded. +fn make_file_stream( + file_rx: mpsc::Receiver>, +) -> IcebergArrowStream { + let stream = futures::stream::unfold(file_rx, |mut rx| async move { + rx.recv().await.map(|item| { + let result = item.map(|buf| { + buf.semaphore.add_permits(buf.byte_len); + STATS.track_buffer_release(buf.byte_len as u64); + buf.batch + }); + (result, rx) + }) + }) + .boxed(); + IcebergArrowStream { + stream: AsyncMutex::new(stream), + } +} + +/// Create the nested file-parallel pipeline and return it as an +/// IcebergFileScanStream. Each item in the outer stream is a FileScan +/// carrying the filename, record count, and a prefetched inner batch stream. +pub async fn create_nested_pipeline( tasks: Vec, file_io: FileIO, batch_size: Option, concurrency: usize, -) -> anyhow::Result { +) -> anyhow::Result { assert!( concurrency <= MAX_FILE_CONCURRENCY, "file concurrency {concurrency} exceeds hard cap {MAX_FILE_CONCURRENCY}" @@ -257,110 +283,158 @@ pub async fn create_pipeline( STATS.reset(); - // Outer channel: run() → Julia (via IcebergArrowStream). - let (tx, rx) = mpsc::channel(concurrency * 2); + let (tx, rx) = mpsc::channel::>(concurrency); - tokio::spawn(run(tasks, file_io, batch_size, concurrency, tx)); + tokio::spawn(run_nested(tasks, file_io, batch_size, concurrency, tx)); - // Wrap the mpsc receiver as a BoxStream for IcebergArrowStream. let stream = futures::stream::unfold(rx, |mut rx| async move { rx.recv().await.map(|item| (item, rx)) }) .boxed(); - Ok(IcebergArrowStream { + Ok(IcebergFileScanStream { stream: AsyncMutex::new(stream), }) } -/// Spawn a single file task. Returns a future that resolves immediately -/// to the receiving end of the file's batch channel. +/// Create the flat file-parallel pipeline and return it as an IcebergArrowStream. /// -/// The actual work (parquet I/O, decode, serialize) happens in the -/// background tokio task. The future resolves to the Receiver so that -/// FuturesOrdered can yield receivers in file order. -fn spawn_file_task( - task: FileScanTask, +/// Implemented by creating the nested pipeline and flattening it, so all +/// ordering, backpressure, and stats logic lives in one place. +pub async fn create_pipeline( + tasks: Vec, file_io: FileIO, batch_size: Option, -) -> impl std::future::Future< - Output = Result>, iceberg::Error>, -> { - // Each file gets its own semaphore for independent backpressure. - let sem = Arc::new(Semaphore::new(MAX_BUFFERED_BYTES_PER_TASK)); - let (file_tx, file_rx) = mpsc::channel(8); + concurrency: usize, +) -> anyhow::Result { + assert!( + concurrency <= MAX_FILE_CONCURRENCY, + "file concurrency {concurrency} exceeds hard cap {MAX_FILE_CONCURRENCY}" + ); - tokio::spawn(process_file(task, file_io, batch_size, sem, file_tx)); + // Outer channel: flatten task → Julia (via IcebergArrowStream). + let (tx, rx) = mpsc::channel(concurrency * 2); - async move { Ok(file_rx) } + let nested = create_nested_pipeline(tasks, file_io, batch_size, concurrency).await?; + + tokio::spawn(run_flat(nested, tx)); + + let stream = futures::stream::unfold(rx, |mut rx| async move { + rx.recv().await.map(|item| (item, rx)) + }) + .boxed(); + + Ok(IcebergArrowStream { + stream: AsyncMutex::new(stream), + }) } -/// Main consumer loop. Orchestrates file-level parallelism while +/// Flatten the nested pipeline into a single batch channel (backwards-compat). +async fn run_flat( + nested: IcebergFileScanStream, + tx: mpsc::Sender>, +) { + let pipeline_start = Instant::now(); + + let mut outer = nested.stream.lock().await; + while let Some(item) = outer.next().await { + match item { + Ok(file_scan) => { + let mut inner = file_scan.stream.stream.lock().await; + while let Some(batch_result) = inner.next().await { + if tx.send(batch_result).await.is_err() { + return; + } + } + } + Err(e) => { + let _ = tx.send(Err(e)).await; + return; + } + } + } + + STATS + .pipeline_wall_ns + .store(pipeline_start.elapsed().as_nanos() as u64, Ordering::Relaxed); +} + +/// Main nested consumer loop. Orchestrates file-level parallelism while /// maintaining strict file ordering. /// /// Uses FuturesOrdered to poll N file tasks concurrently but yield their -/// receivers in push order. For each file, drains its channel batch-by-batch, -/// forwarding to the outer channel and releasing semaphore permits. -async fn run( +/// (filename, record_count, receiver) tuples in push order. Wraps each +/// receiver as an IcebergArrowStream and sends a FileScan to the outer channel. +async fn run_nested( tasks: Vec, file_io: FileIO, batch_size: Option, concurrency: usize, - tx: mpsc::Sender>, + tx: mpsc::Sender>, ) { - use futures::stream::FuturesOrdered; + use futures::stream::FuturesUnordered; - let pipeline_start = Instant::now(); - let mut in_flight = FuturesOrdered::new(); + let mut in_flight = FuturesUnordered::new(); let mut task_iter = tasks.into_iter(); // Seed the first N file tasks. for _ in 0..concurrency { if let Some(task) = task_iter.next() { - in_flight.push_back(spawn_file_task(task, file_io.clone(), batch_size)); + in_flight.push(spawn_file_task_with_meta(task, file_io.clone(), batch_size)); } } - // FuturesOrdered::next() yields results in push order (file 0, 1, 2, ...). while let Some(file_result) = in_flight.next().await { // Eagerly start the next file to keep N tasks in flight. if let Some(task) = task_iter.next() { - in_flight.push_back(spawn_file_task(task, file_io.clone(), batch_size)); + in_flight.push(spawn_file_task_with_meta(task, file_io.clone(), batch_size)); } - let mut file_rx = match file_result { - Ok(rx) => rx, + match file_result { + Ok((filename, record_count, file_rx)) => { + let stream = make_file_stream(file_rx); + let file_scan = FileScan { + filename, + record_count, + stream, + }; + if tx.send(Ok(file_scan)).await.is_err() { + return; // outer consumer dropped the stream + } + } Err(e) => { let _ = tx.send(Err(e)).await; return; } - }; - - // Drain this file's batches in row order, forwarding to Julia. - while let Some(batch_result) = file_rx.recv().await { - match batch_result { - Ok(buf) => { - let sem = buf.semaphore.clone(); - let byte_len = buf.byte_len; - if tx.send(Ok(buf.batch)).await.is_err() { - return; // Julia side dropped the stream - } - // Release permits so the file task can produce more. - sem.add_permits(byte_len); - STATS.track_buffer_release(byte_len as u64); - } - Err(e) => { - let _ = tx.send(Err(e)).await; - return; - } - } } - // file_rx exhausted → file task finished → move to next file } +} - STATS - .pipeline_wall_ns - .store(pipeline_start.elapsed().as_nanos() as u64, Ordering::Relaxed); +/// Spawn a single file task. Returns a future that resolves immediately to +/// the file's metadata and the receiving end of its batch channel. +/// +/// The actual work (parquet I/O, decode, serialize) happens in the background +/// tokio task. The future resolves to (filename, record_count, Receiver) so +/// that FuturesOrdered can yield them in file order. +fn spawn_file_task_with_meta( + task: FileScanTask, + file_io: FileIO, + batch_size: Option, +) -> impl std::future::Future< + Output = Result< + (String, i64, mpsc::Receiver>), + iceberg::Error, + >, +> { + let filename = task.data_file_path().to_string(); + let record_count = task.record_count.unwrap_or(0) as i64; + + let sem = Arc::new(Semaphore::new(MAX_BUFFERED_BYTES_PER_TASK)); + let (file_tx, file_rx) = mpsc::channel(8); + + tokio::spawn(process_file(task, file_io, batch_size, sem, file_tx)); + + async move { Ok((filename, record_count, file_rx)) } } /// Wrapper around process_file_inner that ensures errors are sent to the @@ -374,6 +448,7 @@ async fn process_file( semaphore: Arc, tx: mpsc::Sender>, ) { + STATS.track_task_start(); let result = process_file_inner(task, file_io, batch_size, semaphore, &tx).await; if let Err(e) = result { let _ = tx.send(Err(e)).await; @@ -395,7 +470,6 @@ async fn process_file_inner( semaphore: Arc, tx: &mpsc::Sender>, ) -> Result<(), iceberg::Error> { - STATS.track_task_start(); // ── Phase 1: Reader setup ─────────────────────────────────────────── // Builds a per-file ArrowReader using the same iceberg-rs code path as diff --git a/iceberg_rust_ffi/src/table.rs b/iceberg_rust_ffi/src/table.rs index 2e089a8..e6175f0 100644 --- a/iceberg_rust_ffi/src/table.rs +++ b/iceberg_rust_ffi/src/table.rs @@ -1,6 +1,7 @@ use crate::response::IcebergBoxedResponse; /// Table and streaming support for iceberg_rust_ffi use crate::{CResult, Context, RawResponse}; +use crate::ordered_file_pipeline::FileScan; use iceberg::io::{FileIOBuilder, OpenDalRoutingStorageFactory}; use iceberg::table::StaticTable; use iceberg::table::Table; @@ -77,6 +78,67 @@ impl RawResponse for IcebergBatchResponse { } } +/// Outer stream of per-file scans from the nested pipeline. +pub struct IcebergFileScanStream { + pub stream: AsyncMutex< + futures::stream::BoxStream<'static, Result>, + >, +} + +unsafe impl Send for IcebergFileScanStream {} + +/// C-compatible per-file scan item returned to Julia. +/// Owns `filename` (must be freed via iceberg_file_scan_free) and `stream`. +#[repr(C)] +pub struct IcebergFileScan { + /// Null-terminated file path. Owned; freed by iceberg_file_scan_free. + pub filename: *mut c_char, + pub record_count: i64, + /// Inner batch stream. Owned; freed by iceberg_file_scan_free. + /// Callers must NOT call iceberg_arrow_stream_free on this pointer. + pub stream: *mut IcebergArrowStream, +} + +unsafe impl Send for IcebergFileScan {} + +pub type IcebergFileScanStreamResponse = IcebergBoxedResponse; + +/// Response for iceberg_next_file_scan (mirrors IcebergBatchResponse). +#[repr(transparent)] +pub struct IcebergFileScanResponse(pub IcebergBoxedResponse); + +unsafe impl Send for IcebergFileScanResponse {} + +impl RawResponse for IcebergFileScanResponse { + type Payload = Option; + + fn result_mut(&mut self) -> &mut CResult { + &mut self.0.result + } + fn context_mut(&mut self) -> &mut *const Context { + &mut self.0.context + } + fn error_message_mut(&mut self) -> &mut *mut c_char { + &mut self.0.error_message + } + fn set_payload(&mut self, payload: Option) { + match payload.flatten() { + Some(fs) => { + let filename = std::ffi::CString::new(fs.filename) + .unwrap_or_default() + .into_raw(); + let stream = Box::into_raw(Box::new(fs.stream)); + self.0.value = Box::into_raw(Box::new(IcebergFileScan { + filename, + record_count: fs.record_count, + stream, + })); + } + None => self.0.value = ptr::null_mut(), + } + } +} + /// Synchronous operations for table and batch management /// Free a table @@ -114,6 +176,77 @@ pub extern "C" fn iceberg_arrow_stream_free(stream: *mut IcebergArrowStream) { } } +/// Free a file scan (its owned filename and inner stream). +#[no_mangle] +pub extern "C" fn iceberg_file_scan_free(scan: *mut IcebergFileScan) { + if scan.is_null() { + return; + } + unsafe { + let scan = Box::from_raw(scan); + if !scan.filename.is_null() { + let _ = std::ffi::CString::from_raw(scan.filename); + } + if !scan.stream.is_null() { + let _ = Box::from_raw(scan.stream); + } + } +} + +/// Free a file scan stream. +#[no_mangle] +pub extern "C" fn iceberg_file_scan_stream_free(stream: *mut IcebergFileScanStream) { + if !stream.is_null() { + unsafe { + let _ = Box::from_raw(stream); + } + } +} + +/// Return the record count of a file scan. Returns -1 on null input. +#[no_mangle] +pub extern "C" fn iceberg_file_scan_record_count(scan: *const IcebergFileScan) -> i64 { + if scan.is_null() { + return -1; + } + unsafe { (*scan).record_count } +} + +/// Return a borrowed pointer to the null-terminated filename of a file scan. +/// The pointer is valid for the lifetime of the IcebergFileScan. +/// The caller must NOT free this pointer; iceberg_file_scan_free handles it. +#[no_mangle] +pub extern "C" fn iceberg_file_scan_filename(scan: *const IcebergFileScan) -> *const c_char { + if scan.is_null() { + return ptr::null(); + } + unsafe { (*scan).filename } +} + +// Get next file scan from outer stream (async) +export_runtime_op!( + iceberg_next_file_scan, + IcebergFileScanResponse, + || { + if stream.is_null() { + return Err(anyhow::anyhow!("Null file scan stream pointer")); + } + let stream_ref = unsafe { &*stream }; + Ok(stream_ref) + }, + stream_ref, + async { + use futures::StreamExt; + let mut guard = stream_ref.stream.lock().await; + match guard.next().await { + Some(Ok(fs)) => Ok(Some(fs)), + Some(Err(e)) => Err(anyhow::anyhow!("Error reading file scan: {}", e)), + None => Ok(None), + } + }, + stream: *mut IcebergFileScanStream +); + // FFI Export functions for table operations // These functions are exported to be called from Julia via the FFI diff --git a/src/RustyIceberg.jl b/src/RustyIceberg.jl index 2179f5a..881628d 100644 --- a/src/RustyIceberg.jl +++ b/src/RustyIceberg.jl @@ -17,6 +17,9 @@ export table_location, table_uuid, table_format_version, table_last_sequence_num export select_columns!, with_batch_size!, with_data_file_concurrency_limit!, with_manifest_entry_concurrency_limit! export with_file_column!, with_pos_column! export scan!, next_batch, free_batch, free_stream +export scan_nested!, nested_arrow_stream, next_file_scan +export FileScanStream, file_scan_record_count, file_scan_filename, file_scan_arrow_stream +export free_file_scan!, free_file_scan_stream! export FILE_COLUMN, POS_COLUMN export Catalog, catalog_create_rest, free_catalog export load_table, list_tables, list_namespaces, table_exists, create_table, drop_table, drop_namespace, create_namespace diff --git a/src/full.jl b/src/full.jl index 9ac088b..e34c31c 100644 --- a/src/full.jl +++ b/src/full.jl @@ -297,3 +297,132 @@ function free_scan!(scan::Scan) convert(Ptr{Ptr{Cvoid}}, pointer_from_objref(scan))::Ptr{Ptr{Cvoid}} )::Cvoid end + +# ── Nested file-scan pipeline ────────────────────────────────────────────── +# +# Returns a FileScanStream (outer stream). Each item yielded by next_file_scan +# is an opaque FileScan pointer carrying the filename, record count, and an +# inner IcebergArrowStream of prefetched batches. +# +# Ownership rules: +# - Call free_file_scan! after processing each FileScan. +# - Do NOT call free_stream on the pointer returned by file_scan_arrow_stream; +# iceberg_file_scan_free handles that. +# - Call free_file_scan_stream! after the outer loop completes. + +"""Opaque pointer type for an outer file-scan stream (IcebergFileScanStream).""" +const FileScanStream = Ptr{Cvoid} + +const FileScanStreamResponse = Response{FileScanStream} +Response{FileScanStream}() = Response{FileScanStream}(-1, C_NULL, C_NULL, C_NULL) + +""" + nested_arrow_stream(scan::Scan)::FileScanStream + +Initialize a nested Arrow stream for the scan asynchronously. +Returns an outer FileScanStream whose items are per-file scans. +""" +function nested_arrow_stream(scan::Scan) + response = FileScanStreamResponse() + + async_ccall(response) do handle + @ccall rust_lib.iceberg_file_scan_stream( + scan.ptr::Ptr{Cvoid}, + response::Ref{FileScanStreamResponse}, + handle::Ptr{Cvoid} + )::Cint + end + + @throw_on_error(response, "iceberg_file_scan_stream", IcebergException) + + return response.value +end + +""" + scan_nested!(scan::Scan)::FileScanStream + +Build the scan and return a nested FileScanStream (one item per file). +""" +function scan_nested!(scan::Scan) + build!(scan) + return nested_arrow_stream(scan) +end + +# Response type for next_file_scan +const FileScanResponse = Response{Ptr{Cvoid}} +Response{Ptr{Cvoid}}() = Response{Ptr{Cvoid}}(-1, C_NULL, C_NULL, C_NULL) + +""" + next_file_scan(stream::FileScanStream)::Ptr{Cvoid} + +Wait for the next per-file scan from the outer stream asynchronously. +Returns C_NULL when the stream is exhausted. +""" +function next_file_scan(stream::FileScanStream) + response = FileScanResponse() + + async_ccall(response) do handle + @ccall rust_lib.iceberg_next_file_scan( + stream::FileScanStream, + response::Ref{FileScanResponse}, + handle::Ptr{Cvoid} + )::Cint + end + + @throw_on_error(response, "iceberg_next_file_scan", IcebergException) + + return response.value +end + +""" + file_scan_record_count(fs::Ptr{Cvoid})::Int64 + +Return the record count of the file scan. Returns -1 on null input. +""" +function file_scan_record_count(fs::Ptr{Cvoid})::Int64 + return @ccall rust_lib.iceberg_file_scan_record_count(fs::Ptr{Cvoid})::Int64 +end + +""" + file_scan_filename(fs::Ptr{Cvoid})::String + +Return the file path of the file scan as a Julia String. +""" +function file_scan_filename(fs::Ptr{Cvoid})::String + ptr = @ccall rust_lib.iceberg_file_scan_filename(fs::Ptr{Cvoid})::Ptr{Cchar} + ptr == C_NULL && return "" + return unsafe_string(ptr) +end + +""" + file_scan_arrow_stream(fs::Ptr{Cvoid})::ArrowStream + +Return the inner batch stream of the file scan (borrowed pointer). +Do NOT call free_stream on the returned pointer; use free_file_scan! instead. +""" +function file_scan_arrow_stream(fs::Ptr{Cvoid})::ArrowStream + # IcebergFileScan layout (repr(C)): + # filename: *mut c_char (offset 0, 8 bytes) + # record_count: i64 (offset 8, 8 bytes) + # stream: *mut IcebergArrowStream (offset 16, 8 bytes) + offset = sizeof(Ptr{Cvoid}) + sizeof(Int64) + return unsafe_load(convert(Ptr{Ptr{Cvoid}}, fs + offset)) +end + +""" + free_file_scan!(fs::Ptr{Cvoid}) + +Free the memory associated with a file scan (filename and inner stream). +""" +function free_file_scan!(fs::Ptr{Cvoid}) + @ccall rust_lib.iceberg_file_scan_free(fs::Ptr{Cvoid})::Cvoid +end + +""" + free_file_scan_stream!(stream::FileScanStream) + +Free the memory associated with a file scan stream. +""" +function free_file_scan_stream!(stream::FileScanStream) + @ccall rust_lib.iceberg_file_scan_stream_free(stream::FileScanStream)::Cvoid +end From 0fa63518e8be8db5d900b84420a9f3980f6d0ddf Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 05:54:34 +0000 Subject: [PATCH 04/13] Extract unexpected() helper for iceberg error construction Replace all inline iceberg::Error::new(iceberg::ErrorKind::Unexpected, ...) callsites with a crate-wide pub(crate) helper in lib.rs, removing the boilerplate from catalog.rs, lib.rs, and ordered_file_pipeline.rs. Labels: dismiss-release-notes build:benchmark Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/catalog.rs | 8 +++----- iceberg_rust_ffi/src/lib.rs | 19 +++++++------------ iceberg_rust_ffi/src/ordered_file_pipeline.rs | 19 +++++-------------- 3 files changed, 15 insertions(+), 31 deletions(-) diff --git a/iceberg_rust_ffi/src/catalog.rs b/iceberg_rust_ffi/src/catalog.rs index 9e4f102..48b216b 100644 --- a/iceberg_rust_ffi/src/catalog.rs +++ b/iceberg_rust_ffi/src/catalog.rs @@ -3,7 +3,7 @@ use crate::response::{ IcebergStringListResponse, }; /// Catalog support for iceberg_rust_ffi -use crate::IcebergTable; +use crate::{unexpected, IcebergTable}; use anyhow::Result; use async_trait::async_trait; use iceberg::io::{ @@ -155,16 +155,14 @@ impl StorageCredentialsLoader for RestCredentialsLoader { .catalog .get() .and_then(|w| w.upgrade()) - .ok_or_else(|| { - Error::new(ErrorKind::Unexpected, "Catalog reference is not available") - })?; + .ok_or_else(|| unexpected("Catalog reference is not available"))?; let response = catalog.load_table_credentials(table_ident).await?; response .storage_credentials .into_iter() .filter(|c| location.starts_with(&c.prefix)) .max_by_key(|c| c.prefix.len()) - .ok_or_else(|| Error::new(ErrorKind::Unexpected, "No matching credential for location")) + .ok_or_else(|| unexpected("No matching credential for location")) } } diff --git a/iceberg_rust_ffi/src/lib.rs b/iceberg_rust_ffi/src/lib.rs index 45775a5..02269d3 100644 --- a/iceberg_rust_ffi/src/lib.rs +++ b/iceberg_rust_ffi/src/lib.rs @@ -1,6 +1,10 @@ use futures::StreamExt; use std::ffi::{c_char, c_void}; +pub(crate) fn unexpected(msg: impl std::fmt::Display) -> iceberg::Error { + iceberg::Error::new(iceberg::ErrorKind::Unexpected, msg.to_string()) +} + use anyhow::Result; use arrow_array::RecordBatch; use arrow_ipc::writer::StreamWriter; @@ -162,20 +166,11 @@ pub(crate) fn transform_stream_with_parallel_serialization( .await { Ok(Ok(arrow_batch)) => Ok(arrow_batch), - Ok(Err(e)) => Err(iceberg::Error::new( - iceberg::ErrorKind::Unexpected, - e.to_string(), - )), - Err(e) => Err(iceberg::Error::new( - iceberg::ErrorKind::Unexpected, - format!("Serialization task panicked: {}", e), - )), + Ok(Err(e)) => Err(unexpected(e)), + Err(e) => Err(unexpected(format!("Serialization task panicked: {e}"))), } } - Err(e) => Err(iceberg::Error::new( - iceberg::ErrorKind::Unexpected, - format!("Stream error: {}", e), - )), + Err(e) => Err(unexpected(format!("Stream error: {e}"))), } }) .buffered(concurrency) diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index 364550a..70f86a3 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -38,6 +38,7 @@ use iceberg::scan::FileScanTask; use tokio::sync::{mpsc, Mutex as AsyncMutex, Semaphore}; use crate::table::{ArrowBatch, IcebergArrowStream, IcebergFileScanStream}; +use crate::unexpected; /// Per-file cap on serialized bytes buffered ahead of the consumer. const MAX_BUFFERED_BYTES_PER_TASK: usize = 100 * 1024 * 1024; @@ -485,7 +486,7 @@ async fn process_file_inner( let task_stream = Box::pin(futures::stream::once(async { Ok(task) })); let batch_stream = reader .read(task_stream) - .map_err(|e| iceberg::Error::new(iceberg::ErrorKind::Unexpected, e.to_string()))?; + .map_err(|e| unexpected(e))?; STATS .reader_setup_ns .fetch_add(setup_start.elapsed().as_nanos() as u64, Ordering::Relaxed); @@ -515,13 +516,8 @@ async fn process_file_inner( let ser_start = Instant::now(); let serialized = tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch)) .await - .map_err(|e| { - iceberg::Error::new( - iceberg::ErrorKind::Unexpected, - format!("serialize panicked: {e}"), - ) - })? - .map_err(|e| iceberg::Error::new(iceberg::ErrorKind::Unexpected, e.to_string()))?; + .map_err(|e| unexpected(format!("serialize panicked: {e}")))? + .map_err(|e| unexpected(e))?; STATS .serialize_ns .fetch_add(ser_start.elapsed().as_nanos() as u64, Ordering::Relaxed); @@ -540,12 +536,7 @@ async fn process_file_inner( let _permit = semaphore .acquire_many(byte_len as u32) .await - .map_err(|e| { - iceberg::Error::new( - iceberg::ErrorKind::Unexpected, - format!("semaphore: {e}"), - ) - })?; + .map_err(|e| unexpected(format!("semaphore: {e}")))?; STATS .semaphore_wait_ns .fetch_add(sem_start.elapsed().as_nanos() as u64, Ordering::Relaxed); From bea8bd8bb50420943c1b5695d014e7c50b11db42 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 06:09:51 +0000 Subject: [PATCH 05/13] Add timing helpers to PipelineStats Add add_elapsed, store_elapsed, and async timed methods to PipelineStats to eliminate the repeated start/record/elapsed boilerplate. The three async phases (fetch+decode, serialize, semaphore) now read as single timed() expressions; the sync reader-setup phase uses add_elapsed directly. Labels: dismiss-release-notes build:benchmark Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/ordered_file_pipeline.rs | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index 70f86a3..dea1808 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -134,6 +134,21 @@ impl PipelineStats { self.active_tasks.fetch_sub(1, Ordering::Relaxed); } + fn add_elapsed(&self, field: &AtomicU64, start: Instant) { + field.fetch_add(start.elapsed().as_nanos() as u64, Ordering::Relaxed); + } + + fn store_elapsed(&self, field: &AtomicU64, start: Instant) { + field.store(start.elapsed().as_nanos() as u64, Ordering::Relaxed); + } + + async fn timed(&self, field: &AtomicU64, fut: F) -> F::Output { + let start = Instant::now(); + let result = fut.await; + self.add_elapsed(field, start); + result + } + fn track_buffer_add(&self, bytes: u64) { let prev = self.buffered_bytes.fetch_add(bytes, Ordering::Relaxed); self.peak_buffered_bytes @@ -355,9 +370,7 @@ async fn run_flat( } } - STATS - .pipeline_wall_ns - .store(pipeline_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + STATS.store_elapsed(&STATS.pipeline_wall_ns, pipeline_start); } /// Main nested consumer loop. Orchestrates file-level parallelism while @@ -487,9 +500,7 @@ async fn process_file_inner( let batch_stream = reader .read(task_stream) .map_err(|e| unexpected(e))?; - STATS - .reader_setup_ns - .fetch_add(setup_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + STATS.add_elapsed(&STATS.reader_setup_ns, setup_start); tokio::pin!(batch_stream); @@ -498,11 +509,7 @@ async fn process_file_inner( // Each .next() call fetches compressed parquet pages from storage, // decompresses (ZSTD), decodes column encodings, and assembles a // RecordBatch. These are inseparable without forking parquet-rs. - let fd_start = Instant::now(); - let batch_opt = batch_stream.next().await; - STATS - .fetch_decode_ns - .fetch_add(fd_start.elapsed().as_nanos() as u64, Ordering::Relaxed); + let batch_opt = STATS.timed(&STATS.fetch_decode_ns, batch_stream.next()).await; let batch = match batch_opt { Some(Ok(b)) => b, @@ -513,14 +520,11 @@ async fn process_file_inner( // ── Phase 3: Serialize to Arrow IPC ───────────────────────────── // CPU-bound work, offloaded to the blocking thread pool to avoid // starving the tokio executor. - let ser_start = Instant::now(); - let serialized = tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch)) + let serialized = STATS + .timed(&STATS.serialize_ns, tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch))) .await .map_err(|e| unexpected(format!("serialize panicked: {e}")))? .map_err(|e| unexpected(e))?; - STATS - .serialize_ns - .fetch_add(ser_start.elapsed().as_nanos() as u64, Ordering::Relaxed); let byte_len = serialized.length; STATS.batches_produced.fetch_add(1, Ordering::Relaxed); @@ -532,14 +536,10 @@ async fn process_file_inner( // Acquire permits equal to the serialized size. If this file task // has produced > MAX_BUFFERED_BYTES_PER_TASK ahead of the consumer, // this yields (async, not thread-blocking) until permits are freed. - let sem_start = Instant::now(); - let _permit = semaphore - .acquire_many(byte_len as u32) + let _permit = STATS + .timed(&STATS.semaphore_wait_ns, semaphore.acquire_many(byte_len as u32)) .await .map_err(|e| unexpected(format!("semaphore: {e}")))?; - STATS - .semaphore_wait_ns - .fetch_add(sem_start.elapsed().as_nanos() as u64, Ordering::Relaxed); // Detach the permit — the consumer releases it via add_permits(). std::mem::forget(_permit); From 6d09e32bc99a9f1f1f263a20e36b63cd13ceae55 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 07:31:33 +0000 Subject: [PATCH 06/13] Add Julia tests for nested file-scan API; fix timed! macro usage Tests (test/scan_tests.jl): - Basic iteration: verify each FileScan has a non-empty .parquet filename, record_count > 0, and the nations table totals exactly 25 records. - Row count consistency: nested and flat pipelines over the customer table must produce identical total row counts. - Correct data: nested scan over nations produces the same rows as the existing flat-scan reference test. - Builder methods: select_columns! and with_batch_size! are respected per-batch in the nested path. - Safe early drop: abandon a FileScan mid-stream and drop the outer stream early; background file tasks see the dropped receiver and exit cleanly. Also fixes the timed! macro call for the serialize phase: the block form { expr.await.map_err(...)? } was incorrectly evaluating to () before the macro's own .await ran. The .map_err chains now appear outside the macro invocation where they belong. Labels: dismiss-release-notes build:benchmark Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/ordered_file_pipeline.rs | 39 ++-- src/full.jl | 2 - test/scan_tests.jl | 200 ++++++++++++++++++ 3 files changed, 223 insertions(+), 18 deletions(-) diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index dea1808..ec7ec6c 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -142,13 +142,6 @@ impl PipelineStats { field.store(start.elapsed().as_nanos() as u64, Ordering::Relaxed); } - async fn timed(&self, field: &AtomicU64, fut: F) -> F::Output { - let start = Instant::now(); - let result = fut.await; - self.add_elapsed(field, start); - result - } - fn track_buffer_add(&self, bytes: u64) { let prev = self.buffered_bytes.fetch_add(bytes, Ordering::Relaxed); self.peak_buffered_bytes @@ -229,6 +222,21 @@ impl PipelineStats { static STATS: PipelineStats = PipelineStats::new(); +/// Time an async expression and record its duration into a STATS field. +/// +/// ```ignore +/// let result = timed!(serialize_ns, { tokio::task::spawn_blocking(...) }) +/// .map_err(...)?; +/// ``` +macro_rules! timed { + ($field:ident, $fut:expr) => {{ + let _start = Instant::now(); + let _result = $fut.await; + STATS.add_elapsed(&STATS.$field, _start); + _result + }}; +} + // ── FFI exports for profiling (called from Julia benchmark teardown) ───── #[no_mangle] @@ -509,7 +517,7 @@ async fn process_file_inner( // Each .next() call fetches compressed parquet pages from storage, // decompresses (ZSTD), decodes column encodings, and assembles a // RecordBatch. These are inseparable without forking parquet-rs. - let batch_opt = STATS.timed(&STATS.fetch_decode_ns, batch_stream.next()).await; + let batch_opt = timed!(fetch_decode_ns, batch_stream.next()); let batch = match batch_opt { Some(Ok(b)) => b, @@ -520,11 +528,12 @@ async fn process_file_inner( // ── Phase 3: Serialize to Arrow IPC ───────────────────────────── // CPU-bound work, offloaded to the blocking thread pool to avoid // starving the tokio executor. - let serialized = STATS - .timed(&STATS.serialize_ns, tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch))) - .await - .map_err(|e| unexpected(format!("serialize panicked: {e}")))? - .map_err(|e| unexpected(e))?; + let serialized = timed!( + serialize_ns, + tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch)) + ) + .map_err(|e| unexpected(format!("serialize panicked: {e}")))? + .map_err(|e| unexpected(e))?; let byte_len = serialized.length; STATS.batches_produced.fetch_add(1, Ordering::Relaxed); @@ -536,9 +545,7 @@ async fn process_file_inner( // Acquire permits equal to the serialized size. If this file task // has produced > MAX_BUFFERED_BYTES_PER_TASK ahead of the consumer, // this yields (async, not thread-blocking) until permits are freed. - let _permit = STATS - .timed(&STATS.semaphore_wait_ns, semaphore.acquire_many(byte_len as u32)) - .await + let _permit = timed!(semaphore_wait_ns, semaphore.acquire_many(byte_len as u32)) .map_err(|e| unexpected(format!("semaphore: {e}")))?; // Detach the permit — the consumer releases it via add_permits(). std::mem::forget(_permit); diff --git a/src/full.jl b/src/full.jl index e34c31c..479bed5 100644 --- a/src/full.jl +++ b/src/full.jl @@ -314,7 +314,6 @@ end const FileScanStream = Ptr{Cvoid} const FileScanStreamResponse = Response{FileScanStream} -Response{FileScanStream}() = Response{FileScanStream}(-1, C_NULL, C_NULL, C_NULL) """ nested_arrow_stream(scan::Scan)::FileScanStream @@ -350,7 +349,6 @@ end # Response type for next_file_scan const FileScanResponse = Response{Ptr{Cvoid}} -Response{Ptr{Cvoid}}() = Response{Ptr{Cvoid}}(-1, C_NULL, C_NULL, C_NULL) """ next_file_scan(stream::FileScanStream)::Ptr{Cvoid} diff --git a/test/scan_tests.jl b/test/scan_tests.jl index c8e0358..643dddf 100644 --- a/test/scan_tests.jl +++ b/test/scan_tests.jl @@ -1223,3 +1223,203 @@ end end end end + +@testset "Nested File-Scan API" begin + nations_path = "s3://warehouse/tpch.sf01/nation/metadata/00001-44f668fe-3688-49d5-851f-36e75d143321.metadata.json" + customer_path = "s3://warehouse/tpch.sf01/customer/metadata/00001-76f6e7e4-b34f-492f-b6a1-cc9f8c8f4975.metadata.json" + + @testset "Basic iteration — filenames and record counts" begin + table = RustyIceberg.table_open(nations_path) + scan = RustyIceberg.new_scan(table) + outer = RustyIceberg.scan_nested!(scan) + @test outer != C_NULL + + file_count = 0 + total_records = 0 + try + fs_ptr = RustyIceberg.next_file_scan(outer) + while fs_ptr != C_NULL + file_count += 1 + fname = RustyIceberg.file_scan_filename(fs_ptr) + nrec = RustyIceberg.file_scan_record_count(fs_ptr) + + @test !isempty(fname) + @test endswith(fname, ".parquet") + @test startswith(fname, "s3://") + @test nrec > 0 + total_records += nrec + + # Drain the inner stream. + inner = RustyIceberg.file_scan_arrow_stream(fs_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + while batch_ptr != C_NULL + RustyIceberg.free_batch(batch_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + end + + RustyIceberg.free_file_scan!(fs_ptr) + fs_ptr = RustyIceberg.next_file_scan(outer) + end + finally + RustyIceberg.free_file_scan_stream!(outer) + RustyIceberg.free_scan!(scan) + RustyIceberg.free_table(table) + end + + @test file_count > 0 + @test total_records == 25 # nations table has exactly 25 rows + println("✅ Basic iteration: $file_count file(s), $total_records total records") + end + + @testset "Row counts match flat scan" begin + # Collect total rows via nested pipeline. + table_n = RustyIceberg.table_open(customer_path) + scan_n = RustyIceberg.new_scan(table_n) + outer = RustyIceberg.scan_nested!(scan_n) + + nested_rows = 0 + try + fs_ptr = RustyIceberg.next_file_scan(outer) + while fs_ptr != C_NULL + inner = RustyIceberg.file_scan_arrow_stream(fs_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + while batch_ptr != C_NULL + batch = unsafe_load(batch_ptr) + arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) + nested_rows += length(arrow_table) + RustyIceberg.free_batch(batch_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + end + RustyIceberg.free_file_scan!(fs_ptr) + fs_ptr = RustyIceberg.next_file_scan(outer) + end + finally + RustyIceberg.free_file_scan_stream!(outer) + RustyIceberg.free_scan!(scan_n) + RustyIceberg.free_table(table_n) + end + + # Collect total rows via flat pipeline. + table_f = RustyIceberg.table_open(customer_path) + scan_f = RustyIceberg.new_scan(table_f) + stream_f = RustyIceberg.scan!(scan_f) + + flat_rows = 0 + try + batch_ptr = RustyIceberg.next_batch(stream_f) + while batch_ptr != C_NULL + batch = unsafe_load(batch_ptr) + arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) + flat_rows += length(arrow_table) + RustyIceberg.free_batch(batch_ptr) + batch_ptr = RustyIceberg.next_batch(stream_f) + end + finally + RustyIceberg.free_stream(stream_f) + RustyIceberg.free_scan!(scan_f) + RustyIceberg.free_table(table_f) + end + + @test nested_rows == flat_rows + @test nested_rows > 0 + println("✅ Row count consistency: nested=$nested_rows, flat=$flat_rows") + end + + @testset "Correct data — nations table" begin + table = RustyIceberg.table_open(nations_path) + scan = RustyIceberg.new_scan(table) + outer = RustyIceberg.scan_nested!(scan) + + rows = Tuple[] + try + fs_ptr = RustyIceberg.next_file_scan(outer) + while fs_ptr != C_NULL + inner = RustyIceberg.file_scan_arrow_stream(fs_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + while batch_ptr != C_NULL + batch = unsafe_load(batch_ptr) + arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) + df = DataFrame(arrow_table) + @test names(df) == ["n_nationkey", "n_name", "n_regionkey", "n_comment"] + for row in eachrow(df) + push!(rows, Tuple(row)) + end + RustyIceberg.free_batch(batch_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + end + RustyIceberg.free_file_scan!(fs_ptr) + fs_ptr = RustyIceberg.next_file_scan(outer) + end + finally + RustyIceberg.free_file_scan_stream!(outer) + RustyIceberg.free_scan!(scan) + RustyIceberg.free_table(table) + end + + sort!(rows, by = x -> x[1]) + @test length(rows) == 25 + @test rows[1] == (0, "ALGERIA", 0, "furiously regular requests. platelets affix furious") + @test rows[25] == (24, "UNITED STATES", 1, "ly ironic requests along the slyly bold ideas hang after the blithely special notornis; blithely even accounts") + println("✅ Correct data verified for nations table via nested API") + end + + @testset "Builder methods — select_columns! and with_batch_size!" begin + table = RustyIceberg.table_open(customer_path) + scan = RustyIceberg.new_scan(table) + RustyIceberg.select_columns!(scan, ["c_custkey", "c_name"]) + RustyIceberg.with_batch_size!(scan, UInt(10)) + outer = RustyIceberg.scan_nested!(scan) + + try + fs_ptr = RustyIceberg.next_file_scan(outer) + while fs_ptr != C_NULL + inner = RustyIceberg.file_scan_arrow_stream(fs_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + while batch_ptr != C_NULL + batch = unsafe_load(batch_ptr) + arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) + df = DataFrame(arrow_table) + @test names(df) == ["c_custkey", "c_name"] + @test length(arrow_table) <= 10 + RustyIceberg.free_batch(batch_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + end + RustyIceberg.free_file_scan!(fs_ptr) + fs_ptr = RustyIceberg.next_file_scan(outer) + end + finally + RustyIceberg.free_file_scan_stream!(outer) + RustyIceberg.free_scan!(scan) + RustyIceberg.free_table(table) + end + println("✅ select_columns! and with_batch_size! work with nested API") + end + + @testset "Safe early drop of FileScan" begin + # Drop a FileScan after reading only the first batch — must not crash or hang. + table = RustyIceberg.table_open(customer_path) + scan = RustyIceberg.new_scan(table) + RustyIceberg.with_batch_size!(scan, UInt(1)) + outer = RustyIceberg.scan_nested!(scan) + + try + fs_ptr = RustyIceberg.next_file_scan(outer) + @test fs_ptr != C_NULL + + # Read exactly one batch then abandon the rest. + inner = RustyIceberg.file_scan_arrow_stream(fs_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + if batch_ptr != C_NULL + RustyIceberg.free_batch(batch_ptr) + end + # Drop the file scan without draining inner stream. + RustyIceberg.free_file_scan!(fs_ptr) + # Drop the outer stream without consuming remaining files. + finally + RustyIceberg.free_file_scan_stream!(outer) + RustyIceberg.free_scan!(scan) + RustyIceberg.free_table(table) + end + println("✅ Safe early drop of FileScan does not crash") + end +end From 63e6929822c45123d48797ef69e1904ad80e0b15 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 10:54:21 +0000 Subject: [PATCH 07/13] test nested scan --- test/scan_tests.jl | 38 ++++++++++++++++++++++++++++++++++++-- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/test/scan_tests.jl b/test/scan_tests.jl index 643dddf..5689b21 100644 --- a/test/scan_tests.jl +++ b/test/scan_tests.jl @@ -1228,6 +1228,40 @@ end nations_path = "s3://warehouse/tpch.sf01/nation/metadata/00001-44f668fe-3688-49d5-851f-36e75d143321.metadata.json" customer_path = "s3://warehouse/tpch.sf01/customer/metadata/00001-76f6e7e4-b34f-492f-b6a1-cc9f8c8f4975.metadata.json" + @testset "nested_arrow_stream called directly after build!" begin + # scan_nested! = build! + nested_arrow_stream; exercise the two-step path. + table = RustyIceberg.table_open(nations_path) + scan = RustyIceberg.new_scan(table) + RustyIceberg.build!(scan) + outer = RustyIceberg.nested_arrow_stream(scan) + @test outer != C_NULL + + row_count = 0 + try + fs_ptr = RustyIceberg.next_file_scan(outer) + while fs_ptr != C_NULL + inner = RustyIceberg.file_scan_arrow_stream(fs_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + while batch_ptr != C_NULL + batch = unsafe_load(batch_ptr) + arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) + row_count += nrow(DataFrame(arrow_table)) + RustyIceberg.free_batch(batch_ptr) + batch_ptr = RustyIceberg.next_batch(inner) + end + RustyIceberg.free_file_scan!(fs_ptr) + fs_ptr = RustyIceberg.next_file_scan(outer) + end + finally + RustyIceberg.free_file_scan_stream!(outer) + RustyIceberg.free_scan!(scan) + RustyIceberg.free_table(table) + end + + @test row_count == 25 + println("✅ nested_arrow_stream called directly after build! returns correct data") + end + @testset "Basic iteration — filenames and record counts" begin table = RustyIceberg.table_open(nations_path) scan = RustyIceberg.new_scan(table) @@ -1286,7 +1320,7 @@ end while batch_ptr != C_NULL batch = unsafe_load(batch_ptr) arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) - nested_rows += length(arrow_table) + nested_rows += nrow(DataFrame(arrow_table)) RustyIceberg.free_batch(batch_ptr) batch_ptr = RustyIceberg.next_batch(inner) end @@ -1310,7 +1344,7 @@ end while batch_ptr != C_NULL batch = unsafe_load(batch_ptr) arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) - flat_rows += length(arrow_table) + flat_rows += nrow(DataFrame(arrow_table)) RustyIceberg.free_batch(batch_ptr) batch_ptr = RustyIceberg.next_batch(stream_f) end From fe321b681083a56d59a35a809defd78235bfb512 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 13:38:09 +0000 Subject: [PATCH 08/13] Add pipeline_stats tests, dispatch backpressure metric, and nested wall time MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit PipelineStats unit tests (12 tests): - reset_clears_every_field, task_start/end concurrency tracking, peak high-water mark, buffer add/release/peak, add_elapsed accumulation, store_elapsed overwrite, field isolation. Dispatch backpressure metric: - Rename dead consumer_wait_ns → file_dispatch_wait_ns. Track time run_nested spends blocked on the outer channel (slow consumer) using timed!(file_dispatch_wait_ns, tx.send(...)). Shown as "dispatch stall" in print_summary. Nested pipeline wall time: - run_nested now records pipeline_wall_ns when it finishes dispatching all FileScan handles. For the flat path run_flat overwrites it with the full end-to-end time; for the nested path this gives a non-zero throughput in print_summary instead of always showing 0. Labels: dismiss-release-notes build:benchmark Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/ordered_file_pipeline.rs | 222 +++++++++++++++++- 1 file changed, 212 insertions(+), 10 deletions(-) diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index ec7ec6c..92552a7 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -79,9 +79,10 @@ struct PipelineStats { /// Time blocked on per-file semaphore (backpressure from consumer). semaphore_wait_ns: AtomicU64, - // ── Consumer ── - /// Time the flat consumer (`run_flat`) spends waiting for the next file. - consumer_wait_ns: AtomicU64, + // ── Dispatch ── + /// Time `run_nested` blocks on the outer channel waiting for the consumer + /// to call `next_file_scan` (backpressure from a slow consumer). + file_dispatch_wait_ns: AtomicU64, // ── Memory ── /// Live counter of serialized bytes buffered across all file tasks. @@ -103,7 +104,7 @@ impl PipelineStats { fetch_decode_ns: AtomicU64::new(0), serialize_ns: AtomicU64::new(0), semaphore_wait_ns: AtomicU64::new(0), - consumer_wait_ns: AtomicU64::new(0), + file_dispatch_wait_ns: AtomicU64::new(0), buffered_bytes: AtomicU64::new(0), peak_buffered_bytes: AtomicU64::new(0), } @@ -120,7 +121,7 @@ impl PipelineStats { self.fetch_decode_ns.store(0, Ordering::Relaxed); self.serialize_ns.store(0, Ordering::Relaxed); self.semaphore_wait_ns.store(0, Ordering::Relaxed); - self.consumer_wait_ns.store(0, Ordering::Relaxed); + self.file_dispatch_wait_ns.store(0, Ordering::Relaxed); self.buffered_bytes.store(0, Ordering::Relaxed); self.peak_buffered_bytes.store(0, Ordering::Relaxed); } @@ -162,7 +163,7 @@ impl PipelineStats { let fd_ms = self.fetch_decode_ns.load(Ordering::Relaxed) as f64 / 1e6; let ser_ms = self.serialize_ns.load(Ordering::Relaxed) as f64 / 1e6; let sem_ms = self.semaphore_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; - let con_ms = self.consumer_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; + let dispatch_ms = self.file_dispatch_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; let peak_buf = self.peak_buffered_bytes.load(Ordering::Relaxed) as f64 / (1024.0 * 1024.0); let bytes_mb = bytes as f64 / (1024.0 * 1024.0); @@ -211,9 +212,9 @@ impl PipelineStats { row(&format!("reader setup: {:>9.1} ms (open, metadata, deletes)", setup_ms)); row(&format!("fetch+decode: {:>9.1} ms (I/O + ZSTD + decode)", fd_ms)); row(&format!("serialize IPC: {:>9.1} ms (RecordBatch -> Arrow IPC)", ser_ms)); - row(&format!("semaphore wait: {:>9.1} ms (backpressure)", sem_ms)); - sep("consumer"); - row(&format!("ordering stall: {:>9.1} ms (waiting for head file)", con_ms)); + row(&format!("batch push wait: {:>9.1} ms (backpressure)", sem_ms)); + sep("dispatch"); + row(&format!("file push wait: {:>9.1} ms (backpressure)", dispatch_ms)); sep("memory"); row(&format!("peak buffered: {:>9.1} MB (limit: {} MB/task)", peak_buf, limit_mb)); border('└', '┘'); @@ -396,6 +397,12 @@ async fn run_nested( ) { use futures::stream::FuturesUnordered; + // For the nested path this records "time until last FileScan was handed to + // the consumer". For the flat path run_flat overwrites it with the full + // end-to-end time once all batches are drained, so the flat metric is + // unaffected. + let pipeline_start = Instant::now(); + let mut in_flight = FuturesUnordered::new(); let mut task_iter = tasks.into_iter(); @@ -420,7 +427,7 @@ async fn run_nested( record_count, stream, }; - if tx.send(Ok(file_scan)).await.is_err() { + if timed!(file_dispatch_wait_ns, tx.send(Ok(file_scan))).is_err() { return; // outer consumer dropped the stream } } @@ -430,6 +437,8 @@ async fn run_nested( } } } + + STATS.store_elapsed(&STATS.pipeline_wall_ns, pipeline_start); } /// Spawn a single file task. Returns a future that resolves immediately to @@ -569,3 +578,196 @@ async fn process_file_inner( STATS.files_completed.fetch_add(1, Ordering::Relaxed); Ok(()) } + +// ============================================================================= +// Tests +// ============================================================================= + +#[cfg(test)] +mod tests { + use super::*; + use std::time::Duration; + + fn fresh() -> PipelineStats { + PipelineStats::new() + } + + // ── reset ───────────────────────────────────────────────────────────────── + + #[test] + fn reset_clears_every_field() { + let s = fresh(); + s.pipeline_wall_ns.store(1, Ordering::Relaxed); + s.files_completed.store(2, Ordering::Relaxed); + s.batches_produced.store(3, Ordering::Relaxed); + s.bytes_produced.store(4, Ordering::Relaxed); + s.peak_concurrency.store(5, Ordering::Relaxed); + s.active_tasks.store(6, Ordering::Relaxed); + s.reader_setup_ns.store(7, Ordering::Relaxed); + s.fetch_decode_ns.store(8, Ordering::Relaxed); + s.serialize_ns.store(9, Ordering::Relaxed); + s.semaphore_wait_ns.store(10, Ordering::Relaxed); + s.file_dispatch_wait_ns.store(11, Ordering::Relaxed); + s.buffered_bytes.store(12, Ordering::Relaxed); + s.peak_buffered_bytes.store(13, Ordering::Relaxed); + + s.reset(); + + assert_eq!(s.pipeline_wall_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.files_completed.load(Ordering::Relaxed), 0); + assert_eq!(s.batches_produced.load(Ordering::Relaxed), 0); + assert_eq!(s.bytes_produced.load(Ordering::Relaxed), 0); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 0); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 0); + assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.fetch_decode_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.file_dispatch_wait_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 0); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 0); + } + + // ── track_task_start / track_task_end ───────────────────────────────────── + + #[test] + fn task_start_increments_active_and_updates_peak() { + let s = fresh(); + s.track_task_start(); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 1); + + s.track_task_start(); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 2); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 2); + } + + #[test] + fn task_end_decrements_active_without_touching_peak() { + let s = fresh(); + s.track_task_start(); + s.track_task_start(); + s.track_task_end(); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 2); // high-water mark unchanged + } + + #[test] + fn peak_concurrency_is_high_water_mark() { + let s = fresh(); + s.track_task_start(); // active 1, peak 1 + s.track_task_start(); // active 2, peak 2 + s.track_task_start(); // active 3, peak 3 + s.track_task_end(); // active 2, peak 3 + s.track_task_end(); // active 1, peak 3 + s.track_task_end(); // active 0, peak 3 + s.track_task_start(); // active 1 — below previous peak, peak stays 3 + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 3); + } + + // ── track_buffer_add / track_buffer_release ─────────────────────────────── + + #[test] + fn buffer_add_increments_bytes_and_peak() { + let s = fresh(); + s.track_buffer_add(100); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 100); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 100); + + s.track_buffer_add(50); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 150); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 150); + } + + #[test] + fn buffer_release_decrements_bytes_without_touching_peak() { + let s = fresh(); + s.track_buffer_add(200); + s.track_buffer_release(80); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 120); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 200); // peak unchanged + } + + #[test] + fn buffer_peak_does_not_decrease_after_release_and_smaller_add() { + let s = fresh(); + s.track_buffer_add(500); // peak → 500 + s.track_buffer_release(500); // current → 0 + s.track_buffer_add(10); // current → 10, peak stays 500 + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 10); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 500); + } + + // ── add_elapsed ─────────────────────────────────────────────────────────── + + #[test] + fn add_elapsed_accumulates_into_field() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.reader_setup_ns, start); + let after_first = s.reader_setup_ns.load(Ordering::Relaxed); + assert!(after_first > 0, "first add_elapsed should record > 0 ns"); + + let start2 = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.reader_setup_ns, start2); + let after_second = s.reader_setup_ns.load(Ordering::Relaxed); + assert!(after_second > after_first, "second add_elapsed should accumulate"); + } + + #[test] + fn add_elapsed_is_independent_per_field() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.fetch_decode_ns, start); + + // Unrelated fields stay at 0. + assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); + assert!(s.fetch_decode_ns.load(Ordering::Relaxed) > 0); + } + + // ── store_elapsed ───────────────────────────────────────────────────────── + + #[test] + fn store_elapsed_overwrites_previous_value() { + let s = fresh(); + s.pipeline_wall_ns.store(999_999_999, Ordering::Relaxed); // some large value + + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.store_elapsed(&s.pipeline_wall_ns, start); + let stored = s.pipeline_wall_ns.load(Ordering::Relaxed); + + // Stored value is the elapsed time, which is much less than 999_999_999 ns (1 s). + assert!(stored > 0); + assert!(stored < 999_999_999, "store_elapsed should overwrite, not accumulate"); + } + + #[test] + fn file_dispatch_wait_accumulates_via_add_elapsed() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.file_dispatch_wait_ns, start); + assert!(s.file_dispatch_wait_ns.load(Ordering::Relaxed) > 0); + // Other fields are unaffected. + assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); + } + + #[test] + fn store_elapsed_does_not_affect_other_fields() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.store_elapsed(&s.pipeline_wall_ns, start); + + assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.fetch_decode_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); + } +} From 9219563fe9c6ffb1916b42e590bda442ebdef1e6 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 14:57:00 +0000 Subject: [PATCH 09/13] cleanup Co-authored-by: Copilot --- iceberg_rust_ffi/src/ordered_file_pipeline.rs | 27 +++++++------------ test/scan_tests.jl | 2 +- 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index 92552a7..5561b63 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -301,10 +301,11 @@ pub async fn create_nested_pipeline( batch_size: Option, concurrency: usize, ) -> anyhow::Result { - assert!( - concurrency <= MAX_FILE_CONCURRENCY, - "file concurrency {concurrency} exceeds hard cap {MAX_FILE_CONCURRENCY}" - ); + if concurrency > MAX_FILE_CONCURRENCY { + anyhow::bail!( + "file concurrency {concurrency} exceeds hard cap {MAX_FILE_CONCURRENCY}" + ); + } STATS.reset(); @@ -332,11 +333,6 @@ pub async fn create_pipeline( batch_size: Option, concurrency: usize, ) -> anyhow::Result { - assert!( - concurrency <= MAX_FILE_CONCURRENCY, - "file concurrency {concurrency} exceeds hard cap {MAX_FILE_CONCURRENCY}" - ); - // Outer channel: flatten task → Julia (via IcebergArrowStream). let (tx, rx) = mpsc::channel(concurrency * 2); @@ -382,12 +378,9 @@ async fn run_flat( STATS.store_elapsed(&STATS.pipeline_wall_ns, pipeline_start); } -/// Main nested consumer loop. Orchestrates file-level parallelism while -/// maintaining strict file ordering. -/// -/// Uses FuturesOrdered to poll N file tasks concurrently but yield their -/// (filename, record_count, receiver) tuples in push order. Wraps each -/// receiver as an IcebergArrowStream and sends a FileScan to the outer channel. +/// Main nested consumer loop. Keeps `concurrency` file tasks in flight concurrently using +/// FuturesUnordered and wraps each completed task's receiver as an +/// IcebergArrowStream before sending a FileScan to the outer channel. async fn run_nested( tasks: Vec, file_io: FileIO, @@ -445,8 +438,8 @@ async fn run_nested( /// the file's metadata and the receiving end of its batch channel. /// /// The actual work (parquet I/O, decode, serialize) happens in the background -/// tokio task. The future resolves to (filename, record_count, Receiver) so -/// that FuturesOrdered can yield them in file order. +/// tokio task. The future resolves immediately to (filename, record_count, +/// Receiver) so that FuturesUnordered can poll it alongside other tasks. fn spawn_file_task_with_meta( task: FileScanTask, file_io: FileIO, diff --git a/test/scan_tests.jl b/test/scan_tests.jl index 5689b21..a213e2a 100644 --- a/test/scan_tests.jl +++ b/test/scan_tests.jl @@ -1414,7 +1414,7 @@ end arrow_table = Arrow.Table(unsafe_wrap(Array, batch.data, batch.length)) df = DataFrame(arrow_table) @test names(df) == ["c_custkey", "c_name"] - @test length(arrow_table) <= 10 + @test nrow(df) <= 10 RustyIceberg.free_batch(batch_ptr) batch_ptr = RustyIceberg.next_batch(inner) end From 87b8941018305c20dd1e8984cde942c9622a9a52 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 17:36:03 +0000 Subject: [PATCH 10/13] Move pipeline stats to it's own file --- iceberg_rust_ffi/src/lib.rs | 3 + iceberg_rust_ffi/src/ordered_file_pipeline.rs | 389 +--------------- iceberg_rust_ffi/src/pipeline_stats.rs | 425 ++++++++++++++++++ 3 files changed, 430 insertions(+), 387 deletions(-) create mode 100644 iceberg_rust_ffi/src/pipeline_stats.rs diff --git a/iceberg_rust_ffi/src/lib.rs b/iceberg_rust_ffi/src/lib.rs index 02269d3..6c5107a 100644 --- a/iceberg_rust_ffi/src/lib.rs +++ b/iceberg_rust_ffi/src/lib.rs @@ -35,6 +35,9 @@ mod writer; // Column-based writer module (zero-copy from Julia) mod writer_columns; +// Profiling stats for the file-parallel pipeline +mod pipeline_stats; + // Ordered file-parallel pipeline mod ordered_file_pipeline; diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index 5561b63..b1593ec 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -27,7 +27,7 @@ //! batches and releases permits. This caps each file's buffered output to //! ~100MB independently. -use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering}; +use std::sync::atomic::Ordering; use std::sync::Arc; use std::time::Instant; @@ -37,192 +37,13 @@ use iceberg::io::FileIO; use iceberg::scan::FileScanTask; use tokio::sync::{mpsc, Mutex as AsyncMutex, Semaphore}; +use crate::pipeline_stats::{MAX_BUFFERED_BYTES_PER_TASK, STATS}; use crate::table::{ArrowBatch, IcebergArrowStream, IcebergFileScanStream}; use crate::unexpected; -/// Per-file cap on serialized bytes buffered ahead of the consumer. -const MAX_BUFFERED_BYTES_PER_TASK: usize = 100 * 1024 * 1024; - /// Hard cap on file-level concurrency to keep total memory bounded. const MAX_FILE_CONCURRENCY: usize = 16; -// =========================================================================== -// Temporary profiling — will be removed before merging to production. -// -// Global atomic counters accumulate timing/size data across all file tasks. -// Called from Julia via `@ccall iceberg_print_pipeline_stats()`. -// =========================================================================== - -/// Accumulates profiling data across all file tasks in a pipeline run. -/// All fields are atomics so concurrent file tasks can update without locks. -struct PipelineStats { - // ── Overall ── - pipeline_wall_ns: AtomicU64, - files_completed: AtomicUsize, - batches_produced: AtomicUsize, - bytes_produced: AtomicU64, - - // ── Concurrency ── - peak_concurrency: AtomicUsize, - active_tasks: AtomicUsize, - - // ── Per-phase cumulative time (ns), summed across all file tasks ── - /// Time to build ArrowReader and call reader.read() (opens parquet, - /// loads metadata, resolves schema, loads delete files, builds filters). - reader_setup_ns: AtomicU64, - /// Time inside batch_stream.next() — fetches compressed pages from - /// storage, decompresses (ZSTD), decodes pages, assembles columns. - fetch_decode_ns: AtomicU64, - /// Time in spawn_blocking(serialize_record_batch) — writes RecordBatch - /// to Arrow IPC wire format for transfer to Julia. - serialize_ns: AtomicU64, - /// Time blocked on per-file semaphore (backpressure from consumer). - semaphore_wait_ns: AtomicU64, - - // ── Dispatch ── - /// Time `run_nested` blocks on the outer channel waiting for the consumer - /// to call `next_file_scan` (backpressure from a slow consumer). - file_dispatch_wait_ns: AtomicU64, - - // ── Memory ── - /// Live counter of serialized bytes buffered across all file tasks. - buffered_bytes: AtomicU64, - /// High-water mark of buffered_bytes. - peak_buffered_bytes: AtomicU64, -} - -impl PipelineStats { - const fn new() -> Self { - Self { - pipeline_wall_ns: AtomicU64::new(0), - files_completed: AtomicUsize::new(0), - batches_produced: AtomicUsize::new(0), - bytes_produced: AtomicU64::new(0), - peak_concurrency: AtomicUsize::new(0), - active_tasks: AtomicUsize::new(0), - reader_setup_ns: AtomicU64::new(0), - fetch_decode_ns: AtomicU64::new(0), - serialize_ns: AtomicU64::new(0), - semaphore_wait_ns: AtomicU64::new(0), - file_dispatch_wait_ns: AtomicU64::new(0), - buffered_bytes: AtomicU64::new(0), - peak_buffered_bytes: AtomicU64::new(0), - } - } - - fn reset(&self) { - self.pipeline_wall_ns.store(0, Ordering::Relaxed); - self.files_completed.store(0, Ordering::Relaxed); - self.batches_produced.store(0, Ordering::Relaxed); - self.bytes_produced.store(0, Ordering::Relaxed); - self.peak_concurrency.store(0, Ordering::Relaxed); - self.active_tasks.store(0, Ordering::Relaxed); - self.reader_setup_ns.store(0, Ordering::Relaxed); - self.fetch_decode_ns.store(0, Ordering::Relaxed); - self.serialize_ns.store(0, Ordering::Relaxed); - self.semaphore_wait_ns.store(0, Ordering::Relaxed); - self.file_dispatch_wait_ns.store(0, Ordering::Relaxed); - self.buffered_bytes.store(0, Ordering::Relaxed); - self.peak_buffered_bytes.store(0, Ordering::Relaxed); - } - - fn track_task_start(&self) { - let prev = self.active_tasks.fetch_add(1, Ordering::Relaxed); - self.peak_concurrency.fetch_max(prev + 1, Ordering::Relaxed); - } - - fn track_task_end(&self) { - self.active_tasks.fetch_sub(1, Ordering::Relaxed); - } - - fn add_elapsed(&self, field: &AtomicU64, start: Instant) { - field.fetch_add(start.elapsed().as_nanos() as u64, Ordering::Relaxed); - } - - fn store_elapsed(&self, field: &AtomicU64, start: Instant) { - field.store(start.elapsed().as_nanos() as u64, Ordering::Relaxed); - } - - fn track_buffer_add(&self, bytes: u64) { - let prev = self.buffered_bytes.fetch_add(bytes, Ordering::Relaxed); - self.peak_buffered_bytes - .fetch_max(prev + bytes, Ordering::Relaxed); - } - - fn track_buffer_release(&self, bytes: u64) { - self.buffered_bytes.fetch_sub(bytes, Ordering::Relaxed); - } - - fn print_summary(&self) { - let wall_ms = self.pipeline_wall_ns.load(Ordering::Relaxed) as f64 / 1e6; - let files = self.files_completed.load(Ordering::Relaxed); - let batches = self.batches_produced.load(Ordering::Relaxed); - let bytes = self.bytes_produced.load(Ordering::Relaxed); - let peak = self.peak_concurrency.load(Ordering::Relaxed); - let setup_ms = self.reader_setup_ns.load(Ordering::Relaxed) as f64 / 1e6; - let fd_ms = self.fetch_decode_ns.load(Ordering::Relaxed) as f64 / 1e6; - let ser_ms = self.serialize_ns.load(Ordering::Relaxed) as f64 / 1e6; - let sem_ms = self.semaphore_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; - let dispatch_ms = self.file_dispatch_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; - let peak_buf = self.peak_buffered_bytes.load(Ordering::Relaxed) as f64 / (1024.0 * 1024.0); - - let bytes_mb = bytes as f64 / (1024.0 * 1024.0); - let throughput = if wall_ms > 0.0 { - bytes_mb / (wall_ms / 1000.0) - } else { - 0.0 - }; - let file_wall_ms = setup_ms + fd_ms + ser_ms + sem_ms; - let parallelism = if wall_ms > 0.0 { - file_wall_ms / wall_ms - } else { - 0.0 - }; - let limit_mb = MAX_BUFFERED_BYTES_PER_TASK / (1024 * 1024); - - // Box layout: "│ " + content + padding + " │", total = BOX chars. - // All content uses ASCII only so byte len = display width. - const BOX: usize = 68; // total width including borders - - let row = |content: &str| { - // "│ " = 3, " │" = 3 => content area = BOX - 6 - let pad = (BOX - 6).saturating_sub(content.len()); - println!("│ {}{:pad$} │", content, "", pad = pad); - }; - let dashes = |n: usize| -> String { "─".repeat(n) }; - let sep = |label: &str| { - // "│ ── label ──── │" — label is ASCII, dashes are multi-byte. - // Display width: 3 + 3 + label.len() + 1 + fill + 1 = BOX - // => fill = BOX - 3 - 3 - label.len() - 3 - 1 = BOX - 10 - label.len() - let fill = (BOX - 10).saturating_sub(label.len()); - println!("│ {} {} {} │", dashes(2), label, dashes(fill)); - }; - let border = |left: char, right: char| { - println!("{}{}{}", left, dashes(BOX - 2), right); - }; - - border('┌', '┐'); - row(&format!("Pipeline Stats")); - row(""); - row(&format!("wall time: {:>9.1} ms", wall_ms)); - row(&format!("files: {:>9} peak concurrency: {}", files, peak)); - row(&format!("batches: {:>9} serialized: {:.1} MB", batches, bytes_mb)); - row(&format!("throughput: {:>9.1} MB/s parallelism: {:.1}x", throughput, parallelism)); - sep("time across all file tasks (sum)"); - row(&format!("reader setup: {:>9.1} ms (open, metadata, deletes)", setup_ms)); - row(&format!("fetch+decode: {:>9.1} ms (I/O + ZSTD + decode)", fd_ms)); - row(&format!("serialize IPC: {:>9.1} ms (RecordBatch -> Arrow IPC)", ser_ms)); - row(&format!("batch push wait: {:>9.1} ms (backpressure)", sem_ms)); - sep("dispatch"); - row(&format!("file push wait: {:>9.1} ms (backpressure)", dispatch_ms)); - sep("memory"); - row(&format!("peak buffered: {:>9.1} MB (limit: {} MB/task)", peak_buf, limit_mb)); - border('└', '┘'); - } -} - -static STATS: PipelineStats = PipelineStats::new(); - /// Time an async expression and record its duration into a STATS field. /// /// ```ignore @@ -238,18 +59,6 @@ macro_rules! timed { }}; } -// ── FFI exports for profiling (called from Julia benchmark teardown) ───── - -#[no_mangle] -pub extern "C" fn iceberg_print_pipeline_stats() { - STATS.print_summary(); -} - -#[no_mangle] -pub extern "C" fn iceberg_reset_pipeline_stats() { - STATS.reset(); -} - // =========================================================================== // Pipeline implementation // =========================================================================== @@ -494,7 +303,6 @@ async fn process_file_inner( semaphore: Arc, tx: &mpsc::Sender>, ) -> Result<(), iceberg::Error> { - // ── Phase 1: Reader setup ─────────────────────────────────────────── // Builds a per-file ArrowReader using the same iceberg-rs code path as // to_arrow(): opens parquet metadata, resolves schema, loads delete @@ -571,196 +379,3 @@ async fn process_file_inner( STATS.files_completed.fetch_add(1, Ordering::Relaxed); Ok(()) } - -// ============================================================================= -// Tests -// ============================================================================= - -#[cfg(test)] -mod tests { - use super::*; - use std::time::Duration; - - fn fresh() -> PipelineStats { - PipelineStats::new() - } - - // ── reset ───────────────────────────────────────────────────────────────── - - #[test] - fn reset_clears_every_field() { - let s = fresh(); - s.pipeline_wall_ns.store(1, Ordering::Relaxed); - s.files_completed.store(2, Ordering::Relaxed); - s.batches_produced.store(3, Ordering::Relaxed); - s.bytes_produced.store(4, Ordering::Relaxed); - s.peak_concurrency.store(5, Ordering::Relaxed); - s.active_tasks.store(6, Ordering::Relaxed); - s.reader_setup_ns.store(7, Ordering::Relaxed); - s.fetch_decode_ns.store(8, Ordering::Relaxed); - s.serialize_ns.store(9, Ordering::Relaxed); - s.semaphore_wait_ns.store(10, Ordering::Relaxed); - s.file_dispatch_wait_ns.store(11, Ordering::Relaxed); - s.buffered_bytes.store(12, Ordering::Relaxed); - s.peak_buffered_bytes.store(13, Ordering::Relaxed); - - s.reset(); - - assert_eq!(s.pipeline_wall_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.files_completed.load(Ordering::Relaxed), 0); - assert_eq!(s.batches_produced.load(Ordering::Relaxed), 0); - assert_eq!(s.bytes_produced.load(Ordering::Relaxed), 0); - assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 0); - assert_eq!(s.active_tasks.load(Ordering::Relaxed), 0); - assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.fetch_decode_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.file_dispatch_wait_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 0); - assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 0); - } - - // ── track_task_start / track_task_end ───────────────────────────────────── - - #[test] - fn task_start_increments_active_and_updates_peak() { - let s = fresh(); - s.track_task_start(); - assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); - assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 1); - - s.track_task_start(); - assert_eq!(s.active_tasks.load(Ordering::Relaxed), 2); - assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 2); - } - - #[test] - fn task_end_decrements_active_without_touching_peak() { - let s = fresh(); - s.track_task_start(); - s.track_task_start(); - s.track_task_end(); - assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); - assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 2); // high-water mark unchanged - } - - #[test] - fn peak_concurrency_is_high_water_mark() { - let s = fresh(); - s.track_task_start(); // active 1, peak 1 - s.track_task_start(); // active 2, peak 2 - s.track_task_start(); // active 3, peak 3 - s.track_task_end(); // active 2, peak 3 - s.track_task_end(); // active 1, peak 3 - s.track_task_end(); // active 0, peak 3 - s.track_task_start(); // active 1 — below previous peak, peak stays 3 - assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); - assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 3); - } - - // ── track_buffer_add / track_buffer_release ─────────────────────────────── - - #[test] - fn buffer_add_increments_bytes_and_peak() { - let s = fresh(); - s.track_buffer_add(100); - assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 100); - assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 100); - - s.track_buffer_add(50); - assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 150); - assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 150); - } - - #[test] - fn buffer_release_decrements_bytes_without_touching_peak() { - let s = fresh(); - s.track_buffer_add(200); - s.track_buffer_release(80); - assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 120); - assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 200); // peak unchanged - } - - #[test] - fn buffer_peak_does_not_decrease_after_release_and_smaller_add() { - let s = fresh(); - s.track_buffer_add(500); // peak → 500 - s.track_buffer_release(500); // current → 0 - s.track_buffer_add(10); // current → 10, peak stays 500 - assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 10); - assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 500); - } - - // ── add_elapsed ─────────────────────────────────────────────────────────── - - #[test] - fn add_elapsed_accumulates_into_field() { - let s = fresh(); - let start = Instant::now(); - std::thread::sleep(Duration::from_millis(2)); - s.add_elapsed(&s.reader_setup_ns, start); - let after_first = s.reader_setup_ns.load(Ordering::Relaxed); - assert!(after_first > 0, "first add_elapsed should record > 0 ns"); - - let start2 = Instant::now(); - std::thread::sleep(Duration::from_millis(2)); - s.add_elapsed(&s.reader_setup_ns, start2); - let after_second = s.reader_setup_ns.load(Ordering::Relaxed); - assert!(after_second > after_first, "second add_elapsed should accumulate"); - } - - #[test] - fn add_elapsed_is_independent_per_field() { - let s = fresh(); - let start = Instant::now(); - std::thread::sleep(Duration::from_millis(2)); - s.add_elapsed(&s.fetch_decode_ns, start); - - // Unrelated fields stay at 0. - assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); - assert!(s.fetch_decode_ns.load(Ordering::Relaxed) > 0); - } - - // ── store_elapsed ───────────────────────────────────────────────────────── - - #[test] - fn store_elapsed_overwrites_previous_value() { - let s = fresh(); - s.pipeline_wall_ns.store(999_999_999, Ordering::Relaxed); // some large value - - let start = Instant::now(); - std::thread::sleep(Duration::from_millis(2)); - s.store_elapsed(&s.pipeline_wall_ns, start); - let stored = s.pipeline_wall_ns.load(Ordering::Relaxed); - - // Stored value is the elapsed time, which is much less than 999_999_999 ns (1 s). - assert!(stored > 0); - assert!(stored < 999_999_999, "store_elapsed should overwrite, not accumulate"); - } - - #[test] - fn file_dispatch_wait_accumulates_via_add_elapsed() { - let s = fresh(); - let start = Instant::now(); - std::thread::sleep(Duration::from_millis(2)); - s.add_elapsed(&s.file_dispatch_wait_ns, start); - assert!(s.file_dispatch_wait_ns.load(Ordering::Relaxed) > 0); - // Other fields are unaffected. - assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); - } - - #[test] - fn store_elapsed_does_not_affect_other_fields() { - let s = fresh(); - let start = Instant::now(); - std::thread::sleep(Duration::from_millis(2)); - s.store_elapsed(&s.pipeline_wall_ns, start); - - assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.fetch_decode_ns.load(Ordering::Relaxed), 0); - assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); - } -} diff --git a/iceberg_rust_ffi/src/pipeline_stats.rs b/iceberg_rust_ffi/src/pipeline_stats.rs new file mode 100644 index 0000000..d65a168 --- /dev/null +++ b/iceberg_rust_ffi/src/pipeline_stats.rs @@ -0,0 +1,425 @@ +// =========================================================================== +// Temporary profiling — will be removed before merging to production. +// +// Global atomic counters accumulate timing/size data across all file tasks. +// Called from Julia via `@ccall iceberg_print_pipeline_stats()`. +// =========================================================================== + +use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering}; +use std::time::Instant; + +pub(crate) const MAX_BUFFERED_BYTES_PER_TASK: usize = 100 * 1024 * 1024; + +/// Accumulates profiling data across all file tasks in a pipeline run. +/// All fields are atomics so concurrent file tasks can update without locks. +pub(crate) struct PipelineStats { + // ── Overall ── + pub pipeline_wall_ns: AtomicU64, + pub files_completed: AtomicUsize, + pub batches_produced: AtomicUsize, + pub bytes_produced: AtomicU64, + + // ── Concurrency ── + pub peak_concurrency: AtomicUsize, + pub active_tasks: AtomicUsize, + + // ── Per-phase cumulative time (ns), summed across all file tasks ── + /// Time to build ArrowReader and call reader.read() (opens parquet, + /// loads metadata, resolves schema, loads delete files, builds filters). + pub reader_setup_ns: AtomicU64, + /// Time inside batch_stream.next() — fetches compressed pages from + /// storage, decompresses (ZSTD), decodes pages, assembles columns. + pub fetch_decode_ns: AtomicU64, + /// Time in spawn_blocking(serialize_record_batch) — writes RecordBatch + /// to Arrow IPC wire format for transfer to Julia. + pub serialize_ns: AtomicU64, + /// Time blocked on per-file semaphore (backpressure from consumer). + pub semaphore_wait_ns: AtomicU64, + + // ── Dispatch ── + /// Time `run_nested` blocks on the outer channel waiting for the consumer + /// to call `next_file_scan` (backpressure from a slow consumer). + pub file_dispatch_wait_ns: AtomicU64, + + // ── Memory ── + /// Live counter of serialized bytes buffered across all file tasks. + pub buffered_bytes: AtomicU64, + /// High-water mark of buffered_bytes. + pub peak_buffered_bytes: AtomicU64, +} + +impl PipelineStats { + pub(crate) const fn new() -> Self { + Self { + pipeline_wall_ns: AtomicU64::new(0), + files_completed: AtomicUsize::new(0), + batches_produced: AtomicUsize::new(0), + bytes_produced: AtomicU64::new(0), + peak_concurrency: AtomicUsize::new(0), + active_tasks: AtomicUsize::new(0), + reader_setup_ns: AtomicU64::new(0), + fetch_decode_ns: AtomicU64::new(0), + serialize_ns: AtomicU64::new(0), + semaphore_wait_ns: AtomicU64::new(0), + file_dispatch_wait_ns: AtomicU64::new(0), + buffered_bytes: AtomicU64::new(0), + peak_buffered_bytes: AtomicU64::new(0), + } + } + + pub(crate) fn reset(&self) { + self.pipeline_wall_ns.store(0, Ordering::Relaxed); + self.files_completed.store(0, Ordering::Relaxed); + self.batches_produced.store(0, Ordering::Relaxed); + self.bytes_produced.store(0, Ordering::Relaxed); + self.peak_concurrency.store(0, Ordering::Relaxed); + self.active_tasks.store(0, Ordering::Relaxed); + self.reader_setup_ns.store(0, Ordering::Relaxed); + self.fetch_decode_ns.store(0, Ordering::Relaxed); + self.serialize_ns.store(0, Ordering::Relaxed); + self.semaphore_wait_ns.store(0, Ordering::Relaxed); + self.file_dispatch_wait_ns.store(0, Ordering::Relaxed); + self.buffered_bytes.store(0, Ordering::Relaxed); + self.peak_buffered_bytes.store(0, Ordering::Relaxed); + } + + pub(crate) fn track_task_start(&self) { + let prev = self.active_tasks.fetch_add(1, Ordering::Relaxed); + self.peak_concurrency.fetch_max(prev + 1, Ordering::Relaxed); + } + + pub(crate) fn track_task_end(&self) { + self.active_tasks.fetch_sub(1, Ordering::Relaxed); + } + + pub(crate) fn add_elapsed(&self, field: &AtomicU64, start: Instant) { + field.fetch_add(start.elapsed().as_nanos() as u64, Ordering::Relaxed); + } + + pub(crate) fn store_elapsed(&self, field: &AtomicU64, start: Instant) { + field.store(start.elapsed().as_nanos() as u64, Ordering::Relaxed); + } + + pub(crate) fn track_buffer_add(&self, bytes: u64) { + let prev = self.buffered_bytes.fetch_add(bytes, Ordering::Relaxed); + self.peak_buffered_bytes + .fetch_max(prev + bytes, Ordering::Relaxed); + } + + /// Release buffered bytes. Uses saturating_sub to guard against underflow + /// caused by lingering tasks from a previous scan releasing after a reset(). + pub(crate) fn track_buffer_release(&self, bytes: u64) { + self.buffered_bytes + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |v| { + Some(v.saturating_sub(bytes)) + }) + .ok(); + } + + pub(crate) fn print_summary(&self) { + let wall_ms = self.pipeline_wall_ns.load(Ordering::Relaxed) as f64 / 1e6; + let files = self.files_completed.load(Ordering::Relaxed); + let batches = self.batches_produced.load(Ordering::Relaxed); + let bytes = self.bytes_produced.load(Ordering::Relaxed); + let peak = self.peak_concurrency.load(Ordering::Relaxed); + let setup_ms = self.reader_setup_ns.load(Ordering::Relaxed) as f64 / 1e6; + let fd_ms = self.fetch_decode_ns.load(Ordering::Relaxed) as f64 / 1e6; + let ser_ms = self.serialize_ns.load(Ordering::Relaxed) as f64 / 1e6; + let sem_ms = self.semaphore_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; + let dispatch_ms = self.file_dispatch_wait_ns.load(Ordering::Relaxed) as f64 / 1e6; + let peak_buf = + self.peak_buffered_bytes.load(Ordering::Relaxed) as f64 / (1024.0 * 1024.0); + + let bytes_mb = bytes as f64 / (1024.0 * 1024.0); + let throughput = if wall_ms > 0.0 { + bytes_mb / (wall_ms / 1000.0) + } else { + 0.0 + }; + let file_wall_ms = setup_ms + fd_ms + ser_ms + sem_ms; + let parallelism = if wall_ms > 0.0 { + file_wall_ms / wall_ms + } else { + 0.0 + }; + let limit_mb = MAX_BUFFERED_BYTES_PER_TASK / (1024 * 1024); + + // Box layout: "│ " + content + padding + " │", total = BOX chars. + // All content uses ASCII only so byte len = display width. + const BOX: usize = 68; // total width including borders + + let row = |content: &str| { + let pad = (BOX - 6).saturating_sub(content.len()); + println!("│ {}{:pad$} │", content, "", pad = pad); + }; + let dashes = |n: usize| -> String { "─".repeat(n) }; + let sep = |label: &str| { + let fill = (BOX - 10).saturating_sub(label.len()); + println!("│ {} {} {} │", dashes(2), label, dashes(fill)); + }; + let border = |left: char, right: char| { + println!("{}{}{}", left, dashes(BOX - 2), right); + }; + + border('┌', '┐'); + row("Pipeline Stats"); + row(""); + row(&format!("wall time: {:>9.1} ms", wall_ms)); + row(&format!( + "files: {:>9} peak concurrency: {}", + files, peak + )); + row(&format!( + "batches: {:>9} serialized: {:.1} MB", + batches, bytes_mb + )); + row(&format!( + "throughput: {:>9.1} MB/s parallelism: {:.1}x", + throughput, parallelism + )); + sep("time across all file tasks (sum)"); + row(&format!( + "reader setup: {:>9.1} ms (open, metadata, deletes)", + setup_ms + )); + row(&format!( + "fetch+decode: {:>9.1} ms (I/O + ZSTD + decode)", + fd_ms + )); + row(&format!( + "serialize IPC: {:>9.1} ms (RecordBatch -> Arrow IPC)", + ser_ms + )); + row(&format!( + "batch push wait: {:>9.1} ms (backpressure)", + sem_ms + )); + sep("dispatch"); + row(&format!( + "file push wait: {:>9.1} ms (backpressure)", + dispatch_ms + )); + sep("memory"); + row(&format!( + "peak buffered: {:>9.1} MB (limit: {} MB/task)", + peak_buf, limit_mb + )); + border('└', '┘'); + } +} + +pub(crate) static STATS: PipelineStats = PipelineStats::new(); + +// ── FFI exports (called from Julia benchmark teardown) ──────────────────── + +#[no_mangle] +pub extern "C" fn iceberg_print_pipeline_stats() { + STATS.print_summary(); +} + +#[no_mangle] +pub extern "C" fn iceberg_reset_pipeline_stats() { + STATS.reset(); +} + +// ============================================================================= +// Tests +// ============================================================================= + +#[cfg(test)] +mod tests { + use super::*; + use std::time::Duration; + + fn fresh() -> PipelineStats { + PipelineStats::new() + } + + // ── reset ───────────────────────────────────────────────────────────────── + + #[test] + fn reset_clears_every_field() { + let s = fresh(); + s.pipeline_wall_ns.store(1, Ordering::Relaxed); + s.files_completed.store(2, Ordering::Relaxed); + s.batches_produced.store(3, Ordering::Relaxed); + s.bytes_produced.store(4, Ordering::Relaxed); + s.peak_concurrency.store(5, Ordering::Relaxed); + s.active_tasks.store(6, Ordering::Relaxed); + s.reader_setup_ns.store(7, Ordering::Relaxed); + s.fetch_decode_ns.store(8, Ordering::Relaxed); + s.serialize_ns.store(9, Ordering::Relaxed); + s.semaphore_wait_ns.store(10, Ordering::Relaxed); + s.file_dispatch_wait_ns.store(11, Ordering::Relaxed); + s.buffered_bytes.store(12, Ordering::Relaxed); + s.peak_buffered_bytes.store(13, Ordering::Relaxed); + + s.reset(); + + assert_eq!(s.pipeline_wall_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.files_completed.load(Ordering::Relaxed), 0); + assert_eq!(s.batches_produced.load(Ordering::Relaxed), 0); + assert_eq!(s.bytes_produced.load(Ordering::Relaxed), 0); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 0); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 0); + assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.fetch_decode_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.file_dispatch_wait_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 0); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 0); + } + + // ── track_task_start / track_task_end ───────────────────────────────────── + + #[test] + fn task_start_increments_active_and_updates_peak() { + let s = fresh(); + s.track_task_start(); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 1); + + s.track_task_start(); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 2); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 2); + } + + #[test] + fn task_end_decrements_active_without_touching_peak() { + let s = fresh(); + s.track_task_start(); + s.track_task_start(); + s.track_task_end(); + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 2); // high-water mark unchanged + } + + #[test] + fn peak_concurrency_is_high_water_mark() { + let s = fresh(); + s.track_task_start(); // active 1, peak 1 + s.track_task_start(); // active 2, peak 2 + s.track_task_start(); // active 3, peak 3 + s.track_task_end(); // active 2, peak 3 + s.track_task_end(); // active 1, peak 3 + s.track_task_end(); // active 0, peak 3 + s.track_task_start(); // active 1 — below previous peak, peak stays 3 + assert_eq!(s.active_tasks.load(Ordering::Relaxed), 1); + assert_eq!(s.peak_concurrency.load(Ordering::Relaxed), 3); + } + + // ── track_buffer_add / track_buffer_release ─────────────────────────────── + + #[test] + fn buffer_add_increments_bytes_and_peak() { + let s = fresh(); + s.track_buffer_add(100); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 100); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 100); + + s.track_buffer_add(50); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 150); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 150); + } + + #[test] + fn buffer_release_decrements_bytes_without_touching_peak() { + let s = fresh(); + s.track_buffer_add(200); + s.track_buffer_release(80); + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 120); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 200); // peak unchanged + } + + #[test] + fn buffer_peak_does_not_decrease_after_release_and_smaller_add() { + let s = fresh(); + s.track_buffer_add(500); // peak → 500 + s.track_buffer_release(500); // current → 0 + s.track_buffer_add(10); // current → 10, peak stays 500 + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 10); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 500); + } + + #[test] + fn buffer_release_saturates_at_zero_instead_of_wrapping() { + let s = fresh(); + // Simulate a release arriving after a stats reset (lingering task). + s.track_buffer_release(100); // would have wrapped to u64::MAX without saturating_sub + assert_eq!(s.buffered_bytes.load(Ordering::Relaxed), 0); + assert_eq!(s.peak_buffered_bytes.load(Ordering::Relaxed), 0); + } + + // ── add_elapsed ─────────────────────────────────────────────────────────── + + #[test] + fn add_elapsed_accumulates_into_field() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.reader_setup_ns, start); + let after_first = s.reader_setup_ns.load(Ordering::Relaxed); + assert!(after_first > 0, "first add_elapsed should record > 0 ns"); + + let start2 = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.reader_setup_ns, start2); + let after_second = s.reader_setup_ns.load(Ordering::Relaxed); + assert!(after_second > after_first, "second add_elapsed should accumulate"); + } + + #[test] + fn add_elapsed_is_independent_per_field() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.fetch_decode_ns, start); + + // Unrelated fields stay at 0. + assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); + assert!(s.fetch_decode_ns.load(Ordering::Relaxed) > 0); + } + + // ── store_elapsed ───────────────────────────────────────────────────────── + + #[test] + fn store_elapsed_overwrites_previous_value() { + let s = fresh(); + s.pipeline_wall_ns.store(999_999_999, Ordering::Relaxed); // some large value + + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.store_elapsed(&s.pipeline_wall_ns, start); + let stored = s.pipeline_wall_ns.load(Ordering::Relaxed); + + // Stored value is the elapsed time, which is much less than 999_999_999 ns (1 s). + assert!(stored > 0); + assert!(stored < 999_999_999, "store_elapsed should overwrite, not accumulate"); + } + + #[test] + fn file_dispatch_wait_accumulates_via_add_elapsed() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.add_elapsed(&s.file_dispatch_wait_ns, start); + assert!(s.file_dispatch_wait_ns.load(Ordering::Relaxed) > 0); + // Other fields are unaffected. + assert_eq!(s.semaphore_wait_ns.load(Ordering::Relaxed), 0); + } + + #[test] + fn store_elapsed_does_not_affect_other_fields() { + let s = fresh(); + let start = Instant::now(); + std::thread::sleep(Duration::from_millis(2)); + s.store_elapsed(&s.pipeline_wall_ns, start); + + assert_eq!(s.reader_setup_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.fetch_decode_ns.load(Ordering::Relaxed), 0); + assert_eq!(s.serialize_ns.load(Ordering::Relaxed), 0); + } +} From 69ea668ee5e87fa6c7ef861288abca9729545af8 Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Wed, 29 Apr 2026 21:43:50 +0000 Subject: [PATCH 11/13] Implement with_file_prefetch_depth! end-to-end Adds a new scan configuration parameter that controls how many completed FileScan items are queued in the outer channel of the nested pipeline, decoupling file-I/O concurrency from how far ahead Rust prefetches for the Julia consumer. Previously the outer channel capacity was hardcoded to `file_concurrency`, coupling the two. Now they are independent: `file_concurrency` sets how many files are processed in parallel, while `file_prefetch_depth` controls how many ready FileScan items can buffer up before run_nested blocks. When `file_prefetch_depth` is 0 (default), it falls back to `file_concurrency` preserving existing behaviour. Changes: - IcebergScan/IcebergIncrementalScan: add file_prefetch_depth field - scan_common.rs: propagate field through all struct-reconstructing macros; add impl_with_file_prefetch_depth! macro - full.rs: wire iceberg_scan_with_file_prefetch_depth FFI function; resolve prefetch_depth in both iceberg_arrow_stream and iceberg_file_scan_stream - ordered_file_pipeline: add prefetch_depth parameter to create_nested_pipeline and create_pipeline; use it for the outer FileScan channel capacity - Julia: with_file_prefetch_depth! was already defined; move export to correct line Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/full.rs | 29 +++++++++++++++---- iceberg_rust_ffi/src/incremental.rs | 2 ++ iceberg_rust_ffi/src/ordered_file_pipeline.rs | 6 ++-- iceberg_rust_ffi/src/scan_common.rs | 21 ++++++++++++++ src/RustyIceberg.jl | 2 +- src/full.jl | 17 +++++++++++ 6 files changed, 68 insertions(+), 9 deletions(-) diff --git a/iceberg_rust_ffi/src/full.rs b/iceberg_rust_ffi/src/full.rs index 42b8707..eae8975 100644 --- a/iceberg_rust_ffi/src/full.rs +++ b/iceberg_rust_ffi/src/full.rs @@ -34,6 +34,9 @@ pub struct IcebergScan { /// How many parquet files to process concurrently in the pipeline. /// Set by with_data_file_concurrency_limit (0 = auto-detect). pub file_concurrency: usize, + /// How many FileScan items to queue in the outer channel of the nested pipeline. + /// 0 = use file_concurrency as the default. + pub file_prefetch_depth: usize, } unsafe impl Send for IcebergScan {} @@ -55,6 +58,7 @@ pub extern "C" fn iceberg_new_scan(table: *mut IcebergTable) -> *mut IcebergScan file_io, batch_size: None, file_concurrency: 0, + file_prefetch_depth: 0, })) } @@ -86,6 +90,7 @@ pub extern "C" fn iceberg_scan_with_data_file_concurrency_limit( file_io: scan_ref.file_io, batch_size: scan_ref.batch_size, file_concurrency: n, + file_prefetch_depth: scan_ref.file_prefetch_depth, })); CResult::Ok } @@ -117,6 +122,8 @@ impl_with_serialization_concurrency_limit!( IcebergScan ); +impl_with_file_prefetch_depth!(iceberg_scan_with_file_prefetch_depth, IcebergScan); + impl_scan_builder_method!( iceberg_scan_with_snapshot_id, IcebergScan, @@ -159,12 +166,17 @@ export_runtime_op!( let file_io = scan_ptr.file_io.clone(); let batch_size = scan_ptr.batch_size; + let prefetch_depth = if scan_ptr.file_prefetch_depth == 0 { + concurrency + } else { + scan_ptr.file_prefetch_depth + }; - Ok((scan_ref.as_ref().unwrap(), concurrency, file_io, batch_size)) + Ok((scan_ref.as_ref().unwrap(), concurrency, prefetch_depth, file_io, batch_size)) }, result_tuple, async { - let (scan_ref, concurrency, file_io, batch_size) = result_tuple; + let (scan_ref, concurrency, prefetch_depth, file_io, batch_size) = result_tuple; // Collect the ordered file task list from iceberg-rs. use futures::TryStreamExt; @@ -173,7 +185,7 @@ export_runtime_op!( // Hand off to the file-parallel pipeline. let stream = crate::ordered_file_pipeline::create_pipeline( - tasks, file_io, batch_size, concurrency, + tasks, file_io, batch_size, concurrency, prefetch_depth, ) .await?; @@ -215,19 +227,24 @@ export_runtime_op!( let file_io = scan_ptr.file_io.clone(); let batch_size = scan_ptr.batch_size; + let prefetch_depth = if scan_ptr.file_prefetch_depth == 0 { + concurrency + } else { + scan_ptr.file_prefetch_depth + }; - Ok((scan_ref.as_ref().unwrap(), concurrency, file_io, batch_size)) + Ok((scan_ref.as_ref().unwrap(), concurrency, prefetch_depth, file_io, batch_size)) }, result_tuple, async { - let (scan_ref, concurrency, file_io, batch_size) = result_tuple; + let (scan_ref, concurrency, prefetch_depth, file_io, batch_size) = result_tuple; use futures::TryStreamExt; let tasks: Vec = scan_ref.plan_files().await?.try_collect().await?; let stream = crate::ordered_file_pipeline::create_nested_pipeline( - tasks, file_io, batch_size, concurrency, + tasks, file_io, batch_size, concurrency, prefetch_depth, ) .await?; diff --git a/iceberg_rust_ffi/src/incremental.rs b/iceberg_rust_ffi/src/incremental.rs index 8a281c2..fff8b2c 100644 --- a/iceberg_rust_ffi/src/incremental.rs +++ b/iceberg_rust_ffi/src/incremental.rs @@ -26,6 +26,7 @@ pub struct IcebergIncrementalScan { pub file_io: Option, pub batch_size: Option, pub file_concurrency: usize, + pub file_prefetch_depth: usize, } unsafe impl Send for IcebergIncrementalScan {} @@ -109,6 +110,7 @@ pub extern "C" fn iceberg_new_incremental_scan( file_io: None, batch_size: None, file_concurrency: 0, + file_prefetch_depth: 0, })) } diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index b1593ec..724b13a 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -109,6 +109,7 @@ pub async fn create_nested_pipeline( file_io: FileIO, batch_size: Option, concurrency: usize, + prefetch_depth: usize, ) -> anyhow::Result { if concurrency > MAX_FILE_CONCURRENCY { anyhow::bail!( @@ -118,7 +119,7 @@ pub async fn create_nested_pipeline( STATS.reset(); - let (tx, rx) = mpsc::channel::>(concurrency); + let (tx, rx) = mpsc::channel::>(prefetch_depth); tokio::spawn(run_nested(tasks, file_io, batch_size, concurrency, tx)); @@ -141,11 +142,12 @@ pub async fn create_pipeline( file_io: FileIO, batch_size: Option, concurrency: usize, + prefetch_depth: usize, ) -> anyhow::Result { // Outer channel: flatten task → Julia (via IcebergArrowStream). let (tx, rx) = mpsc::channel(concurrency * 2); - let nested = create_nested_pipeline(tasks, file_io, batch_size, concurrency).await?; + let nested = create_nested_pipeline(tasks, file_io, batch_size, concurrency, prefetch_depth).await?; tokio::spawn(run_flat(nested, tx)); diff --git a/iceberg_rust_ffi/src/scan_common.rs b/iceberg_rust_ffi/src/scan_common.rs index f8c6737..a85645b 100644 --- a/iceberg_rust_ffi/src/scan_common.rs +++ b/iceberg_rust_ffi/src/scan_common.rs @@ -41,6 +41,7 @@ macro_rules! impl_select_columns { file_io: scan_ref.file_io, batch_size: scan_ref.batch_size, file_concurrency: scan_ref.file_concurrency, + file_prefetch_depth: scan_ref.file_prefetch_depth, })); CResult::Ok @@ -90,6 +91,7 @@ macro_rules! impl_scan_builder_method { file_io: scan_ref.file_io, batch_size: scan_ref.batch_size, file_concurrency: scan_ref.file_concurrency, + file_prefetch_depth: scan_ref.file_prefetch_depth, })); CResult::Ok @@ -120,6 +122,7 @@ macro_rules! impl_with_batch_size { file_io: scan_ref.file_io, batch_size: Some(n), file_concurrency: scan_ref.file_concurrency, + file_prefetch_depth: scan_ref.file_prefetch_depth, })); CResult::Ok @@ -150,6 +153,7 @@ macro_rules! impl_scan_build { file_io: scan_ref.file_io, batch_size: scan_ref.batch_size, file_concurrency: scan_ref.file_concurrency, + file_prefetch_depth: scan_ref.file_prefetch_depth, })); CResult::Ok } @@ -190,10 +194,27 @@ macro_rules! impl_with_serialization_concurrency_limit { }; } +/// Macro to generate with_file_prefetch_depth function for any scan type +macro_rules! impl_with_file_prefetch_depth { + ($fn_name:ident, $scan_type:ident) => { + #[no_mangle] + pub extern "C" fn $fn_name(scan: &mut *mut $scan_type, n: usize) -> CResult { + if scan.is_null() || (*scan).is_null() { + return CResult::Error; + } + let mut scan_ref = unsafe { Box::from_raw(*scan) }; + scan_ref.file_prefetch_depth = n; + *scan = Box::into_raw(scan_ref); + CResult::Ok + } + }; +} + // Re-export macros for use in other modules pub(crate) use impl_scan_build; pub(crate) use impl_scan_builder_method; pub(crate) use impl_scan_free; pub(crate) use impl_select_columns; pub(crate) use impl_with_batch_size; +pub(crate) use impl_with_file_prefetch_depth; pub(crate) use impl_with_serialization_concurrency_limit; diff --git a/src/RustyIceberg.jl b/src/RustyIceberg.jl index 881628d..4ed921c 100644 --- a/src/RustyIceberg.jl +++ b/src/RustyIceberg.jl @@ -15,7 +15,7 @@ export new_incremental_scan, free_incremental_scan! export table_open, free_table, new_scan, free_scan! export table_location, table_uuid, table_format_version, table_last_sequence_number, table_last_updated_ms, table_schema export select_columns!, with_batch_size!, with_data_file_concurrency_limit!, with_manifest_entry_concurrency_limit! -export with_file_column!, with_pos_column! +export with_file_column!, with_pos_column!, with_file_prefetch_depth! export scan!, next_batch, free_batch, free_stream export scan_nested!, nested_arrow_stream, next_file_scan export FileScanStream, file_scan_record_count, file_scan_filename, file_scan_arrow_stream diff --git a/src/full.jl b/src/full.jl index 479bed5..a29f074 100644 --- a/src/full.jl +++ b/src/full.jl @@ -206,6 +206,23 @@ function with_serialization_concurrency_limit!(scan::Scan, n::UInt) return nothing end +""" + with_file_prefetch_depth!(scan::Scan, n::UInt) + +Set how many FileScan tasks are queued ahead in the outer FileScanStream. +Higher values keep the Julia consumer busy but use more memory. +""" +function with_file_prefetch_depth!(scan::Scan, n::UInt) + result = GC.@preserve scan @ccall rust_lib.iceberg_scan_with_file_prefetch_depth( + convert(Ptr{Ptr{Cvoid}}, pointer_from_objref(scan))::Ptr{Ptr{Cvoid}}, + n::Csize_t + )::Cint + if result != 0 + throw(IcebergException("Failed to set file prefetch depth", result)) + end + return nothing +end + """ with_snapshot_id!(scan::Scan, snapshot_id::Int64) From eaa30d49bcd43bc888a7d0535a7a617d3067101e Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Thu, 30 Apr 2026 09:47:17 +0000 Subject: [PATCH 12/13] Address review comments: extract helper, fix pipeline_stats comment MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Extract resolve_pipeline_params() in full.rs to deduplicate the concurrency/prefetch_depth/file_io/batch_size resolution logic shared by iceberg_arrow_stream and iceberg_file_scan_stream sync blocks - Update pipeline_stats.rs header comment to remove "temporary / will be removed" framing — the module is intended to stay Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/src/full.rs | 61 ++++++++++---------------- iceberg_rust_ffi/src/pipeline_stats.rs | 7 ++- 2 files changed, 27 insertions(+), 41 deletions(-) diff --git a/iceberg_rust_ffi/src/full.rs b/iceberg_rust_ffi/src/full.rs index eae8975..aa6815d 100644 --- a/iceberg_rust_ffi/src/full.rs +++ b/iceberg_rust_ffi/src/full.rs @@ -141,6 +141,26 @@ impl_scan_builder_method!( // (ordered_file_pipeline.rs) which processes N files concurrently // but yields batches in strict file-then-row order. +/// Resolve the pipeline tuning parameters from a configured scan. +/// Returns `(concurrency, prefetch_depth, file_io, batch_size)`. +/// A stored value of 0 means "auto": concurrency defaults to available +/// parallelism; prefetch_depth defaults to concurrency. +fn resolve_pipeline_params(scan: &IcebergScan) -> (usize, usize, FileIO, Option) { + let concurrency = if scan.file_concurrency == 0 { + std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1) + } else { + scan.file_concurrency + }; + let prefetch_depth = if scan.file_prefetch_depth == 0 { + concurrency + } else { + scan.file_prefetch_depth + }; + (concurrency, prefetch_depth, scan.file_io.clone(), scan.batch_size) +} + export_runtime_op!( iceberg_arrow_stream, IcebergArrowStreamResponse, @@ -153,25 +173,8 @@ export_runtime_op!( if scan_ref.is_none() { return Err(anyhow::anyhow!("Scan not initialized")); } - - // File pipeline concurrency (0 = auto-detect from available CPUs) - let concurrency = scan_ptr.file_concurrency; - let concurrency = if concurrency == 0 { - std::thread::available_parallelism() - .map(|n| n.get()) - .unwrap_or(1) - } else { - concurrency - }; - - let file_io = scan_ptr.file_io.clone(); - let batch_size = scan_ptr.batch_size; - let prefetch_depth = if scan_ptr.file_prefetch_depth == 0 { - concurrency - } else { - scan_ptr.file_prefetch_depth - }; - + let (concurrency, prefetch_depth, file_io, batch_size) = + resolve_pipeline_params(scan_ptr); Ok((scan_ref.as_ref().unwrap(), concurrency, prefetch_depth, file_io, batch_size)) }, result_tuple, @@ -215,24 +218,8 @@ export_runtime_op!( if scan_ref.is_none() { return Err(anyhow::anyhow!("Scan not initialized")); } - - let concurrency = scan_ptr.file_concurrency; - let concurrency = if concurrency == 0 { - std::thread::available_parallelism() - .map(|n| n.get()) - .unwrap_or(1) - } else { - concurrency - }; - - let file_io = scan_ptr.file_io.clone(); - let batch_size = scan_ptr.batch_size; - let prefetch_depth = if scan_ptr.file_prefetch_depth == 0 { - concurrency - } else { - scan_ptr.file_prefetch_depth - }; - + let (concurrency, prefetch_depth, file_io, batch_size) = + resolve_pipeline_params(scan_ptr); Ok((scan_ref.as_ref().unwrap(), concurrency, prefetch_depth, file_io, batch_size)) }, result_tuple, diff --git a/iceberg_rust_ffi/src/pipeline_stats.rs b/iceberg_rust_ffi/src/pipeline_stats.rs index d65a168..252c007 100644 --- a/iceberg_rust_ffi/src/pipeline_stats.rs +++ b/iceberg_rust_ffi/src/pipeline_stats.rs @@ -1,8 +1,7 @@ // =========================================================================== -// Temporary profiling — will be removed before merging to production. -// -// Global atomic counters accumulate timing/size data across all file tasks. -// Called from Julia via `@ccall iceberg_print_pipeline_stats()`. +// Pipeline statistics — timing and throughput counters for the file-parallel +// scan pipeline. Global atomics accumulate data across all file tasks and are +// readable from Julia via `@ccall iceberg_print_pipeline_stats()`. // =========================================================================== use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering}; From b41d53923acb93f551d8a1d334091064169defad Mon Sep 17 00:00:00 2001 From: Robert Buessow Date: Thu, 30 Apr 2026 15:53:12 +0000 Subject: [PATCH 13/13] Address review comments: rayon serialization pool, Julia stats wrappers Replace `tokio::task::spawn_blocking` with a process-global `rayon::ThreadPool` (sized to available CPU cores) for Arrow IPC serialization. All concurrent file tasks share the same pool via `SERIALIZE_POOL`, keeping N worker threads warm for the lifetime of the process rather than borrowing from tokio's unbounded blocking pool on each batch. Also add `print_pipeline_stats()` and `reset_pipeline_stats()` Julia wrappers in `full.jl`, both exported from the module. Co-Authored-By: Claude Sonnet 4.6 (1M context) --- iceberg_rust_ffi/Cargo.lock | 31 +++++++++++++++++ iceberg_rust_ffi/Cargo.toml | 1 + iceberg_rust_ffi/src/ordered_file_pipeline.rs | 34 ++++++++++++++----- src/RustyIceberg.jl | 1 + src/full.jl | 18 ++++++++++ 5 files changed, 76 insertions(+), 9 deletions(-) diff --git a/iceberg_rust_ffi/Cargo.lock b/iceberg_rust_ffi/Cargo.lock index 09bd5de..e8e6d75 100644 --- a/iceberg_rust_ffi/Cargo.lock +++ b/iceberg_rust_ffi/Cargo.lock @@ -608,6 +608,16 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "crossbeam-deque" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + [[package]] name = "crossbeam-epoch" version = "0.9.18" @@ -1588,6 +1598,7 @@ dependencies = [ "object_store_ffi", "once_cell", "parquet", + "rayon", "serde_json", "tempfile", "tokio", @@ -2748,6 +2759,26 @@ dependencies = [ "bitflags", ] +[[package]] +name = "rayon" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb39b166781f92d482534ef4b4b1b2568f42613b53e5b6c160e24cfbfa30926d" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22e18b0f0062d30d4230b2e85ff77fdfe4326feb054b9783a3460d8435c8ab91" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + [[package]] name = "redox_syscall" version = "0.5.18" diff --git a/iceberg_rust_ffi/Cargo.toml b/iceberg_rust_ffi/Cargo.toml index 458fd9a..db55b36 100644 --- a/iceberg_rust_ffi/Cargo.toml +++ b/iceberg_rust_ffi/Cargo.toml @@ -32,6 +32,7 @@ async-trait = "0.1" # (0.5.11+ requires rustc 1.88+) home = "=0.5.9" serde_json = "1.0.142" +rayon = "1" [dev-dependencies] tempfile = "3.0" diff --git a/iceberg_rust_ffi/src/ordered_file_pipeline.rs b/iceberg_rust_ffi/src/ordered_file_pipeline.rs index 724b13a..d34e34c 100644 --- a/iceberg_rust_ffi/src/ordered_file_pipeline.rs +++ b/iceberg_rust_ffi/src/ordered_file_pipeline.rs @@ -12,7 +12,7 @@ //! Each file task is a background tokio task that: //! 1. Builds a per-file ArrowReader (same iceberg-rs code path as to_arrow) //! 2. Reads row groups sequentially → RecordBatch stream -//! 3. Serializes each batch to Arrow IPC (via spawn_blocking) +//! 3. Serializes each batch to Arrow IPC (via a shared rayon thread pool) //! 4. Sends serialized batches into a per-file mpsc channel //! //! A consumer (`run_nested`) uses FuturesUnordered to maintain N tasks in @@ -28,7 +28,7 @@ //! ~100MB independently. use std::sync::atomic::Ordering; -use std::sync::Arc; +use std::sync::{Arc, LazyLock}; use std::time::Instant; use futures::StreamExt; @@ -44,10 +44,23 @@ use crate::unexpected; /// Hard cap on file-level concurrency to keep total memory bounded. const MAX_FILE_CONCURRENCY: usize = 16; +/// Process-global rayon pool for Arrow IPC serialization. Sized to +/// MAX_FILE_CONCURRENCY so it can saturate the maximum file parallelism +/// without spinning up per-pipeline threads. +static SERIALIZE_POOL: LazyLock = LazyLock::new(|| { + let n = std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(MAX_FILE_CONCURRENCY); + rayon::ThreadPoolBuilder::new() + .num_threads(n) + .build() + .expect("failed to build serialization thread pool") +}); + /// Time an async expression and record its duration into a STATS field. /// /// ```ignore -/// let result = timed!(serialize_ns, { tokio::task::spawn_blocking(...) }) +/// let result = timed!(serialize_ns, { rayon_dispatch(&pool, batch) }) /// .map_err(...)?; /// ``` macro_rules! timed { @@ -338,12 +351,15 @@ async fn process_file_inner( }; // ── Phase 3: Serialize to Arrow IPC ───────────────────────────── - // CPU-bound work, offloaded to the blocking thread pool to avoid - // starving the tokio executor. - let serialized = timed!( - serialize_ns, - tokio::task::spawn_blocking(move || crate::serialize_record_batch(batch)) - ) + // Dispatched to the shared rayon pool so N idle workers handle all + // file tasks; the oneshot channel bridges rayon back to async. + let serialized = timed!(serialize_ns, { + let (stx, srx) = tokio::sync::oneshot::channel(); + SERIALIZE_POOL.spawn(move || { + let _ = stx.send(crate::serialize_record_batch(batch)); + }); + srx + }) .map_err(|e| unexpected(format!("serialize panicked: {e}")))? .map_err(|e| unexpected(e))?; diff --git a/src/RustyIceberg.jl b/src/RustyIceberg.jl index 4ed921c..fb016fe 100644 --- a/src/RustyIceberg.jl +++ b/src/RustyIceberg.jl @@ -20,6 +20,7 @@ export scan!, next_batch, free_batch, free_stream export scan_nested!, nested_arrow_stream, next_file_scan export FileScanStream, file_scan_record_count, file_scan_filename, file_scan_arrow_stream export free_file_scan!, free_file_scan_stream! +export print_pipeline_stats, reset_pipeline_stats export FILE_COLUMN, POS_COLUMN export Catalog, catalog_create_rest, free_catalog export load_table, list_tables, list_namespaces, table_exists, create_table, drop_table, drop_namespace, create_namespace diff --git a/src/full.jl b/src/full.jl index a29f074..35e2b24 100644 --- a/src/full.jl +++ b/src/full.jl @@ -441,3 +441,21 @@ Free the memory associated with a file scan stream. function free_file_scan_stream!(stream::FileScanStream) @ccall rust_lib.iceberg_file_scan_stream_free(stream::FileScanStream)::Cvoid end + +""" + print_pipeline_stats() + +Print a summary of file-parallel pipeline timing and throughput counters to stdout. +""" +function print_pipeline_stats() + @ccall rust_lib.iceberg_print_pipeline_stats()::Cvoid +end + +""" + reset_pipeline_stats() + +Reset all file-parallel pipeline statistics counters to zero. +""" +function reset_pipeline_stats() + @ccall rust_lib.iceberg_reset_pipeline_stats()::Cvoid +end