From 2a5a365cc8faa91ac1a053f4a829b1c94be2d6d7 Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Thu, 30 Oct 2025 15:08:25 +0100 Subject: [PATCH 1/7] feat(core): Add incremental scan for appends and positional deletes (#3) * WIP, initial draft of incremental scan * . * . * cargo fmt * Implement unzipped stream * Remove printlns * Add API method for unzipped stream * . * Remove comment * Rename var * Add import * Measure time * Fix typo * Undo some changes * Change type name * Add comment header * Fail when encountering equality deletes * Add comments * Add some preliminary tests * Format * Remove playground * Add more tests * Clippy * . * . * Adapt tests * . * Add test * Add tests * Add tests * Format * Add test * Format * . * Rm newline * Rename trait function * Reuse schema * . * remove clone * Add test for adding file_path column * Make `from_snapshot` mandatory * Error out if incremental scan encounters neither Append nor Delete * . * Add materialized variant of add_file_path_column * . * Allow dead code * Some PR comments * . * More PR comments * . * Add comments * Avoid cloning * Add reference to PR * Some PR comments * . * format * Allow overwrite operation for now * Fix file_path column * Add overwrite test * Unwrap delete vector * . * Add assertion * Avoid cloning the mutex guard * Abort when encountering a deleted delete file * Adjust comment * Update crates/iceberg/src/arrow/reader.rs Co-authored-by: Vukasin Stefanovic * Add check * Update crates/iceberg/src/scan/incremental/mod.rs --------- Co-authored-by: Vukasin Stefanovic --- crates/iceberg/src/arrow/delete_filter.rs | 35 +- crates/iceberg/src/arrow/incremental.rs | 271 +++ crates/iceberg/src/arrow/mod.rs | 2 + crates/iceberg/src/arrow/reader.rs | 463 ++++- crates/iceberg/src/delete_file_index.rs | 28 + crates/iceberg/src/delete_vector.rs | 4 +- crates/iceberg/src/lib.rs | 3 + crates/iceberg/src/scan/context.rs | 29 +- .../iceberg/src/scan/incremental/context.rs | 142 ++ crates/iceberg/src/scan/incremental/mod.rs | 588 ++++++ crates/iceberg/src/scan/incremental/task.rs | 106 + crates/iceberg/src/scan/incremental/tests.rs | 1703 +++++++++++++++++ crates/iceberg/src/scan/mod.rs | 3 + crates/iceberg/src/table.rs | 10 + crates/iceberg/src/util/mod.rs | 19 + crates/iceberg/src/util/snapshot.rs | 74 + 16 files changed, 3456 insertions(+), 24 deletions(-) create mode 100644 crates/iceberg/src/arrow/incremental.rs create mode 100644 crates/iceberg/src/scan/incremental/context.rs create mode 100644 crates/iceberg/src/scan/incremental/mod.rs create mode 100644 crates/iceberg/src/scan/incremental/task.rs create mode 100644 crates/iceberg/src/scan/incremental/tests.rs create mode 100644 crates/iceberg/src/util/mod.rs create mode 100644 crates/iceberg/src/util/snapshot.rs diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index b853baa993..e029912bbe 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -35,11 +35,22 @@ enum EqDelState { } #[derive(Debug, Default)] -struct DeleteFileFilterState { +pub(crate) struct DeleteFileFilterState { delete_vectors: HashMap>>, equality_deletes: HashMap, } +impl DeleteFileFilterState { + pub fn delete_vectors(&self) -> &HashMap>> { + &self.delete_vectors + } + + /// Remove and return the delete vector for the given data file path. + pub fn remove_delete_vector(&mut self, path: &str) -> Option>> { + self.delete_vectors.remove(path) + } +} + #[derive(Clone, Debug, Default)] pub(crate) struct DeleteFilter { state: Arc>, @@ -65,6 +76,28 @@ impl DeleteFilter { .and_then(|st| st.delete_vectors.get(delete_file_path).cloned()) } + pub(crate) fn with_read(&self, f: F) -> Result + where F: FnOnce(&DeleteFileFilterState) -> Result { + let state = self.state.read().map_err(|e| { + Error::new( + ErrorKind::Unexpected, + format!("Failed to acquire read lock: {}", e), + ) + })?; + f(&state) + } + + pub(crate) fn with_write(&self, f: F) -> Result + where F: FnOnce(&mut DeleteFileFilterState) -> Result { + let mut state = self.state.write().map_err(|e| { + Error::new( + ErrorKind::Unexpected, + format!("Failed to acquire write lock: {}", e), + ) + })?; + f(&mut state) + } + pub(crate) fn try_start_eq_del_load(&self, file_path: &str) -> Option> { let mut state = self.state.write().unwrap(); diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs new file mode 100644 index 0000000000..e7ca2521bf --- /dev/null +++ b/crates/iceberg/src/arrow/incremental.rs @@ -0,0 +1,271 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::pin::Pin; +use std::sync::Arc; + +use arrow_array::{RecordBatch, UInt64Array}; +use arrow_schema::{DataType, Field, Schema as ArrowSchema}; +use futures::channel::mpsc::channel; +use futures::stream::select; +use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; + +use crate::arrow::record_batch_transformer::RecordBatchTransformer; +use crate::arrow::{ + ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_FIELD_ID_FILE_PATH, StreamsInto, +}; +use crate::delete_vector::DeleteVector; +use crate::io::FileIO; +use crate::runtime::spawn; +use crate::scan::ArrowRecordBatchStream; +use crate::scan::incremental::{ + AppendedFileScanTask, IncrementalFileScanTask, IncrementalFileScanTaskStream, +}; +use crate::{Error, ErrorKind, Result}; + +/// The type of incremental batch: appended data or deleted records. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum IncrementalBatchType { + /// Appended records. + Append, + /// Deleted records. + Delete, +} + +/// The stream of incremental Arrow `RecordBatch`es with batch type. +pub type CombinedIncrementalBatchRecordStream = + Pin> + Send + 'static>>; + +/// Stream type for obtaining a separate stream of appended and deleted record batches. +pub type UnzippedIncrementalBatchRecordStream = (ArrowRecordBatchStream, ArrowRecordBatchStream); + +impl StreamsInto + for IncrementalFileScanTaskStream +{ + /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a + /// stream of Arrow `RecordBatch`es containing the data from the files. + fn stream(self, reader: ArrowReader) -> Result { + let (appends, deletes) = + StreamsInto::::stream(self, reader)?; + + let left = appends.map(|res| res.map(|batch| (IncrementalBatchType::Append, batch))); + let right = deletes.map(|res| res.map(|batch| (IncrementalBatchType::Delete, batch))); + + Ok(Box::pin(select(left, right)) as CombinedIncrementalBatchRecordStream) + } +} + +impl StreamsInto + for IncrementalFileScanTaskStream +{ + /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two + /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. + fn stream(self, reader: ArrowReader) -> Result { + let (appends_tx, appends_rx) = channel(reader.concurrency_limit_data_files); + let (deletes_tx, deletes_rx) = channel(reader.concurrency_limit_data_files); + + let batch_size = reader.batch_size; + let concurrency_limit_data_files = reader.concurrency_limit_data_files; + + spawn(async move { + let _ = self + .try_for_each_concurrent(concurrency_limit_data_files, |task| { + let file_io = reader.file_io.clone(); + let mut appends_tx = appends_tx.clone(); + let mut deletes_tx = deletes_tx.clone(); + async move { + match task { + IncrementalFileScanTask::Append(append_task) => { + spawn(async move { + let record_batch_stream = process_incremental_append_task( + append_task, + batch_size, + file_io, + ) + .await; + + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = appends_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read appended record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = appends_tx.send(Err(e)).await; + } + } + }); + } + IncrementalFileScanTask::Delete(file_path, delete_vector) => { + spawn(async move { + let record_batch_stream = process_incremental_delete_task( + file_path, + delete_vector, + batch_size, + ); + + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = deletes_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read deleted record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = deletes_tx.send(Err(e)).await; + } + } + }); + } + }; + + Ok(()) + } + }) + .await; + }); + + Ok(( + Box::pin(appends_rx) as ArrowRecordBatchStream, + Box::pin(deletes_rx) as ArrowRecordBatchStream, + )) + } +} + +async fn process_incremental_append_task( + task: AppendedFileScanTask, + batch_size: Option, + file_io: FileIO, +) -> Result { + let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( + &task.data_file_path, + file_io, + true, + ) + .await?; + + // Create a projection mask for the batch stream to select which columns in the + // Parquet file that we want in the response + let projection_mask = ArrowReader::get_arrow_projection_mask( + &task.project_field_ids, + &task.schema_ref(), + record_batch_stream_builder.parquet_schema(), + record_batch_stream_builder.schema(), + )?; + record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); + + // RecordBatchTransformer performs any transformations required on the RecordBatches + // that come back from the file, such as type promotion, default column insertion + // and column re-ordering + let mut record_batch_transformer = + RecordBatchTransformer::build(task.schema_ref(), &task.project_field_ids); + + if let Some(batch_size) = batch_size { + record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); + } + + // Apply positional deletes as row selections. + let row_selection = if let Some(positional_delete_indexes) = task.positional_deletes { + Some(ArrowReader::build_deletes_row_selection( + record_batch_stream_builder.metadata().row_groups(), + &None, + &positional_delete_indexes.lock().unwrap(), + )?) + } else { + None + }; + + if let Some(row_selection) = row_selection { + record_batch_stream_builder = record_batch_stream_builder.with_row_selection(row_selection); + } + + // Build the batch stream and send all the RecordBatches that it generates + // to the requester. + let record_batch_stream = record_batch_stream_builder + .build()? + .map(move |batch| match batch { + Ok(batch) => record_batch_transformer.process_record_batch(batch), + Err(err) => Err(err.into()), + }); + + Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream) +} + +fn process_incremental_delete_task( + file_path: String, + delete_vector: DeleteVector, + batch_size: Option, +) -> Result { + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "pos", + DataType::UInt64, + false, + )])); + + let batch_size = batch_size.unwrap_or(1024); + + let treemap = delete_vector.inner; + + let stream = futures::stream::iter(treemap) + .chunks(batch_size) + .map(move |chunk| { + let array = UInt64Array::from_iter(chunk); + RecordBatch::try_new( + Arc::clone(&schema), // Cheap Arc clone instead of full schema creation + vec![Arc::new(array)], + ) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RecordBatch for DeleteVector", + ) + }) + .and_then(|batch| { + ArrowReader::add_file_path_column( + batch, + &file_path, + RESERVED_COL_NAME_FILE_PATH, + RESERVED_FIELD_ID_FILE_PATH, + ) + }) + }); + + Ok(Box::pin(stream) as ArrowRecordBatchStream) +} diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index c091c45177..db85cd5730 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -33,6 +33,8 @@ pub mod record_batch_projector; pub(crate) mod record_batch_transformer; mod value; +mod incremental; +pub use incremental::*; pub use reader::*; pub use value::*; /// Partition value calculator for computing partition values diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index c6f5af2f2a..ebb5491a16 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -23,11 +23,14 @@ use std::str::FromStr; use std::sync::Arc; use arrow_arith::boolean::{and, and_kleene, is_not_null, is_null, not, or, or_kleene}; -use arrow_array::{Array, ArrayRef, BooleanArray, Datum as ArrowDatum, RecordBatch, Scalar}; +use arrow_array::{ + Array, ArrayRef, BooleanArray, Datum as ArrowDatum, Int32Array, RecordBatch, RunArray, Scalar, + StringArray, +}; use arrow_cast::cast::cast; use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq}; use arrow_schema::{ - ArrowError, DataType, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, + ArrowError, DataType, Field, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, }; use arrow_string::like::starts_with; use bytes::Bytes; @@ -59,6 +62,22 @@ use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; +/// Reserved field ID for the file path (_file) column per Iceberg spec +/// This is dead code for now but will be used when we add the _file column support. +#[allow(dead_code)] +pub(crate) const RESERVED_FIELD_ID_FILE: i32 = 2147483646; + +/// Column name for the file path metadata column per Iceberg spec +/// This is dead code for now but will be used when we add the _file column support. +#[allow(dead_code)] +pub(crate) const RESERVED_COL_NAME_FILE: &str = "_file"; + +/// Reserved field ID for the file path column used in delete file reading. +pub(crate) const RESERVED_FIELD_ID_FILE_PATH: i32 = 2147483546; + +/// Column name for the file path metadata column used in delete file reading. +pub(crate) const RESERVED_COL_NAME_FILE_PATH: &str = "file_path"; + /// Builder to create ArrowReader pub struct ArrowReaderBuilder { batch_size: Option, @@ -126,15 +145,22 @@ impl ArrowReaderBuilder { /// Reads data from Parquet files #[derive(Clone)] pub struct ArrowReader { - batch_size: Option, - file_io: FileIO, - delete_file_loader: CachingDeleteFileLoader, + pub(crate) batch_size: Option, + pub(crate) file_io: FileIO, + pub(crate) delete_file_loader: CachingDeleteFileLoader, /// the maximum number of data files that can be fetched at the same time - concurrency_limit_data_files: usize, + pub(crate) concurrency_limit_data_files: usize, - row_group_filtering_enabled: bool, - row_selection_enabled: bool, + pub(crate) row_group_filtering_enabled: bool, + pub(crate) row_selection_enabled: bool, +} + +/// Trait indicating that the implementing type streams into a stream of type `S` using +/// a reader of type `R`. +pub trait StreamsInto { + /// Stream from the reader and produce a stream of type `S`. + fn stream(self, reader: R) -> Result; } impl ArrowReader { @@ -378,7 +404,7 @@ impl ArrowReader { /// Using the Parquet page index, we build a `RowSelection` that rejects rows that are indicated /// as having been deleted by a positional delete, taking into account any row groups that have /// been skipped entirely by the filter predicate - fn build_deletes_row_selection( + pub(crate) fn build_deletes_row_selection( row_group_metadata_list: &[RowGroupMetaData], selected_row_groups: &Option>, positional_deletes: &DeleteVector, @@ -483,6 +509,103 @@ impl ArrowReader { Ok(results.into()) } + /// Helper function to add a `_file` column to a RecordBatch. + /// Takes the array and field to add, reducing code duplication. + fn create_file_field( + batch: RecordBatch, + file_array: ArrayRef, + file_field: Field, + field_id: i32, + ) -> Result { + let mut columns = batch.columns().to_vec(); + columns.push(file_array); + + let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); + fields.push(Arc::new(file_field.with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + field_id.to_string(), + )])))); + + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to add _file column to RecordBatch", + ) + .with_source(e) + }) + } + + /// Adds a `_file` column to the RecordBatch containing the file path. + /// Uses Run-End Encoding (REE) for maximum memory efficiency when the same + /// file path is repeated across all rows. + /// Note: This is only used in tests for now, for production usage we use the + /// non-REE version as it is Julia-compatible. + #[allow(dead_code)] + pub(crate) fn add_file_path_column_ree( + batch: RecordBatch, + file_path: &str, + field_name: &str, + field_id: i32, + ) -> Result { + let num_rows = batch.num_rows(); + + // Use Run-End Encoded array for optimal memory efficiency + // For a constant value repeated num_rows times, this stores: + // - run_ends: [num_rows] (one i32) for non-empty batches, or [] for empty batches + // - values: [file_path] (one string) for non-empty batches, or [] for empty batches + let run_ends = if num_rows == 0 { + Int32Array::from(Vec::::new()) + } else { + Int32Array::from(vec![num_rows as i32]) + }; + let values = if num_rows == 0 { + StringArray::from(Vec::<&str>::new()) + } else { + StringArray::from(vec![file_path]) + }; + + let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RunArray for _file column", + ) + .with_source(e) + })?; + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + // DataType is RunEndEncoded with Int32 run ends and Utf8 values + // Note: values field is nullable to match what RunArray::try_new(..) expects. + let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); + let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); + let file_field = Field::new( + field_name, + DataType::RunEndEncoded(run_ends_field, values_field), + false, + ); + + Self::create_file_field(batch, Arc::new(file_array), file_field, field_id) + } + + /// Adds a `_file` column to the RecordBatch containing the file path. + /// Materializes the file path string for each row (no compression). + pub(crate) fn add_file_path_column( + batch: RecordBatch, + file_path: &str, + field_name: &str, + field_id: i32, + ) -> Result { + let num_rows = batch.num_rows(); + + // Create a StringArray with the file path repeated num_rows times + let file_array = StringArray::from(vec![file_path; num_rows]); + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + let file_field = Field::new(field_name, DataType::Utf8, false); + + Self::create_file_field(batch, Arc::new(file_array), file_field, field_id) + } + fn build_field_id_set_and_map( parquet_schema: &SchemaDescriptor, predicate: &BoundPredicate, @@ -521,7 +644,7 @@ impl ArrowReader { } } - fn get_arrow_projection_mask( + pub(crate) fn get_arrow_projection_mask( field_ids: &[i32], iceberg_schema_of_task: &Schema, parquet_schema: &SchemaDescriptor, @@ -1479,6 +1602,7 @@ mod tests { use arrow_array::cast::AsArray; use arrow_array::{ArrayRef, LargeStringArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema, TimeUnit}; + use as_any::Downcast; use futures::TryStreamExt; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::arrow::{ArrowWriter, ProjectionMask}; @@ -1492,7 +1616,9 @@ mod tests { use crate::ErrorKind; use crate::arrow::reader::{CollectFieldIdVisitor, PARQUET_FIELD_ID_META_KEY}; - use crate::arrow::{ArrowReader, ArrowReaderBuilder}; + use crate::arrow::{ + ArrowReader, ArrowReaderBuilder, RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE, + }; use crate::delete_vector::DeleteVector; use crate::expr::visitors::bound_predicate_visitor::visit; use crate::expr::{Bind, Predicate, Reference}; @@ -2286,4 +2412,319 @@ message schema { assert!(col_b.is_null(1)); assert!(col_b.is_null(2)); } + + #[test] + fn test_add_file_path_column_ree() { + use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + // Create a simple test batch with 2 columns and 3 rows + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let id_array = Int32Array::from(vec![1, 2, 3]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Charlie"]); + + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(id_array), + Arc::new(name_array), + ]) + .unwrap(); + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Add file path column with REE + let file_path = "/path/to/data/file.parquet"; + let result = ArrowReader::add_file_path_column_ree( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + assert!(result.is_ok(), "Should successfully add file path column"); + + let new_batch = result.unwrap(); + + // Verify the new batch has 3 columns + assert_eq!(new_batch.num_columns(), 3); + assert_eq!(new_batch.num_rows(), 3); + + // Verify schema has the _file column + let schema = new_batch.schema(); + assert_eq!(schema.fields().len(), 3); + + let file_field = schema.field(2); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + assert!(!file_field.is_nullable()); + + // Verify the field has the correct metadata + let metadata = file_field.metadata(); + assert_eq!( + metadata.get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the data type is RunEndEncoded + match file_field.data_type() { + DataType::RunEndEncoded(run_ends_field, values_field) => { + assert_eq!(run_ends_field.name(), "run_ends"); + assert_eq!(run_ends_field.data_type(), &DataType::Int32); + assert!(!run_ends_field.is_nullable()); + + assert_eq!(values_field.name(), "values"); + assert_eq!(values_field.data_type(), &DataType::Utf8); + } + _ => panic!("Expected RunEndEncoded data type for _file column"), + } + + // Verify the original columns are intact + let id_col = new_batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.values(), &[1, 2, 3]); + + let name_col = new_batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + + // Verify the file path column contains the correct value + // The _file column is a RunArray, so we need to decode it + let file_col = new_batch.column(2); + let run_array = file_col + .as_any() + .downcast_ref::>() + .expect("Expected RunArray for _file column"); + + // Verify the run array structure (should be optimally encoded) + let run_ends = run_array.run_ends(); + assert_eq!(run_ends.values().len(), 1, "Should have only 1 run end"); + assert_eq!( + run_ends.values()[0], + new_batch.num_rows() as i32, + "Run end should equal number of rows" + ); + + // Check that the single value in the RunArray is the expected file path + let values = run_array.values(); + let string_values = values.as_string::(); + assert_eq!(string_values.len(), 1, "Should have only 1 value"); + assert_eq!(string_values.value(0), file_path); + + assert!( + string_values + .downcast_ref::() + .unwrap() + .iter() + .all(|v| v == Some(file_path)) + ) + } + + #[test] + fn test_add_file_path_column_ree_empty_batch() { + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + // Create an empty batch + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = arrow_array::Int32Array::from(Vec::::new()); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + assert_eq!(batch.num_rows(), 0); + + // Add file path column to empty batch with REE + let file_path = "/empty/file.parquet"; + let result = ArrowReader::add_file_path_column_ree( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + + // Should succeed with empty RunArray for empty batches + assert!(result.is_ok()); + let new_batch = result.unwrap(); + assert_eq!(new_batch.num_rows(), 0); + assert_eq!(new_batch.num_columns(), 2); + + // Verify the _file column exists with correct schema + let schema = new_batch.schema(); + let file_field = schema.field(1); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + + // Should use RunEndEncoded even for empty batches + match file_field.data_type() { + DataType::RunEndEncoded(run_ends_field, values_field) => { + assert_eq!(run_ends_field.data_type(), &DataType::Int32); + assert_eq!(values_field.data_type(), &DataType::Utf8); + } + _ => panic!("Expected RunEndEncoded data type for _file column"), + } + + // Verify metadata with reserved field ID + assert_eq!( + file_field.metadata().get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the file path column is empty but properly structured + let file_path_column = new_batch.column(1); + assert_eq!(file_path_column.len(), 0); + } + + #[test] + fn test_add_file_path_column_special_characters() { + use arrow_array::{Int32Array, RecordBatch}; + use arrow_schema::{DataType, Field, Schema}; + + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = Int32Array::from(vec![42]); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + // Test with file path containing special characters (materialized version) + let file_path = "/path/with spaces/and-dashes/file_name.parquet"; + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + assert!(result.is_ok()); + + let new_batch = result.unwrap(); + let file_col = new_batch.column(1); + + // Verify the file path is correctly stored as a materialized StringArray + let str_arr = file_col.as_string::(); + assert_eq!(str_arr.value(0), file_path); + } + + #[test] + fn test_add_file_path_column() { + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + // Create a simple test batch with 2 columns and 3 rows + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let id_array = Int32Array::from(vec![1, 2, 3]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Charlie"]); + + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(id_array), + Arc::new(name_array), + ]) + .unwrap(); + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Add file path column with materialization + let file_path = "/path/to/data/file.parquet"; + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + assert!(result.is_ok(), "Should successfully add file path column"); + + let new_batch = result.unwrap(); + + // Verify the new batch has 3 columns + assert_eq!(new_batch.num_columns(), 3); + assert_eq!(new_batch.num_rows(), 3); + + // Verify schema has the _file column + let schema = new_batch.schema(); + assert_eq!(schema.fields().len(), 3); + + let file_field = schema.field(2); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + assert!(!file_field.is_nullable()); + + // Verify the field has the correct metadata + let metadata = file_field.metadata(); + assert_eq!( + metadata.get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the data type is Utf8 (materialized strings) + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify the original columns are intact + let id_col = new_batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.values(), &[1, 2, 3]); + + let name_col = new_batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + + // Verify the file path column contains the correct value for all rows + let file_col = new_batch.column(2).as_string::(); + for i in 0..new_batch.num_rows() { + assert_eq!(file_col.value(i), file_path); + } + } + + #[test] + fn test_add_file_path_column_empty_batch() { + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + // Create an empty batch + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = arrow_array::Int32Array::from(Vec::::new()); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + assert_eq!(batch.num_rows(), 0); + + // Add file path column to empty batch (materialized version) + let file_path = "/empty/file.parquet"; + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + + // Should succeed with empty StringArray + assert!(result.is_ok()); + let new_batch = result.unwrap(); + assert_eq!(new_batch.num_rows(), 0); + assert_eq!(new_batch.num_columns(), 2); + + // Verify the _file column exists with correct schema + let schema = new_batch.schema(); + let file_field = schema.field(1); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + + // Should use Utf8 (materialized strings) + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify metadata with reserved field ID + assert_eq!( + file_field.metadata().get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the file path column is empty but properly structured + let file_path_column = new_batch.column(1); + assert_eq!(file_path_column.len(), 0); + } } diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index 4f6fd28483..3a3dfbb529 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -107,6 +107,26 @@ impl DeleteFileIndex { _ => unreachable!("Cannot be any other state than loaded"), } } + + pub(crate) async fn positional_deletes(&self) -> Vec { + let notifier = { + let guard = self.state.read().unwrap(); + match *guard { + DeleteFileIndexState::Populating(ref notifier) => notifier.clone(), + DeleteFileIndexState::Populated(ref index) => { + return index.positional_deletes(); + } + } + }; + + notifier.notified().await; + + let guard = self.state.read().unwrap(); + match guard.deref() { + DeleteFileIndexState::Populated(index) => index.positional_deletes(), + _ => unreachable!("Cannot be any other state than loaded"), + } + } } impl PopulatedDeleteFileIndex { @@ -210,6 +230,14 @@ impl PopulatedDeleteFileIndex { results } + + fn positional_deletes(&self) -> Vec { + self.pos_deletes_by_partition + .values() + .flatten() + .map(|ctx| ctx.as_ref().into()) + .collect() + } } #[cfg(test)] diff --git a/crates/iceberg/src/delete_vector.rs b/crates/iceberg/src/delete_vector.rs index f382bf079e..1040796034 100644 --- a/crates/iceberg/src/delete_vector.rs +++ b/crates/iceberg/src/delete_vector.rs @@ -23,9 +23,9 @@ use roaring::treemap::BitmapIter; use crate::{Error, ErrorKind, Result}; -#[derive(Debug, Default)] +#[derive(Debug, Default, Clone)] pub struct DeleteVector { - inner: RoaringTreemap, + pub inner: RoaringTreemap, } impl DeleteVector { diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs index aae8efed74..9c9c7460fb 100644 --- a/crates/iceberg/src/lib.rs +++ b/crates/iceberg/src/lib.rs @@ -97,3 +97,6 @@ pub mod writer; mod delete_vector; pub mod puffin; + +/// Utility functions and modules. +pub mod util; diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 3f7c29dbf4..e35c260be4 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -33,19 +33,23 @@ use crate::spec::{ }; use crate::{Error, ErrorKind, Result}; +pub(crate) type ManifestEntryFilterFn = dyn Fn(&ManifestEntryRef) -> bool + Send + Sync; + /// Wraps a [`ManifestFile`] alongside the objects that are needed /// to process it in a thread-safe manner pub(crate) struct ManifestFileContext { - manifest_file: ManifestFile, + pub manifest_file: ManifestFile, - sender: Sender, + pub sender: Sender, - field_ids: Arc>, - bound_predicates: Option>, - object_cache: Arc, - snapshot_schema: SchemaRef, - expression_evaluator_cache: Arc, - delete_file_index: DeleteFileIndex, + pub field_ids: Arc>, + pub bound_predicates: Option>, + pub object_cache: Arc, + pub snapshot_schema: SchemaRef, + pub expression_evaluator_cache: Arc, + pub delete_file_index: DeleteFileIndex, + + pub filter_fn: Option>, } /// Wraps a [`ManifestEntryRef`] alongside the objects that are needed @@ -74,12 +78,15 @@ impl ManifestFileContext { mut sender, expression_evaluator_cache, delete_file_index, + filter_fn, .. } = self; + let filter_fn = filter_fn.unwrap_or_else(|| Arc::new(|_| true)); + let manifest = object_cache.get_manifest(&manifest_file).await?; - for manifest_entry in manifest.entries() { + for manifest_entry in manifest.entries().iter().filter(|e| filter_fn(e)) { let manifest_entry_context = ManifestEntryContext { // TODO: refactor to avoid the expensive ManifestEntry clone manifest_entry: manifest_entry.clone(), @@ -189,7 +196,6 @@ impl PlanContext { ) -> Result> + 'static>> { let manifest_files = manifest_list.entries().iter(); - // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut filtered_mfcs = vec![]; for manifest_file in manifest_files { let tx = if manifest_file.content == ManifestContentType::Deletes { @@ -224,6 +230,7 @@ impl PlanContext { partition_bound_predicate, tx, delete_file_idx.clone(), + None, ); filtered_mfcs.push(Ok(mfc)); @@ -238,6 +245,7 @@ impl PlanContext { partition_filter: Option>, sender: Sender, delete_file_index: DeleteFileIndex, + filter_fn: Option>, ) -> ManifestFileContext { let bound_predicates = if let (Some(ref partition_bound_predicate), Some(snapshot_bound_predicate)) = @@ -260,6 +268,7 @@ impl PlanContext { field_ids: self.field_ids.clone(), expression_evaluator_cache: self.expression_evaluator_cache.clone(), delete_file_index, + filter_fn, } } } diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs new file mode 100644 index 0000000000..04db28a4c4 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -0,0 +1,142 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashSet; +use std::sync::Arc; + +use futures::channel::mpsc::Sender; + +use crate::Result; +use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; +use crate::delete_file_index::DeleteFileIndex; +use crate::io::object_cache::ObjectCache; +use crate::scan::ExpressionEvaluatorCache; +use crate::scan::context::{ManifestEntryContext, ManifestEntryFilterFn, ManifestFileContext}; +use crate::spec::{ + ManifestContentType, ManifestEntryRef, ManifestFile, Operation, SchemaRef, SnapshotRef, + TableMetadataRef, +}; + +#[derive(Debug)] +pub(crate) struct IncrementalPlanContext { + /// The snapshots involved in the incremental scan. + pub snapshots: Vec, + + /// The snapshot to start the incremental scan from. + pub from_snapshot: SnapshotRef, + + /// The metadata of the table being scanned. + pub table_metadata: TableMetadataRef, + + /// The schema of the snapshot to end the incremental scan at. + pub to_snapshot_schema: SchemaRef, + + /// The object cache to use for the scan. + pub object_cache: Arc, + + /// The field IDs to scan. + pub field_ids: Arc>, + + /// The expression evaluator cache to use for the scan. + pub expression_evaluator_cache: Arc, + + /// The caching delete file loader to use for the scan. + pub caching_delete_file_loader: CachingDeleteFileLoader, +} + +impl IncrementalPlanContext { + pub(crate) async fn build_manifest_file_contexts( + &self, + tx_data: Sender, + delete_file_idx: DeleteFileIndex, + delete_file_tx: Sender, + ) -> Result> + 'static>> { + // Validate that all snapshots are Append or Delete operations and collect their IDs + let snapshot_ids: HashSet = { + let mut ids = HashSet::new(); + for snapshot in self.snapshots.iter() { + let operation = &snapshot.summary().operation; + if !matches!( + operation, + Operation::Append | Operation::Overwrite | Operation::Delete + ) { + return Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + format!( + "Incremental scan only supports Append, Overwrite and Delete operations, but snapshot {} has operation {:?}", + snapshot.snapshot_id(), + operation + ), + )); + } + ids.insert(snapshot.snapshot_id()); + } + ids + }; + + let (manifest_files, filter_fn) = { + let mut manifest_files = HashSet::::new(); + for snapshot in self.snapshots.iter() { + let manifest_list = self + .object_cache + .get_manifest_list(snapshot, &self.table_metadata) + .await?; + for entry in manifest_list.entries() { + if !snapshot_ids.contains(&entry.added_snapshot_id) { + continue; + } + manifest_files.insert(entry.clone()); + } + } + let filter_fn: Option> = + Some(Arc::new(move |entry: &ManifestEntryRef| { + entry + .snapshot_id() + .map(|id| snapshot_ids.contains(&id)) + .unwrap_or(true) // Include entries without `snapshot_id`. + })); + + (manifest_files, filter_fn) + }; + + // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. + let mut mfcs = vec![]; + for manifest_file in &manifest_files { + let tx = if manifest_file.content == ManifestContentType::Deletes { + delete_file_tx.clone() + } else { + tx_data.clone() + }; + + let mfc = ManifestFileContext { + manifest_file: manifest_file.clone(), + bound_predicates: None, + sender: tx, + object_cache: self.object_cache.clone(), + snapshot_schema: self.to_snapshot_schema.clone(), + field_ids: self.field_ids.clone(), + expression_evaluator_cache: self.expression_evaluator_cache.clone(), + delete_file_index: delete_file_idx.clone(), + filter_fn: filter_fn.clone(), + }; + + mfcs.push(Ok(mfc)); + } + + Ok(Box::new(mfcs.into_iter())) + } +} diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs new file mode 100644 index 0000000000..1253353b4d --- /dev/null +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -0,0 +1,588 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Incremental table scan implementation. + +use std::collections::HashSet; +use std::sync::Arc; + +use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; +use crate::arrow::delete_filter::DeleteFilter; +use crate::arrow::{ + ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, StreamsInto, + UnzippedIncrementalBatchRecordStream, +}; +use crate::delete_file_index::DeleteFileIndex; +use crate::io::FileIO; +use crate::scan::DeleteFileContext; +use crate::scan::cache::ExpressionEvaluatorCache; +use crate::scan::context::ManifestEntryContext; +use crate::spec::{DataContentType, ManifestStatus, Snapshot, SnapshotRef}; +use crate::table::Table; +use crate::util::snapshot::ancestors_between; +use crate::utils::available_parallelism; +use crate::{Error, ErrorKind, Result}; + +mod context; +use context::*; +mod task; +use futures::channel::mpsc::{Sender, channel}; +use futures::{SinkExt, StreamExt, TryStreamExt}; +use itertools::Itertools; +pub use task::*; + +use crate::runtime::spawn; + +/// Builder for an incremental table scan. +#[derive(Debug)] +pub struct IncrementalTableScanBuilder<'a> { + table: &'a Table, + // Defaults to `None`, which means all columns. + column_names: Option>, + from_snapshot_id: i64, + to_snapshot_id: i64, + batch_size: Option, + concurrency_limit_data_files: usize, + concurrency_limit_manifest_entries: usize, + concurrency_limit_manifest_files: usize, +} + +impl<'a> IncrementalTableScanBuilder<'a> { + pub(crate) fn new(table: &'a Table, from_snapshot_id: i64, to_snapshot_id: i64) -> Self { + let num_cpus = available_parallelism().get(); + Self { + table, + column_names: None, + from_snapshot_id, + to_snapshot_id, + batch_size: None, + concurrency_limit_data_files: num_cpus, + concurrency_limit_manifest_entries: num_cpus, + concurrency_limit_manifest_files: num_cpus, + } + } + + /// Set the batch size for reading data files. + pub fn with_batch_size(mut self, batch_size: Option) -> Self { + self.batch_size = batch_size; + self + } + + /// Select all columns of the table. + pub fn select_all(mut self) -> Self { + self.column_names = None; + self + } + + /// Select no columns of the table. + pub fn select_empty(mut self) -> Self { + self.column_names = Some(vec![]); + self + } + + /// Select some columns of the table. + pub fn select(mut self, column_names: impl IntoIterator) -> Self { + self.column_names = Some( + column_names + .into_iter() + .map(|item| item.to_string()) + .collect(), + ); + self + } + + /// Set the `from_snapshot_id` for the incremental scan. + pub fn from_snapshot_id(mut self, from_snapshot_id: i64) -> Self { + self.from_snapshot_id = from_snapshot_id; + self + } + + /// Set the `to_snapshot_id` for the incremental scan. + pub fn to_snapshot_id(mut self, to_snapshot_id: i64) -> Self { + self.to_snapshot_id = to_snapshot_id; + self + } + + /// Set the concurrency limit for reading data files. + pub fn with_concurrency_limit_data_files(mut self, limit: usize) -> Self { + self.concurrency_limit_data_files = limit; + self + } + + /// Set the concurrency limit for reading manifest entries. + pub fn with_concurrency_limit_manifest_entries(mut self, limit: usize) -> Self { + self.concurrency_limit_manifest_entries = limit; + self + } + + /// Set the concurrency limit for reading manifest files. + pub fn with_concurrency_limit_manifest_files(mut self, limit: usize) -> Self { + self.concurrency_limit_manifest_files = limit; + self + } + + /// Build the incremental table scan. + pub fn build(self) -> Result { + let snapshot_from: Arc = self + .table + .metadata() + .snapshot_by_id(self.from_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", self.from_snapshot_id), + ) + })? + .clone(); + + let snapshot_to: Arc = self + .table + .metadata() + .snapshot_by_id(self.to_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", self.to_snapshot_id), + ) + })? + .clone(); + + let snapshots = ancestors_between( + &self.table.metadata_ref(), + snapshot_to.snapshot_id(), + Some(snapshot_from.snapshot_id()), + ) + .collect_vec(); + + if !snapshots.is_empty() { + assert_eq!( + snapshots.first().map(|s| s.snapshot_id()), + Some(snapshot_to.snapshot_id()) + ); + } + + let schema = snapshot_to.schema(self.table.metadata())?; + + if let Some(column_names) = self.column_names.as_ref() { + for column_name in column_names { + if schema.field_by_name(column_name).is_none() { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Column {} not found in table. Schema: {}", + column_name, schema + ), + )); + } + } + } + + let mut field_ids = vec![]; + let column_names = self.column_names.clone().unwrap_or_else(|| { + schema + .as_struct() + .fields() + .iter() + .map(|f| f.name.clone()) + .collect() + }); + + for column_name in column_names.iter() { + let field_id = schema.field_id_by_name(column_name).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Column {} not found in table. Schema: {}", + column_name, schema + ), + ) + })?; + + schema + .as_struct() + .field_by_id(field_id) + .ok_or_else(|| { + Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Column {} is not a direct child of schema but a nested field, which is not supported now. Schema: {}", + column_name, schema + ), + ) + })?; + + field_ids.push(field_id); + } + + let plan_context = IncrementalPlanContext { + snapshots, + from_snapshot: snapshot_from, + table_metadata: self.table.metadata_ref(), + to_snapshot_schema: schema, + object_cache: self.table.object_cache().clone(), + field_ids: Arc::new(field_ids), + expression_evaluator_cache: Arc::new(ExpressionEvaluatorCache::new()), + caching_delete_file_loader: CachingDeleteFileLoader::new( + self.table.file_io().clone(), + self.concurrency_limit_data_files, + ), + }; + + Ok(IncrementalTableScan { + plan_context, + file_io: self.table.file_io().clone(), + column_names: Some(column_names), + batch_size: self.batch_size, + concurrency_limit_data_files: self.concurrency_limit_data_files, + concurrency_limit_manifest_entries: self.concurrency_limit_manifest_entries, + concurrency_limit_manifest_files: self.concurrency_limit_manifest_files, + }) + } +} + +/// An incremental table scan. +#[derive(Debug)] +pub struct IncrementalTableScan { + plan_context: IncrementalPlanContext, + file_io: FileIO, + column_names: Option>, + batch_size: Option, + concurrency_limit_data_files: usize, + concurrency_limit_manifest_entries: usize, + concurrency_limit_manifest_files: usize, +} + +impl IncrementalTableScan { + /// Returns the `from` snapshot of this incremental table scan. + pub fn snapshot_from(&self) -> &SnapshotRef { + &self.plan_context.from_snapshot + } + + /// Returns the snapshots involved in this incremental table scan. + pub fn snapshots(&self) -> &[SnapshotRef] { + &self.plan_context.snapshots + } + + /// Returns the `to` snapshot of this incremental table scan. + pub fn snapshot_to(&self) -> &SnapshotRef { + self.snapshots() + .first() + .expect("There is always at least one snapshot") + } + + /// Returns the selected column names of this incremental table scan. + /// If `None`, all columns are selected. + pub fn column_names(&self) -> Option<&[String]> { + self.column_names.as_deref() + } + + /// Plans the files to be read in this incremental table scan. + pub async fn plan_files(&self) -> Result { + let concurrency_limit_manifest_files = self.concurrency_limit_manifest_files; + let concurrency_limit_manifest_entries = self.concurrency_limit_manifest_entries; + + // Used to stream `ManifestEntryContexts` between stages of the planning operation. + let (manifest_entry_data_ctx_tx, manifest_entry_data_ctx_rx) = + channel(concurrency_limit_manifest_files); + let (manifest_entry_delete_ctx_tx, manifest_entry_delete_ctx_rx) = + channel(concurrency_limit_manifest_files); + + // Used to stream the results back to the caller. + let (file_scan_task_tx, file_scan_task_rx) = channel(concurrency_limit_manifest_entries); + + let (delete_file_idx, delete_file_tx) = DeleteFileIndex::new(); + + let manifest_file_contexts = self + .plan_context + .build_manifest_file_contexts( + manifest_entry_data_ctx_tx, + delete_file_idx.clone(), + manifest_entry_delete_ctx_tx, + ) + .await?; + + let mut channel_for_manifest_error: Sender> = file_scan_task_tx.clone(); + + // Concurrently load all [`Manifest`]s and stream their [`ManifestEntry`]s + spawn(async move { + let result = futures::stream::iter(manifest_file_contexts) + .try_for_each_concurrent(concurrency_limit_manifest_files, |ctx| async move { + ctx.fetch_manifest_and_stream_manifest_entries().await + }) + .await; + + if let Err(error) = result { + let _ = channel_for_manifest_error.send(Err(error)).await; + } + }); + + let mut channel_for_data_manifest_entry_error = file_scan_task_tx.clone(); + let mut channel_for_delete_manifest_entry_error = file_scan_task_tx.clone(); + + // Process the delete file [`ManifestEntry`] stream in parallel. Builds the delete + // index below. + spawn(async move { + let result = manifest_entry_delete_ctx_rx + .map(|me_ctx| Ok((me_ctx, delete_file_tx.clone()))) + .try_for_each_concurrent( + concurrency_limit_manifest_entries, + |(manifest_entry_context, tx)| async move { + spawn(async move { + Self::process_delete_manifest_entry(tx, manifest_entry_context).await + }) + .await + }, + ) + .await; + + if let Err(error) = result { + let _ = channel_for_delete_manifest_entry_error + .send(Err(error)) + .await; + } + }) + .await; + + // TODO: Streaming this into the delete index seems somewhat redundant, as we + // could directly stream into the CachingDeleteFileLoader and instantly load the + // delete files. + let positional_deletes = delete_file_idx.positional_deletes().await; + let result = self + .plan_context + .caching_delete_file_loader + .load_deletes( + &positional_deletes, + self.plan_context.to_snapshot_schema.clone(), + ) + .await; + + // Build the delete filter from the loaded deletes. + let delete_filter = match result { + Ok(loaded_deletes) => loaded_deletes.unwrap(), + Err(e) => { + return Err(Error::new( + ErrorKind::Unexpected, + format!("Failed to load positional deletes: {}", e), + )); + } + }; + + // Process the data file [`ManifestEntry`] stream in parallel + let filter = delete_filter.clone(); + spawn(async move { + let result = manifest_entry_data_ctx_rx + .map(|me_ctx| Ok((me_ctx, file_scan_task_tx.clone()))) + .try_for_each_concurrent( + concurrency_limit_manifest_entries, + |(manifest_entry_context, tx)| { + let filter = filter.clone(); + async move { + if manifest_entry_context.manifest_entry.status() + == ManifestStatus::Added + { + spawn(async move { + Self::process_data_manifest_entry( + tx, + manifest_entry_context, + &filter, + ) + .await + }) + .await + } else if manifest_entry_context.manifest_entry.status() + == ManifestStatus::Deleted + { + // TODO (RAI-43291): Process deleted files + Err(Error::new( + ErrorKind::FeatureUnsupported, + "Processing deleted data files is not supported yet in incremental scans", + )) + } else { + Ok(()) + } + } + }, + ) + .await; + + if let Err(error) = result { + let _ = channel_for_data_manifest_entry_error.send(Err(error)).await; + } + }); + + // Collect all append tasks. + let mut tasks = file_scan_task_rx.try_collect::>().await?; + + // Compute those file paths that have been appended. + let appended_files = tasks + .iter() + .filter_map(|task| match task { + IncrementalFileScanTask::Append(append_task) => { + Some(append_task.data_file_path.clone()) + } + _ => None, + }) + .collect::>(); + + // Augment `tasks` with delete tasks. + // First collect paths to process (paths that weren't appended in this scan range) + let delete_paths: Vec = delete_filter.with_read(|state| { + Ok(state + .delete_vectors() + .keys() + .filter(|path| !appended_files.contains::(path)) + .cloned() + .collect()) + })?; + + // Now remove and take ownership of each delete vector + for path in delete_paths { + let delete_vector_arc = delete_filter.with_write(|state| { + state.remove_delete_vector(&path).ok_or_else(|| { + Error::new( + ErrorKind::Unexpected, + format!("DeleteVector for path {} not found", path), + ) + }) + })?; + + // Try to unwrap the Arc to avoid cloning the DeleteVector + let delete_vector_inner = Arc::try_unwrap(delete_vector_arc) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "DeleteVector Arc has multiple references, cannot take ownership", + ) + })? + .into_inner() + .map_err(|e| { + Error::new(ErrorKind::Unexpected, "Failed to unwrap DeleteVector Mutex") + .with_source(e) + })?; + + let delete_task = IncrementalFileScanTask::Delete(path, delete_vector_inner); + tasks.push(delete_task); + } + + // We actually would not need a stream here, but we can keep it compatible with + // other scan types. + Ok(futures::stream::iter(tasks).map(Ok).boxed()) + } + + /// Returns an [`CombinedIncrementalBatchRecordStream`] for this incremental table scan. + pub async fn to_arrow(&self) -> Result { + let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); + + if let Some(batch_size) = self.batch_size { + arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); + } + + let arrow_reader = arrow_reader_builder.build(); + let file_scan_task_stream = self.plan_files().await?; + file_scan_task_stream.stream(arrow_reader) + } + + /// Returns an [`UnzippedIncrementalBatchRecordStream`] for this incremental table scan. + /// This stream will yield separate streams for appended and deleted record batches. + pub async fn to_unzipped_arrow(&self) -> Result { + let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); + + if let Some(batch_size) = self.batch_size { + arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); + } + + let arrow_reader = arrow_reader_builder.build(); + let file_scan_task_stream = self.plan_files().await?; + file_scan_task_stream.stream(arrow_reader) + } + + async fn process_delete_manifest_entry( + mut delete_file_ctx_tx: Sender, + manifest_entry_context: ManifestEntryContext, + ) -> Result<()> { + // Abort the plan if we encounter a manifest entry for a data file or equality + // deletes. + if manifest_entry_context.manifest_entry.content_type() == DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a data file in a delete file manifest", + )); + } else if manifest_entry_context.manifest_entry.content_type() + == DataContentType::EqualityDeletes + { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Equality deletes are not supported yet in incremental scans", + )); + } + + // Abort if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() + && manifest_entry_context.manifest_entry.content_type() + == DataContentType::PositionDeletes + { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Processing deleted (position) delete files is not supported yet in incremental scans", + )); + } + + delete_file_ctx_tx + .send(DeleteFileContext { + manifest_entry: manifest_entry_context.manifest_entry.clone(), + partition_spec_id: manifest_entry_context.partition_spec_id, + }) + .await?; + Ok(()) + } + + async fn process_data_manifest_entry( + mut file_scan_task_tx: Sender>, + manifest_entry_context: ManifestEntryContext, + delete_filter: &DeleteFilter, + ) -> Result<()> { + // Skip processing this manifest entry if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() { + return Ok(()); + } + + // Abort the plan if we encounter a manifest entry for a delete file + if manifest_entry_context.manifest_entry.content_type() != DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a delete file in a data file manifest", + )); + } + + let file_scan_task = IncrementalFileScanTask::append_from_manifest_entry( + &manifest_entry_context, + delete_filter, + ); + + file_scan_task_tx.send(Ok(file_scan_task)).await?; + Ok(()) + } +} + +#[cfg(test)] +mod tests; diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs new file mode 100644 index 0000000000..4b2f244410 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::{Arc, Mutex}; + +use futures::stream::BoxStream; + +use crate::Result; +use crate::arrow::delete_filter::DeleteFilter; +use crate::delete_vector::DeleteVector; +use crate::scan::context::ManifestEntryContext; +use crate::spec::{DataFileFormat, Schema, SchemaRef}; + +/// A file scan task for appended data files in an incremental scan. +#[derive(Debug, Clone)] +pub struct AppendedFileScanTask { + /// The start offset of the file to scan. + pub start: u64, + /// The length of the file to scan. + pub length: u64, + /// The number of records in the file. + pub record_count: Option, + /// The path to the data file to scan. + pub data_file_path: String, + /// The format of the data file to scan. + pub data_file_format: DataFileFormat, + /// The schema of the data file to scan. + pub schema: crate::spec::SchemaRef, + /// The field ids to project. + pub project_field_ids: Vec, + /// The optional positional deletes associated with this data file. + pub positional_deletes: Option>>, +} + +impl AppendedFileScanTask { + /// Returns the data file path of this appended file scan task. + pub fn data_file_path(&self) -> &str { + &self.data_file_path + } + + /// Returns the schema of this file scan task as a reference + pub fn schema(&self) -> &Schema { + &self.schema + } + + /// Returns the schema of this file scan task as a SchemaRef + pub fn schema_ref(&self) -> SchemaRef { + self.schema.clone() + } +} + +/// The stream of incremental file scan tasks. +pub type IncrementalFileScanTaskStream = BoxStream<'static, Result>; + +/// An incremental file scan task, which can be either an appended data file or positional +/// deletes. +#[derive(Debug, Clone)] +pub enum IncrementalFileScanTask { + /// An appended data file. + Append(AppendedFileScanTask), + /// Deleted records of a data file. First argument is the file path, second the delete + /// vector. + Delete(String, DeleteVector), +} + +impl IncrementalFileScanTask { + /// Create an `IncrementalFileScanTask::Append` from a `ManifestEntryContext` and `DeleteFilter`. + pub(crate) fn append_from_manifest_entry( + manifest_entry_context: &ManifestEntryContext, + delete_filter: &DeleteFilter, + ) -> Self { + let data_file_path = manifest_entry_context.manifest_entry.file_path(); + IncrementalFileScanTask::Append(AppendedFileScanTask { + start: 0, + length: manifest_entry_context.manifest_entry.file_size_in_bytes(), + record_count: Some(manifest_entry_context.manifest_entry.record_count()), + data_file_path: data_file_path.to_string(), + data_file_format: manifest_entry_context.manifest_entry.file_format(), + schema: manifest_entry_context.snapshot_schema.clone(), + project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), + positional_deletes: delete_filter.get_delete_vector_for_path(data_file_path), + }) + } + + /// Returns the data file path of this incremental file scan task. + pub fn data_file_path(&self) -> &str { + match self { + IncrementalFileScanTask::Append(task) => task.data_file_path(), + IncrementalFileScanTask::Delete(path, _) => path, + } + } +} diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs new file mode 100644 index 0000000000..05724a54a4 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -0,0 +1,1703 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashMap; +use std::fs; +use std::fs::File; +use std::sync::Arc; + +use arrow_array::cast::AsArray; +use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use parquet::arrow::{ArrowWriter, PARQUET_FIELD_ID_META_KEY}; +use parquet::basic::Compression; +use parquet::file::properties::WriterProperties; +use tempfile::TempDir; +use uuid::Uuid; + +use crate::TableIdent; +use crate::io::{FileIO, OutputFile}; +use crate::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, ManifestEntry, ManifestListWriter, + ManifestStatus, ManifestWriterBuilder, PartitionSpec, Struct, TableMetadata, +}; +use crate::table::Table; + +/// Represents an operation to perform on a snapshot of a table with schema (id: Int32, +/// data: String). +#[derive(Debug, Clone)] +pub enum Operation { + /// Add rows with the given (n, data) tuples, and write to the specified parquet file name. + /// Example: `Add(vec![(1, "a".to_string()), (2, "b".to_string())], "data-1.parquet".to_string())` + /// adds two rows with n=1,2 and data="a","b" to a file named "data-1.parquet" + Add(Vec<(i32, String)>, String), + + /// Delete rows by their positions within specific parquet files (uses positional deletes). + /// Takes a vector of (position, file_name) tuples specifying which position in which file to delete. + /// Example: `Delete(vec![(0, "data-1.parquet"), (1, "data-1.parquet")])` deletes positions 0 and 1 from data-1.parquet + Delete(Vec<(i64, String)>), + + /// Overwrite operation that can append new rows, delete specific positions, and remove entire data files. + /// This is a combination of append and delete operations in a single atomic snapshot. + /// + /// Parameters: + /// 1. Rows to append: Vec<(n, data)> tuples and the filename to write them to + /// 2. Positions to delete: Vec<(position, file_name)> tuples for positional deletes + /// 3. Data files to delete: Vec of file names to completely remove + /// + /// All three parameters can be empty, allowing for various combinations: + /// - Pure append: `Overwrite((rows, "file.parquet"), vec![], vec![])` + /// - Pure positional delete: `Overwrite((vec![], ""), vec![(pos, "file")], vec![])` + /// - Pure file deletion: `Overwrite((vec![], ""), vec![], vec!["file.parquet"])` + /// - Delete entire files: `Overwrite((vec![], ""), vec![], vec!["old-file.parquet"])` + /// + /// Example: `Overwrite((vec![(1, "new".to_string())], "new.parquet"), vec![(0, "old.parquet")], vec!["remove.parquet"])` + /// This adds new data to "new.parquet", deletes position 0 from "old.parquet", and removes "remove.parquet" entirely. + Overwrite( + (Vec<(i32, String)>, String), + Vec<(i64, String)>, + Vec, + ), +} + +/// Tracks the state of data files across snapshots +#[derive(Debug, Clone)] +struct DataFileInfo { + path: String, + snapshot_id: i64, + sequence_number: i64, + n_values: Vec, +} + +/// Test fixture that creates a table with custom snapshots based on operations. +/// +/// # Example +/// ``` +/// let fixture = IncrementalTestFixture::new(vec![ +/// Operation::Add(vec![], "empty.parquet".to_string()), // Empty snapshot +/// Operation::Add( +/// vec![ +/// (1, "1".to_string()), +/// (2, "2".to_string()), +/// (3, "3".to_string()), +/// ], +/// "data-1.parquet".to_string(), +/// ), // Add 3 rows +/// Operation::Delete(vec![(1, "data-1.parquet".to_string())]), // Delete position 1 from data-1.parquet +/// ]) +/// .await; +/// ``` +pub struct IncrementalTestFixture { + pub table_location: String, + pub table: Table, + _tmp_dir: TempDir, // Keep temp dir alive +} + +impl IncrementalTestFixture { + /// Create a new test fixture with the given operations. + pub async fn new(operations: Vec) -> Self { + // Use pwd + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().join("incremental_test_table"); + + // Create directory structure + fs::create_dir_all(table_location.join("metadata")).unwrap(); + fs::create_dir_all(table_location.join("data")).unwrap(); + + let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) + .unwrap() + .build() + .unwrap(); + + let num_snapshots = operations.len(); + let current_snapshot_id = num_snapshots as i64; + let last_sequence_number = (num_snapshots - 1) as i64; + + // Build the snapshots JSON dynamically + let mut snapshots_json = Vec::new(); + let mut snapshot_log_json = Vec::new(); + let mut manifest_list_locations = Vec::new(); + + for (i, op) in operations.iter().enumerate() { + let snapshot_id = (i + 1) as i64; + let parent_id = if i == 0 { None } else { Some(i as i64) }; + let sequence_number = i as i64; + let timestamp = 1515100955770 + (i as i64 * 1000); + + let operation_type = match op { + Operation::Add(..) => "append", + Operation::Delete(..) => "delete", + Operation::Overwrite(..) => "overwrite", + }; + + let manifest_list_location = + table_location.join(format!("metadata/snap-{}-manifest-list.avro", snapshot_id)); + manifest_list_locations.push(manifest_list_location.clone()); + + let parent_str = if let Some(pid) = parent_id { + format!(r#""parent-snapshot-id": {},"#, pid) + } else { + String::new() + }; + + snapshots_json.push(format!( + r#" {{ + "snapshot-id": {}, + {} + "timestamp-ms": {}, + "sequence-number": {}, + "summary": {{"operation": "{}"}}, + "manifest-list": "{}", + "schema-id": 0 + }}"#, + snapshot_id, + parent_str, + timestamp, + sequence_number, + operation_type, + manifest_list_location.display() + )); + + snapshot_log_json.push(format!( + r#" {{"snapshot-id": {}, "timestamp-ms": {}}}"#, + snapshot_id, timestamp + )); + } + + let snapshots_str = snapshots_json.join(",\n"); + let snapshot_log_str = snapshot_log_json.join(",\n"); + + // Create the table metadata + let metadata_json = format!( + r#"{{ + "format-version": 2, + "table-uuid": "{}", + "location": "{}", + "last-sequence-number": {}, + "last-updated-ms": 1602638573590, + "last-column-id": 2, + "current-schema-id": 0, + "schemas": [ + {{ + "type": "struct", + "schema-id": 0, + "fields": [ + {{"id": 1, "name": "n", "required": true, "type": "int"}}, + {{"id": 2, "name": "data", "required": true, "type": "string"}} + ] + }} + ], + "default-spec-id": 0, + "partition-specs": [ + {{ + "spec-id": 0, + "fields": [] + }} + ], + "last-partition-id": 0, + "default-sort-order-id": 0, + "sort-orders": [ + {{ + "order-id": 0, + "fields": [] + }} + ], + "properties": {{}}, + "current-snapshot-id": {}, + "snapshots": [ +{} + ], + "snapshot-log": [ +{} + ], + "metadata-log": [] +}}"#, + Uuid::new_v4(), + table_location.display(), + last_sequence_number, + current_snapshot_id, + snapshots_str, + snapshot_log_str + ); + + let table_metadata_location = table_location.join("metadata/v1.json"); + let table_metadata = serde_json::from_str::(&metadata_json).unwrap(); + + let table = Table::builder() + .metadata(table_metadata) + .identifier(TableIdent::from_strs(["db", "incremental_test"]).unwrap()) + .file_io(file_io.clone()) + .metadata_location(table_metadata_location.as_os_str().to_str().unwrap()) + .build() + .unwrap(); + + let mut fixture = Self { + table_location: table_location.to_str().unwrap().to_string(), + table, + _tmp_dir: tmp_dir, + }; + + // Setup all snapshots based on operations + fixture.setup_snapshots(operations).await; + + fixture + } + + fn next_manifest_file(&self) -> OutputFile { + self.table + .file_io() + .new_output(format!( + "{}/metadata/manifest_{}.avro", + self.table_location, + Uuid::new_v4() + )) + .unwrap() + } + + async fn setup_snapshots(&mut self, operations: Vec) { + let current_schema = self + .table + .metadata() + .current_snapshot() + .unwrap() + .schema(self.table.metadata()) + .unwrap(); + let partition_spec = Arc::new(PartitionSpec::unpartition_spec()); + let empty_partition = Struct::empty(); + + // Track all data files and their contents across snapshots + let mut data_files: Vec = Vec::new(); + #[allow(clippy::type_complexity)] + let mut delete_files: Vec<(String, i64, i64, Vec<(String, i64)>)> = Vec::new(); // (path, snapshot_id, sequence_number, [(data_file_path, position)]) + + for (snapshot_idx, operation) in operations.iter().enumerate() { + let snapshot_id = (snapshot_idx + 1) as i64; + let sequence_number = snapshot_idx as i64; + let parent_snapshot_id = if snapshot_idx == 0 { + None + } else { + Some(snapshot_idx as i64) + }; + + match operation { + Operation::Add(rows, file_name) => { + // Extract n_values and data_values from tuples + let n_values: Vec = rows.iter().map(|(n, _)| *n).collect(); + let data_values: Vec = rows.iter().map(|(_, d)| d.clone()).collect(); + + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + // Add existing data files from previous snapshots + for data_file in &data_files { + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new data if not empty + if !n_values.is_empty() { + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + self.write_parquet_file(&data_file_path, &n_values, &data_values) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this data file + data_files.push(DataFileInfo { + path: data_file_path, + snapshot_id, + sequence_number, + n_values, + }); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Create delete manifest if there are any delete files + let mut manifests = vec![data_manifest]; + if !delete_files.is_empty() { + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + manifests.push(delete_writer.write_manifest_file().await.unwrap()); + } + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(manifests.into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + } + + Operation::Delete(positions_to_delete) => { + // Group deletes by file + let mut deletes_by_file: HashMap> = HashMap::new(); + + for (position, file_name) in positions_to_delete { + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + deletes_by_file + .entry(data_file_path) + .or_default() + .push(*position); + } + + // Create data manifest with existing data files + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + for data_file in &data_files { + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Create delete manifest + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + // Add existing delete files + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new delete files + for (data_file_path, positions) in deletes_by_file { + let delete_file_path = format!( + "{}/data/delete-{}-{}.parquet", + &self.table_location, + snapshot_id, + Uuid::new_v4() + ); + self.write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; + + delete_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(positions.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this delete file + delete_files.push(( + delete_file_path, + snapshot_id, + sequence_number, + positions + .into_iter() + .map(|pos| (data_file_path.clone(), pos)) + .collect(), + )); + } + + let delete_manifest = delete_writer.write_manifest_file().await.unwrap(); + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(vec![data_manifest, delete_manifest].into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + } + + Operation::Overwrite((rows, file_name), positions_to_delete, files_to_delete) => { + // Overwrite creates a single snapshot that can: + // 1. Add new data files + // 2. Delete positions from existing files + // 3. Remove entire data files + + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + // Determine which files to delete + let files_to_delete_set: std::collections::HashSet = files_to_delete + .iter() + .map(|f| format!("{}/data/{}", &self.table_location, f)) + .collect(); + + // Add existing data files (mark deleted ones as DELETED, others as EXISTING) + for data_file in &data_files { + if files_to_delete_set.contains(&data_file.path) { + // Mark file for deletion + data_writer + .add_delete_entry( + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } else { + // Keep existing file + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + } + + // Add new data file if rows provided + if !rows.is_empty() { + let n_values: Vec = rows.iter().map(|(n, _)| *n).collect(); + let data_values: Vec = + rows.iter().map(|(_, d)| d.clone()).collect(); + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + + self.write_parquet_file(&data_file_path, &n_values, &data_values) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this new data file + data_files.push(DataFileInfo { + path: data_file_path, + snapshot_id, + sequence_number, + n_values, + }); + } + + // Remove deleted files from tracking + data_files.retain(|df| !files_to_delete_set.contains(&df.path)); + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Handle positional deletes if any + let mut manifests = vec![data_manifest]; + + if !positions_to_delete.is_empty() || !delete_files.is_empty() { + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + // Add existing delete files + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new positional delete files + if !positions_to_delete.is_empty() { + // Group deletes by file + let mut deletes_by_file: HashMap> = HashMap::new(); + for (position, file_name) in positions_to_delete { + let data_file_path = + format!("{}/data/{}", &self.table_location, file_name); + deletes_by_file + .entry(data_file_path) + .or_default() + .push(*position); + } + + for (data_file_path, positions) in deletes_by_file { + let delete_file_path = format!( + "{}/data/delete-{}-{}.parquet", + &self.table_location, + snapshot_id, + Uuid::new_v4() + ); + self.write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; + + delete_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(positions.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this delete file + delete_files.push(( + delete_file_path, + snapshot_id, + sequence_number, + positions + .into_iter() + .map(|pos| (data_file_path.clone(), pos)) + .collect(), + )); + } + } + + manifests.push(delete_writer.write_manifest_file().await.unwrap()); + } + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(manifests.into_iter()) + .unwrap(); + + manifest_list_write.close().await.unwrap(); + } + } + } + } + + async fn write_parquet_file(&self, path: &str, n_values: &[i32], data_values: &[String]) { + let schema = { + let fields = vec![ + arrow_schema::Field::new("n", arrow_schema::DataType::Int32, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ), + arrow_schema::Field::new("data", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let col_n = Arc::new(Int32Array::from(n_values.to_vec())) as ArrayRef; + let col_data = Arc::new(StringArray::from(data_values.to_vec())) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![col_n, col_data]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let file = File::create(path).unwrap(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + + async fn write_positional_delete_file( + &self, + path: &str, + data_file_path: &str, + positions: &[i64], + ) { + let schema = { + let fields = vec![ + arrow_schema::Field::new("file_path", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2147483546".to_string(), + )])), + arrow_schema::Field::new("pos", arrow_schema::DataType::Int64, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2147483545".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let file_paths: Vec<&str> = vec![data_file_path; positions.len()]; + let col_file_path = Arc::new(StringArray::from(file_paths)) as ArrayRef; + let col_pos = Arc::new(arrow_array::Int64Array::from(positions.to_vec())) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![col_file_path, col_pos]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let file = File::create(path).unwrap(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + + /// Verify incremental scan results. + /// + /// Verifies that the incremental scan contains the expected appended and deleted records. + pub async fn verify_incremental_scan( + &self, + from_snapshot_id: i64, + to_snapshot_id: i64, + expected_appends: Vec<(i32, &str)>, + expected_deletes: Vec<(u64, &str)>, + ) { + use arrow_array::cast::AsArray; + use arrow_select::concat::concat_batches; + use futures::TryStreamExt; + + let incremental_scan = self + .table + .incremental_scan(from_snapshot_id, to_snapshot_id) + .build() + .unwrap(); + + let stream = incremental_scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Separate appends and deletes + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + let delete_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Delete) + .map(|(_, b)| b.clone()) + .collect(); + + // Verify appended records + if !append_batches.is_empty() { + let append_batch = + concat_batches(&append_batches[0].schema(), append_batches.iter()).unwrap(); + + let n_array = append_batch + .column(0) + .as_primitive::(); + let data_array = append_batch.column(1).as_string::(); + + let mut appended_pairs: Vec<(i32, String)> = (0..append_batch.num_rows()) + .map(|i| (n_array.value(i), data_array.value(i).to_string())) + .collect(); + appended_pairs.sort(); + + let expected_appends: Vec<(i32, String)> = expected_appends + .into_iter() + .map(|(n, s)| (n, s.to_string())) + .collect(); + + assert_eq!(appended_pairs, expected_appends); + } else { + assert!(expected_appends.is_empty(), "Expected appends but got none"); + } + + // Verify deleted records + if !delete_batches.is_empty() { + let delete_batch = + concat_batches(&delete_batches[0].schema(), delete_batches.iter()).unwrap(); + + let pos_array = delete_batch + .column(0) + .as_primitive::(); + + // The file path column is a StringArray with materialized values + let file_path_column = delete_batch.column(1); + let file_path_array = file_path_column.as_string::(); + + let mut deleted_pairs: Vec<(u64, String)> = (0..delete_batch.num_rows()) + .map(|i| { + let pos = pos_array.value(i); + let file_path = file_path_array.value(i).to_string(); + (pos, file_path) + }) + .collect(); + deleted_pairs.sort(); + + let expected_deletes: Vec<(u64, String)> = expected_deletes + .into_iter() + .map(|(pos, file)| (pos, file.to_string())) + .collect(); + + assert_eq!(deleted_pairs, expected_deletes); + } else { + assert!(expected_deletes.is_empty(), "Expected deletes but got none"); + } + } +} + +#[tokio::test] +async fn test_incremental_fixture_simple() { + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), + Operation::Add( + vec![ + (1, "1".to_string()), + (2, "2".to_string()), + (3, "3".to_string()), + ], + "data-2.parquet".to_string(), + ), + Operation::Delete(vec![(1, "data-2.parquet".to_string())]), // Delete position 1 (n=2, data="2") + ]) + .await; + + // Verify we have 3 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 3); + + // Verify snapshot IDs + assert_eq!(snapshots[0].snapshot_id(), 1); + assert_eq!(snapshots[1].snapshot_id(), 2); + assert_eq!(snapshots[2].snapshot_id(), 3); + + // Verify parent relationships + assert_eq!(snapshots[0].parent_snapshot_id(), None); + assert_eq!(snapshots[1].parent_snapshot_id(), Some(1)); + assert_eq!(snapshots[2].parent_snapshot_id(), Some(2)); + + // Verify incremental scan from snapshot 1 to snapshot 3. + // Expected appends: snapshot 2 adds [1, 2, 3] + // Expected deletes: snapshot 3 deletes [2] + // In total we expect appends [1, 3] and deletes [] + fixture + .verify_incremental_scan(1, 3, vec![(1, "1"), (3, "3")], vec![]) + .await; + + // Verify incremental scan from snapshot 2 to snapshot 3. + let data_file_path = format!("{}/data/data-2.parquet", fixture.table_location); + fixture + .verify_incremental_scan(2, 3, vec![], vec![(1, &data_file_path)]) + .await; + + // Verify incremental scan from snapshot 1 to snapshot 1. + fixture.verify_incremental_scan(1, 1, vec![], vec![]).await; +} + +#[tokio::test] +async fn test_incremental_fixture_complex() { + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), // Snapshot 1: Empty + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + (4, "d".to_string()), + (5, "e".to_string()), + ], + "data-2.parquet".to_string(), + ), // Snapshot 2: Add 5 rows (positions 0-4) + Operation::Delete(vec![ + (1, "data-2.parquet".to_string()), + (3, "data-2.parquet".to_string()), + ]), // Snapshot 3: Delete positions 1,3 (n=2,4; data=b,d) + Operation::Add( + vec![(6, "f".to_string()), (7, "g".to_string())], + "data-4.parquet".to_string(), + ), // Snapshot 4: Add 2 more rows (positions 5-6) + Operation::Delete(vec![ + (0, "data-2.parquet".to_string()), + (2, "data-2.parquet".to_string()), + (4, "data-2.parquet".to_string()), + (0, "data-4.parquet".to_string()), + (1, "data-4.parquet".to_string()), + ]), // Snapshot 5: Delete positions 0,2,4,5,6 (all remaining rows: n=1,3,5,6,7) + ]) + .await; + + // Verify we have 5 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 5); + + // Verify parent chain + assert_eq!(snapshots[0].parent_snapshot_id(), None); + for (i, snapshot) in snapshots.iter().enumerate().take(5).skip(1) { + assert_eq!(snapshot.parent_snapshot_id(), Some(i as i64)); + } + + // Verify current snapshot + assert_eq!( + fixture + .table + .metadata() + .current_snapshot() + .unwrap() + .snapshot_id(), + 5 + ); + + // Verify incremental scan from snapshot 1 to snapshot 5. + // All data has been deleted, so we expect the empty result. + fixture.verify_incremental_scan(1, 5, vec![], vec![]).await; + + // Verify incremental scan from snapshot 2 to snapshot 5. + // Snapshot 2 starts with: (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // Snapshot 5: Deletes positions 0,2,4 from data-2.parquet and 0,1 from data-4.parquet (n=1,3,5,6,7; data=a,c,e,f,g) + // + // The incremental scan computes the NET EFFECT between snapshot 2 and 5: + // - Files added in snapshot 4 were completely deleted in snapshot 5, so NO net appends + // - Net deletes from data-2.parquet: positions 0,1,2,3,4 (all 5 rows deleted across snapshots 3 and 5) + // - Since data-4 was added and deleted between 2 and 5, it doesn't appear in the scan + let data_2_path = format!("{}/data/data-2.parquet", fixture.table_location); + fixture + .verify_incremental_scan( + 2, + 5, + vec![], // No net appends (data-4 was added and fully deleted) + vec![ + (0, data_2_path.as_str()), // All 5 positions from data-2.parquet + (1, data_2_path.as_str()), + (2, data_2_path.as_str()), + (3, data_2_path.as_str()), + (4, data_2_path.as_str()), + ], + ) + .await; + + // Verify incremental scan from snapshot 3 to snapshot 5. + // Snapshot 3 state: (1,a), (3,c), (5,e) remain in data-2.parquet at positions 0,2,4 + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // Snapshot 5: Deletes positions 0,2,4 from data-2.parquet (n=1,3,5) and 0,1 from data-4.parquet (n=6,7) + // + // Net effect from snapshot 3 to 5: + // - No net appends (data-4 was added and fully deleted between 3 and 5) + // - Net deletes from data-2.parquet: positions 0,2,4 (the three remaining rows deleted in snapshot 5) + fixture + .verify_incremental_scan( + 3, + 5, + vec![], // No net appends (data-4 was added and fully deleted) + vec![ + (0, data_2_path.as_str()), // Positions 0,2,4 from data-2.parquet + (2, data_2_path.as_str()), // (n=1,3,5; data=a,c,e) + (4, data_2_path.as_str()), + ], + ) + .await; + + // Verify incremental scan from snapshot 1 to snapshot 4. + // Snapshot 1: Empty + // Snapshot 2: Adds (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // + // Net effect from snapshot 1 to 4: + // - Net appends: (1,a), (3,c), (5,e), (6,f), (7,g) - all rows that exist at snapshot 4 + // - No deletes: rows deleted in snapshot 3 were added after snapshot 1, so they don't count as deletes + fixture + .verify_incremental_scan( + 1, + 4, + vec![(1, "a"), (3, "c"), (5, "e"), (6, "f"), (7, "g")], + vec![], // No deletes (deleted rows were added after snapshot 1) + ) + .await; + + // Verify incremental scan from snapshot 2 to snapshot 4. + // Snapshot 2: Has (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // + // Net effect from snapshot 2 to 4: + // - Net appends: (6,f), (7,g) from data-4.parquet + // - Net deletes: positions 1,3 from data-2.parquet (n=2,4; data=b,d) - existed at snapshot 2 and deleted in 3 + fixture + .verify_incremental_scan(2, 4, vec![(6, "f"), (7, "g")], vec![ + (1, data_2_path.as_str()), // Positions 1,3 from data-2.parquet + (3, data_2_path.as_str()), // (n=2,4; data=b,d) + ]) + .await; +} + +#[tokio::test] +async fn test_incremental_scan_edge_cases() { + // This test covers several edge cases: + // 1. Multiple data files added in separate snapshots + // 2. Deletes spread across multiple data files + // 3. Partial deletes from multiple files + // 4. Cross-file delete operations in a single snapshot + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add file A with 3 rows + Operation::Add( + vec![ + (1, "a1".to_string()), + (2, "a2".to_string()), + (3, "a3".to_string()), + ], + "file-a.parquet".to_string(), + ), + // Snapshot 3: Add file B with 4 rows + Operation::Add( + vec![ + (10, "b1".to_string()), + (20, "b2".to_string()), + (30, "b3".to_string()), + (40, "b4".to_string()), + ], + "file-b.parquet".to_string(), + ), + // Snapshot 4: Partial delete from file A (delete middle row n=2) + Operation::Delete(vec![(1, "file-a.parquet".to_string())]), + // Snapshot 5: Partial delete from file B (delete first and last rows n=10,40) + Operation::Delete(vec![ + (0, "file-b.parquet".to_string()), + (3, "file-b.parquet".to_string()), + ]), + // Snapshot 6: Add file C with 2 rows + Operation::Add( + vec![(100, "c1".to_string()), (200, "c2".to_string())], + "file-c.parquet".to_string(), + ), + // Snapshot 7: Delete from multiple files in one snapshot (cross-file deletes) + Operation::Delete(vec![ + (0, "file-a.parquet".to_string()), // n=1 + (1, "file-b.parquet".to_string()), // n=20 + (0, "file-c.parquet".to_string()), // n=100 + ]), + ]) + .await; + + // Verify we have 7 snapshots + let n_snapshots = fixture.table.metadata().snapshots().count(); + assert_eq!(n_snapshots, 7); + + let file_a_path = format!("{}/data/file-a.parquet", fixture.table_location); + let file_b_path = format!("{}/data/file-b.parquet", fixture.table_location); + + // Test 1: Scan from snapshot 1 to 4 + // Should see: file-a (1,2,3), file-b (10,20,30,40) added, then (2) deleted from file-a + // BUT: The row n=2 was added AFTER snapshot 1, so it won't show as a delete! + // Net: appends (1,3) from file-a (n=2 added then deleted = net zero), (10,20,30,40) from file-b + // No deletes (n=2 was added and deleted between snapshots 1 and 4) + fixture + .verify_incremental_scan( + 1, + 4, + vec![ + (1, "a1"), + (3, "a3"), + (10, "b1"), + (20, "b2"), + (30, "b3"), + (40, "b4"), + ], + vec![], // No deletes - n=2 was added and deleted between snapshots + ) + .await; + + // Test 2: Scan from snapshot 4 to 6 + // Snapshot 4: has file-a (1,3) and file-b (10,20,30,40) + // Snapshot 5: deletes positions 0,3 from file-b (n=10,40) + // Snapshot 6: adds file-c (100,200) + // Net: appends (100,200) from file-c; deletes pos 0,3 from file-b + fixture + .verify_incremental_scan(4, 6, vec![(100, "c1"), (200, "c2")], vec![ + (0, file_b_path.as_str()), // n=10 + (3, file_b_path.as_str()), // n=40 + ]) + .await; + + // Test 3: Scan from snapshot 2 to 7 + // This tests the full lifecycle: multiple adds, partial deletes, more adds, cross-file deletes + // Starting at snapshot 2: file-a (1,2,3) exists + // File-b is added in snapshot 3 (after snapshot 2) + // File-c is added in snapshot 6 (after snapshot 2) + // By snapshot 7: file-a has (3) at position 2, file-b has (30), file-c has (200) + // + // Net appends: file-b (30) and file-c (200) were added after snapshot 2 + // Net deletes: positions 0,1 from file-a (n=1,2) existed at snapshot 2 and were deleted + // Note: (3) from file-a already existed at snapshot 2, so it's not a net append! + fixture + .verify_incremental_scan(2, 7, vec![(30, "b3"), (200, "c2")], vec![ + (0, file_a_path.as_str()), // n=1 + (1, file_a_path.as_str()), // n=2 + ]) + .await; + + // Test 4: Scan from snapshot 5 to 6 + // Simple test: just adding a new file + // Snapshot 5 state: file-a (1,3), file-b (20,30) + // Snapshot 6: adds file-c (100,200) + // Net: appends (100,200) from file-c, no deletes + fixture + .verify_incremental_scan(5, 6, vec![(100, "c1"), (200, "c2")], vec![]) + .await; + + // Test 5: Scan from snapshot 3 to 4 + // Tests a single delete operation + // State at 3: file-a (1,2,3), file-b (10,20,30,40) + // State at 4: file-a (1,3), file-b (10,20,30,40) + // Net: no appends, 1 delete (position 1, n=2) from file-a + fixture + .verify_incremental_scan( + 3, + 4, + vec![], + vec![(1, file_a_path.as_str())], // n=2 + ) + .await; +} + +#[tokio::test] +async fn test_incremental_scan_builder_options() { + // This test demonstrates using the incremental scan builder API with various options: + // - Column projection (selecting specific columns) + // - Batch size configuration + // - Verifying the schema and batch structure + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add 10 rows to test batch size behavior + Operation::Add( + vec![ + (1, "data-1".to_string()), + (2, "data-2".to_string()), + (3, "data-3".to_string()), + (4, "data-4".to_string()), + (5, "data-5".to_string()), + (6, "data-6".to_string()), + (7, "data-7".to_string()), + (8, "data-8".to_string()), + (9, "data-9".to_string()), + (10, "data-10".to_string()), + ], + "data-2.parquet".to_string(), + ), + // Snapshot 3: Delete some rows + Operation::Delete(vec![ + (2, "data-2.parquet".to_string()), // n=3 + (5, "data-2.parquet".to_string()), // n=6 + (8, "data-2.parquet".to_string()), // n=9 + ]), + // Snapshot 4: Add more rows + Operation::Add( + vec![ + (20, "data-20".to_string()), + (21, "data-21".to_string()), + (22, "data-22".to_string()), + (23, "data-23".to_string()), + (24, "data-24".to_string()), + ], + "data-4.parquet".to_string(), + ), + ]) + .await; + + // Test 1: Column projection - select only the "n" column + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Verify we have both append and delete batches + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + assert!(!append_batches.is_empty(), "Should have append batches"); + + // Check schema - should only have "n" column + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 1, + "Should have only 1 column when projecting 'n'" + ); + assert_eq!( + batch.schema().field(0).name(), + "n", + "Projected column should be 'n'" + ); + } + + // Test 2: Column projection - select only the "data" column + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["data"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 1, + "Should have only 1 column when projecting 'data'" + ); + assert_eq!( + batch.schema().field(0).name(), + "data", + "Projected column should be 'data'" + ); + } + + // Test 3: Select both columns explicitly + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n", "data"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 2, + "Should have 2 columns when projecting both" + ); + assert_eq!(batch.schema().field(0).name(), "n"); + assert_eq!(batch.schema().field(1).name(), "data"); + } + + // Test 4: Batch size configuration + let scan = fixture + .table + .incremental_scan(1, 2) + .with_batch_size(Some(3)) // Small batch size to test batching + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()); + + for batch in append_batches { + // Each batch should have at most 3 rows (except possibly the last) + assert!( + batch.num_rows() <= 3, + "Batch size should be <= 3 as configured" + ); + } + + // Test 5: Combining column projection and batch size + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n"]) + .with_batch_size(Some(4)) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()); + + for batch in append_batches { + assert_eq!(batch.schema().fields().len(), 1, "Should project only 'n'"); + assert!(batch.num_rows() <= 4, "Batch size should be <= 4"); + } + + // Test 6: Verify actual data with column projection + let scan = fixture + .table + .incremental_scan(1, 2) + .select(vec!["n"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + if !append_batches.is_empty() { + use arrow_select::concat::concat_batches; + let combined_batch = + concat_batches(&append_batches[0].schema(), append_batches.iter()).unwrap(); + + let n_array = combined_batch + .column(0) + .as_primitive::(); + + let mut n_values: Vec = (0..combined_batch.num_rows()) + .map(|i| n_array.value(i)) + .collect(); + n_values.sort(); + + assert_eq!( + n_values, + vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10], + "Should have all 10 n values from snapshot 2" + ); + } + + // Test 7: Delete batches always have the same schema. + let scan = fixture.table.incremental_scan(2, 3).build().unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let delete_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Delete) + .map(|(_, b)| b.clone()) + .collect(); + + if !delete_batches.is_empty() { + for batch in &delete_batches { + // Delete batches should have "pos" and "_file" columns + assert!( + batch.schema().fields().len() == 2, + "Delete batch should have exactly position and file columns" + ); + assert_eq!( + batch.num_rows(), + 3, + "Should have 3 deleted positions from snapshot 3" + ); + } + } +} + +#[tokio::test] +async fn test_incremental_scan_with_deleted_files_errors() { + // This test verifies that incremental scans properly error out when entire data files + // are deleted (overwrite operation), since this is not yet supported. + // + // Test scenario: + // Snapshot 1: Add file-1.parquet with data + // Snapshot 2: Add file-2.parquet with data + // Snapshot 3: Overwrite - delete file-1.parquet entirely + // Snapshot 4: Add file-3.parquet with data + // + // Incremental scan from snapshot 1 to snapshot 3 should error because file-1 + // was completely removed in the overwrite operation. + + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Add file-1 with rows + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + ], + "file-1.parquet".to_string(), + ), + // Snapshot 2: Add file-2 with rows + Operation::Add( + vec![(10, "x".to_string()), (20, "y".to_string())], + "file-2.parquet".to_string(), + ), + // Snapshot 3: Overwrite - delete file-1 entirely + Operation::Overwrite( + (vec![], "".to_string()), // No new data to add + vec![], // No positional deletes + vec!["file-1.parquet".to_string()], // Delete file-1 completely + ), + // Snapshot 4: Add file-3 (to have more snapshots) + Operation::Add(vec![(100, "p".to_string())], "file-3.parquet".to_string()), + ]) + .await; + + // Test 1: Incremental scan from snapshot 1 to 3 should error when building the stream + // because file-1 was deleted entirely in snapshot 3 + let scan = fixture + .table + .incremental_scan(1, 3) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + match stream_result { + Err(error) => { + assert_eq!( + error.message(), + "Processing deleted data files is not supported yet in incremental scans", + "Error message should indicate that deleted files are not supported. Got: {}", + error + ); + } + Ok(_) => panic!( + "Expected error when building stream over a snapshot that deletes entire data files" + ), + } + + // Test 2: Incremental scan from snapshot 2 to 4 should also error + // because it includes snapshot 3 which deletes a file + let scan = fixture + .table + .incremental_scan(2, 4) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + match stream_result { + Err(_) => { + // Expected error + } + Ok(_) => panic!("Expected error when scan range includes a snapshot that deletes files"), + } + + // Test 3: Incremental scan from snapshot 1 to 2 should work fine + // (no files deleted) + let scan = fixture + .table + .incremental_scan(1, 2) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + assert!( + stream_result.is_ok(), + "Scan should succeed when no files are deleted. Error: {:?}", + stream_result.err() + ); + + // Test 4: Incremental scan from snapshot 3 to 4 should work + // (starting from after the deletion) + let scan = fixture + .table + .incremental_scan(3, 4) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + assert!( + stream_result.is_ok(), + "Scan should succeed when starting after the file deletion. Error: {:?}", + stream_result.err() + ); +} diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 3d14b3cce4..682691dc49 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -21,6 +21,9 @@ mod cache; use cache::*; mod context; use context::*; + +pub mod incremental; + mod task; use std::sync::Arc; diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index d4e696ce84..1c8b43ca04 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -24,6 +24,7 @@ use crate::inspect::MetadataTable; use crate::io::FileIO; use crate::io::object_cache::ObjectCache; use crate::scan::TableScanBuilder; +use crate::scan::incremental::IncrementalTableScanBuilder; use crate::spec::{TableMetadata, TableMetadataRef}; use crate::{Error, ErrorKind, Result, TableIdent}; @@ -224,6 +225,15 @@ impl Table { TableScanBuilder::new(self) } + /// Creates an incremental table scan between two snapshots. + pub fn incremental_scan( + &self, + from_snapshot_id: i64, + to_snapshot_id: i64, + ) -> IncrementalTableScanBuilder<'_> { + IncrementalTableScanBuilder::new(self, from_snapshot_id, to_snapshot_id) + } + /// Creates a metadata table which provides table-like APIs for inspecting metadata. /// See [`MetadataTable`] for more details. pub fn inspect(&self) -> MetadataTable<'_> { diff --git a/crates/iceberg/src/util/mod.rs b/crates/iceberg/src/util/mod.rs new file mode 100644 index 0000000000..b614c981ec --- /dev/null +++ b/crates/iceberg/src/util/mod.rs @@ -0,0 +1,19 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +/// Utilities for working with snapshots. +pub mod snapshot; diff --git a/crates/iceberg/src/util/snapshot.rs b/crates/iceberg/src/util/snapshot.rs new file mode 100644 index 0000000000..513e23c110 --- /dev/null +++ b/crates/iceberg/src/util/snapshot.rs @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Taken from https://github.com/apache/iceberg-rust/pull/1470 + +use crate::spec::{SnapshotRef, TableMetadataRef}; + +struct Ancestors { + next: Option, + get_snapshot: Box Option + Send>, +} + +impl Iterator for Ancestors { + type Item = SnapshotRef; + + fn next(&mut self) -> Option { + let snapshot = self.next.take()?; + let result = snapshot.clone(); + self.next = snapshot + .parent_snapshot_id() + .and_then(|id| (self.get_snapshot)(id)); + Some(result) + } +} + +/// Iterate starting from `snapshot` (inclusive) to the root snapshot. +pub fn ancestors_of( + table_metadata: &TableMetadataRef, + snapshot: i64, +) -> Box + Send> { + if let Some(snapshot) = table_metadata.snapshot_by_id(snapshot) { + let table_metadata = table_metadata.clone(); + Box::new(Ancestors { + next: Some(snapshot.clone()), + get_snapshot: Box::new(move |id| table_metadata.snapshot_by_id(id).cloned()), + }) + } else { + Box::new(std::iter::empty()) + } +} + +/// Iterate starting from `snapshot` (inclusive) to `oldest_snapshot_id` (exclusive). +pub fn ancestors_between( + table_metadata: &TableMetadataRef, + latest_snapshot_id: i64, + oldest_snapshot_id: Option, +) -> Box + Send> { + let Some(oldest_snapshot_id) = oldest_snapshot_id else { + return Box::new(ancestors_of(table_metadata, latest_snapshot_id)); + }; + + if latest_snapshot_id == oldest_snapshot_id { + return Box::new(std::iter::empty()); + } + + Box::new( + ancestors_of(table_metadata, latest_snapshot_id) + .take_while(move |snapshot| snapshot.snapshot_id() != oldest_snapshot_id), + ) +} From c9501dbfbee474ff704ee9daa7b545959297e662 Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Thu, 30 Oct 2025 15:08:25 +0100 Subject: [PATCH 2/7] feat(core): Add incremental scan for appends and positional deletes (#3) * WIP, initial draft of incremental scan * . * . * cargo fmt * Implement unzipped stream * Remove printlns * Add API method for unzipped stream * . * Remove comment * Rename var * Add import * Measure time * Fix typo * Undo some changes * Change type name * Add comment header * Fail when encountering equality deletes * Add comments * Add some preliminary tests * Format * Remove playground * Add more tests * Clippy * . * . * Adapt tests * . * Add test * Add tests * Add tests * Format * Add test * Format * . * Rm newline * Rename trait function * Reuse schema * . * remove clone * Add test for adding file_path column * Make `from_snapshot` mandatory * Error out if incremental scan encounters neither Append nor Delete * . * Add materialized variant of add_file_path_column * . * Allow dead code * Some PR comments * . * More PR comments * . * Add comments * Avoid cloning * Add reference to PR * Some PR comments * . * format * Allow overwrite operation for now * Fix file_path column * Add overwrite test * Unwrap delete vector * . * Add assertion * Avoid cloning the mutex guard * Abort when encountering a deleted delete file * Adjust comment * Update crates/iceberg/src/arrow/reader.rs Co-authored-by: Vukasin Stefanovic * Add check * Update crates/iceberg/src/scan/incremental/mod.rs --------- Co-authored-by: Vukasin Stefanovic --- crates/iceberg/src/arrow/delete_filter.rs | 35 +- crates/iceberg/src/arrow/incremental.rs | 271 +++ crates/iceberg/src/arrow/mod.rs | 2 + crates/iceberg/src/arrow/reader.rs | 463 ++++- crates/iceberg/src/delete_file_index.rs | 28 + crates/iceberg/src/delete_vector.rs | 4 +- crates/iceberg/src/lib.rs | 3 + crates/iceberg/src/scan/context.rs | 29 +- .../iceberg/src/scan/incremental/context.rs | 142 ++ crates/iceberg/src/scan/incremental/mod.rs | 588 ++++++ crates/iceberg/src/scan/incremental/task.rs | 106 + crates/iceberg/src/scan/incremental/tests.rs | 1703 +++++++++++++++++ crates/iceberg/src/scan/mod.rs | 3 + crates/iceberg/src/table.rs | 10 + crates/iceberg/src/util/mod.rs | 19 + crates/iceberg/src/util/snapshot.rs | 74 + 16 files changed, 3456 insertions(+), 24 deletions(-) create mode 100644 crates/iceberg/src/arrow/incremental.rs create mode 100644 crates/iceberg/src/scan/incremental/context.rs create mode 100644 crates/iceberg/src/scan/incremental/mod.rs create mode 100644 crates/iceberg/src/scan/incremental/task.rs create mode 100644 crates/iceberg/src/scan/incremental/tests.rs create mode 100644 crates/iceberg/src/util/mod.rs create mode 100644 crates/iceberg/src/util/snapshot.rs diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index b853baa993..e029912bbe 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -35,11 +35,22 @@ enum EqDelState { } #[derive(Debug, Default)] -struct DeleteFileFilterState { +pub(crate) struct DeleteFileFilterState { delete_vectors: HashMap>>, equality_deletes: HashMap, } +impl DeleteFileFilterState { + pub fn delete_vectors(&self) -> &HashMap>> { + &self.delete_vectors + } + + /// Remove and return the delete vector for the given data file path. + pub fn remove_delete_vector(&mut self, path: &str) -> Option>> { + self.delete_vectors.remove(path) + } +} + #[derive(Clone, Debug, Default)] pub(crate) struct DeleteFilter { state: Arc>, @@ -65,6 +76,28 @@ impl DeleteFilter { .and_then(|st| st.delete_vectors.get(delete_file_path).cloned()) } + pub(crate) fn with_read(&self, f: F) -> Result + where F: FnOnce(&DeleteFileFilterState) -> Result { + let state = self.state.read().map_err(|e| { + Error::new( + ErrorKind::Unexpected, + format!("Failed to acquire read lock: {}", e), + ) + })?; + f(&state) + } + + pub(crate) fn with_write(&self, f: F) -> Result + where F: FnOnce(&mut DeleteFileFilterState) -> Result { + let mut state = self.state.write().map_err(|e| { + Error::new( + ErrorKind::Unexpected, + format!("Failed to acquire write lock: {}", e), + ) + })?; + f(&mut state) + } + pub(crate) fn try_start_eq_del_load(&self, file_path: &str) -> Option> { let mut state = self.state.write().unwrap(); diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs new file mode 100644 index 0000000000..e7ca2521bf --- /dev/null +++ b/crates/iceberg/src/arrow/incremental.rs @@ -0,0 +1,271 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::pin::Pin; +use std::sync::Arc; + +use arrow_array::{RecordBatch, UInt64Array}; +use arrow_schema::{DataType, Field, Schema as ArrowSchema}; +use futures::channel::mpsc::channel; +use futures::stream::select; +use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; + +use crate::arrow::record_batch_transformer::RecordBatchTransformer; +use crate::arrow::{ + ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_FIELD_ID_FILE_PATH, StreamsInto, +}; +use crate::delete_vector::DeleteVector; +use crate::io::FileIO; +use crate::runtime::spawn; +use crate::scan::ArrowRecordBatchStream; +use crate::scan::incremental::{ + AppendedFileScanTask, IncrementalFileScanTask, IncrementalFileScanTaskStream, +}; +use crate::{Error, ErrorKind, Result}; + +/// The type of incremental batch: appended data or deleted records. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum IncrementalBatchType { + /// Appended records. + Append, + /// Deleted records. + Delete, +} + +/// The stream of incremental Arrow `RecordBatch`es with batch type. +pub type CombinedIncrementalBatchRecordStream = + Pin> + Send + 'static>>; + +/// Stream type for obtaining a separate stream of appended and deleted record batches. +pub type UnzippedIncrementalBatchRecordStream = (ArrowRecordBatchStream, ArrowRecordBatchStream); + +impl StreamsInto + for IncrementalFileScanTaskStream +{ + /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a + /// stream of Arrow `RecordBatch`es containing the data from the files. + fn stream(self, reader: ArrowReader) -> Result { + let (appends, deletes) = + StreamsInto::::stream(self, reader)?; + + let left = appends.map(|res| res.map(|batch| (IncrementalBatchType::Append, batch))); + let right = deletes.map(|res| res.map(|batch| (IncrementalBatchType::Delete, batch))); + + Ok(Box::pin(select(left, right)) as CombinedIncrementalBatchRecordStream) + } +} + +impl StreamsInto + for IncrementalFileScanTaskStream +{ + /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two + /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. + fn stream(self, reader: ArrowReader) -> Result { + let (appends_tx, appends_rx) = channel(reader.concurrency_limit_data_files); + let (deletes_tx, deletes_rx) = channel(reader.concurrency_limit_data_files); + + let batch_size = reader.batch_size; + let concurrency_limit_data_files = reader.concurrency_limit_data_files; + + spawn(async move { + let _ = self + .try_for_each_concurrent(concurrency_limit_data_files, |task| { + let file_io = reader.file_io.clone(); + let mut appends_tx = appends_tx.clone(); + let mut deletes_tx = deletes_tx.clone(); + async move { + match task { + IncrementalFileScanTask::Append(append_task) => { + spawn(async move { + let record_batch_stream = process_incremental_append_task( + append_task, + batch_size, + file_io, + ) + .await; + + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = appends_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read appended record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = appends_tx.send(Err(e)).await; + } + } + }); + } + IncrementalFileScanTask::Delete(file_path, delete_vector) => { + spawn(async move { + let record_batch_stream = process_incremental_delete_task( + file_path, + delete_vector, + batch_size, + ); + + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = deletes_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read deleted record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = deletes_tx.send(Err(e)).await; + } + } + }); + } + }; + + Ok(()) + } + }) + .await; + }); + + Ok(( + Box::pin(appends_rx) as ArrowRecordBatchStream, + Box::pin(deletes_rx) as ArrowRecordBatchStream, + )) + } +} + +async fn process_incremental_append_task( + task: AppendedFileScanTask, + batch_size: Option, + file_io: FileIO, +) -> Result { + let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( + &task.data_file_path, + file_io, + true, + ) + .await?; + + // Create a projection mask for the batch stream to select which columns in the + // Parquet file that we want in the response + let projection_mask = ArrowReader::get_arrow_projection_mask( + &task.project_field_ids, + &task.schema_ref(), + record_batch_stream_builder.parquet_schema(), + record_batch_stream_builder.schema(), + )?; + record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); + + // RecordBatchTransformer performs any transformations required on the RecordBatches + // that come back from the file, such as type promotion, default column insertion + // and column re-ordering + let mut record_batch_transformer = + RecordBatchTransformer::build(task.schema_ref(), &task.project_field_ids); + + if let Some(batch_size) = batch_size { + record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); + } + + // Apply positional deletes as row selections. + let row_selection = if let Some(positional_delete_indexes) = task.positional_deletes { + Some(ArrowReader::build_deletes_row_selection( + record_batch_stream_builder.metadata().row_groups(), + &None, + &positional_delete_indexes.lock().unwrap(), + )?) + } else { + None + }; + + if let Some(row_selection) = row_selection { + record_batch_stream_builder = record_batch_stream_builder.with_row_selection(row_selection); + } + + // Build the batch stream and send all the RecordBatches that it generates + // to the requester. + let record_batch_stream = record_batch_stream_builder + .build()? + .map(move |batch| match batch { + Ok(batch) => record_batch_transformer.process_record_batch(batch), + Err(err) => Err(err.into()), + }); + + Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream) +} + +fn process_incremental_delete_task( + file_path: String, + delete_vector: DeleteVector, + batch_size: Option, +) -> Result { + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "pos", + DataType::UInt64, + false, + )])); + + let batch_size = batch_size.unwrap_or(1024); + + let treemap = delete_vector.inner; + + let stream = futures::stream::iter(treemap) + .chunks(batch_size) + .map(move |chunk| { + let array = UInt64Array::from_iter(chunk); + RecordBatch::try_new( + Arc::clone(&schema), // Cheap Arc clone instead of full schema creation + vec![Arc::new(array)], + ) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RecordBatch for DeleteVector", + ) + }) + .and_then(|batch| { + ArrowReader::add_file_path_column( + batch, + &file_path, + RESERVED_COL_NAME_FILE_PATH, + RESERVED_FIELD_ID_FILE_PATH, + ) + }) + }); + + Ok(Box::pin(stream) as ArrowRecordBatchStream) +} diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index c091c45177..db85cd5730 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -33,6 +33,8 @@ pub mod record_batch_projector; pub(crate) mod record_batch_transformer; mod value; +mod incremental; +pub use incremental::*; pub use reader::*; pub use value::*; /// Partition value calculator for computing partition values diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index e0894ad6b4..c4a1b8a5f3 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -23,11 +23,14 @@ use std::str::FromStr; use std::sync::Arc; use arrow_arith::boolean::{and, and_kleene, is_not_null, is_null, not, or, or_kleene}; -use arrow_array::{Array, ArrayRef, BooleanArray, Datum as ArrowDatum, RecordBatch, Scalar}; +use arrow_array::{ + Array, ArrayRef, BooleanArray, Datum as ArrowDatum, Int32Array, RecordBatch, RunArray, Scalar, + StringArray, +}; use arrow_cast::cast::cast; use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq}; use arrow_schema::{ - ArrowError, DataType, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, + ArrowError, DataType, Field, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, }; use arrow_string::like::starts_with; use bytes::Bytes; @@ -59,6 +62,22 @@ use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; +/// Reserved field ID for the file path (_file) column per Iceberg spec +/// This is dead code for now but will be used when we add the _file column support. +#[allow(dead_code)] +pub(crate) const RESERVED_FIELD_ID_FILE: i32 = 2147483646; + +/// Column name for the file path metadata column per Iceberg spec +/// This is dead code for now but will be used when we add the _file column support. +#[allow(dead_code)] +pub(crate) const RESERVED_COL_NAME_FILE: &str = "_file"; + +/// Reserved field ID for the file path column used in delete file reading. +pub(crate) const RESERVED_FIELD_ID_FILE_PATH: i32 = 2147483546; + +/// Column name for the file path metadata column used in delete file reading. +pub(crate) const RESERVED_COL_NAME_FILE_PATH: &str = "file_path"; + /// Builder to create ArrowReader pub struct ArrowReaderBuilder { batch_size: Option, @@ -126,15 +145,22 @@ impl ArrowReaderBuilder { /// Reads data from Parquet files #[derive(Clone)] pub struct ArrowReader { - batch_size: Option, - file_io: FileIO, - delete_file_loader: CachingDeleteFileLoader, + pub(crate) batch_size: Option, + pub(crate) file_io: FileIO, + pub(crate) delete_file_loader: CachingDeleteFileLoader, /// the maximum number of data files that can be fetched at the same time - concurrency_limit_data_files: usize, + pub(crate) concurrency_limit_data_files: usize, - row_group_filtering_enabled: bool, - row_selection_enabled: bool, + pub(crate) row_group_filtering_enabled: bool, + pub(crate) row_selection_enabled: bool, +} + +/// Trait indicating that the implementing type streams into a stream of type `S` using +/// a reader of type `R`. +pub trait StreamsInto { + /// Stream from the reader and produce a stream of type `S`. + fn stream(self, reader: R) -> Result; } impl ArrowReader { @@ -378,7 +404,7 @@ impl ArrowReader { /// Using the Parquet page index, we build a `RowSelection` that rejects rows that are indicated /// as having been deleted by a positional delete, taking into account any row groups that have /// been skipped entirely by the filter predicate - fn build_deletes_row_selection( + pub(crate) fn build_deletes_row_selection( row_group_metadata_list: &[RowGroupMetaData], selected_row_groups: &Option>, positional_deletes: &DeleteVector, @@ -492,6 +518,103 @@ impl ArrowReader { Ok(results.into()) } + /// Helper function to add a `_file` column to a RecordBatch. + /// Takes the array and field to add, reducing code duplication. + fn create_file_field( + batch: RecordBatch, + file_array: ArrayRef, + file_field: Field, + field_id: i32, + ) -> Result { + let mut columns = batch.columns().to_vec(); + columns.push(file_array); + + let mut fields: Vec<_> = batch.schema().fields().iter().cloned().collect(); + fields.push(Arc::new(file_field.with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + field_id.to_string(), + )])))); + + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to add _file column to RecordBatch", + ) + .with_source(e) + }) + } + + /// Adds a `_file` column to the RecordBatch containing the file path. + /// Uses Run-End Encoding (REE) for maximum memory efficiency when the same + /// file path is repeated across all rows. + /// Note: This is only used in tests for now, for production usage we use the + /// non-REE version as it is Julia-compatible. + #[allow(dead_code)] + pub(crate) fn add_file_path_column_ree( + batch: RecordBatch, + file_path: &str, + field_name: &str, + field_id: i32, + ) -> Result { + let num_rows = batch.num_rows(); + + // Use Run-End Encoded array for optimal memory efficiency + // For a constant value repeated num_rows times, this stores: + // - run_ends: [num_rows] (one i32) for non-empty batches, or [] for empty batches + // - values: [file_path] (one string) for non-empty batches, or [] for empty batches + let run_ends = if num_rows == 0 { + Int32Array::from(Vec::::new()) + } else { + Int32Array::from(vec![num_rows as i32]) + }; + let values = if num_rows == 0 { + StringArray::from(Vec::<&str>::new()) + } else { + StringArray::from(vec![file_path]) + }; + + let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RunArray for _file column", + ) + .with_source(e) + })?; + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + // DataType is RunEndEncoded with Int32 run ends and Utf8 values + // Note: values field is nullable to match what RunArray::try_new(..) expects. + let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false)); + let values_field = Arc::new(Field::new("values", DataType::Utf8, true)); + let file_field = Field::new( + field_name, + DataType::RunEndEncoded(run_ends_field, values_field), + false, + ); + + Self::create_file_field(batch, Arc::new(file_array), file_field, field_id) + } + + /// Adds a `_file` column to the RecordBatch containing the file path. + /// Materializes the file path string for each row (no compression). + pub(crate) fn add_file_path_column( + batch: RecordBatch, + file_path: &str, + field_name: &str, + field_id: i32, + ) -> Result { + let num_rows = batch.num_rows(); + + // Create a StringArray with the file path repeated num_rows times + let file_array = StringArray::from(vec![file_path; num_rows]); + + // Per Iceberg spec, the _file column has reserved field ID RESERVED_FIELD_ID_FILE + let file_field = Field::new(field_name, DataType::Utf8, false); + + Self::create_file_field(batch, Arc::new(file_array), file_field, field_id) + } + fn build_field_id_set_and_map( parquet_schema: &SchemaDescriptor, predicate: &BoundPredicate, @@ -530,7 +653,7 @@ impl ArrowReader { } } - fn get_arrow_projection_mask( + pub(crate) fn get_arrow_projection_mask( field_ids: &[i32], iceberg_schema_of_task: &Schema, parquet_schema: &SchemaDescriptor, @@ -1487,6 +1610,7 @@ mod tests { use arrow_array::cast::AsArray; use arrow_array::{ArrayRef, LargeStringArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema, TimeUnit}; + use as_any::Downcast; use futures::TryStreamExt; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::arrow::{ArrowWriter, ProjectionMask}; @@ -1500,7 +1624,9 @@ mod tests { use crate::ErrorKind; use crate::arrow::reader::{CollectFieldIdVisitor, PARQUET_FIELD_ID_META_KEY}; - use crate::arrow::{ArrowReader, ArrowReaderBuilder}; + use crate::arrow::{ + ArrowReader, ArrowReaderBuilder, RESERVED_COL_NAME_FILE, RESERVED_FIELD_ID_FILE, + }; use crate::delete_vector::DeleteVector; use crate::expr::visitors::bound_predicate_visitor::visit; use crate::expr::{Bind, Predicate, Reference}; @@ -2292,6 +2418,321 @@ message schema { assert!(col_b.is_null(2)); } + #[test] + fn test_add_file_path_column_ree() { + use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + // Create a simple test batch with 2 columns and 3 rows + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let id_array = Int32Array::from(vec![1, 2, 3]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Charlie"]); + + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(id_array), + Arc::new(name_array), + ]) + .unwrap(); + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Add file path column with REE + let file_path = "/path/to/data/file.parquet"; + let result = ArrowReader::add_file_path_column_ree( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + assert!(result.is_ok(), "Should successfully add file path column"); + + let new_batch = result.unwrap(); + + // Verify the new batch has 3 columns + assert_eq!(new_batch.num_columns(), 3); + assert_eq!(new_batch.num_rows(), 3); + + // Verify schema has the _file column + let schema = new_batch.schema(); + assert_eq!(schema.fields().len(), 3); + + let file_field = schema.field(2); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + assert!(!file_field.is_nullable()); + + // Verify the field has the correct metadata + let metadata = file_field.metadata(); + assert_eq!( + metadata.get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the data type is RunEndEncoded + match file_field.data_type() { + DataType::RunEndEncoded(run_ends_field, values_field) => { + assert_eq!(run_ends_field.name(), "run_ends"); + assert_eq!(run_ends_field.data_type(), &DataType::Int32); + assert!(!run_ends_field.is_nullable()); + + assert_eq!(values_field.name(), "values"); + assert_eq!(values_field.data_type(), &DataType::Utf8); + } + _ => panic!("Expected RunEndEncoded data type for _file column"), + } + + // Verify the original columns are intact + let id_col = new_batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.values(), &[1, 2, 3]); + + let name_col = new_batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + + // Verify the file path column contains the correct value + // The _file column is a RunArray, so we need to decode it + let file_col = new_batch.column(2); + let run_array = file_col + .as_any() + .downcast_ref::>() + .expect("Expected RunArray for _file column"); + + // Verify the run array structure (should be optimally encoded) + let run_ends = run_array.run_ends(); + assert_eq!(run_ends.values().len(), 1, "Should have only 1 run end"); + assert_eq!( + run_ends.values()[0], + new_batch.num_rows() as i32, + "Run end should equal number of rows" + ); + + // Check that the single value in the RunArray is the expected file path + let values = run_array.values(); + let string_values = values.as_string::(); + assert_eq!(string_values.len(), 1, "Should have only 1 value"); + assert_eq!(string_values.value(0), file_path); + + assert!( + string_values + .downcast_ref::() + .unwrap() + .iter() + .all(|v| v == Some(file_path)) + ) + } + + #[test] + fn test_add_file_path_column_ree_empty_batch() { + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + // Create an empty batch + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = arrow_array::Int32Array::from(Vec::::new()); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + assert_eq!(batch.num_rows(), 0); + + // Add file path column to empty batch with REE + let file_path = "/empty/file.parquet"; + let result = ArrowReader::add_file_path_column_ree( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + + // Should succeed with empty RunArray for empty batches + assert!(result.is_ok()); + let new_batch = result.unwrap(); + assert_eq!(new_batch.num_rows(), 0); + assert_eq!(new_batch.num_columns(), 2); + + // Verify the _file column exists with correct schema + let schema = new_batch.schema(); + let file_field = schema.field(1); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + + // Should use RunEndEncoded even for empty batches + match file_field.data_type() { + DataType::RunEndEncoded(run_ends_field, values_field) => { + assert_eq!(run_ends_field.data_type(), &DataType::Int32); + assert_eq!(values_field.data_type(), &DataType::Utf8); + } + _ => panic!("Expected RunEndEncoded data type for _file column"), + } + + // Verify metadata with reserved field ID + assert_eq!( + file_field.metadata().get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the file path column is empty but properly structured + let file_path_column = new_batch.column(1); + assert_eq!(file_path_column.len(), 0); + } + + #[test] + fn test_add_file_path_column_special_characters() { + use arrow_array::{Int32Array, RecordBatch}; + use arrow_schema::{DataType, Field, Schema}; + + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = Int32Array::from(vec![42]); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + // Test with file path containing special characters (materialized version) + let file_path = "/path/with spaces/and-dashes/file_name.parquet"; + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + assert!(result.is_ok()); + + let new_batch = result.unwrap(); + let file_col = new_batch.column(1); + + // Verify the file path is correctly stored as a materialized StringArray + let str_arr = file_col.as_string::(); + assert_eq!(str_arr.value(0), file_path); + } + + #[test] + fn test_add_file_path_column() { + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + // Create a simple test batch with 2 columns and 3 rows + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let id_array = Int32Array::from(vec![1, 2, 3]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Charlie"]); + + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(id_array), + Arc::new(name_array), + ]) + .unwrap(); + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Add file path column with materialization + let file_path = "/path/to/data/file.parquet"; + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + assert!(result.is_ok(), "Should successfully add file path column"); + + let new_batch = result.unwrap(); + + // Verify the new batch has 3 columns + assert_eq!(new_batch.num_columns(), 3); + assert_eq!(new_batch.num_rows(), 3); + + // Verify schema has the _file column + let schema = new_batch.schema(); + assert_eq!(schema.fields().len(), 3); + + let file_field = schema.field(2); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + assert!(!file_field.is_nullable()); + + // Verify the field has the correct metadata + let metadata = file_field.metadata(); + assert_eq!( + metadata.get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the data type is Utf8 (materialized strings) + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify the original columns are intact + let id_col = new_batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.values(), &[1, 2, 3]); + + let name_col = new_batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + + // Verify the file path column contains the correct value for all rows + let file_col = new_batch.column(2).as_string::(); + for i in 0..new_batch.num_rows() { + assert_eq!(file_col.value(i), file_path); + } + } + + #[test] + fn test_add_file_path_column_empty_batch() { + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + // Create an empty batch + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let id_array = arrow_array::Int32Array::from(Vec::::new()); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(id_array)]).unwrap(); + + assert_eq!(batch.num_rows(), 0); + + // Add file path column to empty batch (materialized version) + let file_path = "/empty/file.parquet"; + let result = ArrowReader::add_file_path_column( + batch, + file_path, + RESERVED_COL_NAME_FILE, + RESERVED_FIELD_ID_FILE, + ); + + // Should succeed with empty StringArray + assert!(result.is_ok()); + let new_batch = result.unwrap(); + assert_eq!(new_batch.num_rows(), 0); + assert_eq!(new_batch.num_columns(), 2); + + // Verify the _file column exists with correct schema + let schema = new_batch.schema(); + let file_field = schema.field(1); + assert_eq!(file_field.name(), RESERVED_COL_NAME_FILE); + + // Should use Utf8 (materialized strings) + assert_eq!(file_field.data_type(), &DataType::Utf8); + + // Verify metadata with reserved field ID + assert_eq!( + file_field.metadata().get(PARQUET_FIELD_ID_META_KEY), + Some(&RESERVED_FIELD_ID_FILE.to_string()) + ); + + // Verify the file path column is empty but properly structured + let file_path_column = new_batch.column(1); + assert_eq!(file_path_column.len(), 0); + } + /// Test for bug where position deletes in later row groups are not applied correctly. /// /// When a file has multiple row groups and a position delete targets a row in a later diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index 4f6fd28483..3a3dfbb529 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -107,6 +107,26 @@ impl DeleteFileIndex { _ => unreachable!("Cannot be any other state than loaded"), } } + + pub(crate) async fn positional_deletes(&self) -> Vec { + let notifier = { + let guard = self.state.read().unwrap(); + match *guard { + DeleteFileIndexState::Populating(ref notifier) => notifier.clone(), + DeleteFileIndexState::Populated(ref index) => { + return index.positional_deletes(); + } + } + }; + + notifier.notified().await; + + let guard = self.state.read().unwrap(); + match guard.deref() { + DeleteFileIndexState::Populated(index) => index.positional_deletes(), + _ => unreachable!("Cannot be any other state than loaded"), + } + } } impl PopulatedDeleteFileIndex { @@ -210,6 +230,14 @@ impl PopulatedDeleteFileIndex { results } + + fn positional_deletes(&self) -> Vec { + self.pos_deletes_by_partition + .values() + .flatten() + .map(|ctx| ctx.as_ref().into()) + .collect() + } } #[cfg(test)] diff --git a/crates/iceberg/src/delete_vector.rs b/crates/iceberg/src/delete_vector.rs index f382bf079e..1040796034 100644 --- a/crates/iceberg/src/delete_vector.rs +++ b/crates/iceberg/src/delete_vector.rs @@ -23,9 +23,9 @@ use roaring::treemap::BitmapIter; use crate::{Error, ErrorKind, Result}; -#[derive(Debug, Default)] +#[derive(Debug, Default, Clone)] pub struct DeleteVector { - inner: RoaringTreemap, + pub inner: RoaringTreemap, } impl DeleteVector { diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs index aae8efed74..9c9c7460fb 100644 --- a/crates/iceberg/src/lib.rs +++ b/crates/iceberg/src/lib.rs @@ -97,3 +97,6 @@ pub mod writer; mod delete_vector; pub mod puffin; + +/// Utility functions and modules. +pub mod util; diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 3f7c29dbf4..e35c260be4 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -33,19 +33,23 @@ use crate::spec::{ }; use crate::{Error, ErrorKind, Result}; +pub(crate) type ManifestEntryFilterFn = dyn Fn(&ManifestEntryRef) -> bool + Send + Sync; + /// Wraps a [`ManifestFile`] alongside the objects that are needed /// to process it in a thread-safe manner pub(crate) struct ManifestFileContext { - manifest_file: ManifestFile, + pub manifest_file: ManifestFile, - sender: Sender, + pub sender: Sender, - field_ids: Arc>, - bound_predicates: Option>, - object_cache: Arc, - snapshot_schema: SchemaRef, - expression_evaluator_cache: Arc, - delete_file_index: DeleteFileIndex, + pub field_ids: Arc>, + pub bound_predicates: Option>, + pub object_cache: Arc, + pub snapshot_schema: SchemaRef, + pub expression_evaluator_cache: Arc, + pub delete_file_index: DeleteFileIndex, + + pub filter_fn: Option>, } /// Wraps a [`ManifestEntryRef`] alongside the objects that are needed @@ -74,12 +78,15 @@ impl ManifestFileContext { mut sender, expression_evaluator_cache, delete_file_index, + filter_fn, .. } = self; + let filter_fn = filter_fn.unwrap_or_else(|| Arc::new(|_| true)); + let manifest = object_cache.get_manifest(&manifest_file).await?; - for manifest_entry in manifest.entries() { + for manifest_entry in manifest.entries().iter().filter(|e| filter_fn(e)) { let manifest_entry_context = ManifestEntryContext { // TODO: refactor to avoid the expensive ManifestEntry clone manifest_entry: manifest_entry.clone(), @@ -189,7 +196,6 @@ impl PlanContext { ) -> Result> + 'static>> { let manifest_files = manifest_list.entries().iter(); - // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut filtered_mfcs = vec![]; for manifest_file in manifest_files { let tx = if manifest_file.content == ManifestContentType::Deletes { @@ -224,6 +230,7 @@ impl PlanContext { partition_bound_predicate, tx, delete_file_idx.clone(), + None, ); filtered_mfcs.push(Ok(mfc)); @@ -238,6 +245,7 @@ impl PlanContext { partition_filter: Option>, sender: Sender, delete_file_index: DeleteFileIndex, + filter_fn: Option>, ) -> ManifestFileContext { let bound_predicates = if let (Some(ref partition_bound_predicate), Some(snapshot_bound_predicate)) = @@ -260,6 +268,7 @@ impl PlanContext { field_ids: self.field_ids.clone(), expression_evaluator_cache: self.expression_evaluator_cache.clone(), delete_file_index, + filter_fn, } } } diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs new file mode 100644 index 0000000000..04db28a4c4 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -0,0 +1,142 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashSet; +use std::sync::Arc; + +use futures::channel::mpsc::Sender; + +use crate::Result; +use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; +use crate::delete_file_index::DeleteFileIndex; +use crate::io::object_cache::ObjectCache; +use crate::scan::ExpressionEvaluatorCache; +use crate::scan::context::{ManifestEntryContext, ManifestEntryFilterFn, ManifestFileContext}; +use crate::spec::{ + ManifestContentType, ManifestEntryRef, ManifestFile, Operation, SchemaRef, SnapshotRef, + TableMetadataRef, +}; + +#[derive(Debug)] +pub(crate) struct IncrementalPlanContext { + /// The snapshots involved in the incremental scan. + pub snapshots: Vec, + + /// The snapshot to start the incremental scan from. + pub from_snapshot: SnapshotRef, + + /// The metadata of the table being scanned. + pub table_metadata: TableMetadataRef, + + /// The schema of the snapshot to end the incremental scan at. + pub to_snapshot_schema: SchemaRef, + + /// The object cache to use for the scan. + pub object_cache: Arc, + + /// The field IDs to scan. + pub field_ids: Arc>, + + /// The expression evaluator cache to use for the scan. + pub expression_evaluator_cache: Arc, + + /// The caching delete file loader to use for the scan. + pub caching_delete_file_loader: CachingDeleteFileLoader, +} + +impl IncrementalPlanContext { + pub(crate) async fn build_manifest_file_contexts( + &self, + tx_data: Sender, + delete_file_idx: DeleteFileIndex, + delete_file_tx: Sender, + ) -> Result> + 'static>> { + // Validate that all snapshots are Append or Delete operations and collect their IDs + let snapshot_ids: HashSet = { + let mut ids = HashSet::new(); + for snapshot in self.snapshots.iter() { + let operation = &snapshot.summary().operation; + if !matches!( + operation, + Operation::Append | Operation::Overwrite | Operation::Delete + ) { + return Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + format!( + "Incremental scan only supports Append, Overwrite and Delete operations, but snapshot {} has operation {:?}", + snapshot.snapshot_id(), + operation + ), + )); + } + ids.insert(snapshot.snapshot_id()); + } + ids + }; + + let (manifest_files, filter_fn) = { + let mut manifest_files = HashSet::::new(); + for snapshot in self.snapshots.iter() { + let manifest_list = self + .object_cache + .get_manifest_list(snapshot, &self.table_metadata) + .await?; + for entry in manifest_list.entries() { + if !snapshot_ids.contains(&entry.added_snapshot_id) { + continue; + } + manifest_files.insert(entry.clone()); + } + } + let filter_fn: Option> = + Some(Arc::new(move |entry: &ManifestEntryRef| { + entry + .snapshot_id() + .map(|id| snapshot_ids.contains(&id)) + .unwrap_or(true) // Include entries without `snapshot_id`. + })); + + (manifest_files, filter_fn) + }; + + // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. + let mut mfcs = vec![]; + for manifest_file in &manifest_files { + let tx = if manifest_file.content == ManifestContentType::Deletes { + delete_file_tx.clone() + } else { + tx_data.clone() + }; + + let mfc = ManifestFileContext { + manifest_file: manifest_file.clone(), + bound_predicates: None, + sender: tx, + object_cache: self.object_cache.clone(), + snapshot_schema: self.to_snapshot_schema.clone(), + field_ids: self.field_ids.clone(), + expression_evaluator_cache: self.expression_evaluator_cache.clone(), + delete_file_index: delete_file_idx.clone(), + filter_fn: filter_fn.clone(), + }; + + mfcs.push(Ok(mfc)); + } + + Ok(Box::new(mfcs.into_iter())) + } +} diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs new file mode 100644 index 0000000000..1253353b4d --- /dev/null +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -0,0 +1,588 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Incremental table scan implementation. + +use std::collections::HashSet; +use std::sync::Arc; + +use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; +use crate::arrow::delete_filter::DeleteFilter; +use crate::arrow::{ + ArrowReaderBuilder, CombinedIncrementalBatchRecordStream, StreamsInto, + UnzippedIncrementalBatchRecordStream, +}; +use crate::delete_file_index::DeleteFileIndex; +use crate::io::FileIO; +use crate::scan::DeleteFileContext; +use crate::scan::cache::ExpressionEvaluatorCache; +use crate::scan::context::ManifestEntryContext; +use crate::spec::{DataContentType, ManifestStatus, Snapshot, SnapshotRef}; +use crate::table::Table; +use crate::util::snapshot::ancestors_between; +use crate::utils::available_parallelism; +use crate::{Error, ErrorKind, Result}; + +mod context; +use context::*; +mod task; +use futures::channel::mpsc::{Sender, channel}; +use futures::{SinkExt, StreamExt, TryStreamExt}; +use itertools::Itertools; +pub use task::*; + +use crate::runtime::spawn; + +/// Builder for an incremental table scan. +#[derive(Debug)] +pub struct IncrementalTableScanBuilder<'a> { + table: &'a Table, + // Defaults to `None`, which means all columns. + column_names: Option>, + from_snapshot_id: i64, + to_snapshot_id: i64, + batch_size: Option, + concurrency_limit_data_files: usize, + concurrency_limit_manifest_entries: usize, + concurrency_limit_manifest_files: usize, +} + +impl<'a> IncrementalTableScanBuilder<'a> { + pub(crate) fn new(table: &'a Table, from_snapshot_id: i64, to_snapshot_id: i64) -> Self { + let num_cpus = available_parallelism().get(); + Self { + table, + column_names: None, + from_snapshot_id, + to_snapshot_id, + batch_size: None, + concurrency_limit_data_files: num_cpus, + concurrency_limit_manifest_entries: num_cpus, + concurrency_limit_manifest_files: num_cpus, + } + } + + /// Set the batch size for reading data files. + pub fn with_batch_size(mut self, batch_size: Option) -> Self { + self.batch_size = batch_size; + self + } + + /// Select all columns of the table. + pub fn select_all(mut self) -> Self { + self.column_names = None; + self + } + + /// Select no columns of the table. + pub fn select_empty(mut self) -> Self { + self.column_names = Some(vec![]); + self + } + + /// Select some columns of the table. + pub fn select(mut self, column_names: impl IntoIterator) -> Self { + self.column_names = Some( + column_names + .into_iter() + .map(|item| item.to_string()) + .collect(), + ); + self + } + + /// Set the `from_snapshot_id` for the incremental scan. + pub fn from_snapshot_id(mut self, from_snapshot_id: i64) -> Self { + self.from_snapshot_id = from_snapshot_id; + self + } + + /// Set the `to_snapshot_id` for the incremental scan. + pub fn to_snapshot_id(mut self, to_snapshot_id: i64) -> Self { + self.to_snapshot_id = to_snapshot_id; + self + } + + /// Set the concurrency limit for reading data files. + pub fn with_concurrency_limit_data_files(mut self, limit: usize) -> Self { + self.concurrency_limit_data_files = limit; + self + } + + /// Set the concurrency limit for reading manifest entries. + pub fn with_concurrency_limit_manifest_entries(mut self, limit: usize) -> Self { + self.concurrency_limit_manifest_entries = limit; + self + } + + /// Set the concurrency limit for reading manifest files. + pub fn with_concurrency_limit_manifest_files(mut self, limit: usize) -> Self { + self.concurrency_limit_manifest_files = limit; + self + } + + /// Build the incremental table scan. + pub fn build(self) -> Result { + let snapshot_from: Arc = self + .table + .metadata() + .snapshot_by_id(self.from_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", self.from_snapshot_id), + ) + })? + .clone(); + + let snapshot_to: Arc = self + .table + .metadata() + .snapshot_by_id(self.to_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", self.to_snapshot_id), + ) + })? + .clone(); + + let snapshots = ancestors_between( + &self.table.metadata_ref(), + snapshot_to.snapshot_id(), + Some(snapshot_from.snapshot_id()), + ) + .collect_vec(); + + if !snapshots.is_empty() { + assert_eq!( + snapshots.first().map(|s| s.snapshot_id()), + Some(snapshot_to.snapshot_id()) + ); + } + + let schema = snapshot_to.schema(self.table.metadata())?; + + if let Some(column_names) = self.column_names.as_ref() { + for column_name in column_names { + if schema.field_by_name(column_name).is_none() { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Column {} not found in table. Schema: {}", + column_name, schema + ), + )); + } + } + } + + let mut field_ids = vec![]; + let column_names = self.column_names.clone().unwrap_or_else(|| { + schema + .as_struct() + .fields() + .iter() + .map(|f| f.name.clone()) + .collect() + }); + + for column_name in column_names.iter() { + let field_id = schema.field_id_by_name(column_name).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Column {} not found in table. Schema: {}", + column_name, schema + ), + ) + })?; + + schema + .as_struct() + .field_by_id(field_id) + .ok_or_else(|| { + Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Column {} is not a direct child of schema but a nested field, which is not supported now. Schema: {}", + column_name, schema + ), + ) + })?; + + field_ids.push(field_id); + } + + let plan_context = IncrementalPlanContext { + snapshots, + from_snapshot: snapshot_from, + table_metadata: self.table.metadata_ref(), + to_snapshot_schema: schema, + object_cache: self.table.object_cache().clone(), + field_ids: Arc::new(field_ids), + expression_evaluator_cache: Arc::new(ExpressionEvaluatorCache::new()), + caching_delete_file_loader: CachingDeleteFileLoader::new( + self.table.file_io().clone(), + self.concurrency_limit_data_files, + ), + }; + + Ok(IncrementalTableScan { + plan_context, + file_io: self.table.file_io().clone(), + column_names: Some(column_names), + batch_size: self.batch_size, + concurrency_limit_data_files: self.concurrency_limit_data_files, + concurrency_limit_manifest_entries: self.concurrency_limit_manifest_entries, + concurrency_limit_manifest_files: self.concurrency_limit_manifest_files, + }) + } +} + +/// An incremental table scan. +#[derive(Debug)] +pub struct IncrementalTableScan { + plan_context: IncrementalPlanContext, + file_io: FileIO, + column_names: Option>, + batch_size: Option, + concurrency_limit_data_files: usize, + concurrency_limit_manifest_entries: usize, + concurrency_limit_manifest_files: usize, +} + +impl IncrementalTableScan { + /// Returns the `from` snapshot of this incremental table scan. + pub fn snapshot_from(&self) -> &SnapshotRef { + &self.plan_context.from_snapshot + } + + /// Returns the snapshots involved in this incremental table scan. + pub fn snapshots(&self) -> &[SnapshotRef] { + &self.plan_context.snapshots + } + + /// Returns the `to` snapshot of this incremental table scan. + pub fn snapshot_to(&self) -> &SnapshotRef { + self.snapshots() + .first() + .expect("There is always at least one snapshot") + } + + /// Returns the selected column names of this incremental table scan. + /// If `None`, all columns are selected. + pub fn column_names(&self) -> Option<&[String]> { + self.column_names.as_deref() + } + + /// Plans the files to be read in this incremental table scan. + pub async fn plan_files(&self) -> Result { + let concurrency_limit_manifest_files = self.concurrency_limit_manifest_files; + let concurrency_limit_manifest_entries = self.concurrency_limit_manifest_entries; + + // Used to stream `ManifestEntryContexts` between stages of the planning operation. + let (manifest_entry_data_ctx_tx, manifest_entry_data_ctx_rx) = + channel(concurrency_limit_manifest_files); + let (manifest_entry_delete_ctx_tx, manifest_entry_delete_ctx_rx) = + channel(concurrency_limit_manifest_files); + + // Used to stream the results back to the caller. + let (file_scan_task_tx, file_scan_task_rx) = channel(concurrency_limit_manifest_entries); + + let (delete_file_idx, delete_file_tx) = DeleteFileIndex::new(); + + let manifest_file_contexts = self + .plan_context + .build_manifest_file_contexts( + manifest_entry_data_ctx_tx, + delete_file_idx.clone(), + manifest_entry_delete_ctx_tx, + ) + .await?; + + let mut channel_for_manifest_error: Sender> = file_scan_task_tx.clone(); + + // Concurrently load all [`Manifest`]s and stream their [`ManifestEntry`]s + spawn(async move { + let result = futures::stream::iter(manifest_file_contexts) + .try_for_each_concurrent(concurrency_limit_manifest_files, |ctx| async move { + ctx.fetch_manifest_and_stream_manifest_entries().await + }) + .await; + + if let Err(error) = result { + let _ = channel_for_manifest_error.send(Err(error)).await; + } + }); + + let mut channel_for_data_manifest_entry_error = file_scan_task_tx.clone(); + let mut channel_for_delete_manifest_entry_error = file_scan_task_tx.clone(); + + // Process the delete file [`ManifestEntry`] stream in parallel. Builds the delete + // index below. + spawn(async move { + let result = manifest_entry_delete_ctx_rx + .map(|me_ctx| Ok((me_ctx, delete_file_tx.clone()))) + .try_for_each_concurrent( + concurrency_limit_manifest_entries, + |(manifest_entry_context, tx)| async move { + spawn(async move { + Self::process_delete_manifest_entry(tx, manifest_entry_context).await + }) + .await + }, + ) + .await; + + if let Err(error) = result { + let _ = channel_for_delete_manifest_entry_error + .send(Err(error)) + .await; + } + }) + .await; + + // TODO: Streaming this into the delete index seems somewhat redundant, as we + // could directly stream into the CachingDeleteFileLoader and instantly load the + // delete files. + let positional_deletes = delete_file_idx.positional_deletes().await; + let result = self + .plan_context + .caching_delete_file_loader + .load_deletes( + &positional_deletes, + self.plan_context.to_snapshot_schema.clone(), + ) + .await; + + // Build the delete filter from the loaded deletes. + let delete_filter = match result { + Ok(loaded_deletes) => loaded_deletes.unwrap(), + Err(e) => { + return Err(Error::new( + ErrorKind::Unexpected, + format!("Failed to load positional deletes: {}", e), + )); + } + }; + + // Process the data file [`ManifestEntry`] stream in parallel + let filter = delete_filter.clone(); + spawn(async move { + let result = manifest_entry_data_ctx_rx + .map(|me_ctx| Ok((me_ctx, file_scan_task_tx.clone()))) + .try_for_each_concurrent( + concurrency_limit_manifest_entries, + |(manifest_entry_context, tx)| { + let filter = filter.clone(); + async move { + if manifest_entry_context.manifest_entry.status() + == ManifestStatus::Added + { + spawn(async move { + Self::process_data_manifest_entry( + tx, + manifest_entry_context, + &filter, + ) + .await + }) + .await + } else if manifest_entry_context.manifest_entry.status() + == ManifestStatus::Deleted + { + // TODO (RAI-43291): Process deleted files + Err(Error::new( + ErrorKind::FeatureUnsupported, + "Processing deleted data files is not supported yet in incremental scans", + )) + } else { + Ok(()) + } + } + }, + ) + .await; + + if let Err(error) = result { + let _ = channel_for_data_manifest_entry_error.send(Err(error)).await; + } + }); + + // Collect all append tasks. + let mut tasks = file_scan_task_rx.try_collect::>().await?; + + // Compute those file paths that have been appended. + let appended_files = tasks + .iter() + .filter_map(|task| match task { + IncrementalFileScanTask::Append(append_task) => { + Some(append_task.data_file_path.clone()) + } + _ => None, + }) + .collect::>(); + + // Augment `tasks` with delete tasks. + // First collect paths to process (paths that weren't appended in this scan range) + let delete_paths: Vec = delete_filter.with_read(|state| { + Ok(state + .delete_vectors() + .keys() + .filter(|path| !appended_files.contains::(path)) + .cloned() + .collect()) + })?; + + // Now remove and take ownership of each delete vector + for path in delete_paths { + let delete_vector_arc = delete_filter.with_write(|state| { + state.remove_delete_vector(&path).ok_or_else(|| { + Error::new( + ErrorKind::Unexpected, + format!("DeleteVector for path {} not found", path), + ) + }) + })?; + + // Try to unwrap the Arc to avoid cloning the DeleteVector + let delete_vector_inner = Arc::try_unwrap(delete_vector_arc) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "DeleteVector Arc has multiple references, cannot take ownership", + ) + })? + .into_inner() + .map_err(|e| { + Error::new(ErrorKind::Unexpected, "Failed to unwrap DeleteVector Mutex") + .with_source(e) + })?; + + let delete_task = IncrementalFileScanTask::Delete(path, delete_vector_inner); + tasks.push(delete_task); + } + + // We actually would not need a stream here, but we can keep it compatible with + // other scan types. + Ok(futures::stream::iter(tasks).map(Ok).boxed()) + } + + /// Returns an [`CombinedIncrementalBatchRecordStream`] for this incremental table scan. + pub async fn to_arrow(&self) -> Result { + let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); + + if let Some(batch_size) = self.batch_size { + arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); + } + + let arrow_reader = arrow_reader_builder.build(); + let file_scan_task_stream = self.plan_files().await?; + file_scan_task_stream.stream(arrow_reader) + } + + /// Returns an [`UnzippedIncrementalBatchRecordStream`] for this incremental table scan. + /// This stream will yield separate streams for appended and deleted record batches. + pub async fn to_unzipped_arrow(&self) -> Result { + let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) + .with_data_file_concurrency_limit(self.concurrency_limit_data_files) + .with_row_group_filtering_enabled(true) + .with_row_selection_enabled(true); + + if let Some(batch_size) = self.batch_size { + arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); + } + + let arrow_reader = arrow_reader_builder.build(); + let file_scan_task_stream = self.plan_files().await?; + file_scan_task_stream.stream(arrow_reader) + } + + async fn process_delete_manifest_entry( + mut delete_file_ctx_tx: Sender, + manifest_entry_context: ManifestEntryContext, + ) -> Result<()> { + // Abort the plan if we encounter a manifest entry for a data file or equality + // deletes. + if manifest_entry_context.manifest_entry.content_type() == DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a data file in a delete file manifest", + )); + } else if manifest_entry_context.manifest_entry.content_type() + == DataContentType::EqualityDeletes + { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Equality deletes are not supported yet in incremental scans", + )); + } + + // Abort if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() + && manifest_entry_context.manifest_entry.content_type() + == DataContentType::PositionDeletes + { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Processing deleted (position) delete files is not supported yet in incremental scans", + )); + } + + delete_file_ctx_tx + .send(DeleteFileContext { + manifest_entry: manifest_entry_context.manifest_entry.clone(), + partition_spec_id: manifest_entry_context.partition_spec_id, + }) + .await?; + Ok(()) + } + + async fn process_data_manifest_entry( + mut file_scan_task_tx: Sender>, + manifest_entry_context: ManifestEntryContext, + delete_filter: &DeleteFilter, + ) -> Result<()> { + // Skip processing this manifest entry if it has been marked as deleted. + if !manifest_entry_context.manifest_entry.is_alive() { + return Ok(()); + } + + // Abort the plan if we encounter a manifest entry for a delete file + if manifest_entry_context.manifest_entry.content_type() != DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a delete file in a data file manifest", + )); + } + + let file_scan_task = IncrementalFileScanTask::append_from_manifest_entry( + &manifest_entry_context, + delete_filter, + ); + + file_scan_task_tx.send(Ok(file_scan_task)).await?; + Ok(()) + } +} + +#[cfg(test)] +mod tests; diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs new file mode 100644 index 0000000000..4b2f244410 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::{Arc, Mutex}; + +use futures::stream::BoxStream; + +use crate::Result; +use crate::arrow::delete_filter::DeleteFilter; +use crate::delete_vector::DeleteVector; +use crate::scan::context::ManifestEntryContext; +use crate::spec::{DataFileFormat, Schema, SchemaRef}; + +/// A file scan task for appended data files in an incremental scan. +#[derive(Debug, Clone)] +pub struct AppendedFileScanTask { + /// The start offset of the file to scan. + pub start: u64, + /// The length of the file to scan. + pub length: u64, + /// The number of records in the file. + pub record_count: Option, + /// The path to the data file to scan. + pub data_file_path: String, + /// The format of the data file to scan. + pub data_file_format: DataFileFormat, + /// The schema of the data file to scan. + pub schema: crate::spec::SchemaRef, + /// The field ids to project. + pub project_field_ids: Vec, + /// The optional positional deletes associated with this data file. + pub positional_deletes: Option>>, +} + +impl AppendedFileScanTask { + /// Returns the data file path of this appended file scan task. + pub fn data_file_path(&self) -> &str { + &self.data_file_path + } + + /// Returns the schema of this file scan task as a reference + pub fn schema(&self) -> &Schema { + &self.schema + } + + /// Returns the schema of this file scan task as a SchemaRef + pub fn schema_ref(&self) -> SchemaRef { + self.schema.clone() + } +} + +/// The stream of incremental file scan tasks. +pub type IncrementalFileScanTaskStream = BoxStream<'static, Result>; + +/// An incremental file scan task, which can be either an appended data file or positional +/// deletes. +#[derive(Debug, Clone)] +pub enum IncrementalFileScanTask { + /// An appended data file. + Append(AppendedFileScanTask), + /// Deleted records of a data file. First argument is the file path, second the delete + /// vector. + Delete(String, DeleteVector), +} + +impl IncrementalFileScanTask { + /// Create an `IncrementalFileScanTask::Append` from a `ManifestEntryContext` and `DeleteFilter`. + pub(crate) fn append_from_manifest_entry( + manifest_entry_context: &ManifestEntryContext, + delete_filter: &DeleteFilter, + ) -> Self { + let data_file_path = manifest_entry_context.manifest_entry.file_path(); + IncrementalFileScanTask::Append(AppendedFileScanTask { + start: 0, + length: manifest_entry_context.manifest_entry.file_size_in_bytes(), + record_count: Some(manifest_entry_context.manifest_entry.record_count()), + data_file_path: data_file_path.to_string(), + data_file_format: manifest_entry_context.manifest_entry.file_format(), + schema: manifest_entry_context.snapshot_schema.clone(), + project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), + positional_deletes: delete_filter.get_delete_vector_for_path(data_file_path), + }) + } + + /// Returns the data file path of this incremental file scan task. + pub fn data_file_path(&self) -> &str { + match self { + IncrementalFileScanTask::Append(task) => task.data_file_path(), + IncrementalFileScanTask::Delete(path, _) => path, + } + } +} diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs new file mode 100644 index 0000000000..05724a54a4 --- /dev/null +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -0,0 +1,1703 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashMap; +use std::fs; +use std::fs::File; +use std::sync::Arc; + +use arrow_array::cast::AsArray; +use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use parquet::arrow::{ArrowWriter, PARQUET_FIELD_ID_META_KEY}; +use parquet::basic::Compression; +use parquet::file::properties::WriterProperties; +use tempfile::TempDir; +use uuid::Uuid; + +use crate::TableIdent; +use crate::io::{FileIO, OutputFile}; +use crate::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, ManifestEntry, ManifestListWriter, + ManifestStatus, ManifestWriterBuilder, PartitionSpec, Struct, TableMetadata, +}; +use crate::table::Table; + +/// Represents an operation to perform on a snapshot of a table with schema (id: Int32, +/// data: String). +#[derive(Debug, Clone)] +pub enum Operation { + /// Add rows with the given (n, data) tuples, and write to the specified parquet file name. + /// Example: `Add(vec![(1, "a".to_string()), (2, "b".to_string())], "data-1.parquet".to_string())` + /// adds two rows with n=1,2 and data="a","b" to a file named "data-1.parquet" + Add(Vec<(i32, String)>, String), + + /// Delete rows by their positions within specific parquet files (uses positional deletes). + /// Takes a vector of (position, file_name) tuples specifying which position in which file to delete. + /// Example: `Delete(vec![(0, "data-1.parquet"), (1, "data-1.parquet")])` deletes positions 0 and 1 from data-1.parquet + Delete(Vec<(i64, String)>), + + /// Overwrite operation that can append new rows, delete specific positions, and remove entire data files. + /// This is a combination of append and delete operations in a single atomic snapshot. + /// + /// Parameters: + /// 1. Rows to append: Vec<(n, data)> tuples and the filename to write them to + /// 2. Positions to delete: Vec<(position, file_name)> tuples for positional deletes + /// 3. Data files to delete: Vec of file names to completely remove + /// + /// All three parameters can be empty, allowing for various combinations: + /// - Pure append: `Overwrite((rows, "file.parquet"), vec![], vec![])` + /// - Pure positional delete: `Overwrite((vec![], ""), vec![(pos, "file")], vec![])` + /// - Pure file deletion: `Overwrite((vec![], ""), vec![], vec!["file.parquet"])` + /// - Delete entire files: `Overwrite((vec![], ""), vec![], vec!["old-file.parquet"])` + /// + /// Example: `Overwrite((vec![(1, "new".to_string())], "new.parquet"), vec![(0, "old.parquet")], vec!["remove.parquet"])` + /// This adds new data to "new.parquet", deletes position 0 from "old.parquet", and removes "remove.parquet" entirely. + Overwrite( + (Vec<(i32, String)>, String), + Vec<(i64, String)>, + Vec, + ), +} + +/// Tracks the state of data files across snapshots +#[derive(Debug, Clone)] +struct DataFileInfo { + path: String, + snapshot_id: i64, + sequence_number: i64, + n_values: Vec, +} + +/// Test fixture that creates a table with custom snapshots based on operations. +/// +/// # Example +/// ``` +/// let fixture = IncrementalTestFixture::new(vec![ +/// Operation::Add(vec![], "empty.parquet".to_string()), // Empty snapshot +/// Operation::Add( +/// vec![ +/// (1, "1".to_string()), +/// (2, "2".to_string()), +/// (3, "3".to_string()), +/// ], +/// "data-1.parquet".to_string(), +/// ), // Add 3 rows +/// Operation::Delete(vec![(1, "data-1.parquet".to_string())]), // Delete position 1 from data-1.parquet +/// ]) +/// .await; +/// ``` +pub struct IncrementalTestFixture { + pub table_location: String, + pub table: Table, + _tmp_dir: TempDir, // Keep temp dir alive +} + +impl IncrementalTestFixture { + /// Create a new test fixture with the given operations. + pub async fn new(operations: Vec) -> Self { + // Use pwd + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().join("incremental_test_table"); + + // Create directory structure + fs::create_dir_all(table_location.join("metadata")).unwrap(); + fs::create_dir_all(table_location.join("data")).unwrap(); + + let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) + .unwrap() + .build() + .unwrap(); + + let num_snapshots = operations.len(); + let current_snapshot_id = num_snapshots as i64; + let last_sequence_number = (num_snapshots - 1) as i64; + + // Build the snapshots JSON dynamically + let mut snapshots_json = Vec::new(); + let mut snapshot_log_json = Vec::new(); + let mut manifest_list_locations = Vec::new(); + + for (i, op) in operations.iter().enumerate() { + let snapshot_id = (i + 1) as i64; + let parent_id = if i == 0 { None } else { Some(i as i64) }; + let sequence_number = i as i64; + let timestamp = 1515100955770 + (i as i64 * 1000); + + let operation_type = match op { + Operation::Add(..) => "append", + Operation::Delete(..) => "delete", + Operation::Overwrite(..) => "overwrite", + }; + + let manifest_list_location = + table_location.join(format!("metadata/snap-{}-manifest-list.avro", snapshot_id)); + manifest_list_locations.push(manifest_list_location.clone()); + + let parent_str = if let Some(pid) = parent_id { + format!(r#""parent-snapshot-id": {},"#, pid) + } else { + String::new() + }; + + snapshots_json.push(format!( + r#" {{ + "snapshot-id": {}, + {} + "timestamp-ms": {}, + "sequence-number": {}, + "summary": {{"operation": "{}"}}, + "manifest-list": "{}", + "schema-id": 0 + }}"#, + snapshot_id, + parent_str, + timestamp, + sequence_number, + operation_type, + manifest_list_location.display() + )); + + snapshot_log_json.push(format!( + r#" {{"snapshot-id": {}, "timestamp-ms": {}}}"#, + snapshot_id, timestamp + )); + } + + let snapshots_str = snapshots_json.join(",\n"); + let snapshot_log_str = snapshot_log_json.join(",\n"); + + // Create the table metadata + let metadata_json = format!( + r#"{{ + "format-version": 2, + "table-uuid": "{}", + "location": "{}", + "last-sequence-number": {}, + "last-updated-ms": 1602638573590, + "last-column-id": 2, + "current-schema-id": 0, + "schemas": [ + {{ + "type": "struct", + "schema-id": 0, + "fields": [ + {{"id": 1, "name": "n", "required": true, "type": "int"}}, + {{"id": 2, "name": "data", "required": true, "type": "string"}} + ] + }} + ], + "default-spec-id": 0, + "partition-specs": [ + {{ + "spec-id": 0, + "fields": [] + }} + ], + "last-partition-id": 0, + "default-sort-order-id": 0, + "sort-orders": [ + {{ + "order-id": 0, + "fields": [] + }} + ], + "properties": {{}}, + "current-snapshot-id": {}, + "snapshots": [ +{} + ], + "snapshot-log": [ +{} + ], + "metadata-log": [] +}}"#, + Uuid::new_v4(), + table_location.display(), + last_sequence_number, + current_snapshot_id, + snapshots_str, + snapshot_log_str + ); + + let table_metadata_location = table_location.join("metadata/v1.json"); + let table_metadata = serde_json::from_str::(&metadata_json).unwrap(); + + let table = Table::builder() + .metadata(table_metadata) + .identifier(TableIdent::from_strs(["db", "incremental_test"]).unwrap()) + .file_io(file_io.clone()) + .metadata_location(table_metadata_location.as_os_str().to_str().unwrap()) + .build() + .unwrap(); + + let mut fixture = Self { + table_location: table_location.to_str().unwrap().to_string(), + table, + _tmp_dir: tmp_dir, + }; + + // Setup all snapshots based on operations + fixture.setup_snapshots(operations).await; + + fixture + } + + fn next_manifest_file(&self) -> OutputFile { + self.table + .file_io() + .new_output(format!( + "{}/metadata/manifest_{}.avro", + self.table_location, + Uuid::new_v4() + )) + .unwrap() + } + + async fn setup_snapshots(&mut self, operations: Vec) { + let current_schema = self + .table + .metadata() + .current_snapshot() + .unwrap() + .schema(self.table.metadata()) + .unwrap(); + let partition_spec = Arc::new(PartitionSpec::unpartition_spec()); + let empty_partition = Struct::empty(); + + // Track all data files and their contents across snapshots + let mut data_files: Vec = Vec::new(); + #[allow(clippy::type_complexity)] + let mut delete_files: Vec<(String, i64, i64, Vec<(String, i64)>)> = Vec::new(); // (path, snapshot_id, sequence_number, [(data_file_path, position)]) + + for (snapshot_idx, operation) in operations.iter().enumerate() { + let snapshot_id = (snapshot_idx + 1) as i64; + let sequence_number = snapshot_idx as i64; + let parent_snapshot_id = if snapshot_idx == 0 { + None + } else { + Some(snapshot_idx as i64) + }; + + match operation { + Operation::Add(rows, file_name) => { + // Extract n_values and data_values from tuples + let n_values: Vec = rows.iter().map(|(n, _)| *n).collect(); + let data_values: Vec = rows.iter().map(|(_, d)| d.clone()).collect(); + + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + // Add existing data files from previous snapshots + for data_file in &data_files { + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new data if not empty + if !n_values.is_empty() { + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + self.write_parquet_file(&data_file_path, &n_values, &data_values) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this data file + data_files.push(DataFileInfo { + path: data_file_path, + snapshot_id, + sequence_number, + n_values, + }); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Create delete manifest if there are any delete files + let mut manifests = vec![data_manifest]; + if !delete_files.is_empty() { + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + manifests.push(delete_writer.write_manifest_file().await.unwrap()); + } + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(manifests.into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + } + + Operation::Delete(positions_to_delete) => { + // Group deletes by file + let mut deletes_by_file: HashMap> = HashMap::new(); + + for (position, file_name) in positions_to_delete { + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + deletes_by_file + .entry(data_file_path) + .or_default() + .push(*position); + } + + // Create data manifest with existing data files + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + for data_file in &data_files { + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Create delete manifest + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + // Add existing delete files + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new delete files + for (data_file_path, positions) in deletes_by_file { + let delete_file_path = format!( + "{}/data/delete-{}-{}.parquet", + &self.table_location, + snapshot_id, + Uuid::new_v4() + ); + self.write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; + + delete_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(positions.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this delete file + delete_files.push(( + delete_file_path, + snapshot_id, + sequence_number, + positions + .into_iter() + .map(|pos| (data_file_path.clone(), pos)) + .collect(), + )); + } + + let delete_manifest = delete_writer.write_manifest_file().await.unwrap(); + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(vec![data_manifest, delete_manifest].into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + } + + Operation::Overwrite((rows, file_name), positions_to_delete, files_to_delete) => { + // Overwrite creates a single snapshot that can: + // 1. Add new data files + // 2. Delete positions from existing files + // 3. Remove entire data files + + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_data(); + + // Determine which files to delete + let files_to_delete_set: std::collections::HashSet = files_to_delete + .iter() + .map(|f| format!("{}/data/{}", &self.table_location, f)) + .collect(); + + // Add existing data files (mark deleted ones as DELETED, others as EXISTING) + for data_file in &data_files { + if files_to_delete_set.contains(&data_file.path) { + // Mark file for deletion + data_writer + .add_delete_entry( + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } else { + // Keep existing file + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + } + + // Add new data file if rows provided + if !rows.is_empty() { + let n_values: Vec = rows.iter().map(|(n, _)| *n).collect(); + let data_values: Vec = + rows.iter().map(|(_, d)| d.clone()).collect(); + let data_file_path = format!("{}/data/{}", &self.table_location, file_name); + + self.write_parquet_file(&data_file_path, &n_values, &data_values) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(1024) + .record_count(n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this new data file + data_files.push(DataFileInfo { + path: data_file_path, + snapshot_id, + sequence_number, + n_values, + }); + } + + // Remove deleted files from tracking + data_files.retain(|df| !files_to_delete_set.contains(&df.path)); + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Handle positional deletes if any + let mut manifests = vec![data_manifest]; + + if !positions_to_delete.is_empty() || !delete_files.is_empty() { + let mut delete_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema.clone(), + partition_spec.as_ref().clone(), + ) + .build_v2_deletes(); + + // Add existing delete files + for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + { + let delete_count = delete_files + .iter() + .filter(|(p, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes)| deletes.len()) + .sum::(); + + delete_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(*del_snapshot_id) + .sequence_number(*del_sequence_number) + .file_sequence_number(*del_sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(delete_count as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + + // Add new positional delete files + if !positions_to_delete.is_empty() { + // Group deletes by file + let mut deletes_by_file: HashMap> = HashMap::new(); + for (position, file_name) in positions_to_delete { + let data_file_path = + format!("{}/data/{}", &self.table_location, file_name); + deletes_by_file + .entry(data_file_path) + .or_default() + .push(*position); + } + + for (data_file_path, positions) in deletes_by_file { + let delete_file_path = format!( + "{}/data/delete-{}-{}.parquet", + &self.table_location, + snapshot_id, + Uuid::new_v4() + ); + self.write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; + + delete_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::PositionDeletes) + .file_path(delete_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(512) + .record_count(positions.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Track this delete file + delete_files.push(( + delete_file_path, + snapshot_id, + sequence_number, + positions + .into_iter() + .map(|pos| (data_file_path.clone(), pos)) + .collect(), + )); + } + } + + manifests.push(delete_writer.write_manifest_file().await.unwrap()); + } + + // Write manifest list + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(manifests.into_iter()) + .unwrap(); + + manifest_list_write.close().await.unwrap(); + } + } + } + } + + async fn write_parquet_file(&self, path: &str, n_values: &[i32], data_values: &[String]) { + let schema = { + let fields = vec![ + arrow_schema::Field::new("n", arrow_schema::DataType::Int32, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ), + arrow_schema::Field::new("data", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let col_n = Arc::new(Int32Array::from(n_values.to_vec())) as ArrayRef; + let col_data = Arc::new(StringArray::from(data_values.to_vec())) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![col_n, col_data]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let file = File::create(path).unwrap(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + + async fn write_positional_delete_file( + &self, + path: &str, + data_file_path: &str, + positions: &[i64], + ) { + let schema = { + let fields = vec![ + arrow_schema::Field::new("file_path", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2147483546".to_string(), + )])), + arrow_schema::Field::new("pos", arrow_schema::DataType::Int64, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2147483545".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let file_paths: Vec<&str> = vec![data_file_path; positions.len()]; + let col_file_path = Arc::new(StringArray::from(file_paths)) as ArrayRef; + let col_pos = Arc::new(arrow_array::Int64Array::from(positions.to_vec())) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![col_file_path, col_pos]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let file = File::create(path).unwrap(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + + /// Verify incremental scan results. + /// + /// Verifies that the incremental scan contains the expected appended and deleted records. + pub async fn verify_incremental_scan( + &self, + from_snapshot_id: i64, + to_snapshot_id: i64, + expected_appends: Vec<(i32, &str)>, + expected_deletes: Vec<(u64, &str)>, + ) { + use arrow_array::cast::AsArray; + use arrow_select::concat::concat_batches; + use futures::TryStreamExt; + + let incremental_scan = self + .table + .incremental_scan(from_snapshot_id, to_snapshot_id) + .build() + .unwrap(); + + let stream = incremental_scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Separate appends and deletes + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + let delete_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Delete) + .map(|(_, b)| b.clone()) + .collect(); + + // Verify appended records + if !append_batches.is_empty() { + let append_batch = + concat_batches(&append_batches[0].schema(), append_batches.iter()).unwrap(); + + let n_array = append_batch + .column(0) + .as_primitive::(); + let data_array = append_batch.column(1).as_string::(); + + let mut appended_pairs: Vec<(i32, String)> = (0..append_batch.num_rows()) + .map(|i| (n_array.value(i), data_array.value(i).to_string())) + .collect(); + appended_pairs.sort(); + + let expected_appends: Vec<(i32, String)> = expected_appends + .into_iter() + .map(|(n, s)| (n, s.to_string())) + .collect(); + + assert_eq!(appended_pairs, expected_appends); + } else { + assert!(expected_appends.is_empty(), "Expected appends but got none"); + } + + // Verify deleted records + if !delete_batches.is_empty() { + let delete_batch = + concat_batches(&delete_batches[0].schema(), delete_batches.iter()).unwrap(); + + let pos_array = delete_batch + .column(0) + .as_primitive::(); + + // The file path column is a StringArray with materialized values + let file_path_column = delete_batch.column(1); + let file_path_array = file_path_column.as_string::(); + + let mut deleted_pairs: Vec<(u64, String)> = (0..delete_batch.num_rows()) + .map(|i| { + let pos = pos_array.value(i); + let file_path = file_path_array.value(i).to_string(); + (pos, file_path) + }) + .collect(); + deleted_pairs.sort(); + + let expected_deletes: Vec<(u64, String)> = expected_deletes + .into_iter() + .map(|(pos, file)| (pos, file.to_string())) + .collect(); + + assert_eq!(deleted_pairs, expected_deletes); + } else { + assert!(expected_deletes.is_empty(), "Expected deletes but got none"); + } + } +} + +#[tokio::test] +async fn test_incremental_fixture_simple() { + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), + Operation::Add( + vec![ + (1, "1".to_string()), + (2, "2".to_string()), + (3, "3".to_string()), + ], + "data-2.parquet".to_string(), + ), + Operation::Delete(vec![(1, "data-2.parquet".to_string())]), // Delete position 1 (n=2, data="2") + ]) + .await; + + // Verify we have 3 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 3); + + // Verify snapshot IDs + assert_eq!(snapshots[0].snapshot_id(), 1); + assert_eq!(snapshots[1].snapshot_id(), 2); + assert_eq!(snapshots[2].snapshot_id(), 3); + + // Verify parent relationships + assert_eq!(snapshots[0].parent_snapshot_id(), None); + assert_eq!(snapshots[1].parent_snapshot_id(), Some(1)); + assert_eq!(snapshots[2].parent_snapshot_id(), Some(2)); + + // Verify incremental scan from snapshot 1 to snapshot 3. + // Expected appends: snapshot 2 adds [1, 2, 3] + // Expected deletes: snapshot 3 deletes [2] + // In total we expect appends [1, 3] and deletes [] + fixture + .verify_incremental_scan(1, 3, vec![(1, "1"), (3, "3")], vec![]) + .await; + + // Verify incremental scan from snapshot 2 to snapshot 3. + let data_file_path = format!("{}/data/data-2.parquet", fixture.table_location); + fixture + .verify_incremental_scan(2, 3, vec![], vec![(1, &data_file_path)]) + .await; + + // Verify incremental scan from snapshot 1 to snapshot 1. + fixture.verify_incremental_scan(1, 1, vec![], vec![]).await; +} + +#[tokio::test] +async fn test_incremental_fixture_complex() { + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), // Snapshot 1: Empty + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + (4, "d".to_string()), + (5, "e".to_string()), + ], + "data-2.parquet".to_string(), + ), // Snapshot 2: Add 5 rows (positions 0-4) + Operation::Delete(vec![ + (1, "data-2.parquet".to_string()), + (3, "data-2.parquet".to_string()), + ]), // Snapshot 3: Delete positions 1,3 (n=2,4; data=b,d) + Operation::Add( + vec![(6, "f".to_string()), (7, "g".to_string())], + "data-4.parquet".to_string(), + ), // Snapshot 4: Add 2 more rows (positions 5-6) + Operation::Delete(vec![ + (0, "data-2.parquet".to_string()), + (2, "data-2.parquet".to_string()), + (4, "data-2.parquet".to_string()), + (0, "data-4.parquet".to_string()), + (1, "data-4.parquet".to_string()), + ]), // Snapshot 5: Delete positions 0,2,4,5,6 (all remaining rows: n=1,3,5,6,7) + ]) + .await; + + // Verify we have 5 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 5); + + // Verify parent chain + assert_eq!(snapshots[0].parent_snapshot_id(), None); + for (i, snapshot) in snapshots.iter().enumerate().take(5).skip(1) { + assert_eq!(snapshot.parent_snapshot_id(), Some(i as i64)); + } + + // Verify current snapshot + assert_eq!( + fixture + .table + .metadata() + .current_snapshot() + .unwrap() + .snapshot_id(), + 5 + ); + + // Verify incremental scan from snapshot 1 to snapshot 5. + // All data has been deleted, so we expect the empty result. + fixture.verify_incremental_scan(1, 5, vec![], vec![]).await; + + // Verify incremental scan from snapshot 2 to snapshot 5. + // Snapshot 2 starts with: (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // Snapshot 5: Deletes positions 0,2,4 from data-2.parquet and 0,1 from data-4.parquet (n=1,3,5,6,7; data=a,c,e,f,g) + // + // The incremental scan computes the NET EFFECT between snapshot 2 and 5: + // - Files added in snapshot 4 were completely deleted in snapshot 5, so NO net appends + // - Net deletes from data-2.parquet: positions 0,1,2,3,4 (all 5 rows deleted across snapshots 3 and 5) + // - Since data-4 was added and deleted between 2 and 5, it doesn't appear in the scan + let data_2_path = format!("{}/data/data-2.parquet", fixture.table_location); + fixture + .verify_incremental_scan( + 2, + 5, + vec![], // No net appends (data-4 was added and fully deleted) + vec![ + (0, data_2_path.as_str()), // All 5 positions from data-2.parquet + (1, data_2_path.as_str()), + (2, data_2_path.as_str()), + (3, data_2_path.as_str()), + (4, data_2_path.as_str()), + ], + ) + .await; + + // Verify incremental scan from snapshot 3 to snapshot 5. + // Snapshot 3 state: (1,a), (3,c), (5,e) remain in data-2.parquet at positions 0,2,4 + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // Snapshot 5: Deletes positions 0,2,4 from data-2.parquet (n=1,3,5) and 0,1 from data-4.parquet (n=6,7) + // + // Net effect from snapshot 3 to 5: + // - No net appends (data-4 was added and fully deleted between 3 and 5) + // - Net deletes from data-2.parquet: positions 0,2,4 (the three remaining rows deleted in snapshot 5) + fixture + .verify_incremental_scan( + 3, + 5, + vec![], // No net appends (data-4 was added and fully deleted) + vec![ + (0, data_2_path.as_str()), // Positions 0,2,4 from data-2.parquet + (2, data_2_path.as_str()), // (n=1,3,5; data=a,c,e) + (4, data_2_path.as_str()), + ], + ) + .await; + + // Verify incremental scan from snapshot 1 to snapshot 4. + // Snapshot 1: Empty + // Snapshot 2: Adds (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // + // Net effect from snapshot 1 to 4: + // - Net appends: (1,a), (3,c), (5,e), (6,f), (7,g) - all rows that exist at snapshot 4 + // - No deletes: rows deleted in snapshot 3 were added after snapshot 1, so they don't count as deletes + fixture + .verify_incremental_scan( + 1, + 4, + vec![(1, "a"), (3, "c"), (5, "e"), (6, "f"), (7, "g")], + vec![], // No deletes (deleted rows were added after snapshot 1) + ) + .await; + + // Verify incremental scan from snapshot 2 to snapshot 4. + // Snapshot 2: Has (1,a), (2,b), (3,c), (4,d), (5,e) in data-2.parquet + // Snapshot 3: Deletes positions 1,3 from data-2.parquet (n=2,4; data=b,d) + // Snapshot 4: Adds (6,f), (7,g) in data-4.parquet + // + // Net effect from snapshot 2 to 4: + // - Net appends: (6,f), (7,g) from data-4.parquet + // - Net deletes: positions 1,3 from data-2.parquet (n=2,4; data=b,d) - existed at snapshot 2 and deleted in 3 + fixture + .verify_incremental_scan(2, 4, vec![(6, "f"), (7, "g")], vec![ + (1, data_2_path.as_str()), // Positions 1,3 from data-2.parquet + (3, data_2_path.as_str()), // (n=2,4; data=b,d) + ]) + .await; +} + +#[tokio::test] +async fn test_incremental_scan_edge_cases() { + // This test covers several edge cases: + // 1. Multiple data files added in separate snapshots + // 2. Deletes spread across multiple data files + // 3. Partial deletes from multiple files + // 4. Cross-file delete operations in a single snapshot + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add file A with 3 rows + Operation::Add( + vec![ + (1, "a1".to_string()), + (2, "a2".to_string()), + (3, "a3".to_string()), + ], + "file-a.parquet".to_string(), + ), + // Snapshot 3: Add file B with 4 rows + Operation::Add( + vec![ + (10, "b1".to_string()), + (20, "b2".to_string()), + (30, "b3".to_string()), + (40, "b4".to_string()), + ], + "file-b.parquet".to_string(), + ), + // Snapshot 4: Partial delete from file A (delete middle row n=2) + Operation::Delete(vec![(1, "file-a.parquet".to_string())]), + // Snapshot 5: Partial delete from file B (delete first and last rows n=10,40) + Operation::Delete(vec![ + (0, "file-b.parquet".to_string()), + (3, "file-b.parquet".to_string()), + ]), + // Snapshot 6: Add file C with 2 rows + Operation::Add( + vec![(100, "c1".to_string()), (200, "c2".to_string())], + "file-c.parquet".to_string(), + ), + // Snapshot 7: Delete from multiple files in one snapshot (cross-file deletes) + Operation::Delete(vec![ + (0, "file-a.parquet".to_string()), // n=1 + (1, "file-b.parquet".to_string()), // n=20 + (0, "file-c.parquet".to_string()), // n=100 + ]), + ]) + .await; + + // Verify we have 7 snapshots + let n_snapshots = fixture.table.metadata().snapshots().count(); + assert_eq!(n_snapshots, 7); + + let file_a_path = format!("{}/data/file-a.parquet", fixture.table_location); + let file_b_path = format!("{}/data/file-b.parquet", fixture.table_location); + + // Test 1: Scan from snapshot 1 to 4 + // Should see: file-a (1,2,3), file-b (10,20,30,40) added, then (2) deleted from file-a + // BUT: The row n=2 was added AFTER snapshot 1, so it won't show as a delete! + // Net: appends (1,3) from file-a (n=2 added then deleted = net zero), (10,20,30,40) from file-b + // No deletes (n=2 was added and deleted between snapshots 1 and 4) + fixture + .verify_incremental_scan( + 1, + 4, + vec![ + (1, "a1"), + (3, "a3"), + (10, "b1"), + (20, "b2"), + (30, "b3"), + (40, "b4"), + ], + vec![], // No deletes - n=2 was added and deleted between snapshots + ) + .await; + + // Test 2: Scan from snapshot 4 to 6 + // Snapshot 4: has file-a (1,3) and file-b (10,20,30,40) + // Snapshot 5: deletes positions 0,3 from file-b (n=10,40) + // Snapshot 6: adds file-c (100,200) + // Net: appends (100,200) from file-c; deletes pos 0,3 from file-b + fixture + .verify_incremental_scan(4, 6, vec![(100, "c1"), (200, "c2")], vec![ + (0, file_b_path.as_str()), // n=10 + (3, file_b_path.as_str()), // n=40 + ]) + .await; + + // Test 3: Scan from snapshot 2 to 7 + // This tests the full lifecycle: multiple adds, partial deletes, more adds, cross-file deletes + // Starting at snapshot 2: file-a (1,2,3) exists + // File-b is added in snapshot 3 (after snapshot 2) + // File-c is added in snapshot 6 (after snapshot 2) + // By snapshot 7: file-a has (3) at position 2, file-b has (30), file-c has (200) + // + // Net appends: file-b (30) and file-c (200) were added after snapshot 2 + // Net deletes: positions 0,1 from file-a (n=1,2) existed at snapshot 2 and were deleted + // Note: (3) from file-a already existed at snapshot 2, so it's not a net append! + fixture + .verify_incremental_scan(2, 7, vec![(30, "b3"), (200, "c2")], vec![ + (0, file_a_path.as_str()), // n=1 + (1, file_a_path.as_str()), // n=2 + ]) + .await; + + // Test 4: Scan from snapshot 5 to 6 + // Simple test: just adding a new file + // Snapshot 5 state: file-a (1,3), file-b (20,30) + // Snapshot 6: adds file-c (100,200) + // Net: appends (100,200) from file-c, no deletes + fixture + .verify_incremental_scan(5, 6, vec![(100, "c1"), (200, "c2")], vec![]) + .await; + + // Test 5: Scan from snapshot 3 to 4 + // Tests a single delete operation + // State at 3: file-a (1,2,3), file-b (10,20,30,40) + // State at 4: file-a (1,3), file-b (10,20,30,40) + // Net: no appends, 1 delete (position 1, n=2) from file-a + fixture + .verify_incremental_scan( + 3, + 4, + vec![], + vec![(1, file_a_path.as_str())], // n=2 + ) + .await; +} + +#[tokio::test] +async fn test_incremental_scan_builder_options() { + // This test demonstrates using the incremental scan builder API with various options: + // - Column projection (selecting specific columns) + // - Batch size configuration + // - Verifying the schema and batch structure + let fixture = IncrementalTestFixture::new(vec![ + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add 10 rows to test batch size behavior + Operation::Add( + vec![ + (1, "data-1".to_string()), + (2, "data-2".to_string()), + (3, "data-3".to_string()), + (4, "data-4".to_string()), + (5, "data-5".to_string()), + (6, "data-6".to_string()), + (7, "data-7".to_string()), + (8, "data-8".to_string()), + (9, "data-9".to_string()), + (10, "data-10".to_string()), + ], + "data-2.parquet".to_string(), + ), + // Snapshot 3: Delete some rows + Operation::Delete(vec![ + (2, "data-2.parquet".to_string()), // n=3 + (5, "data-2.parquet".to_string()), // n=6 + (8, "data-2.parquet".to_string()), // n=9 + ]), + // Snapshot 4: Add more rows + Operation::Add( + vec![ + (20, "data-20".to_string()), + (21, "data-21".to_string()), + (22, "data-22".to_string()), + (23, "data-23".to_string()), + (24, "data-24".to_string()), + ], + "data-4.parquet".to_string(), + ), + ]) + .await; + + // Test 1: Column projection - select only the "n" column + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Verify we have both append and delete batches + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + assert!(!append_batches.is_empty(), "Should have append batches"); + + // Check schema - should only have "n" column + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 1, + "Should have only 1 column when projecting 'n'" + ); + assert_eq!( + batch.schema().field(0).name(), + "n", + "Projected column should be 'n'" + ); + } + + // Test 2: Column projection - select only the "data" column + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["data"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 1, + "Should have only 1 column when projecting 'data'" + ); + assert_eq!( + batch.schema().field(0).name(), + "data", + "Projected column should be 'data'" + ); + } + + // Test 3: Select both columns explicitly + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n", "data"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + for batch in &append_batches { + assert_eq!( + batch.schema().fields().len(), + 2, + "Should have 2 columns when projecting both" + ); + assert_eq!(batch.schema().field(0).name(), "n"); + assert_eq!(batch.schema().field(1).name(), "data"); + } + + // Test 4: Batch size configuration + let scan = fixture + .table + .incremental_scan(1, 2) + .with_batch_size(Some(3)) // Small batch size to test batching + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()); + + for batch in append_batches { + // Each batch should have at most 3 rows (except possibly the last) + assert!( + batch.num_rows() <= 3, + "Batch size should be <= 3 as configured" + ); + } + + // Test 5: Combining column projection and batch size + let scan = fixture + .table + .incremental_scan(1, 4) + .select(vec!["n"]) + .with_batch_size(Some(4)) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()); + + for batch in append_batches { + assert_eq!(batch.schema().fields().len(), 1, "Should project only 'n'"); + assert!(batch.num_rows() <= 4, "Batch size should be <= 4"); + } + + // Test 6: Verify actual data with column projection + let scan = fixture + .table + .incremental_scan(1, 2) + .select(vec!["n"]) + .build() + .unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + if !append_batches.is_empty() { + use arrow_select::concat::concat_batches; + let combined_batch = + concat_batches(&append_batches[0].schema(), append_batches.iter()).unwrap(); + + let n_array = combined_batch + .column(0) + .as_primitive::(); + + let mut n_values: Vec = (0..combined_batch.num_rows()) + .map(|i| n_array.value(i)) + .collect(); + n_values.sort(); + + assert_eq!( + n_values, + vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10], + "Should have all 10 n values from snapshot 2" + ); + } + + // Test 7: Delete batches always have the same schema. + let scan = fixture.table.incremental_scan(2, 3).build().unwrap(); + + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + let delete_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Delete) + .map(|(_, b)| b.clone()) + .collect(); + + if !delete_batches.is_empty() { + for batch in &delete_batches { + // Delete batches should have "pos" and "_file" columns + assert!( + batch.schema().fields().len() == 2, + "Delete batch should have exactly position and file columns" + ); + assert_eq!( + batch.num_rows(), + 3, + "Should have 3 deleted positions from snapshot 3" + ); + } + } +} + +#[tokio::test] +async fn test_incremental_scan_with_deleted_files_errors() { + // This test verifies that incremental scans properly error out when entire data files + // are deleted (overwrite operation), since this is not yet supported. + // + // Test scenario: + // Snapshot 1: Add file-1.parquet with data + // Snapshot 2: Add file-2.parquet with data + // Snapshot 3: Overwrite - delete file-1.parquet entirely + // Snapshot 4: Add file-3.parquet with data + // + // Incremental scan from snapshot 1 to snapshot 3 should error because file-1 + // was completely removed in the overwrite operation. + + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Add file-1 with rows + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + ], + "file-1.parquet".to_string(), + ), + // Snapshot 2: Add file-2 with rows + Operation::Add( + vec![(10, "x".to_string()), (20, "y".to_string())], + "file-2.parquet".to_string(), + ), + // Snapshot 3: Overwrite - delete file-1 entirely + Operation::Overwrite( + (vec![], "".to_string()), // No new data to add + vec![], // No positional deletes + vec!["file-1.parquet".to_string()], // Delete file-1 completely + ), + // Snapshot 4: Add file-3 (to have more snapshots) + Operation::Add(vec![(100, "p".to_string())], "file-3.parquet".to_string()), + ]) + .await; + + // Test 1: Incremental scan from snapshot 1 to 3 should error when building the stream + // because file-1 was deleted entirely in snapshot 3 + let scan = fixture + .table + .incremental_scan(1, 3) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + match stream_result { + Err(error) => { + assert_eq!( + error.message(), + "Processing deleted data files is not supported yet in incremental scans", + "Error message should indicate that deleted files are not supported. Got: {}", + error + ); + } + Ok(_) => panic!( + "Expected error when building stream over a snapshot that deletes entire data files" + ), + } + + // Test 2: Incremental scan from snapshot 2 to 4 should also error + // because it includes snapshot 3 which deletes a file + let scan = fixture + .table + .incremental_scan(2, 4) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + match stream_result { + Err(_) => { + // Expected error + } + Ok(_) => panic!("Expected error when scan range includes a snapshot that deletes files"), + } + + // Test 3: Incremental scan from snapshot 1 to 2 should work fine + // (no files deleted) + let scan = fixture + .table + .incremental_scan(1, 2) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + assert!( + stream_result.is_ok(), + "Scan should succeed when no files are deleted. Error: {:?}", + stream_result.err() + ); + + // Test 4: Incremental scan from snapshot 3 to 4 should work + // (starting from after the deletion) + let scan = fixture + .table + .incremental_scan(3, 4) + .build() + .expect("Building the scan should succeed"); + + let stream_result = scan.to_arrow().await; + + assert!( + stream_result.is_ok(), + "Scan should succeed when starting after the file deletion. Error: {:?}", + stream_result.err() + ); +} diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 6884e00b9b..5c4ea54323 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -21,6 +21,9 @@ mod cache; use cache::*; mod context; use context::*; + +pub mod incremental; + mod task; use std::sync::Arc; diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index 80e10b2fe2..61ed755027 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -24,6 +24,7 @@ use crate::inspect::MetadataTable; use crate::io::FileIO; use crate::io::object_cache::ObjectCache; use crate::scan::TableScanBuilder; +use crate::scan::incremental::IncrementalTableScanBuilder; use crate::spec::{SchemaRef, TableMetadata, TableMetadataRef}; use crate::{Error, ErrorKind, Result, TableIdent}; @@ -224,6 +225,15 @@ impl Table { TableScanBuilder::new(self) } + /// Creates an incremental table scan between two snapshots. + pub fn incremental_scan( + &self, + from_snapshot_id: i64, + to_snapshot_id: i64, + ) -> IncrementalTableScanBuilder<'_> { + IncrementalTableScanBuilder::new(self, from_snapshot_id, to_snapshot_id) + } + /// Creates a metadata table which provides table-like APIs for inspecting metadata. /// See [`MetadataTable`] for more details. pub fn inspect(&self) -> MetadataTable<'_> { diff --git a/crates/iceberg/src/util/mod.rs b/crates/iceberg/src/util/mod.rs new file mode 100644 index 0000000000..b614c981ec --- /dev/null +++ b/crates/iceberg/src/util/mod.rs @@ -0,0 +1,19 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +/// Utilities for working with snapshots. +pub mod snapshot; diff --git a/crates/iceberg/src/util/snapshot.rs b/crates/iceberg/src/util/snapshot.rs new file mode 100644 index 0000000000..513e23c110 --- /dev/null +++ b/crates/iceberg/src/util/snapshot.rs @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Taken from https://github.com/apache/iceberg-rust/pull/1470 + +use crate::spec::{SnapshotRef, TableMetadataRef}; + +struct Ancestors { + next: Option, + get_snapshot: Box Option + Send>, +} + +impl Iterator for Ancestors { + type Item = SnapshotRef; + + fn next(&mut self) -> Option { + let snapshot = self.next.take()?; + let result = snapshot.clone(); + self.next = snapshot + .parent_snapshot_id() + .and_then(|id| (self.get_snapshot)(id)); + Some(result) + } +} + +/// Iterate starting from `snapshot` (inclusive) to the root snapshot. +pub fn ancestors_of( + table_metadata: &TableMetadataRef, + snapshot: i64, +) -> Box + Send> { + if let Some(snapshot) = table_metadata.snapshot_by_id(snapshot) { + let table_metadata = table_metadata.clone(); + Box::new(Ancestors { + next: Some(snapshot.clone()), + get_snapshot: Box::new(move |id| table_metadata.snapshot_by_id(id).cloned()), + }) + } else { + Box::new(std::iter::empty()) + } +} + +/// Iterate starting from `snapshot` (inclusive) to `oldest_snapshot_id` (exclusive). +pub fn ancestors_between( + table_metadata: &TableMetadataRef, + latest_snapshot_id: i64, + oldest_snapshot_id: Option, +) -> Box + Send> { + let Some(oldest_snapshot_id) = oldest_snapshot_id else { + return Box::new(ancestors_of(table_metadata, latest_snapshot_id)); + }; + + if latest_snapshot_id == oldest_snapshot_id { + return Box::new(std::iter::empty()); + } + + Box::new( + ancestors_of(table_metadata, latest_snapshot_id) + .take_while(move |snapshot| snapshot.snapshot_id() != oldest_snapshot_id), + ) +} From 25045a4669483ab80bc693bcf723970835e2a950 Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Mon, 3 Nov 2025 09:46:25 +0100 Subject: [PATCH 3/7] Merge upstream (#6) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * fix(reader): fix position delete bugs with row group skipping (#1806) ## Which issue does this PR close? Partially address #1749. ## What changes are included in this PR? This PR fixes two related correctness bugs in `ArrowReader::build_deletes_row_selection()` where position deletes targeting rows in skipped or skipped-to row groups were not being applied correctly. ### Background: How These Bugs Were Discovered While running Apache Spark + Apache Iceberg integration tests through DataFusion Comet, we discovered that the following tests were failing or hanging: - org.apache.iceberg.spark.extensions.TestMergeOnReadMerge - org.apache.iceberg.spark.extensions.TestMergeOnReadDelete - org.apache.iceberg.spark.extensions.TestMergeOnReadUpdate Investigation revealed that recent work to support Iceberg's file splitting feature (via `filter_row_groups_by_byte_range()`) exposed latent bugs in the position delete logic. While the byte range filtering code itself is correct, it exercises code paths that were previously untested, revealing these pre-existing issues. #### Bug 1: Missing base index increment when skipping row groups **The Issue:** When processing a Parquet file with multiple row groups, if a position delete targets a row in a later row group, the function would skip row groups without deletes but fail to increment `current_row_group_base_idx`. This caused the row index tracking to become desynchronized. **Example scenario:** - File with 2 row groups: rows 0-99 (group 0) and rows 100-199 (group 1) - Position delete targets row 199 (last row in group 1) - When processing group 0: delete (199) is beyond the group's range, so code hits `continue` at lines 469-471 - BUG: `current_row_group_base_idx` is NOT incremented, stays at 0 - When processing group 1: code thinks rows start at 0 instead of 100 - Delete at position 199 is never applied (thinks file only has rows 0-99) **The Fix:** Add `current_row_group_base_idx += row_group_num_rows` before the two `continue` statements at lines ~470 and ~481. This ensures row index tracking stays synchronized when skipping row groups. #### Bug 2: Stale cached delete index when skipping unselected row groups **The Issue:** When row group selection is active (e.g., via byte range filtering for file splits) and an unselected row group is skipped, the cached `next_deleted_row_idx_opt` variable can become stale, leading to either lost deletes or infinite loops depending on the scenario. The function maintains a cached value (`next_deleted_row_idx_opt`) containing the next delete to apply. When skipping unselected row groups, it calls `delete_vector_iter.advance_to(next_row_group_base_idx)` to position the iterator, but this doesn't automatically update the cached variable. **Two problematic scenarios:** 1. Stale cache causes infinite loop (the bug we hit): - File with 2 row groups: rows 0-99 (group 0) and rows 100-199 (group 1) - Position delete at row 0 (in group 0) - Row group selection: read ONLY group 1 - Initial state: `next_deleted_row_idx_opt = Some(0)` (cached) - Skip group 0: `advance_to(100)` positions iterator past delete at 0 - BUG: cached value still `Some(0)` - STALE! - Process group 1: loop condition `0 < 200` is `true`, but `current_idx (100) != next_deleted_row_idx (0)`, so neither branch executes could result in infinite loop 2. Unconditionally calling `next()` loses deletes: - File with 2 row groups: rows 0-99 (group 0) and rows 100-199 (group 1) - Position delete at row 199 (in group 1) - Row group selection: read ONLY group 1 - Initial state: `next_deleted_row_idx_opt = Some(199)` (cached, already correct!) - Skip group 0: `advance_to(100)` - iterator already positioned correctly - If we call `next()`: BUG - consumes delete at 199, advancing past it - Process group 1: iterator exhausted, delete is lost **The Fix:** - If `cached value < next_row_group_base_idx` (stale), update it, thus avoiding infinite loop - If `cached value >= next_row_group_base_idx` (still valid), keep it, thus preserving delete ## Are these changes tested? Yes. This PR adds two comprehensive unit tests in reader.rs: 1. `test_position_delete_across_multiple_row_groups` - Tests bug 1 (missing base index increment) 2. `test_position_delete_with_row_group_selection` - Tests bug 2 scenario where delete is in selected group 3. `test_position_delete_in_skipped_row_group` - Tests bug 2 scenario where delete is in skipped group (would hang without fix) Additionally, these fixes resolve failures in Iceberg Java's spark-extension tests when running with DataFusion Comet’s PR https://github.com/apache/datafusion-comet/pull/2528: - org.apache.iceberg.spark.extensions.TestMergeOnReadMerge - org.apache.iceberg.spark.extensions.TestMergeOnReadDelete - org.apache.iceberg.spark.extensions.TestMergeOnReadUpdate * feat(datafusion): implement the partitioning node for DataFusion to define the partitioning (#1620) ## Which issue does this PR close? - Closes #1543 ## What changes are included in this PR? Implement a physical execution repartition node that determines the relevant DataFusion partitioning strategy based on the Iceberg table schema and metadata. 1. Unpartitioned tables: Uses round-robin partitioning 2. Partitioned tables: It depends on the transform type: - Identity or Bucket transforms: Uses hash partitioning on the _partition column - Temporal transforms (Year, Month, Day, Hour): Uses round-robin partitioning _Minor change: I created a new `schema_ref()` helper method._ ## Are these changes tested? Yes, with unit tests --------- Signed-off-by: Florian Valeye * feat(reader): Date32 from days since epoch for Literal:try_from_json (#1803) * chore(deps): Bump aws-sdk-glue from 1.125.0 to 1.126.0 (#1812) Bumps [aws-sdk-glue](https://github.com/awslabs/aws-sdk-rust) from 1.125.0 to 1.126.0.
Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=aws-sdk-glue&package-manager=cargo&previous-version=1.125.0&new-version=1.126.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> * chore(deps): Bump astral-sh/setup-uv from 6 to 7 (#1811) Bumps [astral-sh/setup-uv](https://github.com/astral-sh/setup-uv) from 6 to 7.
Release notes

Sourced from astral-sh/setup-uv's releases.

v7.0.0 🌈 node24 and a lot of bugfixes

Changes

This release comes with a load of bug fixes and a speed up. Because of switching from node20 to node24 it is also a breaking change. If you are running on GitHub hosted runners this will just work, if you are using self-hosted runners make sure, that your runners are up to date. If you followed the normal installation instructions your self-hosted runner will keep itself updated.

This release also removes the deprecated input server-url which was used to download uv releases from a different server. The manifest-file input supersedes that functionality by adding a flexible way to define available versions and where they should be downloaded from.

Fixes

  • The action now respects when the environment variable UV_CACHE_DIR is already set and does not overwrite it. It now also finds cache-dir settings in config files if you set them.
  • Some users encountered problems that cache pruning took forever because they had some uv processes running in the background. Starting with uv version 0.8.24 this action uses uv cache prune --ci --force to ignore the running processes
  • If you just want to install uv but not have it available in path, this action now respects UV_NO_MODIFY_PATH
  • Some other actions also set the env var UV_CACHE_DIR. This action can now deal with that but as this could lead to unwanted behavior in some edgecases a warning is now displayed.

Improvements

If you are using minimum version specifiers for the version of uv to install for example

[tool.uv]
required-version = ">=0.8.17"

This action now detects that and directly uses the latest version. Previously it would download all available releases from the uv repo to determine the highest matching candidate for the version specifier, which took much more time.

If you are using other specifiers like 0.8.x this action still needs to download all available releases because the specifier defines an upper bound (not 0.9.0 or later) and "latest" would possibly not satisfy that.

🚨 Breaking changes

🐛 Bug fixes

🚀 Enhancements

🧰 Maintenance

... (truncated)

Commits
  • 8585678 Bump dependencies (#664)
  • 22d500a Bump github/codeql-action from 4.30.8 to 4.30.9 (#652)
  • 14d5571 chore: update known checksums for 0.9.5 (#663)
  • 29cd235 Use tar for extracting the uv zip file on Windows too (#660)
  • 2ddd2b9 chore: update known checksums for 0.9.4 (#651)
  • b7bf789 Fix "lowest" resolution strategy with lower-bound only (#649)
  • cb6c0a5 Change version in docs to v7 (#647)
  • dffc629 Use working-directory to detect empty workdir (#645)
  • 6e346e1 chore: update known checksums for 0.9.3 (#644)
  • 3ccd0fd Bump github/codeql-action from 4.30.7 to 4.30.8 (#639)
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=astral-sh/setup-uv&package-manager=github_actions&previous-version=6&new-version=7)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu * chore(deps): Bump crate-ci/typos from 1.38.1 to 1.39.0 (#1810) --------- Signed-off-by: Florian Valeye Signed-off-by: dependabot[bot] Co-authored-by: Matt Butrovich Co-authored-by: Florian Valeye Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu --- .github/workflows/bindings_python_ci.yml | 4 +- .github/workflows/ci_typos.yml | 2 +- Cargo.lock | 18 +- crates/iceberg/src/arrow/reader.rs | 617 +++++++++++- crates/iceberg/src/spec/values.rs | 51 + crates/iceberg/src/table.rs | 7 +- .../datafusion/src/physical_plan/mod.rs | 1 + .../src/physical_plan/repartition.rs | 886 ++++++++++++++++++ 8 files changed, 1568 insertions(+), 18 deletions(-) create mode 100644 crates/integrations/datafusion/src/physical_plan/repartition.rs diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index 55e7e5bbd0..e9eabda2cb 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -56,7 +56,7 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v5 - - uses: astral-sh/setup-uv@v6 + - uses: astral-sh/setup-uv@v7 with: version: "0.9.3" enable-cache: true @@ -90,7 +90,7 @@ jobs: working-directory: "bindings/python" command: build args: --out dist --sdist - - uses: astral-sh/setup-uv@v6 + - uses: astral-sh/setup-uv@v7 with: version: "0.9.3" enable-cache: true diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index 11564caa13..693199eda6 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v5 - name: Check typos - uses: crate-ci/typos@v1.38.1 + uses: crate-ci/typos@v1.39.0 diff --git a/Cargo.lock b/Cargo.lock index 1cacc75fb9..38d284e52d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -710,9 +710,9 @@ dependencies = [ [[package]] name = "aws-sdk-glue" -version = "1.125.0" +version = "1.126.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df436173c8acd8c207ecff1aedfd0992d44d0d6d1e84be73e30dab7746a20363" +checksum = "fd9c10a11584c0b619c9e478143072c4028c39017f98534e206156a7e94188be" dependencies = [ "aws-credential-types", "aws-runtime", @@ -2642,7 +2642,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -2784,7 +2784,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -4523,7 +4523,7 @@ version = "0.50.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -5421,7 +5421,7 @@ dependencies = [ "once_cell", "socket2 0.6.1", "tracing", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] @@ -5878,7 +5878,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -6950,7 +6950,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -7776,7 +7776,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.48.0", ] [[package]] diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 745e63a143..c4a1b8a5f3 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -432,10 +432,17 @@ impl ArrowReader { // selected row group selected_row_groups_idx += 1; } else { - // remove any positional deletes from the skipped page so that - // `positional.deletes.min()` can be used + // Advance iterator past all deletes in the skipped row group. + // advance_to() positions the iterator to the first delete >= next_row_group_base_idx. + // However, if our cached next_deleted_row_idx_opt is in the skipped range, + // we need to call next() to update the cache with the newly positioned value. delete_vector_iter.advance_to(next_row_group_base_idx); - next_deleted_row_idx_opt = delete_vector_iter.next(); + // Only update the cache if the cached value is stale (in the skipped range) + if let Some(cached_idx) = next_deleted_row_idx_opt { + if cached_idx < next_row_group_base_idx { + next_deleted_row_idx_opt = delete_vector_iter.next(); + } + } // still increment the current page base index but then skip to the next row group // in the file @@ -450,6 +457,7 @@ impl ArrowReader { // the remainder of this row group and skip to the next row group if next_deleted_row_idx >= next_row_group_base_idx { results.push(RowSelector::select(row_group_num_rows as usize)); + current_row_group_base_idx += row_group_num_rows; continue; } @@ -459,6 +467,7 @@ impl ArrowReader { // If there are no more pos deletes, add a selector for the entirety of this row group. _ => { results.push(RowSelector::select(row_group_num_rows as usize)); + current_row_group_base_idx += row_group_num_rows; continue; } }; @@ -1622,8 +1631,10 @@ mod tests { use crate::expr::visitors::bound_predicate_visitor::visit; use crate::expr::{Bind, Predicate, Reference}; use crate::io::FileIO; - use crate::scan::{FileScanTask, FileScanTaskStream}; - use crate::spec::{DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, Type}; + use crate::scan::{FileScanTask, FileScanTaskDeleteFile, FileScanTaskStream}; + use crate::spec::{ + DataContentType, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, Type, + }; fn table_schema_simple() -> SchemaRef { Arc::new( @@ -2721,4 +2732,600 @@ message schema { let file_path_column = new_batch.column(1); assert_eq!(file_path_column.len(), 0); } + + /// Test for bug where position deletes in later row groups are not applied correctly. + /// + /// When a file has multiple row groups and a position delete targets a row in a later + /// row group, the `build_deletes_row_selection` function had a bug where it would + /// fail to increment `current_row_group_base_idx` when skipping row groups. + /// + /// This test creates: + /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) + /// - A position delete file that deletes row 199 (last row in second row group) + /// + /// Expected behavior: Should return 199 rows (with id=200 deleted) + /// Bug behavior: Returns 200 rows (delete is not applied) + /// + /// This bug was discovered while running Apache Spark + Apache Iceberg integration tests + /// through DataFusion Comet. The following Iceberg Java tests failed due to this bug: + /// - `org.apache.iceberg.spark.extensions.TestMergeOnReadDelete::testDeleteWithMultipleRowGroupsParquet` + /// - `org.apache.iceberg.spark.extensions.TestMergeOnReadUpdate::testUpdateWithMultipleRowGroupsParquet` + #[tokio::test] + async fn test_position_delete_across_multiple_row_groups() { + use arrow_array::{Int32Array, Int64Array}; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + // Field IDs for positional delete schema + const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; + const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + + // Create table schema with a single 'id' column + let table_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Step 1: Create data file with 200 rows in 2 row groups + // Row group 0: rows 0-99 (ids 1-100) + // Row group 1: rows 100-199 (ids 101-200) + let data_file_path = format!("{}/data.parquet", &table_location); + + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(1..=100), + )]) + .unwrap(); + + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(101..=200), + )]) + .unwrap(); + + // Force each batch into its own row group + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&data_file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.close().unwrap(); + + // Verify we created 2 row groups + let verify_file = File::open(&data_file_path).unwrap(); + let verify_reader = SerializedFileReader::new(verify_file).unwrap(); + assert_eq!( + verify_reader.metadata().num_row_groups(), + 2, + "Should have 2 row groups" + ); + + // Step 2: Create position delete file that deletes row 199 (id=200, last row in row group 1) + let delete_file_path = format!("{}/deletes.parquet", &table_location); + + let delete_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), + )])), + Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_POS.to_string(), + )])), + ])); + + // Delete row at position 199 (0-indexed, so it's the last row: id=200) + let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ + Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), + Arc::new(Int64Array::from_iter_values(vec![199i64])), + ]) + .unwrap(); + + let delete_props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let delete_file = File::create(&delete_file_path).unwrap(); + let mut delete_writer = + ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); + delete_writer.write(&delete_batch).unwrap(); + delete_writer.close().unwrap(); + + // Step 3: Read the data file with the delete applied + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + let task = FileScanTask { + start: 0, + length: 0, + record_count: Some(200), + data_file_path: data_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![FileScanTaskDeleteFile { + file_path: delete_file_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }], + }; + + let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Step 4: Verify we got 199 rows (not 200) + let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); + + println!("Total rows read: {}", total_rows); + println!("Expected: 199 rows (deleted row 199 which had id=200)"); + + // This assertion will FAIL before the fix and PASS after the fix + assert_eq!( + total_rows, 199, + "Expected 199 rows after deleting row 199, but got {} rows. \ + The bug causes position deletes in later row groups to be ignored.", + total_rows + ); + + // Verify the deleted row (id=200) is not present + let all_ids: Vec = result + .iter() + .flat_map(|batch| { + batch + .column(0) + .as_primitive::() + .values() + .iter() + .copied() + }) + .collect(); + + assert!( + !all_ids.contains(&200), + "Row with id=200 should be deleted but was found in results" + ); + + // Verify we have all other ids (1-199) + let expected_ids: Vec = (1..=199).collect(); + assert_eq!( + all_ids, expected_ids, + "Should have ids 1-199 but got different values" + ); + } + + /// Test for bug where position deletes are lost when skipping unselected row groups. + /// + /// This is a variant of `test_position_delete_across_multiple_row_groups` that exercises + /// the row group selection code path (`selected_row_groups: Some([...])`). + /// + /// When a file has multiple row groups and only some are selected for reading, + /// the `build_deletes_row_selection` function must correctly skip over deletes in + /// unselected row groups WITHOUT consuming deletes that belong to selected row groups. + /// + /// This test creates: + /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) + /// - A position delete file that deletes row 199 (last row in second row group) + /// - Row group selection that reads ONLY row group 1 (rows 100-199) + /// + /// Expected behavior: Should return 99 rows (with row 199 deleted) + /// Bug behavior: Returns 100 rows (delete is lost when skipping row group 0) + /// + /// The bug occurs when processing row group 0 (unselected): + /// ```rust + /// delete_vector_iter.advance_to(next_row_group_base_idx); // Position at first delete >= 100 + /// next_deleted_row_idx_opt = delete_vector_iter.next(); // BUG: Consumes delete at 199! + /// ``` + /// + /// The fix is to NOT call `next()` after `advance_to()` when skipping unselected row groups, + /// because `advance_to()` already positions the iterator correctly without consuming elements. + #[tokio::test] + async fn test_position_delete_with_row_group_selection() { + use arrow_array::{Int32Array, Int64Array}; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + // Field IDs for positional delete schema + const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; + const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + + // Create table schema with a single 'id' column + let table_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Step 1: Create data file with 200 rows in 2 row groups + // Row group 0: rows 0-99 (ids 1-100) + // Row group 1: rows 100-199 (ids 101-200) + let data_file_path = format!("{}/data.parquet", &table_location); + + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(1..=100), + )]) + .unwrap(); + + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(101..=200), + )]) + .unwrap(); + + // Force each batch into its own row group + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&data_file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.close().unwrap(); + + // Verify we created 2 row groups + let verify_file = File::open(&data_file_path).unwrap(); + let verify_reader = SerializedFileReader::new(verify_file).unwrap(); + assert_eq!( + verify_reader.metadata().num_row_groups(), + 2, + "Should have 2 row groups" + ); + + // Step 2: Create position delete file that deletes row 199 (id=200, last row in row group 1) + let delete_file_path = format!("{}/deletes.parquet", &table_location); + + let delete_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), + )])), + Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_POS.to_string(), + )])), + ])); + + // Delete row at position 199 (0-indexed, so it's the last row: id=200) + let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ + Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), + Arc::new(Int64Array::from_iter_values(vec![199i64])), + ]) + .unwrap(); + + let delete_props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let delete_file = File::create(&delete_file_path).unwrap(); + let mut delete_writer = + ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); + delete_writer.write(&delete_batch).unwrap(); + delete_writer.close().unwrap(); + + // Step 3: Get byte ranges to read ONLY row group 1 (rows 100-199) + // This exercises the row group selection code path where row group 0 is skipped + let metadata_file = File::open(&data_file_path).unwrap(); + let metadata_reader = SerializedFileReader::new(metadata_file).unwrap(); + let metadata = metadata_reader.metadata(); + + let row_group_0 = metadata.row_group(0); + let row_group_1 = metadata.row_group(1); + + let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" + let rg1_start = rg0_start + row_group_0.compressed_size() as u64; + let rg1_length = row_group_1.compressed_size() as u64; + + println!( + "Row group 0: starts at byte {}, {} bytes compressed", + rg0_start, + row_group_0.compressed_size() + ); + println!( + "Row group 1: starts at byte {}, {} bytes compressed", + rg1_start, + row_group_1.compressed_size() + ); + + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + // Create FileScanTask that reads ONLY row group 1 via byte range filtering + let task = FileScanTask { + start: rg1_start, + length: rg1_length, + record_count: Some(100), // Row group 1 has 100 rows + data_file_path: data_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![FileScanTaskDeleteFile { + file_path: delete_file_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }], + }; + + let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Step 4: Verify we got 99 rows (not 100) + // Row group 1 has 100 rows (ids 101-200), minus 1 delete (id=200) = 99 rows + let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); + + println!("Total rows read from row group 1: {}", total_rows); + println!("Expected: 99 rows (row group 1 has 100 rows, 1 delete at position 199)"); + + // This assertion will FAIL before the fix and PASS after the fix + assert_eq!( + total_rows, 99, + "Expected 99 rows from row group 1 after deleting position 199, but got {} rows. \ + The bug causes position deletes to be lost when advance_to() is followed by next() \ + when skipping unselected row groups.", + total_rows + ); + + // Verify the deleted row (id=200) is not present + let all_ids: Vec = result + .iter() + .flat_map(|batch| { + batch + .column(0) + .as_primitive::() + .values() + .iter() + .copied() + }) + .collect(); + + assert!( + !all_ids.contains(&200), + "Row with id=200 should be deleted but was found in results" + ); + + // Verify we have ids 101-199 (not 101-200) + let expected_ids: Vec = (101..=199).collect(); + assert_eq!( + all_ids, expected_ids, + "Should have ids 101-199 but got different values" + ); + } + /// Test for bug where stale cached delete causes infinite loop when skipping row groups. + /// + /// This test exposes the inverse scenario of `test_position_delete_with_row_group_selection`: + /// - Position delete targets a row in the SKIPPED row group (not the selected one) + /// - After calling advance_to(), the cached delete index is stale + /// - Without updating the cache, the code enters an infinite loop + /// + /// This test creates: + /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) + /// - A position delete file that deletes row 0 (first row in SKIPPED row group 0) + /// - Row group selection that reads ONLY row group 1 (rows 100-199) + /// + /// The bug occurs when skipping row group 0: + /// ```rust + /// let mut next_deleted_row_idx_opt = delete_vector_iter.next(); // Some(0) + /// // ... skip to row group 1 ... + /// delete_vector_iter.advance_to(100); // Iterator advances past delete at 0 + /// // BUG: next_deleted_row_idx_opt is still Some(0) - STALE! + /// // When processing row group 1: + /// // current_idx = 100, next_deleted_row_idx = 0, next_row_group_base_idx = 200 + /// // Loop condition: 0 < 200 (true) + /// // But: current_idx (100) > next_deleted_row_idx (0) + /// // And: current_idx (100) != next_deleted_row_idx (0) + /// // Neither branch executes -> INFINITE LOOP! + /// ``` + /// + /// Expected behavior: Should return 100 rows (delete at 0 doesn't affect row group 1) + /// Bug behavior: Infinite loop in build_deletes_row_selection + #[tokio::test] + async fn test_position_delete_in_skipped_row_group() { + use arrow_array::{Int32Array, Int64Array}; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + // Field IDs for positional delete schema + const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; + const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + + // Create table schema with a single 'id' column + let table_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Step 1: Create data file with 200 rows in 2 row groups + // Row group 0: rows 0-99 (ids 1-100) + // Row group 1: rows 100-199 (ids 101-200) + let data_file_path = format!("{}/data.parquet", &table_location); + + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(1..=100), + )]) + .unwrap(); + + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(101..=200), + )]) + .unwrap(); + + // Force each batch into its own row group + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&data_file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.close().unwrap(); + + // Verify we created 2 row groups + let verify_file = File::open(&data_file_path).unwrap(); + let verify_reader = SerializedFileReader::new(verify_file).unwrap(); + assert_eq!( + verify_reader.metadata().num_row_groups(), + 2, + "Should have 2 row groups" + ); + + // Step 2: Create position delete file that deletes row 0 (id=1, first row in row group 0) + let delete_file_path = format!("{}/deletes.parquet", &table_location); + + let delete_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), + )])), + Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_POS.to_string(), + )])), + ])); + + // Delete row at position 0 (0-indexed, so it's the first row: id=1) + let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ + Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), + Arc::new(Int64Array::from_iter_values(vec![0i64])), + ]) + .unwrap(); + + let delete_props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let delete_file = File::create(&delete_file_path).unwrap(); + let mut delete_writer = + ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); + delete_writer.write(&delete_batch).unwrap(); + delete_writer.close().unwrap(); + + // Step 3: Get byte ranges to read ONLY row group 1 (rows 100-199) + // This exercises the row group selection code path where row group 0 is skipped + let metadata_file = File::open(&data_file_path).unwrap(); + let metadata_reader = SerializedFileReader::new(metadata_file).unwrap(); + let metadata = metadata_reader.metadata(); + + let row_group_0 = metadata.row_group(0); + let row_group_1 = metadata.row_group(1); + + let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" + let rg1_start = rg0_start + row_group_0.compressed_size() as u64; + let rg1_length = row_group_1.compressed_size() as u64; + + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + // Create FileScanTask that reads ONLY row group 1 via byte range filtering + let task = FileScanTask { + start: rg1_start, + length: rg1_length, + record_count: Some(100), // Row group 1 has 100 rows + data_file_path: data_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![FileScanTaskDeleteFile { + file_path: delete_file_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }], + }; + + let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Step 4: Verify we got 100 rows (all of row group 1) + // The delete at position 0 is in row group 0, which is skipped, so it doesn't affect us + let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); + + assert_eq!( + total_rows, 100, + "Expected 100 rows from row group 1 (delete at position 0 is in skipped row group 0). \ + If this hangs or fails, it indicates the cached delete index was not updated after advance_to()." + ); + + // Verify we have all ids from row group 1 (101-200) + let all_ids: Vec = result + .iter() + .flat_map(|batch| { + batch + .column(0) + .as_primitive::() + .values() + .iter() + .copied() + }) + .collect(); + + let expected_ids: Vec = (101..=200).collect(); + assert_eq!( + all_ids, expected_ids, + "Should have ids 101-200 (all of row group 1)" + ); + } } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index d06e754dcc..4c763a3941 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -1888,6 +1888,17 @@ impl Literal { date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), )))) } + (PrimitiveType::Date, JsonValue::Number(number)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( + number + .as_i64() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to date (days since epoch)", + ))? + .try_into()?, + )))) + } (PrimitiveType::Time, JsonValue::String(s)) => { Ok(Some(Literal::Primitive(PrimitiveLiteral::Long( time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), @@ -3942,4 +3953,44 @@ mod tests { assert_eq!(double_sorted, double_expected); } + + /// Test Date deserialization from JSON as number (days since epoch). + /// + /// This reproduces the scenario from Iceberg Java's TestAddFilesProcedure where: + /// - Date partition columns have initial_default values in manifests + /// - These values are serialized as days since epoch (e.g., 18628 for 2021-01-01) + /// - The JSON schema includes: {"type":"date","initial-default":18628} + /// + /// Prior to this fix, Date values in JSON were only parsed from String format ("2021-01-01"), + /// causing initial_default values to be lost during schema deserialization. + /// + /// This test ensures both formats are supported: + /// - String format: "2021-01-01" (used in table metadata) + /// - Number format: 18628 (used in initial-default values from add_files) + /// + /// See: Iceberg Java TestAddFilesProcedure.addDataPartitionedByDateToPartitioned() + #[test] + fn test_date_from_json_as_number() { + use serde_json::json; + + // Test Date as number (days since epoch) - used in initial-default from add_files + let date_number = json!(18628); // 2021-01-01 is 18628 days since 1970-01-01 + let result = + Literal::try_from_json(date_number, &Type::Primitive(PrimitiveType::Date)).unwrap(); + assert_eq!( + result, + Some(Literal::Primitive(PrimitiveLiteral::Int(18628))) + ); + + // Test Date as string - traditional format + let date_string = json!("2021-01-01"); + let result = + Literal::try_from_json(date_string, &Type::Primitive(PrimitiveType::Date)).unwrap(); + assert_eq!( + result, + Some(Literal::Primitive(PrimitiveLiteral::Int(18628))) + ); + + // Both formats should produce the same Literal value + } } diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index 1c8b43ca04..61ed755027 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -25,7 +25,7 @@ use crate::io::FileIO; use crate::io::object_cache::ObjectCache; use crate::scan::TableScanBuilder; use crate::scan::incremental::IncrementalTableScanBuilder; -use crate::spec::{TableMetadata, TableMetadataRef}; +use crate::spec::{SchemaRef, TableMetadata, TableMetadataRef}; use crate::{Error, ErrorKind, Result, TableIdent}; /// Builder to create table scan. @@ -245,6 +245,11 @@ impl Table { self.readonly } + /// Returns the current schema as a shared reference. + pub fn current_schema_ref(&self) -> SchemaRef { + self.metadata.current_schema().clone() + } + /// Create a reader for the table. pub fn reader_builder(&self) -> ArrowReaderBuilder { ArrowReaderBuilder::new(self.file_io.clone()) diff --git a/crates/integrations/datafusion/src/physical_plan/mod.rs b/crates/integrations/datafusion/src/physical_plan/mod.rs index ce923b8662..eb58082fe5 100644 --- a/crates/integrations/datafusion/src/physical_plan/mod.rs +++ b/crates/integrations/datafusion/src/physical_plan/mod.rs @@ -19,6 +19,7 @@ pub(crate) mod commit; pub(crate) mod expr_to_predicate; pub(crate) mod metadata_scan; pub(crate) mod project; +pub(crate) mod repartition; pub(crate) mod scan; pub(crate) mod write; diff --git a/crates/integrations/datafusion/src/physical_plan/repartition.rs b/crates/integrations/datafusion/src/physical_plan/repartition.rs new file mode 100644 index 0000000000..95cdc8472e --- /dev/null +++ b/crates/integrations/datafusion/src/physical_plan/repartition.rs @@ -0,0 +1,886 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::num::NonZeroUsize; +use std::sync::Arc; + +use datafusion::error::{DataFusionError, Result as DFResult}; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::expressions::Column; +use datafusion::physical_plan::repartition::RepartitionExec; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use iceberg::arrow::PROJECTED_PARTITION_VALUE_COLUMN; +use iceberg::spec::{TableMetadata, TableMetadataRef, Transform}; +/// Creates an Iceberg-aware repartition execution plan that optimizes data distribution +/// for parallel processing while respecting Iceberg table partitioning semantics. +/// +/// Automatically determines the optimal partitioning strategy based on the table's +/// partition specification. +/// +/// ## Partitioning Strategies +/// +/// - **Partitioned tables with Identity/Bucket transforms** – Uses hash partitioning on the +/// `_partition` column for optimal data distribution and file clustering. Ensures that rows +/// with the same partition values are co-located in the same task. +/// +/// - **Partitioned tables with temporal transforms** – Uses round-robin partitioning for +/// temporal transforms (Year, Month, Day, Hour) that don't provide uniform hash distribution. +/// +/// - **Unpartitioned tables** – Uses round-robin distribution to balance load evenly across workers. +/// +/// ## Requirements +/// +/// - **For partitioned tables**: The input MUST include the `_partition` column. +/// Add it by calling [`project_with_partition`](crate::physical_plan::project_with_partition) before [`repartition`]. +/// - **For unpartitioned tables**: No special preparation needed. +/// - Returns an error if a partitioned table is missing the `_partition` column. +/// +/// ## Performance Notes +/// +/// - Only adds repartitioning when the input partitioning differs from the target. +/// - Requires an explicit target partition count for deterministic behavior. +/// +/// # Arguments +/// +/// * `input` - The input [`ExecutionPlan`]. For partitioned tables, must include the `_partition` +/// column (added via [`project_with_partition`](crate::physical_plan::project_with_partition)). +/// * `table_metadata` - Iceberg table metadata containing partition spec. +/// * `target_partitions` - Target number of partitions for parallel processing (must be > 0). +/// +/// # Returns +/// +/// An [`ExecutionPlan`] that applies the optimal partitioning strategy, or the original input plan +/// if repartitioning is not needed. +/// +/// # Errors +/// +/// Returns [`DataFusionError::Plan`] if a partitioned table input is missing the `_partition` column. +/// +/// # Examples +/// +/// For partitioned tables, first add the `_partition` column: +/// +/// ```ignore +/// use std::num::NonZeroUsize; +/// use iceberg_datafusion::physical_plan::project_with_partition; +/// +/// let plan_with_partition = project_with_partition(input_plan, &table)?; +/// +/// let repartitioned_plan = repartition( +/// plan_with_partition, +/// table.metadata_ref(), +/// NonZeroUsize::new(4).unwrap(), +/// )?; +/// ``` +#[allow(dead_code)] +pub(crate) fn repartition( + input: Arc, + table_metadata: TableMetadataRef, + target_partitions: NonZeroUsize, +) -> DFResult> { + let partitioning_strategy = + determine_partitioning_strategy(&input, &table_metadata, target_partitions)?; + + Ok(Arc::new(RepartitionExec::try_new( + input, + partitioning_strategy, + )?)) +} + +/// Determine the optimal partitioning strategy based on table metadata. +/// +/// Analyzes the table's partition specification to select the most appropriate +/// DataFusion partitioning strategy for insert operations. +/// +/// ## Partitioning Strategy +/// +/// - **Partitioned tables**: Must have the `_partition` column in the input schema (added via +/// `project_with_partition`). Uses hash partitioning if the partition spec contains Identity +/// or Bucket transforms for good data distribution. Falls back to round-robin for temporal +/// transforms (Year, Month, Day, Hour) that don't provide uniform hash distribution. +/// +/// - **Unpartitioned tables**: Always uses round-robin batch partitioning to ensure even load +/// distribution across workers. +/// +/// ## Requirements +/// +/// - **For partitioned tables**: The input MUST include the `_partition` column +/// (added via `project_with_partition()`). +/// - **For unpartitioned tables**: No special preparation needed. +/// - Returns an error if a partitioned table is missing the `_partition` column. +fn determine_partitioning_strategy( + input: &Arc, + table_metadata: &TableMetadata, + target_partitions: NonZeroUsize, +) -> DFResult { + let partition_spec = table_metadata.default_partition_spec(); + let input_schema = input.schema(); + let target_partition_count = target_partitions.get(); + + // Check if partition spec has transforms suitable for hash partitioning + let has_hash_friendly_transforms = partition_spec + .fields() + .iter() + .any(|pf| matches!(pf.transform, Transform::Identity | Transform::Bucket(_))); + + let partition_col_result = input_schema.index_of(PROJECTED_PARTITION_VALUE_COLUMN); + let is_partitioned_table = !partition_spec.is_unpartitioned(); + + match (is_partitioned_table, partition_col_result) { + // Case 1: Partitioned table with _partition column present + (true, Ok(partition_col_idx)) => { + let partition_expr = Arc::new(Column::new( + PROJECTED_PARTITION_VALUE_COLUMN, + partition_col_idx, + )) as Arc; + + if has_hash_friendly_transforms { + Ok(Partitioning::Hash( + vec![partition_expr], + target_partition_count, + )) + } else { + Ok(Partitioning::RoundRobinBatch(target_partition_count)) + } + } + + // Case 2: Partitioned table missing _partition column (normally this should not happen) + (true, Err(_)) => Err(DataFusionError::Plan(format!( + "Partitioned table input missing {} column. \ + Ensure projection happens before repartitioning.", + PROJECTED_PARTITION_VALUE_COLUMN + ))), + + // Case 3: Unpartitioned table, always use RoundRobinBatch + (false, _) => Ok(Partitioning::RoundRobinBatch(target_partition_count)), + } +} + +#[cfg(test)] +mod tests { + use datafusion::arrow::datatypes::{ + DataType as ArrowDataType, Field as ArrowField, Fields, Schema as ArrowSchema, TimeUnit, + }; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::empty::EmptyExec; + use iceberg::TableIdent; + use iceberg::io::FileIO; + use iceberg::spec::{ + NestedField, NullOrder, PrimitiveType, Schema, SortDirection, SortField, SortOrder, + Transform, Type, + }; + use iceberg::table::Table; + + use super::*; + + fn create_test_table() -> Table { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .build() + .unwrap(); + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + + Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/metadata.json".to_string()) + .build() + .unwrap() + } + + fn create_test_arrow_schema() -> Arc { + Arc::new(ArrowSchema::new(vec![ + ArrowField::new("id", ArrowDataType::Int64, false), + ArrowField::new("data", ArrowDataType::Utf8, false), + ])) + } + + #[tokio::test] + async fn test_repartition_unpartitioned_table() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let repartitioned_plan = repartition( + input.clone(), + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + assert_ne!(input.name(), repartitioned_plan.name()); + assert_eq!(repartitioned_plan.name(), "RepartitionExec"); + } + + #[tokio::test] + async fn test_repartition_explicit_partitions() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(8).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::RoundRobinBatch(n) => { + assert_eq!(*n, 8); + } + _ => panic!("Expected RoundRobinBatch partitioning"), + } + } + + #[tokio::test] + async fn test_repartition_zero_partitions_fails() { + let _table = create_test_table(); + let _input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let result = std::num::NonZeroUsize::new(0); + assert!(result.is_none(), "NonZeroUsize::new(0) should return None"); + + // Test that we can't call repartition with 0 partitions + // This is prevented at compile time by NonZeroUsize + let _ = result; // This would be None, so we can't call repartition + } + + #[tokio::test] + async fn test_partition_count_validation() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let target_partitions = 16; + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(target_partitions).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::RoundRobinBatch(n) => { + assert_eq!(*n, target_partitions); + } + _ => panic!("Expected RoundRobinBatch partitioning"), + } + } + + #[tokio::test] + async fn test_datafusion_repartitioning_integration() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(3).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::RoundRobinBatch(n) => { + assert_eq!(*n, 3, "Should use round-robin for unpartitioned table"); + } + _ => panic!("Expected RoundRobinBatch partitioning for unpartitioned table"), + } + + let task_ctx = Arc::new(TaskContext::default()); + let stream = repartitioned_plan.execute(0, task_ctx.clone()).unwrap(); + + assert!(!stream.schema().fields().is_empty()); + } + + #[tokio::test] + async fn test_bucket_aware_partitioning() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 2, + "category", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(1) + .with_sort_field(SortField { + source_id: 2, + transform: Transform::Bucket(4), + direction: SortDirection::Ascending, + null_order: NullOrder::First, + }) + .build(&schema) + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .build() + .unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/bucketed_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "bucketed_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/bucketed_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("id", ArrowDataType::Int64, false), + ArrowField::new("category", ArrowDataType::Utf8, false), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + // For bucketed tables without _partition column, should use round-robin + // since the new logic prioritizes _partition column when available + match partitioning { + Partitioning::Hash(_, _) => { + // This would happen if _partition column is present + } + Partitioning::RoundRobinBatch(_) => { + // This happens when _partition column is not present + } + _ => panic!("Unexpected partitioning strategy"), + } + } + + #[tokio::test] + async fn test_combined_partition_and_bucket_strategy() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::required( + 2, + "user_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 3, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("date", "date", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(1) + .with_sort_field(SortField { + source_id: 2, + transform: Transform::Bucket(8), + direction: SortDirection::Ascending, + null_order: NullOrder::First, + }) + .build(&schema) + .unwrap(); + + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/partitioned_bucketed_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "partitioned_bucketed_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/partitioned_bucketed_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("date", ArrowDataType::Date32, false), + ArrowField::new("user_id", ArrowDataType::Int64, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ArrowField::new( + PROJECTED_PARTITION_VALUE_COLUMN, + ArrowDataType::Struct(Fields::empty()), + false, + ), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::Hash(exprs, _) => { + // Should use _partition column for hash partitioning + assert_eq!( + exprs.len(), + 1, + "Should have exactly one hash column (_partition)" + ); + + let column_names: Vec = exprs + .iter() + .filter_map(|expr| { + expr.as_any() + .downcast_ref::() + .map(|col| col.name().to_string()) + }) + .collect(); + + assert!( + column_names.contains(&PROJECTED_PARTITION_VALUE_COLUMN.to_string()), + "Should use _partition column, got: {:?}", + column_names + ); + } + _ => panic!("Expected Hash partitioning with Identity transform"), + } + } + + #[tokio::test] + async fn test_none_distribution_mode_fallback() { + let schema = Schema::builder() + .with_fields(vec![Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + ))]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .build() + .unwrap(); + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + + let mut properties = std::collections::HashMap::new(); + properties.insert("write.distribution-mode".to_string(), "none".to_string()); + + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/none_table".to_string(), + iceberg::spec::FormatVersion::V2, + properties, + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "none_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/none_metadata.json".to_string()) + .build() + .unwrap(); + + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::RoundRobinBatch(_)), + "Should use round-robin for 'none' distribution mode" + ); + } + + #[tokio::test] + async fn test_schema_ref_convenience_method() { + let table = create_test_table(); + + let schema_ref_1 = table.current_schema_ref(); + let schema_ref_2 = Arc::clone(table.metadata().current_schema()); + + assert!( + Arc::ptr_eq(&schema_ref_1, &schema_ref_2), + "current_schema_ref() should return the same Arc as manual approach" + ); + } + + #[tokio::test] + async fn test_range_only_partitions_use_round_robin() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::required( + 2, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("date", "date_day", Transform::Day) + .unwrap() + .build() + .unwrap(); + + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/range_only_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "range_only_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/range_only_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("date", ArrowDataType::Date32, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ArrowField::new( + PROJECTED_PARTITION_VALUE_COLUMN, + ArrowDataType::Struct(Fields::empty()), + false, + ), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::RoundRobinBatch(_)), + "Should use round-robin for temporal transforms (Day) that don't provide good hash distribution" + ); + } + + #[tokio::test] + async fn test_mixed_transforms_use_hash_partitioning() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::required( + 2, + "user_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 3, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("date", "date_day", Transform::Day) + .unwrap() + .add_partition_field("user_id", "user_id", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/mixed_transforms_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "mixed_transforms_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/mixed_transforms_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("date", ArrowDataType::Date32, false), + ArrowField::new("user_id", ArrowDataType::Int64, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ArrowField::new( + PROJECTED_PARTITION_VALUE_COLUMN, + ArrowDataType::Struct(Fields::empty()), + false, + ), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::Hash(exprs, _) => { + assert_eq!(exprs.len(), 1, "Should have one hash column (_partition)"); + let column_names: Vec = exprs + .iter() + .filter_map(|expr| { + expr.as_any() + .downcast_ref::() + .map(|col| col.name().to_string()) + }) + .collect(); + assert!( + column_names.contains(&PROJECTED_PARTITION_VALUE_COLUMN.to_string()), + "Should use _partition column for mixed transforms with Identity, got: {:?}", + column_names + ); + } + _ => panic!("Expected Hash partitioning for table with identity transforms"), + } + } + + #[tokio::test] + async fn test_partition_column_with_temporal_transforms_uses_round_robin() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "event_time", + Type::Primitive(PrimitiveType::Timestamp), + )), + Arc::new(NestedField::required( + 2, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("event_time", "event_month", Transform::Month) + .unwrap() + .build() + .unwrap(); + + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/temporal_partition".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "temporal_partition"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/temporal_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new( + "event_time", + ArrowDataType::Timestamp(TimeUnit::Microsecond, None), + false, + ), + ArrowField::new("amount", ArrowDataType::Int64, false), + ArrowField::new( + PROJECTED_PARTITION_VALUE_COLUMN, + ArrowDataType::Struct(Fields::empty()), + false, + ), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::RoundRobinBatch(_)), + "Should use round-robin for _partition column with temporal transforms, not Hash" + ); + } + + #[tokio::test] + async fn test_partition_column_with_identity_transforms_uses_hash() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "user_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 2, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("user_id", "user_id", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/identity_partition".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "identity_partition"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/identity_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("user_id", ArrowDataType::Int64, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ArrowField::new( + PROJECTED_PARTITION_VALUE_COLUMN, + ArrowDataType::Struct(Fields::empty()), + false, + ), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + + let repartitioned_plan = repartition( + input, + table.metadata_ref(), + std::num::NonZeroUsize::new(4).unwrap(), + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::Hash(_, _)), + "Should use Hash partitioning for _partition column with Identity transforms" + ); + } +} From 589b57ccc44e01813e1e5c4dc6cd6c745b3f6505 Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Tue, 4 Nov 2025 11:05:39 +0100 Subject: [PATCH 4/7] feat(core): Support file deletion in incremental scan (#4) * Remove TODO * Implement delete files * PR comments * Fix test comments * Comment fix * Fix upstream changes * Modify test * format --- crates/iceberg/src/arrow/incremental.rs | 121 ++++++- crates/iceberg/src/arrow/reader.rs | 6 + .../iceberg/src/scan/incremental/context.rs | 1 - crates/iceberg/src/scan/incremental/mod.rs | 109 ++++-- crates/iceberg/src/scan/incremental/task.rs | 89 ++++- crates/iceberg/src/scan/incremental/tests.rs | 309 +++++++++++++----- 6 files changed, 499 insertions(+), 136 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index e7ca2521bf..0eeebfd5ef 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; use std::pin::Pin; use std::sync::Arc; @@ -23,10 +24,12 @@ use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use futures::channel::mpsc::channel; use futures::stream::select; use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; +use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::record_batch_transformer::RecordBatchTransformer; use crate::arrow::{ - ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_FIELD_ID_FILE_PATH, StreamsInto, + ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_COL_NAME_POS, RESERVED_FIELD_ID_FILE_PATH, + RESERVED_FIELD_ID_POS, StreamsInto, }; use crate::delete_vector::DeleteVector; use crate::io::FileIO; @@ -37,6 +40,22 @@ use crate::scan::incremental::{ }; use crate::{Error, ErrorKind, Result}; +/// Default batch size for incremental delete operations. +const DEFAULT_BATCH_SIZE: usize = 1024; + +/// Creates the schema for positional delete records containing the "pos" column. +/// The pos field includes the reserved field ID as metadata. +fn create_pos_delete_schema() -> Arc { + let pos_field = + Field::new(RESERVED_COL_NAME_POS, DataType::UInt64, false).with_metadata(HashMap::from([ + ( + PARQUET_FIELD_ID_META_KEY.to_string(), + RESERVED_FIELD_ID_POS.to_string(), + ), + ])); + Arc::new(ArrowSchema::new(vec![pos_field])) +} + /// The type of incremental batch: appended data or deleted records. #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub enum IncrementalBatchType { @@ -120,9 +139,49 @@ impl StreamsInto let _ = appends_tx.send(Err(e)).await; } } - }); + }) + .await + } + IncrementalFileScanTask::Delete(deleted_file_task) => { + spawn(async move { + let file_path = deleted_file_task.data_file_path().to_string(); + let total_records = deleted_file_task.base.record_count.unwrap_or(0); + + let record_batch_stream = process_incremental_deleted_file_task( + file_path, + total_records, + batch_size, + ); + + match record_batch_stream { + Ok(mut stream) => { + while let Some(batch) = stream.next().await { + let result = deletes_tx + .send(batch.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read deleted file record batch", + ) + .with_source(e) + })) + .await; + + if result.is_err() { + break; + } + } + } + Err(e) => { + let _ = deletes_tx.send(Err(e)).await; + } + } + }) + .await } - IncrementalFileScanTask::Delete(file_path, delete_vector) => { + IncrementalFileScanTask::PositionalDeletes( + file_path, + delete_vector, + ) => { spawn(async move { let record_batch_stream = process_incremental_delete_task( file_path, @@ -152,7 +211,8 @@ impl StreamsInto let _ = deletes_tx.send(Err(e)).await; } } - }); + }) + .await } }; @@ -175,19 +235,21 @@ async fn process_incremental_append_task( file_io: FileIO, ) -> Result { let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( - &task.data_file_path, + &task.base.data_file_path, file_io, true, + None, // arrow_reader_options ) .await?; // Create a projection mask for the batch stream to select which columns in the // Parquet file that we want in the response let projection_mask = ArrowReader::get_arrow_projection_mask( - &task.project_field_ids, + &task.base.project_field_ids, &task.schema_ref(), record_batch_stream_builder.parquet_schema(), record_batch_stream_builder.schema(), + false, // use_fallback )?; record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); @@ -195,7 +257,7 @@ async fn process_incremental_append_task( // that come back from the file, such as type promotion, default column insertion // and column re-ordering let mut record_batch_transformer = - RecordBatchTransformer::build(task.schema_ref(), &task.project_field_ids); + RecordBatchTransformer::build(task.schema_ref(), &task.base.project_field_ids); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -233,13 +295,9 @@ fn process_incremental_delete_task( delete_vector: DeleteVector, batch_size: Option, ) -> Result { - let schema = Arc::new(ArrowSchema::new(vec![Field::new( - "pos", - DataType::UInt64, - false, - )])); + let schema = create_pos_delete_schema(); - let batch_size = batch_size.unwrap_or(1024); + let batch_size = batch_size.unwrap_or(DEFAULT_BATCH_SIZE); let treemap = delete_vector.inner; @@ -269,3 +327,40 @@ fn process_incremental_delete_task( Ok(Box::pin(stream) as ArrowRecordBatchStream) } + +fn process_incremental_deleted_file_task( + file_path: String, + total_records: u64, + batch_size: Option, +) -> Result { + let schema = create_pos_delete_schema(); + + let batch_size = batch_size.unwrap_or(DEFAULT_BATCH_SIZE); + + // Create a stream of position values from 0 to total_records-1 (0-indexed) + let stream = futures::stream::iter(0..total_records) + .chunks(batch_size) + .map(move |chunk| { + let array = UInt64Array::from_iter(chunk); + RecordBatch::try_new( + Arc::clone(&schema), // Cheap Arc clone instead of full schema creation + vec![Arc::new(array)], + ) + .map_err(|_| { + Error::new( + ErrorKind::Unexpected, + "Failed to create RecordBatch for deleted file", + ) + }) + .and_then(|batch| { + ArrowReader::add_file_path_column( + batch, + &file_path, + RESERVED_COL_NAME_FILE_PATH, + RESERVED_FIELD_ID_FILE_PATH, + ) + }) + }); + + Ok(Box::pin(stream) as ArrowRecordBatchStream) +} diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index cc44c4f0a9..d44e813fd1 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -78,6 +78,12 @@ pub(crate) const RESERVED_FIELD_ID_FILE_PATH: i32 = 2147483546; /// Column name for the file path metadata column used in delete file reading. pub(crate) const RESERVED_COL_NAME_FILE_PATH: &str = "file_path"; +/// Reserved field ID for the position column used in delete file reading. +pub(crate) const RESERVED_FIELD_ID_POS: i32 = 2147483545; + +/// Column name for the position metadata column used in delete file reading. +pub(crate) const RESERVED_COL_NAME_POS: &str = "pos"; + /// Builder to create ArrowReader pub struct ArrowReaderBuilder { batch_size: Option, diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index 04db28a4c4..3f6169270e 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -113,7 +113,6 @@ impl IncrementalPlanContext { (manifest_files, filter_fn) }; - // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut mfcs = vec![]; for manifest_file in &manifest_files { let tx = if manifest_file.content == ManifestContentType::Deletes { diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index 1253353b4d..af011dfc16 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -406,11 +406,14 @@ impl IncrementalTableScan { } else if manifest_entry_context.manifest_entry.status() == ManifestStatus::Deleted { - // TODO (RAI-43291): Process deleted files - Err(Error::new( - ErrorKind::FeatureUnsupported, - "Processing deleted data files is not supported yet in incremental scans", - )) + spawn(async move { + Self::process_deleted_data_manifest_entry( + tx, + manifest_entry_context, + ) + .await + }) + .await } else { Ok(()) } @@ -424,33 +427,63 @@ impl IncrementalTableScan { } }); - // Collect all append tasks. - let mut tasks = file_scan_task_rx.try_collect::>().await?; + // Collect all tasks from manifest processing. + let all_tasks = file_scan_task_rx.try_collect::>().await?; + + // Separate tasks by type and compute file path sets in a single pass + let mut append_tasks = Vec::new(); + let mut delete_tasks = Vec::new(); + let mut appended_files = HashSet::new(); + let mut deleted_files = HashSet::new(); - // Compute those file paths that have been appended. - let appended_files = tasks - .iter() - .filter_map(|task| match task { + for task in all_tasks { + match task { IncrementalFileScanTask::Append(append_task) => { - Some(append_task.data_file_path.clone()) + appended_files.insert(append_task.data_file_path().to_string()); + append_tasks.push(append_task); } - _ => None, - }) - .collect::>(); + IncrementalFileScanTask::Delete(delete_task) => { + deleted_files.insert(delete_task.data_file_path().to_string()); + delete_tasks.push(delete_task); + } + _ => {} + } + } + + // Build final task list with net changes + // We filter out tasks for files that appear in both sets (cancelled out) + let mut final_tasks: Vec = Vec::new(); + + // Add net append tasks (only files not in deleted_files) + for append_task in append_tasks { + if !deleted_files.contains(append_task.data_file_path()) { + final_tasks.push(IncrementalFileScanTask::Append(append_task)); + } + } + + // Add net delete tasks (only files not in appended_files) + for delete_task in delete_tasks { + if !appended_files.contains(delete_task.data_file_path()) { + final_tasks.push(IncrementalFileScanTask::Delete(delete_task)); + } + } - // Augment `tasks` with delete tasks. - // First collect paths to process (paths that weren't appended in this scan range) - let delete_paths: Vec = delete_filter.with_read(|state| { + // Add positional delete tasks (only for files that haven't been deleted) + let positional_delete_paths: Vec = delete_filter.with_read(|state| { Ok(state .delete_vectors() .keys() - .filter(|path| !appended_files.contains::(path)) + .filter(|path| { + // Only include positional deletes for files that were not appended in + // this range and not deleted. + !appended_files.contains::(path) && !deleted_files.contains::(path) + }) .cloned() .collect()) })?; // Now remove and take ownership of each delete vector - for path in delete_paths { + for path in positional_delete_paths { let delete_vector_arc = delete_filter.with_write(|state| { state.remove_delete_vector(&path).ok_or_else(|| { Error::new( @@ -474,13 +507,14 @@ impl IncrementalTableScan { .with_source(e) })?; - let delete_task = IncrementalFileScanTask::Delete(path, delete_vector_inner); - tasks.push(delete_task); + let positional_delete_task = + IncrementalFileScanTask::PositionalDeletes(path, delete_vector_inner); + final_tasks.push(positional_delete_task); } // We actually would not need a stream here, but we can keep it compatible with // other scan types. - Ok(futures::stream::iter(tasks).map(Ok).boxed()) + Ok(futures::stream::iter(final_tasks).map(Ok).boxed()) } /// Returns an [`CombinedIncrementalBatchRecordStream`] for this incremental table scan. @@ -582,6 +616,35 @@ impl IncrementalTableScan { file_scan_task_tx.send(Ok(file_scan_task)).await?; Ok(()) } + + async fn process_deleted_data_manifest_entry( + mut file_scan_task_tx: Sender>, + manifest_entry_context: ManifestEntryContext, + ) -> Result<()> { + // Abort the plan if we encounter a manifest entry for a delete file + if manifest_entry_context.manifest_entry.content_type() != DataContentType::Data { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Encountered an entry for a delete file in a data file manifest", + )); + } + + let data_file_path = manifest_entry_context.manifest_entry.file_path(); + let file_scan_task = IncrementalFileScanTask::Delete(DeletedFileScanTask { + base: BaseIncrementalFileScanTask { + start: 0, + length: manifest_entry_context.manifest_entry.file_size_in_bytes(), + record_count: Some(manifest_entry_context.manifest_entry.record_count()), + data_file_path: data_file_path.to_string(), + data_file_format: manifest_entry_context.manifest_entry.file_format(), + schema: manifest_entry_context.snapshot_schema.clone(), + project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), + }, + }); + + file_scan_task_tx.send(Ok(file_scan_task)).await?; + Ok(()) + } } #[cfg(test)] diff --git a/crates/iceberg/src/scan/incremental/task.rs b/crates/iceberg/src/scan/incremental/task.rs index 4b2f244410..e05703ae28 100644 --- a/crates/iceberg/src/scan/incremental/task.rs +++ b/crates/iceberg/src/scan/incremental/task.rs @@ -25,9 +25,9 @@ use crate::delete_vector::DeleteVector; use crate::scan::context::ManifestEntryContext; use crate::spec::{DataFileFormat, Schema, SchemaRef}; -/// A file scan task for appended data files in an incremental scan. +/// Base file scan task containing common attributes for incremental scan tasks. #[derive(Debug, Clone)] -pub struct AppendedFileScanTask { +pub struct BaseIncrementalFileScanTask { /// The start offset of the file to scan. pub start: u64, /// The length of the file to scan. @@ -42,6 +42,30 @@ pub struct AppendedFileScanTask { pub schema: crate::spec::SchemaRef, /// The field ids to project. pub project_field_ids: Vec, +} + +impl BaseIncrementalFileScanTask { + /// Returns the data file path of this file scan task. + pub fn data_file_path(&self) -> &str { + &self.data_file_path + } + + /// Returns the schema of this file scan task as a reference + pub fn schema(&self) -> &Schema { + &self.schema + } + + /// Returns the schema of this file scan task as a SchemaRef + pub fn schema_ref(&self) -> SchemaRef { + self.schema.clone() + } +} + +/// A file scan task for appended data files in an incremental scan. +#[derive(Debug, Clone)] +pub struct AppendedFileScanTask { + /// The base file scan task attributes. + pub base: BaseIncrementalFileScanTask, /// The optional positional deletes associated with this data file. pub positional_deletes: Option>>, } @@ -49,32 +73,58 @@ pub struct AppendedFileScanTask { impl AppendedFileScanTask { /// Returns the data file path of this appended file scan task. pub fn data_file_path(&self) -> &str { - &self.data_file_path + self.base.data_file_path() } /// Returns the schema of this file scan task as a reference pub fn schema(&self) -> &Schema { - &self.schema + self.base.schema() } /// Returns the schema of this file scan task as a SchemaRef pub fn schema_ref(&self) -> SchemaRef { - self.schema.clone() + self.base.schema_ref() + } +} + +/// A file scan task for deleted data files in an incremental scan. +#[derive(Debug, Clone)] +pub struct DeletedFileScanTask { + /// The base file scan task attributes. + pub base: BaseIncrementalFileScanTask, +} + +impl DeletedFileScanTask { + /// Returns the data file path of this deleted file scan task. + pub fn data_file_path(&self) -> &str { + self.base.data_file_path() + } + + /// Returns the schema of this file scan task as a reference + pub fn schema(&self) -> &Schema { + self.base.schema() + } + + /// Returns the schema of this file scan task as a SchemaRef + pub fn schema_ref(&self) -> SchemaRef { + self.base.schema_ref() } } /// The stream of incremental file scan tasks. pub type IncrementalFileScanTaskStream = BoxStream<'static, Result>; -/// An incremental file scan task, which can be either an appended data file or positional -/// deletes. +/// An incremental file scan task, which can be an appended data file, deleted data file, +/// or positional deletes. #[derive(Debug, Clone)] pub enum IncrementalFileScanTask { /// An appended data file. Append(AppendedFileScanTask), - /// Deleted records of a data file. First argument is the file path, second the delete - /// vector. - Delete(String, DeleteVector), + /// A deleted data file. + Delete(DeletedFileScanTask), + /// Positional deletes (deleted records of a data file). First argument is the file path, + /// second the delete vector. + PositionalDeletes(String, DeleteVector), } impl IncrementalFileScanTask { @@ -85,13 +135,15 @@ impl IncrementalFileScanTask { ) -> Self { let data_file_path = manifest_entry_context.manifest_entry.file_path(); IncrementalFileScanTask::Append(AppendedFileScanTask { - start: 0, - length: manifest_entry_context.manifest_entry.file_size_in_bytes(), - record_count: Some(manifest_entry_context.manifest_entry.record_count()), - data_file_path: data_file_path.to_string(), - data_file_format: manifest_entry_context.manifest_entry.file_format(), - schema: manifest_entry_context.snapshot_schema.clone(), - project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), + base: BaseIncrementalFileScanTask { + start: 0, + length: manifest_entry_context.manifest_entry.file_size_in_bytes(), + record_count: Some(manifest_entry_context.manifest_entry.record_count()), + data_file_path: data_file_path.to_string(), + data_file_format: manifest_entry_context.manifest_entry.file_format(), + schema: manifest_entry_context.snapshot_schema.clone(), + project_field_ids: manifest_entry_context.field_ids.as_ref().clone(), + }, positional_deletes: delete_filter.get_delete_vector_for_path(data_file_path), }) } @@ -100,7 +152,8 @@ impl IncrementalFileScanTask { pub fn data_file_path(&self) -> &str { match self { IncrementalFileScanTask::Append(task) => task.data_file_path(), - IncrementalFileScanTask::Delete(path, _) => path, + IncrementalFileScanTask::Delete(task) => task.data_file_path(), + IncrementalFileScanTask::PositionalDeletes(path, _) => path, } } } diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 05724a54a4..32f0078736 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -1589,21 +1589,182 @@ async fn test_incremental_scan_builder_options() { } #[tokio::test] -async fn test_incremental_scan_with_deleted_files_errors() { - // This test verifies that incremental scans properly error out when entire data files - // are deleted (overwrite operation), since this is not yet supported. +async fn test_incremental_scan_append_then_delete_file() { + // This test verifies the basic delete file functionality: + // - Snapshot 1: Empty starting point + // - Snapshot 2: Append a file with 3 records + // - Snapshot 3: Overwrite that deletes the entire file + // + // An incremental scan from snapshot 1 to 3 should yield no tuples, as there is no net change. + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Append a file with 3 records + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + ], + "data-1.parquet".to_string(), + ), + // Snapshot 3: Overwrite that deletes the entire file + Operation::Overwrite( + (vec![], "".to_string()), // No new data to add + vec![], // No positional deletes + vec!["data-1.parquet".to_string()], // Delete the file completely + ), + ]) + .await; + + // Verify we have 3 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 3); + + // Verify snapshot IDs + assert_eq!(snapshots[0].snapshot_id(), 1); + assert_eq!(snapshots[1].snapshot_id(), 2); + assert_eq!(snapshots[2].snapshot_id(), 3); + + // Incremental scan from snapshot 1 to 3 should yield no tuples + // The file was added in snapshot 2 and deleted in snapshot 3, so they cancel out + fixture + .verify_incremental_scan( + 1, + 3, + vec![], // No net appends + vec![], // No net deletes + ) + .await; + + // Incremental scan from snapshot 1 to 2 should yield the appended records + fixture + .verify_incremental_scan( + 1, + 2, + vec![(1, "a"), (2, "b"), (3, "c")], // All 3 records appended + vec![], // No deletes + ) + .await; + + // Incremental scan from snapshot 2 to 3 should produce delete records + // The file existed at snapshot 2 and was deleted in snapshot 3 + let data_file_path = format!("{}/data/data-1.parquet", fixture.table_location); + fixture + .verify_incremental_scan( + 2, + 3, + vec![], // No appends + vec![ + // All 3 positions deleted (pos values are 0-indexed: 0, 1, 2) + (0, data_file_path.as_str()), + (1, data_file_path.as_str()), + (2, data_file_path.as_str()), + ], + ) + .await; +} + +#[tokio::test] +async fn test_incremental_scan_positional_deletes_then_file_delete() { + // This test verifies that the system correctly handles the case where: + // - Snapshot 1: Empty starting point + // - Snapshot 2: Append a file with 3 records + // - Snapshot 3: Delete these 3 records using positional deletes + // - Snapshot 4: Delete the file entirely (that was introduced in snapshot 2) + // + // The system should avoid double-deletes by filtering out positional deletes + // for files that are subsequently deleted entirely. + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Append a file with 3 records + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + ], + "data-1.parquet".to_string(), + ), + // Snapshot 3: Delete one record using a positional delete + Operation::Delete(vec![(0, "data-1.parquet".to_string())]), + // Snapshot 4: Delete the file entirely + Operation::Overwrite( + (vec![], "".to_string()), // No new data to add + vec![], // No positional deletes + vec!["data-1.parquet".to_string()], // Delete the file completely + ), + ]) + .await; + + // Verify we have 4 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 4); + + // Incremental scan from snapshot 1 to 4 + // Since data-1.parquet was both appended (snapshot 2) and deleted (snapshot 4) + // in the scan range, BOTH appends and deletes are fully cancelled out. + // Even though there were positional deletes in snapshot 3, those are filtered out + // because the file qualifies for cancellation (appended && deleted). + let data_file_path = format!("{}/data/data-1.parquet", fixture.table_location); + fixture + .verify_incremental_scan( + 1, + 4, + vec![], // No appends (file was added and fully deleted, cancelled out) + vec![], // No deletes (fully cancelled since file was added and deleted in range) + ) + .await; + + // Incremental scan from snapshot 3 to 4 + // This demonstrates the double-delete scenario: + // - At snapshot 3, all records have already been deleted via positional deletes + // - At snapshot 4, the entire file is deleted + // - The scan from 3 to 4 should still emit delete records for the file, + // even though the records were already deleted in snapshot 3 + // This is because the incremental scan doesn't know about the prior state + fixture + .verify_incremental_scan( + 3, + 4, + vec![], // No appends + vec![ + // File deletion emits positions 0, 1, 2 + // These are "redundant" deletes since the records were already + // deleted by positional deletes in snapshot 3 + (0, data_file_path.as_str()), + (1, data_file_path.as_str()), + (2, data_file_path.as_str()), + ], + ) + .await; +} + +#[tokio::test] +async fn test_incremental_scan_with_deleted_files_cancellation() { + // This test verifies that incremental scans properly handle file deletions with cancellation logic: + // - Files added and deleted in the same scan range should cancel out + // - Files deleted that weren't added in the scan range should produce Delete tasks // // Test scenario: - // Snapshot 1: Add file-1.parquet with data - // Snapshot 2: Add file-2.parquet with data - // Snapshot 3: Overwrite - delete file-1.parquet entirely - // Snapshot 4: Add file-3.parquet with data + // Snapshot 1: Empty starting point + // Snapshot 2: Add file-1.parquet with data (3 records) + // Snapshot 3: Add file-2.parquet with data (2 records) + // Snapshot 4: Overwrite - delete file-1.parquet entirely + // Snapshot 5: Add file-3.parquet with data (1 record) // - // Incremental scan from snapshot 1 to snapshot 3 should error because file-1 - // was completely removed in the overwrite operation. + // Incremental scan from snapshot 1 to 4: file-1 added and deleted, should cancel out (only file-2 remains). + // Incremental scan from snapshot 3 to 5: file-1 deleted but not added in range, produces Delete tasks. + // Incremental scan from snapshot 1 to 3: file-1 and file-2 added before any deletion. + // Incremental scan from snapshot 4 to 5: file-3 added after file-1 deletion occurred. let fixture = IncrementalTestFixture::new(vec![ - // Snapshot 1: Add file-1 with rows + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add file-1 with rows Operation::Add( vec![ (1, "a".to_string()), @@ -1612,92 +1773,78 @@ async fn test_incremental_scan_with_deleted_files_errors() { ], "file-1.parquet".to_string(), ), - // Snapshot 2: Add file-2 with rows + // Snapshot 3: Add file-2 with rows Operation::Add( vec![(10, "x".to_string()), (20, "y".to_string())], "file-2.parquet".to_string(), ), - // Snapshot 3: Overwrite - delete file-1 entirely + // Snapshot 4: Overwrite - delete file-1 entirely Operation::Overwrite( (vec![], "".to_string()), // No new data to add vec![], // No positional deletes vec!["file-1.parquet".to_string()], // Delete file-1 completely ), - // Snapshot 4: Add file-3 (to have more snapshots) + // Snapshot 5: Add file-3 (to have more snapshots) Operation::Add(vec![(100, "p".to_string())], "file-3.parquet".to_string()), ]) .await; - // Test 1: Incremental scan from snapshot 1 to 3 should error when building the stream - // because file-1 was deleted entirely in snapshot 3 - let scan = fixture - .table - .incremental_scan(1, 3) - .build() - .expect("Building the scan should succeed"); - - let stream_result = scan.to_arrow().await; - - match stream_result { - Err(error) => { - assert_eq!( - error.message(), - "Processing deleted data files is not supported yet in incremental scans", - "Error message should indicate that deleted files are not supported. Got: {}", - error - ); - } - Ok(_) => panic!( - "Expected error when building stream over a snapshot that deletes entire data files" - ), - } - - // Test 2: Incremental scan from snapshot 2 to 4 should also error - // because it includes snapshot 3 which deletes a file - let scan = fixture - .table - .incremental_scan(2, 4) - .build() - .expect("Building the scan should succeed"); - - let stream_result = scan.to_arrow().await; - - match stream_result { - Err(_) => { - // Expected error - } - Ok(_) => panic!("Expected error when scan range includes a snapshot that deletes files"), - } - - // Test 3: Incremental scan from snapshot 1 to 2 should work fine - // (no files deleted) - let scan = fixture - .table - .incremental_scan(1, 2) - .build() - .expect("Building the scan should succeed"); - - let stream_result = scan.to_arrow().await; + let file_1_path = format!("{}/data/file-1.parquet", fixture.table_location); - assert!( - stream_result.is_ok(), - "Scan should succeed when no files are deleted. Error: {:?}", - stream_result.err() - ); + // Test 1: Incremental scan from snapshot 1 to 4 + // file-1 was added in snapshot 2 and deleted in snapshot 4 + // Both appends and deletes for file-1 are fully cancelled out: + // - Appends: only file-2 records (10, x), (20, y) + // - Deletes: none (file-1 deletions cancelled since it was added in range) + fixture + .verify_incremental_scan( + 1, + 4, + vec![(10, "x"), (20, "y")], // Only file-2 (file-1 cancelled out) + vec![], // No deletes (file-1 fully cancelled) + ) + .await; - // Test 4: Incremental scan from snapshot 3 to 4 should work - // (starting from after the deletion) - let scan = fixture - .table - .incremental_scan(3, 4) - .build() - .expect("Building the scan should succeed"); + // Test 2: Incremental scan from snapshot 3 to 5 + // file-1 was deleted in snapshot 4 but wasn't added in the scan range (added in snapshot 2) + // This produces a net Delete task with positions 0, 1, 2 (all records in file-1) + fixture + .verify_incremental_scan( + 3, + 5, + vec![(100, "p")], // file-3 added in snapshot 5 + vec![ + // file-1 deleted in snapshot 4 (all 3 positions: 0, 1, 2) + (0, file_1_path.as_str()), + (1, file_1_path.as_str()), + (2, file_1_path.as_str()), + ], + ) + .await; - let stream_result = scan.to_arrow().await; + // Test 3: Incremental scan from snapshot 1 to 3 + // Verifies basic append-only operations before any deletions occur. + // Both file-1 (snapshot 2) and file-2 (snapshot 3) are added, no deletions yet. + // Expected: All records from both files appear in appends, no deletes. + fixture + .verify_incremental_scan( + 1, + 3, + vec![(1, "a"), (2, "b"), (3, "c"), (10, "x"), (20, "y")], + vec![], // No deletes + ) + .await; - assert!( - stream_result.is_ok(), - "Scan should succeed when starting after the file deletion. Error: {:?}", - stream_result.err() - ); + // Test 4: Incremental scan from snapshot 4 to 5 + // Verifies scanning after the deletion has already occurred. + // Starting from snapshot 4 (after file-1 deletion), only file-3 is added in snapshot 5. + // Expected: Only file-3 records in appends, no deletes (deletion happened before scan range). + fixture + .verify_incremental_scan( + 4, + 5, + vec![(100, "p")], // file-3 added + vec![], // No deletes + ) + .await; } From 37e79b805407a0340d08823373cafed9cdac0083 Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Mon, 10 Nov 2025 10:39:58 +0100 Subject: [PATCH 5/7] feat(core): Add support for replace in incremental scan (#8) * Add support for replace * Add positional delete test, refactor match arm --- .../iceberg/src/scan/incremental/context.rs | 27 +- crates/iceberg/src/scan/incremental/tests.rs | 598 ++++++++++++++++-- 2 files changed, 562 insertions(+), 63 deletions(-) diff --git a/crates/iceberg/src/scan/incremental/context.rs b/crates/iceberg/src/scan/incremental/context.rs index 3f6169270e..fbd9219d5c 100644 --- a/crates/iceberg/src/scan/incremental/context.rs +++ b/crates/iceberg/src/scan/incremental/context.rs @@ -27,8 +27,7 @@ use crate::io::object_cache::ObjectCache; use crate::scan::ExpressionEvaluatorCache; use crate::scan::context::{ManifestEntryContext, ManifestEntryFilterFn, ManifestFileContext}; use crate::spec::{ - ManifestContentType, ManifestEntryRef, ManifestFile, Operation, SchemaRef, SnapshotRef, - TableMetadataRef, + ManifestContentType, ManifestEntryRef, ManifestFile, SchemaRef, SnapshotRef, TableMetadataRef, }; #[derive(Debug)] @@ -65,28 +64,8 @@ impl IncrementalPlanContext { delete_file_idx: DeleteFileIndex, delete_file_tx: Sender, ) -> Result> + 'static>> { - // Validate that all snapshots are Append or Delete operations and collect their IDs - let snapshot_ids: HashSet = { - let mut ids = HashSet::new(); - for snapshot in self.snapshots.iter() { - let operation = &snapshot.summary().operation; - if !matches!( - operation, - Operation::Append | Operation::Overwrite | Operation::Delete - ) { - return Err(crate::Error::new( - crate::ErrorKind::FeatureUnsupported, - format!( - "Incremental scan only supports Append, Overwrite and Delete operations, but snapshot {} has operation {:?}", - snapshot.snapshot_id(), - operation - ), - )); - } - ids.insert(snapshot.snapshot_id()); - } - ids - }; + // Collect all snapshot IDs (all operation types are supported) + let snapshot_ids: HashSet = self.snapshots.iter().map(|s| s.snapshot_id()).collect(); let (manifest_files, filter_fn) = { let mut manifest_files = HashSet::::new(); diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 32f0078736..0ae836b53f 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -33,7 +33,7 @@ use crate::TableIdent; use crate::io::{FileIO, OutputFile}; use crate::spec::{ DataContentType, DataFileBuilder, DataFileFormat, ManifestEntry, ManifestListWriter, - ManifestStatus, ManifestWriterBuilder, PartitionSpec, Struct, TableMetadata, + ManifestStatus, ManifestWriterBuilder, PartitionSpec, SchemaRef, Struct, TableMetadata, }; use crate::table::Table; @@ -72,6 +72,20 @@ pub enum Operation { Vec<(i64, String)>, Vec, ), + + /// Replace operation for file compaction/reorganization. + /// The logical table content does NOT change - only the physical file representation changes. + /// + /// Parameters: + /// 1. Files to compact: Vec of existing file names that are being compacted + /// 2. Target file: String name of the new compacted file + /// + /// Example: `Replace(vec!["file-a.parquet", "file-b.parquet"], "file-a-b-compacted.parquet")` + /// This compacts two existing files into one new file with the same logical content. + /// + /// For an incremental scan that only contains Replace operations, the result should be + /// zero additions and zero deletions, because the logical data hasn't changed. + Replace(Vec, String), } /// Tracks the state of data files across snapshots @@ -81,6 +95,8 @@ struct DataFileInfo { snapshot_id: i64, sequence_number: i64, n_values: Vec, + data_values: Vec, + file_size: u64, } /// Test fixture that creates a table with custom snapshots based on operations. @@ -110,7 +126,6 @@ pub struct IncrementalTestFixture { impl IncrementalTestFixture { /// Create a new test fixture with the given operations. pub async fn new(operations: Vec) -> Self { - // Use pwd let tmp_dir = TempDir::new().unwrap(); let table_location = tmp_dir.path().join("incremental_test_table"); @@ -142,6 +157,7 @@ impl IncrementalTestFixture { Operation::Add(..) => "append", Operation::Delete(..) => "delete", Operation::Overwrite(..) => "overwrite", + Operation::Replace(..) => "replace", }; let manifest_list_location = @@ -282,7 +298,7 @@ impl IncrementalTestFixture { // Track all data files and their contents across snapshots let mut data_files: Vec = Vec::new(); #[allow(clippy::type_complexity)] - let mut delete_files: Vec<(String, i64, i64, Vec<(String, i64)>)> = Vec::new(); // (path, snapshot_id, sequence_number, [(data_file_path, position)]) + let mut delete_files: Vec<(String, i64, i64, Vec<(String, i64)>, u64)> = Vec::new(); // (path, snapshot_id, sequence_number, [(data_file_path, position)], file_size) for (snapshot_idx, operation) in operations.iter().enumerate() { let snapshot_id = (snapshot_idx + 1) as i64; @@ -324,7 +340,7 @@ impl IncrementalTestFixture { .content(DataContentType::Data) .file_path(data_file.path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(1024) + .file_size_in_bytes(data_file.file_size) .record_count(data_file.n_values.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -339,7 +355,8 @@ impl IncrementalTestFixture { // Add new data if not empty if !n_values.is_empty() { let data_file_path = format!("{}/data/{}", &self.table_location, file_name); - self.write_parquet_file(&data_file_path, &n_values, &data_values) + let file_size = self + .write_parquet_file(&data_file_path, &n_values, &data_values) .await; data_writer @@ -352,7 +369,7 @@ impl IncrementalTestFixture { .content(DataContentType::Data) .file_path(data_file_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(1024) + .file_size_in_bytes(file_size) .record_count(n_values.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -369,6 +386,8 @@ impl IncrementalTestFixture { snapshot_id, sequence_number, n_values, + data_values, + file_size, }); } @@ -386,12 +405,13 @@ impl IncrementalTestFixture { ) .build_v2_deletes(); - for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + for (delete_path, del_snapshot_id, del_sequence_number, _, del_file_size) in + &delete_files { let delete_count = delete_files .iter() - .filter(|(p, _, _, _)| p == delete_path) - .map(|(_, _, _, deletes)| deletes.len()) + .filter(|(p, _, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes, _)| deletes.len()) .sum::(); delete_writer @@ -407,7 +427,7 @@ impl IncrementalTestFixture { .content(DataContentType::PositionDeletes) .file_path(delete_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(512) + .file_size_in_bytes(*del_file_size) .record_count(delete_count as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -477,7 +497,7 @@ impl IncrementalTestFixture { .content(DataContentType::Data) .file_path(data_file.path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(1024) + .file_size_in_bytes(data_file.file_size) .record_count(data_file.n_values.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -502,11 +522,13 @@ impl IncrementalTestFixture { .build_v2_deletes(); // Add existing delete files - for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files { + for (delete_path, del_snapshot_id, del_sequence_number, _, del_file_size) in + &delete_files + { let delete_count = delete_files .iter() - .filter(|(p, _, _, _)| p == delete_path) - .map(|(_, _, _, deletes)| deletes.len()) + .filter(|(p, _, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes, _)| deletes.len()) .sum::(); delete_writer @@ -522,7 +544,7 @@ impl IncrementalTestFixture { .content(DataContentType::PositionDeletes) .file_path(delete_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(512) + .file_size_in_bytes(*del_file_size) .record_count(delete_count as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -542,12 +564,13 @@ impl IncrementalTestFixture { snapshot_id, Uuid::new_v4() ); - self.write_positional_delete_file( - &delete_file_path, - &data_file_path, - &positions, - ) - .await; + let delete_file_size = self + .write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; delete_writer .add_entry( @@ -559,7 +582,7 @@ impl IncrementalTestFixture { .content(DataContentType::PositionDeletes) .file_path(delete_file_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(512) + .file_size_in_bytes(delete_file_size) .record_count(positions.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -579,6 +602,7 @@ impl IncrementalTestFixture { .into_iter() .map(|pos| (data_file_path.clone(), pos)) .collect(), + delete_file_size, )); } @@ -642,7 +666,7 @@ impl IncrementalTestFixture { .content(DataContentType::Data) .file_path(data_file.path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(1024) + .file_size_in_bytes(data_file.file_size) .record_count(data_file.n_values.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -667,7 +691,7 @@ impl IncrementalTestFixture { .content(DataContentType::Data) .file_path(data_file.path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(1024) + .file_size_in_bytes(data_file.file_size) .record_count(data_file.n_values.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -687,7 +711,8 @@ impl IncrementalTestFixture { rows.iter().map(|(_, d)| d.clone()).collect(); let data_file_path = format!("{}/data/{}", &self.table_location, file_name); - self.write_parquet_file(&data_file_path, &n_values, &data_values) + let file_size = self + .write_parquet_file(&data_file_path, &n_values, &data_values) .await; data_writer @@ -700,7 +725,7 @@ impl IncrementalTestFixture { .content(DataContentType::Data) .file_path(data_file_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(1024) + .file_size_in_bytes(file_size) .record_count(n_values.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -717,6 +742,8 @@ impl IncrementalTestFixture { snapshot_id, sequence_number, n_values, + data_values, + file_size, }); } @@ -739,12 +766,13 @@ impl IncrementalTestFixture { .build_v2_deletes(); // Add existing delete files - for (delete_path, del_snapshot_id, del_sequence_number, _) in &delete_files + for (delete_path, del_snapshot_id, del_sequence_number, _, del_file_size) in + &delete_files { let delete_count = delete_files .iter() - .filter(|(p, _, _, _)| p == delete_path) - .map(|(_, _, _, deletes)| deletes.len()) + .filter(|(p, _, _, _, _)| p == delete_path) + .map(|(_, _, _, deletes, _)| deletes.len()) .sum::(); delete_writer @@ -760,7 +788,7 @@ impl IncrementalTestFixture { .content(DataContentType::PositionDeletes) .file_path(delete_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(512) + .file_size_in_bytes(*del_file_size) .record_count(delete_count as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -792,12 +820,13 @@ impl IncrementalTestFixture { snapshot_id, Uuid::new_v4() ); - self.write_positional_delete_file( - &delete_file_path, - &data_file_path, - &positions, - ) - .await; + let delete_file_size = self + .write_positional_delete_file( + &delete_file_path, + &data_file_path, + &positions, + ) + .await; delete_writer .add_entry( @@ -809,7 +838,7 @@ impl IncrementalTestFixture { .content(DataContentType::PositionDeletes) .file_path(delete_file_path.clone()) .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(512) + .file_size_in_bytes(delete_file_size) .record_count(positions.len() as u64) .partition(empty_partition.clone()) .key_metadata(None) @@ -829,6 +858,7 @@ impl IncrementalTestFixture { .into_iter() .map(|pos| (data_file_path.clone(), pos)) .collect(), + delete_file_size, )); } } @@ -855,11 +885,227 @@ impl IncrementalTestFixture { manifest_list_write.close().await.unwrap(); } + + Operation::Replace(files_to_compact, target_file) => { + // Replace operation: compact existing files into a new file + // The logical content doesn't change, only the physical representation + self.handle_replace_operation( + files_to_compact, + target_file, + &mut data_files, + &delete_files, + current_schema.clone(), + &partition_spec, + snapshot_id, + sequence_number, + parent_snapshot_id, + ) + .await + .unwrap(); + } } } } - async fn write_parquet_file(&self, path: &str, n_values: &[i32], data_values: &[String]) { + #[allow(clippy::too_many_arguments, clippy::type_complexity)] + async fn handle_replace_operation( + &mut self, + files_to_compact: &[String], + target_file: &str, + data_files: &mut Vec, + delete_files: &[(String, i64, i64, Vec<(String, i64)>, u64)], + current_schema: SchemaRef, + partition_spec: &PartitionSpec, + snapshot_id: i64, + sequence_number: i64, + parent_snapshot_id: Option, + ) -> Result<(), Box> { + let empty_partition = Struct::empty(); + + // Create data manifest + let mut data_writer = ManifestWriterBuilder::new( + self.next_manifest_file(), + Some(snapshot_id), + None, + current_schema, + partition_spec.clone(), + ) + .build_v2_data(); + + // Determine which files are being compacted + let files_to_compact_set: std::collections::HashSet = files_to_compact + .iter() + .map(|f| format!("{}/data/{}", &self.table_location, f)) + .collect(); + + // Build a set of deleted positions for each file being compacted + let mut deleted_positions: std::collections::HashMap< + String, + std::collections::HashSet, + > = std::collections::HashMap::new(); + for (_, _, _, delete_records, _) in delete_files { + for (file_path, position) in delete_records { + deleted_positions + .entry(file_path.clone()) + .or_default() + .insert(*position); + } + } + + // Track the data being compacted + let mut compacted_data: Vec<(i32, String)> = Vec::new(); + let mut compacted_record_count: u64 = 0; + + // Add existing data files (mark compacted ones as DELETED, others as EXISTING) + for data_file in data_files.iter() { + if files_to_compact_set.contains(&data_file.path) { + // Mark file as deleted (being compacted away) + data_writer + .add_delete_entry( + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(data_file.file_size) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Collect data from compacted files, filtering out deleted records + let file_deleted_positions = deleted_positions.get(&data_file.path); + for (position, (n, d)) in data_file + .n_values + .iter() + .zip(data_file.data_values.iter()) + .enumerate() + { + // Skip this record if it was deleted via positional delete + if let Some(deleted) = file_deleted_positions { + if deleted.contains(&(position as i64)) { + continue; + } + } + compacted_data.push((*n, d.clone())); + } + compacted_record_count += data_file.n_values.len() as u64; + } else { + // Keep existing file + data_writer + .add_existing_entry( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(data_file.snapshot_id) + .sequence_number(data_file.sequence_number) + .file_sequence_number(data_file.sequence_number) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(data_file.path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(data_file.file_size) + .record_count(data_file.n_values.len() as u64) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + } + } + + // Create the compacted file with the collected data + if !compacted_data.is_empty() { + let compacted_n_values: Vec = compacted_data.iter().map(|(n, _)| *n).collect(); + let compacted_data_values: Vec = + compacted_data.iter().map(|(_, d)| d.clone()).collect(); + let compacted_file_path = format!("{}/data/{}", &self.table_location, target_file); + + let file_size = self + .write_parquet_file( + &compacted_file_path, + &compacted_n_values, + &compacted_data_values, + ) + .await; + + data_writer + .add_entry( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path(compacted_file_path.clone()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(file_size) + .record_count(compacted_record_count) + .partition(empty_partition.clone()) + .key_metadata(None) + .build() + .unwrap(), + ) + .build(), + ) + .unwrap(); + + // Update data_files tracking: remove compacted, add new + data_files.retain(|df| !files_to_compact_set.contains(&df.path)); + data_files.push(DataFileInfo { + path: compacted_file_path, + snapshot_id, + sequence_number, + n_values: compacted_n_values, + data_values: compacted_data_values, + file_size, + }); + } + + let data_manifest = data_writer.write_manifest_file().await.unwrap(); + + // Write manifest list (no delete manifests for Replace) + let mut manifest_list_write = ManifestListWriter::v2( + self.table + .file_io() + .new_output(format!( + "{}/metadata/snap-{}-manifest-list.avro", + self.table_location, snapshot_id + )) + .unwrap(), + snapshot_id, + parent_snapshot_id, + sequence_number, + ); + manifest_list_write + .add_manifests(vec![data_manifest].into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + + Ok(()) + } + + async fn write_parquet_file( + &self, + path: &str, + n_values: &[i32], + data_values: &[String], + ) -> u64 { let schema = { let fields = vec![ arrow_schema::Field::new("n", arrow_schema::DataType::Int32, false).with_metadata( @@ -887,6 +1133,9 @@ impl IncrementalTestFixture { let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); writer.write(&batch).unwrap(); writer.close().unwrap(); + + // Return the actual file size + fs::metadata(path).unwrap().len() } async fn write_positional_delete_file( @@ -894,7 +1143,7 @@ impl IncrementalTestFixture { path: &str, data_file_path: &str, positions: &[i64], - ) { + ) -> u64 { let schema = { let fields = vec![ arrow_schema::Field::new("file_path", arrow_schema::DataType::Utf8, false) @@ -925,6 +1174,9 @@ impl IncrementalTestFixture { let mut writer = ArrowWriter::try_new(file, schema, Some(props)).unwrap(); writer.write(&batch).unwrap(); writer.close().unwrap(); + + // Return the actual file size + fs::metadata(path).unwrap().len() } /// Verify incremental scan results. @@ -1743,6 +1995,111 @@ async fn test_incremental_scan_positional_deletes_then_file_delete() { .await; } +#[tokio::test] +async fn test_incremental_scan_with_replace_operation() { + // This test verifies the Replace operation semantics: + // - A Replace operation compacts multiple files into one + // - The logical table content does NOT change (same data) + // - But physically, files are reorganized: old files are DELETED, new file is ADDED + // - For incremental scans, we report these physical changes (additions and deletions) + // even though the logical data is identical + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add file-a with 3 rows + Operation::Add( + vec![ + (1, "a".to_string()), + (2, "b".to_string()), + (3, "c".to_string()), + ], + "file-a.parquet".to_string(), + ), + // Snapshot 3: Add file-b with 2 rows + Operation::Add( + vec![(10, "x".to_string()), (20, "y".to_string())], + "file-b.parquet".to_string(), + ), + // Snapshot 4: Replace (compact file-a and file-b into file-ab-compact) + // This will delete file-a and file-b, and add file-ab-compact with the same data + Operation::Replace( + vec!["file-a.parquet".to_string(), "file-b.parquet".to_string()], + "file-ab-compact.parquet".to_string(), + ), + ]) + .await; + + // Verify we have 4 snapshots + let mut snapshots = fixture.table.metadata().snapshots().collect::>(); + snapshots.sort_by_key(|s| s.snapshot_id()); + assert_eq!(snapshots.len(), 4); + + // Verify snapshot operations + assert_eq!( + snapshots[3].summary().operation, + crate::spec::Operation::Replace, + "Snapshot 4 should be a Replace operation" + ); + + let file_a_path = format!("{}/data/file-a.parquet", fixture.table_location); + let file_b_path = format!("{}/data/file-b.parquet", fixture.table_location); + + // Test 1: Incremental scan from snapshot 3 to 4 (ONLY Replace operation) + // Physical changes: file-a and file-b are deleted, file-ab-compact is added + // But the data is the same, so: + // - Additions: all 5 rows from file-ab-compact (1,2,3,10,20) + // - Deletions: all 5 rows from file-a and file-b (positions sorted by (position, file_path)) + fixture + .verify_incremental_scan( + 3, + 4, + vec![(1, "a"), (2, "b"), (3, "c"), (10, "x"), (20, "y")], + vec![ + (0, &file_a_path), + (0, &file_b_path), + (1, &file_a_path), + (1, &file_b_path), + (2, &file_a_path), + ], + ) + .await; + + // Test 2: Incremental scan from snapshot 1 to 4 (includes Appends and Replace) + // Snapshot 2 adds file-a (1,2,3) + // Snapshot 3 adds file-b (10,20) + // Snapshot 4 replaces both files (deletes file-a, file-b; adds file-ab-compact) + // Since file-a and file-b are added in the scan range and then deleted in the Replace: + // - The additions of file-a and file-b cancel with their deletions + // - Net result: only file-ab-compact is added (1,2,3,10,20) + // - Deletions: empty (file-a and file-b deletions cancelled by their additions) + fixture + .verify_incremental_scan( + 1, + 4, + vec![(1, "a"), (2, "b"), (3, "c"), (10, "x"), (20, "y")], + vec![], + ) + .await; + + // Test 3: Incremental scan from snapshot 2 to 4 + // Starting from snapshot 2: file-a (1,2,3) exists (added before scan start) + // Snapshot 3: adds file-b (10,20) + // Snapshot 4: replaces both files with file-ab-compact + // Since file-a was added before the scan started, its deletion is not cancelled + // But file-b is added and deleted within the scan, so cancels out + // Net result: + // - Additions: file-ab-compact (1,2,3,10,20) + // - Deletions: file-a (0,1,2) since it existed at scan start + fixture + .verify_incremental_scan( + 2, + 4, + vec![(1, "a"), (2, "b"), (3, "c"), (10, "x"), (20, "y")], + vec![(0, &file_a_path), (1, &file_a_path), (2, &file_a_path)], + ) + .await; +} + #[tokio::test] async fn test_incremental_scan_with_deleted_files_cancellation() { // This test verifies that incremental scans properly handle file deletions with cancellation logic: @@ -1848,3 +2205,166 @@ async fn test_incremental_scan_with_deleted_files_cancellation() { ) .await; } + +#[tokio::test] +async fn test_incremental_scan_with_replace_and_positional_deletes() { + // This test verifies Replace operations with positional deletes before and after the replace. + // + // Test scenario: + // Snapshot 1: Empty starting point + // Snapshot 2: Add file-a with 5 records (1,2,3,4,5) + // Snapshot 3: Add file-b with 3 records (10,11,12) + // Snapshot 4: Delete record at position 1 in file-a (delete "2") + // Snapshot 5: Replace - compact file-a and file-b into file-ab-compact + // (containing records 1,3,4,5 from file-a and 10,11,12 from file-b after the previous delete) + // Snapshot 6: Delete record at position 2 in file-ab-compact (delete "4") + + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1: Empty starting point + Operation::Add(vec![], "empty.parquet".to_string()), + // Snapshot 2: Add file-a with 5 rows + Operation::Add( + vec![ + (1, "1".to_string()), + (2, "2".to_string()), + (3, "3".to_string()), + (4, "4".to_string()), + (5, "5".to_string()), + ], + "file-a.parquet".to_string(), + ), + // Snapshot 3: Add file-b with 3 rows + Operation::Add( + vec![ + (10, "10".to_string()), + (11, "11".to_string()), + (12, "12".to_string()), + ], + "file-b.parquet".to_string(), + ), + // Snapshot 4: Delete position 1 (record "2") from file-a + Operation::Delete(vec![(1, "file-a.parquet".to_string())]), + // Snapshot 5: Replace - compact file-a and file-b into file-ab-compact + Operation::Replace( + vec!["file-a.parquet".to_string(), "file-b.parquet".to_string()], + "file-ab-compact.parquet".to_string(), + ), + // Snapshot 6: Delete position 2 (record "4") from file-ab-compact + Operation::Delete(vec![(2, "file-ab-compact.parquet".to_string())]), + ]) + .await; + + // Test 1: Full scan from snapshot 1 to 6 + // Snapshot 2: Add file-a (1,2,3,4,5) + // Snapshot 3: Add file-b (10,11,12) + // Snapshot 4: Delete position 1 from file-a (record "2" deleted) + // Snapshot 5: Replace both files with file-ab-compact (compacts to 1,3,4,5,10,11,12, filtering out position 1) + // Snapshot 6: Delete position 2 from file-ab-compact (record "4" deleted) + // Net result: + // - Additions: records from compacted file with deleted positions filtered (1,3,5,10,11,12) + // Note: position 1 from file-a (record "2") is filtered during compaction + // position 2 from compacted file (record "4") is never added since it's deleted in snapshot 6 + // - Deletions: empty (deletes from Replace are absorbed into the additions of the compacted file) + fixture + .verify_incremental_scan( + 1, + 6, + vec![ + (1, "1"), + (3, "3"), + (5, "5"), + (10, "10"), + (11, "11"), + (12, "12"), + ], + vec![], + ) + .await; + + // Test 2: Scan from snapshot 3 to 6 (after file-b added, through replace and final delete) + // Snapshot 3: Starting point with file-a and file-b (both exist in starting snapshot) + // Snapshot 4: Delete position 1 from file-a (record "2" deleted) + // Snapshot 5: Replace both files with file-ab-compact (filters out position 1 from file-a) + // Snapshot 6: Delete position 2 from file-ab-compact (record "4" deleted) + // Net result: + // - Additions: compacted file records with filtered positions (1,3,5,10,11,12) + // - Deletions: All positions from file-a (0-4) and file-b (0-2) because these files + // existed in the starting snapshot (3) and are deleted/replaced in snapshot 5. + // Deletes are sorted by (position, file_path) tuple ordering + let file_a_path = format!("{}/data/file-a.parquet", fixture.table_location); + let file_b_path = format!("{}/data/file-b.parquet", fixture.table_location); + let test2_deletes = vec![ + (0, file_a_path.as_str()), + (0, file_b_path.as_str()), + (1, file_a_path.as_str()), + (1, file_b_path.as_str()), + (2, file_a_path.as_str()), + (2, file_b_path.as_str()), + (3, file_a_path.as_str()), + (4, file_a_path.as_str()), + ]; + fixture + .verify_incremental_scan( + 3, + 6, + vec![ + (1, "1"), + (3, "3"), + (5, "5"), + (10, "10"), + (11, "11"), + (12, "12"), + ], + test2_deletes, + ) + .await; + + // Test 3: Scan from snapshot 4 to 6 (after first delete, through replace and final delete) + // Snapshot 4: Starting point - file-a already has position 1 deleted (record "2") + // Snapshot 5: Replace both files with file-ab-compact (filters out position 1) + // Snapshot 6: Delete position 2 from file-ab-compact (record "4" deleted) + // Net result: + // - Additions: compacted records with position 1 filtered from file-a (1,3,5,10,11,12) + // - Deletions: All positions from file-a (0-4) and file-b (0-2) because these files + // existed in the starting snapshot (4) and are deleted/replaced in snapshot 5. + // Sorted by (position, file_path) tuple ordering + let file_a_path = format!("{}/data/file-a.parquet", fixture.table_location); + let file_b_path = format!("{}/data/file-b.parquet", fixture.table_location); + let test3_deletes = vec![ + (0, file_a_path.as_str()), + (0, file_b_path.as_str()), + (1, file_a_path.as_str()), + (1, file_b_path.as_str()), + (2, file_a_path.as_str()), + (2, file_b_path.as_str()), + (3, file_a_path.as_str()), + (4, file_a_path.as_str()), + ]; + fixture + .verify_incremental_scan( + 4, + 6, + vec![ + (1, "1"), + (3, "3"), + (5, "5"), + (10, "10"), + (11, "11"), + (12, "12"), + ], + test3_deletes, + ) + .await; + + // Test 4: Scan from snapshot 5 to 6 (after replace, only sees the final delete) + // Snapshot 5: Starting point - file-ab-compact is newly created with records (1,3,4,5,10,11,12) + // Snapshot 6: Delete position 2 from file-ab-compact (record "4" deleted) + // Net result: + // - Additions: empty (file-ab-compact was added in snapshot 5, not in this scan range) + // - Deletions: position 2 from file-ab-compact (the positional delete in snapshot 6) + let file_ab_path = format!("{}/data/file-ab-compact.parquet", fixture.table_location); + let test4_deletes = vec![(2, file_ab_path.as_str())]; + fixture + .verify_incremental_scan(5, 6, vec![], test4_deletes) + .await; +} From 0de5b0387e7bb0297508c9f65461fef9224af1c6 Mon Sep 17 00:00:00 2001 From: Gerald Berger <59661379+gbrgr@users.noreply.github.com> Date: Fri, 14 Nov 2025 14:07:15 +0100 Subject: [PATCH 6/7] feat(incremental): Make from and to in incremental scan optional (#9) * Make from and to in incremental scan optional * Make from inclusive * Fix merge conflicts * . * Format --- crates/iceberg/src/arrow/incremental.rs | 4 +- .../src/arrow/record_batch_transformer.rs | 3 +- crates/iceberg/src/scan/incremental/mod.rs | 77 +++++++++---- crates/iceberg/src/scan/incremental/tests.rs | 103 ++++++++++++++++-- crates/iceberg/src/table.rs | 8 +- crates/iceberg/src/util/snapshot.rs | 14 +++ 6 files changed, 177 insertions(+), 32 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index 0eeebfd5ef..b83335a628 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -26,7 +26,7 @@ use futures::stream::select; use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; -use crate::arrow::record_batch_transformer::RecordBatchTransformer; +use crate::arrow::record_batch_transformer::RecordBatchTransformerBuilder; use crate::arrow::{ ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_COL_NAME_POS, RESERVED_FIELD_ID_FILE_PATH, RESERVED_FIELD_ID_POS, StreamsInto, @@ -257,7 +257,7 @@ async fn process_incremental_append_task( // that come back from the file, such as type promotion, default column insertion // and column re-ordering let mut record_batch_transformer = - RecordBatchTransformer::build(task.schema_ref(), &task.base.project_field_ids); + RecordBatchTransformerBuilder::new(task.schema_ref(), &task.base.project_field_ids).build(); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 07ec43918f..a20adb6a5a 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -792,7 +792,8 @@ mod test { let projected_iceberg_field_ids = [1, 2, 3]; let mut transformer = - RecordBatchTransformer::build(snapshot_schema, &projected_iceberg_field_ids); + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_iceberg_field_ids) + .build(); let file_schema = Arc::new(ArrowSchema::new(vec![ simple_field("id", DataType::Int32, false, "1"), diff --git a/crates/iceberg/src/scan/incremental/mod.rs b/crates/iceberg/src/scan/incremental/mod.rs index af011dfc16..e074f49c98 100644 --- a/crates/iceberg/src/scan/incremental/mod.rs +++ b/crates/iceberg/src/scan/incremental/mod.rs @@ -53,8 +53,10 @@ pub struct IncrementalTableScanBuilder<'a> { table: &'a Table, // Defaults to `None`, which means all columns. column_names: Option>, - from_snapshot_id: i64, - to_snapshot_id: i64, + // None means scan from the first snapshot (inclusive) + from_snapshot_id: Option, + // None means scan to the current/last snapshot + to_snapshot_id: Option, batch_size: Option, concurrency_limit_data_files: usize, concurrency_limit_manifest_entries: usize, @@ -62,7 +64,11 @@ pub struct IncrementalTableScanBuilder<'a> { } impl<'a> IncrementalTableScanBuilder<'a> { - pub(crate) fn new(table: &'a Table, from_snapshot_id: i64, to_snapshot_id: i64) -> Self { + pub(crate) fn new( + table: &'a Table, + from_snapshot_id: Option, + to_snapshot_id: Option, + ) -> Self { let num_cpus = available_parallelism().get(); Self { table, @@ -107,13 +113,13 @@ impl<'a> IncrementalTableScanBuilder<'a> { /// Set the `from_snapshot_id` for the incremental scan. pub fn from_snapshot_id(mut self, from_snapshot_id: i64) -> Self { - self.from_snapshot_id = from_snapshot_id; + self.from_snapshot_id = Some(from_snapshot_id); self } /// Set the `to_snapshot_id` for the incremental scan. pub fn to_snapshot_id(mut self, to_snapshot_id: i64) -> Self { - self.to_snapshot_id = to_snapshot_id; + self.to_snapshot_id = Some(to_snapshot_id); self } @@ -137,37 +143,70 @@ impl<'a> IncrementalTableScanBuilder<'a> { /// Build the incremental table scan. pub fn build(self) -> Result { - let snapshot_from: Arc = self - .table - .metadata() - .snapshot_by_id(self.from_snapshot_id) + let metadata = self.table.metadata(); + + // Resolve to_snapshot_id: if None, use the current (latest) snapshot + let to_snapshot_id = if let Some(id) = self.to_snapshot_id { + id + } else { + metadata + .current_snapshot() + .ok_or_else(|| Error::new(ErrorKind::DataInvalid, "No current snapshot found"))? + .snapshot_id() + }; + + let snapshot_to: Arc = metadata + .snapshot_by_id(to_snapshot_id) .ok_or_else(|| { Error::new( ErrorKind::DataInvalid, - format!("Snapshot with id {} not found", self.from_snapshot_id), + format!("Snapshot with id {} not found", to_snapshot_id), ) })? .clone(); - let snapshot_to: Arc = self - .table - .metadata() - .snapshot_by_id(self.to_snapshot_id) - .ok_or_else(|| { + // Determine oldest_snapshot_id for ancestors_between + // If from was None, include root snapshot by passing None to ancestors_between + // If from was Some(id), exclude that snapshot by passing Some(id) + let oldest_snapshot_id = if let Some(from_id) = self.from_snapshot_id { + // Validate the from snapshot exists + let _ = metadata.snapshot_by_id(from_id).ok_or_else(|| { Error::new( ErrorKind::DataInvalid, - format!("Snapshot with id {} not found", self.to_snapshot_id), + format!("Snapshot with id {} not found", from_id), ) - })? - .clone(); + })?; + Some(from_id) + } else { + None + }; let snapshots = ancestors_between( &self.table.metadata_ref(), snapshot_to.snapshot_id(), - Some(snapshot_from.snapshot_id()), + oldest_snapshot_id, ) .collect_vec(); + // Get the from_snapshot for the plan context + // This is either the user-specified snapshot or the root snapshot + let from_snapshot_id = oldest_snapshot_id.unwrap_or_else(|| { + snapshots + .last() + .map(|s| s.snapshot_id()) + .expect("snapshots should not be empty") + }); + + let snapshot_from: Arc = metadata + .snapshot_by_id(from_snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", from_snapshot_id), + ) + })? + .clone(); + if !snapshots.is_empty() { assert_eq!( snapshots.first().map(|s| s.snapshot_id()), diff --git a/crates/iceberg/src/scan/incremental/tests.rs b/crates/iceberg/src/scan/incremental/tests.rs index 0ae836b53f..276ab9ef12 100644 --- a/crates/iceberg/src/scan/incremental/tests.rs +++ b/crates/iceberg/src/scan/incremental/tests.rs @@ -1195,7 +1195,7 @@ impl IncrementalTestFixture { let incremental_scan = self .table - .incremental_scan(from_snapshot_id, to_snapshot_id) + .incremental_scan(Some(from_snapshot_id), Some(to_snapshot_id)) .build() .unwrap(); @@ -1638,9 +1638,10 @@ async fn test_incremental_scan_builder_options() { .await; // Test 1: Column projection - select only the "n" column + // Scan from root (None) to last (None) let scan = fixture .table - .incremental_scan(1, 4) + .incremental_scan(None, None) .select(vec!["n"]) .build() .unwrap(); @@ -1672,9 +1673,10 @@ async fn test_incremental_scan_builder_options() { } // Test 2: Column projection - select only the "data" column + // Scan from root (None) to last (None) let scan = fixture .table - .incremental_scan(1, 4) + .incremental_scan(None, None) .select(vec!["data"]) .build() .unwrap(); @@ -1702,9 +1704,10 @@ async fn test_incremental_scan_builder_options() { } // Test 3: Select both columns explicitly + // Scan from root (None) to last (None) let scan = fixture .table - .incremental_scan(1, 4) + .incremental_scan(None, None) .select(vec!["n", "data"]) .build() .unwrap(); @@ -1729,9 +1732,10 @@ async fn test_incremental_scan_builder_options() { } // Test 4: Batch size configuration + // Scan from root (None) to snapshot 2 let scan = fixture .table - .incremental_scan(1, 2) + .incremental_scan(None, Some(2)) .with_batch_size(Some(3)) // Small batch size to test batching .build() .unwrap(); @@ -1753,9 +1757,10 @@ async fn test_incremental_scan_builder_options() { } // Test 5: Combining column projection and batch size + // Scan from root (None) to last (None) let scan = fixture .table - .incremental_scan(1, 4) + .incremental_scan(None, None) .select(vec!["n"]) .with_batch_size(Some(4)) .build() @@ -1775,9 +1780,10 @@ async fn test_incremental_scan_builder_options() { } // Test 6: Verify actual data with column projection + // Scan from root (None) to snapshot 2 let scan = fixture .table - .incremental_scan(1, 2) + .incremental_scan(None, Some(2)) .select(vec!["n"]) .build() .unwrap(); @@ -1813,7 +1819,11 @@ async fn test_incremental_scan_builder_options() { } // Test 7: Delete batches always have the same schema. - let scan = fixture.table.incremental_scan(2, 3).build().unwrap(); + let scan = fixture + .table + .incremental_scan(Some(2), Some(3)) + .build() + .unwrap(); let stream = scan.to_arrow().await.unwrap(); let batches: Vec<_> = stream.try_collect().await.unwrap(); @@ -2368,3 +2378,80 @@ async fn test_incremental_scan_with_replace_and_positional_deletes() { .verify_incremental_scan(5, 6, vec![], test4_deletes) .await; } + +#[tokio::test] +async fn test_incremental_scan_includes_root_when_from_is_none() { + // Test that when from=None, the root snapshot is INCLUDED in the scan + // (not excluded like when an explicit from snapshot is specified) + let fixture = IncrementalTestFixture::new(vec![ + // Snapshot 1 (root): Add initial data - this should be INCLUDED when from=None + Operation::Add( + vec![(1, "one".to_string()), (2, "two".to_string())], + "root-data.parquet".to_string(), + ), + // Snapshot 2: Add more data + Operation::Add( + vec![(10, "ten".to_string()), (20, "twenty".to_string())], + "second-data.parquet".to_string(), + ), + // Snapshot 3: Add final data + Operation::Add( + vec![(100, "hundred".to_string())], + "third-data.parquet".to_string(), + ), + ]) + .await; + + // Test 1: Scan with explicit from=1 should EXCLUDE snapshot 1 (normal behavior) + fixture + .verify_incremental_scan( + 1, // from snapshot 1 - EXCLUDED + 3, // to snapshot 3 + vec![(10, "ten"), (20, "twenty"), (100, "hundred")], // Only snapshots 2 and 3 + vec![], + ) + .await; + + // Test 2: Scan using table.incremental_scan(None, None) API + // This should INCLUDE the root snapshot + let scan = fixture.table.incremental_scan(None, None).build().unwrap(); + let stream = scan.to_arrow().await.unwrap(); + let batches: Vec<_> = stream.try_collect().await.unwrap(); + + // Collect all appended data + let append_batches: Vec<_> = batches + .iter() + .filter(|(t, _)| *t == crate::arrow::IncrementalBatchType::Append) + .map(|(_, b)| b.clone()) + .collect(); + + // Extract n and data values + use arrow_array::cast::AsArray; + let mut results = Vec::new(); + for batch in append_batches { + let n_array = batch + .column_by_name("n") + .unwrap() + .as_primitive::(); + let data_array = batch.column_by_name("data").unwrap().as_string::(); + for i in 0..batch.num_rows() { + results.push((n_array.value(i), data_array.value(i).to_string())); + } + } + + // Sort for consistent comparison + results.sort(); + + // Should include ALL data including root snapshot + assert_eq!( + results, + vec![ + (1, "one".to_string()), + (2, "two".to_string()), + (10, "ten".to_string()), + (20, "twenty".to_string()), + (100, "hundred".to_string()), + ], + "Scan with from=None should include root snapshot data" + ); +} diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index 7c8825707a..3ea6f61476 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -226,10 +226,14 @@ impl Table { } /// Creates an incremental table scan between two snapshots. + /// + /// # Arguments + /// * `from_snapshot_id` - Starting snapshot ID. If None, scans from the root (oldest) snapshot. + /// * `to_snapshot_id` - Ending snapshot ID. If None, scans to the current (latest) snapshot. pub fn incremental_scan( &self, - from_snapshot_id: i64, - to_snapshot_id: i64, + from_snapshot_id: Option, + to_snapshot_id: Option, ) -> IncrementalTableScanBuilder<'_> { IncrementalTableScanBuilder::new(self, from_snapshot_id, to_snapshot_id) } diff --git a/crates/iceberg/src/util/snapshot.rs b/crates/iceberg/src/util/snapshot.rs index 513e23c110..d73a255f5e 100644 --- a/crates/iceberg/src/util/snapshot.rs +++ b/crates/iceberg/src/util/snapshot.rs @@ -72,3 +72,17 @@ pub fn ancestors_between( .take_while(move |snapshot| snapshot.snapshot_id() != oldest_snapshot_id), ) } + +/// Returns the root (oldest/first) snapshot ID in the table's snapshot lineage. +/// +/// # Arguments +/// * `table_metadata` - The table metadata +/// +/// # Returns +/// The snapshot ID of the root snapshot, or None if there are no snapshots +pub fn root_snapshot_id(table_metadata: &TableMetadataRef) -> Option { + let current = table_metadata.current_snapshot()?; + ancestors_of(table_metadata, current.snapshot_id()) + .last() + .map(|s| s.snapshot_id()) +} From 4979a205f9175afe50a5638e6059986df058f4a9 Mon Sep 17 00:00:00 2001 From: Vukasin Stefanovic Date: Tue, 18 Nov 2025 15:38:55 +0100 Subject: [PATCH 7/7] True parallelism across full and incremental scan --- crates/iceberg/src/arrow/incremental.rs | 105 +++++++++++++----------- crates/iceberg/src/arrow/reader.rs | 94 +++++++++++++++------ 2 files changed, 128 insertions(+), 71 deletions(-) diff --git a/crates/iceberg/src/arrow/incremental.rs b/crates/iceberg/src/arrow/incremental.rs index b83335a628..ce1b6add5c 100644 --- a/crates/iceberg/src/arrow/incremental.rs +++ b/crates/iceberg/src/arrow/incremental.rs @@ -118,22 +118,25 @@ impl StreamsInto .await; match record_batch_stream { - Ok(mut stream) => { - while let Some(batch) = stream.next().await { - let result = appends_tx - .send(batch.map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "failed to read appended record batch", - ) - .with_source(e) - })) - .await; - - if result.is_err() { - break; - } - } + Ok(stream) => { + // Process batches with parallelism for CPU-heavy operations + let _: Vec<_> = stream + .map(|batch_result| { + let mut appends_tx = appends_tx.clone(); + spawn(async move { + let result = batch_result.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read appended record batch", + ) + .with_source(e) + }); + let _ = appends_tx.send(result).await; + }) + }) + .buffer_unordered(concurrency_limit_data_files) + .collect() + .await; } Err(e) => { let _ = appends_tx.send(Err(e)).await; @@ -154,22 +157,25 @@ impl StreamsInto ); match record_batch_stream { - Ok(mut stream) => { - while let Some(batch) = stream.next().await { - let result = deletes_tx - .send(batch.map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "failed to read deleted file record batch", - ) - .with_source(e) - })) - .await; - - if result.is_err() { - break; - } - } + Ok(stream) => { + // Process batches with parallelism for CPU-heavy operations + let _: Vec<_> = stream + .map(|batch_result| { + let mut deletes_tx = deletes_tx.clone(); + spawn(async move { + let result = batch_result.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read deleted file record batch", + ) + .with_source(e) + }); + let _ = deletes_tx.send(result).await; + }) + }) + .buffer_unordered(concurrency_limit_data_files) + .collect() + .await; } Err(e) => { let _ = deletes_tx.send(Err(e)).await; @@ -190,22 +196,25 @@ impl StreamsInto ); match record_batch_stream { - Ok(mut stream) => { - while let Some(batch) = stream.next().await { - let result = deletes_tx - .send(batch.map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "failed to read deleted record batch", - ) - .with_source(e) - })) - .await; - - if result.is_err() { - break; - } - } + Ok(stream) => { + // Process batches with parallelism for CPU-heavy operations + let _: Vec<_> = stream + .map(|batch_result| { + let mut deletes_tx = deletes_tx.clone(); + spawn(async move { + let result = batch_result.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read deleted record batch", + ) + .with_source(e) + }); + let _ = deletes_tx.send(result).await; + }) + }) + .buffer_unordered(concurrency_limit_data_files) + .collect() + .await; } Err(e) => { let _ = deletes_tx.send(Err(e)).await; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 0dc1ccf68f..f4d4b8981c 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -35,8 +35,9 @@ use arrow_schema::{ use arrow_string::like::starts_with; use bytes::Bytes; use fnv::FnvHashSet; +use futures::channel::mpsc::channel; use futures::future::BoxFuture; -use futures::{FutureExt, StreamExt, TryFutureExt, TryStreamExt, try_join}; +use futures::{FutureExt, SinkExt, StreamExt, TryFutureExt, TryStreamExt, try_join}; use parquet::arrow::arrow_reader::{ ArrowPredicateFn, ArrowReaderOptions, RowFilter, RowSelection, RowSelector, }; @@ -57,6 +58,7 @@ use crate::expr::visitors::page_index_evaluator::PageIndexEvaluator; use crate::expr::visitors::row_group_metrics_evaluator::RowGroupMetricsEvaluator; use crate::expr::{BoundPredicate, BoundReference}; use crate::io::{FileIO, FileMetadata, FileRead}; +use crate::runtime::spawn; use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskStream}; use crate::spec::{Datum, NameMapping, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; @@ -171,34 +173,80 @@ pub trait StreamsInto { impl ArrowReader { /// Take a stream of FileScanTasks and reads all the files. - /// Returns a stream of Arrow RecordBatches containing the data from the files + /// Returns a stream of Arrow RecordBatches containing the data from the files. + /// + /// This implementation provides both file-level and batch-level parallelism: + /// - Multiple files are processed in parallel (IO-heavy operations) + /// - Multiple batches are processed in parallel across all files (CPU-heavy operations) pub fn read(self, tasks: FileScanTaskStream) -> Result { - let file_io = self.file_io.clone(); + let (tx, rx) = channel(self.concurrency_limit_data_files); + + let file_io = self.file_io; let batch_size = self.batch_size; let concurrency_limit_data_files = self.concurrency_limit_data_files; let row_group_filtering_enabled = self.row_group_filtering_enabled; let row_selection_enabled = self.row_selection_enabled; + let delete_file_loader = self.delete_file_loader; + + // Outer spawn for coordination - runs the entire processing pipeline in background + spawn(async move { + let _ = tasks + .try_for_each_concurrent(concurrency_limit_data_files, |task| { + let file_io = file_io.clone(); + let delete_file_loader = delete_file_loader.clone(); + let mut tx = tx.clone(); + + async move { + // Inner spawn for IO-heavy file operations (parallel file processing) + spawn(async move { + let record_batch_stream = Self::process_file_scan_task( + task, + batch_size, + file_io, + delete_file_loader, + row_group_filtering_enabled, + row_selection_enabled, + ) + .await; + + match record_batch_stream { + Ok(stream) => { + // Process batches with parallelism for CPU-heavy operations + // Each batch gets its own spawned task, enabling true parallel processing + let _: Vec<_> = stream + .map(|batch_result| { + let mut tx = tx.clone(); + spawn(async move { + // CPU-heavy batch processing happens here in parallel + let result = batch_result.map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "failed to read record batch", + ) + .with_source(e) + }); + + let _ = tx.send(result).await; + }) + }) + .buffer_unordered(concurrency_limit_data_files) + .collect() + .await; + } + Err(e) => { + let _ = tx.send(Err(e)).await; + } + } + }) + .await; + + Ok(()) + } + }) + .await; + }); - let stream = tasks - .map_ok(move |task| { - let file_io = file_io.clone(); - - Self::process_file_scan_task( - task, - batch_size, - file_io, - self.delete_file_loader.clone(), - row_group_filtering_enabled, - row_selection_enabled, - ) - }) - .map_err(|err| { - Error::new(ErrorKind::Unexpected, "file scan task generate failed").with_source(err) - }) - .try_buffer_unordered(concurrency_limit_data_files) - .try_flatten_unordered(concurrency_limit_data_files); - - Ok(Box::pin(stream) as ArrowRecordBatchStream) + Ok(Box::pin(rx) as ArrowRecordBatchStream) } #[allow(clippy::too_many_arguments)]