From ad879460400ca1dc6c8946e913470b201f930919 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 18 Mar 2025 16:16:46 +0700 Subject: [PATCH 01/33] feat: support incremental scan between 2 snapshots (#13) --- crates/iceberg/src/scan/context.rs | 137 +++++++++++++++++++++++++++-- crates/iceberg/src/scan/mod.rs | 48 ++++++++-- 2 files changed, 173 insertions(+), 12 deletions(-) diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 3f7c29dbf4..a70b77982b 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashSet; use std::sync::Arc; use futures::channel::mpsc::Sender; use futures::{SinkExt, TryFutureExt}; +use itertools::Itertools; use crate::delete_file_index::DeleteFileIndex; use crate::expr::{Bind, BoundPredicate, Predicate}; @@ -28,11 +30,12 @@ use crate::scan::{ PartitionFilterCache, }; use crate::spec::{ - ManifestContentType, ManifestEntryRef, ManifestFile, ManifestList, SchemaRef, SnapshotRef, - TableMetadataRef, + DataContentType, ManifestContentType, ManifestEntryRef, ManifestFile, ManifestList, + ManifestStatus, Operation, SchemaRef, SnapshotRef, TableMetadataRef, }; use crate::{Error, ErrorKind, Result}; +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 { @@ -46,6 +49,10 @@ pub(crate) struct ManifestFileContext { snapshot_schema: SchemaRef, expression_evaluator_cache: Arc, delete_file_index: DeleteFileIndex, + + /// filter manifest entries. + /// Used for different kind of scans, e.g., only scan newly added files without delete files. + filter_fn: Option>, } /// Wraps a [`ManifestEntryRef`] alongside the objects that are needed @@ -74,12 +81,13 @@ 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(), @@ -149,6 +157,11 @@ pub(crate) struct PlanContext { pub partition_filter_cache: Arc, pub manifest_evaluator_cache: Arc, pub expression_evaluator_cache: Arc, + + // for incremental scan. + // If `to_snapshot_id` is set, it means incremental scan. `from_snapshot_id` can be `None`. + pub from_snapshot_id: Option, + pub to_snapshot_id: Option, } impl PlanContext { @@ -180,18 +193,71 @@ impl PlanContext { Ok(partition_filter) } - pub(crate) fn build_manifest_file_contexts( + pub(crate) async fn build_manifest_file_contexts( &self, manifest_list: Arc, tx_data: Sender, delete_file_idx: DeleteFileIndex, delete_file_tx: Sender, ) -> Result> + 'static>> { - let manifest_files = manifest_list.entries().iter(); + let mut filter_fn: Option> = None; + let manifest_files = { + if let Some(to_snapshot_id) = self.to_snapshot_id { + // Incremental scan mode: + // Get all added files between two snapshots. + // - data files in `Append` and `Overwrite` snapshots are included. + // - delete files are ignored + // - `Replace` snapshots (e.g., compaction) are ignored. + // + // `latest_snapshot_id` is inclusive, `oldest_snapshot_id` is exclusive. + + let snapshots = + ancestors_between(&self.table_metadata, to_snapshot_id, self.from_snapshot_id) + .filter(|snapshot| { + matches!( + snapshot.summary().operation, + Operation::Append | Operation::Overwrite + ) + }) + .collect_vec(); + let snapshot_ids: HashSet = snapshots + .iter() + .map(|snapshot| snapshot.snapshot_id()) + .collect(); + + let mut manifest_files = vec![]; + for snapshot in snapshots { + 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.push(entry.clone()); + } + } + + filter_fn = Some(Arc::new(move |entry: &ManifestEntryRef| { + matches!(entry.status(), ManifestStatus::Added) + && matches!(entry.data_file().content_type(), DataContentType::Data) + && ( + // Is it possible that the snapshot id here is not contained? + entry.snapshot_id().is_none() + || snapshot_ids.contains(&entry.snapshot_id().unwrap()) + ) + })); + + manifest_files + } else { + manifest_list.entries().to_vec() + } + }; // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut filtered_mfcs = vec![]; - for manifest_file in manifest_files { + for manifest_file in &manifest_files { let tx = if manifest_file.content == ManifestContentType::Deletes { delete_file_tx.clone() } else { @@ -224,6 +290,7 @@ impl PlanContext { partition_bound_predicate, tx, delete_file_idx.clone(), + filter_fn.clone(), ); filtered_mfcs.push(Ok(mfc)); @@ -238,6 +305,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 +328,61 @@ impl PlanContext { field_ids: self.field_ids.clone(), expression_evaluator_cache: self.expression_evaluator_cache.clone(), delete_file_index, + filter_fn, } } } + +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. +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). +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), + ) +} diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 6884e00b9b..9691f119a3 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -21,6 +21,7 @@ mod cache; use cache::*; mod context; use context::*; +pub use task::*; mod task; use std::sync::Arc; @@ -29,7 +30,6 @@ use arrow_array::RecordBatch; use futures::channel::mpsc::{Sender, channel}; use futures::stream::BoxStream; use futures::{SinkExt, StreamExt, TryStreamExt}; -pub use task::*; use crate::arrow::ArrowReaderBuilder; use crate::delete_file_index::DeleteFileIndex; @@ -51,6 +51,10 @@ pub struct TableScanBuilder<'a> { // Defaults to none which means select all columns column_names: Option>, snapshot_id: Option, + /// Exclusive. Used for incremental scan. + from_snapshot_id: Option, + /// Inclusive. Used for incremental scan. + to_snapshot_id: Option, batch_size: Option, case_sensitive: bool, filter: Option, @@ -69,6 +73,8 @@ impl<'a> TableScanBuilder<'a> { table, column_names: None, snapshot_id: None, + from_snapshot_id: None, + to_snapshot_id: None, batch_size: None, case_sensitive: true, filter: None, @@ -130,6 +136,18 @@ impl<'a> TableScanBuilder<'a> { self } + /// Set the starting snapshot id (exclusive) for incremental scan. + pub fn from_snapshot_id(mut self, from_snapshot_id: i64) -> Self { + self.from_snapshot_id = Some(from_snapshot_id); + self + } + + /// Set the ending snapshot id (inclusive) for incremental scan. + pub fn to_snapshot_id(mut self, to_snapshot_id: i64) -> Self { + self.to_snapshot_id = Some(to_snapshot_id); + self + } + /// Sets the concurrency limit for both manifest files and manifest /// entries for this scan pub fn with_concurrency_limit(mut self, limit: usize) -> Self { @@ -185,6 +203,25 @@ impl<'a> TableScanBuilder<'a> { /// Build the table scan. pub fn build(self) -> Result { + // Validate that we have either a snapshot scan or an incremental scan configuration + if self.from_snapshot_id.is_some() || self.to_snapshot_id.is_some() { + // For incremental scan, we need to_snapshot_id to be set. from_snapshot_id is optional. + if self.to_snapshot_id.is_none() { + return Err(Error::new( + ErrorKind::DataInvalid, + "Incremental scan requires to_snapshot_id to be set", + )); + } + + // snapshot_id should not be set for incremental scan + if self.snapshot_id.is_some() { + return Err(Error::new( + ErrorKind::DataInvalid, + "snapshot_id should not be set for incremental scan. Use from_snapshot_id and to_snapshot_id instead.", + )); + } + } + let snapshot = match self.snapshot_id { Some(snapshot_id) => self .table @@ -214,7 +251,6 @@ impl<'a> TableScanBuilder<'a> { current_snapshot_id.clone() } }; - let schema = snapshot.schema(self.table.metadata())?; // Check that all column names exist in the schema. @@ -277,6 +313,8 @@ impl<'a> TableScanBuilder<'a> { snapshot_bound_predicate: snapshot_bound_predicate.map(Arc::new), object_cache: self.table.object_cache(), field_ids: Arc::new(field_ids), + from_snapshot_id: self.from_snapshot_id, + to_snapshot_id: self.to_snapshot_id, partition_filter_cache: Arc::new(PartitionFilterCache::new()), manifest_evaluator_cache: Arc::new(ManifestEvaluatorCache::new()), expression_evaluator_cache: Arc::new(ExpressionEvaluatorCache::new()), @@ -353,7 +391,7 @@ impl TableScan { manifest_entry_data_ctx_tx, delete_file_idx.clone(), manifest_entry_delete_ctx_tx, - )?; + ).await?; let mut channel_for_manifest_error = file_scan_task_tx.clone(); @@ -383,7 +421,7 @@ impl TableScan { spawn(async move { Self::process_delete_manifest_entry(manifest_entry_context, tx).await }) - .await + .await }, ) .await; @@ -406,7 +444,7 @@ impl TableScan { spawn(async move { Self::process_data_manifest_entry(manifest_entry_context, tx).await }) - .await + .await }, ) .await; From 2d07dcdc2fad7e85c5b91f4078336980cef86dcf Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 11 Nov 2025 13:55:03 +0800 Subject: [PATCH 02/33] expose data file serialized --- crates/iceberg/src/scan/mod.rs | 18 ++++++++++-------- crates/iceberg/src/spec/manifest/_serde.rs | 9 ++++++--- crates/iceberg/src/spec/manifest/mod.rs | 2 ++ crates/iceberg/src/spec/values.rs | 2 +- 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 9691f119a3..9bb48056f0 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -386,12 +386,14 @@ impl TableScan { // get the [`ManifestFile`]s from the [`ManifestList`], filtering out any // whose partitions cannot match this // scan's filter - let manifest_file_contexts = plan_context.build_manifest_file_contexts( - manifest_list, - manifest_entry_data_ctx_tx, - delete_file_idx.clone(), - manifest_entry_delete_ctx_tx, - ).await?; + let manifest_file_contexts = plan_context + .build_manifest_file_contexts( + manifest_list, + manifest_entry_data_ctx_tx, + delete_file_idx.clone(), + manifest_entry_delete_ctx_tx, + ) + .await?; let mut channel_for_manifest_error = file_scan_task_tx.clone(); @@ -421,7 +423,7 @@ impl TableScan { spawn(async move { Self::process_delete_manifest_entry(manifest_entry_context, tx).await }) - .await + .await }, ) .await; @@ -444,7 +446,7 @@ impl TableScan { spawn(async move { Self::process_data_manifest_entry(manifest_entry_context, tx).await }) - .await + .await }, ) .await; diff --git a/crates/iceberg/src/spec/manifest/_serde.rs b/crates/iceberg/src/spec/manifest/_serde.rs index 7738af46d4..68566bba49 100644 --- a/crates/iceberg/src/spec/manifest/_serde.rs +++ b/crates/iceberg/src/spec/manifest/_serde.rs @@ -98,7 +98,8 @@ impl ManifestEntryV1 { #[serde_as] #[derive(Serialize, Deserialize)] -pub(super) struct DataFileSerde { +/// Data file +pub struct DataFileSerde { #[serde(default)] content: i32, file_path: String, @@ -125,6 +126,7 @@ pub(super) struct DataFileSerde { } impl DataFileSerde { + /// Try to convert a `super::DataFile` to a `DataFileSerde`. pub fn try_from( value: super::DataFile, partition_type: &StructType, @@ -163,6 +165,7 @@ impl DataFileSerde { }) } + /// Try to convert a `DataFileSerde` to a `super::DataFile`. pub fn try_into( self, partition_spec_id: i32, @@ -235,7 +238,7 @@ impl DataFileSerde { } #[serde_as] -#[derive(Serialize, Deserialize)] +#[derive(Serialize, Deserialize, Clone)] #[cfg_attr(test, derive(Debug, PartialEq, Eq))] struct BytesEntry { key: i32, @@ -276,7 +279,7 @@ fn to_bytes_entry(v: impl IntoIterator) -> Result Date: Tue, 11 Nov 2025 14:21:46 +0800 Subject: [PATCH 03/33] support set snapshot id for fast append --- crates/iceberg/src/transaction/append.rs | 26 ++++++++++++++++++++++ crates/iceberg/src/transaction/snapshot.rs | 5 +++-- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index f248543df2..cf688a1c94 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -28,6 +28,7 @@ use crate::transaction::snapshot::{ DefaultManifestProcess, SnapshotProduceOperation, SnapshotProducer, }; use crate::transaction::{ActionCommit, TransactionAction}; +use crate::{Error, ErrorKind}; /// FastAppendAction is a transaction action for fast append data files to the table. pub struct FastAppendAction { @@ -35,6 +36,7 @@ pub struct FastAppendAction { // below are properties used to create SnapshotProducer when commit commit_uuid: Option, key_metadata: Option>, + snapshot_id: Option, snapshot_properties: HashMap, added_data_files: Vec, } @@ -45,6 +47,7 @@ impl FastAppendAction { check_duplicate: true, commit_uuid: None, key_metadata: None, + snapshot_id: None, snapshot_properties: HashMap::default(), added_data_files: vec![], } @@ -79,15 +82,38 @@ impl FastAppendAction { self.snapshot_properties = snapshot_properties; self } + + pub fn set_snapshot_id(mut self, snapshot_id: i64) -> Self { + self.snapshot_id = Some(snapshot_id); + self + } + + /// Generate snapshot id ahead which is used by exactly once delivery. + pub fn generate_snapshot_id(table: &Table) -> i64 { + SnapshotProducer::generate_unique_snapshot_id(table) + } } #[async_trait] impl TransactionAction for FastAppendAction { async fn commit(self: Arc, table: &Table) -> Result { + if let Some(snapshot_id) = self.snapshot_id { + if table + .metadata() + .snapshots() + .any(|s| s.snapshot_id() == snapshot_id) + { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Snapshot id {} already exists", snapshot_id), + )); + } + } let snapshot_producer = SnapshotProducer::new( table, self.commit_uuid.unwrap_or_else(Uuid::now_v7), self.key_metadata.clone(), + self.snapshot_id, self.snapshot_properties.clone(), self.added_data_files.clone(), ); diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 4f85962ff1..6b3bf90e73 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -85,12 +85,13 @@ impl<'a> SnapshotProducer<'a> { table: &'a Table, commit_uuid: Uuid, key_metadata: Option>, + snapshot_id: Option, snapshot_properties: HashMap, added_data_files: Vec, ) -> Self { Self { table, - snapshot_id: Self::generate_unique_snapshot_id(table), + snapshot_id: snapshot_id.unwrap_or_else(|| Self::generate_unique_snapshot_id(table)), commit_uuid, key_metadata, snapshot_properties, @@ -163,7 +164,7 @@ impl<'a> SnapshotProducer<'a> { Ok(()) } - fn generate_unique_snapshot_id(table: &Table) -> i64 { + pub(crate) fn generate_unique_snapshot_id(table: &Table) -> i64 { let generate_random_id = || -> i64 { let (lhs, rhs) = Uuid::new_v4().as_u64_pair(); let snapshot_id = (lhs ^ rhs) as i64; From 8c60928ba1205000e2dcdbc48ed85108297c98a0 Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 26 Sep 2025 15:29:35 +0800 Subject: [PATCH 04/33] feat(iceberg): introduce remove snapshot action address comments --- Cargo.lock | 2 + crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/spec/snapshot.rs | 28 +- crates/iceberg/src/transaction/mod.rs | 7 + .../src/transaction/remove_snapshots.rs | 517 ++++++++++++++++++ .../TableMetadataV2ValidMultiSnapshot.json | 167 ++++++ crates/integration_tests/Cargo.toml | 1 + .../tests/shared_tests/mod.rs | 1 + .../shared_tests/remove_snapshots_test.rs | 131 +++++ 9 files changed, 854 insertions(+), 1 deletion(-) create mode 100644 crates/iceberg/src/transaction/remove_snapshots.rs create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMultiSnapshot.json create mode 100644 crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs diff --git a/Cargo.lock b/Cargo.lock index 2edc51a4ea..e17e403cfe 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3548,6 +3548,7 @@ dependencies = [ "tempfile", "thrift", "tokio", + "tracing", "typed-builder", "url", "uuid", @@ -3702,6 +3703,7 @@ version = "0.7.0" dependencies = [ "arrow-array", "arrow-schema", + "chrono", "ctor", "datafusion", "futures", diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 895a5cf5e4..789e6ef0d0 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -89,6 +89,7 @@ smol = { workspace = true, optional = true } strum = { workspace = true, features = ["derive"] } thrift = { workspace = true } tokio = { workspace = true, optional = false, features = ["sync"] } +tracing = { workspace = true } typed-builder = { workspace = true } url = { workspace = true } uuid = { workspace = true } diff --git a/crates/iceberg/src/spec/snapshot.rs b/crates/iceberg/src/spec/snapshot.rs index 5371cf68f2..132634c2b6 100644 --- a/crates/iceberg/src/spec/snapshot.rs +++ b/crates/iceberg/src/spec/snapshot.rs @@ -193,7 +193,6 @@ impl Snapshot { } /// Get parent snapshot. - #[cfg(test)] pub(crate) fn parent_snapshot(&self, table_metadata: &TableMetadata) -> Option { match self.parent_snapshot_id { Some(id) => table_metadata.snapshot_by_id(id).cloned(), @@ -511,6 +510,33 @@ impl SnapshotRetention { } } +/// An iterator over the ancestors of a snapshot. +pub struct AncestorIterator<'a> { + current: Option, + table_metadata: &'a TableMetadata, +} + +impl Iterator for AncestorIterator<'_> { + type Item = SnapshotRef; + + fn next(&mut self) -> Option { + let current = self.current.take()?; + + let next = current.parent_snapshot(self.table_metadata); + self.current = next; + + Some(current) + } +} + +/// Returns an iterator over the ancestors of a snapshot. +pub fn ancestors_of(snapshot: SnapshotRef, table_metadata: &TableMetadata) -> AncestorIterator<'_> { + AncestorIterator { + current: Some(snapshot), + table_metadata, + } +} + #[cfg(test)] mod tests { use std::collections::HashMap; diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 4116264a14..ba80c94080 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -54,6 +54,7 @@ mod action; pub use action::*; mod append; +pub mod remove_snapshots; mod snapshot; mod sort_order; mod update_location; @@ -71,6 +72,7 @@ use crate::spec::TableProperties; use crate::table::Table; use crate::transaction::action::BoxedTransactionAction; use crate::transaction::append::FastAppendAction; +use crate::transaction::remove_snapshots::RemoveSnapshotAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::transaction::update_location::UpdateLocationAction; use crate::transaction::update_properties::UpdatePropertiesAction; @@ -151,6 +153,11 @@ impl Transaction { UpdateLocationAction::new() } + /// Creates remove snapshot action. + pub fn expire_snapshot(&self) -> RemoveSnapshotAction { + RemoveSnapshotAction::new() + } + /// Update the statistics of table pub fn update_statistics(&self) -> UpdateStatisticsAction { UpdateStatisticsAction::new() diff --git a/crates/iceberg/src/transaction/remove_snapshots.rs b/crates/iceberg/src/transaction/remove_snapshots.rs new file mode 100644 index 0000000000..f86fce079b --- /dev/null +++ b/crates/iceberg/src/transaction/remove_snapshots.rs @@ -0,0 +1,517 @@ +// 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. + +//! Transaction action for removing snapshot. + +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use async_trait::async_trait; +use itertools::Itertools; + +use crate::error::Result; +use crate::spec::{MAIN_BRANCH, SnapshotReference, SnapshotRetention, TableMetadata, ancestors_of}; +use crate::table::Table; +use crate::transaction::{ActionCommit, TransactionAction}; +use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; + +/// Property key for max snapshot age in milliseconds. +pub const MAX_SNAPSHOT_AGE_MS: &str = "history.expire.max-snapshot-age-ms"; +/// Default value for max snapshot age in milliseconds. +pub const MAX_SNAPSHOT_AGE_MS_DEFAULT: i64 = 5 * 24 * 60 * 60 * 1000; // 5 days +/// Property key for min snapshots to keep. +pub const MIN_SNAPSHOTS_TO_KEEP: &str = "history.expire.min-snapshots-to-keep"; +/// Default value for min snapshots to keep. +pub const MIN_SNAPSHOTS_TO_KEEP_DEFAULT: i32 = 1; +/// Property key for max reference age in milliseconds. +pub const MAX_REF_AGE_MS: &str = "history.expire.max-ref-age-ms"; +/// Default value for max reference age in milliseconds. +pub const MAX_REF_AGE_MS_DEFAULT: i64 = i64::MAX; + +/// RemoveSnapshotAction is a transaction action for removing snapshot. +pub struct RemoveSnapshotAction { + clear_expire_files: bool, + ids_to_remove: HashSet, + default_expired_older_than: i64, + default_min_num_snapshots: i32, + default_max_ref_age_ms: i64, + clear_expired_meta_data: bool, + + now: i64, +} + +impl Default for RemoveSnapshotAction { + fn default() -> Self { + Self::new() + } +} + +impl RemoveSnapshotAction { + /// Creates a new action. + pub fn new() -> Self { + let now = chrono::Utc::now().timestamp_millis(); + + Self { + clear_expire_files: false, + ids_to_remove: HashSet::new(), + default_expired_older_than: now - MAX_SNAPSHOT_AGE_MS_DEFAULT, + default_min_num_snapshots: MIN_SNAPSHOTS_TO_KEEP_DEFAULT, + default_max_ref_age_ms: MAX_REF_AGE_MS_DEFAULT, + now, + clear_expired_meta_data: false, + } + } + + /// Finished building the action and apply it to the transaction. + pub fn clear_expire_files(mut self, clear_expire_files: bool) -> Self { + self.clear_expire_files = clear_expire_files; + self + } + + /// Finished building the action and apply it to the transaction. + pub fn expire_snapshot_id(mut self, expire_snapshot_id: i64) -> Self { + self.ids_to_remove.insert(expire_snapshot_id); + self + } + + /// Finished building the action and apply it to the transaction. + pub fn expire_older_than(mut self, timestamp_ms: i64) -> Self { + self.default_expired_older_than = timestamp_ms; + self + } + + /// Finished building the action and apply it to the transaction. + pub fn retain_last(mut self, min_num_snapshots: i32) -> Self { + self.default_min_num_snapshots = min_num_snapshots; + self + } + + /// Finished building the action and apply it to the transaction. + pub fn clear_expired_meta_data(mut self, clear_expired_meta_data: bool) -> Self { + self.clear_expired_meta_data = clear_expired_meta_data; + self + } + + fn compute_retained_refs( + &self, + snapshot_refs: &HashMap, + table_meta: &TableMetadata, + ) -> HashMap { + let mut retained_refs = HashMap::new(); + + for (ref_name, snapshot_ref) in snapshot_refs { + if ref_name == MAIN_BRANCH { + retained_refs.insert(ref_name.clone(), snapshot_ref.clone()); + continue; + } + + let snapshot = table_meta.snapshot_by_id(snapshot_ref.snapshot_id); + let max_ref_age_ms = match &snapshot_ref.retention { + SnapshotRetention::Branch { + min_snapshots_to_keep: _, + max_snapshot_age_ms: _, + max_ref_age_ms, + } => max_ref_age_ms, + SnapshotRetention::Tag { max_ref_age_ms } => max_ref_age_ms, + } + .unwrap_or(self.default_max_ref_age_ms); + + if let Some(snapshot) = snapshot { + let ref_age_ms = self.now - snapshot.timestamp_ms(); + if ref_age_ms <= max_ref_age_ms { + retained_refs.insert(ref_name.clone(), snapshot_ref.clone()); + } + } else { + tracing::warn!( + "Snapshot {} for reference {} not found, removing the reference", + snapshot_ref.snapshot_id, + ref_name + ); + } + } + + retained_refs + } + + fn compute_all_branch_snapshots_to_retain( + &self, + refs: impl Iterator, + table_meta: &TableMetadata, + ) -> HashSet { + let mut branch_snapshots_to_retain = HashSet::new(); + for snapshot_ref in refs { + if snapshot_ref.is_branch() { + let max_snapshot_age_ms = match snapshot_ref.retention { + SnapshotRetention::Branch { + min_snapshots_to_keep: _, + max_snapshot_age_ms, + max_ref_age_ms: _, + } => max_snapshot_age_ms, + SnapshotRetention::Tag { max_ref_age_ms: _ } => None, + }; + + let expire_snapshot_older_than = + if let Some(max_snapshot_age_ms) = max_snapshot_age_ms { + self.now - max_snapshot_age_ms + } else { + self.default_expired_older_than + }; + + let min_snapshots_to_keep = match snapshot_ref.retention { + SnapshotRetention::Branch { + min_snapshots_to_keep, + max_snapshot_age_ms: _, + max_ref_age_ms: _, + } => min_snapshots_to_keep, + SnapshotRetention::Tag { max_ref_age_ms: _ } => None, + } + .unwrap_or(self.default_min_num_snapshots); + + branch_snapshots_to_retain.extend(self.compute_branch_snapshots_to_retain( + snapshot_ref.snapshot_id, + expire_snapshot_older_than, + min_snapshots_to_keep as usize, + table_meta, + )); + } + } + + branch_snapshots_to_retain + } + + fn compute_branch_snapshots_to_retain( + &self, + snapshot_id: i64, + expire_snapshots_older_than: i64, + min_snapshots_to_keep: usize, + table_meta: &TableMetadata, + ) -> HashSet { + let mut ids_to_retain = HashSet::new(); + if let Some(snapshot) = table_meta.snapshot_by_id(snapshot_id) { + let ancestors = ancestors_of(snapshot.clone(), table_meta); + for ancestor in ancestors { + if ids_to_retain.len() < min_snapshots_to_keep + || ancestor.timestamp_ms() >= expire_snapshots_older_than + { + ids_to_retain.insert(ancestor.snapshot_id()); + } else { + return ids_to_retain; + } + } + } + + ids_to_retain + } + + fn unreferenced_snapshots_to_retain( + &self, + refs: impl Iterator, + table_meta: &TableMetadata, + ) -> HashSet { + let mut ids_to_retain = HashSet::new(); + let mut referenced_snapshots = HashSet::new(); + + for snapshot_ref in refs { + if snapshot_ref.is_branch() { + if let Some(snapshot) = table_meta.snapshot_by_id(snapshot_ref.snapshot_id) { + let ancestors = ancestors_of(snapshot.clone(), table_meta); + for ancestor in ancestors { + referenced_snapshots.insert(ancestor.snapshot_id()); + } + } + } else { + referenced_snapshots.insert(snapshot_ref.snapshot_id); + } + } + + for snapshot in table_meta.snapshots() { + if !referenced_snapshots.contains(&snapshot.snapshot_id()) + && snapshot.timestamp_ms() >= self.default_expired_older_than + { + ids_to_retain.insert(snapshot.snapshot_id()); + } + } + + ids_to_retain + } +} + +#[async_trait] +impl TransactionAction for RemoveSnapshotAction { + async fn commit(self: Arc, table: &Table) -> Result { + if table.metadata().refs.is_empty() { + return Ok(ActionCommit::new(vec![], vec![])); + } + + let table_meta = table.metadata().clone(); + + let mut ids_to_retain = HashSet::new(); + let retained_refs = self.compute_retained_refs(&table_meta.refs, &table_meta); + let mut retained_id_to_refs = HashMap::new(); + for (ref_name, snapshot_ref) in &retained_refs { + let snapshot_id = snapshot_ref.snapshot_id; + retained_id_to_refs + .entry(snapshot_id) + .or_insert_with(Vec::new) + .push(ref_name.clone()); + + ids_to_retain.insert(snapshot_id); + } + + for id_to_remove in &self.ids_to_remove { + if let Some(refs_for_id) = retained_id_to_refs.get(id_to_remove) { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot remove snapshot {:?} with retained references: {:?}", + id_to_remove, refs_for_id + ), + )); + } + } + + ids_to_retain.extend(self.compute_all_branch_snapshots_to_retain( + table_meta.refs.values().cloned(), + &table_meta, + )); + ids_to_retain.extend( + self.unreferenced_snapshots_to_retain(table_meta.refs.values().cloned(), &table_meta), + ); + + let mut updates = vec![]; + let mut requirements = vec![]; + + for ref_name in table_meta.refs.keys() { + if !retained_refs.contains_key(ref_name) { + updates.push(TableUpdate::RemoveSnapshotRef { + ref_name: ref_name.clone(), + }); + } + } + + let mut snapshot_to_remove = Vec::from_iter(self.ids_to_remove.iter().cloned()); + for snapshot in table_meta.snapshots() { + if !ids_to_retain.contains(&snapshot.snapshot_id()) { + snapshot_to_remove.push(snapshot.snapshot_id()); + } + } + + if !snapshot_to_remove.is_empty() { + // TODO: batch remove when server supports it + for snapshot_id in snapshot_to_remove { + updates.push(TableUpdate::RemoveSnapshots { + snapshot_ids: vec![snapshot_id], + }); + } + } + + if self.clear_expired_meta_data { + let mut reachable_specs = HashSet::new(); + reachable_specs.insert(table_meta.current_schema_id()); + let mut reachable_schemas = HashSet::new(); + reachable_schemas.insert(table_meta.current_schema_id()); + + // TODO: parallelize loading manifest list and get reachable specs and schemas to reduce latency + for snapshot in table_meta.snapshots() { + if ids_to_retain.contains(&snapshot.snapshot_id()) { + let manifest_list = snapshot + .load_manifest_list(table.file_io(), &table_meta) + .await?; + + for manifest in manifest_list.entries() { + reachable_specs.insert(manifest.partition_spec_id); + } + + if let Some(schema_id) = snapshot.schema_id() { + reachable_schemas.insert(schema_id); + } + } + } + + let spec_to_remove: Vec = table_meta + .partition_specs_iter() + .filter_map(|spec| { + if !reachable_specs.contains(&spec.spec_id()) { + Some(spec.spec_id()) + } else { + None + } + }) + .unique() + .collect(); + + if !spec_to_remove.is_empty() { + updates.push(TableUpdate::RemovePartitionSpecs { + spec_ids: spec_to_remove, + }); + } + + let schema_to_remove: Vec = table_meta + .schemas_iter() + .filter_map(|schema| { + if !reachable_schemas.contains(&schema.schema_id()) { + Some(schema.schema_id()) + } else { + None + } + }) + .unique() + .collect(); + + if !schema_to_remove.is_empty() { + updates.push(TableUpdate::RemoveSchemas { + schema_ids: schema_to_remove, + }); + } + } + + requirements.push(TableRequirement::UuidMatch { + uuid: table_meta.uuid(), + }); + requirements.push(TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: table_meta.current_snapshot_id(), + }); + + Ok(ActionCommit::new(updates, requirements)) + } +} + +#[cfg(test)] +mod tests { + use std::fs::File; + use std::io::BufReader; + use std::sync::Arc; + + use crate::io::FileIOBuilder; + use crate::spec::{MAIN_BRANCH, TableMetadata}; + use crate::table::Table; + use crate::transaction::{Transaction, TransactionAction}; + use crate::{TableIdent, TableRequirement}; + + fn make_v2_table_with_mutli_snapshot() -> Table { + let file = File::open(format!( + "{}/testdata/table_metadata/{}", + env!("CARGO_MANIFEST_DIR"), + "TableMetadataV2ValidMultiSnapshot.json" + )) + .unwrap(); + let reader = BufReader::new(file); + let resp = serde_json::from_reader::<_, TableMetadata>(reader).unwrap(); + + Table::builder() + .metadata(resp) + .metadata_location("s3://bucket/test/location/metadata/v1.json".to_string()) + .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) + .file_io(FileIOBuilder::new("memory").build().unwrap()) + .build() + .unwrap() + } + + #[tokio::test] + async fn test_remove_snapshot_action() { + let table = make_v2_table_with_mutli_snapshot(); + let table_meta = table.metadata().clone(); + assert_eq!(5, table_meta.snapshots().count()); + { + let tx = Transaction::new(&table); + let act = tx.expire_snapshot(); + let mut v = Arc::new(act).commit(&table).await.unwrap(); + + let updates = v.take_updates(); + assert_eq!(4, updates.len()); + + let requirements = v.take_requirements(); + assert_eq!(2, requirements.len()); + // assert_eq!(4, tx.updates.len()); + + assert_eq!( + vec![ + TableRequirement::UuidMatch { + uuid: table.metadata().uuid() + }, + TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: table.metadata().current_snapshot_id() + } + ], + requirements + ); + } + + { + let tx = Transaction::new(&table); + let act = tx.expire_snapshot().retain_last(2); + let mut action_commit = Arc::new(act).commit(&table).await.unwrap(); + + let updates = action_commit.take_updates(); + let requirements = action_commit.take_requirements(); + + assert_eq!(3, updates.len()); + assert_eq!( + vec![ + TableRequirement::UuidMatch { + uuid: table.metadata().uuid() + }, + TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: table.metadata().current_snapshot_id(), + } + ], + requirements + ); + } + + { + let tx = Transaction::new(&table); + let act = tx.expire_snapshot().retain_last(100).expire_older_than(100); + + let mut action_commit = Arc::new(act).commit(&table).await.unwrap(); + + let updates = action_commit.take_updates(); + let requirements = action_commit.take_requirements(); + + assert_eq!(0, updates.len()); + assert_eq!( + vec![ + TableRequirement::UuidMatch { + uuid: table.metadata().uuid() + }, + TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: table.metadata().current_snapshot_id(), + } + ], + requirements + ); + } + + { + // test remove main current snapshot + let tx = Transaction::new(&table); + let act = tx + .expire_snapshot() + .expire_snapshot_id(table.metadata().current_snapshot_id().unwrap()); + + let err = Arc::new(act).commit(&table).await.err().unwrap(); + assert_eq!( + "DataInvalid => Cannot remove snapshot 3067729675574597004 with retained references: [\"main\"]", + err.to_string() + ) + } + } +} diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMultiSnapshot.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMultiSnapshot.json new file mode 100644 index 0000000000..a52fa2a906 --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMultiSnapshot.json @@ -0,0 +1,167 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1675100955770, + "last-column-id": 3, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 1, + "identifier-field-ids": [ + 1, + 2 + ], + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": 3067729675574597004, + "snapshots": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "sequence-number": 0, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/1.avro" + }, + { + "snapshot-id": 3055729675574597004, + "parent-snapshot-id": 3051729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 1 + }, + { + "snapshot-id": 3059729675574597004, + "parent-snapshot-id": 3055729675574597004, + "timestamp-ms": 1595100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/3.avro", + "schema-id": 1 + }, + { + "snapshot-id": 3063729675574597004, + "parent-snapshot-id": 3059729675574597004, + "timestamp-ms": 1635100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/4.avro", + "schema-id": 1 + }, + { + "snapshot-id": 3067729675574597004, + "parent-snapshot-id": 3063729675574597004, + "timestamp-ms": 1675100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/5.avro", + "schema-id": 1 + } + ], + "snapshot-log": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770 + }, + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770 + }, + { + "snapshot-id": 3059729675574597004, + "timestamp-ms": 1595100955770 + }, + { + "snapshot-id": 3063729675574597004, + "timestamp-ms": 1635100955770 + }, + { + "snapshot-id": 3067729675574597004, + "timestamp-ms": 1675100955770 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/integration_tests/Cargo.toml b/crates/integration_tests/Cargo.toml index 07eea5f375..0f2a7563aa 100644 --- a/crates/integration_tests/Cargo.toml +++ b/crates/integration_tests/Cargo.toml @@ -27,6 +27,7 @@ version = { workspace = true } [dependencies] arrow-array = { workspace = true } arrow-schema = { workspace = true } +chrono = { workspace = true } ctor = { workspace = true } datafusion = { workspace = true } futures = { workspace = true } diff --git a/crates/integration_tests/tests/shared_tests/mod.rs b/crates/integration_tests/tests/shared_tests/mod.rs index 065b14d5da..70c041def2 100644 --- a/crates/integration_tests/tests/shared_tests/mod.rs +++ b/crates/integration_tests/tests/shared_tests/mod.rs @@ -29,6 +29,7 @@ mod conflict_commit_test; mod datafusion; mod read_evolved_schema; mod read_positional_deletes; +mod remove_snapshots_test; mod scan_all_type; pub async fn random_ns() -> Namespace { diff --git a/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs b/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs new file mode 100644 index 0000000000..7546575f8e --- /dev/null +++ b/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs @@ -0,0 +1,131 @@ +// 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. + +//! Integration tests for rest catalog. + +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use iceberg::transaction::{ApplyTransactionAction, Transaction}; +use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, CatalogBuilder, TableCreation}; +use iceberg_catalog_rest::RestCatalogBuilder; +use parquet::file::properties::WriterProperties; + +use crate::get_shared_containers; +use crate::shared_tests::{random_ns, test_schema}; + +#[tokio::test] +async fn test_expire_snapshots_by_count() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let mut table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // commit result + for i in 0..10 { + // Create a new data file writer for each iteration + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + + // Create different data for each iteration + let col1 = StringArray::from(vec![ + Some(format!("foo_{}", i)), + Some(format!("bar_{}", i)), + None, + Some(format!("baz_{}", i)), + ]); + let col2 = Int32Array::from(vec![Some(i), Some(i + 1), Some(i + 2), Some(i + 3)]); + let col3 = BooleanArray::from(vec![ + Some(i % 2 == 0), + Some(i % 2 == 1), + None, + Some(i % 3 == 0), + ]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + + // Write the unique data and get the data file + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + let tx = Transaction::new(&table); + let append_action = tx.fast_append(); + let tx = append_action.add_data_files(data_file).apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + } + + // check snapshot count + let snapshot_counts = table.metadata().snapshots().count(); + assert_eq!(10, snapshot_counts); + + let tx = Transaction::new(&table); + let now = chrono::Utc::now().timestamp_millis(); + let remove_action = tx.expire_snapshot().retain_last(5).expire_older_than(now); + let tx = remove_action.apply(tx).unwrap(); + let t = tx.commit(&rest_catalog).await.unwrap(); + assert_eq!(5, t.metadata().snapshots().count()); +} From 6d4339e30dfb305313faa1f2fe2d0b323f38b9a3 Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 18 Aug 2025 18:08:29 +0800 Subject: [PATCH 05/33] feat: support append delete file Signed-off-by: xxchan Co-authored-by: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Co-authored-by: ZENOTME --- crates/iceberg/src/transaction/append.rs | 19 ++++++- crates/iceberg/src/transaction/snapshot.rs | 63 +++++++++++++++------- 2 files changed, 62 insertions(+), 20 deletions(-) diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index cf688a1c94..0114abe72f 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -39,6 +39,7 @@ pub struct FastAppendAction { snapshot_id: Option, snapshot_properties: HashMap, added_data_files: Vec, + added_delete_files: Vec, } impl FastAppendAction { @@ -50,6 +51,7 @@ impl FastAppendAction { snapshot_id: None, snapshot_properties: HashMap::default(), added_data_files: vec![], + added_delete_files: vec![], } } @@ -61,7 +63,16 @@ impl FastAppendAction { /// Add data files to the snapshot. pub fn add_data_files(mut self, data_files: impl IntoIterator) -> Self { - self.added_data_files.extend(data_files); + for file in data_files { + match file.content_type() { + crate::spec::DataContentType::Data => self.added_data_files.push(file), + crate::spec::DataContentType::PositionDeletes + | crate::spec::DataContentType::EqualityDeletes => { + self.added_delete_files.push(file) + } + } + } + self } @@ -116,16 +127,22 @@ impl TransactionAction for FastAppendAction { self.snapshot_id, self.snapshot_properties.clone(), self.added_data_files.clone(), + self.added_delete_files.clone(), ); // validate added files snapshot_producer.validate_added_data_files(&self.added_data_files)?; + snapshot_producer.validate_added_data_files(&self.added_delete_files)?; // Checks duplicate files if self.check_duplicate { snapshot_producer .validate_duplicate_files(&self.added_data_files) .await?; + + snapshot_producer + .validate_duplicate_files(&self.added_delete_files) + .await?; } snapshot_producer diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 6b3bf90e73..d4a4551fda 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -23,10 +23,10 @@ use uuid::Uuid; use crate::error::Result; use crate::spec::{ - DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, ManifestEntry, - ManifestFile, ManifestListWriter, ManifestWriter, ManifestWriterBuilder, Operation, Snapshot, - SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, StructType, Summary, - TableProperties, update_snapshot_summaries, + DataContentType, DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, + ManifestEntry, ManifestFile, ManifestListWriter, ManifestWriter, ManifestWriterBuilder, + Operation, Snapshot, SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, + StructType, Summary, TableProperties, update_snapshot_summaries, }; use crate::table::Table; use crate::transaction::ActionCommit; @@ -73,7 +73,8 @@ pub(crate) struct SnapshotProducer<'a> { commit_uuid: Uuid, key_metadata: Option>, snapshot_properties: HashMap, - added_data_files: Vec, + pub added_data_files: Vec, + added_delete_files: Vec, // A counter used to generate unique manifest file names. // It starts from 0 and increments for each new manifest file. // Note: This counter is limited to the range of (0..u64::MAX). @@ -88,6 +89,7 @@ impl<'a> SnapshotProducer<'a> { snapshot_id: Option, snapshot_properties: HashMap, added_data_files: Vec, + added_delete_files: Vec, ) -> Self { Self { table, @@ -96,18 +98,13 @@ impl<'a> SnapshotProducer<'a> { key_metadata, snapshot_properties, added_data_files, + added_delete_files, manifest_counter: (0..), } } pub(crate) fn validate_added_data_files(&self, added_data_files: &[DataFile]) -> Result<()> { for data_file in added_data_files { - if data_file.content_type() != crate::spec::DataContentType::Data { - return Err(Error::new( - ErrorKind::DataInvalid, - "Only data content type is allowed for fast append", - )); - } // Check if the data file partition spec id matches the table default partition spec id. if self.table.metadata().default_partition_spec_id() != data_file.partition_spec_id { return Err(Error::new( @@ -252,18 +249,42 @@ impl<'a> SnapshotProducer<'a> { } // Write manifest file for added data files and return the ManifestFile for ManifestList. - async fn write_added_manifest(&mut self) -> Result { - let added_data_files = std::mem::take(&mut self.added_data_files); - if added_data_files.is_empty() { + async fn write_added_manifest(&mut self, added_files: Vec) -> Result { + if added_files.is_empty() { return Err(Error::new( ErrorKind::PreconditionFailed, "No added data files found when write an added manifest file", )); } + let file_count = added_files.len(); + + let manifest_content_type = { + let mut data_num = 0; + let mut delete_num = 0; + for f in &added_files { + match f.content_type() { + DataContentType::Data => data_num += 1, + DataContentType::PositionDeletes | DataContentType::EqualityDeletes => { + delete_num += 1 + } + } + } + if data_num == file_count { + ManifestContentType::Data + } else if delete_num == file_count { + ManifestContentType::Deletes + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "added DataFile for a ManifestFile should be same type (Data or Delete)", + )); + } + }; + let snapshot_id = self.snapshot_id; let format_version = self.table.metadata().format_version(); - let manifest_entries = added_data_files.into_iter().map(|data_file| { + let manifest_entries = added_files.into_iter().map(|data_file| { let builder = ManifestEntry::builder() .status(crate::spec::ManifestStatus::Added) .data_file(data_file); @@ -275,7 +296,7 @@ impl<'a> SnapshotProducer<'a> { builder.build() } }); - let mut writer = self.new_manifest_writer(ManifestContentType::Data)?; + let mut writer = self.new_manifest_writer(manifest_content_type)?; for entry in manifest_entries { writer.add_entry(entry)?; } @@ -304,12 +325,16 @@ impl<'a> SnapshotProducer<'a> { // Process added entries. if !self.added_data_files.is_empty() { - let added_manifest = self.write_added_manifest().await?; + let added_data_files = std::mem::take(&mut self.added_data_files); + let added_manifest = self.write_added_manifest(added_data_files).await?; manifest_files.push(added_manifest); } - // # TODO - // Support process delete entries. + if !self.added_delete_files.is_empty() { + let added_delete_files = std::mem::take(&mut self.added_delete_files); + let added_manifest = self.write_added_manifest(added_delete_files).await?; + manifest_files.push(added_manifest); + } let manifest_files = manifest_process.process_manifests(self, manifest_files); Ok(manifest_files) From 6519c9857da736ccb11f4502eac5ef67c1cc71eb Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 5 Nov 2025 17:20:53 +0800 Subject: [PATCH 06/33] feat: support merge append Signed-off-by: xxchan Co-authored-by: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Co-authored-by: ZENOTME --- crates/iceberg/src/spec/manifest/entry.rs | 6 + crates/iceberg/src/transaction/append.rs | 131 ++++++++- crates/iceberg/src/transaction/mod.rs | 13 +- .../src/transaction/remove_snapshots.rs | 6 - crates/iceberg/src/transaction/snapshot.rs | 222 ++++++++++++++- crates/iceberg/src/utils.rs | 145 ++++++++++ .../testdata/docker-compose.yaml | 24 ++ .../tests/shared_tests/merge_append_test.rs | 260 ++++++++++++++++++ .../tests/shared_tests/mod.rs | 1 + 9 files changed, 784 insertions(+), 24 deletions(-) create mode 100644 crates/integration_tests/tests/shared_tests/merge_append_test.rs diff --git a/crates/iceberg/src/spec/manifest/entry.rs b/crates/iceberg/src/spec/manifest/entry.rs index e8fe0f223a..93f99e9195 100644 --- a/crates/iceberg/src/spec/manifest/entry.rs +++ b/crates/iceberg/src/spec/manifest/entry.rs @@ -147,6 +147,12 @@ impl ManifestEntry { pub fn data_file(&self) -> &DataFile { &self.data_file } + + /// File sequence number indicating when the file was added. Inherited when null and status is 1 (added). + #[inline] + pub fn file_sequence_number(&self) -> Option { + self.file_sequence_number + } } /// Used to track additions and deletions in ManifestEntry. diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index 0114abe72f..b77259c9f8 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -25,11 +25,21 @@ use crate::error::Result; use crate::spec::{DataFile, ManifestEntry, ManifestFile, Operation}; use crate::table::Table; use crate::transaction::snapshot::{ - DefaultManifestProcess, SnapshotProduceOperation, SnapshotProducer, + DefaultManifestProcess, MergeManifestProcess, SnapshotProduceOperation, SnapshotProducer, }; use crate::transaction::{ActionCommit, TransactionAction}; use crate::{Error, ErrorKind}; +/// Target size of manifest file when merging manifests. +pub const MANIFEST_TARGET_SIZE_BYTES: &str = "commit.manifest.target-size-bytes"; +const MANIFEST_TARGET_SIZE_BYTES_DEFAULT: u32 = 8 * 1024 * 1024; // 8 MB +/// Minimum number of manifests to merge. +pub const MANIFEST_MIN_MERGE_COUNT: &str = "commit.manifest.min-count-to-merge"; +const MANIFEST_MIN_MERGE_COUNT_DEFAULT: u32 = 100; +/// Whether allow to merge manifests. +pub const MANIFEST_MERGE_ENABLED: &str = "commit.manifest-merge.enabled"; +const MANIFEST_MERGE_ENABLED_DEFAULT: bool = false; + /// FastAppendAction is a transaction action for fast append data files to the table. pub struct FastAppendAction { check_duplicate: bool, @@ -189,6 +199,125 @@ impl SnapshotProduceOperation for FastAppendOperation { } } +/// MergeAppendAction is a transaction action similar to fast append except that it will merge manifests +/// based on the target size. +pub struct MergeAppendAction { + // snapshot_produce_action: SnapshotProducer<'_>, + target_size_bytes: u32, + min_count_to_merge: u32, + merge_enabled: bool, + + check_duplicate: bool, + // below are properties used to create SnapshotProducer when commit + commit_uuid: Option, + key_metadata: Option>, + snapshot_properties: HashMap, + added_data_files: Vec, + added_delete_files: Vec, + snapshot_id: Option, +} + +impl MergeAppendAction { + #[allow(clippy::too_many_arguments)] + pub(crate) fn new() -> Self { + Self { + target_size_bytes: MANIFEST_TARGET_SIZE_BYTES_DEFAULT, + min_count_to_merge: MANIFEST_MIN_MERGE_COUNT_DEFAULT, + merge_enabled: MANIFEST_MERGE_ENABLED_DEFAULT, + check_duplicate: true, + commit_uuid: None, + key_metadata: None, + snapshot_properties: HashMap::default(), + added_data_files: vec![], + added_delete_files: vec![], + snapshot_id: None, + } + } + + pub fn set_target_size_bytes(mut self, v: u32) -> Self { + self.target_size_bytes = v; + self + } + + pub fn set_min_count_to_merge(mut self, v: u32) -> Self { + self.min_count_to_merge = v; + self + } + + pub fn set_merge_enabled(mut self, v: bool) -> Self { + self.merge_enabled = v; + self + } + + pub fn set_snapshot_properties(mut self, snapshot_properties: HashMap) -> Self { + let target_size_bytes: u32 = snapshot_properties + .get(MANIFEST_TARGET_SIZE_BYTES) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + let min_count_to_merge: u32 = snapshot_properties + .get(MANIFEST_MIN_MERGE_COUNT) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MIN_MERGE_COUNT_DEFAULT); + let merge_enabled = snapshot_properties + .get(MANIFEST_MERGE_ENABLED) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MERGE_ENABLED_DEFAULT); + + self.snapshot_properties = snapshot_properties; + self.target_size_bytes = target_size_bytes; + self.min_count_to_merge = min_count_to_merge; + self.merge_enabled = merge_enabled; + + self + } + + /// Add data files to the snapshot. + pub fn add_data_files(mut self, data_files: impl IntoIterator) -> Self { + self.added_data_files.extend(data_files); + self + } +} + +#[async_trait] +impl TransactionAction for MergeAppendAction { + async fn commit(self: Arc, table: &Table) -> Result { + let snapshot_producer = SnapshotProducer::new( + table, + self.commit_uuid.unwrap_or_else(Uuid::now_v7), + self.key_metadata.clone(), + self.snapshot_id, + self.snapshot_properties.clone(), + self.added_data_files.clone(), + self.added_delete_files.clone(), + ); + + // validate added files + snapshot_producer.validate_added_data_files(&self.added_data_files)?; + snapshot_producer.validate_added_data_files(&self.added_delete_files)?; + + // Checks duplicate files + if self.check_duplicate { + snapshot_producer + .validate_duplicate_files(&self.added_data_files) + .await?; + + snapshot_producer + .validate_duplicate_files(&self.added_delete_files) + .await?; + } + + if self.merge_enabled { + let process = + MergeManifestProcess::new(self.target_size_bytes, self.min_count_to_merge); + snapshot_producer.commit(FastAppendOperation, process).await + } else { + snapshot_producer + .commit(FastAppendOperation, DefaultManifestProcess) + .await + } + } +} + #[cfg(test)] mod tests { use std::collections::HashMap; diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index ba80c94080..7b6a1f6b7c 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -54,7 +54,7 @@ mod action; pub use action::*; mod append; -pub mod remove_snapshots; +mod remove_snapshots; mod snapshot; mod sort_order; mod update_location; @@ -65,14 +65,14 @@ mod upgrade_format_version; use std::sync::Arc; use std::time::Duration; +pub use append::{MANIFEST_MERGE_ENABLED, MANIFEST_MIN_MERGE_COUNT, MANIFEST_TARGET_SIZE_BYTES}; use backon::{BackoffBuilder, ExponentialBackoff, ExponentialBuilder, RetryableWithContext}; +use remove_snapshots::RemoveSnapshotAction; use crate::error::Result; use crate::spec::TableProperties; use crate::table::Table; -use crate::transaction::action::BoxedTransactionAction; -use crate::transaction::append::FastAppendAction; -use crate::transaction::remove_snapshots::RemoveSnapshotAction; +use crate::transaction::append::{FastAppendAction, MergeAppendAction}; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::transaction::update_location::UpdateLocationAction; use crate::transaction::update_properties::UpdatePropertiesAction; @@ -143,6 +143,11 @@ impl Transaction { FastAppendAction::new() } + /// Creates a merge append action. + pub fn merge_append(&self) -> MergeAppendAction { + MergeAppendAction::new() + } + /// Creates replace sort order action. pub fn replace_sort_order(&self) -> ReplaceSortOrderAction { ReplaceSortOrderAction::new() diff --git a/crates/iceberg/src/transaction/remove_snapshots.rs b/crates/iceberg/src/transaction/remove_snapshots.rs index f86fce079b..ac805e0ec3 100644 --- a/crates/iceberg/src/transaction/remove_snapshots.rs +++ b/crates/iceberg/src/transaction/remove_snapshots.rs @@ -29,16 +29,10 @@ use crate::table::Table; use crate::transaction::{ActionCommit, TransactionAction}; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; -/// Property key for max snapshot age in milliseconds. -pub const MAX_SNAPSHOT_AGE_MS: &str = "history.expire.max-snapshot-age-ms"; /// Default value for max snapshot age in milliseconds. pub const MAX_SNAPSHOT_AGE_MS_DEFAULT: i64 = 5 * 24 * 60 * 60 * 1000; // 5 days -/// Property key for min snapshots to keep. -pub const MIN_SNAPSHOTS_TO_KEEP: &str = "history.expire.min-snapshots-to-keep"; /// Default value for min snapshots to keep. pub const MIN_SNAPSHOTS_TO_KEEP_DEFAULT: i32 = 1; -/// Property key for max reference age in milliseconds. -pub const MAX_REF_AGE_MS: &str = "history.expire.max-ref-age-ms"; /// Default value for max reference age in milliseconds. pub const MAX_REF_AGE_MS_DEFAULT: i64 = i64::MAX; diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index d4a4551fda..414753b588 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -15,21 +15,26 @@ // specific language governing permissions and limitations // under the License. -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::future::Future; use std::ops::RangeFrom; +use std::pin::Pin; +use async_trait::async_trait; use uuid::Uuid; use crate::error::Result; +use crate::io::FileIO; use crate::spec::{ DataContentType, DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, - ManifestEntry, ManifestFile, ManifestListWriter, ManifestWriter, ManifestWriterBuilder, - Operation, Snapshot, SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, - StructType, Summary, TableProperties, update_snapshot_summaries, + ManifestEntry, ManifestFile, ManifestListWriter, ManifestStatus, ManifestWriter, + ManifestWriterBuilder, Operation, Snapshot, SnapshotReference, SnapshotRetention, + SnapshotSummaryCollector, Struct, StructType, Summary, TableProperties, + update_snapshot_summaries, }; use crate::table::Table; use crate::transaction::ActionCommit; +use crate::utils::bin::ListPacker; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; const META_ROOT_PATH: &str = "metadata"; @@ -49,22 +54,24 @@ pub(crate) trait SnapshotProduceOperation: Send + Sync { pub(crate) struct DefaultManifestProcess; +#[async_trait] impl ManifestProcess for DefaultManifestProcess { - fn process_manifests( + async fn process_manifests( &self, - _snapshot_produce: &SnapshotProducer<'_>, + _snapshot_produce: &mut SnapshotProducer<'_>, manifests: Vec, - ) -> Vec { - manifests + ) -> Result> { + Ok(manifests) } } +#[async_trait] pub(crate) trait ManifestProcess: Send + Sync { - fn process_manifests( + async fn process_manifests( &self, - snapshot_produce: &SnapshotProducer<'_>, + snapshot_produce: &mut SnapshotProducer<'_>, manifests: Vec, - ) -> Vec; + ) -> Result>; } pub(crate) struct SnapshotProducer<'a> { @@ -336,8 +343,9 @@ impl<'a> SnapshotProducer<'a> { manifest_files.push(added_manifest); } - let manifest_files = manifest_process.process_manifests(self, manifest_files); - Ok(manifest_files) + manifest_process + .process_manifests(self, manifest_files) + .await } // Returns a `Summary` of the current snapshot @@ -499,3 +507,191 @@ impl<'a> SnapshotProducer<'a> { Ok(ActionCommit::new(updates, requirements)) } } + +pub(crate) struct MergeManifestProcess { + target_size_bytes: u32, + min_count_to_merge: u32, +} + +impl MergeManifestProcess { + pub fn new(target_size_bytes: u32, min_count_to_merge: u32) -> Self { + Self { + target_size_bytes, + min_count_to_merge, + } + } +} + +#[async_trait] +impl ManifestProcess for MergeManifestProcess { + async fn process_manifests( + &self, + snapshot_produce: &mut SnapshotProducer<'_>, + manifests: Vec, + ) -> Result> { + let (unmerge_data_manifest, unmerge_delete_manifest): (Vec<_>, Vec<_>) = manifests + .into_iter() + .partition(|manifest| matches!(manifest.content, ManifestContentType::Data)); + let mut data_manifest = { + let manifest_merge_manager = MergeManifestManager::new( + self.target_size_bytes, + self.min_count_to_merge, + ManifestContentType::Data, + ); + manifest_merge_manager + .merge_manifest(snapshot_produce, unmerge_data_manifest) + .await? + }; + data_manifest.extend(unmerge_delete_manifest); + Ok(data_manifest) + } +} + +struct MergeManifestManager { + target_size_bytes: u32, + min_count_to_merge: u32, + content: ManifestContentType, +} + +impl MergeManifestManager { + pub fn new( + target_size_bytes: u32, + min_count_to_merge: u32, + content: ManifestContentType, + ) -> Self { + Self { + target_size_bytes, + min_count_to_merge, + content, + } + } + + fn group_by_spec(&self, manifests: Vec) -> BTreeMap> { + let mut grouped_manifests = BTreeMap::new(); + for manifest in manifests { + grouped_manifests + .entry(manifest.partition_spec_id) + .or_insert_with(Vec::new) + .push(manifest); + } + grouped_manifests + } + + async fn merge_bin( + &self, + snapshot_id: i64, + file_io: FileIO, + manifest_bin: Vec, + mut writer: ManifestWriter, + ) -> Result { + for manifest_file in manifest_bin { + let manifest_file = manifest_file.load_manifest(&file_io).await?; + for manifest_entry in manifest_file.entries() { + if manifest_entry.status() == ManifestStatus::Deleted + && manifest_entry + .snapshot_id() + .is_some_and(|id| id == snapshot_id) + { + //only files deleted by this snapshot should be added to the new manifest + writer.add_delete_entry(manifest_entry.as_ref().clone())?; + } else if manifest_entry.status() == ManifestStatus::Added + && manifest_entry + .snapshot_id() + .is_some_and(|id| id == snapshot_id) + { + //added entries from this snapshot are still added, otherwise they should be existing + writer.add_entry(manifest_entry.as_ref().clone())?; + } else if manifest_entry.status() != ManifestStatus::Deleted { + // add all non-deleted files from the old manifest as existing files + writer.add_existing_entry(manifest_entry.as_ref().clone())?; + } + } + } + + writer.write_manifest_file().await + } + + async fn merge_group( + &self, + snapshot_produce: &mut SnapshotProducer<'_>, + first_manifest: &ManifestFile, + group_manifests: Vec, + ) -> Result> { + let packer: ListPacker = ListPacker::new(self.target_size_bytes); + let manifest_bins = + packer.pack(group_manifests, |manifest| manifest.manifest_length as u32); + + let manifest_merge_futures = manifest_bins + .into_iter() + .map(|manifest_bin| { + if manifest_bin.len() == 1 { + Ok(Box::pin(async { Ok(manifest_bin) }) + as Pin< + Box>> + Send>, + >) + } + // if the bin has the first manifest (the new data files or an appended manifest file) then only + // merge it if the number of manifests is above the minimum count. this is applied only to bins + // with an in-memory manifest so that large manifests don't prevent merging older groups. + else if manifest_bin + .iter() + .any(|manifest| manifest == first_manifest) + && manifest_bin.len() < self.min_count_to_merge as usize + { + Ok(Box::pin(async { Ok(manifest_bin) }) + as Pin< + Box>> + Send>, + >) + } else { + let writer = snapshot_produce.new_manifest_writer(self.content)?; + let snapshot_id = snapshot_produce.snapshot_id; + let file_io = snapshot_produce.table.file_io().clone(); + Ok((Box::pin(async move { + Ok(vec![ + self.merge_bin( + snapshot_id, + file_io, + manifest_bin, + writer, + ) + .await?, + ]) + })) + as Pin>> + Send>>) + } + }) + .collect::>> + Send>>>>>()?; + + let merged_bins: Vec> = + futures::future::join_all(manifest_merge_futures.into_iter()) + .await + .into_iter() + .collect::>>()?; + + Ok(merged_bins.into_iter().flatten().collect()) + } + + pub(crate) async fn merge_manifest( + &self, + snapshot_produce: &mut SnapshotProducer<'_>, + manifests: Vec, + ) -> Result> { + if manifests.is_empty() { + return Ok(manifests); + } + + let first_manifest = manifests[0].clone(); + + let group_manifests = self.group_by_spec(manifests); + + let mut merge_manifests = vec![]; + for (_spec_id, manifests) in group_manifests.into_iter().rev() { + merge_manifests.extend( + self.merge_group(snapshot_produce, &first_manifest, manifests) + .await?, + ); + } + + Ok(merge_manifests) + } +} diff --git a/crates/iceberg/src/utils.rs b/crates/iceberg/src/utils.rs index 00d3e69bd3..1a60c82a77 100644 --- a/crates/iceberg/src/utils.rs +++ b/crates/iceberg/src/utils.rs @@ -40,3 +40,148 @@ pub(crate) fn available_parallelism() -> NonZeroUsize { NonZeroUsize::new(DEFAULT_PARALLELISM).unwrap() }) } + +pub mod bin { + use std::iter::Iterator; + use std::marker::PhantomData; + + use itertools::Itertools; + + struct Bin { + bin_weight: u32, + target_weight: u32, + items: Vec, + } + + impl Bin { + pub fn new(target_weight: u32) -> Self { + Bin { + bin_weight: 0, + target_weight, + items: Vec::new(), + } + } + + pub fn can_add(&self, weight: u32) -> bool { + self.bin_weight + weight <= self.target_weight + } + + pub fn add(&mut self, item: T, weight: u32) { + self.bin_weight += weight; + self.items.push(item); + } + + pub fn into_vec(self) -> Vec { + self.items + } + } + + /// ListPacker help to pack item into bin of item. Each bin has close to + /// target_weight. + pub(crate) struct ListPacker { + target_weight: u32, + _marker: PhantomData, + } + + impl ListPacker { + pub fn new(target_weight: u32) -> Self { + ListPacker { + target_weight, + _marker: PhantomData, + } + } + + pub fn pack(&self, items: Vec, weight_func: F) -> Vec> + where F: Fn(&T) -> u32 { + let mut bins: Vec> = vec![]; + for item in items { + let cur_weight = weight_func(&item); + let addable_bin = + if let Some(bin) = bins.iter_mut().find(|bin| bin.can_add(cur_weight)) { + bin + } else { + bins.push(Bin::new(self.target_weight)); + bins.last_mut().unwrap() + }; + addable_bin.add(item, cur_weight); + } + + bins.into_iter().map(|bin| bin.into_vec()).collect_vec() + } + } + + #[cfg(test)] + mod tests { + use super::*; + + #[test] + fn test_list_packer_basic_packing() { + let packer = ListPacker::new(10); + let items = vec![3, 4, 5, 6, 2, 1]; + + let packed = packer.pack(items, |&x| x); + + assert_eq!(packed.len(), 3); + assert!(packed[0].iter().sum::() == 10); + assert!(packed[1].iter().sum::() == 5); + assert!(packed[2].iter().sum::() == 6); + } + + #[test] + fn test_list_packer_with_complex_items() { + #[derive(Debug, PartialEq)] + struct Item { + name: String, + size: u32, + } + + let packer = ListPacker::new(15); + let items = vec![ + Item { + name: "A".to_string(), + size: 7, + }, + Item { + name: "B".to_string(), + size: 8, + }, + Item { + name: "C".to_string(), + size: 5, + }, + Item { + name: "D".to_string(), + size: 6, + }, + ]; + + let packed = packer.pack(items, |item| item.size); + + assert_eq!(packed.len(), 2); + assert!(packed[0].iter().map(|x| x.size).sum::() <= 15); + assert!(packed[1].iter().map(|x| x.size).sum::() <= 15); + } + + #[test] + fn test_list_packer_single_large_item() { + let packer = ListPacker::new(10); + let items = vec![15, 5, 3]; + + let packed = packer.pack(items, |&x| x); + + assert_eq!(packed.len(), 2); + assert!(packed[0].contains(&15)); + assert!(packed[1].iter().sum::() <= 10); + } + + #[test] + fn test_list_packer_empty_input() { + let packer = ListPacker::new(10); + let items: Vec = vec![]; + + let packed = packer.pack(items, |&x| x); + + assert_eq!(packed.len(), 0); + } + } +} diff --git a/crates/integration_tests/testdata/docker-compose.yaml b/crates/integration_tests/testdata/docker-compose.yaml index cf0240d1a5..d9f4c30c71 100644 --- a/crates/integration_tests/testdata/docker-compose.yaml +++ b/crates/integration_tests/testdata/docker-compose.yaml @@ -22,6 +22,12 @@ services: rest: image: apache/iceberg-rest-fixture:1.10.0 environment: + - http_proxy= + - https_proxy= + - HTTP_PROXY= + - HTTPS_PROXY= + - NO_PROXY=localhost,127.0.0.1,minio + - no_proxy=localhost,127.0.0.1,minio - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password - AWS_REGION=us-east-1 @@ -42,6 +48,12 @@ services: minio: image: minio/minio:RELEASE.2025-05-24T17-08-30Z environment: + - http_proxy= + - https_proxy= + - HTTP_PROXY= + - HTTPS_PROXY= + - NO_PROXY=localhost,127.0.0.1,minio + - no_proxy=localhost,127.0.0.1,minio - MINIO_ROOT_USER=admin - MINIO_ROOT_PASSWORD=password - MINIO_DOMAIN=minio @@ -61,6 +73,12 @@ services: - minio image: minio/mc:RELEASE.2025-05-21T01-59-54Z environment: + - http_proxy= + - https_proxy= + - HTTP_PROXY= + - HTTPS_PROXY= + - NO_PROXY=localhost,127.0.0.1,minio + - no_proxy=localhost,127.0.0.1,minio - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password - AWS_REGION=us-east-1 @@ -77,6 +95,12 @@ services: - rest - minio environment: + - http_proxy= + - https_proxy= + - HTTP_PROXY= + - HTTPS_PROXY= + - NO_PROXY=localhost,127.0.0.1,minio + - no_proxy=localhost,127.0.0.1,minio - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password - AWS_REGION=us-east-1 diff --git a/crates/integration_tests/tests/shared_tests/merge_append_test.rs b/crates/integration_tests/tests/shared_tests/merge_append_test.rs new file mode 100644 index 0000000000..fb4ea363ff --- /dev/null +++ b/crates/integration_tests/tests/shared_tests/merge_append_test.rs @@ -0,0 +1,260 @@ +// 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. + +//! Integration tests for rest catalog. + +use std::collections::HashMap; +use std::sync::Arc; +use std::sync::atomic::{AtomicU64, Ordering}; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use iceberg::spec::{DataFile, NestedField, PrimitiveType, Schema, Type}; +use iceberg::table::Table; +use iceberg::transaction::{ + ApplyTransactionAction, MANIFEST_MERGE_ENABLED, MANIFEST_MIN_MERGE_COUNT, + MANIFEST_TARGET_SIZE_BYTES, Transaction, +}; +use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, CatalogBuilder, TableCreation}; +use iceberg_catalog_rest::{RestCatalog, RestCatalogBuilder}; +use parquet::file::properties::WriterProperties; + +use crate::get_shared_containers; + +static FILE_COUNTER: AtomicU64 = AtomicU64::new(0); +use crate::shared_tests::random_ns; + +async fn write_new_data_file(table: &Table) -> Vec { + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + // Use a short unique prefix each time so repeated calls don't produce identical paths + // while keeping file name length small to mimic original manifest sizing for merge logic. + let unique_prefix = format!("test{:04}", FILE_COUNTER.fetch_add(1, Ordering::Relaxed)); + let file_name_generator = + DefaultFileNameGenerator::new(unique_prefix, None, iceberg::spec::DataFileFormat::Parquet); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"); 100]); + let col2 = Int32Array::from(vec![Some(1); 100]); + let col3 = BooleanArray::from(vec![Some(true); 100]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + data_file_writer.close().await.unwrap() +} + +// Helper: create table with 3 single-entry manifests produced by fast append +async fn create_table_with_three_manifests() -> (RestCatalog, Table) { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .with_fields(vec![ + NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + let mut table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + for _ in 0..3 { + let data_file = write_new_data_file(&table).await; + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + } + (rest_catalog, table) +} + +// Scenario 1: Every manifest is packed into its own bin so merge is skipped (covers `manifest_bin.len()==1` branch) +#[tokio::test] +async fn test_merge_append_no_merge_each_single_bin() { + let (rest_catalog, mut table) = create_table_with_three_manifests().await; + let manifest_list_before = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list_before.entries().len(), 3); + + let snapshot_properties = HashMap::from([ + (MANIFEST_MERGE_ENABLED.to_string(), "true".to_string()), + (MANIFEST_MIN_MERGE_COUNT.to_string(), "4".to_string()), // original setting + (MANIFEST_TARGET_SIZE_BYTES.to_string(), "7000".to_string()), // small target -> each manifest isolated + ]); + + let data_file = write_new_data_file(&table).await; + let tx = Transaction::new(&table); + let merge_action = tx + .merge_append() + .set_snapshot_properties(snapshot_properties) + .add_data_files(data_file); + let tx = merge_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + + let manifest_list_after = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + // No merge => 3 (existing) + 1 (new) = 4 + assert_eq!(manifest_list_after.entries().len(), 4); + for entry in manifest_list_after.entries() { + let m = entry.load_manifest(table.file_io()).await.unwrap(); + assert_eq!( + m.entries().len(), + 1, + "each manifest should still have exactly one data file" + ); + } +} + +// Scenario 2: All manifests fit into one bin but merge is skipped because bin contains the first manifest and count < min_count_to_merge +#[tokio::test] +async fn test_merge_append_no_merge_min_count_protects() { + let (rest_catalog, mut table) = create_table_with_three_manifests().await; + let manifest_list_before = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list_before.entries().len(), 3); + + let snapshot_properties = HashMap::from([ + (MANIFEST_MERGE_ENABLED.to_string(), "true".to_string()), + (MANIFEST_MIN_MERGE_COUNT.to_string(), "10".to_string()), // larger than manifest count (4) + (MANIFEST_TARGET_SIZE_BYTES.to_string(), "500000".to_string()), // large target -> all in one bin + ]); + + let data_file = write_new_data_file(&table).await; + let tx = Transaction::new(&table); + let merge_action = tx + .merge_append() + .set_snapshot_properties(snapshot_properties) + .add_data_files(data_file); + let tx = merge_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + + let manifest_list_after = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + // len(4) < min_count(10) and bin contains first manifest => skip merge + assert_eq!(manifest_list_after.entries().len(), 4); +} + +// Scenario 3: Merge all manifests into a single manifest +#[tokio::test] +async fn test_merge_append_full_merge() { + let (rest_catalog, mut table) = create_table_with_three_manifests().await; + let manifest_list_before = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list_before.entries().len(), 3); + + let snapshot_properties = HashMap::from([ + (MANIFEST_MERGE_ENABLED.to_string(), "true".to_string()), + (MANIFEST_MIN_MERGE_COUNT.to_string(), "2".to_string()), // low threshold allows merge + (MANIFEST_TARGET_SIZE_BYTES.to_string(), "500000".to_string()), // large -> all in one bin + ]); + + let data_file = write_new_data_file(&table).await; + let tx = Transaction::new(&table); + let merge_action = tx + .merge_append() + .set_snapshot_properties(snapshot_properties) + .add_data_files(data_file); + let tx = merge_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + + let manifest_list_after = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + + // Expect all 4 single-file manifests merged into 1 manifest + assert_eq!( + manifest_list_after.entries().len(), + 1, + "should merge into a single manifest file" + ); + let merged_manifest = manifest_list_after.entries()[0] + .load_manifest(table.file_io()) + .await + .unwrap(); + assert_eq!( + merged_manifest.entries().len(), + 4, + "merged manifest should contain 4 data file entries" + ); +} diff --git a/crates/integration_tests/tests/shared_tests/mod.rs b/crates/integration_tests/tests/shared_tests/mod.rs index 70c041def2..da1ea9fa72 100644 --- a/crates/integration_tests/tests/shared_tests/mod.rs +++ b/crates/integration_tests/tests/shared_tests/mod.rs @@ -27,6 +27,7 @@ mod append_data_file_test; mod append_partition_data_file_test; mod conflict_commit_test; mod datafusion; +mod merge_append_test; mod read_evolved_schema; mod read_positional_deletes; mod remove_snapshots_test; From 3201d8c4e4f36297e8794c8d24bee1dcb7663007 Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 5 Nov 2025 17:29:22 +0800 Subject: [PATCH 07/33] feat: add process delete enrty in snapshot produce --------- Signed-off-by: xxchan Co-authored-by: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Co-authored-by: ZENOTME Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- crates/iceberg/src/transaction/append.rs | 2 + crates/iceberg/src/transaction/snapshot.rs | 84 +++++++++++++++++-- .../src/writer/file_writer/parquet_writer.rs | 3 +- .../shared_tests/remove_snapshots_test.rs | 35 +++++++- 4 files changed, 117 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index b77259c9f8..04e92748cb 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -418,6 +418,7 @@ mod tests { // check add data file with incompatible partition value let data_file = DataFileBuilder::default() + .partition_spec_id(0) .content(DataContentType::Data) .file_path("test/3.parquet".to_string()) .file_format(DataFileFormat::Parquet) @@ -440,6 +441,7 @@ mod tests { let action = tx.fast_append(); let data_file = DataFileBuilder::default() + .partition_spec_id(0) .content(DataContentType::Data) .file_path("test/3.parquet".to_string()) .file_format(DataFileFormat::Parquet) diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 414753b588..b5fb428181 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -41,7 +41,6 @@ const META_ROOT_PATH: &str = "metadata"; pub(crate) trait SnapshotProduceOperation: Send + Sync { fn operation(&self) -> Operation; - #[allow(unused)] fn delete_entries( &self, snapshot_produce: &SnapshotProducer, @@ -190,7 +189,11 @@ impl<'a> SnapshotProducer<'a> { snapshot_id } - fn new_manifest_writer(&mut self, content: ManifestContentType) -> Result { + fn new_manifest_writer( + &mut self, + content: ManifestContentType, + partition_spec_id: i32, + ) -> Result { let new_manifest_path = format!( "{}/{}/{}-m{}.{}", self.table.metadata().location(), @@ -207,7 +210,14 @@ impl<'a> SnapshotProducer<'a> { self.table.metadata().current_schema().clone(), self.table .metadata() - .default_partition_spec() + .partition_spec_by_id(partition_spec_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "Invalid partition spec id for new manifest writer", + ) + .with_context("partition spec id", partition_spec_id.to_string()) + })? .as_ref() .clone(), ); @@ -303,13 +313,72 @@ impl<'a> SnapshotProducer<'a> { builder.build() } }); - let mut writer = self.new_manifest_writer(manifest_content_type)?; + + let mut writer = self.new_manifest_writer( + manifest_content_type, + self.table.metadata().default_partition_spec_id(), + )?; for entry in manifest_entries { writer.add_entry(entry)?; } writer.write_manifest_file().await } + async fn write_delete_manifest( + &mut self, + deleted_entries: Vec, + ) -> Result> { + if deleted_entries.is_empty() { + return Ok(vec![]); + } + + // Group deleted entries by spec_id + let mut partition_groups = HashMap::new(); + for entry in deleted_entries { + partition_groups + .entry(entry.data_file().partition_spec_id) + .or_insert_with(Vec::new) + .push(entry); + } + + // Write a delete manifest per spec_id group + let mut deleted_manifests = Vec::new(); + for (spec_id, entries) in partition_groups { + let mut data_file_writer: Option = None; + let mut delete_file_writer: Option = None; + for entry in entries { + match entry.content_type() { + DataContentType::Data => { + if data_file_writer.is_none() { + data_file_writer = + Some(self.new_manifest_writer(ManifestContentType::Data, spec_id)?); + } + data_file_writer.as_mut().unwrap().add_delete_entry(entry)?; + } + DataContentType::EqualityDeletes | DataContentType::PositionDeletes => { + if delete_file_writer.is_none() { + delete_file_writer = Some( + self.new_manifest_writer(ManifestContentType::Deletes, spec_id)?, + ); + } + delete_file_writer + .as_mut() + .unwrap() + .add_delete_entry(entry)?; + } + } + } + if let Some(writer) = data_file_writer { + deleted_manifests.push(writer.write_manifest_file().await?); + } + if let Some(writer) = delete_file_writer { + deleted_manifests.push(writer.write_manifest_file().await?); + } + } + + Ok(deleted_manifests) + } + async fn manifest_file( &mut self, snapshot_produce_operation: &OP, @@ -343,6 +412,11 @@ impl<'a> SnapshotProducer<'a> { manifest_files.push(added_manifest); } + let delete_manifests = self + .write_delete_manifest(snapshot_produce_operation.delete_entries(self).await?) + .await?; + manifest_files.extend(delete_manifests); + manifest_process .process_manifests(self, manifest_files) .await @@ -643,7 +717,7 @@ impl MergeManifestManager { Box>> + Send>, >) } else { - let writer = snapshot_produce.new_manifest_writer(self.content)?; + let writer = snapshot_produce.new_manifest_writer(self.content, snapshot_produce.table.metadata().default_partition_spec_id())?; let snapshot_id = snapshot_produce.snapshot_id; let file_io = snapshot_produce.table.file_io().clone(); Ok((Box::pin(async move { diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 3e9d1715c9..0c179ac3c9 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -428,6 +428,7 @@ impl ParquetWriter { .file_path(file_path) .file_format(DataFileFormat::Parquet) .partition(Struct::empty()) + .partition_spec_id(0) .record_count(metadata.file_metadata().num_rows() as u64) .file_size_in_bytes(written_size as u64) .column_sizes(column_sizes) @@ -1571,7 +1572,7 @@ mod tests { // .next() // .unwrap() // .content(crate::spec::DataContentType::Data) - // .partition(Struct::empty()) + // .partition(Struct::empty()).partition_spec_id(0) // .build() // .unwrap(); // assert_eq!( diff --git a/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs b/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs index 7546575f8e..ac94d2b3ac 100644 --- a/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs +++ b/crates/integration_tests/tests/shared_tests/remove_snapshots_test.rs @@ -30,6 +30,7 @@ use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; +use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::file::properties::WriterProperties; use crate::get_shared_containers; @@ -81,12 +82,44 @@ async fn test_expire_snapshots_by_count() { file_name_generator, ); let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + // check parquet file schema + let content = table + .file_io() + .new_input(data_file[0].file_path()) + .unwrap() + .read() + .await + .unwrap(); + let parquet_reader = parquet::arrow::arrow_reader::ArrowReaderMetadata::load( + &content, + ArrowReaderOptions::default(), + ) + .unwrap(); + let field_ids: Vec = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.self_type().get_basic_info().id()) + .collect(); + assert_eq!(field_ids, vec![1, 2, 3]); // commit result for i in 0..10 { // Create a new data file writer for each iteration let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); - // Create different data for each iteration let col1 = StringArray::from(vec![ Some(format!("foo_{}", i)), From cee1fa7fa03e75d8a2505f82cc99b121ee9c5b51 Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 5 Nov 2025 17:34:26 +0800 Subject: [PATCH 08/33] fix: cherry-pick #27 Co-authored-by: Dylan --- .../src/arrow/caching_delete_file_loader.rs | 52 ++++++-- .../iceberg/src/arrow/delete_file_loader.rs | 13 +- crates/iceberg/src/arrow/delete_filter.rs | 51 ++++---- crates/iceberg/src/arrow/reader.rs | 122 +++++++++++++++--- crates/iceberg/src/delete_file_index.rs | 30 +++-- crates/iceberg/src/scan/context.rs | 3 + crates/iceberg/src/scan/mod.rs | 8 ++ crates/iceberg/src/scan/task.rs | 34 ++++- crates/iceberg/src/spec/manifest/mod.rs | 3 +- 9 files changed, 243 insertions(+), 73 deletions(-) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index 8a3ab3a955..28d1655a8e 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -30,7 +30,7 @@ use crate::delete_vector::DeleteVector; use crate::expr::Predicate::AlwaysTrue; use crate::expr::{Predicate, Reference}; use crate::io::FileIO; -use crate::scan::{ArrowRecordBatchStream, FileScanTaskDeleteFile}; +use crate::scan::{ArrowRecordBatchStream, FileScanTask}; use crate::spec::{ DataContentType, Datum, ListType, MapType, NestedField, NestedFieldRef, PartnerAccessor, PrimitiveType, Schema, SchemaRef, SchemaWithPartnerVisitor, StructType, Type, @@ -138,7 +138,7 @@ impl CachingDeleteFileLoader { /// ``` pub(crate) fn load_deletes( &self, - delete_file_entries: &[FileScanTaskDeleteFile], + delete_file_entries: &[FileScanTask], schema: SchemaRef, ) -> Receiver> { let (tx, rx) = channel(); @@ -204,32 +204,32 @@ impl CachingDeleteFileLoader { } async fn load_file_for_task( - task: &FileScanTaskDeleteFile, + task: &FileScanTask, basic_delete_file_loader: BasicDeleteFileLoader, del_filter: DeleteFilter, schema: SchemaRef, ) -> Result { - match task.file_type { + match task.data_file_content { DataContentType::PositionDeletes => Ok(DeleteFileContext::PosDels( basic_delete_file_loader - .parquet_to_batch_stream(&task.file_path) + .parquet_to_batch_stream(task.data_file_path()) .await?, )), DataContentType::EqualityDeletes => { - let Some(notify) = del_filter.try_start_eq_del_load(&task.file_path) else { + let Some(notify) = del_filter.try_start_eq_del_load(task.data_file_path()) else { return Ok(DeleteFileContext::ExistingEqDel); }; let (sender, receiver) = channel(); - del_filter.insert_equality_delete(&task.file_path, receiver); + del_filter.insert_equality_delete(task.data_file_path(), receiver); // Per the Iceberg spec, evolve schema for equality deletes but only for the // equality_ids columns, not all table columns. let equality_ids_vec = task.equality_ids.clone().unwrap(); let evolved_stream = BasicDeleteFileLoader::evolve_schema( basic_delete_file_loader - .parquet_to_batch_stream(&task.file_path) + .parquet_to_batch_stream(task.data_file_path()) .await?, schema, &equality_ids_vec, @@ -552,6 +552,7 @@ mod tests { use super::*; use crate::arrow::delete_filter::tests::setup; + use crate::scan::FileScanTaskDeleteFile; #[tokio::test] async fn test_delete_file_loader_parse_equality_deletes() { @@ -871,6 +872,36 @@ mod tests { equality_ids: Some(vec![2, 3]), // Only use field IDs that exist in both schemas }; + let pos_del_scan_task = FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: pos_del.file_path, + data_file_content: DataContentType::PositionDeletes, + data_file_format: DataFileFormat::Parquet, + schema: data_file_schema.clone(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + sequence_number: 0, + equality_ids: None, + }; + + let eq_del_scan_task = FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: eq_del.file_path.clone(), + data_file_content: DataContentType::EqualityDeletes, + data_file_format: DataFileFormat::Parquet, + schema: data_file_schema.clone(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + sequence_number: 0, + equality_ids: eq_del.equality_ids.clone(), + }; + let file_scan_task = FileScanTask { start: 0, length: 0, @@ -880,7 +911,10 @@ mod tests { schema: data_file_schema.clone(), project_field_ids: vec![2, 3], predicate: None, - deletes: vec![pos_del, eq_del], + deletes: vec![pos_del_scan_task, eq_del_scan_task], + sequence_number: 0, + data_file_content: DataContentType::Data, + equality_ids: None, }; // Load the deletes - should handle both types without error diff --git a/crates/iceberg/src/arrow/delete_file_loader.rs b/crates/iceberg/src/arrow/delete_file_loader.rs index c0b1392dc6..8018e786f6 100644 --- a/crates/iceberg/src/arrow/delete_file_loader.rs +++ b/crates/iceberg/src/arrow/delete_file_loader.rs @@ -120,6 +120,7 @@ mod tests { use super::*; use crate::arrow::delete_filter::tests::setup; + use crate::spec::DataContentType; #[tokio::test] async fn test_basic_delete_file_loader_read_delete_file() { @@ -134,11 +135,15 @@ mod tests { let file_scan_tasks = setup(table_location); + let delete_task = FileScanTaskDeleteFile { + file_path: file_scan_tasks[0].deletes[0].data_file_path.clone(), + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }; + let result = delete_file_loader - .read_delete_file( - &file_scan_tasks[0].deletes[0], - file_scan_tasks[0].schema_ref(), - ) + .read_delete_file(&delete_task, file_scan_tasks[0].schema_ref()) .await .unwrap(); diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index 4250974bcd..bc0e5d1aba 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -24,7 +24,7 @@ use tokio::sync::oneshot::Receiver; use crate::delete_vector::DeleteVector; use crate::expr::Predicate::AlwaysTrue; use crate::expr::{Bind, BoundPredicate, Predicate}; -use crate::scan::{FileScanTask, FileScanTaskDeleteFile}; +use crate::scan::FileScanTask; use crate::spec::DataContentType; use crate::{Error, ErrorKind, Result}; @@ -122,14 +122,14 @@ impl DeleteFilter { } let Some(predicate) = self - .get_equality_delete_predicate_for_delete_file_path(&delete.file_path) + .get_equality_delete_predicate_for_delete_file_path(delete.data_file_path()) .await else { return Err(Error::new( ErrorKind::Unexpected, format!( "Missing predicate for equality delete file '{}'", - delete.file_path + delete.data_file_path() ), )); }; @@ -192,8 +192,8 @@ impl DeleteFilter { } } -pub(crate) fn is_equality_delete(f: &FileScanTaskDeleteFile) -> bool { - matches!(f.file_type, DataContentType::EqualityDeletes) +pub(crate) fn is_equality_delete(f: &FileScanTask) -> bool { + matches!(f.data_file_content, DataContentType::EqualityDeletes) } #[cfg(test)] @@ -309,24 +309,19 @@ pub(crate) mod tests { writer.close().unwrap(); } - let pos_del_1 = FileScanTaskDeleteFile { - file_path: format!("{}/pos-del-1.parquet", table_location.to_str().unwrap()), - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, - equality_ids: None, - }; - - let pos_del_2 = FileScanTaskDeleteFile { - file_path: format!("{}/pos-del-2.parquet", table_location.to_str().unwrap()), - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, - equality_ids: None, - }; - - let pos_del_3 = FileScanTaskDeleteFile { - file_path: format!("{}/pos-del-3.parquet", table_location.to_str().unwrap()), - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, + // Helper to build a positional delete task with minimal fields + let make_pos_del_task = |n: u8| FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: format!("{}/pos-del-{}.parquet", table_location.to_str().unwrap(), n), + data_file_content: DataContentType::PositionDeletes, + data_file_format: DataFileFormat::Parquet, + schema: data_file_schema.clone(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + sequence_number: 0, equality_ids: None, }; @@ -336,22 +331,28 @@ pub(crate) mod tests { length: 0, record_count: None, data_file_path: format!("{}/1.parquet", table_location.to_str().unwrap()), + data_file_content: DataContentType::Data, data_file_format: DataFileFormat::Parquet, schema: data_file_schema.clone(), project_field_ids: vec![], predicate: None, - deletes: vec![pos_del_1, pos_del_2.clone()], + deletes: vec![make_pos_del_task(1), make_pos_del_task(2)], + sequence_number: 0, + equality_ids: None, }, FileScanTask { start: 0, length: 0, record_count: None, data_file_path: format!("{}/2.parquet", table_location.to_str().unwrap()), + data_file_content: DataContentType::Data, data_file_format: DataFileFormat::Parquet, schema: data_file_schema.clone(), project_field_ids: vec![], predicate: None, - deletes: vec![pos_del_3], + deletes: vec![make_pos_del_task(3)], + sequence_number: 0, + equality_ids: None, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index fed8f19c05..1baa7675a2 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -55,7 +55,7 @@ use crate::expr::visitors::row_group_metrics_evaluator::RowGroupMetricsEvaluator use crate::expr::{BoundPredicate, BoundReference}; use crate::io::{FileIO, FileMetadata, FileRead}; use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskStream}; -use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type}; +use crate::spec::{DataContentType, Datum, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; @@ -370,13 +370,16 @@ impl ArrowReader { // 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 { + let record_batch_stream = record_batch_stream_builder.build()?.map(move |batch| { + if matches!(task.data_file_content, DataContentType::PositionDeletes) { + Ok(batch?) + } else { + match batch { Ok(batch) => record_batch_transformer.process_record_batch(batch), Err(err) => Err(err.into()), - }); + } + } + }); Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream) } @@ -1644,7 +1647,7 @@ mod tests { use crate::expr::visitors::bound_predicate_visitor::visit; use crate::expr::{Bind, Predicate, Reference}; use crate::io::FileIO; - use crate::scan::{FileScanTask, FileScanTaskDeleteFile, FileScanTaskStream}; + use crate::scan::{FileScanTask, FileScanTaskStream}; use crate::spec::{ DataContentType, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, Type, }; @@ -1665,6 +1668,21 @@ mod tests { ) } + fn position_del_schema() -> SchemaRef { + Arc::new( + Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![]) + .with_fields(vec![ + NestedField::required(1, "file_path", Type::Primitive(PrimitiveType::String)) + .into(), + NestedField::required(2, "pos", Type::Primitive(PrimitiveType::Long)).into(), + ]) + .build() + .unwrap(), + ) + } + #[test] fn test_collect_field_id() { let schema = table_schema_simple(); @@ -1943,11 +1961,14 @@ message schema { length: 0, record_count: None, data_file_path: format!("{table_location}/1.parquet"), + data_file_content: DataContentType::Data, data_file_format: DataFileFormat::Parquet, schema: schema.clone(), project_field_ids: vec![1], predicate: Some(predicate.bind(schema, true).unwrap()), deletes: vec![], + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2266,6 +2287,9 @@ message schema { project_field_ids: vec![1], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, }; // Task 2: read the second and third row groups @@ -2279,6 +2303,9 @@ message schema { project_field_ids: vec![1], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, }; let tasks1 = Box::pin(futures::stream::iter(vec![Ok(task1)])) as FileScanTaskStream; @@ -2403,6 +2430,9 @@ message schema { project_field_ids: vec![1, 2], // Request both columns 'a' and 'b' predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2565,12 +2595,23 @@ message schema { 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, + deletes: vec![FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: delete_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::PositionDeletes, + sequence_number: 0, equality_ids: None, }], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -2780,12 +2821,23 @@ message schema { 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, + deletes: vec![FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: delete_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: position_del_schema(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::PositionDeletes, + sequence_number: 0, equality_ids: None, }], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -2988,12 +3040,23 @@ message schema { 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, + deletes: vec![FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: delete_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: position_del_schema(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::PositionDeletes, + sequence_number: 0, equality_ids: None, }], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3094,6 +3157,9 @@ message schema { project_field_ids: vec![1, 2], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3188,6 +3254,9 @@ message schema { project_field_ids: vec![1, 3], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3271,6 +3340,9 @@ message schema { project_field_ids: vec![1, 2, 3], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3368,6 +3440,9 @@ message schema { project_field_ids: vec![1, 2], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3494,6 +3569,9 @@ message schema { project_field_ids: vec![1, 2], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3587,6 +3665,9 @@ message schema { project_field_ids: vec![1, 5, 2], predicate: None, deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3693,6 +3774,9 @@ message schema { project_field_ids: vec![1, 2, 3], predicate: Some(predicate.bind(schema, true).unwrap()), deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, })] .into_iter(), )) as FileScanTaskStream; diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index 4f6fd28483..8692e8acf3 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -24,7 +24,7 @@ use futures::channel::mpsc::{Sender, channel}; use tokio::sync::Notify; use crate::runtime::spawn; -use crate::scan::{DeleteFileContext, FileScanTaskDeleteFile}; +use crate::scan::{DeleteFileContext, FileScanTask}; use crate::spec::{DataContentType, DataFile, Struct}; /// Index of delete files @@ -86,7 +86,7 @@ impl DeleteFileIndex { &self, data_file: &DataFile, seq_num: Option, - ) -> Vec { + ) -> Vec { let notifier = { let guard = self.state.read().unwrap(); match *guard { @@ -165,7 +165,7 @@ impl PopulatedDeleteFileIndex { &self, data_file: &DataFile, seq_num: Option, - ) -> Vec { + ) -> Vec { let mut results = vec![]; self.global_equality_deletes @@ -219,7 +219,7 @@ mod tests { use super::*; use crate::spec::{ DataContentType, DataFileBuilder, DataFileFormat, Literal, ManifestEntry, ManifestStatus, - Struct, + Schema, Struct, }; #[test] @@ -241,6 +241,8 @@ mod tests { .map(|entry| DeleteFileContext { manifest_entry: entry.into(), partition_spec_id: 0, + snapshot_schema: Arc::new(Schema::builder().with_schema_id(1).build().unwrap()), // hack + field_ids: Arc::new(vec![]), // hack }) .collect(); @@ -263,7 +265,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&data_file, Some(4)); let actual_paths_to_apply_for_seq_4: Vec = delete_files_to_apply_for_seq_4 .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( @@ -276,7 +278,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&data_file, Some(5)); let actual_paths_to_apply_for_seq_5: Vec = delete_files_to_apply_for_seq_5 .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( actual_paths_to_apply_for_seq_5, @@ -288,7 +290,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&data_file, Some(6)); let actual_paths_to_apply_for_seq_6: Vec = delete_files_to_apply_for_seq_6 .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( actual_paths_to_apply_for_seq_6, @@ -304,7 +306,7 @@ mod tests { let actual_paths_to_apply_for_partitioned_file: Vec = delete_files_to_apply_for_partitioned_file .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( actual_paths_to_apply_for_partitioned_file, @@ -333,6 +335,8 @@ mod tests { .map(|entry| DeleteFileContext { manifest_entry: entry.into(), partition_spec_id: spec_id, + snapshot_schema: Arc::new(Schema::builder().with_schema_id(1).build().unwrap()), // hack + field_ids: Arc::new(vec![]), // hack }) .collect(); @@ -356,7 +360,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(4)); let actual_paths_to_apply_for_seq_4: Vec = delete_files_to_apply_for_seq_4 .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( @@ -369,7 +373,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(5)); let actual_paths_to_apply_for_seq_5: Vec = delete_files_to_apply_for_seq_5 .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( actual_paths_to_apply_for_seq_5, @@ -381,7 +385,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(6)); let actual_paths_to_apply_for_seq_6: Vec = delete_files_to_apply_for_seq_6 .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert_eq!( actual_paths_to_apply_for_seq_6, @@ -396,7 +400,7 @@ mod tests { let actual_paths_to_apply_for_different_partition: Vec = delete_files_to_apply_for_different_partition .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert!(actual_paths_to_apply_for_different_partition.is_empty()); @@ -407,7 +411,7 @@ mod tests { let actual_paths_to_apply_for_different_spec: Vec = delete_files_to_apply_for_different_spec .into_iter() - .map(|file| file.file_path) + .map(|file| file.data_file_path) .collect(); assert!(actual_paths_to_apply_for_different_spec.is_empty()); } diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index a70b77982b..36d1d02e3d 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -127,6 +127,7 @@ impl ManifestEntryContext { record_count: Some(self.manifest_entry.record_count()), data_file_path: self.manifest_entry.file_path().to_string(), + data_file_content: self.manifest_entry.data_file().content_type(), data_file_format: self.manifest_entry.file_format(), schema: self.snapshot_schema, @@ -136,6 +137,8 @@ impl ManifestEntryContext { .map(|x| x.as_ref().snapshot_bound_predicate.clone()), deletes, + sequence_number: self.manifest_entry.sequence_number().unwrap_or(0), + equality_ids: self.manifest_entry.data_file().equality_ids(), }) } } diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 9bb48056f0..57e9a8e331 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -577,6 +577,8 @@ impl TableScan { .send(DeleteFileContext { manifest_entry: manifest_entry_context.manifest_entry.clone(), partition_spec_id: manifest_entry_context.partition_spec_id, + snapshot_schema: manifest_entry_context.snapshot_schema.clone(), + field_ids: manifest_entry_context.field_ids.clone(), }) .await?; @@ -1815,8 +1817,11 @@ pub mod tests { predicate: None, schema: schema.clone(), record_count: Some(100), + data_file_content: DataContentType::Data, data_file_format: DataFileFormat::Parquet, deletes: vec![], + sequence_number: 0, + equality_ids: None, }; test_fn(task); @@ -1829,8 +1834,11 @@ pub mod tests { predicate: Some(BoundPredicate::AlwaysTrue), schema, record_count: None, + data_file_content: DataContentType::Data, data_file_format: DataFileFormat::Avro, deletes: vec![], + sequence_number: 0, + equality_ids: None, }; test_fn(task); } diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 32fe3ae309..cf0b9e3692 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use futures::stream::BoxStream; use serde::{Deserialize, Serialize}; @@ -41,6 +43,9 @@ pub struct FileScanTask { /// The data file path corresponding to the task. pub data_file_path: String, + /// The content type of the file to scan. + pub data_file_content: DataContentType, + /// The format of the file to scan. pub data_file_format: DataFileFormat, @@ -53,7 +58,11 @@ pub struct FileScanTask { pub predicate: Option, /// The list of delete files that may need to be applied to this data file - pub deletes: Vec, + pub deletes: Vec, + /// sequence number + pub sequence_number: i64, + /// equality ids + pub equality_ids: Option>, } impl FileScanTask { @@ -87,6 +96,8 @@ impl FileScanTask { pub(crate) struct DeleteFileContext { pub(crate) manifest_entry: ManifestEntryRef, pub(crate) partition_spec_id: i32, + pub(crate) snapshot_schema: SchemaRef, + pub(crate) field_ids: Arc>, } impl From<&DeleteFileContext> for FileScanTaskDeleteFile { @@ -100,6 +111,27 @@ impl From<&DeleteFileContext> for FileScanTaskDeleteFile { } } +impl From<&DeleteFileContext> for FileScanTask { + fn from(ctx: &DeleteFileContext) -> Self { + FileScanTask { + start: 0, + length: ctx.manifest_entry.file_size_in_bytes(), + record_count: Some(ctx.manifest_entry.record_count()), + + data_file_path: ctx.manifest_entry.file_path().to_string(), + data_file_content: ctx.manifest_entry.content_type(), + data_file_format: ctx.manifest_entry.file_format(), + + schema: ctx.snapshot_schema.clone(), + project_field_ids: ctx.field_ids.to_vec(), + predicate: None, + deletes: vec![], + sequence_number: ctx.manifest_entry.sequence_number().unwrap_or(0), + equality_ids: ctx.manifest_entry.data_file().equality_ids(), + } + } +} + /// A task to scan part of file. #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct FileScanTaskDeleteFile { diff --git a/crates/iceberg/src/spec/manifest/mod.rs b/crates/iceberg/src/spec/manifest/mod.rs index 5a8c1f79c3..f01a2fff07 100644 --- a/crates/iceberg/src/spec/manifest/mod.rs +++ b/crates/iceberg/src/spec/manifest/mod.rs @@ -1112,8 +1112,7 @@ mod tests { // Create a partition spec let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(1) - .add_partition_field("id", "id_partition", Transform::Identity) - .unwrap() + // No partition fields for this serialization round-trip test .build() .unwrap(); From 494ca9052ee2f2c2ec9f8aeb243bab9bec919472 Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 5 Nov 2025 17:35:21 +0800 Subject: [PATCH 09/33] chore: pick public function generate_unique_snapshot_id for exactly once sink (#82) --- crates/iceberg/src/spec/manifest/_serde.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/manifest/_serde.rs b/crates/iceberg/src/spec/manifest/_serde.rs index 68566bba49..f677ed6a85 100644 --- a/crates/iceberg/src/spec/manifest/_serde.rs +++ b/crates/iceberg/src/spec/manifest/_serde.rs @@ -97,7 +97,7 @@ impl ManifestEntryV1 { } #[serde_as] -#[derive(Serialize, Deserialize)] +#[derive(Serialize, Deserialize, Clone)] /// Data file pub struct DataFileSerde { #[serde(default)] From a2b6cc4c6d2052325ccc764ed4c15e5f83e62220 Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 5 Nov 2025 17:46:31 +0800 Subject: [PATCH 10/33] feat(iceberg): rewrite files action (#47) (#86) * feat(iceberg): introduce rewrite files action * fix(iceberg): add test * fix test --- crates/iceberg/src/spec/snapshot_summary.rs | 1 + crates/iceberg/src/transaction/append.rs | 22 +- crates/iceberg/src/transaction/mod.rs | 21 +- .../iceberg/src/transaction/rewrite_files.rs | 321 ++++++++++++++++ crates/iceberg/src/transaction/snapshot.rs | 27 +- .../tests/shared_tests/mod.rs | 1 + .../tests/shared_tests/rewrite_files_test.rs | 359 ++++++++++++++++++ 7 files changed, 736 insertions(+), 16 deletions(-) create mode 100644 crates/iceberg/src/transaction/rewrite_files.rs create mode 100644 crates/integration_tests/tests/shared_tests/rewrite_files_test.rs diff --git a/crates/iceberg/src/spec/snapshot_summary.rs b/crates/iceberg/src/spec/snapshot_summary.rs index 4cd3715e06..57b36522e4 100644 --- a/crates/iceberg/src/spec/snapshot_summary.rs +++ b/crates/iceberg/src/spec/snapshot_summary.rs @@ -339,6 +339,7 @@ pub(crate) fn update_snapshot_summaries( if summary.operation != Operation::Append && summary.operation != Operation::Overwrite && summary.operation != Operation::Delete + && summary.operation != Operation::Replace { return Err(Error::new( ErrorKind::DataInvalid, diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index 04e92748cb..f6aea7263e 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -21,6 +21,11 @@ use std::sync::Arc; use async_trait::async_trait; use uuid::Uuid; +use super::{ + MANIFEST_MERGE_ENABLED, MANIFEST_MERGE_ENABLED_DEFAULT, MANIFEST_MIN_MERGE_COUNT, + MANIFEST_MIN_MERGE_COUNT_DEFAULT, MANIFEST_TARGET_SIZE_BYTES, + MANIFEST_TARGET_SIZE_BYTES_DEFAULT, +}; use crate::error::Result; use crate::spec::{DataFile, ManifestEntry, ManifestFile, Operation}; use crate::table::Table; @@ -30,16 +35,6 @@ use crate::transaction::snapshot::{ use crate::transaction::{ActionCommit, TransactionAction}; use crate::{Error, ErrorKind}; -/// Target size of manifest file when merging manifests. -pub const MANIFEST_TARGET_SIZE_BYTES: &str = "commit.manifest.target-size-bytes"; -const MANIFEST_TARGET_SIZE_BYTES_DEFAULT: u32 = 8 * 1024 * 1024; // 8 MB -/// Minimum number of manifests to merge. -pub const MANIFEST_MIN_MERGE_COUNT: &str = "commit.manifest.min-count-to-merge"; -const MANIFEST_MIN_MERGE_COUNT_DEFAULT: u32 = 100; -/// Whether allow to merge manifests. -pub const MANIFEST_MERGE_ENABLED: &str = "commit.manifest-merge.enabled"; -const MANIFEST_MERGE_ENABLED_DEFAULT: bool = false; - /// FastAppendAction is a transaction action for fast append data files to the table. pub struct FastAppendAction { check_duplicate: bool, @@ -138,6 +133,8 @@ impl TransactionAction for FastAppendAction { self.snapshot_properties.clone(), self.added_data_files.clone(), self.added_delete_files.clone(), + vec![], + vec![], ); // validate added files @@ -177,7 +174,7 @@ impl SnapshotProduceOperation for FastAppendOperation { async fn existing_manifest( &self, - snapshot_produce: &SnapshotProducer<'_>, + snapshot_produce: &mut SnapshotProducer<'_>, ) -> Result> { let Some(snapshot) = snapshot_produce.table.metadata().current_snapshot() else { return Ok(vec![]); @@ -218,7 +215,6 @@ pub struct MergeAppendAction { } impl MergeAppendAction { - #[allow(clippy::too_many_arguments)] pub(crate) fn new() -> Self { Self { target_size_bytes: MANIFEST_TARGET_SIZE_BYTES_DEFAULT, @@ -289,6 +285,8 @@ impl TransactionAction for MergeAppendAction { self.snapshot_properties.clone(), self.added_data_files.clone(), self.added_delete_files.clone(), + vec![], + vec![], ); // validate added files diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 7b6a1f6b7c..0a8058c6a8 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -55,6 +55,7 @@ mod action; pub use action::*; mod append; mod remove_snapshots; +mod rewrite_files; mod snapshot; mod sort_order; mod update_location; @@ -65,9 +66,9 @@ mod upgrade_format_version; use std::sync::Arc; use std::time::Duration; -pub use append::{MANIFEST_MERGE_ENABLED, MANIFEST_MIN_MERGE_COUNT, MANIFEST_TARGET_SIZE_BYTES}; use backon::{BackoffBuilder, ExponentialBackoff, ExponentialBuilder, RetryableWithContext}; use remove_snapshots::RemoveSnapshotAction; +use rewrite_files::RewriteFilesAction; use crate::error::Result; use crate::spec::TableProperties; @@ -80,6 +81,19 @@ use crate::transaction::update_statistics::UpdateStatisticsAction; use crate::transaction::upgrade_format_version::UpgradeFormatVersionAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; +/// Target size of manifest file when merging manifests. +pub const MANIFEST_TARGET_SIZE_BYTES: &str = "commit.manifest.target-size-bytes"; +/// This is the default value for `MANIFEST_TARGET_SIZE_BYTES`. +pub const MANIFEST_TARGET_SIZE_BYTES_DEFAULT: u32 = 8 * 1024 * 1024; // 8 MB +/// Minimum number of manifests to merge. +pub const MANIFEST_MIN_MERGE_COUNT: &str = "commit.manifest.min-count-to-merge"; +/// This is the default value for `MANIFEST_MIN_MERGE_COUNT`. +pub const MANIFEST_MIN_MERGE_COUNT_DEFAULT: u32 = 100; +/// Whether allow to merge manifests. +pub const MANIFEST_MERGE_ENABLED: &str = "commit.manifest-merge.enabled"; +/// This is the default value for `MANIFEST_MERGE_ENABLED`. +pub const MANIFEST_MERGE_ENABLED_DEFAULT: bool = false; + /// Table transaction. #[derive(Clone)] pub struct Transaction { @@ -168,6 +182,11 @@ impl Transaction { UpdateStatisticsAction::new() } + /// Creates rewrite files action. + pub fn rewrite_files(self) -> RewriteFilesAction { + RewriteFilesAction::new() + } + /// Commit transaction. pub async fn commit(self, catalog: &dyn Catalog) -> Result { if self.actions.is_empty() { diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs new file mode 100644 index 0000000000..93dc356755 --- /dev/null +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -0,0 +1,321 @@ +// 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, HashSet}; +use std::sync::Arc; + +use uuid::Uuid; + +use super::snapshot::{ + DefaultManifestProcess, MergeManifestProcess, SnapshotProduceOperation, SnapshotProducer, +}; +use super::{ + MANIFEST_MERGE_ENABLED, MANIFEST_MERGE_ENABLED_DEFAULT, MANIFEST_MIN_MERGE_COUNT, + MANIFEST_MIN_MERGE_COUNT_DEFAULT, MANIFEST_TARGET_SIZE_BYTES, + MANIFEST_TARGET_SIZE_BYTES_DEFAULT, +}; +use crate::error::Result; +use crate::spec::{ + DataContentType, DataFile, ManifestContentType, ManifestEntry, ManifestFile, ManifestStatus, + Operation, +}; +use crate::table::Table; +use crate::transaction::{ActionCommit, TransactionAction}; + +/// Transaction action for rewriting files. +pub struct RewriteFilesAction { + // snapshot_produce_action: SnapshotProduceAction<'a>, + target_size_bytes: u32, + min_count_to_merge: u32, + merge_enabled: bool, + + check_duplicate: bool, + // below are properties used to create SnapshotProducer when commit + commit_uuid: Option, + key_metadata: Option>, + snapshot_properties: HashMap, + added_data_files: Vec, + added_delete_files: Vec, + removed_data_files: Vec, + removed_delete_files: Vec, + snapshot_id: Option, +} + +struct RewriteFilesOperation; + +impl RewriteFilesAction { + pub fn new() -> Self { + Self { + target_size_bytes: MANIFEST_TARGET_SIZE_BYTES_DEFAULT, + min_count_to_merge: MANIFEST_MIN_MERGE_COUNT_DEFAULT, + merge_enabled: MANIFEST_MERGE_ENABLED_DEFAULT, + check_duplicate: true, + commit_uuid: None, + key_metadata: None, + snapshot_properties: HashMap::new(), + added_data_files: Vec::new(), + added_delete_files: Vec::new(), + removed_data_files: Vec::new(), + removed_delete_files: Vec::new(), + snapshot_id: None, + } + } + + /// Add data files to the snapshot. + pub fn add_data_files(mut self, data_files: impl IntoIterator) -> Self { + for file in data_files { + match file.content_type() { + DataContentType::Data => self.added_data_files.push(file), + DataContentType::PositionDeletes | DataContentType::EqualityDeletes => { + self.added_delete_files.push(file) + } + } + } + + self + } + + /// Add remove files to the snapshot. + pub fn delete_files(mut self, remove_data_files: impl IntoIterator) -> Self { + for file in remove_data_files { + match file.content_type() { + DataContentType::Data => self.removed_data_files.push(file), + DataContentType::PositionDeletes | DataContentType::EqualityDeletes => { + self.removed_delete_files.push(file) + } + } + } + + self + } + + pub fn set_snapshot_properties(&mut self, properties: HashMap) -> &mut Self { + let target_size_bytes: u32 = properties + .get(MANIFEST_TARGET_SIZE_BYTES) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + let min_count_to_merge: u32 = properties + .get(MANIFEST_MIN_MERGE_COUNT) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MIN_MERGE_COUNT_DEFAULT); + let merge_enabled = properties + .get(MANIFEST_MERGE_ENABLED) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MERGE_ENABLED_DEFAULT); + + self.target_size_bytes = target_size_bytes; + self.min_count_to_merge = min_count_to_merge; + self.merge_enabled = merge_enabled; + self.snapshot_properties = properties; + self + } + + /// Set commit UUID for the snapshot. + pub fn set_commit_uuid(&mut self, commit_uuid: Uuid) -> &mut Self { + self.commit_uuid = Some(commit_uuid); + self + } + + /// Set key metadata for manifest files. + pub fn set_key_metadata(mut self, key_metadata: Vec) -> Self { + self.key_metadata = Some(key_metadata); + self + } + + /// Set snapshot id + pub fn set_snapshot_id(mut self, snapshot_id: i64) -> Self { + self.snapshot_id = Some(snapshot_id); + self + } +} + +impl SnapshotProduceOperation for RewriteFilesOperation { + fn operation(&self) -> Operation { + Operation::Replace + } + + async fn delete_entries( + &self, + snapshot_produce: &SnapshotProducer<'_>, + ) -> Result> { + // generate delete manifest entries from removed files + let snapshot = snapshot_produce.table.metadata().current_snapshot(); + + if let Some(snapshot) = snapshot { + let gen_manifest_entry = |old_entry: &Arc| { + let builder = ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(old_entry.snapshot_id().unwrap()) + .sequence_number(old_entry.sequence_number().unwrap()) + .file_sequence_number(old_entry.file_sequence_number().unwrap()) + .data_file(old_entry.data_file().clone()); + + builder.build() + }; + + let manifest_list = snapshot + .load_manifest_list( + snapshot_produce.table.file_io(), + snapshot_produce.table.metadata(), + ) + .await?; + + let mut deleted_entries = Vec::new(); + + for manifest_file in manifest_list.entries() { + let manifest = manifest_file + .load_manifest(snapshot_produce.table.file_io()) + .await?; + + for entry in manifest.entries() { + if entry.content_type() == DataContentType::Data + && snapshot_produce + .removed_data_file_paths + .contains(entry.data_file().file_path()) + { + deleted_entries.push(gen_manifest_entry(entry)); + } + + if entry.content_type() == DataContentType::PositionDeletes + || entry.content_type() == DataContentType::EqualityDeletes + && snapshot_produce + .removed_delete_file_paths + .contains(entry.data_file().file_path()) + { + deleted_entries.push(gen_manifest_entry(entry)); + } + } + } + + Ok(deleted_entries) + } else { + Ok(vec![]) + } + } + + async fn existing_manifest( + &self, + snapshot_produce: &mut SnapshotProducer<'_>, + ) -> Result> { + let table_metadata_ref = snapshot_produce.table.metadata(); + let file_io_ref = snapshot_produce.table.file_io(); + + let Some(snapshot) = table_metadata_ref.current_snapshot() else { + return Ok(vec![]); + }; + + let manifest_list = snapshot + .load_manifest_list(file_io_ref, table_metadata_ref) + .await?; + + let mut existing_files = Vec::new(); + + for manifest_file in manifest_list.entries() { + let manifest = manifest_file.load_manifest(file_io_ref).await?; + + let found_deleted_files: HashSet<_> = manifest + .entries() + .iter() + .filter_map(|entry| { + if snapshot_produce + .removed_data_file_paths + .contains(entry.data_file().file_path()) + || snapshot_produce + .removed_delete_file_paths + .contains(entry.data_file().file_path()) + { + Some(entry.data_file().file_path().to_string()) + } else { + None + } + }) + .collect(); + + if found_deleted_files.is_empty() { + existing_files.push(manifest_file.clone()); + } else { + // Rewrite the manifest file without the deleted data files + if manifest + .entries() + .iter() + .any(|entry| !found_deleted_files.contains(entry.data_file().file_path())) + { + let mut manifest_writer = snapshot_produce.new_manifest_writer( + ManifestContentType::Data, + table_metadata_ref.default_partition_spec_id(), + )?; + + for entry in manifest.entries() { + if !found_deleted_files.contains(entry.data_file().file_path()) { + manifest_writer.add_entry((**entry).clone())?; + } + } + + existing_files.push(manifest_writer.write_manifest_file().await?); + } + } + } + + Ok(existing_files) + } +} + +#[async_trait::async_trait] +impl TransactionAction for RewriteFilesAction { + async fn commit(self: Arc, table: &Table) -> Result { + let snapshot_producer = SnapshotProducer::new( + table, + self.commit_uuid.unwrap_or_else(Uuid::now_v7), + self.key_metadata.clone(), + self.snapshot_id, + self.snapshot_properties.clone(), + self.added_data_files.clone(), + self.added_delete_files.clone(), + self.removed_data_files.clone(), + self.removed_delete_files.clone(), + ); + + // Checks duplicate files + if self.check_duplicate { + snapshot_producer + .validate_duplicate_files(&self.added_data_files) + .await?; + + snapshot_producer + .validate_duplicate_files(&self.added_delete_files) + .await?; + } + + if self.merge_enabled { + let process = + MergeManifestProcess::new(self.target_size_bytes, self.min_count_to_merge); + snapshot_producer + .commit(RewriteFilesOperation, process) + .await + } else { + snapshot_producer + .commit(RewriteFilesOperation, DefaultManifestProcess) + .await + } + } +} + +impl Default for RewriteFilesAction { + fn default() -> Self { + Self::new() + } +} diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index b5fb428181..b007827349 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -45,9 +45,10 @@ pub(crate) trait SnapshotProduceOperation: Send + Sync { &self, snapshot_produce: &SnapshotProducer, ) -> impl Future>> + Send; + fn existing_manifest( &self, - snapshot_produce: &SnapshotProducer<'_>, + snapshot_produce: &mut SnapshotProducer<'_>, ) -> impl Future>> + Send; } @@ -80,7 +81,12 @@ pub(crate) struct SnapshotProducer<'a> { key_metadata: Option>, snapshot_properties: HashMap, pub added_data_files: Vec, - added_delete_files: Vec, + pub added_delete_files: Vec, + + // for filtering out files that are removed by action + pub removed_data_file_paths: HashSet, + pub removed_delete_file_paths: HashSet, + // A counter used to generate unique manifest file names. // It starts from 0 and increments for each new manifest file. // Note: This counter is limited to the range of (0..u64::MAX). @@ -88,6 +94,7 @@ pub(crate) struct SnapshotProducer<'a> { } impl<'a> SnapshotProducer<'a> { + #[allow(clippy::too_many_arguments)] pub(crate) fn new( table: &'a Table, commit_uuid: Uuid, @@ -96,7 +103,19 @@ impl<'a> SnapshotProducer<'a> { snapshot_properties: HashMap, added_data_files: Vec, added_delete_files: Vec, + removed_data_file_paths: Vec, + removed_delete_file_paths: Vec, ) -> Self { + let removed_data_file_paths = removed_data_file_paths + .into_iter() + .map(|df| df.file_path) + .collect(); + + let removed_delete_file_paths = removed_delete_file_paths + .into_iter() + .map(|df| df.file_path) + .collect(); + Self { table, snapshot_id: snapshot_id.unwrap_or_else(|| Self::generate_unique_snapshot_id(table)), @@ -105,6 +124,8 @@ impl<'a> SnapshotProducer<'a> { snapshot_properties, added_data_files, added_delete_files, + removed_data_file_paths, + removed_delete_file_paths, manifest_counter: (0..), } } @@ -189,7 +210,7 @@ impl<'a> SnapshotProducer<'a> { snapshot_id } - fn new_manifest_writer( + pub(crate) fn new_manifest_writer( &mut self, content: ManifestContentType, partition_spec_id: i32, diff --git a/crates/integration_tests/tests/shared_tests/mod.rs b/crates/integration_tests/tests/shared_tests/mod.rs index da1ea9fa72..0ebe77ab7f 100644 --- a/crates/integration_tests/tests/shared_tests/mod.rs +++ b/crates/integration_tests/tests/shared_tests/mod.rs @@ -31,6 +31,7 @@ mod merge_append_test; mod read_evolved_schema; mod read_positional_deletes; mod remove_snapshots_test; +mod rewrite_files_test; mod scan_all_type; pub async fn random_ns() -> Namespace { diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs new file mode 100644 index 0000000000..14dc180f7e --- /dev/null +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -0,0 +1,359 @@ +// 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. + +//! Integration tests for rest catalog. + +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use iceberg::transaction::{ApplyTransactionAction, Transaction}; +use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, CatalogBuilder, TableCreation}; +use iceberg_catalog_rest::RestCatalogBuilder; +use parquet::arrow::arrow_reader::ArrowReaderOptions; +use parquet::file::properties::WriterProperties; + +use crate::get_shared_containers; +use crate::shared_tests::{random_ns, test_schema}; + +#[tokio::test] +async fn test_rewrite_data_files() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + // check parquet file schema + let content = table + .file_io() + .new_input(data_file[0].file_path()) + .unwrap() + .read() + .await + .unwrap(); + let parquet_reader = parquet::arrow::arrow_reader::ArrowReaderMetadata::load( + &content, + ArrowReaderOptions::default(), + ) + .unwrap(); + let field_ids: Vec = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.self_type().get_basic_info().id()) + .collect(); + assert_eq!(field_ids, vec![1, 2, 3]); + + // commit result + let tx = Transaction::new(&table); + // First append + let append_action = tx.fast_append().add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + + // commit result again + // Second append of the SAME data file: disable duplicate check to preserve original test logic + let tx = Transaction::new(&table); + let append_action = tx + .fast_append() + .with_check_duplicate(false) + .add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check result again + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0], batch); + assert_eq!(batches[1], batch); + + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file_rewrite = data_file_writer.close().await.unwrap(); + + // commit result again + let tx = Transaction::new(&table); + // Clone tx so we can consume one for building the action (rewrite_files takes self) + let rewrite_action = tx + .clone() + .rewrite_files() + .add_data_files(data_file_rewrite.clone()) + .delete_files(data_file.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); +} + +#[tokio::test] +async fn test_multiple_file_rewrite() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t3".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file1 = data_file_writer.close().await.unwrap(); + + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file2 = data_file_writer.close().await.unwrap(); + + let tx = Transaction::new(&table); + let rewrite_action = tx + .clone() + .rewrite_files() + .add_data_files(data_file1.clone()) + .add_data_files(data_file2.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0], batch); + assert_eq!(batches[1], batch); +} + +#[tokio::test] +async fn test_rewrite_nonexistent_file() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t4".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create a valid data file + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let valid_data_file = data_file_writer.close().await.unwrap(); + + // Create a nonexistent data file (simulated by not writing it) + let nonexistent_data_file = valid_data_file.clone(); + + // Build rewrite action deleting a nonexistent file; we only ensure builder compiles and does not panic + let _unused_action = Transaction::new(&table) + .rewrite_files() + .delete_files(nonexistent_data_file); + // No commit since removing a nonexistent file would not create a valid snapshot under new semantics +} From c06158b31da469a08544a8bb8f8ab1b599cbac02 Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 17 Jul 2025 18:53:53 +0800 Subject: [PATCH 11/33] fix: fix plan files with deletes (#61) * fix plan files with deletes * fmt --- crates/iceberg/src/scan/context.rs | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 36d1d02e3d..82a1255198 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -259,7 +259,8 @@ impl PlanContext { }; // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. - let mut filtered_mfcs = vec![]; + let mut filtered_deletes_mfcs = vec![]; + let mut filtered_data_mfcs = vec![]; for manifest_file in &manifest_files { let tx = if manifest_file.content == ManifestContentType::Deletes { delete_file_tx.clone() @@ -296,10 +297,22 @@ impl PlanContext { filter_fn.clone(), ); - filtered_mfcs.push(Ok(mfc)); + match manifest_file.content { + ManifestContentType::Deletes => { + filtered_deletes_mfcs.push(Ok(mfc)); + } + ManifestContentType::Data => { + filtered_data_mfcs.push(Ok(mfc)); + } + } } - Ok(Box::new(filtered_mfcs.into_iter())) + // Push deletes manifest first then data manifest files. + Ok(Box::new( + filtered_deletes_mfcs + .into_iter() + .chain(filtered_data_mfcs.into_iter()), + )) } fn create_manifest_file_context( From ef44e889fd2f13923205f18dde74813605cdcb28 Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 24 Jul 2025 18:37:43 +0800 Subject: [PATCH 12/33] feat: optimize plan files memory consumption (#64) --- .../src/arrow/caching_delete_file_loader.rs | 4 +- crates/iceberg/src/arrow/delete_filter.rs | 4 +- crates/iceberg/src/arrow/reader.rs | 93 ++++++++++--------- crates/iceberg/src/delete_file_index.rs | 59 ++++++------ crates/iceberg/src/scan/task.rs | 2 +- 5 files changed, 86 insertions(+), 76 deletions(-) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index 28d1655a8e..4cf239e017 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -138,7 +138,7 @@ impl CachingDeleteFileLoader { /// ``` pub(crate) fn load_deletes( &self, - delete_file_entries: &[FileScanTask], + delete_file_entries: &[Arc], schema: SchemaRef, ) -> Receiver> { let (tx, rx) = channel(); @@ -911,7 +911,7 @@ mod tests { schema: data_file_schema.clone(), project_field_ids: vec![2, 3], predicate: None, - deletes: vec![pos_del_scan_task, eq_del_scan_task], + deletes: vec![pos_del_scan_task.into(), eq_del_scan_task.into()], sequence_number: 0, data_file_content: DataContentType::Data, equality_ids: None, diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index bc0e5d1aba..f83b8b5f98 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -336,7 +336,7 @@ pub(crate) mod tests { schema: data_file_schema.clone(), project_field_ids: vec![], predicate: None, - deletes: vec![make_pos_del_task(1), make_pos_del_task(2)], + deletes: vec![make_pos_del_task(1).into(), make_pos_del_task(2).into()], sequence_number: 0, equality_ids: None, }, @@ -350,7 +350,7 @@ pub(crate) mod tests { schema: data_file_schema.clone(), project_field_ids: vec![], predicate: None, - deletes: vec![make_pos_del_task(3)], + deletes: vec![make_pos_del_task(3).into()], sequence_number: 0, equality_ids: None, }, diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 1baa7675a2..ca12d89fe2 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -2595,20 +2595,23 @@ message schema { schema: table_schema.clone(), project_field_ids: vec![1], predicate: None, - deletes: vec![FileScanTask { - start: 0, - length: 0, - record_count: None, - data_file_path: delete_file_path.clone(), - data_file_format: DataFileFormat::Parquet, - schema: table_schema.clone(), - project_field_ids: vec![], - predicate: None, - deletes: vec![], - data_file_content: DataContentType::PositionDeletes, - sequence_number: 0, - equality_ids: None, - }], + deletes: vec![ + FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: delete_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::PositionDeletes, + sequence_number: 0, + equality_ids: None, + } + .into(), + ], data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, @@ -2821,20 +2824,23 @@ message schema { schema: table_schema.clone(), project_field_ids: vec![1], predicate: None, - deletes: vec![FileScanTask { - start: 0, - length: 0, - record_count: None, - data_file_path: delete_file_path.clone(), - data_file_format: DataFileFormat::Parquet, - schema: position_del_schema(), - project_field_ids: vec![], - predicate: None, - deletes: vec![], - data_file_content: DataContentType::PositionDeletes, - sequence_number: 0, - equality_ids: None, - }], + deletes: vec![ + FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: delete_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: position_del_schema(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::PositionDeletes, + sequence_number: 0, + equality_ids: None, + } + .into(), + ], data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, @@ -3040,20 +3046,23 @@ message schema { schema: table_schema.clone(), project_field_ids: vec![1], predicate: None, - deletes: vec![FileScanTask { - start: 0, - length: 0, - record_count: None, - data_file_path: delete_file_path.clone(), - data_file_format: DataFileFormat::Parquet, - schema: position_del_schema(), - project_field_ids: vec![], - predicate: None, - deletes: vec![], - data_file_content: DataContentType::PositionDeletes, - sequence_number: 0, - equality_ids: None, - }], + deletes: vec![ + FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: delete_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: position_del_schema(), + project_field_ids: vec![], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::PositionDeletes, + sequence_number: 0, + equality_ids: None, + } + .into(), + ], data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index 8692e8acf3..9ba9a46702 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -39,12 +39,14 @@ enum DeleteFileIndexState { Populated(PopulatedDeleteFileIndex), } +type DeleteFileContextAndTask = (Arc, Arc); + #[derive(Debug)] struct PopulatedDeleteFileIndex { #[allow(dead_code)] - global_equality_deletes: Vec>, - eq_deletes_by_partition: HashMap>>, - pos_deletes_by_partition: HashMap>>, + global_equality_deletes: Vec, + eq_deletes_by_partition: HashMap>, + pos_deletes_by_partition: HashMap>, // TODO: do we need this? // pos_deletes_by_path: HashMap>>, @@ -86,7 +88,7 @@ impl DeleteFileIndex { &self, data_file: &DataFile, seq_num: Option, - ) -> Vec { + ) -> Vec> { let notifier = { let guard = self.state.read().unwrap(); match *guard { @@ -118,15 +120,14 @@ impl PopulatedDeleteFileIndex { /// it is added to the `global_equality_deletes` vector /// 3. Otherwise, the delete file is added to one of two hash maps based on its content type. fn new(files: Vec) -> PopulatedDeleteFileIndex { - let mut eq_deletes_by_partition: HashMap>> = - HashMap::default(); - let mut pos_deletes_by_partition: HashMap>> = - HashMap::default(); + let mut eq_deletes_by_partition = HashMap::default(); + let mut pos_deletes_by_partition = HashMap::default(); - let mut global_equality_deletes: Vec> = vec![]; + let mut global_equality_deletes: Vec<(Arc, Arc)> = vec![]; files.into_iter().for_each(|ctx| { let arc_ctx = Arc::new(ctx); + let file_scan_task: Arc = Arc::new(arc_ctx.as_ref().into()); let partition = arc_ctx.manifest_entry.data_file().partition(); @@ -134,7 +135,7 @@ impl PopulatedDeleteFileIndex { if partition.fields().is_empty() { // TODO: confirm we're good to skip here if we encounter a pos del if arc_ctx.manifest_entry.content_type() != DataContentType::PositionDeletes { - global_equality_deletes.push(arc_ctx); + global_equality_deletes.push((arc_ctx, file_scan_task.clone())); return; } } @@ -147,10 +148,10 @@ impl PopulatedDeleteFileIndex { destination_map .entry(partition.clone()) - .and_modify(|entry| { - entry.push(arc_ctx.clone()); + .and_modify(|entry: &mut Vec| { + entry.push((arc_ctx.clone(), file_scan_task.clone())); }) - .or_insert(vec![arc_ctx.clone()]); + .or_insert(vec![(arc_ctx.clone(), file_scan_task)]); }); PopulatedDeleteFileIndex { @@ -165,30 +166,30 @@ impl PopulatedDeleteFileIndex { &self, data_file: &DataFile, seq_num: Option, - ) -> Vec { + ) -> Vec> { let mut results = vec![]; self.global_equality_deletes .iter() // filter that returns true if the provided delete file's sequence number is **greater than** `seq_num` - .filter(|&delete| { + .filter(|&(delete, _)| { seq_num .map(|seq_num| delete.manifest_entry.sequence_number() > Some(seq_num)) .unwrap_or_else(|| true) }) - .for_each(|delete| results.push(delete.as_ref().into())); + .for_each(|(_, task)| results.push(task.clone())); if let Some(deletes) = self.eq_deletes_by_partition.get(data_file.partition()) { deletes .iter() // filter that returns true if the provided delete file's sequence number is **greater than** `seq_num` - .filter(|&delete| { + .filter(|&(delete, _)| { seq_num .map(|seq_num| delete.manifest_entry.sequence_number() > Some(seq_num)) .unwrap_or_else(|| true) && data_file.partition_spec_id == delete.partition_spec_id }) - .for_each(|delete| results.push(delete.as_ref().into())); + .for_each(|(_, task)| results.push(task.clone())); } // TODO: the spec states that: @@ -199,13 +200,13 @@ impl PopulatedDeleteFileIndex { deletes .iter() // filter that returns true if the provided delete file's sequence number is **greater than or equal to** `seq_num` - .filter(|&delete| { + .filter(|&(delete, _)| { seq_num .map(|seq_num| delete.manifest_entry.sequence_number() >= Some(seq_num)) .unwrap_or_else(|| true) && data_file.partition_spec_id == delete.partition_spec_id }) - .for_each(|delete| results.push(delete.as_ref().into())); + .for_each(|(_, task)| results.push(task.clone())); } results @@ -265,7 +266,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&data_file, Some(4)); let actual_paths_to_apply_for_seq_4: Vec = delete_files_to_apply_for_seq_4 .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( @@ -278,7 +279,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&data_file, Some(5)); let actual_paths_to_apply_for_seq_5: Vec = delete_files_to_apply_for_seq_5 .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( actual_paths_to_apply_for_seq_5, @@ -290,7 +291,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&data_file, Some(6)); let actual_paths_to_apply_for_seq_6: Vec = delete_files_to_apply_for_seq_6 .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( actual_paths_to_apply_for_seq_6, @@ -306,7 +307,7 @@ mod tests { let actual_paths_to_apply_for_partitioned_file: Vec = delete_files_to_apply_for_partitioned_file .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( actual_paths_to_apply_for_partitioned_file, @@ -360,7 +361,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(4)); let actual_paths_to_apply_for_seq_4: Vec = delete_files_to_apply_for_seq_4 .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( @@ -373,7 +374,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(5)); let actual_paths_to_apply_for_seq_5: Vec = delete_files_to_apply_for_seq_5 .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( actual_paths_to_apply_for_seq_5, @@ -385,7 +386,7 @@ mod tests { delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(6)); let actual_paths_to_apply_for_seq_6: Vec = delete_files_to_apply_for_seq_6 .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert_eq!( actual_paths_to_apply_for_seq_6, @@ -400,7 +401,7 @@ mod tests { let actual_paths_to_apply_for_different_partition: Vec = delete_files_to_apply_for_different_partition .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert!(actual_paths_to_apply_for_different_partition.is_empty()); @@ -411,7 +412,7 @@ mod tests { let actual_paths_to_apply_for_different_spec: Vec = delete_files_to_apply_for_different_spec .into_iter() - .map(|file| file.data_file_path) + .map(|file| file.data_file_path.clone()) .collect(); assert!(actual_paths_to_apply_for_different_spec.is_empty()); } diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index cf0b9e3692..3f8c3acd77 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -58,7 +58,7 @@ pub struct FileScanTask { pub predicate: Option, /// The list of delete files that may need to be applied to this data file - pub deletes: Vec, + pub deletes: Vec>, /// sequence number pub sequence_number: i64, /// equality ids From fd23d8e2b86f19c1e92cf2ac9a3a23912a8392da Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 11 Nov 2025 18:48:08 +0800 Subject: [PATCH 13/33] azblob --- Cargo.lock | 1 + crates/iceberg/Cargo.toml | 3 +- crates/iceberg/src/io/file_io.rs | 1 + crates/iceberg/src/io/mod.rs | 5 ++ crates/iceberg/src/io/storage.rs | 22 +++++++++ crates/iceberg/src/io/storage_azblob.rs | 63 +++++++++++++++++++++++++ 6 files changed, 94 insertions(+), 1 deletion(-) create mode 100644 crates/iceberg/src/io/storage_azblob.rs diff --git a/Cargo.lock b/Cargo.lock index e17e403cfe..911ddde8b8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4741,6 +4741,7 @@ dependencies = [ "reqwest", "serde", "serde_json", + "sha2", "tokio", "uuid", ] diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 789e6ef0d0..f0540e06e5 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -30,7 +30,7 @@ repository = { workspace = true } [features] default = ["storage-memory", "storage-fs", "storage-s3", "tokio"] -storage-all = ["storage-memory", "storage-fs", "storage-s3", "storage-gcs"] +storage-all = ["storage-memory", "storage-fs", "storage-s3", "storage-gcs", "storage-azblob"] storage-azdls = ["opendal/services-azdls"] storage-fs = ["opendal/services-fs"] @@ -38,6 +38,7 @@ storage-gcs = ["opendal/services-gcs"] storage-memory = ["opendal/services-memory"] storage-oss = ["opendal/services-oss"] storage-s3 = ["opendal/services-s3", "reqsign"] +storage-azblob = ["opendal/services-azblob"] smol = ["dep:smol"] tokio = ["tokio/rt-multi-thread"] diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 6e2d152ed7..48d19c7723 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -44,6 +44,7 @@ use crate::{Error, ErrorKind, Result}; /// | GCS | `storage-gcs` | `gs`, `gcs` | `gs:///path/to/file` | /// | OSS | `storage-oss` | `oss` | `oss:///path/to/file` | /// | Azure Datalake | `storage-azdls` | `abfs`, `abfss`, `wasb`, `wasbs` | `abfs://@.dfs.core.windows.net/path/to/file` or `wasb://@.blob.core.windows.net/path/to/file` | +/// | AZBLOB | `storage-azblob` | `azblob` | `azblob:///path/to/file` | #[derive(Clone, Debug)] pub struct FileIO { builder: FileIOBuilder, diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 5eb5964345..b46d468288 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -98,6 +98,11 @@ pub use storage_oss::*; #[cfg(feature = "storage-s3")] pub use storage_s3::*; +#[cfg(feature = "storage-azblob")] +mod storage_azblob; +#[cfg(feature = "storage-azblob")] +pub use storage_azblob::*; + pub(crate) fn is_truthy(value: &str) -> bool { ["true", "t", "1", "on"].contains(&value.to_lowercase().as_str()) } diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index d5f2ad8fab..66fef0f24a 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -18,6 +18,8 @@ use std::sync::Arc; use opendal::layers::RetryLayer; +#[cfg(feature = "storage-azblob")] +use opendal::services::AzblobConfig; #[cfg(feature = "storage-azdls")] use opendal::services::AzdlsConfig; #[cfg(feature = "storage-gcs")] @@ -53,6 +55,8 @@ pub(crate) enum Storage { }, #[cfg(feature = "storage-gcs")] Gcs { config: Arc }, + #[cfg(feature = "storage-azblob")] + Azblob { config: Arc }, #[cfg(feature = "storage-oss")] Oss { config: Arc }, /// Expects paths of the form @@ -90,6 +94,10 @@ impl Storage { Scheme::Gcs => Ok(Self::Gcs { config: super::gcs_config_parse(props)?.into(), }), + #[cfg(feature = "storage-azblob")] + Scheme::Azblob => Ok(Self::Azblob { + config: super::azblob_config_parse(props)?.into(), + }), #[cfg(feature = "storage-oss")] Scheme::Oss => Ok(Self::Oss { config: super::oss_config_parse(props)?.into(), @@ -179,6 +187,19 @@ impl Storage { )) } } + #[cfg(feature = "storage-azblob")] + Storage::Azblob { config } => { + let operator = super::azblob_config_build(config, path)?; + let prefix = format!("azblob://{}/", operator.info().name()); + if path.starts_with(&prefix) { + Ok((operator, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid azblob url: {}, should start with {}", path, prefix), + )) + } + } #[cfg(feature = "storage-oss")] Storage::Oss { config } => { let op = super::oss_config_build(config, path)?; @@ -203,6 +224,7 @@ impl Storage { not(feature = "storage-s3"), not(feature = "storage-fs"), not(feature = "storage-gcs"), + not(feature = "storage-azblob"), not(feature = "storage-oss"), not(feature = "storage-azdls"), ))] diff --git a/crates/iceberg/src/io/storage_azblob.rs b/crates/iceberg/src/io/storage_azblob.rs new file mode 100644 index 0000000000..54f4a86661 --- /dev/null +++ b/crates/iceberg/src/io/storage_azblob.rs @@ -0,0 +1,63 @@ +// 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. +//! Azure blob storage properties +use std::collections::HashMap; + +use opendal::Operator; +use opendal::services::AzblobConfig; +use url::Url; + +use crate::{Error, ErrorKind, Result}; + +/// Azure blob account name. +pub const AZBLOB_ACCOUNT_NAME: &str = "azblob.account-name"; +/// Azure blob account key. +pub const AZBLOB_ACCOUNT_KEY: &str = "azblob.account-key"; +/// Azure blob account endpoint. +pub const AZBLOB_ENDPOINT: &str = "azblob.endpoint"; + +/// Parse iceberg properties to [`AzblobConfig`]. +pub(crate) fn azblob_config_parse(mut m: HashMap) -> Result { + let mut cfg = AzblobConfig::default(); + + if let Some(account_name) = m.remove(AZBLOB_ACCOUNT_NAME) { + cfg.account_name = Some(account_name); + }; + if let Some(account_key) = m.remove(AZBLOB_ACCOUNT_KEY) { + cfg.account_key = Some(account_key); + }; + if let Some(endpoint) = m.remove(AZBLOB_ENDPOINT) { + cfg.endpoint = Some(endpoint); + }; + + Ok(cfg) +} + +/// Build a new OpenDAL [`Operator`] based on a provided [`AzblobConfig`]. +pub(crate) fn azblob_config_build(cfg: &AzblobConfig, path: &str) -> Result { + let url = Url::parse(path)?; + let container = url.host_str().ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Invalid azblob url: {}, container is required", path), + ) + })?; + + let mut cfg = cfg.clone(); + cfg.container = container.to_string(); + Ok(Operator::from_config(cfg)?.finish()) +} From ccb48c38d0dd7c567021c0ce157623a723e60113 Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 23 May 2025 15:24:29 +0800 Subject: [PATCH 14/33] fix(iceberg): Introduce new data sequence for RewriteFilesAction (#51) * feat(iceberg): rewrite_files support use_starting_sequence_number * chore(test): add test_sequence_number_in_manifest_entry --- crates/iceberg/src/scan/context.rs | 55 +---------- crates/iceberg/src/spec/snapshot.rs | 28 +----- .../src/transaction/remove_snapshots.rs | 17 ++-- .../iceberg/src/transaction/rewrite_files.rs | 16 +++- crates/iceberg/src/transaction/snapshot.rs | 25 ++++- crates/iceberg/src/utils.rs | 56 +++++++++++ .../testdata/docker-compose.yaml | 24 ----- .../tests/shared_tests/rewrite_files_test.rs | 96 +++++++++++++++++++ 8 files changed, 199 insertions(+), 118 deletions(-) diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 82a1255198..81beb0c5ba 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -33,6 +33,7 @@ use crate::spec::{ DataContentType, ManifestContentType, ManifestEntryRef, ManifestFile, ManifestList, ManifestStatus, Operation, SchemaRef, SnapshotRef, TableMetadataRef, }; +use crate::utils::ancestors_between; use crate::{Error, ErrorKind, Result}; type ManifestEntryFilterFn = dyn Fn(&ManifestEntryRef) -> bool + Send + Sync; @@ -348,57 +349,3 @@ impl PlanContext { } } } - -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. -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). -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), - ) -} diff --git a/crates/iceberg/src/spec/snapshot.rs b/crates/iceberg/src/spec/snapshot.rs index 132634c2b6..5371cf68f2 100644 --- a/crates/iceberg/src/spec/snapshot.rs +++ b/crates/iceberg/src/spec/snapshot.rs @@ -193,6 +193,7 @@ impl Snapshot { } /// Get parent snapshot. + #[cfg(test)] pub(crate) fn parent_snapshot(&self, table_metadata: &TableMetadata) -> Option { match self.parent_snapshot_id { Some(id) => table_metadata.snapshot_by_id(id).cloned(), @@ -510,33 +511,6 @@ impl SnapshotRetention { } } -/// An iterator over the ancestors of a snapshot. -pub struct AncestorIterator<'a> { - current: Option, - table_metadata: &'a TableMetadata, -} - -impl Iterator for AncestorIterator<'_> { - type Item = SnapshotRef; - - fn next(&mut self) -> Option { - let current = self.current.take()?; - - let next = current.parent_snapshot(self.table_metadata); - self.current = next; - - Some(current) - } -} - -/// Returns an iterator over the ancestors of a snapshot. -pub fn ancestors_of(snapshot: SnapshotRef, table_metadata: &TableMetadata) -> AncestorIterator<'_> { - AncestorIterator { - current: Some(snapshot), - table_metadata, - } -} - #[cfg(test)] mod tests { use std::collections::HashMap; diff --git a/crates/iceberg/src/transaction/remove_snapshots.rs b/crates/iceberg/src/transaction/remove_snapshots.rs index ac805e0ec3..37a9565b35 100644 --- a/crates/iceberg/src/transaction/remove_snapshots.rs +++ b/crates/iceberg/src/transaction/remove_snapshots.rs @@ -24,9 +24,10 @@ use async_trait::async_trait; use itertools::Itertools; use crate::error::Result; -use crate::spec::{MAIN_BRANCH, SnapshotReference, SnapshotRetention, TableMetadata, ancestors_of}; +use crate::spec::{MAIN_BRANCH, SnapshotReference, SnapshotRetention, TableMetadataRef}; use crate::table::Table; use crate::transaction::{ActionCommit, TransactionAction}; +use crate::utils::ancestors_of; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; /// Default value for max snapshot age in milliseconds. @@ -103,7 +104,7 @@ impl RemoveSnapshotAction { fn compute_retained_refs( &self, snapshot_refs: &HashMap, - table_meta: &TableMetadata, + table_meta: &TableMetadataRef, ) -> HashMap { let mut retained_refs = HashMap::new(); @@ -144,7 +145,7 @@ impl RemoveSnapshotAction { fn compute_all_branch_snapshots_to_retain( &self, refs: impl Iterator, - table_meta: &TableMetadata, + table_meta: &TableMetadataRef, ) -> HashSet { let mut branch_snapshots_to_retain = HashSet::new(); for snapshot_ref in refs { @@ -192,11 +193,11 @@ impl RemoveSnapshotAction { snapshot_id: i64, expire_snapshots_older_than: i64, min_snapshots_to_keep: usize, - table_meta: &TableMetadata, + table_meta: &TableMetadataRef, ) -> HashSet { let mut ids_to_retain = HashSet::new(); if let Some(snapshot) = table_meta.snapshot_by_id(snapshot_id) { - let ancestors = ancestors_of(snapshot.clone(), table_meta); + let ancestors = ancestors_of(table_meta, snapshot.snapshot_id()); for ancestor in ancestors { if ids_to_retain.len() < min_snapshots_to_keep || ancestor.timestamp_ms() >= expire_snapshots_older_than @@ -214,7 +215,7 @@ impl RemoveSnapshotAction { fn unreferenced_snapshots_to_retain( &self, refs: impl Iterator, - table_meta: &TableMetadata, + table_meta: &TableMetadataRef, ) -> HashSet { let mut ids_to_retain = HashSet::new(); let mut referenced_snapshots = HashSet::new(); @@ -222,7 +223,7 @@ impl RemoveSnapshotAction { for snapshot_ref in refs { if snapshot_ref.is_branch() { if let Some(snapshot) = table_meta.snapshot_by_id(snapshot_ref.snapshot_id) { - let ancestors = ancestors_of(snapshot.clone(), table_meta); + let ancestors = ancestors_of(table_meta, snapshot.snapshot_id()); for ancestor in ancestors { referenced_snapshots.insert(ancestor.snapshot_id()); } @@ -251,7 +252,7 @@ impl TransactionAction for RemoveSnapshotAction { return Ok(ActionCommit::new(vec![], vec![])); } - let table_meta = table.metadata().clone(); + let table_meta = table.metadata_ref(); let mut ids_to_retain = HashSet::new(); let retained_refs = self.compute_retained_refs(&table_meta.refs, &table_meta); diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs index 93dc356755..e3315b8f72 100644 --- a/crates/iceberg/src/transaction/rewrite_files.rs +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -53,6 +53,8 @@ pub struct RewriteFilesAction { removed_data_files: Vec, removed_delete_files: Vec, snapshot_id: Option, + + new_data_file_sequence_number: Option, } struct RewriteFilesOperation; @@ -72,6 +74,7 @@ impl RewriteFilesAction { removed_data_files: Vec::new(), removed_delete_files: Vec::new(), snapshot_id: None, + new_data_file_sequence_number: None, } } @@ -121,6 +124,7 @@ impl RewriteFilesAction { self.min_count_to_merge = min_count_to_merge; self.merge_enabled = merge_enabled; self.snapshot_properties = properties; + self } @@ -141,6 +145,12 @@ impl RewriteFilesAction { self.snapshot_id = Some(snapshot_id); self } + + pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Result { + self.new_data_file_sequence_number = Some(seq); + + Ok(self) + } } impl SnapshotProduceOperation for RewriteFilesOperation { @@ -277,7 +287,7 @@ impl SnapshotProduceOperation for RewriteFilesOperation { #[async_trait::async_trait] impl TransactionAction for RewriteFilesAction { async fn commit(self: Arc, table: &Table) -> Result { - let snapshot_producer = SnapshotProducer::new( + let mut snapshot_producer = SnapshotProducer::new( table, self.commit_uuid.unwrap_or_else(Uuid::now_v7), self.key_metadata.clone(), @@ -300,6 +310,10 @@ impl TransactionAction for RewriteFilesAction { .await?; } + if let Some(seq) = self.new_data_file_sequence_number { + snapshot_producer.set_new_data_file_sequence_number(seq); + } + if self.merge_enabled { let process = MergeManifestProcess::new(self.target_size_bytes, self.min_count_to_merge); diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index b007827349..34d187866e 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -91,6 +91,8 @@ pub(crate) struct SnapshotProducer<'a> { // It starts from 0 and increments for each new manifest file. // Note: This counter is limited to the range of (0..u64::MAX). manifest_counter: RangeFrom, + + new_data_file_sequence_number: Option, } impl<'a> SnapshotProducer<'a> { @@ -127,6 +129,7 @@ impl<'a> SnapshotProducer<'a> { removed_data_file_paths, removed_delete_file_paths, manifest_counter: (0..), + new_data_file_sequence_number: None, } } @@ -287,7 +290,11 @@ impl<'a> SnapshotProducer<'a> { } // Write manifest file for added data files and return the ManifestFile for ManifestList. - async fn write_added_manifest(&mut self, added_files: Vec) -> Result { + async fn write_added_manifest( + &mut self, + added_files: Vec, + data_seq: Option, + ) -> Result { if added_files.is_empty() { return Err(Error::new( ErrorKind::PreconditionFailed, @@ -325,7 +332,9 @@ impl<'a> SnapshotProducer<'a> { let manifest_entries = added_files.into_iter().map(|data_file| { let builder = ManifestEntry::builder() .status(crate::spec::ManifestStatus::Added) - .data_file(data_file); + .data_file(data_file) + .sequence_number_opt(data_seq); + if format_version == FormatVersion::V1 { builder.snapshot_id(snapshot_id).build() } else { @@ -423,13 +432,17 @@ impl<'a> SnapshotProducer<'a> { // Process added entries. if !self.added_data_files.is_empty() { let added_data_files = std::mem::take(&mut self.added_data_files); - let added_manifest = self.write_added_manifest(added_data_files).await?; + let added_manifest = self + .write_added_manifest(added_data_files, self.new_data_file_sequence_number) + .await?; manifest_files.push(added_manifest); } if !self.added_delete_files.is_empty() { let added_delete_files = std::mem::take(&mut self.added_delete_files); - let added_manifest = self.write_added_manifest(added_delete_files).await?; + let added_manifest = self + .write_added_manifest(added_delete_files, self.new_data_file_sequence_number) + .await?; manifest_files.push(added_manifest); } @@ -601,6 +614,10 @@ impl<'a> SnapshotProducer<'a> { Ok(ActionCommit::new(updates, requirements)) } + + pub fn set_new_data_file_sequence_number(&mut self, new_data_file_sequence_number: i64) { + self.new_data_file_sequence_number = Some(new_data_file_sequence_number); + } } pub(crate) struct MergeManifestProcess { diff --git a/crates/iceberg/src/utils.rs b/crates/iceberg/src/utils.rs index 1a60c82a77..f9822e26d6 100644 --- a/crates/iceberg/src/utils.rs +++ b/crates/iceberg/src/utils.rs @@ -17,6 +17,8 @@ use std::num::NonZeroUsize; +use crate::spec::{SnapshotRef, TableMetadataRef}; + // Use a default value of 1 as the safest option. // See https://doc.rust-lang.org/std/thread/fn.available_parallelism.html#limitations // for more details. @@ -185,3 +187,57 @@ pub mod bin { } } } + +pub 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), + ) +} diff --git a/crates/integration_tests/testdata/docker-compose.yaml b/crates/integration_tests/testdata/docker-compose.yaml index d9f4c30c71..cf0240d1a5 100644 --- a/crates/integration_tests/testdata/docker-compose.yaml +++ b/crates/integration_tests/testdata/docker-compose.yaml @@ -22,12 +22,6 @@ services: rest: image: apache/iceberg-rest-fixture:1.10.0 environment: - - http_proxy= - - https_proxy= - - HTTP_PROXY= - - HTTPS_PROXY= - - NO_PROXY=localhost,127.0.0.1,minio - - no_proxy=localhost,127.0.0.1,minio - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password - AWS_REGION=us-east-1 @@ -48,12 +42,6 @@ services: minio: image: minio/minio:RELEASE.2025-05-24T17-08-30Z environment: - - http_proxy= - - https_proxy= - - HTTP_PROXY= - - HTTPS_PROXY= - - NO_PROXY=localhost,127.0.0.1,minio - - no_proxy=localhost,127.0.0.1,minio - MINIO_ROOT_USER=admin - MINIO_ROOT_PASSWORD=password - MINIO_DOMAIN=minio @@ -73,12 +61,6 @@ services: - minio image: minio/mc:RELEASE.2025-05-21T01-59-54Z environment: - - http_proxy= - - https_proxy= - - HTTP_PROXY= - - HTTPS_PROXY= - - NO_PROXY=localhost,127.0.0.1,minio - - no_proxy=localhost,127.0.0.1,minio - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password - AWS_REGION=us-east-1 @@ -95,12 +77,6 @@ services: - rest - minio environment: - - http_proxy= - - https_proxy= - - HTTP_PROXY= - - HTTPS_PROXY= - - NO_PROXY=localhost,127.0.0.1,minio - - no_proxy=localhost,127.0.0.1,minio - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password - AWS_REGION=us-east-1 diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs index 14dc180f7e..af3881a526 100644 --- a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -357,3 +357,99 @@ async fn test_rewrite_nonexistent_file() { .delete_files(nonexistent_data_file); // No commit since removing a nonexistent file would not create a valid snapshot under new semantics } + +#[tokio::test] +async fn test_sequence_number_in_manifest_entry() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t3".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file1 = data_file_writer.close().await.unwrap(); + + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file2 = data_file_writer.close().await.unwrap(); + + // Commit with sequence number + + let tx = Transaction::new(&table); + let rewrite_action = tx + .clone() + .rewrite_files() + .add_data_files(data_file1.clone()) + .add_data_files(data_file2.clone()); + // Set sequence number to 12345 + let rewrite_action = rewrite_action + .set_new_data_file_sequence_number(12345) + .unwrap(); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify manifest entry has correct sequence number + let snapshot = table.metadata().current_snapshot().unwrap(); + let manifest_list = snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + + assert_eq!(manifest_list.entries().len(), 1); + + for manifest_file in manifest_list.entries() { + let manifest = manifest_file.load_manifest(table.file_io()).await.unwrap(); + for entry in manifest.entries() { + assert_eq!(entry.sequence_number(), Some(12345)); + } + } +} From d5efb443af288825dee194d96197e9b1cf579534 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Fri, 30 May 2025 12:43:37 +0800 Subject: [PATCH 15/33] feat: exposed file_size_in_bytes (#46) * fix: delete file lost wake Signed-off-by: xxchan * . Signed-off-by: xxchan * . Signed-off-by: xxchan * revert * typo --------- Signed-off-by: xxchan Co-authored-by: xxchan Co-authored-by: Li0k --- .../src/arrow/caching_delete_file_loader.rs | 3 +++ crates/iceberg/src/arrow/delete_filter.rs | 3 +++ crates/iceberg/src/arrow/reader.rs | 17 +++++++++++++++++ crates/iceberg/src/scan/context.rs | 1 + crates/iceberg/src/scan/mod.rs | 2 ++ crates/iceberg/src/scan/task.rs | 4 ++++ .../datafusion/src/physical_plan/mod.rs | 2 +- 7 files changed, 31 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index 4cf239e017..568fa2d8df 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -885,6 +885,7 @@ mod tests { deletes: vec![], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; let eq_del_scan_task = FileScanTask { @@ -900,6 +901,7 @@ mod tests { deletes: vec![], sequence_number: 0, equality_ids: eq_del.equality_ids.clone(), + file_size_in_bytes: 0, }; let file_scan_task = FileScanTask { @@ -915,6 +917,7 @@ mod tests { sequence_number: 0, data_file_content: DataContentType::Data, equality_ids: None, + file_size_in_bytes: 0, }; // Load the deletes - should handle both types without error diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index f83b8b5f98..8b154ff62a 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -323,6 +323,7 @@ pub(crate) mod tests { deletes: vec![], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; let file_scan_tasks = vec![ @@ -339,6 +340,7 @@ pub(crate) mod tests { deletes: vec![make_pos_del_task(1).into(), make_pos_del_task(2).into()], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }, FileScanTask { start: 0, @@ -353,6 +355,7 @@ pub(crate) mod tests { deletes: vec![make_pos_del_task(3).into()], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index ca12d89fe2..da3131c2c3 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -1969,6 +1969,7 @@ message schema { deletes: vec![], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -2290,6 +2291,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; // Task 2: read the second and third row groups @@ -2306,6 +2308,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; let tasks1 = Box::pin(futures::stream::iter(vec![Ok(task1)])) as FileScanTaskStream; @@ -2433,6 +2436,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -2609,12 +2613,14 @@ message schema { data_file_content: DataContentType::PositionDeletes, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, } .into(), ], data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -2838,12 +2844,14 @@ message schema { data_file_content: DataContentType::PositionDeletes, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, } .into(), ], data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3060,12 +3068,14 @@ message schema { data_file_content: DataContentType::PositionDeletes, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, } .into(), ], data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3169,6 +3179,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -3266,6 +3277,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -3352,6 +3364,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -3452,6 +3465,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -3581,6 +3595,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -3677,6 +3692,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; @@ -3786,6 +3802,7 @@ message schema { data_file_content: DataContentType::Data, sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, })] .into_iter(), )) as FileScanTaskStream; diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 81beb0c5ba..4170420dcf 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -140,6 +140,7 @@ impl ManifestEntryContext { deletes, sequence_number: self.manifest_entry.sequence_number().unwrap_or(0), equality_ids: self.manifest_entry.data_file().equality_ids(), + file_size_in_bytes: self.manifest_entry.data_file().file_size_in_bytes(), }) } } diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 57e9a8e331..cc60b7a3ec 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -1822,6 +1822,7 @@ pub mod tests { deletes: vec![], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; test_fn(task); @@ -1839,6 +1840,7 @@ pub mod tests { deletes: vec![], sequence_number: 0, equality_ids: None, + file_size_in_bytes: 0, }; test_fn(task); } diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 3f8c3acd77..de9e8ec806 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -63,6 +63,9 @@ pub struct FileScanTask { pub sequence_number: i64, /// equality ids pub equality_ids: Option>, + + /// The size of the file in bytes. + pub file_size_in_bytes: u64, } impl FileScanTask { @@ -128,6 +131,7 @@ impl From<&DeleteFileContext> for FileScanTask { deletes: vec![], sequence_number: ctx.manifest_entry.sequence_number().unwrap_or(0), equality_ids: ctx.manifest_entry.data_file().equality_ids(), + file_size_in_bytes: ctx.manifest_entry.data_file().file_size_in_bytes(), } } } diff --git a/crates/integrations/datafusion/src/physical_plan/mod.rs b/crates/integrations/datafusion/src/physical_plan/mod.rs index eb58082fe5..049a30c52b 100644 --- a/crates/integrations/datafusion/src/physical_plan/mod.rs +++ b/crates/integrations/datafusion/src/physical_plan/mod.rs @@ -16,7 +16,7 @@ // under the License. pub(crate) mod commit; -pub(crate) mod expr_to_predicate; +pub mod expr_to_predicate; pub(crate) mod metadata_scan; pub(crate) mod project; pub(crate) mod repartition; From 64e7e21600405e6d0d323c8b7600f6d4e4624f60 Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 30 May 2025 17:40:01 +0800 Subject: [PATCH 16/33] fix(iceberg): fix rewrite-files partition-spec-id (#54) * fix(iceberg): fix rewrite-files partition-spec-id * fix(docker): update docker file * add test * update minio * Revert "update minio" This reverts commit 4464d905ff42c0fd631a07fcfb4311941af2450d. --- crates/iceberg/src/spec/manifest/data_file.rs | 6 + crates/iceberg/src/transaction/mod.rs | 2 +- .../iceberg/src/transaction/rewrite_files.rs | 2 +- .../testdata/spark/Dockerfile | 2 +- .../tests/shared_tests/rewrite_files_test.rs | 113 ++++++++++++++++++ 5 files changed, 122 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/spec/manifest/data_file.rs b/crates/iceberg/src/spec/manifest/data_file.rs index a9c041f540..5af13438e4 100644 --- a/crates/iceberg/src/spec/manifest/data_file.rs +++ b/crates/iceberg/src/spec/manifest/data_file.rs @@ -274,6 +274,12 @@ impl DataFile { pub fn referenced_data_file(&self) -> Option { self.referenced_data_file.clone() } + + /// Get the partition spec id of the data file. + pub fn partition_spec_id(&self) -> i32 { + self.partition_spec_id + } + /// Get the offset in the file where the blob content starts. /// Only meaningful for puffin blobs, and required for deletion vectors. pub fn content_offset(&self) -> Option { diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 0a8058c6a8..5c24b0bb4b 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -183,7 +183,7 @@ impl Transaction { } /// Creates rewrite files action. - pub fn rewrite_files(self) -> RewriteFilesAction { + pub fn rewrite_files(&self) -> RewriteFilesAction { RewriteFilesAction::new() } diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs index e3315b8f72..1fc06dc28e 100644 --- a/crates/iceberg/src/transaction/rewrite_files.rs +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -266,7 +266,7 @@ impl SnapshotProduceOperation for RewriteFilesOperation { { let mut manifest_writer = snapshot_produce.new_manifest_writer( ManifestContentType::Data, - table_metadata_ref.default_partition_spec_id(), + manifest_file.partition_spec_id, )?; for entry in manifest.entries() { diff --git a/crates/integration_tests/testdata/spark/Dockerfile b/crates/integration_tests/testdata/spark/Dockerfile index e31e2273d9..4ac7489109 100644 --- a/crates/integration_tests/testdata/spark/Dockerfile +++ b/crates/integration_tests/testdata/spark/Dockerfile @@ -31,7 +31,7 @@ ENV SPARK_VERSION=3.5.7 ENV ICEBERG_SPARK_RUNTIME_VERSION=3.5_2.12 ENV ICEBERG_VERSION=1.10.0 -RUN curl --retry 5 -s -C - https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop3.tgz -o spark-${SPARK_VERSION}-bin-hadoop3.tgz \ +RUN curl --retry 5 -s -C - https://archive.apache.org/dist/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop3.tgz -o spark-${SPARK_VERSION}-bin-hadoop3.tgz \ && tar xzf spark-${SPARK_VERSION}-bin-hadoop3.tgz --directory /opt/spark --strip-components 1 \ && rm -rf spark-${SPARK_VERSION}-bin-hadoop3.tgz diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs index af3881a526..96115bc75f 100644 --- a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -21,6 +21,8 @@ use std::sync::Arc; use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; use futures::TryStreamExt; +use iceberg::spec::DataFile; +use iceberg::table::Table; use iceberg::transaction::{ApplyTransactionAction, Transaction}; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; use iceberg::writer::file_writer::ParquetWriterBuilder; @@ -453,3 +455,114 @@ async fn test_sequence_number_in_manifest_entry() { } } } + +#[tokio::test] +async fn test_partition_spec_id_in_manifest() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let mut table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + + // commit result + let mut data_files_vec = Vec::default(); + + async fn build_data_file_f( + schema: Arc, + table: &Table, + location_generator: DefaultLocationGenerator, + file_name_generator: DefaultFileNameGenerator, + ) -> DataFile { + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + data_file_writer.close().await.unwrap()[0].clone() + } + + for _ in 0..10 { + let data_file = build_data_file_f( + schema.clone(), + &table, + location_generator.clone(), + file_name_generator.clone(), + ) + .await; + data_files_vec.push(data_file.clone()); + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(vec![data_file]); + let tx = append_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + } + + let last_data_files = data_files_vec.last().unwrap(); + let partition_spec_id = last_data_files.partition_spec_id(); + + // remove the data files by RewriteAction + for data_file in &data_files_vec { + let tx = Transaction::new(&table); + let rewrite_action = tx.rewrite_files().delete_files(vec![data_file.clone()]); + let tx = rewrite_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + } + + // TODO: test update partition spec + // Verify that the partition spec ID is correctly set + + let last_snapshot = table.metadata().current_snapshot().unwrap(); + let manifest_list = last_snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list.entries().len(), 1); + for manifest_file in manifest_list.entries() { + assert_eq!(manifest_file.partition_spec_id, partition_spec_id); + } +} From 89808e434b923386d7e16efc1580dd9025588286 Mon Sep 17 00:00:00 2001 From: Alex Graham Date: Fri, 27 Jun 2025 10:52:25 -0500 Subject: [PATCH 17/33] Feature: Optionally configure consistent chunk sizes for multi-part uploads (#57) --- crates/iceberg/src/io/file_io.rs | 49 ++++++++++++++++++++++++++++++-- 1 file changed, 46 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 48d19c7723..5dab5ed898 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -27,6 +27,13 @@ use url::Url; use super::storage::Storage; use crate::{Error, ErrorKind, Result}; +/// Configuration property for setting the chunk size for IO write operations. +/// +/// This is useful for FileIO operations which may use multipart uploads (e.g. for S3) +/// where consistent chunk sizes of a certain size may be more optimal. Some services +/// like Cloudlare R2 requires all chunk sizes to be consistent except for the last. +pub const IO_CHUNK_SIZE: &str = "io.write.chunk-size"; + /// FileIO implementation, used to manipulate files in underlying storage. /// /// # Note @@ -165,8 +172,27 @@ impl FileIO { op, path, relative_path_pos, + chunk_size: self.get_write_chunk_size()?, }) } + + fn get_write_chunk_size(&self) -> Result> { + match self.builder.props.get(IO_CHUNK_SIZE) { + Some(chunk_size) => { + let parsed_chunk_size = chunk_size.parse::().map_err(|_err| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Invalid {}: Cannot parse to unsigned integer.", + IO_CHUNK_SIZE, + ), + ) + })?; + Ok(Some(parsed_chunk_size)) + } + _ => Ok(None), + } + } } /// Container for storing type-safe extensions used to configure underlying FileIO behavior. @@ -410,6 +436,8 @@ pub struct OutputFile { path: String, // Relative path of file to uri, starts at [`relative_path_pos`] relative_path_pos: usize, + // Chunk size for write operations to ensure consistent size of multipart chunks + chunk_size: Option, } impl OutputFile { @@ -457,9 +485,11 @@ impl OutputFile { /// /// For one-time writing, use [`Self::write`] instead. pub async fn writer(&self) -> crate::Result> { - Ok(Box::new( - self.op.writer(&self.path[self.relative_path_pos..]).await?, - )) + let mut writer = self.op.writer_with(&self.path[self.relative_path_pos..]); + if let Some(chunk_size) = self.chunk_size { + writer = writer.chunk(chunk_size); + } + Ok(Box::new(writer.await?)) } } @@ -475,6 +505,7 @@ mod tests { use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; + use crate::io::IO_CHUNK_SIZE; fn create_local_file_io() -> FileIO { FileIOBuilder::new_fs_io().build().unwrap() @@ -618,4 +649,16 @@ mod tests { io.delete(&path).await.unwrap(); assert!(!io.exists(&path).await.unwrap()); } + + #[tokio::test] + async fn test_set_chunk_size() { + let io = FileIOBuilder::new("memory") + .with_prop(IO_CHUNK_SIZE, 32 * 1024 * 1024) + .build() + .unwrap(); + + let path = format!("{}/1.txt", TempDir::new().unwrap().path().to_str().unwrap()); + let output_file = io.new_output(&path).unwrap(); + assert_eq!(Some(32 * 1024 * 1024), output_file.chunk_size); + } } From 118fab3c1e4258dfe92607e7ddba289803483c89 Mon Sep 17 00:00:00 2001 From: Li0k Date: Tue, 22 Jul 2025 18:07:21 +0800 Subject: [PATCH 18/33] feat: support write to branch (#62) * feat: support to branch * fix: fix ref name * fix: current_snapshot_id * refactor: refactor interface * fmt --- crates/iceberg/src/transaction/append.rs | 119 ++++++++++++- .../iceberg/src/transaction/rewrite_files.rs | 24 ++- crates/iceberg/src/transaction/snapshot.rs | 40 +++-- .../shared_tests/append_data_file_test.rs | 156 ++++++++++++++++++ .../tests/shared_tests/rewrite_files_test.rs | 6 +- 5 files changed, 320 insertions(+), 25 deletions(-) diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index f6aea7263e..b67831f888 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -45,6 +45,7 @@ pub struct FastAppendAction { snapshot_properties: HashMap, added_data_files: Vec, added_delete_files: Vec, + target_branch: Option, } impl FastAppendAction { @@ -57,15 +58,21 @@ impl FastAppendAction { snapshot_properties: HashMap::default(), added_data_files: vec![], added_delete_files: vec![], + target_branch: None, } } /// Set whether to check duplicate files - pub fn with_check_duplicate(mut self, v: bool) -> Self { + pub fn set_check_duplicate(mut self, v: bool) -> Self { self.check_duplicate = v; self } + pub fn set_target_branch(mut self, target_branch: String) -> Self { + self.target_branch = Some(target_branch); + self + } + /// Add data files to the snapshot. pub fn add_data_files(mut self, data_files: impl IntoIterator) -> Self { for file in data_files { @@ -125,7 +132,8 @@ impl TransactionAction for FastAppendAction { )); } } - let snapshot_producer = SnapshotProducer::new( + + let mut snapshot_producer = SnapshotProducer::new( table, self.commit_uuid.unwrap_or_else(Uuid::now_v7), self.key_metadata.clone(), @@ -137,6 +145,10 @@ impl TransactionAction for FastAppendAction { vec![], ); + if let Some(target_branch) = &self.target_branch { + snapshot_producer.set_target_branch(target_branch.clone()); + } + // validate added files snapshot_producer.validate_added_data_files(&self.added_data_files)?; snapshot_producer.validate_added_data_files(&self.added_delete_files)?; @@ -212,6 +224,7 @@ pub struct MergeAppendAction { added_data_files: Vec, added_delete_files: Vec, snapshot_id: Option, + target_branch: Option, } impl MergeAppendAction { @@ -227,6 +240,7 @@ impl MergeAppendAction { added_data_files: vec![], added_delete_files: vec![], snapshot_id: None, + target_branch: None, } } @@ -267,6 +281,11 @@ impl MergeAppendAction { self } + pub fn set_target_branch(mut self, target_branch: String) -> Self { + self.target_branch = Some(target_branch); + self + } + /// Add data files to the snapshot. pub fn add_data_files(mut self, data_files: impl IntoIterator) -> Self { self.added_data_files.extend(data_files); @@ -277,7 +296,7 @@ impl MergeAppendAction { #[async_trait] impl TransactionAction for MergeAppendAction { async fn commit(self: Arc, table: &Table) -> Result { - let snapshot_producer = SnapshotProducer::new( + let mut snapshot_producer = SnapshotProducer::new( table, self.commit_uuid.unwrap_or_else(Uuid::now_v7), self.key_metadata.clone(), @@ -289,6 +308,10 @@ impl TransactionAction for MergeAppendAction { vec![], ); + if let Some(target_branch) = &self.target_branch { + snapshot_producer.set_target_branch(target_branch.clone()); + } + // validate added files snapshot_producer.validate_added_data_files(&self.added_data_files)?; snapshot_producer.validate_added_data_files(&self.added_delete_files)?; @@ -507,4 +530,94 @@ mod tests { ); assert_eq!(data_file, *manifest.entries()[0].data_file()); } + + #[tokio::test] + async fn test_fast_append_with_branch() { + let table = make_v2_minimal_table(); + let tx = Transaction::new(&table); + + // Test creating new branch + let branch_name = "test_branch"; + + let data_file = DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path("test/3.parquet".to_string()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition_spec_id(table.metadata().default_partition_spec_id()) + .partition(Struct::from_iter([Some(Literal::long(300))])) + .build() + .unwrap(); + let action = tx + .fast_append() + .set_target_branch(branch_name.to_string()) + .add_data_files(vec![data_file.clone()]); + + let mut action_commit = Arc::new(action).commit(&table).await.unwrap(); + let updates = action_commit.take_updates(); + + // Check branch reference was created + assert!( + matches!(&updates[1], TableUpdate::SetSnapshotRef { ref_name, .. } + if ref_name == branch_name) + ); + + // Test updating existing branch + let action2 = tx + .fast_append() + .set_target_branch(branch_name.to_string()) + .add_data_files(vec![data_file.clone()]); + let mut action_commit2 = Arc::new(action2).commit(&table).await.unwrap(); + let requirements = action_commit2.take_requirements(); + + // Check requirements contain branch validation + assert!(requirements.iter().any( + |r| matches!(r, TableRequirement::RefSnapshotIdMatch { r#ref, .. } + if r#ref == branch_name) + )); + } + + #[tokio::test] + async fn test_branch_operations() { + let table = make_v2_minimal_table(); + + // Test creating new branch + let branch_name = "test_branch"; + let tx = Transaction::new(&table); + + let data_file = DataFileBuilder::default() + .partition_spec_id(0) + .content(DataContentType::Data) + .file_path("test/3.parquet".to_string()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition_spec_id(table.metadata().default_partition_spec_id()) + .partition(Struct::from_iter([Some(Literal::long(300))])) + .build() + .unwrap(); + + let action = tx + .fast_append() + .set_target_branch(branch_name.to_string()) + .add_data_files(vec![data_file.clone()]); + + let mut action_commit = Arc::new(action).commit(&table).await.unwrap(); + let updates = action_commit.take_updates(); + let requirements = action_commit.take_requirements(); + + // Verify branch was created + assert!(matches!( + &updates[1], + TableUpdate::SetSnapshotRef { ref_name, .. } if ref_name == branch_name + )); + + // Verify requirements + assert!(requirements.iter().any( + |r| matches!(r, TableRequirement::RefSnapshotIdMatch { r#ref, .. } + if r#ref == branch_name) + )); + } } diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs index 1fc06dc28e..ec33bac263 100644 --- a/crates/iceberg/src/transaction/rewrite_files.rs +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -55,6 +55,8 @@ pub struct RewriteFilesAction { snapshot_id: Option, new_data_file_sequence_number: Option, + + target_branch: Option, } struct RewriteFilesOperation; @@ -75,6 +77,7 @@ impl RewriteFilesAction { removed_delete_files: Vec::new(), snapshot_id: None, new_data_file_sequence_number: None, + target_branch: None, } } @@ -146,10 +149,15 @@ impl RewriteFilesAction { self } - pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Result { + pub fn set_target_branch(mut self, target_branch: String) -> Self { + self.target_branch = Some(target_branch); + self + } + + pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Self { self.new_data_file_sequence_number = Some(seq); - Ok(self) + self } } @@ -299,6 +307,14 @@ impl TransactionAction for RewriteFilesAction { self.removed_delete_files.clone(), ); + if let Some(seq) = self.new_data_file_sequence_number { + snapshot_producer.set_new_data_file_sequence_number(seq); + } + + if let Some(branch) = &self.target_branch { + snapshot_producer.set_target_branch(branch.clone()); + } + // Checks duplicate files if self.check_duplicate { snapshot_producer @@ -310,10 +326,6 @@ impl TransactionAction for RewriteFilesAction { .await?; } - if let Some(seq) = self.new_data_file_sequence_number { - snapshot_producer.set_new_data_file_sequence_number(seq); - } - if self.merge_enabled { let process = MergeManifestProcess::new(self.target_size_bytes, self.min_count_to_merge); diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 34d187866e..cc7c1580c4 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -93,6 +93,8 @@ pub(crate) struct SnapshotProducer<'a> { manifest_counter: RangeFrom, new_data_file_sequence_number: Option, + + target_branch: String, } impl<'a> SnapshotProducer<'a> { @@ -130,6 +132,7 @@ impl<'a> SnapshotProducer<'a> { removed_delete_file_paths, manifest_counter: (0..), new_data_file_sequence_number: None, + target_branch: MAIN_BRANCH.to_string(), } } @@ -161,7 +164,8 @@ impl<'a> SnapshotProducer<'a> { .collect(); let mut referenced_files = Vec::new(); - if let Some(current_snapshot) = self.table.metadata().current_snapshot() { + if let Some(current_snapshot) = self.table.metadata().snapshot_for_ref(&self.target_branch) + { let manifest_list = current_snapshot .load_manifest_list(self.table.file_io(), &self.table.metadata_ref()) .await?; @@ -526,22 +530,29 @@ impl<'a> SnapshotProducer<'a> { process: MP, ) -> Result { let manifest_list_path = self.generate_manifest_list_file_path(0); - let next_seq_num = self.table.metadata().next_sequence_number(); - let first_row_id = self.table.metadata().next_row_id(); - let mut manifest_list_writer = match self.table.metadata().format_version() { + let metadata_ref = self.table.metadata_ref(); + let next_seq_num = metadata_ref.next_sequence_number(); + let first_row_id = metadata_ref.next_row_id(); + let parent_snapshot = metadata_ref.snapshot_for_ref(&self.target_branch); + + let parent_snapshot_id = parent_snapshot + .map(|s| Some(s.snapshot_id())) + .unwrap_or(None); + + let mut manifest_list_writer = match metadata_ref.format_version() { FormatVersion::V1 => ManifestListWriter::v1( self.table .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, - self.table.metadata().current_snapshot_id(), + parent_snapshot_id, ), FormatVersion::V2 => ManifestListWriter::v2( self.table .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, - self.table.metadata().current_snapshot_id(), + parent_snapshot_id, next_seq_num, ), FormatVersion::V3 => ManifestListWriter::v3( @@ -549,7 +560,7 @@ impl<'a> SnapshotProducer<'a> { .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, - self.table.metadata().current_snapshot_id(), + parent_snapshot_id, next_seq_num, Some(first_row_id), ), @@ -565,7 +576,6 @@ impl<'a> SnapshotProducer<'a> { let new_manifests = self .manifest_file(&snapshot_produce_operation, &process) .await?; - manifest_list_writer.add_manifests(new_manifests.into_iter())?; let writer_next_row_id = manifest_list_writer.next_row_id(); manifest_list_writer.close().await?; @@ -574,7 +584,7 @@ impl<'a> SnapshotProducer<'a> { let new_snapshot = Snapshot::builder() .with_manifest_list(manifest_list_path) .with_snapshot_id(self.snapshot_id) - .with_parent_snapshot_id(self.table.metadata().current_snapshot_id()) + .with_parent_snapshot_id(parent_snapshot_id) .with_sequence_number(next_seq_num) .with_summary(summary) .with_schema_id(self.table.metadata().current_schema_id()) @@ -594,7 +604,7 @@ impl<'a> SnapshotProducer<'a> { snapshot: new_snapshot, }, TableUpdate::SetSnapshotRef { - ref_name: MAIN_BRANCH.to_string(), + ref_name: self.target_branch.clone(), reference: SnapshotReference::new( self.snapshot_id, SnapshotRetention::branch(None, None, None), @@ -607,8 +617,8 @@ impl<'a> SnapshotProducer<'a> { uuid: self.table.metadata().uuid(), }, TableRequirement::RefSnapshotIdMatch { - r#ref: MAIN_BRANCH.to_string(), - snapshot_id: self.table.metadata().current_snapshot_id(), + r#ref: self.target_branch.clone(), + snapshot_id: parent_snapshot_id, }, ]; @@ -618,6 +628,10 @@ impl<'a> SnapshotProducer<'a> { pub fn set_new_data_file_sequence_number(&mut self, new_data_file_sequence_number: i64) { self.new_data_file_sequence_number = Some(new_data_file_sequence_number); } + + pub fn set_target_branch(&mut self, target_branch: String) { + self.target_branch = target_branch; + } } pub(crate) struct MergeManifestProcess { @@ -644,6 +658,7 @@ impl ManifestProcess for MergeManifestProcess { let (unmerge_data_manifest, unmerge_delete_manifest): (Vec<_>, Vec<_>) = manifests .into_iter() .partition(|manifest| matches!(manifest.content, ManifestContentType::Data)); + let mut data_manifest = { let manifest_merge_manager = MergeManifestManager::new( self.target_size_bytes, @@ -654,6 +669,7 @@ impl ManifestProcess for MergeManifestProcess { .merge_manifest(snapshot_produce, unmerge_data_manifest) .await? }; + data_manifest.extend(unmerge_delete_manifest); Ok(data_manifest) } diff --git a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs index bedc975102..34680de249 100644 --- a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs @@ -136,3 +136,159 @@ async fn test_append_data_file() { assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); } + +#[tokio::test] +async fn test_append_data_file_target_branch() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + // Test 1: Append to main branch (default behavior) + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify main branch has the data + assert!(table.metadata().current_snapshot().is_some()); + let main_snapshot_id = table.metadata().current_snapshot().unwrap().snapshot_id(); + + // Verify main branch ref points to the snapshot using snapshot_for_ref + let main_snapshot = table.metadata().snapshot_for_ref("main").unwrap(); + assert_eq!(main_snapshot.snapshot_id(), main_snapshot_id); + + // Test 2: Append to a custom branch + let branch_name = "test-branch"; + let tx = Transaction::new(&table); + let append_action = tx + .fast_append() + .set_target_branch(branch_name.to_string()) + .add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify the custom branch was created and points to a new snapshot + let branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); + assert_ne!(branch_snapshot.snapshot_id(), main_snapshot_id); + + // Verify the main branch is unchanged + let main_snapshot_after = table.metadata().snapshot_for_ref("main").unwrap(); + assert_eq!(main_snapshot_after.snapshot_id(), main_snapshot_id); + + // Test 3: Append to the same custom branch again + let tx = Transaction::new(&table); + let append_action = tx + .fast_append() + .set_target_branch(branch_name.to_string()) + .add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify the custom branch now points to a newer snapshot + let branch_snapshot_final = table.metadata().snapshot_for_ref(branch_name).unwrap(); + assert_ne!( + branch_snapshot_final.snapshot_id(), + branch_snapshot.snapshot_id() + ); + assert_ne!(branch_snapshot_final.snapshot_id(), main_snapshot_id); + + // Verify we have 3 snapshots total (1 main + 2 branch) + assert_eq!(table.metadata().snapshots().count(), 3); + + // Test 4: Test merge append to branch + let another_branch = "merge-branch"; + let tx = Transaction::new(&table); + let merge_append_action = tx + .merge_append() + .set_target_branch(another_branch.to_string()) + .add_data_files(data_file.clone()); + let tx = merge_append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify the merge branch was created + let merge_branch_snapshot = table.metadata().snapshot_for_ref(another_branch).unwrap(); + assert_ne!(merge_branch_snapshot.snapshot_id(), main_snapshot_id); + assert_ne!( + merge_branch_snapshot.snapshot_id(), + branch_snapshot_final.snapshot_id() + ); + + // Verify we now have 4 snapshots total + assert_eq!(table.metadata().snapshots().count(), 4); + + // Verify all branches exist and can be accessed via snapshot_for_ref + assert!(table.metadata().snapshot_for_ref("main").is_some()); + assert!(table.metadata().snapshot_for_ref(branch_name).is_some()); + assert!(table.metadata().snapshot_for_ref(another_branch).is_some()); + + // Verify each branch points to different snapshots + let final_main_snapshot = table.metadata().snapshot_for_ref("main").unwrap(); + let final_branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); + let final_merge_snapshot = table.metadata().snapshot_for_ref(another_branch).unwrap(); + + assert_ne!( + final_main_snapshot.snapshot_id(), + final_branch_snapshot.snapshot_id() + ); + assert_ne!( + final_main_snapshot.snapshot_id(), + final_merge_snapshot.snapshot_id() + ); + assert_ne!( + final_branch_snapshot.snapshot_id(), + final_merge_snapshot.snapshot_id() + ); +} diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs index 96115bc75f..28d85fc847 100644 --- a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -144,7 +144,7 @@ async fn test_rewrite_data_files() { let tx = Transaction::new(&table); let append_action = tx .fast_append() - .with_check_duplicate(false) + .set_check_duplicate(false) .add_data_files(data_file.clone()); let tx = append_action.apply(tx).unwrap(); let table = tx.commit(&rest_catalog).await.unwrap(); @@ -433,9 +433,7 @@ async fn test_sequence_number_in_manifest_entry() { .add_data_files(data_file1.clone()) .add_data_files(data_file2.clone()); // Set sequence number to 12345 - let rewrite_action = rewrite_action - .set_new_data_file_sequence_number(12345) - .unwrap(); + let rewrite_action = rewrite_action.set_new_data_file_sequence_number(12345); let tx = rewrite_action.apply(tx).unwrap(); let table = tx.commit(&rest_catalog).await.unwrap(); From 75eb48d24b7e5469bd386633966608dfba554cf7 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 24 Jul 2025 15:30:34 +0800 Subject: [PATCH 19/33] feat: support overwrite files action (#63) * fix: fix snapshot for entries * refactor: refactor starting_sequence_number * feat: OverwriteFiles Action * fix: fix compile * fix: add more test * typo * fix: fix ut * fix: fix check --- crates/iceberg/src/transaction/mod.rs | 7 + .../src/transaction/overwrite_files.rs | 356 ++++++ .../iceberg/src/transaction/rewrite_files.rs | 21 +- crates/iceberg/src/transaction/snapshot.rs | 4 + .../shared_tests/append_data_file_test.rs | 109 +- .../tests/shared_tests/mod.rs | 1 + .../shared_tests/overwrite_files_test.rs | 1068 +++++++++++++++++ .../tests/shared_tests/rewrite_files_test.rs | 552 +++++++-- 8 files changed, 2005 insertions(+), 113 deletions(-) create mode 100644 crates/iceberg/src/transaction/overwrite_files.rs create mode 100644 crates/integration_tests/tests/shared_tests/overwrite_files_test.rs diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 5c24b0bb4b..0f1b82a114 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -54,6 +54,7 @@ mod action; pub use action::*; mod append; +mod overwrite_files; mod remove_snapshots; mod rewrite_files; mod snapshot; @@ -74,6 +75,7 @@ use crate::error::Result; use crate::spec::TableProperties; use crate::table::Table; use crate::transaction::append::{FastAppendAction, MergeAppendAction}; +use crate::transaction::overwrite_files::OverwriteFilesAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::transaction::update_location::UpdateLocationAction; use crate::transaction::update_properties::UpdatePropertiesAction; @@ -187,6 +189,11 @@ impl Transaction { RewriteFilesAction::new() } + /// Creates an overwrite files action. + pub fn overwrite_files(&self) -> OverwriteFilesAction { + OverwriteFilesAction::new() + } + /// Commit transaction. pub async fn commit(self, catalog: &dyn Catalog) -> Result
{ if self.actions.is_empty() { diff --git a/crates/iceberg/src/transaction/overwrite_files.rs b/crates/iceberg/src/transaction/overwrite_files.rs new file mode 100644 index 0000000000..14ff34f354 --- /dev/null +++ b/crates/iceberg/src/transaction/overwrite_files.rs @@ -0,0 +1,356 @@ +// 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, HashSet}; +use std::sync::Arc; + +use uuid::Uuid; + +use super::snapshot::{DefaultManifestProcess, MergeManifestProcess, SnapshotProducer}; +use super::{ + MANIFEST_MERGE_ENABLED, MANIFEST_MERGE_ENABLED_DEFAULT, MANIFEST_MIN_MERGE_COUNT, + MANIFEST_MIN_MERGE_COUNT_DEFAULT, MANIFEST_TARGET_SIZE_BYTES, + MANIFEST_TARGET_SIZE_BYTES_DEFAULT, +}; +use crate::error::Result; +use crate::spec::{ + DataContentType, DataFile, ManifestContentType, ManifestEntry, ManifestFile, ManifestStatus, + Operation, +}; +use crate::table::Table; +use crate::transaction::snapshot::SnapshotProduceOperation; +use crate::transaction::{ActionCommit, TransactionAction}; + +/// Transaction action for rewriting files. +pub struct OverwriteFilesAction { + // snapshot_produce_action: SnapshotProduceAction<'a>, + target_size_bytes: u32, + min_count_to_merge: u32, + merge_enabled: bool, + + check_duplicate: bool, + // below are properties used to create SnapshotProducer when commit + commit_uuid: Option, + key_metadata: Option>, + snapshot_properties: HashMap, + added_data_files: Vec, + added_delete_files: Vec, + removed_data_files: Vec, + removed_delete_files: Vec, + snapshot_id: Option, + + new_data_file_sequence_number: Option, + + target_branch: Option, +} + +pub struct OverwriteFilesOperation; + +impl OverwriteFilesAction { + pub fn new() -> Self { + Self { + target_size_bytes: MANIFEST_TARGET_SIZE_BYTES_DEFAULT, + min_count_to_merge: MANIFEST_MIN_MERGE_COUNT_DEFAULT, + merge_enabled: MANIFEST_MERGE_ENABLED_DEFAULT, + check_duplicate: true, + commit_uuid: None, + key_metadata: None, + snapshot_properties: HashMap::new(), + added_data_files: Vec::new(), + added_delete_files: Vec::new(), + removed_data_files: Vec::new(), + removed_delete_files: Vec::new(), + snapshot_id: None, + new_data_file_sequence_number: None, + target_branch: None, + } + } + + /// Add data files to the snapshot. + pub fn add_data_files(mut self, data_files: impl IntoIterator) -> Self { + for file in data_files { + match file.content_type() { + DataContentType::Data => self.added_data_files.push(file), + DataContentType::PositionDeletes | DataContentType::EqualityDeletes => { + self.added_delete_files.push(file) + } + } + } + + self + } + + /// Add remove files to the snapshot. + pub fn delete_files(mut self, remove_data_files: impl IntoIterator) -> Self { + for file in remove_data_files { + match file.content_type() { + DataContentType::Data => self.removed_data_files.push(file), + DataContentType::PositionDeletes | DataContentType::EqualityDeletes => { + self.removed_delete_files.push(file) + } + } + } + + self + } + + pub fn set_snapshot_properties(&mut self, properties: HashMap) -> &mut Self { + let target_size_bytes: u32 = properties + .get(MANIFEST_TARGET_SIZE_BYTES) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + let min_count_to_merge: u32 = properties + .get(MANIFEST_MIN_MERGE_COUNT) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MIN_MERGE_COUNT_DEFAULT); + let merge_enabled = properties + .get(MANIFEST_MERGE_ENABLED) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MERGE_ENABLED_DEFAULT); + + self.target_size_bytes = target_size_bytes; + self.min_count_to_merge = min_count_to_merge; + self.merge_enabled = merge_enabled; + self.snapshot_properties = properties; + + self + } + + /// Set commit UUID for the snapshot. + pub fn set_commit_uuid(&mut self, commit_uuid: Uuid) -> &mut Self { + self.commit_uuid = Some(commit_uuid); + self + } + + /// Set key metadata for manifest files. + pub fn set_key_metadata(mut self, key_metadata: Vec) -> Self { + self.key_metadata = Some(key_metadata); + self + } + + /// Set snapshot id + pub fn set_snapshot_id(mut self, snapshot_id: i64) -> Self { + self.snapshot_id = Some(snapshot_id); + self + } + + pub fn set_target_branch(mut self, target_branch: String) -> Self { + self.target_branch = Some(target_branch); + self + } + + // If the compaction should use the sequence number of the snapshot at compaction start time for + // new data files, instead of using the sequence number of the newly produced snapshot. + // This avoids commit conflicts with updates that add newer equality deletes at a higher sequence number. + pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Self { + self.new_data_file_sequence_number = Some(seq); + + self + } +} + +impl SnapshotProduceOperation for OverwriteFilesOperation { + fn operation(&self) -> Operation { + Operation::Overwrite + } + + async fn delete_entries( + &self, + snapshot_produce: &SnapshotProducer<'_>, + ) -> Result> { + // generate delete manifest entries from removed files + let snapshot = snapshot_produce + .table + .metadata() + .snapshot_for_ref(snapshot_produce.target_branch()); + + if let Some(snapshot) = snapshot { + let gen_manifest_entry = |old_entry: &Arc| { + let builder = ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(old_entry.snapshot_id().unwrap()) + .sequence_number(old_entry.sequence_number().unwrap()) + .file_sequence_number(old_entry.file_sequence_number().unwrap()) + .data_file(old_entry.data_file().clone()); + + builder.build() + }; + + let manifest_list = snapshot + .load_manifest_list( + snapshot_produce.table.file_io(), + snapshot_produce.table.metadata(), + ) + .await?; + + let mut deleted_entries = Vec::new(); + + for manifest_file in manifest_list.entries() { + let manifest = manifest_file + .load_manifest(snapshot_produce.table.file_io()) + .await?; + + for entry in manifest.entries() { + if entry.content_type() == DataContentType::Data + && snapshot_produce + .removed_data_file_paths + .contains(entry.data_file().file_path()) + { + deleted_entries.push(gen_manifest_entry(entry)); + } + + if entry.content_type() == DataContentType::PositionDeletes + || entry.content_type() == DataContentType::EqualityDeletes + && snapshot_produce + .removed_delete_file_paths + .contains(entry.data_file().file_path()) + { + deleted_entries.push(gen_manifest_entry(entry)); + } + } + } + + Ok(deleted_entries) + } else { + Ok(vec![]) + } + } + + async fn existing_manifest( + &self, + snapshot_produce: &mut SnapshotProducer<'_>, + ) -> Result> { + let table_metadata_ref = snapshot_produce.table.metadata(); + let file_io_ref = snapshot_produce.table.file_io(); + + let Some(snapshot) = snapshot_produce + .table + .metadata() + .snapshot_for_ref(snapshot_produce.target_branch()) + else { + return Ok(vec![]); + }; + + let manifest_list = snapshot + .load_manifest_list(file_io_ref, table_metadata_ref) + .await?; + + let mut existing_files = Vec::new(); + + for manifest_file in manifest_list.entries() { + let manifest = manifest_file.load_manifest(file_io_ref).await?; + + let found_deleted_files: HashSet<_> = manifest + .entries() + .iter() + .filter_map(|entry| { + if snapshot_produce + .removed_data_file_paths + .contains(entry.data_file().file_path()) + || snapshot_produce + .removed_delete_file_paths + .contains(entry.data_file().file_path()) + { + Some(entry.data_file().file_path().to_string()) + } else { + None + } + }) + .collect(); + + if found_deleted_files.is_empty() { + existing_files.push(manifest_file.clone()); + } else { + // Rewrite the manifest file without the deleted data files + if manifest + .entries() + .iter() + .any(|entry| !found_deleted_files.contains(entry.data_file().file_path())) + { + let mut manifest_writer = snapshot_produce.new_manifest_writer( + ManifestContentType::Data, + manifest_file.partition_spec_id, + )?; + + for entry in manifest.entries() { + if !found_deleted_files.contains(entry.data_file().file_path()) { + manifest_writer.add_entry((**entry).clone())?; + } + } + + existing_files.push(manifest_writer.write_manifest_file().await?); + } + } + } + + Ok(existing_files) + } +} + +#[async_trait::async_trait] +impl TransactionAction for OverwriteFilesAction { + async fn commit(self: Arc, table: &Table) -> Result { + let mut snapshot_producer = SnapshotProducer::new( + table, + self.commit_uuid.unwrap_or_else(Uuid::now_v7), + self.key_metadata.clone(), + self.snapshot_id, + self.snapshot_properties.clone(), + self.added_data_files.clone(), + self.added_delete_files.clone(), + self.removed_data_files.clone(), + self.removed_delete_files.clone(), + ); + + if let Some(seq) = self.new_data_file_sequence_number { + snapshot_producer.set_new_data_file_sequence_number(seq); + } + + if let Some(branch) = &self.target_branch { + snapshot_producer.set_target_branch(branch.clone()); + } + + // Checks duplicate files + if self.check_duplicate { + snapshot_producer + .validate_duplicate_files(&self.added_data_files) + .await?; + + snapshot_producer + .validate_duplicate_files(&self.added_delete_files) + .await?; + } + + if self.merge_enabled { + let process = + MergeManifestProcess::new(self.target_size_bytes, self.min_count_to_merge); + snapshot_producer + .commit(OverwriteFilesOperation, process) + .await + } else { + snapshot_producer + .commit(OverwriteFilesOperation, DefaultManifestProcess) + .await + } + } +} + +impl Default for OverwriteFilesAction { + fn default() -> Self { + Self::new() + } +} diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs index ec33bac263..000f3dd1d8 100644 --- a/crates/iceberg/src/transaction/rewrite_files.rs +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -20,9 +20,7 @@ use std::sync::Arc; use uuid::Uuid; -use super::snapshot::{ - DefaultManifestProcess, MergeManifestProcess, SnapshotProduceOperation, SnapshotProducer, -}; +use super::snapshot::{DefaultManifestProcess, MergeManifestProcess, SnapshotProducer}; use super::{ MANIFEST_MERGE_ENABLED, MANIFEST_MERGE_ENABLED_DEFAULT, MANIFEST_MIN_MERGE_COUNT, MANIFEST_MIN_MERGE_COUNT_DEFAULT, MANIFEST_TARGET_SIZE_BYTES, @@ -34,6 +32,7 @@ use crate::spec::{ Operation, }; use crate::table::Table; +use crate::transaction::snapshot::SnapshotProduceOperation; use crate::transaction::{ActionCommit, TransactionAction}; /// Transaction action for rewriting files. @@ -59,7 +58,7 @@ pub struct RewriteFilesAction { target_branch: Option, } -struct RewriteFilesOperation; +pub struct RewriteFilesOperation; impl RewriteFilesAction { pub fn new() -> Self { @@ -154,6 +153,9 @@ impl RewriteFilesAction { self } + // If the compaction should use the sequence number of the snapshot at compaction start time for + // new data files, instead of using the sequence number of the newly produced snapshot. + // This avoids commit conflicts with updates that add newer equality deletes at a higher sequence number. pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Self { self.new_data_file_sequence_number = Some(seq); @@ -171,7 +173,10 @@ impl SnapshotProduceOperation for RewriteFilesOperation { snapshot_produce: &SnapshotProducer<'_>, ) -> Result> { // generate delete manifest entries from removed files - let snapshot = snapshot_produce.table.metadata().current_snapshot(); + let snapshot = snapshot_produce + .table + .metadata() + .snapshot_for_ref(snapshot_produce.target_branch()); if let Some(snapshot) = snapshot { let gen_manifest_entry = |old_entry: &Arc| { @@ -232,7 +237,11 @@ impl SnapshotProduceOperation for RewriteFilesOperation { let table_metadata_ref = snapshot_produce.table.metadata(); let file_io_ref = snapshot_produce.table.file_io(); - let Some(snapshot) = table_metadata_ref.current_snapshot() else { + let Some(snapshot) = snapshot_produce + .table + .metadata() + .snapshot_for_ref(snapshot_produce.target_branch()) + else { return Ok(vec![]); }; diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index cc7c1580c4..4ef4b9b787 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -632,6 +632,10 @@ impl<'a> SnapshotProducer<'a> { pub fn set_target_branch(&mut self, target_branch: String) { self.target_branch = target_branch; } + + pub fn target_branch(&self) -> &str { + &self.target_branch + } } pub(crate) struct MergeManifestProcess { diff --git a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs index 34680de249..d6abb5c0b7 100644 --- a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs @@ -208,7 +208,29 @@ async fn test_append_data_file_target_branch() { // Verify main branch ref points to the snapshot using snapshot_for_ref let main_snapshot = table.metadata().snapshot_for_ref("main").unwrap(); - assert_eq!(main_snapshot.snapshot_id(), main_snapshot_id); + // First commit should have no parent snapshot + assert_eq!(main_snapshot.parent_snapshot_id(), None); + + let main_snapshot = table.metadata().snapshot_for_ref("main").unwrap(); + + // Verify main branch data + let main_batch_stream = table + .scan() + .snapshot_id(main_snapshot.snapshot_id()) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let main_batches: Vec<_> = main_batch_stream.try_collect().await.unwrap(); + assert_eq!(main_batches.len(), 1); + assert_eq!(main_batches[0], batch); + assert_eq!( + main_batches[0].schema(), + batch.schema(), + "Main branch schema mismatch" + ); // Test 2: Append to a custom branch let branch_name = "test-branch"; @@ -223,6 +245,34 @@ async fn test_append_data_file_target_branch() { // Verify the custom branch was created and points to a new snapshot let branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); assert_ne!(branch_snapshot.snapshot_id(), main_snapshot_id); + // New branch should have no parent snapshot + assert_eq!( + table + .metadata() + .snapshot_for_ref(branch_name) + .unwrap() + .parent_snapshot_id(), + None + ); + + // Verify test-branch data + let branch_batch_stream = table + .scan() + .snapshot_id(branch_snapshot.snapshot_id()) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let branch_batches: Vec<_> = branch_batch_stream.try_collect().await.unwrap(); + assert_eq!(branch_batches.len(), 1); + assert_eq!(branch_batches[0], batch); + assert_eq!( + branch_batches[0].schema(), + batch.schema(), + "Test branch schema mismatch" + ); // Verify the main branch is unchanged let main_snapshot_after = table.metadata().snapshot_for_ref("main").unwrap(); @@ -244,6 +294,35 @@ async fn test_append_data_file_target_branch() { branch_snapshot.snapshot_id() ); assert_ne!(branch_snapshot_final.snapshot_id(), main_snapshot_id); + // Second append should have previous branch snapshot as parent + assert_eq!( + table + .metadata() + .snapshot_for_ref(branch_name) + .unwrap() + .parent_snapshot_id(), + Some(branch_snapshot.snapshot_id()) + ); + + // Verify test-branch data after second append + let branch_batch_stream = table + .scan() + .snapshot_id(branch_snapshot_final.snapshot_id()) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let branch_batches: Vec<_> = branch_batch_stream.try_collect().await.unwrap(); + assert_eq!(branch_batches.len(), 2); + assert_eq!(branch_batches[0], batch); + assert_eq!(branch_batches[1], batch); + assert_eq!( + branch_batches[0].schema(), + batch.schema(), + "Test branch schema mismatch after second append" + ); // Verify we have 3 snapshots total (1 main + 2 branch) assert_eq!(table.metadata().snapshots().count(), 3); @@ -265,10 +344,38 @@ async fn test_append_data_file_target_branch() { merge_branch_snapshot.snapshot_id(), branch_snapshot_final.snapshot_id() ); + // Merge branch should have no parent snapshot + assert_eq!( + table + .metadata() + .snapshot_for_ref(another_branch) + .unwrap() + .parent_snapshot_id(), + None + ); // Verify we now have 4 snapshots total assert_eq!(table.metadata().snapshots().count(), 4); + // Verify merge-branch data + let merge_batch_stream = table + .scan() + .snapshot_id(merge_branch_snapshot.snapshot_id()) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let merge_batches: Vec<_> = merge_batch_stream.try_collect().await.unwrap(); + assert_eq!(merge_batches.len(), 1); + assert_eq!(merge_batches[0], batch); + assert_eq!( + merge_batches[0].schema(), + batch.schema(), + "Merge branch schema mismatch" + ); + // Verify all branches exist and can be accessed via snapshot_for_ref assert!(table.metadata().snapshot_for_ref("main").is_some()); assert!(table.metadata().snapshot_for_ref(branch_name).is_some()); diff --git a/crates/integration_tests/tests/shared_tests/mod.rs b/crates/integration_tests/tests/shared_tests/mod.rs index 0ebe77ab7f..05dbac2587 100644 --- a/crates/integration_tests/tests/shared_tests/mod.rs +++ b/crates/integration_tests/tests/shared_tests/mod.rs @@ -28,6 +28,7 @@ mod append_partition_data_file_test; mod conflict_commit_test; mod datafusion; mod merge_append_test; +mod overwrite_files_test; mod read_evolved_schema; mod read_positional_deletes; mod remove_snapshots_test; diff --git a/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs b/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs new file mode 100644 index 0000000000..d0227969db --- /dev/null +++ b/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs @@ -0,0 +1,1068 @@ +// 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. + +//! Integration tests for rest catalog. + +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use iceberg::spec::{DataFile, Operation}; +use iceberg::table::Table; +use iceberg::transaction::{ApplyTransactionAction, Transaction}; +use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, CatalogBuilder, TableCreation}; +use iceberg_catalog_rest::RestCatalogBuilder; +use parquet::arrow::arrow_reader::ArrowReaderOptions; +use parquet::file::properties::WriterProperties; + +use crate::get_shared_containers; +use crate::shared_tests::{random_ns, test_schema}; + +// Helper function to create a data file writer builder +fn create_data_file_writer_builder( + table: &Table, +) -> DataFileWriterBuilder +{ + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + DataFileWriterBuilder::new(rolling_writer_builder) +} + +#[tokio::test] +async fn test_overwrite_data_files() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let data_file_writer_builder = create_data_file_writer_builder(&table); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + // check parquet file schema + let content = table + .file_io() + .new_input(data_file[0].file_path()) + .unwrap() + .read() + .await + .unwrap(); + let parquet_reader = parquet::arrow::arrow_reader::ArrowReaderMetadata::load( + &content, + ArrowReaderOptions::default(), + ) + .unwrap(); + let field_ids: Vec = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.self_type().get_basic_info().id()) + .collect(); + assert_eq!(field_ids, vec![1, 2, 3]); + + // commit result + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + + // commit result again + let tx = Transaction::new(&table); + let append_action = tx + .fast_append() + .set_check_duplicate(false) + .add_data_files(data_file.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check result again + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0], batch); + assert_eq!(batches[1], batch); + + let data_file_writer_builder = create_data_file_writer_builder(&table); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file_overwrite = data_file_writer.close().await.unwrap(); + + // commit result again + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .add_data_files(data_file_overwrite.clone()) + .delete_files(data_file.clone()); + + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + { + let snapshot = table.metadata().current_snapshot().unwrap(); + assert_eq!( + Operation::Overwrite, + snapshot.summary().operation, + "Expected operation to be Overwrite after rewriting files" + ); + } + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); +} + +#[tokio::test] +async fn test_empty_overwrite() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t2".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let tx = Transaction::new(&table); + let overwrite_action = tx.overwrite_files(); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert!(batches.is_empty()); +} + +#[tokio::test] +async fn test_multiple_file_overwrite() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t3".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file1 = data_file_writer.close().await.unwrap(); + + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file2 = data_file_writer.close().await.unwrap(); + + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .add_data_files(data_file1.clone()) + .add_data_files(data_file2.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0], batch); + assert_eq!(batches[1], batch); +} + +#[tokio::test] +async fn test_overwrite_nonexistent_file() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t4".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create a valid data file + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let valid_data_file = data_file_writer.close().await.unwrap(); + + // Create a nonexistent data file (simulated by not writing it) + let nonexistent_data_file = valid_data_file.clone(); + + let tx = Transaction::new(&table); + let overwrite_action = tx.overwrite_files(); + + // Attempt to delete the nonexistent file + let _overwrite_action = overwrite_action.delete_files(nonexistent_data_file); + + // No assertion needed - just verify builder compiles +} + +#[tokio::test] +async fn test_sequence_number_in_manifest_entry() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t3".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file1 = data_file_writer.close().await.unwrap(); + + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file2 = data_file_writer.close().await.unwrap(); + + // Commit with sequence number + + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .set_new_data_file_sequence_number(12345) + .add_data_files(data_file1.clone()) + .add_data_files(data_file2.clone()); + + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify manifest entry has correct sequence number + let snapshot = table.metadata().current_snapshot().unwrap(); + let manifest_list = snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + + assert_eq!(manifest_list.entries().len(), 1); + + for manifest_file in manifest_list.entries() { + let manifest = manifest_file.load_manifest(table.file_io()).await.unwrap(); + for entry in manifest.entries() { + assert_eq!(entry.sequence_number(), Some(12345)); + } + } +} + +#[tokio::test] +async fn test_partition_spec_id_in_manifest() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let mut table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + + // commit result + let mut data_files_vec = Vec::default(); + + async fn build_data_file_f( + schema: Arc, + table: &Table, + location_generator: DefaultLocationGenerator, + file_name_generator: DefaultFileNameGenerator, + ) -> DataFile { + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + data_file_writer.close().await.unwrap()[0].clone() + } + + for _ in 0..10 { + let data_file = build_data_file_f( + schema.clone(), + &table, + location_generator.clone(), + file_name_generator.clone(), + ) + .await; + data_files_vec.push(data_file.clone()); + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(vec![data_file]); + let tx = append_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + } + + let last_data_files = data_files_vec.last().unwrap(); + let partition_spec_id = last_data_files.partition_spec_id(); + + // remove the data files by overwriteAction + for data_file in &data_files_vec { + let tx = Transaction::new(&table); + let overwrite_action = tx.overwrite_files().delete_files(vec![data_file.clone()]); + let tx = overwrite_action.apply(tx).unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + } + + // TODO: test update partition spec + // Verify that the partition spec ID is correctly set + + let last_snapshot = table.metadata().current_snapshot().unwrap(); + let manifest_list = last_snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list.entries().len(), 1); + for manifest_file in manifest_list.entries() { + assert_eq!(manifest_file.partition_spec_id, partition_spec_id); + } +} + +#[tokio::test] +async fn test_overwrite_files_to_branch() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("branch_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Prepare test data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let original_files = data_file_writer.close().await.unwrap(); + + // First append to main branch + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(original_files.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify main branch has the data + let main_snapshot = table.metadata().current_snapshot().unwrap(); + assert_eq!(main_snapshot.parent_snapshot_id(), None); + + let branch_name = "test_branch"; + + // Prepare overwrite files + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let overwrite_files = data_file_writer.close().await.unwrap(); + + // overwrite files to the new branch + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .set_target_branch(branch_name.to_string()) + .add_data_files(overwrite_files.clone()) + .delete_files(original_files.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branch snapshot + let branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); + assert_eq!( + branch_snapshot.parent_snapshot_id(), + None, + "Branch snapshot should not have a parent snapshot" + ); + + // Verify main branch remains unchanged + let current_main_snapshot = table.metadata().snapshot_for_ref("main").unwrap(); + assert_eq!( + current_main_snapshot.snapshot_id(), + main_snapshot.snapshot_id() + ); + + // Verify data in branch + let branch_batch_stream = table + .scan() + .snapshot_id( + table + .metadata() + .snapshot_for_ref(branch_name) + .unwrap() + .snapshot_id(), + ) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let branch_batches: Vec<_> = branch_batch_stream.try_collect().await.unwrap(); + assert_eq!(branch_batches.len(), 1); + assert_eq!(branch_batches[0], batch); + + // Verify data in main branch remains unchanged + let main_batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let main_batches: Vec<_> = main_batch_stream.try_collect().await.unwrap(); + assert_eq!(main_batches.len(), 1); + assert_eq!(main_batches[0], batch); +} + +#[tokio::test] +async fn test_branch_snapshot_isolation() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("branch_snapshot_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create initial snapshot in main branch + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_files = data_file_writer.close().await.unwrap(); + + // Commit to main branch + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(data_files.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Create branch1 from main + let branch1 = "branch1"; + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .set_target_branch(branch1.to_string()) + .add_data_files(data_files.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branch1 snapshot + let branch1_snapshot = table.metadata().snapshot_for_ref(branch1).unwrap(); + assert_eq!( + branch1_snapshot.parent_snapshot_id(), + None, + "Branch1 should have main snapshot as parent" + ); + + // Create branch2 from branch1 + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .set_target_branch(branch1.to_string()) + .add_data_files(data_files.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branch1 snapshot + let branch1_snapshot_2 = table.metadata().snapshot_for_ref(branch1).unwrap(); + assert_eq!( + branch1_snapshot_2.parent_snapshot_id(), + Some(branch1_snapshot.snapshot_id()), + " Should have branch1 snapshot as parent" + ); +} + +#[tokio::test] +async fn test_overwrite_files_with_sequence_number_from_branch() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("branch_seq_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Prepare test data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let original_files = data_file_writer.close().await.unwrap(); + + // First append to main branch + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(original_files.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + let branch_name = "seq_branch"; + let tx = Transaction::new(&table); + let sequence_from_main = table + .metadata() + .snapshot_for_ref("main") + .unwrap() + .sequence_number(); + let overwrite_action = tx + .overwrite_files() + .set_target_branch(branch_name.to_string()) + .set_new_data_file_sequence_number(sequence_from_main) + .add_data_files(original_files.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check manifest entry for sequence number + let branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); + let manifest_list = branch_snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + + assert_eq!(manifest_list.entries().len(), 1); + let manifest_file = &manifest_list.entries()[0]; + let manifest = manifest_file.load_manifest(table.file_io()).await.unwrap(); + assert_eq!(manifest.entries().len(), 1); + let entry = &manifest.entries()[0]; + assert_eq!(entry.sequence_number(), Some(1)); +} + +#[tokio::test] +async fn test_multiple_branches_isolation() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("multi_branch_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Prepare test data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1.clone()) as ArrayRef, + Arc::new(col2.clone()) as ArrayRef, + Arc::new(col3.clone()) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let original_files = data_file_writer.close().await.unwrap(); + + // Create branch1 with original files + let branch1 = "branch1"; + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .set_target_branch(branch1.to_string()) + .add_data_files(original_files.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Create branch2 with modified data + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let modified_col1 = StringArray::from(vec![Some("modified"), Some("data"), None, Some("here")]); + let modified_batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(modified_col1) as ArrayRef, + Arc::new(col2.clone()) as ArrayRef, + Arc::new(col3.clone()) as ArrayRef, + ]) + .unwrap(); + data_file_writer + .write(modified_batch.clone()) + .await + .unwrap(); + let modified_files = data_file_writer.close().await.unwrap(); + + let branch2 = "branch2"; + let tx = Transaction::new(&table); + let overwrite_action = tx + .overwrite_files() + .set_target_branch(branch2.to_string()) + .add_data_files(modified_files.clone()); + let tx = overwrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branches are isolated + let branch1_data: Vec<_> = table + .scan() + .snapshot_id( + table + .metadata() + .snapshot_for_ref(branch1) + .unwrap() + .snapshot_id(), + ) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap() + .try_collect() + .await + .unwrap(); + assert_eq!(branch1_data[0], batch); + + let branch2_data: Vec<_> = table + .scan() + .snapshot_id( + table + .metadata() + .snapshot_for_ref(branch2) + .unwrap() + .snapshot_id(), + ) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap() + .try_collect() + .await + .unwrap(); + assert_eq!(branch2_data[0], modified_batch); +} diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs index 28d85fc847..f0ccf674dd 100644 --- a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; use futures::TryStreamExt; -use iceberg::spec::DataFile; +use iceberg::spec::{DataFile, Operation}; use iceberg::table::Table; use iceberg::transaction::{ApplyTransactionAction, Transaction}; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; @@ -39,6 +39,33 @@ use parquet::file::properties::WriterProperties; use crate::get_shared_containers; use crate::shared_tests::{random_ns, test_schema}; +// Helper function to create a data file writer builder +fn create_data_file_writer_builder( + table: &Table, +) -> DataFileWriterBuilder< + ParquetWriterBuilder, + DefaultLocationGenerator, + DefaultFileNameGenerator, +> { + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + table.file_io().clone(), + location_generator, + file_name_generator, + ); + DataFileWriterBuilder::new(rolling_writer_builder) +} + #[tokio::test] async fn test_rewrite_data_files() { let fixture = get_shared_containers(); @@ -68,23 +95,7 @@ async fn test_rewrite_data_files() { .try_into() .unwrap(), ); - let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); - let file_name_generator = DefaultFileNameGenerator::new( - "test".to_string(), - None, - iceberg::spec::DataFileFormat::Parquet, - ); - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::default(), - table.metadata().current_schema().clone(), - ); - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - table.file_io().clone(), - location_generator.clone(), - file_name_generator.clone(), - ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let data_file_writer_builder = create_data_file_writer_builder(&table); let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); @@ -163,18 +174,7 @@ async fn test_rewrite_data_files() { assert_eq!(batches[0], batch); assert_eq!(batches[1], batch); - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::default(), - table.metadata().current_schema().clone(), - ); - - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - table.file_io().clone(), - location_generator, - file_name_generator, - ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let data_file_writer_builder = create_data_file_writer_builder(&table); let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); data_file_writer.write(batch.clone()).await.unwrap(); let data_file_rewrite = data_file_writer.close().await.unwrap(); @@ -190,6 +190,15 @@ async fn test_rewrite_data_files() { let tx = rewrite_action.apply(tx).unwrap(); let table = tx.commit(&rest_catalog).await.unwrap(); + { + let snapshot = table.metadata().current_snapshot().unwrap(); + assert_eq!( + Operation::Replace, + snapshot.summary().operation, + "Expected operation to be Replace after rewriting files" + ); + } + // check result let batch_stream = table .scan() @@ -232,24 +241,7 @@ async fn test_multiple_file_rewrite() { .try_into() .unwrap(), ); - let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); - let file_name_generator = DefaultFileNameGenerator::new( - "test".to_string(), - None, - iceberg::spec::DataFileFormat::Parquet, - ); - - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::default(), - table.metadata().current_schema().clone(), - ); - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - table.file_io().clone(), - location_generator, - file_name_generator, - ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let data_file_writer_builder = create_data_file_writer_builder(&table); let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); @@ -318,23 +310,7 @@ async fn test_rewrite_nonexistent_file() { .try_into() .unwrap(), ); - let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); - let file_name_generator = DefaultFileNameGenerator::new( - "test".to_string(), - None, - iceberg::spec::DataFileFormat::Parquet, - ); - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::default(), - table.metadata().current_schema().clone(), - ); - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - table.file_io().clone(), - location_generator, - file_name_generator, - ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let data_file_writer_builder = create_data_file_writer_builder(&table); // Create a valid data file let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); @@ -388,25 +364,7 @@ async fn test_sequence_number_in_manifest_entry() { .try_into() .unwrap(), ); - let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); - let file_name_generator = DefaultFileNameGenerator::new( - "test".to_string(), - None, - iceberg::spec::DataFileFormat::Parquet, - ); - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::default(), - table.metadata().current_schema().clone(), - ); - - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - table.file_io().clone(), - location_generator, - file_name_generator, - ); - - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + let data_file_writer_builder = create_data_file_writer_builder(&table); let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); @@ -483,12 +441,6 @@ async fn test_partition_spec_id_in_manifest() { .try_into() .unwrap(), ); - let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); - let file_name_generator = DefaultFileNameGenerator::new( - "test".to_string(), - None, - iceberg::spec::DataFileFormat::Parquet, - ); // commit result let mut data_files_vec = Vec::default(); @@ -496,21 +448,8 @@ async fn test_partition_spec_id_in_manifest() { async fn build_data_file_f( schema: Arc, table: &Table, - location_generator: DefaultLocationGenerator, - file_name_generator: DefaultFileNameGenerator, ) -> DataFile { - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::default(), - table.metadata().current_schema().clone(), - ); - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - table.file_io().clone(), - location_generator.clone(), - file_name_generator.clone(), - ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); - + let data_file_writer_builder = create_data_file_writer_builder(table); let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); @@ -529,8 +468,6 @@ async fn test_partition_spec_id_in_manifest() { let data_file = build_data_file_f( schema.clone(), &table, - location_generator.clone(), - file_name_generator.clone(), ) .await; data_files_vec.push(data_file.clone()); @@ -564,3 +501,406 @@ async fn test_partition_spec_id_in_manifest() { assert_eq!(manifest_file.partition_spec_id, partition_spec_id); } } + +#[tokio::test] +async fn test_rewrite_files_to_branch() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("branch_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Prepare test data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let data_file_writer_builder = create_data_file_writer_builder(&table); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let original_files = data_file_writer.close().await.unwrap(); + + // First append to main branch + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(original_files.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify main branch has the data + let main_snapshot = table.metadata().current_snapshot().unwrap(); + assert_eq!(main_snapshot.parent_snapshot_id(), None); + + let branch_name = "test_branch"; + + // Prepare rewrite files + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let rewrite_files = data_file_writer.close().await.unwrap(); + + // Rewrite files to the new branch + let tx = Transaction::new(&table); + let rewrite_action = tx + .rewrite_files() + .set_target_branch(branch_name.to_string()) + .add_data_files(rewrite_files.clone()) + .delete_files(original_files.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branch snapshot + let branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); + assert_eq!( + branch_snapshot.parent_snapshot_id(), + None, + "Branch snapshot should not have a parent snapshot" + ); + + // Verify main branch remains unchanged + let current_main_snapshot = table.metadata().snapshot_for_ref("main").unwrap(); + assert_eq!( + current_main_snapshot.snapshot_id(), + main_snapshot.snapshot_id() + ); + + // Verify data in branch + let branch_batch_stream = table + .scan() + .snapshot_id( + table + .metadata() + .snapshot_for_ref(branch_name) + .unwrap() + .snapshot_id(), + ) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let branch_batches: Vec<_> = branch_batch_stream.try_collect().await.unwrap(); + assert_eq!(branch_batches.len(), 1); + assert_eq!(branch_batches[0], batch); + + // Verify data in main branch remains unchanged + let main_batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let main_batches: Vec<_> = main_batch_stream.try_collect().await.unwrap(); + assert_eq!(main_batches.len(), 1); + assert_eq!(main_batches[0], batch); +} + +#[tokio::test] +async fn test_branch_snapshot_isolation() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("branch_snapshot_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create initial snapshot in main branch + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let data_file_writer_builder = create_data_file_writer_builder(&table); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_files = data_file_writer.close().await.unwrap(); + + // Commit to main branch + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(data_files.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Create branch1 from main + let branch1 = "branch1"; + let tx = Transaction::new(&table); + let rewrite_action = tx + .rewrite_files() + .set_target_branch(branch1.to_string()) + .add_data_files(data_files.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branch1 snapshot + let branch1_snapshot = table.metadata().snapshot_for_ref(branch1).unwrap(); + assert_eq!( + branch1_snapshot.parent_snapshot_id(), + None, + "Branch1 should have main snapshot as parent" + ); + + // Create branch2 from branch1 + let tx = Transaction::new(&table); + let rewrite_action = tx + .rewrite_files() + .set_target_branch(branch1.to_string()) + .add_data_files(data_files.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branch1 snapshot + let branch1_snapshot_2 = table.metadata().snapshot_for_ref(branch1).unwrap(); + assert_eq!( + branch1_snapshot_2.parent_snapshot_id(), + Some(branch1_snapshot.snapshot_id()), + " Should have branch1 snapshot as parent" + ); +} + +#[tokio::test] +async fn test_rewrite_files_with_sequence_number_from_branch() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("branch_seq_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Prepare test data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let data_file_writer_builder = create_data_file_writer_builder(&table); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let original_files = data_file_writer.close().await.unwrap(); + + // First append to main branch + let tx = Transaction::new(&table); + let append_action = tx.fast_append().add_data_files(original_files.clone()); + let tx = append_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + let branch_name = "seq_branch"; + let tx = Transaction::new(&table); + let sequence_from_branch = table + .metadata() + .snapshot_for_ref("main") + .unwrap() + .sequence_number(); + + let rewrite_action = tx + .rewrite_files() + .set_target_branch(branch_name.to_string()) + .set_new_data_file_sequence_number(sequence_from_branch) + .add_data_files(original_files.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // check manifest entry for sequence number + let branch_snapshot = table.metadata().snapshot_for_ref(branch_name).unwrap(); + let manifest_list = branch_snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + + assert_eq!(manifest_list.entries().len(), 1); + let manifest_file = &manifest_list.entries()[0]; + let manifest = manifest_file.load_manifest(table.file_io()).await.unwrap(); + assert_eq!(manifest.entries().len(), 1); + let entry = &manifest.entries()[0]; + assert_eq!(entry.sequence_number(), Some(1)); +} + +#[tokio::test] +async fn test_multiple_branches_isolation() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalogBuilder::default() + .load("rest", fixture.catalog_config.clone()) + .await + .unwrap(); + let ns = random_ns().await; + let schema = test_schema(); + + let table_creation = TableCreation::builder() + .name("multi_branch_test".to_string()) + .schema(schema.clone()) + .build(); + + let table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Prepare test data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let data_file_writer_builder = create_data_file_writer_builder(&table); + let mut data_file_writer = data_file_writer_builder.clone().build(None).await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1.clone()) as ArrayRef, + Arc::new(col2.clone()) as ArrayRef, + Arc::new(col3.clone()) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let original_files = data_file_writer.close().await.unwrap(); + + // Create branch1 with original files + let branch1 = "branch1"; + let tx = Transaction::new(&table); + let rewrite_action = tx + .rewrite_files() + .set_target_branch(branch1.to_string()) + .add_data_files(original_files.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Create branch2 with modified data + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); + let modified_col1 = StringArray::from(vec![Some("modified"), Some("data"), None, Some("here")]); + let modified_batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(modified_col1) as ArrayRef, + Arc::new(col2.clone()) as ArrayRef, + Arc::new(col3.clone()) as ArrayRef, + ]) + .unwrap(); + data_file_writer + .write(modified_batch.clone()) + .await + .unwrap(); + let modified_files = data_file_writer.close().await.unwrap(); + + let branch2 = "branch2"; + let tx = Transaction::new(&table); + let rewrite_action = tx + .rewrite_files() + .set_target_branch(branch2.to_string()) + .add_data_files(modified_files.clone()); + let tx = rewrite_action.apply(tx).unwrap(); + let table = tx.commit(&rest_catalog).await.unwrap(); + + // Verify branches are isolated + let branch1_data: Vec<_> = table + .scan() + .snapshot_id( + table + .metadata() + .snapshot_for_ref(branch1) + .unwrap() + .snapshot_id(), + ) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap() + .try_collect() + .await + .unwrap(); + assert_eq!(branch1_data[0], batch); + + let branch2_data: Vec<_> = table + .scan() + .snapshot_id( + table + .metadata() + .snapshot_for_ref(branch2) + .unwrap() + .snapshot_id(), + ) + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap() + .try_collect() + .await + .unwrap(); + assert_eq!(branch2_data[0], modified_batch); +} From 80d553d5d5ec65ba7314e3512ea201860e519188 Mon Sep 17 00:00:00 2001 From: Li0k Date: Tue, 14 Oct 2025 17:47:03 +0800 Subject: [PATCH 20/33] feat: manifest filter manager (#72) * feat: manifest filter feat: manifest filter * feat: add ut * feat: add snapshot-id * feat: integrate filter manager and snapshot producer * feat: remove drop partition * feat: intergrate snapshot producer and filter manager * address comments * fix: address comments and UT * typo * update UT * feat: add configuration to enable filter manager * typo fix: fix drop dangling delete files (#88) --- Cargo.lock | 1 + crates/examples/Cargo.toml | 1 + .../src/transaction/manifest_filter.rs | 1059 +++++++++++++++++ crates/iceberg/src/transaction/mod.rs | 5 +- .../src/transaction/overwrite_files.rs | 15 +- .../iceberg/src/transaction/rewrite_files.rs | 25 +- crates/iceberg/src/transaction/snapshot.rs | 162 ++- 7 files changed, 1241 insertions(+), 27 deletions(-) create mode 100644 crates/iceberg/src/transaction/manifest_filter.rs diff --git a/Cargo.lock b/Cargo.lock index 911ddde8b8..d195a2abda 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3695,6 +3695,7 @@ dependencies = [ "iceberg", "iceberg-catalog-rest", "tokio", + "uuid", ] [[package]] diff --git a/crates/examples/Cargo.toml b/crates/examples/Cargo.toml index c7874d9a17..cb15de80d8 100644 --- a/crates/examples/Cargo.toml +++ b/crates/examples/Cargo.toml @@ -29,6 +29,7 @@ futures = { workspace = true } iceberg = { workspace = true } iceberg-catalog-rest = { workspace = true } tokio = { workspace = true, features = ["full"] } +uuid = { workspace = true } [[example]] name = "rest-catalog-namespace" diff --git a/crates/iceberg/src/transaction/manifest_filter.rs b/crates/iceberg/src/transaction/manifest_filter.rs new file mode 100644 index 0000000000..ecd737c6e8 --- /dev/null +++ b/crates/iceberg/src/transaction/manifest_filter.rs @@ -0,0 +1,1059 @@ +// 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, HashSet}; +use std::sync::{Arc, atomic::{AtomicU64, Ordering}}; + +use uuid::Uuid; + +use crate::error::Result; +use crate::io::FileIO; +use crate::spec::{ + DataFile, FormatVersion, ManifestContentType, ManifestFile, ManifestStatus, ManifestWriter, + ManifestWriterBuilder, PartitionSpec, Schema +}; +use crate::transaction::snapshot::new_manifest_path; +use crate::{Error, ErrorKind}; + +/// Context for creating manifest writers during filtering operations +pub struct ManifestWriterContext { + metadata_location: String, + meta_root_path: String, + commit_uuid: Uuid, + manifest_counter: Arc, + format_version: FormatVersion, + snapshot_id: i64, + file_io: FileIO, + key_metadata: Option>, +} + +impl ManifestWriterContext { + /// Create a new ManifestWriterContext + #[allow(clippy::too_many_arguments)] + pub fn new( + metadata_location: String, + meta_root_path: String, + commit_uuid: Uuid, + manifest_counter: Arc, + format_version: FormatVersion, + snapshot_id: i64, + file_io: FileIO, + key_metadata: Option>, + ) -> Self { + Self { + metadata_location, + meta_root_path, + commit_uuid, + manifest_counter, + format_version, + snapshot_id, + file_io, + key_metadata, + } + } + + /// Create a manifest writer for the specified content type + pub fn new_manifest_writer( + &self, + content_type: ManifestContentType, + table_schema: &Schema, + partition_spec: &PartitionSpec, + ) -> Result { + let new_manifest_path = new_manifest_path( + &self.metadata_location, + &self.meta_root_path, + self.commit_uuid, + self.manifest_counter.fetch_add(1, Ordering::SeqCst), + crate::spec::DataFileFormat::Avro, + ); + + let output = self.file_io.new_output(&new_manifest_path)?; + let builder = ManifestWriterBuilder::new( + output, + Some(self.snapshot_id), + self.key_metadata.clone(), + table_schema.clone().into(), + partition_spec.clone(), + ); + + match self.format_version { + FormatVersion::V1 => Ok(builder.build_v1()), + FormatVersion::V2 => match content_type { + ManifestContentType::Data => Ok(builder.build_v2_data()), + ManifestContentType::Deletes => Ok(builder.build_v2_deletes()), + }, + FormatVersion::V3 => match content_type { + ManifestContentType::Data => Ok(builder.build_v3_data()), + ManifestContentType::Deletes => Ok(builder.build_v3_deletes()), + }, + } + } +} + +/// Manager for filtering manifest files and handling delete operations +pub struct ManifestFilterManager { + /// Files to be deleted by path + files_to_delete: HashMap, + + /// Minimum sequence number for removing old delete files + min_sequence_number: i64, + /// Whether to fail if any delete operation is attempted + fail_any_delete: bool, + /// Whether to fail if required delete paths are missing + fail_missing_delete_paths: bool, + /// Cache of filtered manifests to avoid reprocessing + filtered_manifests: HashMap, // manifest_path -> filtered_manifest + /// Tracking where files were deleted to validate retries quickly + filtered_manifest_to_deleted_files: HashMap>, // manifest_path -> deleted_files + /// this is only being used for the DeleteManifestFilterManager to detect orphaned deletes for removed data file paths + removed_data_file_path: HashSet, + + file_io: FileIO, + writer_context: ManifestWriterContext, +} + +impl ManifestFilterManager { + /// Create a new ManifestFilterManager + pub fn new(file_io: FileIO, writer_context: ManifestWriterContext) -> Self { + Self { + files_to_delete: HashMap::new(), + min_sequence_number: 0, + fail_any_delete: false, + fail_missing_delete_paths: false, + filtered_manifests: HashMap::new(), + filtered_manifest_to_deleted_files: HashMap::new(), + removed_data_file_path: HashSet::new(), + file_io, + writer_context, + } + } + + /// Set whether to fail if any delete operation is attempted + pub fn fail_any_delete(mut self) -> Self { + self.fail_any_delete = true; + self + } + + /// Get files marked for deletion + pub fn files_to_be_deleted(&self) -> Vec<&DataFile> { + self.files_to_delete.values().collect() + } + + /// Set minimum sequence number for removing old delete files + pub(crate) fn drop_delete_files_older_than(&mut self, sequence_number: i64) { + assert!( + sequence_number >= 0, + "Invalid minimum data sequence number: {}", + sequence_number + ); + self.min_sequence_number = sequence_number; + } + + /// Set whether to fail if required delete paths are missing + pub fn fail_missing_delete_paths(mut self) -> Self { + self.fail_missing_delete_paths = true; + self + } + + /// Mark a data file for deletion + pub fn delete_file(&mut self, file: DataFile) -> Result<()> { + // Todo: check all deletes references in manifests? + let file_path = file.file_path.clone(); + + self.files_to_delete.insert(file_path, file); + + Ok(()) + } + + /// Check if this manager contains any delete operations + pub fn contains_deletes(&self) -> bool { + !self.files_to_delete.is_empty() + } + + /// Filter manifest files, removing entries marked for deletion + pub async fn filter_manifests( + &mut self, + table_schema: &Schema, + manifests: Vec, + ) -> Result> { + if manifests.is_empty() { + self.validate_required_deletes(&[])?; + return Ok(vec![]); + } + + let mut filtered = Vec::with_capacity(manifests.len()); + + for manifest in manifests { + let filtered_manifest = self + .filter_manifest(table_schema, manifest) + .await?; + filtered.push(filtered_manifest); + } + + self.validate_required_deletes(&filtered)?; + Ok(filtered) + } + + /// Filter a single manifest file + async fn filter_manifest( + &mut self, + table_schema: &Schema, + manifest: ManifestFile, + ) -> Result { + // Check cache first + if let Some(cached) = self.filtered_manifests.get(&manifest.manifest_path) { + return Ok(cached.clone()); + } + + // Check if this manifest can contain files to delete + if !self.can_contain_deleted_files(&manifest) { + self.filtered_manifests + .insert(manifest.manifest_path.clone(), manifest.clone()); + return Ok(manifest); + } + + if self.manifest_has_deleted_files(&manifest).await? { + // Load and filter the manifest + self.filter_manifest_with_deleted_files(table_schema, manifest).await + } else { + // If no deleted files are found, just return the original manifest + self.filtered_manifests + .insert(manifest.manifest_path.clone(), manifest.clone()); + Ok(manifest) + } + } + + /// Check if a manifest can potentially contain files that need to be deleted + fn can_contain_deleted_files(&self, manifest: &ManifestFile) -> bool { + // If manifest has no live files, it can't contain files to delete + if Self::manifest_has_no_live_files(manifest) { + return false; + } + + // If we have file-based deletes, any manifest with live files might contain them + !self.files_to_delete.is_empty() || !self.removed_data_file_path.is_empty() + } + + /// Filter a manifest that is known to contain files to delete + async fn filter_manifest_with_deleted_files( + &mut self, + table_schema: &Schema, + manifest: ManifestFile, + ) -> Result { + // Load the original manifest + let original_manifest = manifest.load_manifest(&self.file_io).await?; + + let ( + entries, + manifest_meta_data, + ) = original_manifest.into_parts(); + + // Check if this is a delete manifest + let is_delete = manifest.content == ManifestContentType::Deletes; + + // Track deleted files for duplicate detection + let mut deleted_files = HashMap::new(); + + // Create an output path for the filtered manifest using writer context + let partition_spec = manifest_meta_data.partition_spec.clone(); + + // Create the manifest writer using the writer context + let mut writer = self.writer_context.new_manifest_writer( + manifest.content, + table_schema, + &partition_spec, + )?; + + // Process each live entry in the manifest + for entry in &entries{ + if !entry.is_alive() { + continue; + } + + let entry = entry.as_ref(); + let file = entry.data_file(); + + // Check if file is marked for deletion based on various criteria + let marked_for_delete = + // Check if file is in delete files collection + self.files_to_delete.contains_key(file.file_path()) || + // For delete manifests, check sequence number for old delete files + (is_delete && matches!(entry.sequence_number(), Some(seq_num) if seq_num != crate::spec::UNASSIGNED_SEQUENCE_NUMBER + && seq_num > 0 + && seq_num < self.min_sequence_number)); + + // TODO: Add expression evaluation logic + if marked_for_delete { + // Check if all rows match + let all_rows_match = marked_for_delete; + + // Validation check: cannot delete file where some, but not all, rows match filter + // unless it's a delete file (ignore delete files where some records may not match) + if !all_rows_match && !is_delete { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot delete file where some, but not all, rows match filter: {}", + file.file_path() + ), + )); + } + + if all_rows_match { + // Mark this entry as deleted + writer.add_delete_entry(entry.clone())?; + + // Create a copy of the file without stats + let file_copy = file.clone(); + + // For file that it was deleted using an expression + self.files_to_delete.insert(file.file_path().to_string(), file_copy.clone()); + + // TODO: add file to removed_data_file_path once we implement drop_partition + + // Track deleted files for duplicate detection and validation + if deleted_files.contains_key(file_copy.file_path()) { + // TODO: Log warning about duplicate + } else { + // Only add the file to deletes if it is a new delete + // This keeps the snapshot summary accurate for non-duplicate data + deleted_files.insert(file_copy.file_path.to_owned(), file_copy.clone()); + } + } else { + // Keep the entry as existing + writer.add_existing_entry(entry.clone())?; + } + } else { + // Keep the entry as existing + writer.add_existing_entry(entry.clone())?; + } + } + + // Write the filtered manifest + let filtered_manifest = writer.write_manifest_file().await?; + + // Update caches + self.filtered_manifests + .insert(manifest.manifest_path.clone(), filtered_manifest.clone()); + + // Track deleted files for validation - convert HashSet to Vec of file paths + let deleted_file_paths: Vec = deleted_files + .keys().cloned().collect(); + + self.filtered_manifest_to_deleted_files + .insert(filtered_manifest.manifest_path.clone(), deleted_file_paths); + + Ok(filtered_manifest) + } + + /// Validate that all required delete operations were found + fn validate_required_deletes(&self, manifests: &[ManifestFile]) -> Result<()> { + if self.fail_missing_delete_paths { + let deleted_files = self.deleted_files(manifests); + // check deleted_files contains all files in self.delete_files + + for file_path in self.files_to_delete.keys() { + if !deleted_files.contains(file_path) { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Required delete path missing: {}", file_path), + )); + } + } + } + Ok(()) + } + + fn deleted_files(&self, manifests: &[ManifestFile]) -> HashSet { + let mut deleted_files = HashSet::new(); + for manifest in manifests { + if let Some(deleted) = self + .filtered_manifest_to_deleted_files + .get(manifest.manifest_path.as_str()) + { + deleted_files.extend(deleted.clone()); + } + } + deleted_files + } + + fn manifest_has_no_live_files(manifest: &ManifestFile) -> bool { + !manifest.has_added_files() && !manifest.has_existing_files() + } + + async fn manifest_has_deleted_files(&self, manifest_file: &ManifestFile) -> Result { + let manifest = manifest_file.load_manifest(&self.file_io).await?; + + let is_delete = manifest_file.content == ManifestContentType::Deletes; + + for entry in manifest.entries() { + let entry = entry.as_ref(); + + // Skip entries that are already deleted + if entry.status() == ManifestStatus::Deleted { + continue; + } + + let file = entry.data_file(); + + // Check if file is marked for deletion based on various criteria + let marked_for_delete = + // Check if file path is in files to delete + self.files_to_delete.contains_key(file.file_path()) || + // For delete manifests, check sequence number for old delete files + (is_delete && + entry.status() != ManifestStatus::Deleted && + matches!(entry.sequence_number(), Some(seq_num) if seq_num != crate::spec::UNASSIGNED_SEQUENCE_NUMBER + && seq_num > 0 + && seq_num < self.min_sequence_number)); + + // TODO: Add dangling delete vector check: (is_delete && self.is_dangling_dv(file)) + + // TODO: Add expression evaluation logic + if marked_for_delete { + // Check if all rows match + let all_rows_match = marked_for_delete; // || evaluator.rowsMustMatch(file) equivalent + + // Validation check: cannot delete file where some, but not all, rows match filter + // unless it's a delete file + if !all_rows_match && !is_delete { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot delete file where some, but not all, rows match filter: {}", + file.file_path() + ), + )); + } + + if all_rows_match { + // Check fail_any_delete flag + if self.fail_any_delete { + return Err(Error::new( + ErrorKind::DataInvalid, + "Operation would delete existing data".to_string(), + )); + } + + // As soon as a deleted file is detected, stop scanning and return true + return Ok(true); + } + } + } + + Ok(false) + } + + pub(crate) fn remove_dangling_deletes_for(&mut self, file_paths: &HashSet) { + self.removed_data_file_path.extend(file_paths.iter().cloned()); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::io::FileIOBuilder; + use crate::spec::{ + DataContentType, DataFileFormat, NestedField, PrimitiveType, Type, + ManifestEntry, ManifestStatus, ManifestFile, ManifestContentType, Struct, Schema, + PartitionSpec, FormatVersion, ManifestWriterBuilder + }; + use std::collections::HashMap; + use std::sync::Arc; + use tempfile::TempDir; + use uuid::Uuid; + + // Helper function to create a test schema + fn create_test_schema() -> Schema { + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "name", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap() + } + + // Helper function to create a test DataFile + fn create_test_data_file(file_path: &str, partition_spec_id: i32) -> DataFile { + DataFile { + content: DataContentType::Data, + file_path: file_path.to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::empty(), + partition_spec_id, + record_count: 100, + file_size_in_bytes: 1024, + column_sizes: HashMap::new(), + value_counts: HashMap::new(), + null_value_counts: HashMap::new(), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: None, + split_offsets: vec![], + equality_ids: None, + sort_order_id: None, + first_row_id: None, + referenced_data_file: None, + content_offset: None, + content_size_in_bytes: None, + } + } + + // Helper function to create a test ManifestFile with default values + fn create_test_manifest_file(manifest_path: &str, content: ManifestContentType) -> ManifestFile { + create_manifest_with_counts(manifest_path, content, 10, 5, 0) + } + + // Helper function to create a ManifestFile with custom counts + fn create_manifest_with_counts( + manifest_path: &str, + content: ManifestContentType, + added_files: u32, + existing_files: u32, + deleted_files: u32, + ) -> ManifestFile { + ManifestFile { + manifest_path: manifest_path.to_string(), + manifest_length: 5000, + partition_spec_id: 0, + content, + sequence_number: 1, + min_sequence_number: 1, + added_snapshot_id: 12345, + added_files_count: Some(added_files), + existing_files_count: Some(existing_files), + deleted_files_count: Some(deleted_files), + added_rows_count: Some(added_files as u64 * 100), + existing_rows_count: Some(existing_files as u64 * 100), + deleted_rows_count: Some(deleted_files as u64 * 100), + partitions: None, + key_metadata: None, + first_row_id: None, + } + } + + // Helper function to create a ManifestFile with specific sequence numbers + fn create_manifest_with_sequence( + manifest_path: &str, + content: ManifestContentType, + sequence_number: i64, + min_sequence_number: i64, + ) -> ManifestFile { + let mut manifest = create_test_manifest_file(manifest_path, content); + manifest.sequence_number = sequence_number; + manifest.min_sequence_number = min_sequence_number; + manifest + } + + // Helper function to setup test environment + fn setup_test_manager() -> (ManifestFilterManager, TempDir) { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + let metadata_location = temp_dir.path().join("metadata.json").to_string_lossy().to_string(); + let meta_root_path = temp_dir.path().to_string_lossy().to_string(); + + let writer_context = ManifestWriterContext::new( + metadata_location, + meta_root_path, + Uuid::new_v4(), + Arc::new(AtomicU64::new(0)), + FormatVersion::V2, + 1, + file_io.clone(), + None, + ); + + let manager = ManifestFilterManager::new(file_io, writer_context); + + (manager, temp_dir) + } + + // Helper function to write manifest entries to file + async fn write_manifest_with_entries( + manager: &ManifestFilterManager, + manifest_path: &str, + schema: &Schema, + entries: Vec, + snapshot_id: i64, + ) -> Result<()> { + let partition_spec = PartitionSpec::unpartition_spec(); + let output_file = manager.file_io.new_output(manifest_path)?; + let mut writer = ManifestWriterBuilder::new( + output_file, + Some(snapshot_id), + None, + schema.clone().into(), + partition_spec, + ).build_v2_data(); + + for entry in entries { + writer.add_entry(entry)?; + } + writer.write_manifest_file().await?; + Ok(()) + } + + // Helper function to create manifest entries from data files + fn create_entries_from_files(files: Vec) -> Vec { + files.into_iter() + .map(|file| ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file(file) + .build()) + .collect() + } + + // Helper function to create a full manifest file metadata + fn create_manifest_metadata( + manifest_path: &str, + content: ManifestContentType, + sequence_number: i64, + snapshot_id: i64, + file_counts: (u32, u32, u32), // (added, existing, deleted) + ) -> ManifestFile { + let (added, existing, deleted) = file_counts; + ManifestFile { + manifest_path: manifest_path.to_string(), + manifest_length: 1000, + partition_spec_id: 0, + content, + sequence_number, + min_sequence_number: sequence_number, + added_snapshot_id: snapshot_id, + added_files_count: Some(added), + existing_files_count: Some(existing), + deleted_files_count: Some(deleted), + added_rows_count: Some(added as u64 * 100), + existing_rows_count: Some(existing as u64 * 100), + deleted_rows_count: Some(deleted as u64 * 100), + partitions: None, + key_metadata: None, + first_row_id: None, + } + } + + #[test] + fn test_new_manifest_filter_manager() { + let (manager, _temp_dir) = setup_test_manager(); + + // Test initial state + assert!(!manager.contains_deletes()); + assert_eq!(manager.min_sequence_number, 0); + assert!(!manager.fail_any_delete); + assert!(!manager.fail_missing_delete_paths); + assert!(manager.files_to_delete.is_empty()); + } + + #[test] + fn test_configuration_flags() { + let (manager, _temp_dir) = setup_test_manager(); + + let mut configured_manager = manager + .fail_any_delete() + .fail_missing_delete_paths(); + configured_manager.drop_delete_files_older_than(100); + + assert!(configured_manager.fail_any_delete); + assert!(configured_manager.fail_missing_delete_paths); + assert_eq!(configured_manager.min_sequence_number, 100); + } + + #[test] + fn test_delete_file() { + let (mut manager, _temp_dir) = setup_test_manager(); + + // Test 1: Delete file by path + let file_path = "/test/path/file.parquet"; + let test_file1 = create_test_data_file(file_path, 0); + + // Initially no deletes + assert!(!manager.contains_deletes()); + + // Add file to delete + manager.delete_file(test_file1).unwrap(); + + // Should now contain deletes + assert!(manager.contains_deletes()); + assert!(manager.files_to_delete.contains_key(file_path)); + + // Test 2: Delete another file and verify tracking + let test_file2 = create_test_data_file("/test/data/file2.parquet", 0); + let file_path2 = test_file2.file_path.clone(); + manager.delete_file(test_file2).unwrap(); + + // Should track both files for deletion + let deleted_files = manager.files_to_be_deleted(); + assert_eq!(deleted_files.len(), 2); + assert!(manager.files_to_delete.contains_key(&file_path2)); + } + + #[test] + fn test_manifest_has_no_live_files() { + // Test manifest with no live files + let manifest_no_live = create_manifest_with_counts( + "/test/manifest1.avro", + ManifestContentType::Data, + 0, 0, 5 + ); + assert!(ManifestFilterManager::manifest_has_no_live_files(&manifest_no_live)); + + // Test manifest with live files + let manifest_with_live = create_test_manifest_file("/test/manifest2.avro", ManifestContentType::Data); + assert!(!ManifestFilterManager::manifest_has_no_live_files(&manifest_with_live)); + } + + #[test] + fn test_can_contain_deleted_files() { + let (mut manager, _temp_dir) = setup_test_manager(); + + // Manifest with no live files should not contain deleted files + let manifest_no_live = create_manifest_with_counts( + "/test/manifest1.avro", + ManifestContentType::Data, + 0, 0, 5 + ); + assert!(!manager.can_contain_deleted_files(&manifest_no_live)); + + // Manifest with live files but no deletes + let manifest_with_live = create_test_manifest_file("/test/manifest2.avro", ManifestContentType::Data); + assert!(!manager.can_contain_deleted_files(&manifest_with_live)); + + // Add deletes and test again + let test_file = create_test_data_file("/test/file.parquet", 0); + manager.delete_file(test_file).unwrap(); + assert!(manager.can_contain_deleted_files(&manifest_with_live)); + } + + #[tokio::test] + async fn test_filter_manifests_empty_input() { + let (mut manager, _temp_dir) = setup_test_manager(); + let schema = create_test_schema(); + + let result = manager.filter_manifests(&schema, vec![]).await.unwrap(); + assert!(result.is_empty()); + } + + #[test] + fn test_validate_required_deletes_success() { + let (manager, _temp_dir) = setup_test_manager(); + + // Test validation with no required deletes + let manifests = vec![create_test_manifest_file("/test/manifest.avro", ManifestContentType::Data)]; + let result = manager.validate_required_deletes(&manifests); + assert!(result.is_ok()); + } + + #[test] + fn test_validate_required_deletes_failure() { + let (mut manager, _temp_dir) = setup_test_manager(); + + // Enable fail_missing_delete_paths + manager.fail_missing_delete_paths = true; + + // Add a required delete file that won't be found + let missing_file = create_test_data_file("/missing/file.parquet", 0); + manager.delete_file(missing_file).unwrap(); + + let manifests = vec![create_test_manifest_file("/test/manifest.avro", ManifestContentType::Data)]; + let result = manager.validate_required_deletes(&manifests); + + assert!(result.is_err()); + assert!(result.unwrap_err().to_string().contains("Required delete path missing")); + } + + #[tokio::test] + async fn test_comprehensive_deletion_logic() { + let (mut manager, temp_dir) = setup_test_manager(); + let schema = create_test_schema(); + + // Create test data files + let keep_file = create_test_data_file("/test/keep_me.parquet", 0); + let delete_file = create_test_data_file("/test/delete_me.parquet", 0); + manager.delete_file(delete_file.clone()).unwrap(); + + // Write manifest with both files + let manifest_path = temp_dir.path().join("test_manifest.avro"); + let manifest_path_str = manifest_path.to_str().unwrap(); + let entries = create_entries_from_files(vec![keep_file.clone(), delete_file.clone()]); + write_manifest_with_entries(&manager, manifest_path_str, &schema, entries, 12345).await.unwrap(); + + // Create ManifestFile metadata + let manifest = create_manifest_metadata( + manifest_path_str, + ManifestContentType::Data, + 10, + 12345, + (2, 0, 0) + ); + + // Verify manifest filtering capabilities + assert!(manager.can_contain_deleted_files(&manifest)); + assert!(manager.manifest_has_deleted_files(&manifest).await.unwrap()); + assert!(manager.files_to_delete.contains_key(&delete_file.file_path)); + assert!(!manager.files_to_delete.contains_key(&keep_file.file_path)); + + // Verify manager state + assert!(manager.contains_deletes()); + let files_to_delete = manager.files_to_be_deleted(); + assert_eq!(files_to_delete.len(), 1); + assert_eq!(files_to_delete[0].file_path, delete_file.file_path); + } + + #[test] + fn test_min_sequence_number_logic() { + let (mut manager, _temp_dir) = setup_test_manager(); + manager.min_sequence_number = 5; + + // Test basic sequence number comparison logic + assert_eq!(manager.min_sequence_number, 5); + let (old_sequence, new_sequence) = (3, 10); + assert!(old_sequence < manager.min_sequence_number); + assert!(new_sequence >= manager.min_sequence_number); + + // Create manifests with different sequence numbers + let old_manifest = create_manifest_with_sequence( + "/test/old.avro", ManifestContentType::Data, old_sequence, old_sequence + ); + let new_manifest = create_manifest_with_sequence( + "/test/new.avro", ManifestContentType::Data, new_sequence, new_sequence + ); + + // Add files to delete for testing + manager.delete_file(create_test_data_file("/test/file.parquet", 0)).unwrap(); + + // Both manifests should be able to contain deleted files (key filtering behavior) + assert!(manager.can_contain_deleted_files(&old_manifest)); + assert!(manager.can_contain_deleted_files(&new_manifest)); + + // Verify sequence number properties + assert!(old_manifest.min_sequence_number < manager.min_sequence_number); + assert!(new_manifest.min_sequence_number >= manager.min_sequence_number); + } + + #[test] + fn test_deletion_tracking_and_validation() { + let (mut manager, _temp_dir) = setup_test_manager(); + + let delete_file = create_test_data_file("/test/delete_me.parquet", 0); + + // Initially no deletes + assert!(!manager.contains_deletes()); + assert_eq!(manager.files_to_be_deleted().len(), 0); + + // Add the file to be deleted + manager.delete_file(delete_file.clone()).unwrap(); + + // Verify deletion tracking + assert!(manager.contains_deletes()); + assert_eq!(manager.files_to_be_deleted().len(), 1); + assert_eq!(manager.files_to_be_deleted()[0].file_path, delete_file.file_path); + + // Create a manifest that could contain deleted files + let manifest = create_manifest_with_sequence( + "/test/test_manifest.avro", + ManifestContentType::Data, + 10, 1 + ); + + // Verify manifest can contain deleted files + assert!(manager.can_contain_deleted_files(&manifest)); + assert!(manager.files_to_delete.contains_key(&delete_file.file_path)); + + // Validation should pass when no required deletes are set + assert!(manager.validate_required_deletes(&[manifest]).is_ok()); + } + + #[tokio::test] + async fn test_filter_manifests_with_entries_and_rewrite() { + let (mut manager, temp_dir) = setup_test_manager(); + let schema = create_test_schema(); + + // Create test files + let files_to_keep = vec![ + create_test_data_file("/test/keep1.parquet", 0), + create_test_data_file("/test/keep2.parquet", 0), + ]; + let files_to_delete = vec![ + create_test_data_file("/test/delete1.parquet", 0), + create_test_data_file("/test/delete2.parquet", 0), + ]; + + // Mark files for deletion + for file in &files_to_delete { + manager.delete_file(file.clone()).unwrap(); + } + + // Create two manifests with mixed files + let manifest_paths: Vec<_> = (1..=2).map(|i| { + temp_dir.path().join(format!("manifest{}.avro", i)).to_string_lossy().to_string() + }).collect(); + + for (idx, path) in manifest_paths.iter().enumerate() { + let entries = create_entries_from_files(vec![ + files_to_keep[idx].clone(), + files_to_delete[idx].clone(), + ]); + write_manifest_with_entries(&manager, path, &schema, entries, 12345 + idx as i64).await.unwrap(); + } + + // Create ManifestFile metadata objects + let input_manifests: Vec<_> = manifest_paths.iter().enumerate().map(|(idx, path)| { + create_manifest_metadata(path, ManifestContentType::Data, 10, 12345 + idx as i64, (2, 0, 0)) + }).collect(); + + // Filter manifests + let filtered_manifests = manager.filter_manifests(&schema, input_manifests.clone()).await.unwrap(); + + // Verify results + assert_eq!(filtered_manifests.len(), 2); + assert_ne!(filtered_manifests[0].manifest_path, input_manifests[0].manifest_path); + assert_ne!(filtered_manifests[1].manifest_path, input_manifests[1].manifest_path); + + // Verify deletion tracking + assert_eq!(manager.files_to_be_deleted().len(), 2); + let deleted_paths: std::collections::HashSet<_> = manager.files_to_be_deleted() + .into_iter().map(|f| f.file_path.clone()).collect(); + for file in &files_to_delete { + assert!(deleted_paths.contains(&file.file_path)); + } + + // Verify filtered manifest entries + let filtered_manifest1 = filtered_manifests[0].load_manifest(&manager.file_io).await.unwrap(); + let (entries_filtered, _) = filtered_manifest1.into_parts(); + + let (live_count, deleted_count) = entries_filtered.iter().fold((0, 0), |(live, deleted), entry| { + match entry.status() { + ManifestStatus::Added | ManifestStatus::Existing => (live + 1, deleted), + ManifestStatus::Deleted => (live, deleted + 1), + } + }); + + assert_eq!(live_count, 1); + assert_eq!(deleted_count, 1); + + // Verify cache and tracking + assert!(manager.filtered_manifests.contains_key(&input_manifests[0].manifest_path)); + assert!(manager.filtered_manifest_to_deleted_files.contains_key(&filtered_manifests[0].manifest_path)); + } + + #[test] + fn test_unassigned_sequence_number_handling() { + let (mut manager, _temp_dir) = setup_test_manager(); + manager.drop_delete_files_older_than(100); + + // Create manifest with UNASSIGNED_SEQUENCE_NUMBER and no live files + let manifest_unassigned = create_manifest_with_sequence( + "/test/unassigned.avro", + ManifestContentType::Deletes, + crate::spec::UNASSIGNED_SEQUENCE_NUMBER, + crate::spec::UNASSIGNED_SEQUENCE_NUMBER, + ); + + // Manifest with no live files should not contain deleted files + assert!(!manager.can_contain_deleted_files(&manifest_unassigned)); + } + + #[tokio::test] + async fn test_cache_behavior() { + let (mut manager, temp_dir) = setup_test_manager(); + let schema = create_test_schema(); + + // Create and write a manifest without deleted files + let manifest_path = temp_dir.path().join("cache_test.avro"); + let manifest_path_str = manifest_path.to_str().unwrap(); + + let entries = create_entries_from_files(vec![create_test_data_file("/test/keep.parquet", 0)]); + write_manifest_with_entries(&manager, manifest_path_str, &schema, entries, 12345).await.unwrap(); + + let manifest = create_manifest_metadata(manifest_path_str, ManifestContentType::Data, 10, 12345, (1, 0, 0)); + + // First and second calls should return the same cached result + let result1 = manager.filter_manifest(&schema, manifest.clone()).await.unwrap(); + let result2 = manager.filter_manifest(&schema, manifest.clone()).await.unwrap(); + + assert_eq!(result1.manifest_path, result2.manifest_path); + assert!(manager.filtered_manifests.contains_key(&manifest.manifest_path)); + } + + #[test] + fn test_batch_delete_operations() { + let (mut manager, _temp_dir) = setup_test_manager(); + + assert!(!manager.contains_deletes()); + + // Add multiple files for deletion + for i in 1..=3 { + manager.delete_file(create_test_data_file(&format!("/test/batch{}.parquet", i), 0)).unwrap(); + } + + // Verify all files are tracked + assert!(manager.contains_deletes()); + assert_eq!(manager.files_to_be_deleted().len(), 3); + + let deleted_paths: std::collections::HashSet<_> = + manager.files_to_be_deleted().iter().map(|f| f.file_path.as_str()).collect(); + assert!(deleted_paths.contains("/test/batch1.parquet")); + assert!(deleted_paths.contains("/test/batch2.parquet")); + assert!(deleted_paths.contains("/test/batch3.parquet")); + } + + #[test] + fn test_edge_case_empty_partition_specs() { + let (mut manager, _temp_dir) = setup_test_manager(); + + // Create a data file with different partition spec + let file_with_different_spec = DataFile { + content: crate::spec::DataContentType::Data, + file_path: "/test/different_spec.parquet".to_string(), + file_format: crate::spec::DataFileFormat::Parquet, + partition: crate::spec::Struct::empty(), + partition_spec_id: 999, // Different spec ID + record_count: 100, + file_size_in_bytes: 1024, + column_sizes: HashMap::new(), + value_counts: HashMap::new(), + null_value_counts: HashMap::new(), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: None, + split_offsets: vec![], + equality_ids: None, + sort_order_id: None, + first_row_id: None, + referenced_data_file: None, + content_offset: None, + content_size_in_bytes: None, + }; + + // Should be able to add file with different partition spec + manager.delete_file(file_with_different_spec).unwrap(); + assert!(manager.contains_deletes()); + assert_eq!(manager.files_to_be_deleted().len(), 1); + } +} diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 0f1b82a114..6e38ea97db 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -54,6 +54,9 @@ mod action; pub use action::*; mod append; +mod manifest_filter; + +pub use manifest_filter::*; mod overwrite_files; mod remove_snapshots; mod rewrite_files; @@ -550,7 +553,7 @@ mod test_row_lineage { use crate::transaction::tests::make_v3_minimal_table_in_catalog; use crate::transaction::{ApplyTransactionAction, Transaction}; - #[tokio::test] + #[tokio::test] async fn test_fast_append_with_row_lineage() { // Helper function to create a data file with specified number of rows fn file_with_rows(record_count: u64) -> DataFile { diff --git a/crates/iceberg/src/transaction/overwrite_files.rs b/crates/iceberg/src/transaction/overwrite_files.rs index 14ff34f354..27fdb4a408 100644 --- a/crates/iceberg/src/transaction/overwrite_files.rs +++ b/crates/iceberg/src/transaction/overwrite_files.rs @@ -52,10 +52,9 @@ pub struct OverwriteFilesAction { removed_data_files: Vec, removed_delete_files: Vec, snapshot_id: Option, - new_data_file_sequence_number: Option, - target_branch: Option, + enable_delete_filter_manager: bool, } pub struct OverwriteFilesOperation; @@ -77,6 +76,7 @@ impl OverwriteFilesAction { snapshot_id: None, new_data_file_sequence_number: None, target_branch: None, + enable_delete_filter_manager: false, } } @@ -148,6 +148,13 @@ impl OverwriteFilesAction { self } + /// Enable delete filter manager for this snapshot. + /// By default, delete filter manager is disabled. + pub fn set_enable_delete_filter_manager(mut self, enable_delete_filter_manager: bool) -> Self { + self.enable_delete_filter_manager = enable_delete_filter_manager; + self + } + pub fn set_target_branch(mut self, target_branch: String) -> Self { self.target_branch = Some(target_branch); self @@ -324,6 +331,10 @@ impl TransactionAction for OverwriteFilesAction { snapshot_producer.set_target_branch(branch.clone()); } + if self.enable_delete_filter_manager { + snapshot_producer.enable_delete_filter_manager(); + } + // Checks duplicate files if self.check_duplicate { snapshot_producer diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs index 000f3dd1d8..d18de28de0 100644 --- a/crates/iceberg/src/transaction/rewrite_files.rs +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -52,10 +52,9 @@ pub struct RewriteFilesAction { removed_data_files: Vec, removed_delete_files: Vec, snapshot_id: Option, - new_data_file_sequence_number: Option, - target_branch: Option, + enable_delete_filter_manager: bool, } pub struct RewriteFilesOperation; @@ -77,6 +76,7 @@ impl RewriteFilesAction { snapshot_id: None, new_data_file_sequence_number: None, target_branch: None, + enable_delete_filter_manager: false, } } @@ -136,6 +136,13 @@ impl RewriteFilesAction { self } + /// Enable delete filter manager for this snapshot. + /// By default, delete filter manager is disabled. + pub fn set_enable_delete_filter_manager(mut self, enable_delete_filter_manager: bool) -> Self { + self.enable_delete_filter_manager = enable_delete_filter_manager; + self + } + /// Set key metadata for manifest files. pub fn set_key_metadata(mut self, key_metadata: Vec) -> Self { self.key_metadata = Some(key_metadata); @@ -213,11 +220,11 @@ impl SnapshotProduceOperation for RewriteFilesOperation { deleted_entries.push(gen_manifest_entry(entry)); } - if entry.content_type() == DataContentType::PositionDeletes - || entry.content_type() == DataContentType::EqualityDeletes - && snapshot_produce - .removed_delete_file_paths - .contains(entry.data_file().file_path()) + if (entry.content_type() == DataContentType::PositionDeletes + || entry.content_type() == DataContentType::EqualityDeletes) + && snapshot_produce + .removed_delete_file_paths + .contains(entry.data_file().file_path()) { deleted_entries.push(gen_manifest_entry(entry)); } @@ -324,6 +331,10 @@ impl TransactionAction for RewriteFilesAction { snapshot_producer.set_target_branch(branch.clone()); } + if self.enable_delete_filter_manager { + snapshot_producer.enable_delete_filter_manager(); + } + // Checks duplicate files if self.check_duplicate { snapshot_producer diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 4ef4b9b787..fe4d0aa5d4 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -17,8 +17,9 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::future::Future; -use std::ops::RangeFrom; use std::pin::Pin; +use std::sync::Arc; +use std::sync::atomic::{AtomicU64, Ordering}; use async_trait::async_trait; use uuid::Uuid; @@ -30,10 +31,10 @@ use crate::spec::{ ManifestEntry, ManifestFile, ManifestListWriter, ManifestStatus, ManifestWriter, ManifestWriterBuilder, Operation, Snapshot, SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, StructType, Summary, TableProperties, - update_snapshot_summaries, + UNASSIGNED_SEQUENCE_NUMBER, update_snapshot_summaries, }; use crate::table::Table; -use crate::transaction::ActionCommit; +use crate::transaction::{ActionCommit, ManifestFilterManager, ManifestWriterContext}; use crate::utils::bin::ListPacker; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; @@ -86,15 +87,18 @@ pub(crate) struct SnapshotProducer<'a> { // for filtering out files that are removed by action pub removed_data_file_paths: HashSet, pub removed_delete_file_paths: HashSet, + pub removed_delete_files: Vec, // A counter used to generate unique manifest file names. // It starts from 0 and increments for each new manifest file. - // Note: This counter is limited to the range of (0..u64::MAX). - manifest_counter: RangeFrom, + // This counter is shared with ManifestWriterContext to avoid naming conflicts. + manifest_counter: Arc, new_data_file_sequence_number: Option, target_branch: String, + + delete_filter_manager: Option, } impl<'a> SnapshotProducer<'a> { @@ -107,32 +111,38 @@ impl<'a> SnapshotProducer<'a> { snapshot_properties: HashMap, added_data_files: Vec, added_delete_files: Vec, - removed_data_file_paths: Vec, - removed_delete_file_paths: Vec, + removed_data_files: Vec, + removed_delete_files: Vec, ) -> Self { - let removed_data_file_paths = removed_data_file_paths + let removed_data_file_paths = removed_data_files .into_iter() .map(|df| df.file_path) .collect(); - - let removed_delete_file_paths = removed_delete_file_paths - .into_iter() - .map(|df| df.file_path) + let removed_delete_file_paths = removed_delete_files + .iter() + .map(|df| df.file_path.clone()) .collect(); + let manifest_counter = Arc::new(AtomicU64::new(0)); + + // Default: disable delete filter manager (need to explicitly enable) + let delete_filter_manager = None; + Self { table, snapshot_id: snapshot_id.unwrap_or_else(|| Self::generate_unique_snapshot_id(table)), commit_uuid, - key_metadata, snapshot_properties, + manifest_counter, + key_metadata, added_data_files, added_delete_files, removed_data_file_paths, removed_delete_file_paths, - manifest_counter: (0..), + removed_delete_files, new_data_file_sequence_number: None, target_branch: MAIN_BRANCH.to_string(), + delete_filter_manager, } } @@ -227,7 +237,7 @@ impl<'a> SnapshotProducer<'a> { self.table.metadata().location(), META_ROOT_PATH, self.commit_uuid, - self.manifest_counter.next().unwrap(), + self.manifest_counter.fetch_add(1, Ordering::SeqCst), DataFileFormat::Avro ); let output_file = self.table.file_io().new_output(new_manifest_path)?; @@ -430,10 +440,78 @@ impl<'a> SnapshotProducer<'a> { )); } + // Get existing manifests and prepare them for the manifest list. + // Existing manifests must come before new manifests in the final list + // to ensure correct first_row_id assignment by ManifestListWriter. let existing_manifests = snapshot_produce_operation.existing_manifest(self).await?; - let mut manifest_files = existing_manifests; - // Process added entries. + let mut manifest_files = + if let Some(delete_filter_manager) = self.delete_filter_manager.as_mut() { + // When delete filter manager is enabled, filter existing manifests + let metadata_ref = self.table.metadata_ref(); + let branch_snapshot_ref = metadata_ref.snapshot_for_ref(&self.target_branch); + + let schema_id = if let Some(branch_snapshot_ref) = branch_snapshot_ref { + branch_snapshot_ref + .schema_id() + .unwrap_or(metadata_ref.current_schema_id()) + } else { + metadata_ref.current_schema_id() + }; + + let schema = metadata_ref + .schema_by_id(schema_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "Invalid schema id for existing manifest filtering", + ) + .with_context("schema id", schema_id.to_string()) + })? + .as_ref() + .clone(); + + let last_seq = metadata_ref.last_sequence_number(); + + // Partition manifests by type to avoid cloning + let (existing_data_manifests, existing_delete_manifests): (Vec<_>, Vec<_>) = + existing_manifests + .into_iter() + .partition(|m| matches!(m.content, ManifestContentType::Data)); + + let min_data_seq = existing_data_manifests + .iter() + .map(|m| m.min_sequence_number) + .filter(|seq| *seq != UNASSIGNED_SEQUENCE_NUMBER) + .reduce(std::cmp::min) + .map(|min_seq| std::cmp::min(min_seq, last_seq)) + .unwrap_or(last_seq); + + let mut filtered_manifests = existing_data_manifests; + + delete_filter_manager.drop_delete_files_older_than(min_data_seq); + delete_filter_manager.remove_dangling_deletes_for(&self.removed_data_file_paths); + + let filtered_delete_manifests: Vec = delete_filter_manager + .filter_manifests(&schema, existing_delete_manifests) + .await?; + filtered_manifests.extend(filtered_delete_manifests); + + filtered_manifests.retain(|m| { + m.has_added_files() + || m.has_existing_files() + || m.added_snapshot_id == self.snapshot_id + }); + + filtered_manifests + } else { + // No filtering, use existing manifests as-is + existing_manifests + }; + + // Now append new manifests created in this snapshot. + // Order matters: existing manifests first, then new manifests. + // This ensures ManifestListWriter assigns first_row_id correctly. if !self.added_data_files.is_empty() { let added_data_files = std::mem::take(&mut self.added_data_files); let added_manifest = self @@ -636,6 +714,43 @@ impl<'a> SnapshotProducer<'a> { pub fn target_branch(&self) -> &str { &self.target_branch } + + /// Enable delete filter manager for this snapshot (lazy initialization) + /// This will also populate the manager with files already marked for removal + pub fn enable_delete_filter_manager(&mut self) { + if self.delete_filter_manager.is_none() { + let metadata_ref = self.table.metadata_ref(); + let file_io = self.table.file_io(); + + let mut manager = ManifestFilterManager::new( + file_io.clone(), + ManifestWriterContext::new( + metadata_ref.location().to_string(), + META_ROOT_PATH.to_string(), + self.commit_uuid, + self.manifest_counter.clone(), + metadata_ref.format_version(), + self.snapshot_id, + file_io.clone(), + self.key_metadata.clone(), + ), + ); + + // Populate the manager with files that were already marked for deletion + // This bridges the gap between Action's delete_files() and SnapshotProducer + for delete_file in &self.removed_delete_files { + // Only add delete files (not data files) to the filter manager + if matches!( + delete_file.content_type(), + DataContentType::PositionDeletes | DataContentType::EqualityDeletes + ) { + let _ = manager.delete_file(delete_file.clone()); + } + } + + self.delete_filter_manager = Some(manager); + } + } } pub(crate) struct MergeManifestProcess { @@ -827,3 +942,16 @@ impl MergeManifestManager { Ok(merge_manifests) } } + +pub(crate) fn new_manifest_path( + metadata_location: &str, + meta_root_path: &str, + commit_uuid: Uuid, + manifest_counter: u64, + format: DataFileFormat, +) -> String { + format!( + "{}/{}/{}-m{}.{}", + metadata_location, meta_root_path, commit_uuid, manifest_counter, format + ) +} From cf4fdd0300358949ba1a251273ec9f5dc5beecb4 Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 31 Oct 2025 18:43:32 +0800 Subject: [PATCH 21/33] fix(azdls): enable append mode for AZDLS write operations (#89) * fix(azdls): enable append mode for AZDLS write operations * fix: fix doc --- crates/iceberg/src/io/file_io.rs | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 5dab5ed898..24fc42ebbe 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -168,11 +168,19 @@ impl FileIO { let (op, relative_path) = self.inner.create_operator(&path)?; let path = path.as_ref().to_string(); let relative_path_pos = path.len() - relative_path.len(); + + // ADLS requires append mode for writes + #[cfg(feature = "storage-azdls")] + let append_file = matches!(self.inner.as_ref(), Storage::Azdls { .. }); + #[cfg(not(feature = "storage-azdls"))] + let append_file = false; + Ok(OutputFile { op, path, relative_path_pos, chunk_size: self.get_write_chunk_size()?, + append_file, }) } @@ -438,6 +446,8 @@ pub struct OutputFile { relative_path_pos: usize, // Chunk size for write operations to ensure consistent size of multipart chunks chunk_size: Option, + // Whether to use append mode for writes (required for some storage backends like AZDLS) + append_file: bool, } impl OutputFile { @@ -485,7 +495,10 @@ impl OutputFile { /// /// For one-time writing, use [`Self::write`] instead. pub async fn writer(&self) -> crate::Result> { - let mut writer = self.op.writer_with(&self.path[self.relative_path_pos..]); + let mut writer = self + .op + .writer_with(&self.path[self.relative_path_pos..]) + .append(self.append_file); if let Some(chunk_size) = self.chunk_size { writer = writer.chunk(chunk_size); } From cb4753a41a907b0e93940a2334859db214917d2f Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 10 Nov 2025 14:23:13 +0800 Subject: [PATCH 22/33] feat: check file existence (#92) fix: use branch_snapshot instead of current snashot (#94) --- crates/iceberg/src/transaction/append.rs | 16 +- .../src/transaction/overwrite_files.rs | 19 +- .../iceberg/src/transaction/rewrite_files.rs | 19 +- crates/iceberg/src/transaction/snapshot.rs | 189 ++++++++++++++---- 4 files changed, 167 insertions(+), 76 deletions(-) diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index b67831f888..72e747f608 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -155,13 +155,7 @@ impl TransactionAction for FastAppendAction { // Checks duplicate files if self.check_duplicate { - snapshot_producer - .validate_duplicate_files(&self.added_data_files) - .await?; - - snapshot_producer - .validate_duplicate_files(&self.added_delete_files) - .await?; + snapshot_producer.validate_data_file_changes().await?; } snapshot_producer @@ -318,13 +312,7 @@ impl TransactionAction for MergeAppendAction { // Checks duplicate files if self.check_duplicate { - snapshot_producer - .validate_duplicate_files(&self.added_data_files) - .await?; - - snapshot_producer - .validate_duplicate_files(&self.added_delete_files) - .await?; + snapshot_producer.validate_data_file_changes().await?; } if self.merge_enabled { diff --git a/crates/iceberg/src/transaction/overwrite_files.rs b/crates/iceberg/src/transaction/overwrite_files.rs index 27fdb4a408..d3d08d3c37 100644 --- a/crates/iceberg/src/transaction/overwrite_files.rs +++ b/crates/iceberg/src/transaction/overwrite_files.rs @@ -42,7 +42,6 @@ pub struct OverwriteFilesAction { min_count_to_merge: u32, merge_enabled: bool, - check_duplicate: bool, // below are properties used to create SnapshotProducer when commit commit_uuid: Option, key_metadata: Option>, @@ -55,6 +54,7 @@ pub struct OverwriteFilesAction { new_data_file_sequence_number: Option, target_branch: Option, enable_delete_filter_manager: bool, + check_file_existence: bool, } pub struct OverwriteFilesOperation; @@ -65,7 +65,6 @@ impl OverwriteFilesAction { target_size_bytes: MANIFEST_TARGET_SIZE_BYTES_DEFAULT, min_count_to_merge: MANIFEST_MIN_MERGE_COUNT_DEFAULT, merge_enabled: MANIFEST_MERGE_ENABLED_DEFAULT, - check_duplicate: true, commit_uuid: None, key_metadata: None, snapshot_properties: HashMap::new(), @@ -77,6 +76,7 @@ impl OverwriteFilesAction { new_data_file_sequence_number: None, target_branch: None, enable_delete_filter_manager: false, + check_file_existence: false, } } @@ -165,7 +165,11 @@ impl OverwriteFilesAction { // This avoids commit conflicts with updates that add newer equality deletes at a higher sequence number. pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Self { self.new_data_file_sequence_number = Some(seq); + self + } + pub fn set_check_file_existence(mut self, check: bool) -> Self { + self.check_file_existence = check; self } } @@ -335,15 +339,8 @@ impl TransactionAction for OverwriteFilesAction { snapshot_producer.enable_delete_filter_manager(); } - // Checks duplicate files - if self.check_duplicate { - snapshot_producer - .validate_duplicate_files(&self.added_data_files) - .await?; - - snapshot_producer - .validate_duplicate_files(&self.added_delete_files) - .await?; + if self.check_file_existence { + snapshot_producer.validate_data_file_changes().await?; } if self.merge_enabled { diff --git a/crates/iceberg/src/transaction/rewrite_files.rs b/crates/iceberg/src/transaction/rewrite_files.rs index d18de28de0..a3d5db0bd9 100644 --- a/crates/iceberg/src/transaction/rewrite_files.rs +++ b/crates/iceberg/src/transaction/rewrite_files.rs @@ -42,7 +42,6 @@ pub struct RewriteFilesAction { min_count_to_merge: u32, merge_enabled: bool, - check_duplicate: bool, // below are properties used to create SnapshotProducer when commit commit_uuid: Option, key_metadata: Option>, @@ -55,6 +54,7 @@ pub struct RewriteFilesAction { new_data_file_sequence_number: Option, target_branch: Option, enable_delete_filter_manager: bool, + check_file_existence: bool, } pub struct RewriteFilesOperation; @@ -65,7 +65,6 @@ impl RewriteFilesAction { target_size_bytes: MANIFEST_TARGET_SIZE_BYTES_DEFAULT, min_count_to_merge: MANIFEST_MIN_MERGE_COUNT_DEFAULT, merge_enabled: MANIFEST_MERGE_ENABLED_DEFAULT, - check_duplicate: true, commit_uuid: None, key_metadata: None, snapshot_properties: HashMap::new(), @@ -77,6 +76,7 @@ impl RewriteFilesAction { new_data_file_sequence_number: None, target_branch: None, enable_delete_filter_manager: false, + check_file_existence: false, } } @@ -165,7 +165,11 @@ impl RewriteFilesAction { // This avoids commit conflicts with updates that add newer equality deletes at a higher sequence number. pub fn set_new_data_file_sequence_number(mut self, seq: i64) -> Self { self.new_data_file_sequence_number = Some(seq); + self + } + pub fn set_check_file_existence(mut self, check: bool) -> Self { + self.check_file_existence = check; self } } @@ -335,15 +339,8 @@ impl TransactionAction for RewriteFilesAction { snapshot_producer.enable_delete_filter_manager(); } - // Checks duplicate files - if self.check_duplicate { - snapshot_producer - .validate_duplicate_files(&self.added_data_files) - .await?; - - snapshot_producer - .validate_duplicate_files(&self.added_delete_files) - .await?; + if self.check_file_existence { + snapshot_producer.validate_data_file_changes().await?; } if self.merge_enabled { diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index fe4d0aa5d4..552d43f183 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -164,46 +164,46 @@ impl<'a> SnapshotProducer<'a> { Ok(()) } - pub(crate) async fn validate_duplicate_files( - &self, - added_data_files: &[DataFile], - ) -> Result<()> { - let new_files: HashSet<&str> = added_data_files - .iter() - .map(|df| df.file_path.as_str()) - .collect(); - - let mut referenced_files = Vec::new(); - if let Some(current_snapshot) = self.table.metadata().snapshot_for_ref(&self.target_branch) - { - let manifest_list = current_snapshot - .load_manifest_list(self.table.file_io(), &self.table.metadata_ref()) - .await?; - for manifest_list_entry in manifest_list.entries() { - let manifest = manifest_list_entry - .load_manifest(self.table.file_io()) - .await?; - for entry in manifest.entries() { - let file_path = entry.file_path(); - if new_files.contains(file_path) && entry.is_alive() { - referenced_files.push(file_path.to_string()); - } - } - } - } - - if !referenced_files.is_empty() { - return Err(Error::new( - ErrorKind::DataInvalid, - format!( - "Cannot add files that are already referenced by table, files: {}", - referenced_files.join(", ") - ), - )); - } - - Ok(()) - } + // pub(crate) async fn validate_duplicate_files( + // &self, + // added_data_files: &[DataFile], + // ) -> Result<()> { + // let new_files: HashSet<&str> = added_data_files + // .iter() + // .map(|df| df.file_path.as_str()) + // .collect(); + + // let mut referenced_files = Vec::new(); + // if let Some(current_snapshot) = self.table.metadata().snapshot_for_ref(&self.target_branch) + // { + // let manifest_list = current_snapshot + // .load_manifest_list(self.table.file_io(), &self.table.metadata_ref()) + // .await?; + // for manifest_list_entry in manifest_list.entries() { + // let manifest = manifest_list_entry + // .load_manifest(self.table.file_io()) + // .await?; + // for entry in manifest.entries() { + // let file_path = entry.file_path(); + // if new_files.contains(file_path) && entry.is_alive() { + // referenced_files.push(file_path.to_string()); + // } + // } + // } + // } + + // if !referenced_files.is_empty() { + // return Err(Error::new( + // ErrorKind::DataInvalid, + // format!( + // "Cannot add files that are already referenced by table, files: {}", + // referenced_files.join(", ") + // ), + // )); + // } + + // Ok(()) + // } pub(crate) fn generate_unique_snapshot_id(table: &Table) -> i64 { let generate_random_id = || -> i64 { @@ -751,6 +751,115 @@ impl<'a> SnapshotProducer<'a> { self.delete_filter_manager = Some(manager); } } + + /// Validate data file operations in a single pass through manifests. + /// This checks both: + /// 1. Added files don't already exist in the table (duplicate prevention) + /// 2. Deleted files actually exist in the table (existence validation) + pub(crate) async fn validate_data_file_changes(&self) -> Result<()> { + // Early return if nothing to validate + if self.added_data_files.is_empty() + && self.added_delete_files.is_empty() + && self.removed_data_file_paths.is_empty() + { + return Ok(()); + } + + // Use a mutable set - remove files as we find them + let mut files_to_delete: HashSet<&str> = self + .removed_data_file_paths + .iter() + .map(|s| s.as_str()) + .collect(); + + let table = &self.table; + let branch_snapshot_ref = table.metadata().snapshot_for_ref(self.target_branch()); + + // If trying to delete files but no snapshot exists, that's an error + if !files_to_delete.is_empty() && branch_snapshot_ref.is_none() { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot delete files from a table with no current snapshot, files: {}", + files_to_delete + .iter() + .copied() + .collect::>() + .join(", ") + ), + )); + } + + let files_to_add: HashSet<&str> = self + .added_data_files + .iter() + .chain(self.added_delete_files.iter()) + .map(|df| df.file_path.as_str()) + .collect(); + + let mut duplicate_files = Vec::new(); + + // Single pass through all manifests + if let Some(current_snapshot) = branch_snapshot_ref { + let manifest_list = current_snapshot + .load_manifest_list(table.file_io(), table.metadata_ref().as_ref()) + .await?; + + for manifest_list_entry in manifest_list.entries() { + let manifest = manifest_list_entry.load_manifest(table.file_io()).await?; + for entry in manifest.entries() { + if !entry.is_alive() { + continue; + } + + let file_path = entry.file_path(); + + // Check for duplicate adds + if files_to_add.contains(file_path) { + duplicate_files.push(file_path.to_string()); + } + + // Remove from files_to_delete as we find them + // Remaining files in the set don't exist in the snapshot + if !files_to_delete.is_empty() { + files_to_delete.remove(file_path); + } + + // Early exit optimization: if both checks are done, stop scanning + if duplicate_files.len() == files_to_add.len() && files_to_delete.is_empty() { + break; + } + } + } + } + + // Validate no duplicate files are being added + if !duplicate_files.is_empty() { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot add files that are already referenced by table, files: {}", + duplicate_files.join(", ") + ), + )); + } + + // Any remaining files in files_to_delete don't exist in the snapshot + if !files_to_delete.is_empty() { + let non_existent_files: Vec = + files_to_delete.iter().map(|s| s.to_string()).collect(); + + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot delete files that are not in the current snapshot, files: {}", + non_existent_files.join(", ") + ), + )); + } + + Ok(()) + } } pub(crate) struct MergeManifestProcess { From bda9a37b27c4e7e400a31c6918e3cdc4bc4cb885 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 13 Nov 2025 16:22:57 +0800 Subject: [PATCH 23/33] enable ci --- .github/workflows/ci.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index d547397808..265c4c94a0 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -21,7 +21,10 @@ on: push: branches: - main + - dev_rebase_main_20251111 pull_request: + branches: + - dev_rebase_main_20251111 paths: - '**' # Include all files and directories in the repository by default. - '!.github/ISSUE_TEMPLATE/**' # Exclude files and directories that don't impact tests or code like templates, metadata, and documentation. From 9374588d67a0bb2ec9a5de7584efd9a23e1289de Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 13 Nov 2025 16:28:55 +0800 Subject: [PATCH 24/33] fmt --- crates/iceberg/src/io/file_io.rs | 4 +- .../src/transaction/manifest_filter.rs | 491 +++++++++++------- crates/iceberg/src/transaction/mod.rs | 2 +- .../tests/shared_tests/rewrite_files_test.rs | 18 +- 4 files changed, 304 insertions(+), 211 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 24fc42ebbe..e0a519aecb 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -168,13 +168,13 @@ impl FileIO { let (op, relative_path) = self.inner.create_operator(&path)?; let path = path.as_ref().to_string(); let relative_path_pos = path.len() - relative_path.len(); - + // ADLS requires append mode for writes #[cfg(feature = "storage-azdls")] let append_file = matches!(self.inner.as_ref(), Storage::Azdls { .. }); #[cfg(not(feature = "storage-azdls"))] let append_file = false; - + Ok(OutputFile { op, path, diff --git a/crates/iceberg/src/transaction/manifest_filter.rs b/crates/iceberg/src/transaction/manifest_filter.rs index ecd737c6e8..783b3d4503 100644 --- a/crates/iceberg/src/transaction/manifest_filter.rs +++ b/crates/iceberg/src/transaction/manifest_filter.rs @@ -16,15 +16,16 @@ // under the License. use std::collections::{HashMap, HashSet}; -use std::sync::{Arc, atomic::{AtomicU64, Ordering}}; +use std::sync::Arc; +use std::sync::atomic::{AtomicU64, Ordering}; use uuid::Uuid; use crate::error::Result; use crate::io::FileIO; use crate::spec::{ - DataFile, FormatVersion, ManifestContentType, ManifestFile, ManifestStatus, ManifestWriter, - ManifestWriterBuilder, PartitionSpec, Schema + DataFile, FormatVersion, ManifestContentType, ManifestFile, ManifestStatus, ManifestWriter, + ManifestWriterBuilder, PartitionSpec, Schema, }; use crate::transaction::snapshot::new_manifest_path; use crate::{Error, ErrorKind}; @@ -52,7 +53,7 @@ impl ManifestWriterContext { format_version: FormatVersion, snapshot_id: i64, file_io: FileIO, - key_metadata: Option>, + key_metadata: Option>, ) -> Self { Self { metadata_location, @@ -80,7 +81,7 @@ impl ManifestWriterContext { self.manifest_counter.fetch_add(1, Ordering::SeqCst), crate::spec::DataFileFormat::Avro, ); - + let output = self.file_io.new_output(&new_manifest_path)?; let builder = ManifestWriterBuilder::new( output, @@ -173,7 +174,7 @@ impl ManifestFilterManager { pub fn delete_file(&mut self, file: DataFile) -> Result<()> { // Todo: check all deletes references in manifests? let file_path = file.file_path.clone(); - + self.files_to_delete.insert(file_path, file); Ok(()) @@ -198,9 +199,7 @@ impl ManifestFilterManager { let mut filtered = Vec::with_capacity(manifests.len()); for manifest in manifests { - let filtered_manifest = self - .filter_manifest(table_schema, manifest) - .await?; + let filtered_manifest = self.filter_manifest(table_schema, manifest).await?; filtered.push(filtered_manifest); } @@ -228,7 +227,8 @@ impl ManifestFilterManager { if self.manifest_has_deleted_files(&manifest).await? { // Load and filter the manifest - self.filter_manifest_with_deleted_files(table_schema, manifest).await + self.filter_manifest_with_deleted_files(table_schema, manifest) + .await } else { // If no deleted files are found, just return the original manifest self.filtered_manifests @@ -257,17 +257,14 @@ impl ManifestFilterManager { // Load the original manifest let original_manifest = manifest.load_manifest(&self.file_io).await?; - let ( - entries, - manifest_meta_data, - ) = original_manifest.into_parts(); - + let (entries, manifest_meta_data) = original_manifest.into_parts(); + // Check if this is a delete manifest let is_delete = manifest.content == ManifestContentType::Deletes; - + // Track deleted files for duplicate detection let mut deleted_files = HashMap::new(); - + // Create an output path for the filtered manifest using writer context let partition_spec = manifest_meta_data.partition_spec.clone(); @@ -277,30 +274,30 @@ impl ManifestFilterManager { table_schema, &partition_spec, )?; - + // Process each live entry in the manifest - for entry in &entries{ + for entry in &entries { if !entry.is_alive() { continue; } let entry = entry.as_ref(); let file = entry.data_file(); - + // Check if file is marked for deletion based on various criteria - let marked_for_delete = + let marked_for_delete = // Check if file is in delete files collection self.files_to_delete.contains_key(file.file_path()) || // For delete manifests, check sequence number for old delete files - (is_delete && matches!(entry.sequence_number(), Some(seq_num) if seq_num != crate::spec::UNASSIGNED_SEQUENCE_NUMBER - && seq_num > 0 + (is_delete && matches!(entry.sequence_number(), Some(seq_num) if seq_num != crate::spec::UNASSIGNED_SEQUENCE_NUMBER + && seq_num > 0 && seq_num < self.min_sequence_number)); - // TODO: Add expression evaluation logic + // TODO: Add expression evaluation logic if marked_for_delete { // Check if all rows match let all_rows_match = marked_for_delete; - + // Validation check: cannot delete file where some, but not all, rows match filter // unless it's a delete file (ignore delete files where some records may not match) if !all_rows_match && !is_delete { @@ -312,19 +309,20 @@ impl ManifestFilterManager { ), )); } - + if all_rows_match { // Mark this entry as deleted writer.add_delete_entry(entry.clone())?; - + // Create a copy of the file without stats let file_copy = file.clone(); // For file that it was deleted using an expression - self.files_to_delete.insert(file.file_path().to_string(), file_copy.clone()); + self.files_to_delete + .insert(file.file_path().to_string(), file_copy.clone()); // TODO: add file to removed_data_file_path once we implement drop_partition - + // Track deleted files for duplicate detection and validation if deleted_files.contains_key(file_copy.file_path()) { // TODO: Log warning about duplicate @@ -342,21 +340,20 @@ impl ManifestFilterManager { writer.add_existing_entry(entry.clone())?; } } - + // Write the filtered manifest let filtered_manifest = writer.write_manifest_file().await?; - + // Update caches self.filtered_manifests .insert(manifest.manifest_path.clone(), filtered_manifest.clone()); - + // Track deleted files for validation - convert HashSet to Vec of file paths - let deleted_file_paths: Vec = deleted_files - .keys().cloned().collect(); - + let deleted_file_paths: Vec = deleted_files.keys().cloned().collect(); + self.filtered_manifest_to_deleted_files .insert(filtered_manifest.manifest_path.clone(), deleted_file_paths); - + Ok(filtered_manifest) } @@ -397,37 +394,37 @@ impl ManifestFilterManager { async fn manifest_has_deleted_files(&self, manifest_file: &ManifestFile) -> Result { let manifest = manifest_file.load_manifest(&self.file_io).await?; - + let is_delete = manifest_file.content == ManifestContentType::Deletes; for entry in manifest.entries() { let entry = entry.as_ref(); - + // Skip entries that are already deleted if entry.status() == ManifestStatus::Deleted { continue; } - + let file = entry.data_file(); - + // Check if file is marked for deletion based on various criteria - let marked_for_delete = + let marked_for_delete = // Check if file path is in files to delete self.files_to_delete.contains_key(file.file_path()) || // For delete manifests, check sequence number for old delete files - (is_delete && + (is_delete && entry.status() != ManifestStatus::Deleted && - matches!(entry.sequence_number(), Some(seq_num) if seq_num != crate::spec::UNASSIGNED_SEQUENCE_NUMBER - && seq_num > 0 + matches!(entry.sequence_number(), Some(seq_num) if seq_num != crate::spec::UNASSIGNED_SEQUENCE_NUMBER + && seq_num > 0 && seq_num < self.min_sequence_number)); - - // TODO: Add dangling delete vector check: (is_delete && self.is_dangling_dv(file)) - + + // TODO: Add dangling delete vector check: (is_delete && self.is_dangling_dv(file)) + // TODO: Add expression evaluation logic if marked_for_delete { // Check if all rows match let all_rows_match = marked_for_delete; // || evaluator.rowsMustMatch(file) equivalent - + // Validation check: cannot delete file where some, but not all, rows match filter // unless it's a delete file if !all_rows_match && !is_delete { @@ -439,7 +436,7 @@ impl ManifestFilterManager { ), )); } - + if all_rows_match { // Check fail_any_delete flag if self.fail_any_delete { @@ -448,35 +445,38 @@ impl ManifestFilterManager { "Operation would delete existing data".to_string(), )); } - + // As soon as a deleted file is detected, stop scanning and return true return Ok(true); } } } - + Ok(false) } pub(crate) fn remove_dangling_deletes_for(&mut self, file_paths: &HashSet) { - self.removed_data_file_path.extend(file_paths.iter().cloned()); + self.removed_data_file_path + .extend(file_paths.iter().cloned()); } } #[cfg(test)] mod tests { - use super::*; - use crate::io::FileIOBuilder; - use crate::spec::{ - DataContentType, DataFileFormat, NestedField, PrimitiveType, Type, - ManifestEntry, ManifestStatus, ManifestFile, ManifestContentType, Struct, Schema, - PartitionSpec, FormatVersion, ManifestWriterBuilder - }; use std::collections::HashMap; use std::sync::Arc; + use tempfile::TempDir; use uuid::Uuid; + use super::*; + use crate::io::FileIOBuilder; + use crate::spec::{ + DataContentType, DataFileFormat, FormatVersion, ManifestContentType, ManifestEntry, + ManifestFile, ManifestStatus, ManifestWriterBuilder, NestedField, PartitionSpec, + PrimitiveType, Schema, Struct, Type, + }; + // Helper function to create a test schema fn create_test_schema() -> Schema { Schema::builder() @@ -489,7 +489,7 @@ mod tests { )), Arc::new(NestedField::optional( 2, - "name", + "name", Type::Primitive(PrimitiveType::String), )), ]) @@ -509,7 +509,7 @@ mod tests { file_size_in_bytes: 1024, column_sizes: HashMap::new(), value_counts: HashMap::new(), - null_value_counts: HashMap::new(), + null_value_counts: HashMap::new(), nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), @@ -525,7 +525,10 @@ mod tests { } // Helper function to create a test ManifestFile with default values - fn create_test_manifest_file(manifest_path: &str, content: ManifestContentType) -> ManifestFile { + fn create_test_manifest_file( + manifest_path: &str, + content: ManifestContentType, + ) -> ManifestFile { create_manifest_with_counts(manifest_path, content, 10, 5, 0) } @@ -574,9 +577,13 @@ mod tests { fn setup_test_manager() -> (ManifestFilterManager, TempDir) { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let metadata_location = temp_dir.path().join("metadata.json").to_string_lossy().to_string(); + let metadata_location = temp_dir + .path() + .join("metadata.json") + .to_string_lossy() + .to_string(); let meta_root_path = temp_dir.path().to_string_lossy().to_string(); - + let writer_context = ManifestWriterContext::new( metadata_location, meta_root_path, @@ -587,9 +594,9 @@ mod tests { file_io.clone(), None, ); - + let manager = ManifestFilterManager::new(file_io, writer_context); - + (manager, temp_dir) } @@ -609,8 +616,9 @@ mod tests { None, schema.clone().into(), partition_spec, - ).build_v2_data(); - + ) + .build_v2_data(); + for entry in entries { writer.add_entry(entry)?; } @@ -620,11 +628,14 @@ mod tests { // Helper function to create manifest entries from data files fn create_entries_from_files(files: Vec) -> Vec { - files.into_iter() - .map(|file| ManifestEntry::builder() - .status(ManifestStatus::Added) - .data_file(file) - .build()) + files + .into_iter() + .map(|file| { + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file(file) + .build() + }) .collect() } @@ -660,7 +671,7 @@ mod tests { #[test] fn test_new_manifest_filter_manager() { let (manager, _temp_dir) = setup_test_manager(); - + // Test initial state assert!(!manager.contains_deletes()); assert_eq!(manager.min_sequence_number, 0); @@ -672,12 +683,10 @@ mod tests { #[test] fn test_configuration_flags() { let (manager, _temp_dir) = setup_test_manager(); - - let mut configured_manager = manager - .fail_any_delete() - .fail_missing_delete_paths(); + + let mut configured_manager = manager.fail_any_delete().fail_missing_delete_paths(); configured_manager.drop_delete_files_older_than(100); - + assert!(configured_manager.fail_any_delete); assert!(configured_manager.fail_missing_delete_paths); assert_eq!(configured_manager.min_sequence_number, 100); @@ -686,26 +695,26 @@ mod tests { #[test] fn test_delete_file() { let (mut manager, _temp_dir) = setup_test_manager(); - + // Test 1: Delete file by path let file_path = "/test/path/file.parquet"; let test_file1 = create_test_data_file(file_path, 0); - + // Initially no deletes assert!(!manager.contains_deletes()); - + // Add file to delete manager.delete_file(test_file1).unwrap(); - + // Should now contain deletes assert!(manager.contains_deletes()); assert!(manager.files_to_delete.contains_key(file_path)); - + // Test 2: Delete another file and verify tracking let test_file2 = create_test_data_file("/test/data/file2.parquet", 0); let file_path2 = test_file2.file_path.clone(); manager.delete_file(test_file2).unwrap(); - + // Should track both files for deletion let deleted_files = manager.files_to_be_deleted(); assert_eq!(deleted_files.len(), 2); @@ -715,34 +724,34 @@ mod tests { #[test] fn test_manifest_has_no_live_files() { // Test manifest with no live files - let manifest_no_live = create_manifest_with_counts( - "/test/manifest1.avro", - ManifestContentType::Data, - 0, 0, 5 - ); - assert!(ManifestFilterManager::manifest_has_no_live_files(&manifest_no_live)); - + let manifest_no_live = + create_manifest_with_counts("/test/manifest1.avro", ManifestContentType::Data, 0, 0, 5); + assert!(ManifestFilterManager::manifest_has_no_live_files( + &manifest_no_live + )); + // Test manifest with live files - let manifest_with_live = create_test_manifest_file("/test/manifest2.avro", ManifestContentType::Data); - assert!(!ManifestFilterManager::manifest_has_no_live_files(&manifest_with_live)); + let manifest_with_live = + create_test_manifest_file("/test/manifest2.avro", ManifestContentType::Data); + assert!(!ManifestFilterManager::manifest_has_no_live_files( + &manifest_with_live + )); } #[test] fn test_can_contain_deleted_files() { let (mut manager, _temp_dir) = setup_test_manager(); - + // Manifest with no live files should not contain deleted files - let manifest_no_live = create_manifest_with_counts( - "/test/manifest1.avro", - ManifestContentType::Data, - 0, 0, 5 - ); + let manifest_no_live = + create_manifest_with_counts("/test/manifest1.avro", ManifestContentType::Data, 0, 0, 5); assert!(!manager.can_contain_deleted_files(&manifest_no_live)); - + // Manifest with live files but no deletes - let manifest_with_live = create_test_manifest_file("/test/manifest2.avro", ManifestContentType::Data); + let manifest_with_live = + create_test_manifest_file("/test/manifest2.avro", ManifestContentType::Data); assert!(!manager.can_contain_deleted_files(&manifest_with_live)); - + // Add deletes and test again let test_file = create_test_data_file("/test/file.parquet", 0); manager.delete_file(test_file).unwrap(); @@ -753,7 +762,7 @@ mod tests { async fn test_filter_manifests_empty_input() { let (mut manager, _temp_dir) = setup_test_manager(); let schema = create_test_schema(); - + let result = manager.filter_manifests(&schema, vec![]).await.unwrap(); assert!(result.is_empty()); } @@ -761,62 +770,75 @@ mod tests { #[test] fn test_validate_required_deletes_success() { let (manager, _temp_dir) = setup_test_manager(); - + // Test validation with no required deletes - let manifests = vec![create_test_manifest_file("/test/manifest.avro", ManifestContentType::Data)]; + let manifests = vec![create_test_manifest_file( + "/test/manifest.avro", + ManifestContentType::Data, + )]; let result = manager.validate_required_deletes(&manifests); assert!(result.is_ok()); } - #[test] + #[test] fn test_validate_required_deletes_failure() { let (mut manager, _temp_dir) = setup_test_manager(); - + // Enable fail_missing_delete_paths manager.fail_missing_delete_paths = true; - + // Add a required delete file that won't be found let missing_file = create_test_data_file("/missing/file.parquet", 0); manager.delete_file(missing_file).unwrap(); - - let manifests = vec![create_test_manifest_file("/test/manifest.avro", ManifestContentType::Data)]; + + let manifests = vec![create_test_manifest_file( + "/test/manifest.avro", + ManifestContentType::Data, + )]; let result = manager.validate_required_deletes(&manifests); - + assert!(result.is_err()); - assert!(result.unwrap_err().to_string().contains("Required delete path missing")); + assert!( + result + .unwrap_err() + .to_string() + .contains("Required delete path missing") + ); } #[tokio::test] async fn test_comprehensive_deletion_logic() { let (mut manager, temp_dir) = setup_test_manager(); let schema = create_test_schema(); - + // Create test data files let keep_file = create_test_data_file("/test/keep_me.parquet", 0); let delete_file = create_test_data_file("/test/delete_me.parquet", 0); manager.delete_file(delete_file.clone()).unwrap(); - + // Write manifest with both files let manifest_path = temp_dir.path().join("test_manifest.avro"); let manifest_path_str = manifest_path.to_str().unwrap(); let entries = create_entries_from_files(vec![keep_file.clone(), delete_file.clone()]); - write_manifest_with_entries(&manager, manifest_path_str, &schema, entries, 12345).await.unwrap(); - + write_manifest_with_entries(&manager, manifest_path_str, &schema, entries, 12345) + .await + .unwrap(); + // Create ManifestFile metadata let manifest = create_manifest_metadata( manifest_path_str, ManifestContentType::Data, 10, 12345, - (2, 0, 0) + (2, 0, 0), ); - + // Verify manifest filtering capabilities assert!(manager.can_contain_deleted_files(&manifest)); assert!(manager.manifest_has_deleted_files(&manifest).await.unwrap()); assert!(manager.files_to_delete.contains_key(&delete_file.file_path)); assert!(!manager.files_to_delete.contains_key(&keep_file.file_path)); - + // Verify manager state assert!(manager.contains_deletes()); let files_to_delete = manager.files_to_be_deleted(); @@ -828,28 +850,36 @@ mod tests { fn test_min_sequence_number_logic() { let (mut manager, _temp_dir) = setup_test_manager(); manager.min_sequence_number = 5; - + // Test basic sequence number comparison logic assert_eq!(manager.min_sequence_number, 5); let (old_sequence, new_sequence) = (3, 10); assert!(old_sequence < manager.min_sequence_number); assert!(new_sequence >= manager.min_sequence_number); - + // Create manifests with different sequence numbers let old_manifest = create_manifest_with_sequence( - "/test/old.avro", ManifestContentType::Data, old_sequence, old_sequence + "/test/old.avro", + ManifestContentType::Data, + old_sequence, + old_sequence, ); let new_manifest = create_manifest_with_sequence( - "/test/new.avro", ManifestContentType::Data, new_sequence, new_sequence + "/test/new.avro", + ManifestContentType::Data, + new_sequence, + new_sequence, ); - + // Add files to delete for testing - manager.delete_file(create_test_data_file("/test/file.parquet", 0)).unwrap(); - + manager + .delete_file(create_test_data_file("/test/file.parquet", 0)) + .unwrap(); + // Both manifests should be able to contain deleted files (key filtering behavior) assert!(manager.can_contain_deleted_files(&old_manifest)); assert!(manager.can_contain_deleted_files(&new_manifest)); - + // Verify sequence number properties assert!(old_manifest.min_sequence_number < manager.min_sequence_number); assert!(new_manifest.min_sequence_number >= manager.min_sequence_number); @@ -858,32 +888,36 @@ mod tests { #[test] fn test_deletion_tracking_and_validation() { let (mut manager, _temp_dir) = setup_test_manager(); - + let delete_file = create_test_data_file("/test/delete_me.parquet", 0); - + // Initially no deletes assert!(!manager.contains_deletes()); assert_eq!(manager.files_to_be_deleted().len(), 0); - + // Add the file to be deleted manager.delete_file(delete_file.clone()).unwrap(); - + // Verify deletion tracking assert!(manager.contains_deletes()); assert_eq!(manager.files_to_be_deleted().len(), 1); - assert_eq!(manager.files_to_be_deleted()[0].file_path, delete_file.file_path); - + assert_eq!( + manager.files_to_be_deleted()[0].file_path, + delete_file.file_path + ); + // Create a manifest that could contain deleted files let manifest = create_manifest_with_sequence( - "/test/test_manifest.avro", + "/test/test_manifest.avro", ManifestContentType::Data, - 10, 1 + 10, + 1, ); - + // Verify manifest can contain deleted files assert!(manager.can_contain_deleted_files(&manifest)); assert!(manager.files_to_delete.contains_key(&delete_file.file_path)); - + // Validation should pass when no required deletes are set assert!(manager.validate_required_deletes(&[manifest]).is_ok()); } @@ -892,7 +926,7 @@ mod tests { async fn test_filter_manifests_with_entries_and_rewrite() { let (mut manager, temp_dir) = setup_test_manager(); let schema = create_test_schema(); - + // Create test files let files_to_keep = vec![ create_test_data_file("/test/keep1.parquet", 0), @@ -902,70 +936,112 @@ mod tests { create_test_data_file("/test/delete1.parquet", 0), create_test_data_file("/test/delete2.parquet", 0), ]; - + // Mark files for deletion for file in &files_to_delete { manager.delete_file(file.clone()).unwrap(); } - + // Create two manifests with mixed files - let manifest_paths: Vec<_> = (1..=2).map(|i| { - temp_dir.path().join(format!("manifest{}.avro", i)).to_string_lossy().to_string() - }).collect(); - + let manifest_paths: Vec<_> = (1..=2) + .map(|i| { + temp_dir + .path() + .join(format!("manifest{}.avro", i)) + .to_string_lossy() + .to_string() + }) + .collect(); + for (idx, path) in manifest_paths.iter().enumerate() { let entries = create_entries_from_files(vec![ files_to_keep[idx].clone(), files_to_delete[idx].clone(), ]); - write_manifest_with_entries(&manager, path, &schema, entries, 12345 + idx as i64).await.unwrap(); + write_manifest_with_entries(&manager, path, &schema, entries, 12345 + idx as i64) + .await + .unwrap(); } - + // Create ManifestFile metadata objects - let input_manifests: Vec<_> = manifest_paths.iter().enumerate().map(|(idx, path)| { - create_manifest_metadata(path, ManifestContentType::Data, 10, 12345 + idx as i64, (2, 0, 0)) - }).collect(); - + let input_manifests: Vec<_> = manifest_paths + .iter() + .enumerate() + .map(|(idx, path)| { + create_manifest_metadata( + path, + ManifestContentType::Data, + 10, + 12345 + idx as i64, + (2, 0, 0), + ) + }) + .collect(); + // Filter manifests - let filtered_manifests = manager.filter_manifests(&schema, input_manifests.clone()).await.unwrap(); - + let filtered_manifests = manager + .filter_manifests(&schema, input_manifests.clone()) + .await + .unwrap(); + // Verify results assert_eq!(filtered_manifests.len(), 2); - assert_ne!(filtered_manifests[0].manifest_path, input_manifests[0].manifest_path); - assert_ne!(filtered_manifests[1].manifest_path, input_manifests[1].manifest_path); - + assert_ne!( + filtered_manifests[0].manifest_path, + input_manifests[0].manifest_path + ); + assert_ne!( + filtered_manifests[1].manifest_path, + input_manifests[1].manifest_path + ); + // Verify deletion tracking assert_eq!(manager.files_to_be_deleted().len(), 2); - let deleted_paths: std::collections::HashSet<_> = manager.files_to_be_deleted() - .into_iter().map(|f| f.file_path.clone()).collect(); + let deleted_paths: std::collections::HashSet<_> = manager + .files_to_be_deleted() + .into_iter() + .map(|f| f.file_path.clone()) + .collect(); for file in &files_to_delete { assert!(deleted_paths.contains(&file.file_path)); } - + // Verify filtered manifest entries - let filtered_manifest1 = filtered_manifests[0].load_manifest(&manager.file_io).await.unwrap(); + let filtered_manifest1 = filtered_manifests[0] + .load_manifest(&manager.file_io) + .await + .unwrap(); let (entries_filtered, _) = filtered_manifest1.into_parts(); - - let (live_count, deleted_count) = entries_filtered.iter().fold((0, 0), |(live, deleted), entry| { - match entry.status() { - ManifestStatus::Added | ManifestStatus::Existing => (live + 1, deleted), - ManifestStatus::Deleted => (live, deleted + 1), - } - }); - + + let (live_count, deleted_count) = + entries_filtered + .iter() + .fold((0, 0), |(live, deleted), entry| match entry.status() { + ManifestStatus::Added | ManifestStatus::Existing => (live + 1, deleted), + ManifestStatus::Deleted => (live, deleted + 1), + }); + assert_eq!(live_count, 1); assert_eq!(deleted_count, 1); - + // Verify cache and tracking - assert!(manager.filtered_manifests.contains_key(&input_manifests[0].manifest_path)); - assert!(manager.filtered_manifest_to_deleted_files.contains_key(&filtered_manifests[0].manifest_path)); + assert!( + manager + .filtered_manifests + .contains_key(&input_manifests[0].manifest_path) + ); + assert!( + manager + .filtered_manifest_to_deleted_files + .contains_key(&filtered_manifests[0].manifest_path) + ); } #[test] fn test_unassigned_sequence_number_handling() { let (mut manager, _temp_dir) = setup_test_manager(); manager.drop_delete_files_older_than(100); - + // Create manifest with UNASSIGNED_SEQUENCE_NUMBER and no live files let manifest_unassigned = create_manifest_with_sequence( "/test/unassigned.avro", @@ -973,7 +1049,7 @@ mod tests { crate::spec::UNASSIGNED_SEQUENCE_NUMBER, crate::spec::UNASSIGNED_SEQUENCE_NUMBER, ); - + // Manifest with no live files should not contain deleted files assert!(!manager.can_contain_deleted_files(&manifest_unassigned)); } @@ -982,41 +1058,68 @@ mod tests { async fn test_cache_behavior() { let (mut manager, temp_dir) = setup_test_manager(); let schema = create_test_schema(); - + // Create and write a manifest without deleted files let manifest_path = temp_dir.path().join("cache_test.avro"); let manifest_path_str = manifest_path.to_str().unwrap(); - - let entries = create_entries_from_files(vec![create_test_data_file("/test/keep.parquet", 0)]); - write_manifest_with_entries(&manager, manifest_path_str, &schema, entries, 12345).await.unwrap(); - - let manifest = create_manifest_metadata(manifest_path_str, ManifestContentType::Data, 10, 12345, (1, 0, 0)); - + + let entries = + create_entries_from_files(vec![create_test_data_file("/test/keep.parquet", 0)]); + write_manifest_with_entries(&manager, manifest_path_str, &schema, entries, 12345) + .await + .unwrap(); + + let manifest = create_manifest_metadata( + manifest_path_str, + ManifestContentType::Data, + 10, + 12345, + (1, 0, 0), + ); + // First and second calls should return the same cached result - let result1 = manager.filter_manifest(&schema, manifest.clone()).await.unwrap(); - let result2 = manager.filter_manifest(&schema, manifest.clone()).await.unwrap(); - + let result1 = manager + .filter_manifest(&schema, manifest.clone()) + .await + .unwrap(); + let result2 = manager + .filter_manifest(&schema, manifest.clone()) + .await + .unwrap(); + assert_eq!(result1.manifest_path, result2.manifest_path); - assert!(manager.filtered_manifests.contains_key(&manifest.manifest_path)); + assert!( + manager + .filtered_manifests + .contains_key(&manifest.manifest_path) + ); } #[test] fn test_batch_delete_operations() { let (mut manager, _temp_dir) = setup_test_manager(); - + assert!(!manager.contains_deletes()); - + // Add multiple files for deletion for i in 1..=3 { - manager.delete_file(create_test_data_file(&format!("/test/batch{}.parquet", i), 0)).unwrap(); + manager + .delete_file(create_test_data_file( + &format!("/test/batch{}.parquet", i), + 0, + )) + .unwrap(); } - + // Verify all files are tracked assert!(manager.contains_deletes()); assert_eq!(manager.files_to_be_deleted().len(), 3); - - let deleted_paths: std::collections::HashSet<_> = - manager.files_to_be_deleted().iter().map(|f| f.file_path.as_str()).collect(); + + let deleted_paths: std::collections::HashSet<_> = manager + .files_to_be_deleted() + .iter() + .map(|f| f.file_path.as_str()) + .collect(); assert!(deleted_paths.contains("/test/batch1.parquet")); assert!(deleted_paths.contains("/test/batch2.parquet")); assert!(deleted_paths.contains("/test/batch3.parquet")); @@ -1025,7 +1128,7 @@ mod tests { #[test] fn test_edge_case_empty_partition_specs() { let (mut manager, _temp_dir) = setup_test_manager(); - + // Create a data file with different partition spec let file_with_different_spec = DataFile { content: crate::spec::DataContentType::Data, @@ -1037,7 +1140,7 @@ mod tests { file_size_in_bytes: 1024, column_sizes: HashMap::new(), value_counts: HashMap::new(), - null_value_counts: HashMap::new(), + null_value_counts: HashMap::new(), nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), @@ -1050,7 +1153,7 @@ mod tests { content_offset: None, content_size_in_bytes: None, }; - + // Should be able to add file with different partition spec manager.delete_file(file_with_different_spec).unwrap(); assert!(manager.contains_deletes()); diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 6e38ea97db..99271f1823 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -553,7 +553,7 @@ mod test_row_lineage { use crate::transaction::tests::make_v3_minimal_table_in_catalog; use crate::transaction::{ApplyTransactionAction, Transaction}; - #[tokio::test] + #[tokio::test] async fn test_fast_append_with_row_lineage() { // Helper function to create a data file with specified number of rows fn file_with_rows(record_count: u64) -> DataFile { diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs index f0ccf674dd..b77db2d6e7 100644 --- a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -42,11 +42,8 @@ use crate::shared_tests::{random_ns, test_schema}; // Helper function to create a data file writer builder fn create_data_file_writer_builder( table: &Table, -) -> DataFileWriterBuilder< - ParquetWriterBuilder, - DefaultLocationGenerator, - DefaultFileNameGenerator, -> { +) -> DataFileWriterBuilder +{ let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); let file_name_generator = DefaultFileNameGenerator::new( "test".to_string(), @@ -445,10 +442,7 @@ async fn test_partition_spec_id_in_manifest() { // commit result let mut data_files_vec = Vec::default(); - async fn build_data_file_f( - schema: Arc, - table: &Table, - ) -> DataFile { + async fn build_data_file_f(schema: Arc, table: &Table) -> DataFile { let data_file_writer_builder = create_data_file_writer_builder(table); let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); @@ -465,11 +459,7 @@ async fn test_partition_spec_id_in_manifest() { } for _ in 0..10 { - let data_file = build_data_file_f( - schema.clone(), - &table, - ) - .await; + let data_file = build_data_file_f(schema.clone(), &table).await; data_files_vec.push(data_file.clone()); let tx = Transaction::new(&table); let append_action = tx.fast_append().add_data_files(vec![data_file]); From 396313594a571bc5daad9567be69d665af56a8ad Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 27 Nov 2025 14:11:50 +0800 Subject: [PATCH 25/33] feat: support position delete writer (#95) * support position writer * fmt * fix: fix integration-tests and bug * fix --------- Co-authored-by: Li0k --- Cargo.lock | 1 - crates/examples/Cargo.toml | 1 - crates/iceberg/src/transaction/append.rs | 6 +- crates/iceberg/src/transaction/snapshot.rs | 7 +- crates/iceberg/src/writer/base_writer/mod.rs | 1 + .../position_delete_file_writer.rs | 380 ++++++++++++++++++ .../shared_tests/append_data_file_test.rs | 24 +- .../shared_tests/overwrite_files_test.rs | 14 +- .../tests/shared_tests/rewrite_files_test.rs | 5 +- 9 files changed, 427 insertions(+), 12 deletions(-) create mode 100644 crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs diff --git a/Cargo.lock b/Cargo.lock index d195a2abda..911ddde8b8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3695,7 +3695,6 @@ dependencies = [ "iceberg", "iceberg-catalog-rest", "tokio", - "uuid", ] [[package]] diff --git a/crates/examples/Cargo.toml b/crates/examples/Cargo.toml index cb15de80d8..c7874d9a17 100644 --- a/crates/examples/Cargo.toml +++ b/crates/examples/Cargo.toml @@ -29,7 +29,6 @@ futures = { workspace = true } iceberg = { workspace = true } iceberg-catalog-rest = { workspace = true } tokio = { workspace = true, features = ["full"] } -uuid = { workspace = true } [[example]] name = "rest-catalog-namespace" diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index 72e747f608..ee5661b8dd 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -182,7 +182,11 @@ impl SnapshotProduceOperation for FastAppendOperation { &self, snapshot_produce: &mut SnapshotProducer<'_>, ) -> Result> { - let Some(snapshot) = snapshot_produce.table.metadata().current_snapshot() else { + let Some(snapshot) = snapshot_produce + .table + .metadata() + .snapshot_for_ref(snapshot_produce.target_branch()) + else { return Ok(vec![]); }; diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 552d43f183..c5a725e574 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -433,7 +433,12 @@ impl<'a> SnapshotProducer<'a> { // TODO: Allowing snapshot property setup with no added data files is a workaround. // We should clean it up after all necessary actions are supported. // For details, please refer to https://github.com/apache/iceberg-rust/issues/1548 - if self.added_data_files.is_empty() && self.snapshot_properties.is_empty() { + if self.added_data_files.is_empty() + && self.snapshot_properties.is_empty() + && self.added_delete_files.is_empty() + && self.removed_data_file_paths.is_empty() + && self.removed_delete_file_paths.is_empty() + { return Err(Error::new( ErrorKind::PreconditionFailed, "No added data files or added snapshot properties found when write a manifest file", diff --git a/crates/iceberg/src/writer/base_writer/mod.rs b/crates/iceberg/src/writer/base_writer/mod.rs index 37ab97eb6d..a5f2adc676 100644 --- a/crates/iceberg/src/writer/base_writer/mod.rs +++ b/crates/iceberg/src/writer/base_writer/mod.rs @@ -19,3 +19,4 @@ pub mod data_file_writer; pub mod equality_delete_writer; +pub mod position_delete_file_writer; diff --git a/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs b/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs new file mode 100644 index 0000000000..7db8e71497 --- /dev/null +++ b/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs @@ -0,0 +1,380 @@ +// 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. + +//! Position delete file writer. +//! +//! This writer assumes that incoming delete records are ordered by the tuple `(file_path, pos)` +//! as required by the Iceberg specification. Ordering and deduplication must be handled by the +//! caller (e.g. by using a sorting writer) before passing records to this writer. + +use std::sync::Arc; + +use arrow_array::RecordBatch; +use arrow_array::builder::{PrimitiveBuilder, StringBuilder}; +use arrow_array::types::Int64Type; +use arrow_schema::SchemaRef as ArrowSchemaRef; +use once_cell::sync::Lazy; + +use crate::arrow::schema_to_arrow_schema; +use crate::spec::{ + DataContentType, DataFile, NestedField, NestedFieldRef, PartitionKey, PrimitiveType, Schema, + Struct, Type, +}; +use crate::writer::file_writer::FileWriterBuilder; +use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; +use crate::writer::file_writer::rolling_writer::{RollingFileWriter, RollingFileWriterBuilder}; +use crate::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; + +static DELETE_FILE_PATH: Lazy = Lazy::new(|| { + Arc::new(NestedField::required( + 2147483546, + "file_path", + Type::Primitive(PrimitiveType::String), + )) +}); +static DELETE_FILE_POS: Lazy = Lazy::new(|| { + Arc::new(NestedField::required( + 2147483545, + "pos", + Type::Primitive(PrimitiveType::Long), + )) +}); +static POSITION_DELETE_SCHEMA: Lazy = Lazy::new(|| { + Schema::builder() + .with_fields(vec![DELETE_FILE_PATH.clone(), DELETE_FILE_POS.clone()]) + .build() + .unwrap() +}); +static POSITION_DELETE_ARROW_SCHEMA: Lazy = Lazy::new(|| { + Arc::new( + schema_to_arrow_schema(&POSITION_DELETE_SCHEMA) + .expect("position delete schema should always convert to arrow schema"), + ) +}); + +/// Position delete input. +#[derive(Clone, PartialEq, Eq, Ord, PartialOrd, Debug)] +pub struct PositionDeleteInput { + /// The path of the file. + pub path: Arc, + /// The row number in data file + pub pos: i64, +} + +impl PositionDeleteInput { + /// Create a new `PositionDeleteInput`. + pub fn new(path: Arc, row: i64) -> Self { + Self { path, pos: row } + } +} +/// Builder for `PositionDeleteFileWriter`. +#[derive(Clone, Debug)] +pub struct PositionDeleteFileWriterBuilder< + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +> { + inner: RollingFileWriterBuilder, +} + +impl PositionDeleteFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +{ + /// Create a new builder using an underlying `RollingFileWriterBuilder`. + pub fn new(inner: RollingFileWriterBuilder) -> Self { + Self { inner } + } +} + +#[async_trait::async_trait] +impl IcebergWriterBuilder> + for PositionDeleteFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +{ + type R = PositionDeleteFileWriter; + + async fn build(self, partition_key: Option) -> Result { + Ok(PositionDeleteFileWriter { + inner: Some(self.inner.clone().build()), + partition_key, + }) + } +} + +/// Position delete writer. +pub struct PositionDeleteFileWriter< + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +> { + inner: Option>, + partition_key: Option, +} + +#[async_trait::async_trait] +impl IcebergWriter> for PositionDeleteFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +{ + async fn write(&mut self, input: Vec) -> Result<()> { + if input.is_empty() { + return Ok(()); + } + + let batch = build_position_delete_batch(input)?; + + if let Some(writer) = self.inner.as_mut() { + writer.write(&self.partition_key, &batch).await + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Position delete writer has been closed.", + )) + } + } + + async fn close(&mut self) -> Result> { + if let Some(writer) = self.inner.take() { + writer + .close() + .await? + .into_iter() + .map(|mut builder| { + builder.content(DataContentType::PositionDeletes); + if let Some(pk) = self.partition_key.as_ref() { + builder.partition(pk.data().clone()); + builder.partition_spec_id(pk.spec().spec_id()); + } else { + builder.partition(Struct::empty()); + builder.partition_spec_id(0); + } + builder.build().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to build position delete file: {e}"), + ) + }) + }) + .collect() + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Position delete writer has been closed.", + )) + } + } +} + +impl CurrentFileStatus for PositionDeleteFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +{ + fn current_file_path(&self) -> String { + self.inner.as_ref().unwrap().current_file_path() + } + + fn current_row_num(&self) -> usize { + self.inner.as_ref().unwrap().current_row_num() + } + + fn current_written_size(&self) -> usize { + self.inner.as_ref().unwrap().current_written_size() + } +} + +fn build_position_delete_batch(input: Vec) -> Result { + let mut path_column_builder = StringBuilder::new(); + let mut offset_column_builder = PrimitiveBuilder::::new(); + + for pd_input in input { + path_column_builder.append_value(pd_input.path.as_ref()); + offset_column_builder.append_value(pd_input.pos); + } + + RecordBatch::try_new(Arc::clone(&POSITION_DELETE_ARROW_SCHEMA), vec![ + Arc::new(path_column_builder.finish()), + Arc::new(offset_column_builder.finish()), + ]) + .map_err(|e| Error::new(ErrorKind::DataInvalid, e.to_string())) +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use arrow_array::{Int64Array, StringArray}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + use parquet::file::properties::WriterProperties; + use tempfile::TempDir; + + use super::*; + use crate::io::FileIOBuilder; + use crate::spec::{ + DataFileFormat, Literal, NestedField, PartitionKey, PartitionSpec, PrimitiveType, Schema, + Struct, Type, + }; + use crate::writer::file_writer::ParquetWriterBuilder; + use crate::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, + }; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; + use crate::writer::tests::check_parquet_data_file; + use crate::writer::{IcebergWriter, IcebergWriterBuilder}; + + #[tokio::test] + async fn test_position_delete_writer_unpartitioned() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("pos_del".to_string(), None, DataFileFormat::Parquet); + + let schema = Arc::new(POSITION_DELETE_SCHEMA.clone()); + let parquet_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()); + + let rolling_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + let mut writer = PositionDeleteFileWriterBuilder::new(rolling_builder) + .build(None) + .await?; + + let deletes = vec![ + PositionDeleteInput::new(Arc::from("s3://bucket/data/file-1.parquet"), 1), + PositionDeleteInput::new(Arc::from("s3://bucket/data/file-1.parquet"), 2), + PositionDeleteInput::new(Arc::from("s3://bucket/data/file-2.parquet"), 5), + ]; + + let expected_batch = RecordBatch::try_new( + Arc::new(arrow_schema::Schema::new(vec![ + arrow_schema::Field::new("file_path", arrow_schema::DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + DELETE_FILE_PATH.id.to_string(), + )])), + arrow_schema::Field::new("pos", arrow_schema::DataType::Int64, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + DELETE_FILE_POS.id.to_string(), + )])), + ])), + vec![ + Arc::new(StringArray::from(vec![ + "s3://bucket/data/file-1.parquet", + "s3://bucket/data/file-1.parquet", + "s3://bucket/data/file-2.parquet", + ])), + Arc::new(Int64Array::from(vec![1, 2, 5])), + ], + )?; + + writer.write(deletes).await?; + let data_files = writer.close().await?; + + assert_eq!(data_files.len(), 1); + assert_eq!( + data_files[0].content_type(), + DataContentType::PositionDeletes + ); + assert_eq!(data_files[0].partition(), &Struct::empty()); + + check_parquet_data_file(&file_io, &data_files[0], &expected_batch).await; + + Ok(()) + } + + #[tokio::test] + async fn test_position_delete_writer_partitioned() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = DefaultFileNameGenerator::new( + "pos_del_part".to_string(), + None, + DataFileFormat::Parquet, + ); + + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(10, "region", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build()?; + let schema_ref = Arc::new(schema); + + let parquet_builder = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(POSITION_DELETE_SCHEMA.clone()), + ); + + let rolling_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + let partition_value = Struct::from_iter([Some(Literal::string("us"))]); + let partition_key = PartitionKey::new( + PartitionSpec::builder(schema_ref.clone()).build()?, + schema_ref, + partition_value.clone(), + ); + + let mut writer = PositionDeleteFileWriterBuilder::new(rolling_builder) + .build(Some(partition_key)) + .await?; + + writer + .write(vec![PositionDeleteInput::new( + Arc::from("s3://bucket/data/file.parquet"), + 10, + )]) + .await?; + + let data_files = writer.close().await?; + assert_eq!(data_files.len(), 1); + assert_eq!( + data_files[0].content_type(), + DataContentType::PositionDeletes + ); + assert_eq!(data_files[0].partition(), &partition_value); + + Ok(()) + } +} diff --git a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs index d6abb5c0b7..f004ab8e8f 100644 --- a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs @@ -234,11 +234,33 @@ async fn test_append_data_file_target_branch() { // Test 2: Append to a custom branch let branch_name = "test-branch"; + let second_location_generator = + DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let second_file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + Some(uuid::Uuid::now_v7().to_string()), + iceberg::spec::DataFileFormat::Parquet, + ); + let second_parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + ); + let second_rolling_writer = RollingFileWriterBuilder::new_with_default_file_size( + second_parquet_writer_builder, + table.file_io().clone(), + second_location_generator, + second_file_name_generator, + ); + let second_writer_builder = DataFileWriterBuilder::new(second_rolling_writer); + let mut second_writer = second_writer_builder.build(None).await.unwrap(); + second_writer.write(batch.clone()).await.unwrap(); + let second_data_file = second_writer.close().await.unwrap(); + let tx = Transaction::new(&table); let append_action = tx .fast_append() .set_target_branch(branch_name.to_string()) - .add_data_files(data_file.clone()); + .add_data_files(second_data_file.clone()); let tx = append_action.apply(tx).unwrap(); let table = tx.commit(&rest_catalog).await.unwrap(); diff --git a/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs b/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs index d0227969db..7dbbf4ece5 100644 --- a/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/overwrite_files_test.rs @@ -31,10 +31,11 @@ use iceberg::writer::file_writer::location_generator::{ }; use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; -use iceberg::{Catalog, CatalogBuilder, TableCreation}; +use iceberg::{Catalog, CatalogBuilder, ErrorKind, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::file::properties::WriterProperties; +use uuid::Uuid; use crate::get_shared_containers; use crate::shared_tests::{random_ns, test_schema}; @@ -47,7 +48,7 @@ fn create_data_file_writer_builder( let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); let file_name_generator = DefaultFileNameGenerator::new( "test".to_string(), - None, + Some(Uuid::now_v7().to_string()), iceberg::spec::DataFileFormat::Parquet, ); let parquet_writer_builder = ParquetWriterBuilder::new( @@ -230,7 +231,11 @@ async fn test_empty_overwrite() { let tx = Transaction::new(&table); let overwrite_action = tx.overwrite_files(); let tx = overwrite_action.apply(tx).unwrap(); - let table = tx.commit(&rest_catalog).await.unwrap(); + let err = tx + .commit(&rest_catalog) + .await + .expect_err("empty overwrite should fail"); + assert_eq!(err.kind(), ErrorKind::PreconditionFailed); let batch_stream = table .scan() @@ -589,13 +594,12 @@ async fn test_partition_spec_id_in_manifest() { // TODO: test update partition spec // Verify that the partition spec ID is correctly set - let last_snapshot = table.metadata().current_snapshot().unwrap(); let manifest_list = last_snapshot .load_manifest_list(table.file_io(), table.metadata()) .await .unwrap(); - assert_eq!(manifest_list.entries().len(), 1); + assert!(!manifest_list.entries().is_empty()); for manifest_file in manifest_list.entries() { assert_eq!(manifest_file.partition_spec_id, partition_spec_id); } diff --git a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs index b77db2d6e7..1bf6727603 100644 --- a/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs +++ b/crates/integration_tests/tests/shared_tests/rewrite_files_test.rs @@ -35,6 +35,7 @@ use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::file::properties::WriterProperties; +use uuid::Uuid; use crate::get_shared_containers; use crate::shared_tests::{random_ns, test_schema}; @@ -47,7 +48,7 @@ fn create_data_file_writer_builder( let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); let file_name_generator = DefaultFileNameGenerator::new( "test".to_string(), - None, + Some(Uuid::now_v7().to_string()), iceberg::spec::DataFileFormat::Parquet, ); let parquet_writer_builder = ParquetWriterBuilder::new( @@ -486,7 +487,7 @@ async fn test_partition_spec_id_in_manifest() { .load_manifest_list(table.file_io(), table.metadata()) .await .unwrap(); - assert_eq!(manifest_list.entries().len(), 1); + assert!(!manifest_list.entries().is_empty()); for manifest_file in manifest_list.entries() { assert_eq!(manifest_file.partition_spec_id, partition_spec_id); } From 702698416db1f5b1ab246d0b9ff69c12d9803054 Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 15 Dec 2025 15:14:32 +0800 Subject: [PATCH 26/33] feat: expose task writer & support delta writer (#97) * feat: expose task writer * feat: impl IcebergWriter * add new_with_partition_splitter for TaskWriter * support equality delta writer * fix CurrentFileStatus so that it can be called when uninitialized * fix task writer close * fmt * expose position delete schema and genreate snapshot id * better validate_partition_value error msg * rename equality_delta_writer to delta_writer --------- Co-authored-by: Dylan Chen --- Cargo.lock | 1 + Cargo.toml | 1 + crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/transaction/append.rs | 2 + crates/iceberg/src/transaction/mod.rs | 4 +- crates/iceberg/src/transaction/snapshot.rs | 52 +- .../writer/base_writer/data_file_writer.rs | 8 +- .../position_delete_file_writer.rs | 11 +- crates/iceberg/src/writer/delta_writer/mod.rs | 22 + .../iceberg/src/writer/delta_writer/writer.rs | 606 ++++++++++++++++++ crates/iceberg/src/writer/file_writer/mod.rs | 2 +- .../src/writer/file_writer/rolling_writer.rs | 8 +- crates/iceberg/src/writer/mod.rs | 4 + crates/iceberg/src/writer/task_writer.rs | 527 +++++++++++++++ crates/integrations/datafusion/src/lib.rs | 2 - .../datafusion/src/task_writer.rs | 517 +-------------- 16 files changed, 1235 insertions(+), 533 deletions(-) create mode 100644 crates/iceberg/src/writer/delta_writer/mod.rs create mode 100644 crates/iceberg/src/writer/delta_writer/writer.rs create mode 100644 crates/iceberg/src/writer/task_writer.rs diff --git a/Cargo.lock b/Cargo.lock index 911ddde8b8..b3b58bc0b7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3503,6 +3503,7 @@ dependencies = [ "arrow-buffer", "arrow-cast", "arrow-ord", + "arrow-row", "arrow-schema", "arrow-select", "arrow-string", diff --git a/Cargo.toml b/Cargo.toml index c10c01d94a..faeb866b38 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -47,6 +47,7 @@ arrow-array = "56.2" arrow-buffer = "56.2" arrow-cast = "56.2" arrow-ord = "56.2" +arrow-row = "56.2" arrow-schema = "56.2" arrow-select = "56.2" arrow-string = "56.2" diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index f0540e06e5..c130b5a6d0 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -52,6 +52,7 @@ arrow-array = { workspace = true } arrow-buffer = { workspace = true } arrow-cast = { workspace = true } arrow-ord = { workspace = true } +arrow-row = { workspace = true } arrow-schema = { workspace = true } arrow-select = { workspace = true } arrow-string = { workspace = true } diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index ee5661b8dd..58be7a080f 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -68,6 +68,7 @@ impl FastAppendAction { self } + /// Set target branch for the snapshot. pub fn set_target_branch(mut self, target_branch: String) -> Self { self.target_branch = Some(target_branch); self @@ -106,6 +107,7 @@ impl FastAppendAction { self } + /// Set snapshot id for the snapshot. pub fn set_snapshot_id(mut self, snapshot_id: i64) -> Self { self.snapshot_id = Some(snapshot_id); self diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 99271f1823..2a31c83e7c 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -54,6 +54,8 @@ mod action; pub use action::*; mod append; + +pub use append::FastAppendAction; mod manifest_filter; pub use manifest_filter::*; @@ -77,7 +79,7 @@ use rewrite_files::RewriteFilesAction; use crate::error::Result; use crate::spec::TableProperties; use crate::table::Table; -use crate::transaction::append::{FastAppendAction, MergeAppendAction}; +use crate::transaction::append::MergeAppendAction; use crate::transaction::overwrite_files::OverwriteFilesAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::transaction::update_location::UpdateLocationAction; diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index c5a725e574..9065497fc5 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -29,8 +29,8 @@ use crate::io::FileIO; use crate::spec::{ DataContentType, DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, ManifestEntry, ManifestFile, ManifestListWriter, ManifestStatus, ManifestWriter, - ManifestWriterBuilder, Operation, Snapshot, SnapshotReference, SnapshotRetention, - SnapshotSummaryCollector, Struct, StructType, Summary, TableProperties, + ManifestWriterBuilder, Operation, PrimitiveLiteral, Snapshot, SnapshotReference, + SnapshotRetention, SnapshotSummaryCollector, Struct, StructType, Summary, TableProperties, UNASSIGNED_SEQUENCE_NUMBER, update_snapshot_summaries, }; use crate::table::Table; @@ -277,14 +277,41 @@ impl<'a> SnapshotProducer<'a> { partition_value: &Struct, partition_type: &StructType, ) -> Result<()> { + fn literal_type_name(literal: &PrimitiveLiteral) -> &'static str { + match literal { + PrimitiveLiteral::Boolean(_) => "boolean", + PrimitiveLiteral::Int(_) => "int", + PrimitiveLiteral::Long(_) => "long", + PrimitiveLiteral::Float(_) => "float", + PrimitiveLiteral::Double(_) => "double", + PrimitiveLiteral::String(_) => "string", + PrimitiveLiteral::Binary(_) => "binary", + PrimitiveLiteral::Int128(_) => "decimal", + PrimitiveLiteral::UInt128(_) => "uuid", + PrimitiveLiteral::AboveMax => "above_max", + PrimitiveLiteral::BelowMin => "below_min", + } + } + if partition_value.fields().len() != partition_type.fields().len() { return Err(Error::new( ErrorKind::DataInvalid, - "Partition value is not compatible with partition type", + format!( + "Partition struct field count mismatch. struct_value: {:?} ({} fields), partition_type: {:?} ({} fields)", + partition_value, + partition_value.fields().len(), + partition_type, + partition_type.fields().len() + ), )); } - for (value, field) in partition_value.fields().iter().zip(partition_type.fields()) { + for (idx, (value, field)) in partition_value + .fields() + .iter() + .zip(partition_type.fields()) + .enumerate() + { let field = field.field_type.as_primitive_type().ok_or_else(|| { Error::new( ErrorKind::Unexpected, @@ -292,10 +319,20 @@ impl<'a> SnapshotProducer<'a> { ) })?; if let Some(value) = value { - if !field.compatible(&value.as_primitive_literal().unwrap()) { + let literal = value.as_primitive_literal().unwrap(); + if !field.compatible(&literal) { return Err(Error::new( ErrorKind::DataInvalid, - "Partition value is not compatible partition type", + format!( + "Partition field value {:?} (type {}) is not compatible with partition field type {} at index {}. \ +partition_struct: {:?}, partition_type: {:?}", + literal, + literal_type_name(&literal), + field, + idx, + partition_value, + partition_type + ), )); } } @@ -708,14 +745,17 @@ impl<'a> SnapshotProducer<'a> { Ok(ActionCommit::new(updates, requirements)) } + /// Set the new data file sequence number for this snapshot pub fn set_new_data_file_sequence_number(&mut self, new_data_file_sequence_number: i64) { self.new_data_file_sequence_number = Some(new_data_file_sequence_number); } + /// Set the target branch for this snapshot pub fn set_target_branch(&mut self, target_branch: String) { self.target_branch = target_branch; } + /// Get the target branch for this snapshot pub fn target_branch(&self) -> &str { &self.target_branch } diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index dcaa56cc97..46b2c9f442 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -126,11 +126,15 @@ where } fn current_row_num(&self) -> usize { - self.inner.as_ref().unwrap().current_row_num() + self.inner + .as_ref() + .map_or(0, |inner| inner.current_row_num()) } fn current_written_size(&self) -> usize { - self.inner.as_ref().unwrap().current_written_size() + self.inner + .as_ref() + .map_or(0, |inner| inner.current_written_size()) } } diff --git a/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs b/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs index 7db8e71497..153c1a7940 100644 --- a/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/position_delete_file_writer.rs @@ -54,7 +54,8 @@ static DELETE_FILE_POS: Lazy = Lazy::new(|| { Type::Primitive(PrimitiveType::Long), )) }); -static POSITION_DELETE_SCHEMA: Lazy = Lazy::new(|| { +/// Iceberg schema used for position delete files (file_path, pos). +pub static POSITION_DELETE_SCHEMA: Lazy = Lazy::new(|| { Schema::builder() .with_fields(vec![DELETE_FILE_PATH.clone(), DELETE_FILE_POS.clone()]) .build() @@ -199,11 +200,15 @@ where } fn current_row_num(&self) -> usize { - self.inner.as_ref().unwrap().current_row_num() + self.inner + .as_ref() + .map_or(0, |inner| inner.current_row_num()) } fn current_written_size(&self) -> usize { - self.inner.as_ref().unwrap().current_written_size() + self.inner + .as_ref() + .map_or(0, |inner| inner.current_written_size()) } } diff --git a/crates/iceberg/src/writer/delta_writer/mod.rs b/crates/iceberg/src/writer/delta_writer/mod.rs new file mode 100644 index 0000000000..376e4ff63d --- /dev/null +++ b/crates/iceberg/src/writer/delta_writer/mod.rs @@ -0,0 +1,22 @@ +// 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. + +//! Higher-level delta writers built on top of base Iceberg writers. + +pub mod writer; + +pub use writer::{DELETE_OP, DeltaWriter, DeltaWriterBuilder, INSERT_OP}; diff --git a/crates/iceberg/src/writer/delta_writer/writer.rs b/crates/iceberg/src/writer/delta_writer/writer.rs new file mode 100644 index 0000000000..9fdab7a956 --- /dev/null +++ b/crates/iceberg/src/writer/delta_writer/writer.rs @@ -0,0 +1,606 @@ +// 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. + +//! Delta writer that produces data files, position delete files and equality delete files +//! in a single pass. + +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::builder::BooleanBuilder; +use arrow_array::{Array, Int32Array, RecordBatch}; +use arrow_ord::partition::partition; +use arrow_row::{OwnedRow, RowConverter, Rows, SortField}; +use arrow_select::filter::filter_record_batch; +use itertools::Itertools; +use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + +use crate::arrow::record_batch_projector::RecordBatchProjector; +use crate::arrow::schema_to_arrow_schema; +use crate::spec::{DataFile, PartitionKey, SchemaRef}; +use crate::writer::base_writer::position_delete_file_writer::PositionDeleteInput; +use crate::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; + +/// Insert operation marker. +pub const INSERT_OP: i32 = 1; +/// Delete operation marker. +pub const DELETE_OP: i32 = 2; + +/// Builder for [`DeltaWriter`]. +#[derive(Clone)] +pub struct DeltaWriterBuilder { + data_writer_builder: DB, + position_delete_writer_builder: PDB, + equality_delete_writer_builder: EDB, + unique_column_ids: Vec, + schema: SchemaRef, +} + +impl DeltaWriterBuilder { + /// Create a new `DeltaWriterBuilder`. + pub fn new( + data_writer_builder: DB, + position_delete_writer_builder: PDB, + equality_delete_writer_builder: EDB, + unique_column_ids: Vec, + schema: SchemaRef, + ) -> Self { + Self { + data_writer_builder, + position_delete_writer_builder, + equality_delete_writer_builder, + unique_column_ids, + schema, + } + } +} + +#[async_trait::async_trait] +impl IcebergWriterBuilder for DeltaWriterBuilder +where + DB: IcebergWriterBuilder, + PDB: IcebergWriterBuilder>, + EDB: IcebergWriterBuilder, + DB::R: CurrentFileStatus, +{ + type R = DeltaWriter; + + async fn build(self, partition_key: Option) -> Result { + Self::R::try_new( + self.data_writer_builder + .build(partition_key.clone()) + .await?, + self.position_delete_writer_builder + .build(partition_key.clone()) + .await?, + self.equality_delete_writer_builder + .build(partition_key) + .await?, + self.schema, + self.unique_column_ids, + ) + } +} + +/// Writer that handles insert and delete operations in a single stream. +pub struct DeltaWriter { + data_writer: D, + position_delete_writer: PD, + equality_delete_writer: ED, + projector: RecordBatchProjector, + inserted_row: HashMap, + row_converter: RowConverter, +} + +impl DeltaWriter +where + D: IcebergWriter + CurrentFileStatus, + PD: IcebergWriter>, + ED: IcebergWriter, +{ + pub(crate) fn try_new( + data_writer: D, + position_delete_writer: PD, + equality_delete_writer: ED, + schema: SchemaRef, + unique_column_ids: Vec, + ) -> Result { + let projector = RecordBatchProjector::new( + Arc::new(schema_to_arrow_schema(&schema)?), + &unique_column_ids, + |field| { + if field.data_type().is_nested() { + return Ok(None); + } + field + .metadata() + .get(PARQUET_FIELD_ID_META_KEY) + .map(|s| { + s.parse::() + .map_err(|e| Error::new(ErrorKind::Unexpected, e.to_string())) + }) + .transpose() + }, + |_| true, + )?; + let row_converter = RowConverter::new( + projector + .projected_schema_ref() + .fields() + .iter() + .map(|field| SortField::new(field.data_type().clone())) + .collect(), + )?; + Ok(Self { + data_writer, + position_delete_writer, + equality_delete_writer, + projector, + inserted_row: HashMap::new(), + row_converter, + }) + } + + async fn insert(&mut self, batch: RecordBatch) -> Result<()> { + let rows = self.extract_unique_column(&batch)?; + let row_count = batch.num_rows(); + + // Initialise the writer by writing the batch first; offsets are derived from the end state. + self.data_writer.write(batch).await?; + + let end_offset = self.data_writer.current_row_num(); + let start_offset = end_offset.saturating_sub(row_count); + let current_file_path: Arc = Arc::from(self.data_writer.current_file_path()); + + let mut position_deletes = Vec::new(); + for (idx, row) in rows.iter().enumerate() { + let previous_input = self.inserted_row.insert( + row.owned(), + PositionDeleteInput::new(current_file_path.clone(), (start_offset + idx) as i64), + ); + if let Some(previous_input) = previous_input { + position_deletes.push(previous_input); + } + } + + self.write_position_deletes(position_deletes).await + } + + async fn delete(&mut self, batch: RecordBatch) -> Result<()> { + let rows = self.extract_unique_column(&batch)?; + let mut delete_row = BooleanBuilder::with_capacity(rows.num_rows()); + let mut position_deletes = Vec::new(); + for row in rows.iter() { + if let Some(previous_input) = self.inserted_row.remove(&row.owned()) { + position_deletes.push(previous_input); + delete_row.append_value(false); + } else { + delete_row.append_value(true); + } + } + + self.write_position_deletes(position_deletes).await?; + + let delete_mask = delete_row.finish(); + if delete_mask.null_count() == delete_mask.len() { + return Ok(()); + } + + let delete_batch = filter_record_batch(&batch, &delete_mask).map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to filter record batch, error: {err}"), + ) + })?; + if delete_batch.num_rows() == 0 { + return Ok(()); + } + self.equality_delete_writer.write(delete_batch).await + } + + fn extract_unique_column(&mut self, batch: &RecordBatch) -> Result { + self.row_converter + .convert_columns(&self.projector.project_column(batch.columns())?) + .map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to convert columns, error: {err}"), + ) + }) + } + + async fn write_position_deletes( + &mut self, + mut deletes: Vec, + ) -> Result<()> { + if deletes.is_empty() { + return Ok(()); + } + deletes.sort_by(|a, b| { + let path_cmp = a.path.as_ref().cmp(b.path.as_ref()); + if path_cmp == std::cmp::Ordering::Equal { + a.pos.cmp(&b.pos) + } else { + path_cmp + } + }); + self.position_delete_writer.write(deletes).await + } +} + +#[async_trait::async_trait] +impl IcebergWriter for DeltaWriter +where + D: IcebergWriter + CurrentFileStatus, + PD: IcebergWriter>, + ED: IcebergWriter, +{ + async fn write(&mut self, batch: RecordBatch) -> Result<()> { + if batch.num_columns() == 0 { + return Err(Error::new( + ErrorKind::DataInvalid, + "Delta writer expects at least one column for operation markers", + )); + } + + let ops = batch + .column(batch.num_columns() - 1) + .as_any() + .downcast_ref::() + .ok_or(Error::new( + ErrorKind::DataInvalid, + "The last column must be an Int32Array of operation markers", + ))?; + + let partitions = + partition(&[batch.column(batch.num_columns() - 1).clone()]).map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to partition ops, error: {err}"), + ) + })?; + for range in partitions.ranges() { + let batch = batch + .project(&(0..batch.num_columns() - 1).collect_vec()) + .unwrap() + .slice(range.start, range.end - range.start); + match ops.value(range.start) { + INSERT_OP => self.insert(batch).await?, + DELETE_OP => self.delete(batch).await?, + op => { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid ops: {op}"), + )); + } + } + } + Ok(()) + } + + async fn close(&mut self) -> Result> { + let data_files = self.data_writer.close().await?; + let position_delete_files = self.position_delete_writer.close().await?; + let equality_delete_files = self.equality_delete_writer.close().await?; + Ok(data_files + .into_iter() + .chain(position_delete_files) + .chain(equality_delete_files) + .collect()) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use arrow_array::{Int32Array, Int64Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + use arrow_select::concat::concat_batches; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; + use parquet::file::properties::WriterProperties; + use tempfile::TempDir; + + use crate::Result; + use crate::arrow::{arrow_schema_to_schema, schema_to_arrow_schema}; + use crate::io::FileIOBuilder; + use crate::spec::{DataContentType, DataFileFormat, NestedField, PrimitiveType, Schema, Type}; + use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; + use crate::writer::base_writer::equality_delete_writer::{ + EqualityDeleteFileWriterBuilder, EqualityDeleteWriterConfig, + }; + use crate::writer::base_writer::position_delete_file_writer::PositionDeleteFileWriterBuilder; + use crate::writer::delta_writer::{DELETE_OP, DeltaWriterBuilder, INSERT_OP}; + use crate::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, + }; + use crate::writer::file_writer::parquet_writer::ParquetWriterBuilder; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; + use crate::writer::{IcebergWriter, IcebergWriterBuilder}; + + type WriterBuildersResult = ( + DataFileWriterBuilder< + ParquetWriterBuilder, + DefaultLocationGenerator, + DefaultFileNameGenerator, + >, + PositionDeleteFileWriterBuilder< + ParquetWriterBuilder, + DefaultLocationGenerator, + DefaultFileNameGenerator, + >, + EqualityDeleteFileWriterBuilder< + ParquetWriterBuilder, + DefaultLocationGenerator, + DefaultFileNameGenerator, + >, + EqualityDeleteWriterConfig, + ); + + fn position_delete_arrow_schema() -> ArrowSchema { + schema_to_arrow_schema( + &Schema::builder() + .with_fields(vec![ + NestedField::required( + 2147483546, + "file_path", + Type::Primitive(PrimitiveType::String), + ) + .into(), + NestedField::required(2147483545, "pos", Type::Primitive(PrimitiveType::Long)) + .into(), + ]) + .build() + .unwrap(), + ) + .unwrap() + } + + fn create_writer_builders( + data_schema: Arc, + file_io: &crate::io::FileIO, + location_gen: DefaultLocationGenerator, + file_name_gen: DefaultFileNameGenerator, + ) -> Result { + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), data_schema.clone()); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen.clone(), + file_name_gen.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + let position_delete_parquet_builder = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new( + Schema::builder() + .with_fields(vec![ + NestedField::required( + 2147483546, + "file_path", + Type::Primitive(PrimitiveType::String), + ) + .into(), + NestedField::required( + 2147483545, + "pos", + Type::Primitive(PrimitiveType::Long), + ) + .into(), + ]) + .build() + .unwrap(), + ), + ); + let position_delete_writer_builder = PositionDeleteFileWriterBuilder::new( + RollingFileWriterBuilder::new_with_default_file_size( + position_delete_parquet_builder, + file_io.clone(), + location_gen.clone(), + file_name_gen.clone(), + ), + ); + + let equality_config = EqualityDeleteWriterConfig::new(vec![1, 2], data_schema.clone())?; + let equality_delete_writer_builder = { + let schema = + arrow_schema_to_schema(equality_config.projected_arrow_schema_ref())?.into(); + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + EqualityDeleteFileWriterBuilder::new(rolling_writer_builder, equality_config.clone()) + }; + + Ok(( + data_file_writer_builder, + position_delete_writer_builder, + equality_delete_writer_builder, + equality_config, + )) + } + + #[tokio::test] + async fn test_delta_writer() -> Result<()> { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id".to_string(), + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 2, + "name".to_string(), + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(), + ); + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = + DefaultLocationGenerator::with_data_location(temp_dir.path().to_string_lossy().into()); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + + let ( + data_file_writer_builder, + position_delete_writer_builder, + equality_delete_writer_builder, + equality_config, + ) = create_writer_builders( + schema.clone(), + &file_io, + location_gen.clone(), + file_name_gen.clone(), + )?; + + let mut delta_writer = DeltaWriterBuilder::new( + data_file_writer_builder, + position_delete_writer_builder, + equality_delete_writer_builder, + vec![1, 2], + schema.clone(), + ) + .build(None) + .await?; + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int64, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("data", DataType::Utf8, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + Field::new("op", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + ])); + + let batch_one = RecordBatch::try_new(arrow_schema.clone(), vec![ + Arc::new(Int64Array::from(vec![1, 2, 1, 3, 2, 3, 1])), + Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e", "f", "g"])), + Arc::new(Int32Array::from(vec![INSERT_OP; 7])), + ])?; + delta_writer.write(batch_one).await?; + + let batch_two = RecordBatch::try_new(arrow_schema.clone(), vec![ + Arc::new(Int64Array::from(vec![1, 2, 3, 4])), + Arc::new(StringArray::from(vec!["a", "b", "k", "l"])), + Arc::new(Int32Array::from(vec![ + DELETE_OP, DELETE_OP, DELETE_OP, INSERT_OP, + ])), + ])?; + delta_writer.write(batch_two).await?; + + let data_files = delta_writer.close().await?; + assert_eq!(data_files.len(), 3); + + let data_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int64, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("data", DataType::Utf8, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + ])); + + let data_file = data_files + .iter() + .find(|file| file.content == DataContentType::Data) + .unwrap(); + let data_file_path = data_file.file_path().to_string(); + let input_content = file_io.new_input(data_file_path.clone())?.read().await?; + let reader = ParquetRecordBatchReaderBuilder::try_new(input_content)? + .build() + .unwrap(); + let batches = reader.map(|batch| batch.unwrap()).collect::>(); + let res = concat_batches(&data_schema, &batches).unwrap(); + let expected_batches = RecordBatch::try_new(data_schema.clone(), vec![ + Arc::new(Int64Array::from(vec![1, 2, 1, 3, 2, 3, 1, 4])), + Arc::new(StringArray::from(vec![ + "a", "b", "c", "d", "e", "f", "g", "l", + ])), + ])?; + assert_eq!(expected_batches, res); + + let position_delete_file = data_files + .iter() + .find(|file| file.content == DataContentType::PositionDeletes) + .unwrap(); + let position_input = file_io + .new_input(position_delete_file.file_path.clone())? + .read() + .await?; + let reader = ParquetRecordBatchReaderBuilder::try_new(position_input)? + .build() + .unwrap(); + let batches = reader.map(|batch| batch.unwrap()).collect::>(); + let position_schema = Arc::new(position_delete_arrow_schema()); + let res = concat_batches(&position_schema, &batches).unwrap(); + let expected_batches = RecordBatch::try_new(position_schema.clone(), vec![ + Arc::new(StringArray::from(vec![ + data_file_path.clone(), + data_file_path.clone(), + ])), + Arc::new(Int64Array::from(vec![0, 1])), + ])?; + assert_eq!(expected_batches, res); + + let equality_delete_file = data_files + .iter() + .find(|file| file.content == DataContentType::EqualityDeletes) + .unwrap(); + let equality_input = file_io + .new_input(equality_delete_file.file_path.clone())? + .read() + .await?; + let reader = ParquetRecordBatchReaderBuilder::try_new(equality_input)? + .build() + .unwrap(); + let batches = reader.map(|batch| batch.unwrap()).collect::>(); + let equality_schema = Arc::new(arrow_schema_to_schema( + equality_config.projected_arrow_schema_ref(), + )?); + let equality_arrow_schema = Arc::new(schema_to_arrow_schema(&equality_schema)?); + let res = concat_batches(&equality_arrow_schema, &batches).unwrap(); + let expected_batches = RecordBatch::try_new(equality_arrow_schema.clone(), vec![ + Arc::new(Int64Array::from(vec![3])), + Arc::new(StringArray::from(vec!["k"])), + ])?; + assert_eq!(expected_batches, res); + + Ok(()) + } +} diff --git a/crates/iceberg/src/writer/file_writer/mod.rs b/crates/iceberg/src/writer/file_writer/mod.rs index 2ed6414ce8..cb2f52179d 100644 --- a/crates/iceberg/src/writer/file_writer/mod.rs +++ b/crates/iceberg/src/writer/file_writer/mod.rs @@ -24,7 +24,7 @@ use super::CurrentFileStatus; use crate::Result; use crate::spec::DataFileBuilder; -mod parquet_writer; +pub mod parquet_writer; pub use parquet_writer::{ParquetWriter, ParquetWriterBuilder}; use crate::io::OutputFile; diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index 8f03654786..3a950bf640 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -249,11 +249,15 @@ impl CurrentFi } fn current_row_num(&self) -> usize { - self.inner.as_ref().unwrap().current_row_num() + self.inner + .as_ref() + .map_or(0, |inner| inner.current_row_num()) } fn current_written_size(&self) -> usize { - self.inner.as_ref().unwrap().current_written_size() + self.inner + .as_ref() + .map_or(0, |inner| inner.current_written_size()) } } diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs index a7892d49e1..1d2f5e8d4d 100644 --- a/crates/iceberg/src/writer/mod.rs +++ b/crates/iceberg/src/writer/mod.rs @@ -385,8 +385,10 @@ //! ``` pub mod base_writer; +pub mod delta_writer; pub mod file_writer; pub mod partitioning; +pub mod task_writer; use arrow_array::RecordBatch; @@ -419,6 +421,8 @@ pub trait IcebergWriter: Send + 'static { async fn close(&mut self) -> Result; } +pub use task_writer::TaskWriter; + /// The current file status of the Iceberg writer. /// This is implemented for writers that write a single file at a time. pub trait CurrentFileStatus { diff --git a/crates/iceberg/src/writer/task_writer.rs b/crates/iceberg/src/writer/task_writer.rs new file mode 100644 index 0000000000..1bb527192e --- /dev/null +++ b/crates/iceberg/src/writer/task_writer.rs @@ -0,0 +1,527 @@ +// 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. + +//! High-level Iceberg writer that coordinates partition routing for `RecordBatch` input. +//! +//! `TaskWriter` sits on top of the generic writer abstractions and provides a convenient entry +//! point for users that start from Arrow `RecordBatch` values. It lazily constructs the +//! appropriate partitioning writer (unpartitioned, fanout, or clustered) and routes batches to it. +//! +//! # Example +//! +//! ```rust,ignore +//! use iceberg::spec::{PartitionSpec, Schema}; +//! use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +//! use iceberg::writer::file_writer::ParquetWriterBuilder; +//! use iceberg::writer::file_writer::location_generator::{ +//! DefaultFileNameGenerator, DefaultLocationGenerator, +//! }; +//! use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +//! use iceberg::writer::task_writer::TaskWriter; +//! use parquet::file::properties::WriterProperties; +//! +//! # async fn build_task_writer( +//! # data_file_writer_builder: +//! # DataFileWriterBuilder, +//! # schema: Schema, +//! # partition_spec: PartitionSpec, +//! # ) -> iceberg::Result<()> { +//! let mut task_writer = TaskWriter::new( +//! data_file_writer_builder, +//! false, // fanout_enabled +//! Schema::from(schema).into(), +//! PartitionSpec::from(partition_spec).into(), +//! ); +//! +//! // task_writer.write(record_batch).await?; +//! // let data_files = task_writer.close().await?; +//! # Ok(()) +//! # } +//! ``` + +use arrow_array::RecordBatch; + +use crate::Result; +use crate::arrow::RecordBatchPartitionSplitter; +use crate::spec::{DataFile, PartitionSpecRef, SchemaRef}; +use crate::writer::partitioning::PartitioningWriter; +use crate::writer::partitioning::clustered_writer::ClusteredWriter; +use crate::writer::partitioning::fanout_writer::FanoutWriter; +use crate::writer::partitioning::unpartitioned_writer::UnpartitionedWriter; +use crate::writer::{IcebergWriter, IcebergWriterBuilder}; + +/// High-level writer that handles partitioning and routing of `RecordBatch` data to Iceberg tables. +pub struct TaskWriter { + /// The underlying writer (unpartitioned, fanout, or clustered) + writer: Option>, + /// Lazily initialized partition splitter for partitioned tables + partition_splitter: Option, + /// Iceberg schema reference used for partition splitting + schema: SchemaRef, + /// Partition specification reference used for partition splitting + partition_spec: PartitionSpecRef, +} + +/// Internal enum holding the writer implementation for each partitioning strategy. +enum SupportedWriter { + /// Writer for unpartitioned tables + Unpartitioned(UnpartitionedWriter), + /// Writer for partitioned tables with unsorted data (maintains multiple active writers) + Fanout(FanoutWriter), + /// Writer for partitioned tables with sorted data (maintains a single active writer) + Clustered(ClusteredWriter), +} + +impl TaskWriter { + /// Create a new `TaskWriter`. + /// + /// * `writer_builder` - The writer builder used to create underlying writers + /// * `fanout_enabled` - Controls whether `FanoutWriter` is used for partitioned tables; when + /// `false` the `ClusteredWriter` is selected instead + /// * `schema` - The Iceberg schema reference for the incoming `RecordBatch` + /// * `partition_spec` - The partition specification reference for the target table + pub fn new( + writer_builder: B, + fanout_enabled: bool, + schema: SchemaRef, + partition_spec: PartitionSpecRef, + ) -> Self { + Self::new_with_partition_splitter( + writer_builder, + fanout_enabled, + schema, + partition_spec, + None, + ) + } + + /// Create a new `TaskWriter` with a pre-configured partition splitter. + /// + /// This allows callers to provide a custom [`RecordBatchPartitionSplitter`], enabling use cases + /// such as computing partition values at runtime rather than expecting a pre-computed + /// `_partition` column in incoming batches. + pub fn new_with_partition_splitter( + writer_builder: B, + fanout_enabled: bool, + schema: SchemaRef, + partition_spec: PartitionSpecRef, + partition_splitter: Option, + ) -> Self { + let writer = if partition_spec.is_unpartitioned() { + SupportedWriter::Unpartitioned(UnpartitionedWriter::new(writer_builder)) + } else if fanout_enabled { + SupportedWriter::Fanout(FanoutWriter::new(writer_builder)) + } else { + SupportedWriter::Clustered(ClusteredWriter::new(writer_builder)) + }; + + Self { + writer: Some(writer), + partition_splitter, + schema, + partition_spec, + } + } + + /// Write a `RecordBatch` to the `TaskWriter`. + /// + /// For the first write against a partitioned table, the partition splitter is initialised + /// lazily. Unpartitioned tables bypass the splitter entirely. + pub async fn write(&mut self, batch: RecordBatch) -> Result<()> { + let writer = self.writer.as_mut().ok_or_else(|| { + crate::Error::new( + crate::ErrorKind::Unexpected, + "TaskWriter has been closed and cannot be used", + ) + })?; + + match writer { + SupportedWriter::Unpartitioned(writer) => writer.write(batch).await, + SupportedWriter::Fanout(writer) => { + if self.partition_splitter.is_none() { + self.partition_splitter = + Some(RecordBatchPartitionSplitter::new_with_precomputed_values( + self.schema.clone(), + self.partition_spec.clone(), + )?); + } + + Self::write_partitioned_batches(writer, &self.partition_splitter, &batch).await + } + SupportedWriter::Clustered(writer) => { + if self.partition_splitter.is_none() { + self.partition_splitter = + Some(RecordBatchPartitionSplitter::new_with_precomputed_values( + self.schema.clone(), + self.partition_spec.clone(), + )?); + } + + Self::write_partitioned_batches(writer, &self.partition_splitter, &batch).await + } + } + } + + /// Close the `TaskWriter` and return all written data files. + pub async fn close(self) -> Result> { + if let Some(writer) = self.writer { + match writer { + SupportedWriter::Unpartitioned(writer) => writer.close().await, + SupportedWriter::Fanout(writer) => writer.close().await, + SupportedWriter::Clustered(writer) => writer.close().await, + } + } else { + Err(crate::Error::new( + crate::ErrorKind::Unexpected, + "TaskWriter has already been closed", + )) + } + } + + async fn write_partitioned_batches( + writer: &mut W, + partition_splitter: &Option, + batch: &RecordBatch, + ) -> Result<()> { + let splitter = partition_splitter + .as_ref() + .expect("partition splitter must be initialised before use"); + let partitioned_batches = splitter.split(batch)?; + + for (partition_key, partition_batch) in partitioned_batches { + writer.write(partition_key, partition_batch).await?; + } + + Ok(()) + } +} + +#[async_trait::async_trait] +impl IcebergWriter for TaskWriter { + async fn write(&mut self, input: RecordBatch) -> Result<()> { + self.write(input).await + } + + async fn close(&mut self) -> Result> { + if let Some(writer) = self.writer.take() { + match writer { + SupportedWriter::Unpartitioned(writer) => writer.close().await, + SupportedWriter::Fanout(writer) => writer.close().await, + SupportedWriter::Clustered(writer) => writer.close().await, + } + } else { + Err(crate::Error::new( + crate::ErrorKind::Unexpected, + "TaskWriter has already been closed", + )) + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray, StructArray}; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + use parquet::file::properties::WriterProperties; + use tempfile::TempDir; + + use crate::Result; + use crate::arrow::{PROJECTED_PARTITION_VALUE_COLUMN, RecordBatchPartitionSplitter}; + use crate::io::FileIOBuilder; + use crate::spec::{ + DataFile, DataFileFormat, NestedField, PartitionSpec, PrimitiveLiteral, PrimitiveType, Type, + }; + use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; + use crate::writer::file_writer::ParquetWriterBuilder; + use crate::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, + }; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; + use crate::writer::task_writer::TaskWriter; + + fn create_test_schema() -> Result> { + Ok(Arc::new( + crate::spec::Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build()?, + )) + } + + fn create_arrow_schema() -> Arc { + Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + ])) + } + + fn create_arrow_schema_with_partition() -> Arc { + let partition_field = Field::new("region", DataType::Utf8, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), + ); + let partition_struct_field = Field::new( + PROJECTED_PARTITION_VALUE_COLUMN, + DataType::Struct(vec![partition_field.clone()].into()), + false, + ); + + Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + partition_struct_field, + ])) + } + + fn create_writer_builder( + temp_dir: &TempDir, + schema: Arc, + ) -> Result< + DataFileWriterBuilder< + ParquetWriterBuilder, + DefaultLocationGenerator, + DefaultFileNameGenerator, + >, + > { + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_string_lossy().into_owned(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema); + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io, + location_gen, + file_name_gen, + ); + Ok(DataFileWriterBuilder::new(rolling_writer_builder)) + } + + #[tokio::test] + async fn test_task_writer_unpartitioned() -> Result<()> { + let temp_dir = TempDir::new()?; + let schema = create_test_schema()?; + let arrow_schema = create_arrow_schema(); + + let partition_spec = Arc::new(PartitionSpec::builder(schema.clone()).build()?); + + let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; + let mut task_writer = TaskWriter::new(writer_builder, false, schema, partition_spec); + + let batch = RecordBatch::try_new(arrow_schema, vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef, + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])) as ArrayRef, + Arc::new(StringArray::from(vec!["US", "EU", "US"])) as ArrayRef, + ])?; + + task_writer.write(batch).await?; + let data_files = task_writer.close().await?; + + assert!(!data_files.is_empty()); + assert_eq!(data_files[0].record_count(), 3); + + Ok(()) + } + + fn verify_partition_files( + data_files: &[DataFile], + expected_total: u64, + ) -> HashMap { + let total_records: u64 = data_files.iter().map(|f| f.record_count()).sum(); + assert_eq!(total_records, expected_total, "total record count mismatch"); + + let mut partition_counts = HashMap::new(); + for data_file in data_files { + let partition_value = data_file.partition(); + let region_literal = partition_value.fields()[0] + .as_ref() + .expect("partition value should not be null"); + let region = match region_literal + .as_primitive_literal() + .expect("expected primitive literal") + { + PrimitiveLiteral::String(s) => s.clone(), + _ => panic!("expected string partition value"), + }; + + *partition_counts.entry(region.clone()).or_insert(0) += data_file.record_count(); + + assert!( + data_file.file_path().contains("region="), + "file path should encode partition info" + ); + } + partition_counts + } + + #[tokio::test] + async fn test_task_writer_partitioned_with_computed_partitions() -> Result<()> { + let temp_dir = TempDir::new()?; + let schema = create_test_schema()?; + let arrow_schema = create_arrow_schema(); + + let partition_spec = Arc::new( + PartitionSpec::builder(schema.clone()) + .with_spec_id(1) + .add_partition_field("region", "region", crate::spec::Transform::Identity)? + .build()?, + ); + let partition_splitter = RecordBatchPartitionSplitter::new_with_computed_values( + schema.clone(), + partition_spec.clone(), + )?; + + let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; + let mut task_writer = TaskWriter::new_with_partition_splitter( + writer_builder, + true, + schema, + partition_spec, + Some(partition_splitter), + ); + + let batch = RecordBatch::try_new(arrow_schema, vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as ArrayRef, + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])) as ArrayRef, + Arc::new(StringArray::from(vec!["US", "EU", "US", "EU"])) as ArrayRef, + ])?; + + task_writer.write(batch).await?; + let data_files = task_writer.close().await?; + + let partition_counts = verify_partition_files(&data_files, 4); + assert_eq!(partition_counts.get("US"), Some(&2)); + assert_eq!(partition_counts.get("EU"), Some(&2)); + + Ok(()) + } + + #[tokio::test] + async fn test_task_writer_partitioned_fanout() -> Result<()> { + let temp_dir = TempDir::new()?; + let schema = create_test_schema()?; + let arrow_schema = create_arrow_schema_with_partition(); + + let partition_spec = Arc::new( + PartitionSpec::builder(schema.clone()) + .with_spec_id(1) + .add_partition_field("region", "region", crate::spec::Transform::Identity)? + .build()?, + ); + + let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; + let mut task_writer = TaskWriter::new(writer_builder, true, schema, partition_spec); + + let partition_field = Field::new("region", DataType::Utf8, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), + ); + let partition_values = StringArray::from(vec!["US", "EU", "US", "EU"]); + let partition_struct = StructArray::from(vec![( + Arc::new(partition_field), + Arc::new(partition_values) as ArrayRef, + )]); + + let batch = RecordBatch::try_new(arrow_schema, vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as ArrayRef, + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])) as ArrayRef, + Arc::new(StringArray::from(vec!["US", "EU", "US", "EU"])) as ArrayRef, + Arc::new(partition_struct) as ArrayRef, + ])?; + + task_writer.write(batch).await?; + let data_files = task_writer.close().await?; + + let partition_counts = verify_partition_files(&data_files, 4); + assert_eq!(partition_counts.get("US"), Some(&2)); + assert_eq!(partition_counts.get("EU"), Some(&2)); + + Ok(()) + } + + #[tokio::test] + async fn test_task_writer_partitioned_clustered() -> Result<()> { + let temp_dir = TempDir::new()?; + let schema = create_test_schema()?; + let arrow_schema = create_arrow_schema_with_partition(); + + let partition_spec = Arc::new( + PartitionSpec::builder(schema.clone()) + .with_spec_id(1) + .add_partition_field("region", "region", crate::spec::Transform::Identity)? + .build()?, + ); + + let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; + let mut task_writer = TaskWriter::new(writer_builder, false, schema, partition_spec); + + let partition_field = Field::new("region", DataType::Utf8, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), + ); + let partition_values = StringArray::from(vec!["ASIA", "ASIA", "EU", "EU"]); + let partition_struct = StructArray::from(vec![( + Arc::new(partition_field), + Arc::new(partition_values) as ArrayRef, + )]); + + let batch = RecordBatch::try_new(arrow_schema, vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as ArrayRef, + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])) as ArrayRef, + Arc::new(StringArray::from(vec!["ASIA", "ASIA", "EU", "EU"])) as ArrayRef, + Arc::new(partition_struct) as ArrayRef, + ])?; + + task_writer.write(batch).await?; + let data_files = task_writer.close().await?; + + let partition_counts = verify_partition_files(&data_files, 4); + assert_eq!(partition_counts.get("ASIA"), Some(&2)); + assert_eq!(partition_counts.get("EU"), Some(&2)); + + Ok(()) + } +} diff --git a/crates/integrations/datafusion/src/lib.rs b/crates/integrations/datafusion/src/lib.rs index 4b0ea8606d..09d1cac4ce 100644 --- a/crates/integrations/datafusion/src/lib.rs +++ b/crates/integrations/datafusion/src/lib.rs @@ -26,5 +26,3 @@ mod schema; pub mod table; pub use table::table_provider_factory::IcebergTableProviderFactory; pub use table::*; - -pub(crate) mod task_writer; diff --git a/crates/integrations/datafusion/src/task_writer.rs b/crates/integrations/datafusion/src/task_writer.rs index d27b2e6fbf..30013fe5e6 100644 --- a/crates/integrations/datafusion/src/task_writer.rs +++ b/crates/integrations/datafusion/src/task_writer.rs @@ -15,520 +15,5 @@ // specific language governing permissions and limitations // under the License. -//! TaskWriter for DataFusion integration. -//! -//! This module provides a high-level writer that handles partitioning and routing -//! of RecordBatch data to Iceberg tables. - -use datafusion::arrow::array::RecordBatch; -use iceberg::Result; -use iceberg::arrow::RecordBatchPartitionSplitter; -use iceberg::spec::{DataFile, PartitionSpecRef, SchemaRef}; -use iceberg::writer::IcebergWriterBuilder; -use iceberg::writer::partitioning::PartitioningWriter; -use iceberg::writer::partitioning::clustered_writer::ClusteredWriter; -use iceberg::writer::partitioning::fanout_writer::FanoutWriter; -use iceberg::writer::partitioning::unpartitioned_writer::UnpartitionedWriter; - /// High-level writer for DataFusion that handles partitioning and routing of RecordBatch data. -/// -/// TaskWriter coordinates writing data to Iceberg tables by: -/// - Selecting the appropriate partitioning strategy (unpartitioned, fanout, or clustered) -/// - Lazily initializing the partition splitter on first write -/// - Routing data to the underlying writer -/// - Collecting all written data files -/// -/// # Type Parameters -/// -/// * `B` - The IcebergWriterBuilder type used to create underlying writers -/// -/// # Example -/// -/// ```rust,ignore -/// use iceberg::spec::{PartitionSpec, Schema}; -/// use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; -/// use iceberg_datafusion::writer::task_writer::TaskWriter; -/// -/// // Create a TaskWriter for an unpartitioned table -/// let task_writer = TaskWriter::new( -/// data_file_writer_builder, -/// false, // fanout_enabled -/// schema, -/// partition_spec, -/// ); -/// -/// // Write data -/// task_writer.write(record_batch).await?; -/// -/// // Close and get data files -/// let data_files = task_writer.close().await?; -/// ``` -#[allow(dead_code)] -pub(crate) struct TaskWriter { - /// The underlying writer (UnpartitionedWriter, FanoutWriter, or ClusteredWriter) - writer: SupportedWriter, - /// Lazily initialized partition splitter for partitioned tables - partition_splitter: Option, - /// Iceberg schema reference - schema: SchemaRef, - /// Partition specification reference - partition_spec: PartitionSpecRef, -} - -/// Internal enum to hold the different writer types. -/// -/// This enum allows TaskWriter to work with different partitioning strategies -/// while maintaining a unified interface. -#[allow(dead_code)] -enum SupportedWriter { - /// Writer for unpartitioned tables - Unpartitioned(UnpartitionedWriter), - /// Writer for partitioned tables with unsorted data (maintains multiple active writers) - Fanout(FanoutWriter), - /// Writer for partitioned tables with sorted data (maintains single active writer) - Clustered(ClusteredWriter), -} - -#[allow(dead_code)] -impl TaskWriter { - /// Create a new TaskWriter. - /// - /// # Parameters - /// - /// * `writer_builder` - The IcebergWriterBuilder to use for creating underlying writers - /// * `fanout_enabled` - If true, use FanoutWriter for partitioned tables; otherwise use ClusteredWriter - /// * `schema` - The Iceberg schema reference - /// * `partition_spec` - The partition specification reference - /// - /// # Returns - /// - /// Returns a new TaskWriter instance. - /// - /// # Writer Selection Logic - /// - /// - If partition_spec is unpartitioned: creates UnpartitionedWriter - /// - If partition_spec is partitioned AND fanout_enabled is true: creates FanoutWriter - /// - If partition_spec is partitioned AND fanout_enabled is false: creates ClusteredWriter - /// - /// # Example - /// - /// ```rust,ignore - /// use iceberg::spec::{PartitionSpec, Schema}; - /// use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; - /// use iceberg_datafusion::writer::task_writer::TaskWriter; - /// - /// // Create a TaskWriter for an unpartitioned table - /// let task_writer = TaskWriter::new( - /// data_file_writer_builder, - /// false, // fanout_enabled - /// schema, - /// partition_spec, - /// ); - /// ``` - pub fn new( - writer_builder: B, - fanout_enabled: bool, - schema: SchemaRef, - partition_spec: PartitionSpecRef, - ) -> Self { - let writer = if partition_spec.is_unpartitioned() { - SupportedWriter::Unpartitioned(UnpartitionedWriter::new(writer_builder)) - } else if fanout_enabled { - SupportedWriter::Fanout(FanoutWriter::new(writer_builder)) - } else { - SupportedWriter::Clustered(ClusteredWriter::new(writer_builder)) - }; - - Self { - writer, - partition_splitter: None, - schema, - partition_spec, - } - } - - /// Write a RecordBatch to the TaskWriter. - /// - /// For the first write to a partitioned table, this method initializes the partition splitter. - /// For unpartitioned tables, data is written directly without splitting. - /// - /// # Parameters - /// - /// * `batch` - The RecordBatch to write - /// - /// # Returns - /// - /// Returns `Ok(())` on success, or an error if the write fails. - /// - /// # Errors - /// - /// This method will return an error if: - /// - Partition splitter initialization fails - /// - Splitting the batch by partition fails - /// - Writing to the underlying writer fails - /// - /// # Example - /// - /// ```rust,ignore - /// use arrow_array::RecordBatch; - /// use iceberg_datafusion::writer::task_writer::TaskWriter; - /// - /// // Write a RecordBatch - /// task_writer.write(record_batch).await?; - /// ``` - pub async fn write(&mut self, batch: RecordBatch) -> Result<()> { - match &mut self.writer { - SupportedWriter::Unpartitioned(writer) => { - // Unpartitioned: write directly without splitting - writer.write(batch).await - } - SupportedWriter::Fanout(writer) => { - // Initialize splitter on first write if needed - if self.partition_splitter.is_none() { - self.partition_splitter = - Some(RecordBatchPartitionSplitter::new_with_precomputed_values( - self.schema.clone(), - self.partition_spec.clone(), - )?); - } - - // Split and write partitioned data - Self::write_partitioned_batches(writer, &self.partition_splitter, &batch).await - } - SupportedWriter::Clustered(writer) => { - // Initialize splitter on first write if needed - if self.partition_splitter.is_none() { - self.partition_splitter = - Some(RecordBatchPartitionSplitter::new_with_precomputed_values( - self.schema.clone(), - self.partition_spec.clone(), - )?); - } - - // Split and write partitioned data - Self::write_partitioned_batches(writer, &self.partition_splitter, &batch).await - } - } - } - - /// Helper method to split and write partitioned data. - /// - /// This method handles the common logic for both FanoutWriter and ClusteredWriter: - /// - Splits the batch by partition key using the provided splitter - /// - Writes each partition to the underlying writer - /// - /// # Parameters - /// - /// * `writer` - The underlying PartitioningWriter (FanoutWriter or ClusteredWriter) - /// * `partition_splitter` - The partition splitter (must be initialized) - /// * `batch` - The RecordBatch to write - /// - /// # Returns - /// - /// Returns `Ok(())` on success, or an error if the operation fails. - async fn write_partitioned_batches( - writer: &mut W, - partition_splitter: &Option, - batch: &RecordBatch, - ) -> Result<()> { - // Split batch by partition - let splitter = partition_splitter - .as_ref() - .expect("Partition splitter should be initialized"); - let partitioned_batches = splitter.split(batch)?; - - // Write each partition - for (partition_key, partition_batch) in partitioned_batches { - writer.write(partition_key, partition_batch).await?; - } - - Ok(()) - } - - /// Close the TaskWriter and return all written data files. - /// - /// This method consumes the TaskWriter to prevent further use. - /// - /// # Returns - /// - /// Returns a `Vec` containing all written files, or an error if closing fails. - /// - /// # Errors - /// - /// This method will return an error if: - /// - Closing the underlying writer fails - /// - Any I/O operation fails during the close process - /// - /// # Example - /// - /// ```rust,ignore - /// use iceberg_datafusion::writer::task_writer::TaskWriter; - /// - /// // Close the writer and get all data files - /// let data_files = task_writer.close().await?; - /// ``` - pub async fn close(self) -> Result> { - match self.writer { - SupportedWriter::Unpartitioned(writer) => writer.close().await, - SupportedWriter::Fanout(writer) => writer.close().await, - SupportedWriter::Clustered(writer) => writer.close().await, - } - } -} - -#[cfg(test)] -mod tests { - use std::collections::HashMap; - use std::sync::Arc; - - use datafusion::arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray, StructArray}; - use datafusion::arrow::datatypes::{DataType, Field, Schema}; - use iceberg::arrow::PROJECTED_PARTITION_VALUE_COLUMN; - use iceberg::io::FileIOBuilder; - use iceberg::spec::{DataFileFormat, NestedField, PartitionSpec, PrimitiveType, Type}; - use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; - use iceberg::writer::file_writer::ParquetWriterBuilder; - use iceberg::writer::file_writer::location_generator::{ - DefaultFileNameGenerator, DefaultLocationGenerator, - }; - use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; - use parquet::arrow::PARQUET_FIELD_ID_META_KEY; - use parquet::file::properties::WriterProperties; - use tempfile::TempDir; - - use super::*; - - fn create_test_schema() -> Result> { - Ok(Arc::new( - iceberg::spec::Schema::builder() - .with_schema_id(1) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) - .into(), - ]) - .build()?, - )) - } - - fn create_arrow_schema() -> Arc { - Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), - Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "2".to_string(), - )])), - Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "3".to_string(), - )])), - ])) - } - - fn create_arrow_schema_with_partition() -> Arc { - let partition_field = Field::new("region", DataType::Utf8, false).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), - ); - let partition_struct_field = Field::new( - PROJECTED_PARTITION_VALUE_COLUMN, - DataType::Struct(vec![partition_field].into()), - false, - ); - - Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), - Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "2".to_string(), - )])), - Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "3".to_string(), - )])), - partition_struct_field, - ])) - } - - fn create_writer_builder( - temp_dir: &TempDir, - schema: Arc, - ) -> Result< - DataFileWriterBuilder< - ParquetWriterBuilder, - DefaultLocationGenerator, - DefaultFileNameGenerator, - >, - > { - let file_io = FileIOBuilder::new_fs_io().build()?; - let location_gen = DefaultLocationGenerator::with_data_location( - temp_dir.path().to_str().unwrap().to_string(), - ); - let file_name_gen = - DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); - let parquet_writer_builder = - ParquetWriterBuilder::new(WriterProperties::builder().build(), schema); - let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( - parquet_writer_builder, - file_io, - location_gen, - file_name_gen, - ); - Ok(DataFileWriterBuilder::new(rolling_writer_builder)) - } - - #[tokio::test] - async fn test_task_writer_unpartitioned() -> Result<()> { - let temp_dir = TempDir::new()?; - let schema = create_test_schema()?; - let arrow_schema = create_arrow_schema(); - - // Create unpartitioned spec - let partition_spec = Arc::new(PartitionSpec::builder(schema.clone()).build()?); - - let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; - let mut task_writer = TaskWriter::new(writer_builder, false, schema, partition_spec); - - // Write data - let batch = RecordBatch::try_new(arrow_schema, vec![ - Arc::new(Int32Array::from(vec![1, 2, 3])), - Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), - Arc::new(StringArray::from(vec!["US", "EU", "US"])), - ])?; - - task_writer.write(batch).await?; - let data_files = task_writer.close().await?; - - // Verify results - assert!(!data_files.is_empty()); - assert_eq!(data_files[0].record_count(), 3); - - Ok(()) - } - - /// Helper to verify partition data files - fn verify_partition_files( - data_files: &[iceberg::spec::DataFile], - expected_total: u64, - ) -> HashMap { - let total_records: u64 = data_files.iter().map(|f| f.record_count()).sum(); - assert_eq!(total_records, expected_total, "Total record count mismatch"); - - let mut partition_counts = HashMap::new(); - for data_file in data_files { - let partition_value = data_file.partition(); - let region_literal = partition_value.fields()[0] - .as_ref() - .expect("Partition value should not be null"); - let region = match region_literal - .as_primitive_literal() - .expect("Should be primitive literal") - { - iceberg::spec::PrimitiveLiteral::String(s) => s.clone(), - _ => panic!("Expected string partition value"), - }; - - *partition_counts.entry(region.clone()).or_insert(0) += data_file.record_count(); - - // Verify file path contains partition information - assert!( - data_file.file_path().contains("region="), - "File path should contain partition info" - ); - } - partition_counts - } - - #[tokio::test] - async fn test_task_writer_partitioned_fanout() -> Result<()> { - let temp_dir = TempDir::new()?; - let schema = create_test_schema()?; - let arrow_schema = create_arrow_schema_with_partition(); - - let partition_spec = Arc::new( - PartitionSpec::builder(schema.clone()) - .with_spec_id(1) - .add_partition_field("region", "region", iceberg::spec::Transform::Identity)? - .build()?, - ); - - let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; - let mut task_writer = TaskWriter::new(writer_builder, true, schema, partition_spec); - - // Create partition column - let partition_field = Field::new("region", DataType::Utf8, false).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), - ); - let partition_values = StringArray::from(vec!["US", "EU", "US", "EU"]); - let partition_struct = StructArray::from(vec![( - Arc::new(partition_field), - Arc::new(partition_values) as ArrayRef, - )]); - - let batch = RecordBatch::try_new(arrow_schema, vec![ - Arc::new(Int32Array::from(vec![1, 2, 3, 4])), - Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])), - Arc::new(StringArray::from(vec!["US", "EU", "US", "EU"])), - Arc::new(partition_struct), - ])?; - - task_writer.write(batch).await?; - let data_files = task_writer.close().await?; - - let partition_counts = verify_partition_files(&data_files, 4); - assert_eq!(partition_counts.get("US"), Some(&2)); - assert_eq!(partition_counts.get("EU"), Some(&2)); - - Ok(()) - } - - #[tokio::test] - async fn test_task_writer_partitioned_clustered() -> Result<()> { - let temp_dir = TempDir::new()?; - let schema = create_test_schema()?; - let arrow_schema = create_arrow_schema_with_partition(); - - let partition_spec = Arc::new( - PartitionSpec::builder(schema.clone()) - .with_spec_id(1) - .add_partition_field("region", "region", iceberg::spec::Transform::Identity)? - .build()?, - ); - - let writer_builder = create_writer_builder(&temp_dir, schema.clone())?; - let mut task_writer = TaskWriter::new(writer_builder, false, schema, partition_spec); - - // Create partition column - let partition_field = Field::new("region", DataType::Utf8, false).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), - ); - let partition_values = StringArray::from(vec!["ASIA", "ASIA", "EU", "EU"]); - let partition_struct = StructArray::from(vec![( - Arc::new(partition_field), - Arc::new(partition_values) as ArrayRef, - )]); - - // ClusteredWriter expects data to be pre-sorted by partition - let batch = RecordBatch::try_new(arrow_schema, vec![ - Arc::new(Int32Array::from(vec![1, 2, 3, 4])), - Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])), - Arc::new(StringArray::from(vec!["ASIA", "ASIA", "EU", "EU"])), - Arc::new(partition_struct), - ])?; - - task_writer.write(batch).await?; - let data_files = task_writer.close().await?; - - let partition_counts = verify_partition_files(&data_files, 4); - assert_eq!(partition_counts.get("ASIA"), Some(&2)); - assert_eq!(partition_counts.get("EU"), Some(&2)); - - Ok(()) - } -} +pub use iceberg::writer::task_writer::TaskWriter; From 20fb0cf17b949a4ca68a3d65b12743cf79ab095e Mon Sep 17 00:00:00 2001 From: Dylan Date: Mon, 15 Dec 2025 15:45:57 +0800 Subject: [PATCH 27/33] fix: fix record to struct deserialization (#102) * fix record to struct deserialization * refine the test case --- crates/iceberg/src/spec/values.rs | 85 ++++++++++++++++++++++++++----- 1 file changed, 73 insertions(+), 12 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 9b082b39ed..e265b07b60 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -2254,6 +2254,8 @@ mod timestamptz { } mod _serde { + use std::collections::HashMap; + use serde::de::Visitor; use serde::ser::{SerializeMap, SerializeSeq, SerializeStruct}; use serde::{Deserialize, Serialize}; @@ -2904,15 +2906,13 @@ mod _serde { } _ => Err(invalid_err("list")), }, - RawLiteralEnum::Record(Record { - required, - optional: _, - }) => match ty { + RawLiteralEnum::Record(Record { required, optional }) => match ty { Type::Struct(struct_ty) => { - let iters: Vec> = required - .into_iter() - .map(|(field_name, value)| { - let field = struct_ty + let mut value_by_name = HashMap::new(); + + for (field_name, value) in required.into_iter() { + let field = + struct_ty .field_by_name(field_name.as_str()) .ok_or_else(|| { invalid_err_with_reason( @@ -2920,10 +2920,49 @@ mod _serde { &format!("field {} is not exist", &field_name), ) })?; - let value = value.try_into(&field.field_type)?; - Ok(value) - }) - .collect::>()?; + let value = value.try_into(&field.field_type)?; + if value.is_none() && field.required { + return Err(invalid_err_with_reason( + "record", + &format!("required field {} is null", &field_name), + )); + } + value_by_name.insert(field.name.clone(), value); + } + + for (field_name, value) in optional.into_iter() { + let field = + struct_ty + .field_by_name(field_name.as_str()) + .ok_or_else(|| { + invalid_err_with_reason( + "record", + &format!("field {} is not exist", &field_name), + ) + })?; + let value = match value { + Some(v) => v.try_into(&field.field_type)?, + None => None, + }; + value_by_name.insert(field.name.clone(), value); + } + + let mut iters = Vec::with_capacity(struct_ty.fields().len()); + for field in struct_ty.fields() { + match value_by_name.remove(&field.name) { + Some(value) => iters.push(value), + None => { + if field.required { + return Err(invalid_err_with_reason( + "record", + &format!("required field {} is missing", field.name), + )); + } + iters.push(None); + } + } + } + Ok(Some(Literal::Struct(super::Struct::from_iter(iters)))) } Type::Map(map_ty) => { @@ -3052,6 +3091,28 @@ mod tests { } } + #[test] + fn json_struct_preserves_schema_order() { + // struct fields are deliberately ordered as b, then a to detect ordering drift + let struct_type = StructType::new(vec![ + NestedField::required(2, "b", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(1, "a", Type::Primitive(PrimitiveType::Long)).into(), + ]); + let literal = Literal::Struct(Struct::from_iter(vec![ + Some(Literal::int(5)), + Some(Literal::long(10)), + ])); + + let raw = + RawLiteral::try_from(literal.clone(), &Type::Struct(struct_type.clone())).unwrap(); + + // serde_json::Value uses BTreeMap (sorted keys), which mimics the RW metadata path. + let value = serde_json::to_value(&raw).unwrap(); + let deser: RawLiteral = serde_json::from_value(value).unwrap(); + let roundtrip = deser.try_into(&Type::Struct(struct_type)).unwrap().unwrap(); + + assert_eq!(roundtrip, literal); + } #[test] fn json_boolean() { let record = r#"true"#; From fc3867d81aa279bdfecd8703b56144cfb80e9c12 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 16 Dec 2025 14:31:06 +0800 Subject: [PATCH 28/33] public upper bounds (#103) --- crates/iceberg/src/spec/manifest/data_file.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/spec/manifest/data_file.rs b/crates/iceberg/src/spec/manifest/data_file.rs index 5af13438e4..7c40222102 100644 --- a/crates/iceberg/src/spec/manifest/data_file.rs +++ b/crates/iceberg/src/spec/manifest/data_file.rs @@ -104,7 +104,7 @@ pub struct DataFile { /// /// - [Binary single-value serialization](https://iceberg.apache.org/spec/#binary-single-value-serialization) #[builder(default)] - pub(crate) lower_bounds: HashMap, + pub lower_bounds: HashMap, /// field id: 128 /// key field id: 129 /// value field id: 130 @@ -117,7 +117,7 @@ pub struct DataFile { /// /// - [Binary single-value serialization](https://iceberg.apache.org/spec/#binary-single-value-serialization) #[builder(default)] - pub(crate) upper_bounds: HashMap, + pub upper_bounds: HashMap, /// field id: 131 /// /// Implementation-specific key metadata for encryption From 15602441f017b5a36d88dfedc52e3c21116a63e1 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 16 Dec 2025 16:30:58 +0800 Subject: [PATCH 29/33] public schema change needed api (#104) --- crates/iceberg/src/catalog/mod.rs | 1 - crates/iceberg/src/spec/table_metadata.rs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 27d5edaedb..66bcca5304 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -302,7 +302,6 @@ pub struct TableCreation { /// [`TableCommit`] directly. /// Users are supposed to use [`crate::transaction::Transaction`] to update table. #[derive(Debug, TypedBuilder)] -#[builder(build_method(vis = "pub(crate)"))] pub struct TableCommit { /// The table ident. ident: TableIdent, diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 06b32cc847..8332bce578 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -74,7 +74,7 @@ pub struct TableMetadata { /// Timestamp in milliseconds from the unix epoch when the table was last updated. pub(crate) last_updated_ms: i64, /// An integer; the highest assigned column ID for the table. - pub(crate) last_column_id: i32, + pub last_column_id: i32, /// A list of schemas, stored as objects with schema-id. pub(crate) schemas: HashMap, /// ID of the table’s current schema. From 54498bc5189b8d1f68419b53ba2d0288ce375a87 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed, 17 Dec 2025 16:30:30 +0800 Subject: [PATCH 30/33] feat: add update_schema action for Transaction (#105) * add update_schema action for Transaction * minor * fmt --- crates/iceberg/src/catalog/mod.rs | 1 + crates/iceberg/src/spec/table_metadata.rs | 2 +- crates/iceberg/src/transaction/mod.rs | 7 + .../iceberg/src/transaction/update_schema.rs | 299 ++++++++++++++++++ 4 files changed, 308 insertions(+), 1 deletion(-) create mode 100644 crates/iceberg/src/transaction/update_schema.rs diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 66bcca5304..27d5edaedb 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -302,6 +302,7 @@ pub struct TableCreation { /// [`TableCommit`] directly. /// Users are supposed to use [`crate::transaction::Transaction`] to update table. #[derive(Debug, TypedBuilder)] +#[builder(build_method(vis = "pub(crate)"))] pub struct TableCommit { /// The table ident. ident: TableIdent, diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 8332bce578..06b32cc847 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -74,7 +74,7 @@ pub struct TableMetadata { /// Timestamp in milliseconds from the unix epoch when the table was last updated. pub(crate) last_updated_ms: i64, /// An integer; the highest assigned column ID for the table. - pub last_column_id: i32, + pub(crate) last_column_id: i32, /// A list of schemas, stored as objects with schema-id. pub(crate) schemas: HashMap, /// ID of the table’s current schema. diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 2a31c83e7c..faf755c86f 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -56,6 +56,7 @@ pub use action::*; mod append; pub use append::FastAppendAction; +pub use update_schema::UpdateSchemaAction; mod manifest_filter; pub use manifest_filter::*; @@ -66,6 +67,7 @@ mod snapshot; mod sort_order; mod update_location; mod update_properties; +mod update_schema; mod update_statistics; mod upgrade_format_version; @@ -159,6 +161,11 @@ impl Transaction { UpdatePropertiesAction::new() } + /// Update table's schema. + pub fn update_schema(&self) -> UpdateSchemaAction { + UpdateSchemaAction::new() + } + /// Creates a fast append action. pub fn fast_append(&self) -> FastAppendAction { FastAppendAction::new() diff --git a/crates/iceberg/src/transaction/update_schema.rs b/crates/iceberg/src/transaction/update_schema.rs new file mode 100644 index 0000000000..78195e22f2 --- /dev/null +++ b/crates/iceberg/src/transaction/update_schema.rs @@ -0,0 +1,299 @@ +// 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; + +use async_trait::async_trait; + +use crate::error::Result; +use crate::spec::{NestedFieldRef, Schema}; +use crate::table::Table; +use crate::transaction::{ActionCommit, TransactionAction}; +use crate::{TableRequirement, TableUpdate}; + +/// Transaction action for updating table schema. +/// +/// This action allows adding new fields to a table's schema during a transaction. +/// It supports adding single or multiple fields at once. +pub struct UpdateSchemaAction { + fields_to_add: Vec, +} + +impl UpdateSchemaAction { + /// Creates a new [`UpdateSchemaAction`] with no fields to add. + pub fn new() -> Self { + UpdateSchemaAction { + fields_to_add: vec![], + } + } + + /// Adds a single field to the schema. + /// + /// # Arguments + /// + /// * `field` - The field to add to the schema. + /// + /// # Returns + /// + /// The updated [`UpdateSchemaAction`] with the field added. + pub fn add_field(mut self, field: NestedFieldRef) -> Self { + self.fields_to_add.push(field); + self + } + + /// Adds multiple fields to the schema. + /// + /// # Arguments + /// + /// * `fields` - An iterable of fields to add to the schema. + /// + /// # Returns + /// + /// The updated [`UpdateSchemaAction`] with the fields added. + pub fn add_fields(mut self, fields: impl IntoIterator) -> Self { + self.fields_to_add.extend(fields); + self + } +} + +impl Default for UpdateSchemaAction { + fn default() -> Self { + Self::new() + } +} + +#[async_trait] +impl TransactionAction for UpdateSchemaAction { + async fn commit(self: Arc, table: &Table) -> Result { + let current_schema = table.metadata().current_schema(); + let metadata = table.metadata(); + + // Build new schema with existing fields + new fields + let mut all_fields = current_schema.as_struct().fields().to_vec(); + all_fields.extend(self.fields_to_add.iter().cloned()); + + // Create new schema with incremented schema ID + // Find the max schema ID and increment it + let new_schema_id = metadata + .schemas_iter() + .map(|s| s.schema_id()) + .max() + .unwrap_or(0) + + 1; + let new_schema = Schema::builder() + .with_schema_id(new_schema_id) + .with_fields(all_fields) + .with_identifier_field_ids(current_schema.identifier_field_ids()) + .build()?; + + let updates = vec![ + TableUpdate::AddSchema { schema: new_schema }, + TableUpdate::SetCurrentSchema { schema_id: -1 }, + ]; + + let requirements = vec![ + TableRequirement::LastAssignedFieldIdMatch { + last_assigned_field_id: metadata.last_column_id(), + }, + TableRequirement::CurrentSchemaIdMatch { + current_schema_id: current_schema.schema_id(), + }, + ]; + + Ok(ActionCommit::new(updates, requirements)) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use as_any::Downcast; + + use crate::spec::{NestedField, PrimitiveType, Type}; + use crate::transaction::tests::make_v2_table; + use crate::transaction::update_schema::UpdateSchemaAction; + use crate::transaction::{ApplyTransactionAction, Transaction}; + + #[test] + fn test_add_single_field() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + + let new_field = Arc::new(NestedField::optional( + 100, + "new_field", + Type::Primitive(PrimitiveType::String), + )); + + let tx = tx + .update_schema() + .add_field(new_field.clone()) + .apply(tx) + .unwrap(); + + assert_eq!(tx.actions.len(), 1); + + let action = (*tx.actions[0]) + .downcast_ref::() + .unwrap(); + + assert_eq!(action.fields_to_add.len(), 1); + assert_eq!(action.fields_to_add[0].name, "new_field"); + } + + #[test] + fn test_add_multiple_fields() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + + let field1 = Arc::new(NestedField::optional( + 100, + "field1", + Type::Primitive(PrimitiveType::String), + )); + let field2 = Arc::new(NestedField::required( + 101, + "field2", + Type::Primitive(PrimitiveType::Int), + )); + + let tx = tx + .update_schema() + .add_fields(vec![field1.clone(), field2.clone()]) + .apply(tx) + .unwrap(); + + assert_eq!(tx.actions.len(), 1); + + let action = (*tx.actions[0]) + .downcast_ref::() + .unwrap(); + + assert_eq!(action.fields_to_add.len(), 2); + assert_eq!(action.fields_to_add[0].name, "field1"); + assert_eq!(action.fields_to_add[1].name, "field2"); + } + + #[test] + fn test_chained_add_field() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + + let field1 = Arc::new(NestedField::optional( + 100, + "field1", + Type::Primitive(PrimitiveType::String), + )); + let field2 = Arc::new(NestedField::required( + 101, + "field2", + Type::Primitive(PrimitiveType::Int), + )); + + let tx = tx + .update_schema() + .add_field(field1.clone()) + .add_field(field2.clone()) + .apply(tx) + .unwrap(); + + assert_eq!(tx.actions.len(), 1); + + let action = (*tx.actions[0]) + .downcast_ref::() + .unwrap(); + + assert_eq!(action.fields_to_add.len(), 2); + assert_eq!(action.fields_to_add[0].name, "field1"); + assert_eq!(action.fields_to_add[1].name, "field2"); + } + + #[tokio::test] + async fn test_commit_action() { + use crate::transaction::TransactionAction; + use crate::{TableRequirement, TableUpdate}; + + let table = make_v2_table(); + let original_schema = table.metadata().current_schema(); + let original_schema_id = original_schema.schema_id(); + let original_field_count = original_schema.as_struct().fields().len(); + let last_column_id = table.metadata().last_column_id(); + + let new_field = Arc::new(NestedField::optional( + 100, + "new_column", + Type::Primitive(PrimitiveType::String), + )); + + let action = Arc::new(UpdateSchemaAction::new().add_field(new_field)); + + let action_commit = action.commit(&table).await.unwrap(); + let mut action_commit = action_commit; + + let updates = action_commit.take_updates(); + let requirements = action_commit.take_requirements(); + + // Verify we have the correct updates + assert_eq!(updates.len(), 2); + + // First update should be AddSchema + match &updates[0] { + TableUpdate::AddSchema { schema } => { + // New schema should have incremented ID + assert_eq!(schema.schema_id(), original_schema_id + 1); + // New schema should have one more field + assert_eq!(schema.as_struct().fields().len(), original_field_count + 1); + // New field should be present + assert!(schema.field_by_name("new_column").is_some()); + // Old fields should still be present + for field in original_schema.as_struct().fields() { + assert!( + schema.field_by_name(&field.name).is_some(), + "Field {} should be present in new schema", + field.name + ); + } + } + _ => panic!("Expected AddSchema update"), + } + + // Second update should be SetCurrentSchema with -1 + match &updates[1] { + TableUpdate::SetCurrentSchema { schema_id } => { + assert_eq!(schema_id, &-1); + } + _ => panic!("Expected SetCurrentSchema update"), + } + + // Verify requirements + assert_eq!(requirements.len(), 2); + + // Should have LastAssignedFieldIdMatch requirement + assert!(requirements.iter().any(|r| matches!( + r, + TableRequirement::LastAssignedFieldIdMatch { + last_assigned_field_id + } if last_assigned_field_id == &last_column_id + ))); + + // Should have CurrentSchemaIdMatch requirement + assert!(requirements.iter().any( + |r| matches!(r, TableRequirement::CurrentSchemaIdMatch { current_schema_id } if current_schema_id == &original_schema_id) + )); + } +} From 93be8e6e0c147ce051eceae2de1e762231b23580 Mon Sep 17 00:00:00 2001 From: August Date: Tue, 6 Jan 2026 12:39:30 +0800 Subject: [PATCH 31/33] chore: add public access of object cache (#106) * chore: add public access of object cache --- crates/iceberg/src/io/mod.rs | 3 ++- crates/iceberg/src/io/object_cache.rs | 2 +- crates/iceberg/src/table.rs | 8 +++++++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index b46d468288..de9db7cb74 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -70,7 +70,8 @@ mod file_io; mod storage; pub use file_io::*; -pub(crate) mod object_cache; +/// object cache module +pub mod object_cache; #[cfg(feature = "storage-azdls")] mod storage_azdls; diff --git a/crates/iceberg/src/io/object_cache.rs b/crates/iceberg/src/io/object_cache.rs index af297bebb5..2f43728b82 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -120,7 +120,7 @@ impl ObjectCache { /// Retrieves an Arc [`ManifestList`] from the cache /// or retrieves one from FileIO and parses it if not present - pub(crate) async fn get_manifest_list( + pub async fn get_manifest_list( &self, snapshot: &SnapshotRef, table_metadata: &TableMetadataRef, diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index 9c789e2186..b1026123e7 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -215,7 +215,7 @@ impl Table { } /// Returns this table's object cache - pub(crate) fn object_cache(&self) -> Arc { + pub fn object_cache(&self) -> Arc { self.object_cache.clone() } @@ -244,6 +244,12 @@ impl Table { pub fn reader_builder(&self) -> ArrowReaderBuilder { ArrowReaderBuilder::new(self.file_io.clone()) } + + /// Returns a new `Table` instance with the provided object cache. + pub fn with_object_cache(mut self, object_cache: Arc) -> Self { + self.object_cache = object_cache; + self + } } /// `StaticTable` is a read-only table struct that can be created from a metadata file or from `TableMetaData` without a catalog. From 4bf15b94052f507c4cf0a820863294a65d8ac1a2 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sun, 4 Jan 2026 17:57:47 -0800 Subject: [PATCH 32/33] fix(spec): Include delete file content to V3 manifest (#1979) ## Which issue does this PR close? - Closes #1973 ## What changes are included in this PR? - Write `content` to V3 manifest so the field is preserved correctly for delete entry ## Are these changes tested? Yes --- crates/iceberg/src/spec/manifest/writer.rs | 98 +++++++++++++++++++++- 1 file changed, 95 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index ebb0590bcf..782693056c 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -437,9 +437,12 @@ impl ManifestWriter { "format-version".to_string(), (self.metadata.format_version as u8).to_string(), )?; - if self.metadata.format_version == FormatVersion::V2 { - avro_writer - .add_user_metadata("content".to_string(), self.metadata.content.to_string())?; + match self.metadata.format_version { + FormatVersion::V1 => {} + FormatVersion::V2 | FormatVersion::V3 => { + avro_writer + .add_user_metadata("content".to_string(), self.metadata.content.to_string())?; + } } let partition_summary = self.construct_partition_summaries(&partition_type)?; @@ -708,4 +711,93 @@ mod tests { entries[0].file_sequence_number = None; assert_eq!(actual_manifest, Manifest::new(metadata, entries)); } + + #[tokio::test] + async fn test_v3_delete_manifest_delete_file_roundtrip() { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(), + ); + + let partition_spec = PartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .build() + .unwrap(); + + // Create a position delete file entry + let delete_entry = ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::PositionDeletes, + file_path: "s3://bucket/table/data/delete-00000.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::empty(), + record_count: 10, + file_size_in_bytes: 1024, + column_sizes: HashMap::new(), + value_counts: HashMap::new(), + null_value_counts: HashMap::new(), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: None, + split_offsets: None, + equality_ids: None, + sort_order_id: None, + partition_spec_id: 0, + first_row_id: None, + referenced_data_file: None, + content_offset: None, + content_size_in_bytes: None, + }, + }; + + // Write a V3 delete manifest + let tmp_dir = TempDir::new().unwrap(); + let path = tmp_dir.path().join("v3_delete_manifest.avro"); + let io = FileIOBuilder::new_fs_io().build().unwrap(); + let output_file = io.new_output(path.to_str().unwrap()).unwrap(); + + let mut writer = ManifestWriterBuilder::new( + output_file, + Some(1), + None, + schema.clone(), + partition_spec.clone(), + ) + .build_v3_deletes(); + + writer.add_entry(delete_entry).unwrap(); + let manifest_file = writer.write_manifest_file().await.unwrap(); + + // The returned ManifestFile correctly reports Deletes content + assert_eq!(manifest_file.content, ManifestContentType::Deletes); + + // Read back the manifest file + let actual_manifest = + Manifest::parse_avro(fs::read(&path).expect("read_file must succeed").as_slice()) + .unwrap(); + + // Verify the content type is correctly preserved as Deletes + assert_eq!( + actual_manifest.metadata().content, + ManifestContentType::Deletes, + ); + } } From 15e43f41595602f5266de407a37972e1329b6e32 Mon Sep 17 00:00:00 2001 From: Alex Graham Date: Wed, 7 Jan 2026 05:06:44 -0600 Subject: [PATCH 33/33] feat(reader): Add PartitionSpec support to FileScanTask and RecordBatchTransformer (#1821) (#107) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Partially address #1749. This PR adds partition spec handling to `FileScanTask` and `RecordBatchTransformer` to correctly implement the Iceberg spec's "Column Projection" rules for fields "not present" in data files. Prior to this PR, `iceberg-rust`'s `FileScanTask` had no mechanism to pass partition information to `RecordBatchTransformer`, causing two issues: 1. **Incorrect handling of bucket partitioning**: Couldn't distinguish identity transforms (which should use partition metadata constants) from non-identity transforms like bucket/truncate/year/month (which must read from data file). For example, `bucket(4, id)` stores `id_bucket = 2` (bucket number) in partition metadata, but actual `id` values (100, 200, 300) are only in the data file. iceberg-rust was incorrectly treating bucket-partitioned source columns as constants, breaking runtime filtering and returning incorrect query results. 2. **Field ID conflicts in add_files scenarios**: When importing Hive tables via `add_files`, partition columns could have field IDs conflicting with Parquet data columns. Example: Parquet has field_id=1→"name", but Iceberg expects field_id=1→"id" (partition). Per spec, the correct field is "not present" and requires name mapping fallback. Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection), when a field ID is "not present" in a data file, it must be resolved using these rules: 1. Return the value from partition metadata if an **Identity Transform** exists 2. Use `schema.name-mapping.default` metadata to map field id to columns without field id 3. Return the default value if it has a defined `initial-default` 4. Return null in all other cases **Why this matters:** - **Identity transforms** (e.g., `identity(dept)`) store actual column values in partition metadata that can be used as constants without reading the data file - **Non-identity transforms** (e.g., `bucket(4, id)`, `day(timestamp)`) store transformed values in partition metadata (e.g., bucket number 2, not the actual `id` values 100, 200, 300) and must read source columns from the data file 1. **Added partition fields to `FileScanTask`** (`scan/task.rs`): - `partition: Option` - Partition data from manifest entry - `partition_spec: Option>` - For transform-aware constant detection - `name_mapping: Option>` - Name mapping from table metadata 2. **Implemented `constants_map()` function** (`arrow/record_batch_transformer.rs`): - Replicates Java's `PartitionUtil.constantsMap()` behavior - Only includes fields where transform is `Transform::Identity` - Used to determine which fields use partition metadata constants vs. reading from data files 3. **Enhanced `RecordBatchTransformer`** (`arrow/record_batch_transformer.rs`): - Added `build_with_partition_data()` method to accept partition spec, partition data, and name mapping - Implements all 4 spec rules for column resolution with identity-transform awareness - Detects field ID conflicts by verifying both field ID AND name match - Falls back to name mapping when field IDs are missing/conflicting (spec rule #2) 4. **Updated `ArrowReader`** (`arrow/reader.rs`): - Uses `build_with_partition_data()` when partition information is available - Falls back to `build()` when not available 5. **Updated manifest entry processing** (`scan/context.rs`): - Populates partition fields in `FileScanTask` from manifest entry data 1. **`bucket_partitioning_reads_source_column_from_file`** - Verifies that bucket-partitioned source columns are read from data files (not treated as constants from partition metadata) 2. **`identity_partition_uses_constant_from_metadata`** - Verifies that identity-transformed fields correctly use partition metadata constants 3. **`test_bucket_partitioning_with_renamed_source_column`** - Verifies field-ID-based mapping works despite column rename 4. **`add_files_partition_columns_without_field_ids`** - Verifies name mapping resolution for Hive table imports without field IDs (spec rule 5. **`add_files_with_true_field_id_conflict`** - Verifies correct field ID conflict detection with name mapping fallback (spec rule #2) 6. **`test_all_four_spec_rules`** - Integration test verifying all 4 spec rules work together Yes, there are 6 new unit tests covering all 4 Iceberg spec rules. This also resolved approximately 50 Iceberg Java tests when running with DataFusion Comet's experimental https://github.com/apache/datafusion-comet/pull/2528 PR. --------- Co-authored-by: Matt Butrovich Co-authored-by: Renjie Liu --- .../src/arrow/caching_delete_file_loader.rs | 9 + .../iceberg/src/arrow/delete_file_loader.rs | 4 +- crates/iceberg/src/arrow/delete_filter.rs | 9 + crates/iceberg/src/arrow/reader.rs | 352 ++++++- .../src/arrow/record_batch_transformer.rs | 894 ++++++++++++++++-- crates/iceberg/src/scan/context.rs | 7 + crates/iceberg/src/scan/mod.rs | 6 + crates/iceberg/src/scan/task.rs | 61 +- crates/iceberg/src/spec/manifest/writer.rs | 2 +- 9 files changed, 1250 insertions(+), 94 deletions(-) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index 568fa2d8df..eef55afcfa 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -886,6 +886,9 @@ mod tests { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let eq_del_scan_task = FileScanTask { @@ -902,6 +905,9 @@ mod tests { sequence_number: 0, equality_ids: eq_del.equality_ids.clone(), file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let file_scan_task = FileScanTask { @@ -918,6 +924,9 @@ mod tests { data_file_content: DataContentType::Data, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; // Load the deletes - should handle both types without error diff --git a/crates/iceberg/src/arrow/delete_file_loader.rs b/crates/iceberg/src/arrow/delete_file_loader.rs index 8018e786f6..17acb47bfc 100644 --- a/crates/iceberg/src/arrow/delete_file_loader.rs +++ b/crates/iceberg/src/arrow/delete_file_loader.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use futures::{StreamExt, TryStreamExt}; use crate::arrow::ArrowReader; -use crate::arrow::record_batch_transformer::RecordBatchTransformer; +use crate::arrow::record_batch_transformer::RecordBatchTransformerBuilder; use crate::io::FileIO; use crate::scan::{ArrowRecordBatchStream, FileScanTaskDeleteFile}; use crate::spec::{Schema, SchemaRef}; @@ -82,7 +82,7 @@ impl BasicDeleteFileLoader { equality_ids: &[i32], ) -> Result { let mut record_batch_transformer = - RecordBatchTransformer::build(target_schema.clone(), equality_ids); + RecordBatchTransformerBuilder::new(target_schema.clone(), equality_ids).build(); let record_batch_stream = record_batch_stream.map(move |record_batch| { record_batch.and_then(|record_batch| { diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index 8b154ff62a..e7fe3d6553 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -324,6 +324,9 @@ pub(crate) mod tests { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let file_scan_tasks = vec![ @@ -341,6 +344,9 @@ pub(crate) mod tests { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }, FileScanTask { start: 0, @@ -356,6 +362,9 @@ pub(crate) mod tests { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index da3131c2c3..57625c2999 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -45,7 +45,7 @@ use parquet::file::metadata::{ use parquet::schema::types::{SchemaDescriptor, Type as ParquetType}; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; -use crate::arrow::record_batch_transformer::RecordBatchTransformer; +use crate::arrow::record_batch_transformer::RecordBatchTransformerBuilder; use crate::arrow::{arrow_schema_to_schema, get_arrow_datum}; use crate::delete_vector::DeleteVector; use crate::error::Result; @@ -55,7 +55,7 @@ use crate::expr::visitors::row_group_metrics_evaluator::RowGroupMetricsEvaluator use crate::expr::{BoundPredicate, BoundReference}; use crate::io::{FileIO, FileMetadata, FileRead}; use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskStream}; -use crate::spec::{DataContentType, Datum, NestedField, PrimitiveType, Schema, Type}; +use crate::spec::{DataContentType, Datum, NameMapping, NestedField, PrimitiveType, Schema, Type}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; @@ -181,7 +181,8 @@ impl ArrowReader { let should_load_page_index = (row_selection_enabled && task.predicate.is_some()) || !task.deletes.is_empty(); - let delete_filter_rx = delete_file_loader.load_deletes(&task.deletes, task.schema.clone()); + let delete_filter_rx = + delete_file_loader.load_deletes(&task.deletes, Arc::clone(&task.schema)); // Migrated tables lack field IDs, requiring us to inspect the schema to choose // between field-ID-based or position-based projection @@ -193,7 +194,9 @@ impl ArrowReader { ) .await?; - // Parquet files from Hive/Spark migrations lack field IDs in their metadata + // Check if Parquet file has embedded field IDs + // Corresponds to Java's ParquetSchemaUtil.hasIds() + // Reference: parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java:118 let missing_field_ids = initial_stream_builder .schema() .fields() @@ -201,11 +204,38 @@ impl ArrowReader { .next() .is_some_and(|f| f.metadata().get(PARQUET_FIELD_ID_META_KEY).is_none()); - // Adding position-based fallback IDs at schema level (not per-batch) enables projection - // on files that lack embedded field IDs. We recreate the builder to apply the modified schema. + // Three-branch schema resolution strategy matching Java's ReadConf constructor + // + // Per Iceberg spec Column Projection rules: + // "Columns in Iceberg data files are selected by field id. The table schema's column + // names and order may change after a data file is written, and projection must be done + // using field ids." + // https://iceberg.apache.org/spec/#column-projection + // + // When Parquet files lack field IDs (e.g., Hive/Spark migrations via add_files), + // we must assign field IDs BEFORE reading data to enable correct projection. + // + // Java's ReadConf determines field ID strategy: + // - Branch 1: hasIds(fileSchema) → trust embedded field IDs, use pruneColumns() + // - Branch 2: nameMapping present → applyNameMapping(), then pruneColumns() + // - Branch 3: fallback → addFallbackIds(), then pruneColumnsFallback() let mut record_batch_stream_builder = if missing_field_ids { - let arrow_schema = - add_fallback_field_ids_to_arrow_schema(initial_stream_builder.schema()); + // Parquet file lacks field IDs - must assign them before reading + let arrow_schema = if let Some(name_mapping) = &task.name_mapping { + // Branch 2: Apply name mapping to assign correct Iceberg field IDs + // Per spec rule #2: "Use schema.name-mapping.default metadata to map field id + // to columns without field id" + // Corresponds to Java's ParquetSchemaUtil.applyNameMapping() + apply_name_mapping_to_arrow_schema( + Arc::clone(initial_stream_builder.schema()), + name_mapping, + )? + } else { + // Branch 3: No name mapping - use position-based fallback IDs + // Corresponds to Java's ParquetSchemaUtil.addFallbackIds() + add_fallback_field_ids_to_arrow_schema(initial_stream_builder.schema()) + }; + let options = ArrowReaderOptions::new().with_schema(arrow_schema); Self::create_parquet_record_batch_stream_builder( @@ -216,11 +246,14 @@ impl ArrowReader { ) .await? } else { + // Branch 1: File has embedded field IDs - trust them initial_stream_builder }; - // Fallback IDs don't match Parquet's embedded field IDs (since they don't exist), - // so we must use position-based projection instead of field-ID matching + // Create projection mask based on field IDs + // - If file has embedded IDs: field-ID-based projection (missing_field_ids=false) + // - If name mapping applied: field-ID-based projection (missing_field_ids=true but IDs now match) + // - If fallback IDs: position-based projection (missing_field_ids=true) let projection_mask = Self::get_arrow_projection_mask( &task.project_field_ids, &task.schema, @@ -234,9 +267,18 @@ impl ArrowReader { // 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()); + // and column re-ordering. + let mut record_batch_transformer_builder = + RecordBatchTransformerBuilder::new(task.schema_ref(), task.project_field_ids()); + + if let (Some(partition_spec), Some(partition_data)) = + (task.partition_spec.clone(), task.partition.clone()) + { + record_batch_transformer_builder = + record_batch_transformer_builder.with_partition(partition_spec, partition_data); + } + + let mut record_batch_transformer = record_batch_transformer_builder.build(); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -922,6 +964,77 @@ fn build_fallback_field_id_map(parquet_schema: &SchemaDescriptor) -> HashMap Result> { + debug_assert!( + arrow_schema + .fields() + .iter() + .next() + .is_none_or(|f| f.metadata().get(PARQUET_FIELD_ID_META_KEY).is_none()), + "Schema already has field IDs - name mapping should not be applied" + ); + + use arrow_schema::Field; + + let fields_with_mapped_ids: Vec<_> = arrow_schema + .fields() + .iter() + .map(|field| { + // Look up this column name in name mapping to get the Iceberg field ID. + // Corresponds to Java's ApplyNameMapping visitor which calls + // nameMapping.find(currentPath()) and returns field.withId() if found. + // + // If the field isn't in the mapping, leave it WITHOUT assigning an ID + // (matching Java's behavior of returning the field unchanged). + // Later, during projection, fields without IDs are filtered out. + let mapped_field_opt = name_mapping + .fields() + .iter() + .find(|f| f.names().contains(&field.name().to_string())); + + let mut metadata = field.metadata().clone(); + + if let Some(mapped_field) = mapped_field_opt { + if let Some(field_id) = mapped_field.field_id() { + // Field found in mapping with a field_id → assign it + metadata.insert(PARQUET_FIELD_ID_META_KEY.to_string(), field_id.to_string()); + } + // If field_id is None, leave the field without an ID (will be filtered by projection) + } + // If field not found in mapping, leave it without an ID (will be filtered by projection) + + Field::new(field.name(), field.data_type().clone(), field.is_nullable()) + .with_metadata(metadata) + }) + .collect(); + + Ok(Arc::new(ArrowSchema::new_with_metadata( + fields_with_mapped_ids, + arrow_schema.metadata().clone(), + ))) +} + /// Add position-based fallback field IDs to Arrow schema for Parquet files lacking them. /// Enables projection on migrated files (e.g., from Hive/Spark). /// @@ -1970,6 +2083,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2292,6 +2408,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; // Task 2: read the second and third row groups @@ -2309,6 +2428,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let tasks1 = Box::pin(futures::stream::iter(vec![Ok(task1)])) as FileScanTaskStream; @@ -2437,6 +2559,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2614,6 +2739,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, } .into(), ], @@ -2621,6 +2749,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -2845,6 +2976,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, } .into(), ], @@ -2852,6 +2986,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3069,6 +3206,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, } .into(), ], @@ -3076,6 +3216,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3180,6 +3323,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3278,6 +3424,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3365,6 +3514,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3466,6 +3618,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3596,6 +3751,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3693,6 +3851,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3803,6 +3964,9 @@ message schema { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3818,4 +3982,166 @@ message schema { // Should return empty results assert!(result.is_empty() || result.iter().all(|batch| batch.num_rows() == 0)); } + + /// Test bucket partitioning reads source column from data file (not partition metadata). + /// + /// This is an integration test verifying the complete ArrowReader pipeline with bucket partitioning. + /// It corresponds to TestRuntimeFiltering tests in Iceberg Java (e.g., testRenamedSourceColumnTable). + /// + /// # Iceberg Spec Requirements + /// + /// Per the Iceberg spec "Column Projection" section: + /// > "Return the value from partition metadata if an **Identity Transform** exists for the field" + /// + /// This means: + /// - Identity transforms (e.g., `identity(dept)`) use constants from partition metadata + /// - Non-identity transforms (e.g., `bucket(4, id)`) must read source columns from data files + /// - Partition metadata for bucket transforms stores bucket numbers (0-3), NOT source values + /// + /// Java's PartitionUtil.constantsMap() implements this via: + /// ```java + /// if (field.transform().isIdentity()) { + /// idToConstant.put(field.sourceId(), converted); + /// } + /// ``` + /// + /// # What This Test Verifies + /// + /// This test ensures the full ArrowReader → RecordBatchTransformer pipeline correctly handles + /// bucket partitioning when FileScanTask provides partition_spec and partition_data: + /// + /// - Parquet file has field_id=1 named "id" with actual data [1, 5, 9, 13] + /// - FileScanTask specifies partition_spec with bucket(4, id) and partition_data with bucket=1 + /// - RecordBatchTransformer.constants_map() excludes bucket-partitioned field from constants + /// - ArrowReader correctly reads [1, 5, 9, 13] from the data file + /// - Values are NOT replaced with constant 1 from partition metadata + /// + /// # Why This Matters + /// + /// Without correct handling: + /// - Runtime filtering would break (e.g., `WHERE id = 5` would fail) + /// - Query results would be incorrect (all rows would have id=1) + /// - Bucket partitioning would be unusable for query optimization + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" + "Partition Transforms" + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + #[tokio::test] + async fn test_bucket_partitioning_reads_source_column_from_file() { + use arrow_array::Int32Array; + + use crate::spec::{Literal, PartitionSpec, Struct, Transform}; + + // Iceberg schema with id and name columns + let schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: bucket(4, id) + let partition_spec = Arc::new( + PartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .add_partition_field("id", "id_bucket", Transform::Bucket(4)) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: bucket value is 1 + let partition_data = Struct::from_iter(vec![Some(Literal::int(1))]); + + // Create Arrow schema with field IDs for Parquet file + 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(), + )])), + Field::new("name", DataType::Utf8, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + ])); + + // Write Parquet file with data + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + + let id_data = Arc::new(Int32Array::from(vec![1, 5, 9, 13])) as ArrayRef; + let name_data = + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])) as ArrayRef; + + let to_write = + RecordBatch::try_new(arrow_schema.clone(), vec![id_data, name_data]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + let file = File::create(format!("{}/data.parquet", &table_location)).unwrap(); + let mut writer = ArrowWriter::try_new(file, to_write.schema(), Some(props)).unwrap(); + writer.write(&to_write).expect("Writing batch"); + writer.close().unwrap(); + + // Read the Parquet file with partition spec and data + let reader = ArrowReaderBuilder::new(file_io).build(); + let tasks = Box::pin(futures::stream::iter( + vec![Ok(FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: format!("{}/data.parquet", table_location), + data_file_format: DataFileFormat::Parquet, + schema: schema.clone(), + project_field_ids: vec![1, 2], + predicate: None, + deletes: vec![], + data_file_content: DataContentType::Data, + sequence_number: 0, + equality_ids: None, + file_size_in_bytes: 0, + partition: Some(partition_data), + partition_spec: Some(partition_spec), + name_mapping: None, + })] + .into_iter(), + )) as FileScanTaskStream; + + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Verify we got the correct data + assert_eq!(result.len(), 1); + let batch = &result[0]; + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 4); + + // The id column MUST contain actual values from the Parquet file [1, 5, 9, 13], + // NOT the constant partition value 1 + let id_col = batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.value(0), 1); + assert_eq!(id_col.value(1), 5); + assert_eq!(id_col.value(2), 9); + assert_eq!(id_col.value(3), 13); + + let name_col = 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"); + assert_eq!(name_col.value(3), "Dave"); + } } diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 5fbbbb106a..e7d8b8f0fb 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -29,9 +29,44 @@ use arrow_schema::{ use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::schema_to_arrow_schema; -use crate::spec::{Literal, PrimitiveLiteral, Schema as IcebergSchema}; +use crate::spec::{ + Literal, PartitionSpec, PrimitiveLiteral, Schema as IcebergSchema, Struct, Transform, +}; use crate::{Error, ErrorKind, Result}; +/// Build a map of field ID to constant value for identity-partitioned fields. +/// +/// Implements Iceberg spec "Column Projection" rule #1: use partition metadata constants +/// only for identity-transformed fields. Non-identity transforms (bucket, truncate, year, etc.) +/// store derived values in partition metadata, so source columns must be read from data files. +/// +/// Example: For `bucket(4, id)`, partition metadata has `id_bucket = 2` (bucket number), +/// but the actual `id` values (100, 200, 300) are only in the data file. +/// +/// Matches Java's `PartitionUtil.constantsMap()` which filters `if (field.transform().isIdentity())`. +/// +/// # References +/// - Spec: https://iceberg.apache.org/spec/#column-projection +/// - Java: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java:constantsMap() +fn constants_map( + partition_spec: &PartitionSpec, + partition_data: &Struct, +) -> HashMap { + let mut constants = HashMap::new(); + + for (pos, field) in partition_spec.fields().iter().enumerate() { + // Only identity transforms should use constant values from partition metadata + if matches!(field.transform, Transform::Identity) { + // Get the partition value for this field + if let Some(Literal::Primitive(value)) = &partition_data[pos] { + constants.insert(field.source_id, value.clone()); + } + } + } + + constants +} + /// Indicates how a particular column in a processed RecordBatch should /// be sourced. #[derive(Debug)] @@ -107,32 +142,107 @@ enum SchemaComparison { Different, } +/// Builder for RecordBatchTransformer to improve ergonomics when constructing with optional parameters. +/// +/// See [`RecordBatchTransformer`] for details on partition spec and partition data. #[derive(Debug)] -pub(crate) struct RecordBatchTransformer { +pub(crate) struct RecordBatchTransformerBuilder { snapshot_schema: Arc, projected_iceberg_field_ids: Vec, - - // BatchTransform gets lazily constructed based on the schema of - // the first RecordBatch we receive from the file - batch_transform: Option, + partition_spec: Option>, + partition_data: Option, } -impl RecordBatchTransformer { - /// Build a RecordBatchTransformer for a given - /// Iceberg snapshot schema and list of projected field ids. - pub(crate) fn build( +impl RecordBatchTransformerBuilder { + pub(crate) fn new( snapshot_schema: Arc, projected_iceberg_field_ids: &[i32], ) -> Self { - let projected_iceberg_field_ids = projected_iceberg_field_ids.to_vec(); - Self { snapshot_schema, - projected_iceberg_field_ids, + projected_iceberg_field_ids: projected_iceberg_field_ids.to_vec(), + partition_spec: None, + partition_data: None, + } + } + + /// Set partition spec and data together for identifying identity-transformed partition columns. + /// + /// Both partition_spec and partition_data must be provided together since the spec defines + /// which fields are identity-partitioned, and the data provides their constant values. + /// One without the other cannot produce a valid constants map. + pub(crate) fn with_partition( + mut self, + partition_spec: Arc, + partition_data: Struct, + ) -> Self { + self.partition_spec = Some(partition_spec); + self.partition_data = Some(partition_data); + self + } + + pub(crate) fn build(self) -> RecordBatchTransformer { + RecordBatchTransformer { + snapshot_schema: self.snapshot_schema, + projected_iceberg_field_ids: self.projected_iceberg_field_ids, + partition_spec: self.partition_spec, + partition_data: self.partition_data, batch_transform: None, } } +} + +/// Transforms RecordBatches from Parquet files to match the Iceberg table schema. +/// +/// Handles schema evolution, column reordering, type promotion, and implements the Iceberg spec's +/// "Column Projection" rules for resolving field IDs "not present" in data files: +/// 1. Return the value from partition metadata if an Identity Transform exists +/// 2. Use schema.name-mapping.default metadata to map field id to columns without field id (applied in ArrowReader) +/// 3. Return the default value if it has a defined initial-default +/// 4. Return null in all other cases +/// +/// # Field ID Resolution +/// +/// Field ID resolution happens in ArrowReader before data is read (matching Java's ReadConf): +/// - If file has embedded field IDs: trust them (ParquetSchemaUtil.hasIds() = true) +/// - If file lacks IDs and name_mapping exists: apply name mapping (ParquetSchemaUtil.applyNameMapping()) +/// - If file lacks IDs and no name_mapping: use position-based fallback (ParquetSchemaUtil.addFallbackIds()) +/// +/// By the time RecordBatchTransformer processes data, all field IDs are trustworthy. +/// This transformer only handles remaining projection rules (#1, #3, #4) for fields still "not present". +/// +/// # Partition Spec and Data +/// +/// **Bucket partitioning**: Distinguish identity transforms (use partition metadata constants) +/// from non-identity transforms like bucket (read from data file) to enable runtime filtering on +/// bucket-partitioned columns. For example, `bucket(4, id)` stores only the bucket number in +/// partition metadata, so actual `id` values must be read from the data file. +/// +/// # References +/// - Spec: https://iceberg.apache.org/spec/#column-projection +/// - Java: parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java (field ID resolution) +/// - Java: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java (partition constants) +#[derive(Debug)] +pub(crate) struct RecordBatchTransformer { + snapshot_schema: Arc, + projected_iceberg_field_ids: Vec, + + /// Partition spec for identifying identity-transformed partition columns (spec rule #1). + /// Only fields with identity transforms use partition data constants; non-identity transforms + /// (bucket, truncate, etc.) must read source columns from data files. + partition_spec: Option>, + /// Partition data providing constant values for identity-transformed partition columns (spec rule #1). + /// For example, in a file at path `dept=engineering/file.parquet`, this would contain + /// the value "engineering" for the dept field. + partition_data: Option, + + // BatchTransform gets lazily constructed based on the schema of + // the first RecordBatch we receive from the file + batch_transform: Option, +} + +impl RecordBatchTransformer { pub(crate) fn process_record_batch( &mut self, record_batch: RecordBatch, @@ -147,7 +257,7 @@ impl RecordBatchTransformer { .with_match_field_names(false) .with_row_count(Some(record_batch.num_rows())); RecordBatch::try_new_with_options( - target_schema.clone(), + Arc::clone(target_schema), self.transform_columns(record_batch.columns(), operations)?, &options, )? @@ -157,7 +267,7 @@ impl RecordBatchTransformer { .with_match_field_names(false) .with_row_count(Some(record_batch.num_rows())); RecordBatch::try_new_with_options( - target_schema.clone(), + Arc::clone(target_schema), record_batch.columns().to_vec(), &options, )? @@ -167,6 +277,8 @@ impl RecordBatchTransformer { record_batch.schema_ref(), self.snapshot_schema.as_ref(), &self.projected_iceberg_field_ids, + self.partition_spec.as_ref().map(|s| s.as_ref()), + self.partition_data.as_ref(), )?); self.process_record_batch(record_batch)? @@ -185,6 +297,8 @@ impl RecordBatchTransformer { source_schema: &ArrowSchemaRef, snapshot_schema: &IcebergSchema, projected_iceberg_field_ids: &[i32], + partition_spec: Option<&PartitionSpec>, + partition_data: Option<&Struct>, ) -> Result { let mapped_unprojected_arrow_schema = Arc::new(schema_to_arrow_schema(snapshot_schema)?); let field_id_to_mapped_schema_map = @@ -205,6 +319,12 @@ impl RecordBatchTransformer { let target_schema = Arc::new(ArrowSchema::new(fields?)); + let constants_map = if let (Some(spec), Some(data)) = (partition_spec, partition_data) { + constants_map(spec, data) + } else { + HashMap::new() + }; + match Self::compare_schemas(source_schema, &target_schema) { SchemaComparison::Equivalent => Ok(BatchTransform::PassThrough), SchemaComparison::NameChangesOnly => Ok(BatchTransform::ModifySchema { target_schema }), @@ -214,6 +334,8 @@ impl RecordBatchTransformer { snapshot_schema, projected_iceberg_field_ids, field_id_to_mapped_schema_map, + constants_map, + partition_spec, )?, target_schema, }), @@ -270,57 +392,92 @@ impl RecordBatchTransformer { snapshot_schema: &IcebergSchema, projected_iceberg_field_ids: &[i32], field_id_to_mapped_schema_map: HashMap, + constants_map: HashMap, + _partition_spec: Option<&PartitionSpec>, ) -> Result> { let field_id_to_source_schema_map = Self::build_field_id_to_arrow_schema_map(source_schema)?; - projected_iceberg_field_ids.iter().map(|field_id|{ - let (target_field, _) = field_id_to_mapped_schema_map.get(field_id).ok_or( - Error::new(ErrorKind::Unexpected, "could not find field in schema") - )?; - let target_type = target_field.data_type(); - - Ok(if let Some((source_field, source_index)) = field_id_to_source_schema_map.get(field_id) { - // column present in source + projected_iceberg_field_ids + .iter() + .map(|field_id| { + let (target_field, _) = + field_id_to_mapped_schema_map + .get(field_id) + .ok_or(Error::new( + ErrorKind::Unexpected, + "could not find field in schema", + ))?; + let target_type = target_field.data_type(); - if source_field.data_type().equals_datatype(target_type) { - // no promotion required - ColumnSource::PassThrough { - source_index: *source_index + let iceberg_field = snapshot_schema.field_by_id(*field_id).ok_or(Error::new( + ErrorKind::Unexpected, + "Field not found in snapshot schema", + ))?; + + // Iceberg spec's "Column Projection" rules (https://iceberg.apache.org/spec/#column-projection). + // For fields "not present" in data files: + // 1. Use partition metadata (identity transforms only) + // 2. Use name mapping + // 3. Use initial_default + // 4. Return null + // + // Why check partition constants before Parquet field IDs (Java: BaseParquetReaders.java:299): + // In add_files scenarios, partition columns may exist in BOTH Parquet AND partition metadata. + // Partition metadata is authoritative - it defines which partition this file belongs to. + + // Field ID resolution now happens in ArrowReader via: + // 1. Embedded field IDs (ParquetSchemaUtil.hasIds() = true) - trust them + // 2. Name mapping (ParquetSchemaUtil.applyNameMapping()) - applied upfront + // 3. Position-based fallback (ParquetSchemaUtil.addFallbackIds()) - applied upfront + // + // At this point, all field IDs in the source schema are trustworthy. + // No conflict detection needed - schema resolution happened in reader.rs. + let field_by_id = field_id_to_source_schema_map.get(field_id).map( + |(source_field, source_index)| { + if source_field.data_type().equals_datatype(target_type) { + ColumnSource::PassThrough { + source_index: *source_index, + } + } else { + ColumnSource::Promote { + target_type: target_type.clone(), + source_index: *source_index, + } + } + }, + ); + + // Apply spec's fallback steps for "not present" fields. + let column_source = if let Some(constant_value) = constants_map.get(field_id) { + // Rule #1: Identity partition constant + ColumnSource::Add { + value: Some(constant_value.clone()), + target_type: target_type.clone(), } + } else if let Some(source) = field_by_id { + source } else { - // promotion required - ColumnSource::Promote { + // Rules #2, #3 and #4: + // Rule #2 (name mapping) was already applied in reader.rs if needed. + // If field_id is still not found, the column doesn't exist in the Parquet file. + // Fall through to rule #3 (initial_default) or rule #4 (null). + let default_value = iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) + } else { + None + } + }); + ColumnSource::Add { + value: default_value, target_type: target_type.clone(), - source_index: *source_index, } - } - } else { - // column must be added - let iceberg_field = snapshot_schema.field_by_id(*field_id).ok_or( - Error::new(ErrorKind::Unexpected, "Field not found in snapshot schema") - )?; - - let default_value = if let Some(iceberg_default_value) = - &iceberg_field.initial_default - { - let Literal::Primitive(primitive_literal) = iceberg_default_value else { - return Err(Error::new( - ErrorKind::Unexpected, - format!("Default value for column must be primitive type, but encountered {iceberg_default_value:?}") - )); - }; - Some(primitive_literal.clone()) - } else { - None }; - ColumnSource::Add { - value: default_value, - target_type: target_type.clone(), - } + Ok(column_source) }) - }).collect() + .collect() } fn build_field_id_to_arrow_schema_map( @@ -328,25 +485,19 @@ impl RecordBatchTransformer { ) -> Result> { let mut field_id_to_source_schema = HashMap::new(); for (source_field_idx, source_field) in source_schema.fields.iter().enumerate() { - let this_field_id = source_field - .metadata() - .get(PARQUET_FIELD_ID_META_KEY) - .ok_or_else(|| { + // Check if field has a field ID in metadata + if let Some(field_id_str) = source_field.metadata().get(PARQUET_FIELD_ID_META_KEY) { + let this_field_id = field_id_str.parse().map_err(|e| { Error::new( ErrorKind::DataInvalid, - "field ID not present in parquet metadata", - ) - })? - .parse() - .map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("field id not parseable as an i32: {e}"), + format!("field id not parseable as an i32: {}", e), ) })?; - field_id_to_source_schema - .insert(this_field_id, (source_field.clone(), source_field_idx)); + field_id_to_source_schema + .insert(this_field_id, (source_field.clone(), source_field_idx)); + } + // If field doesn't have a field ID, skip it - name mapping will handle it } Ok(field_id_to_source_schema) @@ -447,7 +598,7 @@ impl RecordBatchTransformer { (dt, _) => { return Err(Error::new( ErrorKind::Unexpected, - format!("unexpected target column type {dt}"), + format!("unexpected target column type {}", dt), )); } }) @@ -466,8 +617,10 @@ mod test { use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; - use crate::arrow::record_batch_transformer::RecordBatchTransformer; - use crate::spec::{Literal, NestedField, PrimitiveType, Schema, Type}; + use crate::arrow::record_batch_transformer::{ + RecordBatchTransformer, RecordBatchTransformerBuilder, + }; + use crate::spec::{Literal, NestedField, PrimitiveType, Schema, Struct, Type}; #[test] fn build_field_id_to_source_schema_map_works() { @@ -492,7 +645,9 @@ mod test { let snapshot_schema = Arc::new(iceberg_table_schema()); let projected_iceberg_field_ids = [13, 14]; - let mut inst = RecordBatchTransformer::build(snapshot_schema, &projected_iceberg_field_ids); + let mut inst = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_iceberg_field_ids) + .build(); let result = inst .process_record_batch(source_record_batch_no_migration_required()) @@ -508,7 +663,9 @@ mod test { let snapshot_schema = Arc::new(iceberg_table_schema()); let projected_iceberg_field_ids = [10, 11, 12, 14, 15]; // a, b, c, e, f - let mut inst = RecordBatchTransformer::build(snapshot_schema, &projected_iceberg_field_ids); + let mut inst = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_iceberg_field_ids) + .build(); let result = inst.process_record_batch(source_record_batch()).unwrap(); @@ -537,7 +694,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"), @@ -696,4 +854,592 @@ mod test { value.to_string(), )])) } + + /// Test for add_files with Parquet files that have NO field IDs (Hive tables). + /// + /// This reproduces the scenario from Iceberg spec where: + /// - Hive-style partitioned Parquet files are imported via add_files procedure + /// - Parquet files originally DO NOT have field IDs (typical for Hive tables) + /// - ArrowReader applies name mapping to assign correct Iceberg field IDs + /// - Iceberg schema assigns field IDs: id (1), name (2), dept (3), subdept (4) + /// - Partition columns (id, dept) have initial_default values + /// + /// Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection), + /// this scenario requires `schema.name-mapping.default` from table metadata + /// to correctly map Parquet columns by name to Iceberg field IDs. + /// This mapping is now applied in ArrowReader before data is processed. + /// + /// Expected behavior: + /// 1. id=1 (from initial_default) - spec rule #3 + /// 2. name="John Doe" (from Parquet with field_id=2 assigned by reader) - found by field ID + /// 3. dept="hr" (from initial_default) - spec rule #3 + /// 4. subdept="communications" (from Parquet with field_id=4 assigned by reader) - found by field ID + #[test] + fn add_files_with_name_mapping_applied_in_reader() { + // Iceberg schema after add_files: id (partition), name, dept (partition), subdept + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::optional(1, "id", Type::Primitive(PrimitiveType::Int)) + .with_initial_default(Literal::int(1)) + .into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "dept", Type::Primitive(PrimitiveType::String)) + .with_initial_default(Literal::string("hr")) + .into(), + NestedField::optional(4, "subdept", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build() + .unwrap(), + ); + + // Simulate ArrowReader having applied name mapping: + // Original Parquet: name, subdept (NO field IDs) + // After reader.rs applies name mapping: name (field_id=2), subdept (field_id=4) + // + // Note: Partition columns (id, dept) are NOT in the Parquet file - they're in directory paths + use std::collections::HashMap; + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("name", DataType::Utf8, true).with_metadata(HashMap::from([( + "PARQUET:field_id".to_string(), + "2".to_string(), + )])), + Field::new("subdept", DataType::Utf8, true).with_metadata(HashMap::from([( + "PARQUET:field_id".to_string(), + "4".to_string(), + )])), + ])); + + let projected_field_ids = [1, 2, 3, 4]; // id, name, dept, subdept + + let mut transformer = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_field_ids).build(); + + // Create a Parquet RecordBatch with data for: name="John Doe", subdept="communications" + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(StringArray::from(vec!["John Doe"])), + Arc::new(StringArray::from(vec!["communications"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch has: + // - id=1 (from initial_default, not from Parquet) + // - name="John Doe" (from Parquet with correct field_id=2) + // - dept="hr" (from initial_default, not from Parquet) + // - subdept="communications" (from Parquet with correct field_id=4) + assert_eq!(result.num_columns(), 4); + assert_eq!(result.num_rows(), 1); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 1); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "John Doe"); + + let dept_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(dept_column.value(0), "hr"); + + let subdept_column = result + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(subdept_column.value(0), "communications"); + } + + /// Test for bucket partitioning where source columns must be read from data files. + /// + /// This test verifies correct implementation of the Iceberg spec's "Column Projection" rules: + /// > "Return the value from partition metadata if an **Identity Transform** exists for the field" + /// + /// # Why this test is critical + /// + /// The key insight is that partition metadata stores TRANSFORMED values, not source values: + /// - For `bucket(4, id)`, partition metadata has `id_bucket = 2` (the bucket number) + /// - The actual `id` column values (100, 200, 300) are ONLY in the data file + /// + /// If iceberg-rust incorrectly treated bucket-partitioned fields as constants, it would: + /// 1. Replace all `id` values with the constant `2` from partition metadata + /// 2. Break runtime filtering (e.g., `WHERE id = 100` would match no rows) + /// 3. Return incorrect query results + /// + /// # What this test verifies + /// + /// - Bucket-partitioned fields (e.g., `bucket(4, id)`) are read from the data file + /// - The source column `id` contains actual values (100, 200, 300), not constants + /// - Java's `PartitionUtil.constantsMap()` behavior is correctly replicated: + /// ```java + /// if (field.transform().isIdentity()) { // FALSE for bucket transforms + /// idToConstant.put(field.sourceId(), converted); + /// } + /// ``` + /// + /// # Real-world impact + /// + /// This reproduces the failure scenario from Iceberg Java's TestRuntimeFiltering: + /// - Tables partitioned by `bucket(N, col)` are common for load balancing + /// - Queries filter on the source column: `SELECT * FROM tbl WHERE col = value` + /// - Runtime filtering pushes predicates down to Iceberg file scans + /// - Without this fix, the filter would match against constant partition values instead of data + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" + "Partition Transforms" + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java + #[test] + fn bucket_partitioning_reads_source_column_from_file() { + use crate::spec::{Struct, Transform}; + + // Table schema: id (data column), name (data column), id_bucket (partition column) + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: bucket(4, id) - the id field is bucketed + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("id", "id_bucket", Transform::Bucket(4)) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: bucket value is 2 + // In Iceberg, partition data is a Struct where each field corresponds to a partition field + let partition_data = Struct::from_iter(vec![Some(Literal::int(2))]); + + // Parquet file contains both id and name columns + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "2"), + ])); + + let projected_field_ids = [1, 2]; // id, name + + let mut transformer = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_field_ids) + .with_partition(partition_spec, partition_data) + .build(); + + // Create a Parquet RecordBatch with actual data + // The id column MUST be read from here, not treated as a constant + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200, 300])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch correctly reads id from the file + // (NOT as a constant from partition metadata) + assert_eq!(result.num_columns(), 2); + assert_eq!(result.num_rows(), 3); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + // These values MUST come from the Parquet file, not be replaced by constants + assert_eq!(id_column.value(0), 100); + assert_eq!(id_column.value(1), 200); + assert_eq!(id_column.value(2), 300); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + assert_eq!(name_column.value(2), "Charlie"); + } + + /// Test that identity-transformed partition fields ARE treated as constants. + /// + /// This is the complement to `bucket_partitioning_reads_source_column_from_file`, + /// verifying that constants_map() correctly identifies identity-transformed + /// partition fields per the Iceberg spec. + /// + /// # Spec requirement (format/spec.md "Column Projection") + /// + /// > "Return the value from partition metadata if an Identity Transform exists for the field + /// > and the partition value is present in the `partition` struct on `data_file` object + /// > in the manifest. This allows for metadata only migrations of Hive tables." + /// + /// # Why identity transforms use constants + /// + /// Unlike bucket/truncate/year/etc., identity transforms don't modify the value: + /// - `identity(dept)` stores the actual `dept` value in partition metadata + /// - Partition metadata has `dept = "engineering"` (the real value, not a hash/bucket) + /// - This value can be used directly without reading the data file + /// + /// # Performance benefit + /// + /// For Hive migrations where partition columns aren't in data files: + /// - Partition metadata provides the column values + /// - No need to read from data files (metadata-only query optimization) + /// - Common pattern: `dept=engineering/subdept=backend/file.parquet` + /// - `dept` and `subdept` are in directory structure, not in `file.parquet` + /// - Iceberg populates these from partition metadata as constants + /// + /// # What this test verifies + /// + /// - Identity-partitioned fields use constants from partition metadata + /// - The `dept` column is populated with `"engineering"` (not read from file) + /// - Java's `PartitionUtil.constantsMap()` behavior is matched: + /// ```java + /// if (field.transform().isIdentity()) { // TRUE for identity + /// idToConstant.put(field.sourceId(), converted); + /// } + /// ``` + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + #[test] + fn identity_partition_uses_constant_from_metadata() { + use crate::spec::{Struct, Transform}; + + // Table schema: id (data column), dept (partition column), name (data column) + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "dept", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: identity(dept) - the dept field uses identity transform + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("dept", "dept", Transform::Identity) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: dept="engineering" + let partition_data = Struct::from_iter(vec![Some(Literal::string("engineering"))]); + + // Parquet file contains only id and name (dept is in partition path) + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "3"), + ])); + + let projected_field_ids = [1, 2, 3]; // id, dept, name + + let mut transformer = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_field_ids) + .with_partition(partition_spec, partition_data) + .build(); + + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the dept column is populated with the constant from partition metadata + assert_eq!(result.num_columns(), 3); + assert_eq!(result.num_rows(), 2); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 100); + assert_eq!(id_column.value(1), 200); + + let dept_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + // This value MUST come from partition metadata (constant) + assert_eq!(dept_column.value(0), "engineering"); + assert_eq!(dept_column.value(1), "engineering"); + + let name_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + } + + /// Test bucket partitioning with renamed source column. + /// + /// This verifies correct behavior for TestRuntimeFiltering.testRenamedSourceColumnTable() in Iceberg Java. + /// When a source column is renamed after partitioning is established, field-ID-based mapping + /// must still correctly identify the column in Parquet files. + /// + /// # Scenario + /// + /// 1. Table created with `bucket(4, id)` partitioning + /// 2. Data written to Parquet files (field_id=1, name="id") + /// 3. Column renamed: `ALTER TABLE ... RENAME COLUMN id TO row_id` + /// 4. Iceberg schema now has: field_id=1, name="row_id" + /// 5. Parquet files still have: field_id=1, name="id" + /// + /// # Expected Behavior Per Iceberg Spec + /// + /// Per the Iceberg spec "Column Projection" section and Java's PartitionUtil.constantsMap(): + /// - Bucket transforms are NON-identity, so partition metadata stores bucket numbers (0-3), not source values + /// - Source columns for non-identity transforms MUST be read from data files + /// - Field-ID-based mapping should find the column by field_id=1 (ignoring name mismatch) + /// - Runtime filtering on `row_id` should work correctly + /// + /// # What This Tests + /// + /// This test ensures that when FileScanTask provides partition_spec and partition_data: + /// - constants_map() correctly identifies that bucket(4, row_id) is NOT an identity transform + /// - The source column (field_id=1) is NOT added to constants_map + /// - Field-ID-based mapping reads actual values from the Parquet file + /// - Values [100, 200, 300] are read, not replaced with bucket constant 2 + /// + /// # References + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java::testRenamedSourceColumnTable + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java::constantsMap() + /// - Iceberg spec: format/spec.md "Column Projection" section + #[test] + fn test_bucket_partitioning_with_renamed_source_column() { + use crate::spec::{Struct, Transform}; + + // Iceberg schema after rename: row_id (was id), name + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "row_id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: bucket(4, row_id) - but source_id still points to field_id=1 + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("row_id", "row_id_bucket", Transform::Bucket(4)) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: bucket value is 2 + let partition_data = Struct::from_iter(vec![Some(Literal::int(2))]); + + // Parquet file has OLD column name "id" but SAME field_id=1 + // Field-ID-based mapping should find this despite name mismatch + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "2"), + ])); + + let projected_field_ids = [1, 2]; // row_id (field_id=1), name (field_id=2) + + let mut transformer = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_field_ids) + .with_partition(partition_spec, partition_data) + .build(); + + // Create a Parquet RecordBatch with actual data + // Despite column rename, data should be read via field_id=1 + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200, 300])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch correctly reads data despite name mismatch + assert_eq!(result.num_columns(), 2); + assert_eq!(result.num_rows(), 3); + + let row_id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + // These values MUST come from the Parquet file via field_id=1, + // not be replaced by the bucket constant (2) + assert_eq!(row_id_column.value(0), 100); + assert_eq!(row_id_column.value(1), 200); + assert_eq!(row_id_column.value(2), 300); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + assert_eq!(name_column.value(2), "Charlie"); + } + + /// Comprehensive integration test that verifies all 4 Iceberg spec rules work correctly. + /// + /// Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection), + /// "Values for field ids which are not present in a data file must be resolved + /// according the following rules:" + /// + /// This test creates a scenario where each rule is exercised: + /// - Rule #1: dept (identity-partitioned) -> constant from partition metadata + /// - Rule #2: data (via name mapping) -> read from Parquet file by name + /// - Rule #3: category (initial_default) -> use default value + /// - Rule #4: notes (no default) -> return null + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" section + #[test] + fn test_all_four_spec_rules() { + use crate::spec::Transform; + + // Iceberg schema with columns designed to exercise each spec rule + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + // Field in Parquet by field ID (normal case) + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + // Rule #1: Identity-partitioned field - should use partition metadata + NestedField::required(2, "dept", Type::Primitive(PrimitiveType::String)).into(), + // Rule #2: Field resolved by name mapping (ArrowReader already applied) + NestedField::required(3, "data", Type::Primitive(PrimitiveType::String)).into(), + // Rule #3: Field with initial_default + NestedField::optional(4, "category", Type::Primitive(PrimitiveType::String)) + .with_initial_default(Literal::string("default_category")) + .into(), + // Rule #4: Field with no default - should be null + NestedField::optional(5, "notes", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: identity transform on dept + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("dept", "dept", Transform::Identity) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: dept="engineering" + let partition_data = Struct::from_iter(vec![Some(Literal::string("engineering"))]); + + // Parquet schema: simulates post-ArrowReader state where name mapping already applied + // Has id (field_id=1) and data (field_id=3, assigned by ArrowReader via name mapping) + // Missing: dept (in partition), category (has default), notes (no default) + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("data", DataType::Utf8, false, "3"), + ])); + + let projected_field_ids = [1, 2, 3, 4, 5]; // id, dept, data, category, notes + + let mut transformer = + RecordBatchTransformerBuilder::new(snapshot_schema, &projected_field_ids) + .with_partition(partition_spec, partition_data) + .build(); + + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200])), + Arc::new(StringArray::from(vec!["value1", "value2"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + assert_eq!(result.num_columns(), 5); + assert_eq!(result.num_rows(), 2); + + // Verify each column demonstrates the correct spec rule: + + // Normal case: id from Parquet by field ID + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 100); + assert_eq!(id_column.value(1), 200); + + // Rule #1: dept from partition metadata (identity transform) + let dept_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(dept_column.value(0), "engineering"); + assert_eq!(dept_column.value(1), "engineering"); + + // Rule #2: data from Parquet via name mapping + let data_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(data_column.value(0), "value1"); + assert_eq!(data_column.value(1), "value2"); + + // Rule #3: category from initial_default + let category_column = result + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(category_column.value(0), "default_category"); + assert_eq!(category_column.value(1), "default_category"); + + // Rule #4: notes is null (no default, not in Parquet, not in partition) + let notes_column = result + .column(4) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(notes_column.is_null(0)); + assert!(notes_column.is_null(1)); + } } diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 4170420dcf..378d8d3806 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -141,6 +141,13 @@ impl ManifestEntryContext { sequence_number: self.manifest_entry.sequence_number().unwrap_or(0), equality_ids: self.manifest_entry.data_file().equality_ids(), file_size_in_bytes: self.manifest_entry.data_file().file_size_in_bytes(), + + // Include partition data and spec from manifest entry + partition: Some(self.manifest_entry.data_file.partition.clone()), + // TODO: Pass actual PartitionSpec through context chain for native flow + partition_spec: None, + // TODO: Extract name_mapping from table metadata property "schema.name-mapping.default" + name_mapping: None, }) } } diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index cc60b7a3ec..8c3997e128 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -1823,6 +1823,9 @@ pub mod tests { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; test_fn(task); @@ -1841,6 +1844,9 @@ pub mod tests { sequence_number: 0, equality_ids: None, file_size_in_bytes: 0, + partition: None, + partition_spec: None, + name_mapping: None, }; test_fn(task); } diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index de9e8ec806..fe79c0edcc 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -18,15 +18,36 @@ use std::sync::Arc; use futures::stream::BoxStream; -use serde::{Deserialize, Serialize}; +use serde::{Deserialize, Serialize, Serializer}; use crate::Result; use crate::expr::BoundPredicate; -use crate::spec::{DataContentType, DataFileFormat, ManifestEntryRef, Schema, SchemaRef}; +use crate::spec::{ + DataContentType, DataFileFormat, ManifestEntryRef, NameMapping, PartitionSpec, Schema, + SchemaRef, Struct, +}; /// A stream of [`FileScanTask`]. pub type FileScanTaskStream = BoxStream<'static, Result>; +/// Serialization helper that always returns NotImplementedError. +/// Used for fields that should not be serialized but we want to be explicit about it. +fn serialize_not_implemented(_: &T, _: S) -> std::result::Result +where S: Serializer { + Err(serde::ser::Error::custom( + "Serialization not implemented for this field", + )) +} + +/// Deserialization helper that always returns NotImplementedError. +/// Used for fields that should not be deserialized but we want to be explicit about it. +fn deserialize_not_implemented<'de, D, T>(_: D) -> std::result::Result +where D: serde::Deserializer<'de> { + Err(serde::de::Error::custom( + "Deserialization not implemented for this field", + )) +} + /// A task to scan part of file. #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct FileScanTask { @@ -57,8 +78,6 @@ pub struct FileScanTask { #[serde(skip_serializing_if = "Option::is_none")] pub predicate: Option, - /// The list of delete files that may need to be applied to this data file - pub deletes: Vec>, /// sequence number pub sequence_number: i64, /// equality ids @@ -66,6 +85,36 @@ pub struct FileScanTask { /// The size of the file in bytes. pub file_size_in_bytes: u64, + + /// The list of delete files that may need to be applied to this data file + pub deletes: Vec>, + + /// Partition data from the manifest entry, used to identify which columns can use + /// constant values from partition metadata vs. reading from the data file. + /// Per the Iceberg spec, only identity-transformed partition fields should use constants. + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(serialize_with = "serialize_not_implemented")] + #[serde(deserialize_with = "deserialize_not_implemented")] + pub partition: Option, + + /// The partition spec for this file, used to distinguish identity transforms + /// (which use partition metadata constants) from non-identity transforms like + /// bucket/truncate (which must read source columns from the data file). + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(serialize_with = "serialize_not_implemented")] + #[serde(deserialize_with = "deserialize_not_implemented")] + pub partition_spec: Option>, + + /// Name mapping from table metadata (property: schema.name-mapping.default), + /// used to resolve field IDs from column names when Parquet files lack field IDs + /// or have field ID conflicts. + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(serialize_with = "serialize_not_implemented")] + #[serde(deserialize_with = "deserialize_not_implemented")] + pub name_mapping: Option>, } impl FileScanTask { @@ -132,6 +181,10 @@ impl From<&DeleteFileContext> for FileScanTask { sequence_number: ctx.manifest_entry.sequence_number().unwrap_or(0), equality_ids: ctx.manifest_entry.data_file().equality_ids(), file_size_in_bytes: ctx.manifest_entry.data_file().file_size_in_bytes(), + + partition: Some(ctx.manifest_entry.data_file().partition().clone()), + partition_spec: None, // TODO: pass through partition spec info + name_mapping: None, // TODO: implement name mapping } } } diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index 782693056c..d6b3ea560e 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -757,7 +757,7 @@ mod tests { lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), key_metadata: None, - split_offsets: None, + split_offsets: vec![], equality_ids: None, sort_order_id: None, partition_spec_id: 0,