From 8ed0ddc807535551476ea96f7f5fc833c987183b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 30 May 2025 16:58:51 -0700 Subject: [PATCH 01/25] add tx action-ish for set_location (does not work well :( --- crates/catalog/memory/src/catalog.rs | 25 ++++++- crates/catalog/memory/src/namespace_state.rs | 18 +++++ crates/iceberg/src/transaction/action/mod.rs | 69 ++++++++++++++++++++ crates/iceberg/src/transaction/mod.rs | 34 ++++++++-- 4 files changed, 138 insertions(+), 8 deletions(-) create mode 100644 crates/iceberg/src/transaction/action/mod.rs diff --git a/crates/catalog/memory/src/catalog.rs b/crates/catalog/memory/src/catalog.rs index 7454de126..fe4d203dc 100644 --- a/crates/catalog/memory/src/catalog.rs +++ b/crates/catalog/memory/src/catalog.rs @@ -26,7 +26,7 @@ use iceberg::spec::{TableMetadata, TableMetadataBuilder}; use iceberg::table::Table; use iceberg::{ Catalog, Error, ErrorKind, Namespace, NamespaceIdent, Result, TableCommit, TableCreation, - TableIdent, + TableIdent, TableRequirement, TableUpdate, }; use itertools::Itertools; use uuid::Uuid; @@ -277,7 +277,28 @@ impl Catalog for MemoryCatalog { } /// Update a table to the catalog. - async fn update_table(&self, _commit: TableCommit) -> Result { + async fn update_table(&self, commit: TableCommit) -> Result
{ + // TODO persist the updated metadata + // let mut root_namespace_state = self.root_namespace_state.lock().await; + // let current_table = self.load_table(commit.identifier()).await?; + // let updated_staged_table = update_and_stage_table(Some(¤t_table), &commit)?; + // + // if current_table.metadata() == updated_staged_table.metadata() { + // // no changes + // return Ok(current_table); + // } + // + // // write metadata + // self.file_io + // .new_output(&updated_staged_table.metadata_location())? + // .write(serde_json::to_vec(updated_staged_table.metadata())?.into()) + // .await?; + // + // root_namespace_state.update_existing_table_location( + // commit.identifier(), + // updated_staged_table.metadata_location(), + // )?; + // Ok(updated_staged_table) Err(Error::new( ErrorKind::FeatureUnsupported, "MemoryCatalog does not currently support updating tables.", diff --git a/crates/catalog/memory/src/namespace_state.rs b/crates/catalog/memory/src/namespace_state.rs index e324e7a3d..815bf55e9 100644 --- a/crates/catalog/memory/src/namespace_state.rs +++ b/crates/catalog/memory/src/namespace_state.rs @@ -262,6 +262,24 @@ impl NamespaceState { } } + /// TODO fix this + pub(crate) fn update_existing_table_location( + &mut self, + table_ident: &TableIdent, + new_metadata_location: Option<&str>, + ) -> Result<()> { + if new_metadata_location.is_none() { + return Ok(()); + } + + let mut namespace = self.get_mut_namespace(table_ident.namespace())?; + namespace + .table_metadata_locations + .entry(table_ident.name().to_string()) + .insert_entry(new_metadata_location.unwrap().into_string()); + Ok(()) + } + // Inserts the given table or returns an error if it already exists pub(crate) fn insert_new_table( &mut self, diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs new file mode 100644 index 000000000..bcbafd723 --- /dev/null +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -0,0 +1,69 @@ +// 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 crate::transaction::Transaction; +use crate::{Result, TableUpdate}; + +pub type ActionElement<'a> = Box>; + +pub(crate) trait TransactionAction<'a>: Sync { + /// Apply the pending changes and return the uncommitted changes + /// TODO is this even needed? + fn apply(&mut self) -> Result>; + + /// Commit the changes and apply the changes to the associated transaction + fn commit(self) -> Result>; +} + +pub struct SetLocation<'a> { + pub tx: Transaction<'a>, + location: Option, +} + +impl<'a> SetLocation<'a> { + pub fn new(tx: Transaction<'a>) -> Self { + SetLocation { + tx, + location: None + } + } + + pub fn set_location(mut self, location: String) -> Self { + self.location = Some(location); + self + } +} + +impl<'a> TransactionAction<'a> for SetLocation<'a> { + fn apply(&mut self) -> Result> { + if self.location.is_none() { + return Ok(None) + } + Ok(Some(TableUpdate::SetLocation { location: self.location.clone().unwrap() })) + } + + fn commit(mut self) -> Result> { + let location = &mut self.apply()?; + if location.is_none() { + return Ok(self.tx) + } + + self.tx.apply(vec![location.clone().unwrap()], vec![])?; + Ok(self.tx) + // self.tx.actions.push(Box::new(self)); + } +} diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index ba79d60bb..0d21fcd3e 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -17,6 +17,7 @@ //! This module contains transaction api. +mod action; mod append; mod snapshot; mod sort_order; @@ -32,6 +33,7 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; +use crate::transaction::action::{ActionElement, SetLocation}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -40,6 +42,7 @@ use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdat pub struct Transaction<'a> { base_table: &'a Table, current_table: Table, + _actions: Vec>, // TODO unused for now, should we use this to reapply actions? updates: Vec, requirements: Vec, } @@ -50,6 +53,7 @@ impl<'a> Transaction<'a> { Self { base_table: table, current_table: table.clone(), + _actions: vec![], updates: vec![], requirements: vec![], } @@ -67,6 +71,7 @@ impl<'a> Transaction<'a> { Ok(()) } + // TODO deprecate this and move the logic to TransactionAction fn apply( &mut self, updates: Vec, @@ -184,9 +189,8 @@ impl<'a> Transaction<'a> { } /// Set the location of table - pub fn set_location(mut self, location: String) -> Result { - self.apply(vec![TableUpdate::SetLocation { location }], vec![])?; - Ok(self) + pub fn set_location(self) -> Result> { + Ok(SetLocation::new(self)) } /// Commit transaction. @@ -196,6 +200,20 @@ impl<'a> Transaction<'a> { .updates(self.updates) .requirements(self.requirements) .build(); + if self.base_table.metadata() == self.current_table.metadata() { + return Ok(self.current_table); + } + + // TODO add refresh() in catalog? + let refreshed_table = catalog + .load_table(table_commit.identifier()) + .await + .expect(format!("Failed to refresh table {}", table_commit.identifier()).as_str()); + + if self.base_table.metadata() != refreshed_table.metadata() { + // TODO raise a real error and retry + panic!("Stale base table!") + } catalog.update_table(table_commit).await } @@ -212,6 +230,7 @@ mod tests { use crate::table::Table; use crate::transaction::Transaction; use crate::{TableIdent, TableUpdate}; + use crate::transaction::action::TransactionAction; fn make_v1_table() -> Table { let file = File::open(format!( @@ -345,9 +364,12 @@ mod tests { fn test_set_location() { let table = make_v2_table(); let tx = Transaction::new(&table); - let tx = tx - .set_location(String::from("s3://bucket/prefix/new_table")) - .unwrap(); + let set_location = tx + .set_location() + .unwrap() + .set_location(String::from("s3://bucket/prefix/new_table")); + + let tx = set_location.commit().unwrap(); assert_eq!( vec![TableUpdate::SetLocation { From d631b3c93649acc4f075fe8a78fc9ccb30cb829d Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sat, 31 May 2025 15:07:28 -0700 Subject: [PATCH 02/25] Add logic to commit metadata changes (no retry(yet)) --- crates/catalog/memory/src/catalog.rs | 85 +++++++++++++++----- crates/catalog/memory/src/namespace_state.rs | 23 +++++- crates/iceberg/src/catalog/mod.rs | 2 + crates/iceberg/src/error.rs | 2 +- crates/iceberg/src/transaction/action/mod.rs | 21 +++-- crates/iceberg/src/transaction/mod.rs | 54 +++++++++---- 6 files changed, 137 insertions(+), 50 deletions(-) diff --git a/crates/catalog/memory/src/catalog.rs b/crates/catalog/memory/src/catalog.rs index fe4d203dc..11f82d8b6 100644 --- a/crates/catalog/memory/src/catalog.rs +++ b/crates/catalog/memory/src/catalog.rs @@ -29,6 +29,7 @@ use iceberg::{ TableIdent, TableRequirement, TableUpdate, }; use itertools::Itertools; +use regex::Regex; use uuid::Uuid; use crate::namespace_state::NamespaceState; @@ -278,32 +279,74 @@ impl Catalog for MemoryCatalog { /// Update a table to the catalog. async fn update_table(&self, commit: TableCommit) -> Result
{ - // TODO persist the updated metadata - // let mut root_namespace_state = self.root_namespace_state.lock().await; - // let current_table = self.load_table(commit.identifier()).await?; - // let updated_staged_table = update_and_stage_table(Some(¤t_table), &commit)?; - // - // if current_table.metadata() == updated_staged_table.metadata() { - // // no changes - // return Ok(current_table); - // } - // - // // write metadata - // self.file_io - // .new_output(&updated_staged_table.metadata_location())? - // .write(serde_json::to_vec(updated_staged_table.metadata())?.into()) - // .await?; - // - // root_namespace_state.update_existing_table_location( - // commit.identifier(), - // updated_staged_table.metadata_location(), - // )?; - // Ok(updated_staged_table) Err(Error::new( ErrorKind::FeatureUnsupported, "MemoryCatalog does not currently support updating tables.", )) } + + async fn commit_table(&self, base: &Table, current: Table) -> Result
{ + if base.metadata() == current.metadata() { + // no change + return Ok(current); + } + + let mut root_namespace_state = self.root_namespace_state.lock().await; + // TODO: caller needs to retry on the error below + let _ = root_namespace_state + .check_metadata_location(base.identifier(), base.metadata_location())?; + + let next_metadata_version = if let Some(base_metadata_location) = base.metadata_location() { + self.parse_metadata_version(base_metadata_location) + 1 + } else { + 0 + }; + + // write metadata + let metadata_location = format!( + "{}/metadata/{}-{}.metadata.json", + current.metadata().location(), + next_metadata_version, + Uuid::new_v4() + ); + + // TODO instead of using current.metadata(), build a new metadata with some properties like last_updated_ms updated + self.file_io + .new_output(&metadata_location)? + .write(serde_json::to_vec(current.metadata())?.into()) + .await?; + + root_namespace_state + .update_existing_table_location(current.identifier(), current.metadata_location())?; + + // TODO same here, need to update the metadata location + Ok(current) + } +} + +// todo move this to metadata? +fn parse_metadata_version(metadata_location: &str) -> Result { + let pattern = r"(\d+)-([\w-]{36})(?:\.\w+)?\.metadata\.json"; // todo make this constant + + if let Some(metadata_file_name) = metadata_location.split('/').last() { + let re = Regex::new(pattern).expect("Failed to parse regex for metadata file!"); + if let Some(caps) = re.captures(metadata_file_name) { + let metadata_version_str = &caps[1]; + let uuid_str = &caps[2]; + + let metadata_version = metadata_version_str + .parse() + .expect(format!("Invalid metadata version: {metadata_version_str}").as_str()); + let uuid = Uuid::parse_str(uuid_str)?; + + return Ok(metadata_version); + } + } + + Err(Error::new( + ErrorKind::Unexpected, + format!("Unrecognizable metadata location: {metadata_location}"), + )) } #[cfg(test)] diff --git a/crates/catalog/memory/src/namespace_state.rs b/crates/catalog/memory/src/namespace_state.rs index 815bf55e9..c406b6c0a 100644 --- a/crates/catalog/memory/src/namespace_state.rs +++ b/crates/catalog/memory/src/namespace_state.rs @@ -262,7 +262,28 @@ impl NamespaceState { } } - /// TODO fix this + pub(crate) fn check_metadata_location( + &self, + table_ident: &TableIdent, + metadata_location: Option<&str>, + ) -> Result<()> { + let namespace = self.get_namespace(table_ident.namespace())?; + + if namespace + .table_metadata_locations + .get(table_ident.name()) + .map(|s| s.as_str()) + != metadata_location + { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Metadata location does not match for table: {table_ident}!"), + )); + } + + Ok(()) + } + pub(crate) fn update_existing_table_location( &mut self, table_ident: &TableIdent, diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 3457f8361..edad72e15 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -94,6 +94,8 @@ pub trait Catalog: Debug + Sync + Send { /// Update a table to the catalog. async fn update_table(&self, commit: TableCommit) -> Result
; + + async fn commit_table(&self, base: &Table, current: Table) -> Result
; } /// NamespaceIdent represents the identifier of a namespace in the catalog. diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index 37529ee6f..07fd26413 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -39,7 +39,7 @@ pub enum ErrorKind { /// Iceberg data is invalid. /// /// This error is returned when we try to read a table from iceberg but - /// failed to parse it's metadata or data file correctly. + /// failed to parse its metadata or data file correctly. /// /// The table could be invalid or corrupted. DataInvalid, diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index bcbafd723..89792d9d5 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -24,7 +24,7 @@ pub(crate) trait TransactionAction<'a>: Sync { /// Apply the pending changes and return the uncommitted changes /// TODO is this even needed? fn apply(&mut self) -> Result>; - + /// Commit the changes and apply the changes to the associated transaction fn commit(self) -> Result>; } @@ -36,12 +36,9 @@ pub struct SetLocation<'a> { impl<'a> SetLocation<'a> { pub fn new(tx: Transaction<'a>) -> Self { - SetLocation { - tx, - location: None - } + SetLocation { tx, location: None } } - + pub fn set_location(mut self, location: String) -> Self { self.location = Some(location); self @@ -51,17 +48,19 @@ impl<'a> SetLocation<'a> { impl<'a> TransactionAction<'a> for SetLocation<'a> { fn apply(&mut self) -> Result> { if self.location.is_none() { - return Ok(None) + return Ok(None); } - Ok(Some(TableUpdate::SetLocation { location: self.location.clone().unwrap() })) + Ok(Some(TableUpdate::SetLocation { + location: self.location.clone().unwrap(), + })) } - + fn commit(mut self) -> Result> { let location = &mut self.apply()?; if location.is_none() { - return Ok(self.tx) + return Ok(self.tx); } - + self.tx.apply(vec![location.clone().unwrap()], vec![])?; Ok(self.tx) // self.tx.actions.push(Box::new(self)); diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 0d21fcd3e..0cc1fb0f5 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -71,7 +71,6 @@ impl<'a> Transaction<'a> { Ok(()) } - // TODO deprecate this and move the logic to TransactionAction fn apply( &mut self, updates: Vec, @@ -193,29 +192,52 @@ impl<'a> Transaction<'a> { Ok(SetLocation::new(self)) } + fn refresh(&mut self, refreshed: Table) { + self.base_table = &refreshed; + self.current_table = refreshed.clone(); + } + /// Commit transaction. - pub async fn commit(self, catalog: &dyn Catalog) -> Result
{ - let table_commit = TableCommit::builder() - .ident(self.base_table.identifier().clone()) - .updates(self.updates) - .requirements(self.requirements) - .build(); + pub async fn commit(mut self, catalog: &dyn Catalog) -> Result
{ + // let table_commit = TableCommit::builder() + // .ident(self.base_table.identifier().clone()) + // .updates(self.updates) + // .requirements(self.requirements) + // .build(); if self.base_table.metadata() == self.current_table.metadata() { return Ok(self.current_table); } // TODO add refresh() in catalog? - let refreshed_table = catalog - .load_table(table_commit.identifier()) + let refreshed = catalog + .load_table(self.base_table.identifier()) .await - .expect(format!("Failed to refresh table {}", table_commit.identifier()).as_str()); + .expect(format!("Failed to refresh table {}", self.base_table.identifier()).as_str()); - if self.base_table.metadata() != refreshed_table.metadata() { - // TODO raise a real error and retry - panic!("Stale base table!") + if self.base_table.metadata() != refreshed.metadata() + || self.base_table.metadata_location() != refreshed.metadata_location() + { + self.refresh(refreshed); + self.apply(self.updates, self.requirements) // TODO need create new requirements based on the refreshed table + .expect("Failed to re-apply updates"); // re-apply updates + // TODO retry on this error + return Err(Error::new( + ErrorKind::DataInvalid, + "Cannot commit: stale base table metadata".to_string(), + )); } - catalog.update_table(table_commit).await + if self.base_table.metadata() == self.current_table.metadata() + && self.base_table.metadata_location() == self.current_table.metadata_location() + { + // nothing to commit, return current table + return Ok(self.current_table); + } + + catalog + .commit_table(self.base_table, self.current_table) + .await + // catalog.update_table(table_commit).await } } @@ -229,8 +251,8 @@ mod tests { use crate::spec::{FormatVersion, TableMetadata}; use crate::table::Table; use crate::transaction::Transaction; - use crate::{TableIdent, TableUpdate}; use crate::transaction::action::TransactionAction; + use crate::{TableIdent, TableUpdate}; fn make_v1_table() -> Table { let file = File::open(format!( @@ -368,7 +390,7 @@ mod tests { .set_location() .unwrap() .set_location(String::from("s3://bucket/prefix/new_table")); - + let tx = set_location.commit().unwrap(); assert_eq!( From 7a6f3d1e2f5812825f4e279e61cf672ea41f1c4e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sat, 31 May 2025 21:49:59 -0700 Subject: [PATCH 03/25] refresh-ish action --- crates/iceberg/src/transaction/action/mod.rs | 46 ++++++------------ crates/iceberg/src/transaction/mod.rs | 51 ++++++++++---------- 2 files changed, 41 insertions(+), 56 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 89792d9d5..618b88d0b 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -18,25 +18,20 @@ use crate::transaction::Transaction; use crate::{Result, TableUpdate}; -pub type ActionElement<'a> = Box>; +pub type PendingAction = Box; -pub(crate) trait TransactionAction<'a>: Sync { - /// Apply the pending changes and return the uncommitted changes - /// TODO is this even needed? - fn apply(&mut self) -> Result>; - - /// Commit the changes and apply the changes to the associated transaction - fn commit(self) -> Result>; +pub(crate) trait TransactionAction: Sync { + /// Commit the changes and apply the changes to the transaction, return the updated transaction + fn commit(self, tx: Transaction) -> Result; } -pub struct SetLocation<'a> { - pub tx: Transaction<'a>, - location: Option, +pub struct SetLocation { + pub location: Option, } -impl<'a> SetLocation<'a> { - pub fn new(tx: Transaction<'a>) -> Self { - SetLocation { tx, location: None } +impl SetLocation { + pub fn new() -> Self { + SetLocation { location: None } } pub fn set_location(mut self, location: String) -> Self { @@ -45,24 +40,13 @@ impl<'a> SetLocation<'a> { } } -impl<'a> TransactionAction<'a> for SetLocation<'a> { - fn apply(&mut self) -> Result> { - if self.location.is_none() { - return Ok(None); - } - Ok(Some(TableUpdate::SetLocation { - location: self.location.clone().unwrap(), - })) - } - - fn commit(mut self) -> Result> { - let location = &mut self.apply()?; - if location.is_none() { - return Ok(self.tx); +impl TransactionAction for SetLocation { + fn commit(self, mut tx: Transaction) -> Result { + if let Some(location) = self.location.clone() { + tx.apply(vec![TableUpdate::SetLocation { location }], vec![])?; } + tx.actions.push(Box::new(self)); - self.tx.apply(vec![location.clone().unwrap()], vec![])?; - Ok(self.tx) - // self.tx.actions.push(Box::new(self)); + Ok(tx) } } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 0cc1fb0f5..c406602d7 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -33,7 +33,7 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{ActionElement, SetLocation}; +use crate::transaction::action::{SetLocation, TransactionAction, PendingAction}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -42,7 +42,7 @@ use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdat pub struct Transaction<'a> { base_table: &'a Table, current_table: Table, - _actions: Vec>, // TODO unused for now, should we use this to reapply actions? + actions: Vec, updates: Vec, requirements: Vec, } @@ -53,12 +53,21 @@ impl<'a> Transaction<'a> { Self { base_table: table, current_table: table.clone(), - _actions: vec![], + actions: vec![], updates: vec![], requirements: vec![], } } + pub fn refresh(old_tx: Transaction<'a>, refreshed: Table) -> Result { + let mut new_tx = Transaction::new(&refreshed.clone()); + for action in &old_tx.actions { + new_tx = action.commit(new_tx)? + } + + Ok(new_tx) + } + fn update_table_metadata(&mut self, updates: &[TableUpdate]) -> Result<()> { let mut metadata_builder = self.current_table.metadata().clone().into_builder(None); for update in updates { @@ -188,17 +197,12 @@ impl<'a> Transaction<'a> { } /// Set the location of table - pub fn set_location(self) -> Result> { - Ok(SetLocation::new(self)) - } - - fn refresh(&mut self, refreshed: Table) { - self.base_table = &refreshed; - self.current_table = refreshed.clone(); + pub fn set_location(self, location: String) -> Result> { + Ok(SetLocation::new().set_location(location).commit(self)?) } /// Commit transaction. - pub async fn commit(mut self, catalog: &dyn Catalog) -> Result
{ + pub async fn commit(mut self: Transaction<'a>, catalog: &dyn Catalog) -> Result
{ // let table_commit = TableCommit::builder() // .ident(self.base_table.identifier().clone()) // .updates(self.updates) @@ -217,14 +221,14 @@ impl<'a> Transaction<'a> { if self.base_table.metadata() != refreshed.metadata() || self.base_table.metadata_location() != refreshed.metadata_location() { - self.refresh(refreshed); - self.apply(self.updates, self.requirements) // TODO need create new requirements based on the refreshed table - .expect("Failed to re-apply updates"); // re-apply updates - // TODO retry on this error - return Err(Error::new( - ErrorKind::DataInvalid, - "Cannot commit: stale base table metadata".to_string(), - )); + // refresh table + let new_tx = Transaction::refresh(self, refreshed)?; + return new_tx.commit(catalog).await + // TODO instead of refreshing directly, retry on this error + // return Err(Error::new( + // ErrorKind::DataInvalid, + // "Cannot commit: stale base table metadata".to_string(), + // )); } if self.base_table.metadata() == self.current_table.metadata() @@ -386,12 +390,9 @@ mod tests { fn test_set_location() { let table = make_v2_table(); let tx = Transaction::new(&table); - let set_location = tx - .set_location() - .unwrap() - .set_location(String::from("s3://bucket/prefix/new_table")); - - let tx = set_location.commit().unwrap(); + let tx = tx + .set_location(String::from("s3://bucket/prefix/new_table")) + .unwrap(); assert_eq!( vec![TableUpdate::SetLocation { From d481d69ec21eae9d26ffa760456c009d9f6a764b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sat, 31 May 2025 22:24:01 -0700 Subject: [PATCH 04/25] minor --- crates/iceberg/src/transaction/action/mod.rs | 3 ++- crates/iceberg/src/transaction/mod.rs | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 618b88d0b..ffda1792b 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -21,7 +21,8 @@ use crate::{Result, TableUpdate}; pub type PendingAction = Box; pub(crate) trait TransactionAction: Sync { - /// Commit the changes and apply the changes to the transaction, return the updated transaction + /// Commit the changes and apply the changes to the transaction, + /// return the transaction with the updated current_table fn commit(self, tx: Transaction) -> Result; } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index c406602d7..5556615cc 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -33,7 +33,7 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{SetLocation, TransactionAction, PendingAction}; +use crate::transaction::action::{PendingAction, SetLocation, TransactionAction}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -64,7 +64,7 @@ impl<'a> Transaction<'a> { for action in &old_tx.actions { new_tx = action.commit(new_tx)? } - + Ok(new_tx) } @@ -223,7 +223,7 @@ impl<'a> Transaction<'a> { { // refresh table let new_tx = Transaction::refresh(self, refreshed)?; - return new_tx.commit(catalog).await + return new_tx.commit(catalog).await; // TODO instead of refreshing directly, retry on this error // return Err(Error::new( // ErrorKind::DataInvalid, From 434cdb60c1800b1aeb1040feb79a99bc59e29a7b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sun, 1 Jun 2025 22:39:01 -0700 Subject: [PATCH 05/25] box things up --- crates/iceberg/src/transaction/action/mod.rs | 12 ++++++------ crates/iceberg/src/transaction/mod.rs | 11 +++++++---- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index ffda1792b..ae1bffd62 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -18,12 +18,12 @@ use crate::transaction::Transaction; use crate::{Result, TableUpdate}; -pub type PendingAction = Box; +pub type PendingAction<'a> = Box>; -pub(crate) trait TransactionAction: Sync { +pub(crate) trait TransactionAction<'a>: Sync { /// Commit the changes and apply the changes to the transaction, /// return the transaction with the updated current_table - fn commit(self, tx: Transaction) -> Result; + fn commit(self: Box, tx: Transaction<'a>) -> Result; } pub struct SetLocation { @@ -41,12 +41,12 @@ impl SetLocation { } } -impl TransactionAction for SetLocation { - fn commit(self, mut tx: Transaction) -> Result { +impl<'a> TransactionAction<'a> for SetLocation { + fn commit(self: Box, mut tx: Transaction<'a>) -> Result> { if let Some(location) = self.location.clone() { tx.apply(vec![TableUpdate::SetLocation { location }], vec![])?; } - tx.actions.push(Box::new(self)); + tx.actions.push(self); Ok(tx) } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 5556615cc..8ed5fbc24 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -42,7 +42,7 @@ use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdat pub struct Transaction<'a> { base_table: &'a Table, current_table: Table, - actions: Vec, + actions: Vec>, updates: Vec, requirements: Vec, } @@ -60,8 +60,9 @@ impl<'a> Transaction<'a> { } pub fn refresh(old_tx: Transaction<'a>, refreshed: Table) -> Result { - let mut new_tx = Transaction::new(&refreshed.clone()); - for action in &old_tx.actions { + let refreshed_clone = refreshed.clone(); + let mut new_tx = Transaction::new(&refreshed_clone); + for action in old_tx.actions { new_tx = action.commit(new_tx)? } @@ -198,7 +199,9 @@ impl<'a> Transaction<'a> { /// Set the location of table pub fn set_location(self, location: String) -> Result> { - Ok(SetLocation::new().set_location(location).commit(self)?) + let set_location = SetLocation::new().set_location(location); + let tx = Box::new(set_location).commit(self)?; + Ok(tx) } /// Commit transaction. From 11260a0384540e617bcf8eeb2ce9805eeef8007e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 2 Jun 2025 14:42:29 -0700 Subject: [PATCH 06/25] clean up tx.commit --- crates/iceberg/src/catalog/mod.rs | 21 ++++- crates/iceberg/src/transaction/action/mod.rs | 12 ++- crates/iceberg/src/transaction/mod.rs | 85 +++++++++----------- 3 files changed, 66 insertions(+), 52 deletions(-) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index edad72e15..0a6f47657 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -30,8 +30,8 @@ use uuid::Uuid; use crate::spec::{ FormatVersion, PartitionStatisticsFile, Schema, SchemaId, Snapshot, SnapshotReference, - SortOrder, StatisticsFile, TableMetadata, TableMetadataBuilder, UnboundPartitionSpec, - ViewFormatVersion, ViewRepresentations, ViewVersion, + SortOrder, StatisticsFile, TableMetadata, TableMetadataBuildResult, TableMetadataBuilder, + UnboundPartitionSpec, ViewFormatVersion, ViewRepresentations, ViewVersion, }; use crate::table::Table; use crate::{Error, ErrorKind, Result}; @@ -302,6 +302,23 @@ impl TableCommit { pub fn take_updates(&mut self) -> Vec { take(&mut self.updates) } + + /// Apply updates to a table + pub fn apply(&mut self, table: &Table) -> Result { + // 1. check requirements + for requirement in self.requirements { + requirement.check(Some(table.metadata()))?; + } + + // 2. Apply updates to metadata builder + let mut metadata_builder = table.metadata().clone().into_builder(None); + + for update in self.updates { + metadata_builder = update.apply(metadata_builder)?; + } + + metadata_builder.build() + } } /// TableRequirement represents a requirement for a table in the catalog. diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index ae1bffd62..10ee917fe 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -23,7 +23,13 @@ pub type PendingAction<'a> = Box>; pub(crate) trait TransactionAction<'a>: Sync { /// Commit the changes and apply the changes to the transaction, /// return the transaction with the updated current_table - fn commit(self: Box, tx: Transaction<'a>) -> Result; + fn commit(self: Box, tx: &'a mut Transaction<'a>) -> Result<()>; +} + +impl<'a> Clone for PendingAction<'a> { + fn clone(&self) -> Self { + Box::new(self) + } } pub struct SetLocation { @@ -42,12 +48,12 @@ impl SetLocation { } impl<'a> TransactionAction<'a> for SetLocation { - fn commit(self: Box, mut tx: Transaction<'a>) -> Result> { + fn commit(self: Box, tx: &'a mut Transaction<'a>) -> Result<()> { if let Some(location) = self.location.clone() { tx.apply(vec![TableUpdate::SetLocation { location }], vec![])?; } tx.actions.push(self); - Ok(tx) + Ok(()) } } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 8ed5fbc24..c7dc0cbbe 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -59,16 +59,6 @@ impl<'a> Transaction<'a> { } } - pub fn refresh(old_tx: Transaction<'a>, refreshed: Table) -> Result { - let refreshed_clone = refreshed.clone(); - let mut new_tx = Transaction::new(&refreshed_clone); - for action in old_tx.actions { - new_tx = action.commit(new_tx)? - } - - Ok(new_tx) - } - fn update_table_metadata(&mut self, updates: &[TableUpdate]) -> Result<()> { let mut metadata_builder = self.current_table.metadata().clone().into_builder(None); for update in updates { @@ -198,53 +188,55 @@ impl<'a> Transaction<'a> { } /// Set the location of table - pub fn set_location(self, location: String) -> Result> { + pub fn set_location(mut self, location: String) -> Result> { let set_location = SetLocation::new().set_location(location); - let tx = Box::new(set_location).commit(self)?; - Ok(tx) + let _commit = Box::new(set_location).commit(&mut self)?; + Ok(self) } /// Commit transaction. - pub async fn commit(mut self: Transaction<'a>, catalog: &dyn Catalog) -> Result
{ - // let table_commit = TableCommit::builder() - // .ident(self.base_table.identifier().clone()) - // .updates(self.updates) - // .requirements(self.requirements) - // .build(); - if self.base_table.metadata() == self.current_table.metadata() { + pub async fn commit(mut self, catalog: &dyn Catalog) -> Result
{ + let base_table_identifier = self.base_table.identifier().to_owned(); + + if self.actions.is_empty() + || (self.base_table.metadata() == self.current_table.metadata() + && self.base_table.metadata_location() == self.current_table.metadata_location()) + { + // nothing to commit return Ok(self.current_table); } - // TODO add refresh() in catalog? - let refreshed = catalog - .load_table(self.base_table.identifier()) - .await - .expect(format!("Failed to refresh table {}", self.base_table.identifier()).as_str()); + // TODO use an actual retry + for _attempt in 0..3 { + let refreshed = catalog + .load_table(&base_table_identifier.clone()) + .await + .expect(format!("Failed to refresh table {}", base_table_identifier).as_str()); - if self.base_table.metadata() != refreshed.metadata() - || self.base_table.metadata_location() != refreshed.metadata_location() - { - // refresh table - let new_tx = Transaction::refresh(self, refreshed)?; - return new_tx.commit(catalog).await; - // TODO instead of refreshing directly, retry on this error - // return Err(Error::new( - // ErrorKind::DataInvalid, - // "Cannot commit: stale base table metadata".to_string(), - // )); - } + if self.base_table.metadata() != refreshed.metadata() + || self.base_table.metadata_location() != refreshed.metadata_location() + { + // current base is stale, use refreshed as base and re-apply transaction actions + self.base_table = &refreshed.clone(); + self.current_table = refreshed.clone(); - if self.base_table.metadata() == self.current_table.metadata() - && self.base_table.metadata_location() == self.current_table.metadata_location() - { - // nothing to commit, return current table - return Ok(self.current_table); + let pending_actions = self.actions.clone(); + + for action in pending_actions { + action.commit(&mut self)?; + } + } + + let table_commit = TableCommit::builder() + .ident(base_table_identifier.clone()) + .updates(self.updates) + .requirements(self.requirements) + .build(); + + return catalog.update_table(table_commit).await; } - catalog - .commit_table(self.base_table, self.current_table) - .await - // catalog.update_table(table_commit).await + Err(Error::new(ErrorKind::DataInvalid, "Failed to commit!")) } } @@ -258,7 +250,6 @@ mod tests { use crate::spec::{FormatVersion, TableMetadata}; use crate::table::Table; use crate::transaction::Transaction; - use crate::transaction::action::TransactionAction; use crate::{TableIdent, TableUpdate}; fn make_v1_table() -> Table { From a64cc21fe0c00a32ba2f0a7e6ee7b492b6742090 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 2 Jun 2025 16:26:29 -0700 Subject: [PATCH 07/25] more cleanup --- crates/iceberg/src/transaction/action/mod.rs | 47 +++++++++++++++----- crates/iceberg/src/transaction/mod.rs | 28 +++++++----- 2 files changed, 52 insertions(+), 23 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 10ee917fe..bc267eced 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -15,20 +15,35 @@ // specific language governing permissions and limitations // under the License. +use std::mem::take; use crate::transaction::Transaction; -use crate::{Result, TableUpdate}; +use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; -pub type PendingAction<'a> = Box>; +pub type PendingAction = Box; -pub(crate) trait TransactionAction<'a>: Sync { +pub(crate) trait TransactionAction: Sync { /// Commit the changes and apply the changes to the transaction, /// return the transaction with the updated current_table - fn commit(self: Box, tx: &'a mut Transaction<'a>) -> Result<()>; + fn commit(self: Box) -> Result; } -impl<'a> Clone for PendingAction<'a> { - fn clone(&self) -> Self { - Box::new(self) +pub struct TransactionActionCommitResult { + action: Option, + updates: Vec, + requirements: Vec, +} + +impl TransactionActionCommitResult { + pub fn take_action(&mut self) -> Option { + take(&mut self.action) + } + + pub fn take_updates(&mut self) -> Vec { + take(&mut self.updates) + } + + pub fn take_requirements(&mut self) -> Vec { + take(&mut self.requirements) } } @@ -47,13 +62,21 @@ impl SetLocation { } } -impl<'a> TransactionAction<'a> for SetLocation { - fn commit(self: Box, tx: &'a mut Transaction<'a>) -> Result<()> { +impl TransactionAction for SetLocation { + fn commit(self: Box) -> Result { + let updates: Vec; + let requirements: Vec; if let Some(location) = self.location.clone() { - tx.apply(vec![TableUpdate::SetLocation { location }], vec![])?; + updates = vec![TableUpdate::SetLocation { location }]; + requirements = vec![]; + } else { + return Err(Error::new(ErrorKind::DataInvalid, "TODO error msg")) } - tx.actions.push(self); - Ok(()) + Ok(TransactionActionCommitResult { + action: Some(self), + updates, + requirements + }) } } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index c7dc0cbbe..b20bd6252 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -33,7 +33,7 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{PendingAction, SetLocation, TransactionAction}; +use crate::transaction::action::{PendingAction, SetLocation, TransactionAction, TransactionActionCommitResult}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -42,7 +42,7 @@ use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdat pub struct Transaction<'a> { base_table: &'a Table, current_table: Table, - actions: Vec>, + actions: Vec, updates: Vec, requirements: Vec, } @@ -70,6 +70,12 @@ impl<'a> Transaction<'a> { Ok(()) } + + fn apply_commit_result(&mut self, + mut tx_commit_res: TransactionActionCommitResult) -> Result<()> { + self.actions.push(tx_commit_res.take_action().unwrap()); + self.apply(tx_commit_res.take_updates(), tx_commit_res.take_requirements()) + } fn apply( &mut self, @@ -188,14 +194,14 @@ impl<'a> Transaction<'a> { } /// Set the location of table - pub fn set_location(mut self, location: String) -> Result> { + pub fn set_location(mut self, location: String) -> Result { let set_location = SetLocation::new().set_location(location); - let _commit = Box::new(set_location).commit(&mut self)?; + self.apply_commit_result(Box::new(set_location).commit()?).expect("Some error msg"); Ok(self) } /// Commit transaction. - pub async fn commit(mut self, catalog: &dyn Catalog) -> Result
{ + pub async fn commit(&mut self, catalog: &dyn Catalog) -> Result
{ let base_table_identifier = self.base_table.identifier().to_owned(); if self.actions.is_empty() @@ -203,7 +209,7 @@ impl<'a> Transaction<'a> { && self.base_table.metadata_location() == self.current_table.metadata_location()) { // nothing to commit - return Ok(self.current_table); + return Ok(self.current_table.clone()); } // TODO use an actual retry @@ -220,17 +226,17 @@ impl<'a> Transaction<'a> { self.base_table = &refreshed.clone(); self.current_table = refreshed.clone(); - let pending_actions = self.actions.clone(); + // let pending_actions = self.actions.clone(); - for action in pending_actions { - action.commit(&mut self)?; + for action in self.actions { + action.commit()?; } } let table_commit = TableCommit::builder() .ident(base_table_identifier.clone()) - .updates(self.updates) - .requirements(self.requirements) + .updates(self.updates.clone()) + .requirements(self.requirements.clone()) .build(); return catalog.update_table(table_commit).await; From f2c3399ed7e77277d9f5b5a1d1fb92bd4dd1572b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 2 Jun 2025 17:16:51 -0700 Subject: [PATCH 08/25] clean up, need to fix the referenced logic.. or that could be fixed by using a real retry operator? --- crates/iceberg/src/catalog/mod.rs | 15 ++++++---- crates/iceberg/src/transaction/action/mod.rs | 13 ++++---- crates/iceberg/src/transaction/mod.rs | 31 +++++++++++++------- 3 files changed, 39 insertions(+), 20 deletions(-) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 0a6f47657..26c7e85b7 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -21,6 +21,7 @@ use std::collections::HashMap; use std::fmt::{Debug, Display}; use std::mem::take; use std::ops::Deref; +use std::sync::Arc; use _serde::deserialize_snapshot; use async_trait::async_trait; @@ -30,7 +31,7 @@ use uuid::Uuid; use crate::spec::{ FormatVersion, PartitionStatisticsFile, Schema, SchemaId, Snapshot, SnapshotReference, - SortOrder, StatisticsFile, TableMetadata, TableMetadataBuildResult, TableMetadataBuilder, + SortOrder, StatisticsFile, TableMetadata, TableMetadataBuilder, UnboundPartitionSpec, ViewFormatVersion, ViewRepresentations, ViewVersion, }; use crate::table::Table; @@ -304,20 +305,24 @@ impl TableCommit { } /// Apply updates to a table - pub fn apply(&mut self, table: &Table) -> Result { + pub fn apply(&mut self, mut table: Table) -> Result
{ // 1. check requirements - for requirement in self.requirements { + let requirements = self.take_requirements(); + for requirement in requirements { requirement.check(Some(table.metadata()))?; } // 2. Apply updates to metadata builder let mut metadata_builder = table.metadata().clone().into_builder(None); - for update in self.updates { + let updates = self.take_updates(); + for update in updates { metadata_builder = update.apply(metadata_builder)?; } - metadata_builder.build() + table.with_metadata(Arc::new(metadata_builder.build()?.metadata)); + + Ok(table) } } diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index bc267eced..72e4c4f5c 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -16,7 +16,7 @@ // under the License. use std::mem::take; -use crate::transaction::Transaction; + use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; pub type PendingAction = Box; @@ -37,11 +37,11 @@ impl TransactionActionCommitResult { pub fn take_action(&mut self) -> Option { take(&mut self.action) } - + pub fn take_updates(&mut self) -> Vec { take(&mut self.updates) } - + pub fn take_requirements(&mut self) -> Vec { take(&mut self.requirements) } @@ -70,13 +70,16 @@ impl TransactionAction for SetLocation { updates = vec![TableUpdate::SetLocation { location }]; requirements = vec![]; } else { - return Err(Error::new(ErrorKind::DataInvalid, "TODO error msg")) + return Err(Error::new( + ErrorKind::DataInvalid, + "Location is not set for SetLocation!", + )); } Ok(TransactionActionCommitResult { action: Some(self), updates, - requirements + requirements, }) } } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index b20bd6252..602a246a3 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -33,7 +33,9 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{PendingAction, SetLocation, TransactionAction, TransactionActionCommitResult}; +use crate::transaction::action::{ + PendingAction, SetLocation, TransactionAction, TransactionActionCommitResult, +}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -70,11 +72,20 @@ impl<'a> Transaction<'a> { Ok(()) } - - fn apply_commit_result(&mut self, - mut tx_commit_res: TransactionActionCommitResult) -> Result<()> { - self.actions.push(tx_commit_res.take_action().unwrap()); - self.apply(tx_commit_res.take_updates(), tx_commit_res.take_requirements()) + + fn apply_commit_result( + &mut self, + mut tx_commit_res: TransactionActionCommitResult, + ) -> Result<()> { + if let Some(action) = tx_commit_res.take_action() { + self.actions.push(action); + return self.apply( + tx_commit_res.take_updates(), + tx_commit_res.take_requirements(), + ); + } + + Err(Error::new(ErrorKind::DataInvalid, "Action cannot be none!")) } fn apply( @@ -196,7 +207,8 @@ impl<'a> Transaction<'a> { /// Set the location of table pub fn set_location(mut self, location: String) -> Result { let set_location = SetLocation::new().set_location(location); - self.apply_commit_result(Box::new(set_location).commit()?).expect("Some error msg"); + self.apply_commit_result(Box::new(set_location).commit()?) + .expect("Some error msg"); Ok(self) } @@ -226,10 +238,9 @@ impl<'a> Transaction<'a> { self.base_table = &refreshed.clone(); self.current_table = refreshed.clone(); - // let pending_actions = self.actions.clone(); - for action in self.actions { - action.commit()?; + self.apply_commit_result(action.commit()?) + .expect("Failed to apply updates!"); } } From f7cb068e545951e6e30622e12685cb00d831c49b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 2 Jun 2025 22:34:47 -0700 Subject: [PATCH 09/25] More clean up.. retry is not working yet --- Cargo.lock | 15 +++- crates/iceberg/src/catalog/mod.rs | 4 +- crates/iceberg/src/transaction/action/mod.rs | 10 +-- crates/iceberg/src/transaction/mod.rs | 86 ++++++++++++-------- 4 files changed, 72 insertions(+), 43 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6528d13f6..4e9bd47c9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1029,9 +1029,9 @@ dependencies = [ [[package]] name = "backon" -version = "1.3.0" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba5289ec98f68f28dd809fd601059e6aa908bb8f6108620930828283d4ee23d7" +checksum = "302eaff5357a264a2c42f127ecb8bac761cf99749fc3dc95677e2743991f99e7" dependencies = [ "fastrand", "gloo-timers", @@ -3523,6 +3523,7 @@ dependencies = [ "tera", "thrift", "tokio", + "tokio-retry2", "typed-builder 0.20.1", "url", "uuid", @@ -7012,6 +7013,16 @@ dependencies = [ "syn 2.0.101", ] +[[package]] +name = "tokio-retry2" +version = "0.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1264d076dd34560544a2799e40e457bd07c43d30f4a845686b031bcd8455c84f" +dependencies = [ + "pin-project", + "tokio", +] + [[package]] name = "tokio-rustls" version = "0.24.1" diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 26c7e85b7..bf5059421 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -31,8 +31,8 @@ use uuid::Uuid; use crate::spec::{ FormatVersion, PartitionStatisticsFile, Schema, SchemaId, Snapshot, SnapshotReference, - SortOrder, StatisticsFile, TableMetadata, TableMetadataBuilder, - UnboundPartitionSpec, ViewFormatVersion, ViewRepresentations, ViewVersion, + SortOrder, StatisticsFile, TableMetadata, TableMetadataBuilder, UnboundPartitionSpec, + ViewFormatVersion, ViewRepresentations, ViewVersion, }; use crate::table::Table; use crate::{Error, ErrorKind, Result}; diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 72e4c4f5c..11024763d 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -24,16 +24,16 @@ pub type PendingAction = Box; pub(crate) trait TransactionAction: Sync { /// Commit the changes and apply the changes to the transaction, /// return the transaction with the updated current_table - fn commit(self: Box) -> Result; + fn commit(self: Box) -> Result; } -pub struct TransactionActionCommitResult { +pub struct TransactionActionCommit { action: Option, updates: Vec, requirements: Vec, } -impl TransactionActionCommitResult { +impl TransactionActionCommit { pub fn take_action(&mut self) -> Option { take(&mut self.action) } @@ -63,7 +63,7 @@ impl SetLocation { } impl TransactionAction for SetLocation { - fn commit(self: Box) -> Result { + fn commit(self: Box) -> Result { let updates: Vec; let requirements: Vec; if let Some(location) = self.location.clone() { @@ -76,7 +76,7 @@ impl TransactionAction for SetLocation { )); } - Ok(TransactionActionCommitResult { + Ok(TransactionActionCommit { action: Some(self), updates, requirements, diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 602a246a3..e9ef03477 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -24,17 +24,18 @@ mod sort_order; use std::cmp::Ordering; use std::collections::HashMap; -use std::mem::discriminant; +use std::mem::{discriminant, take}; use std::sync::Arc; - use uuid::Uuid; +use tokio_retry2::{Retry, RetryError}; +use tokio_retry2::strategy::ExponentialBackoff; use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; use crate::transaction::action::{ - PendingAction, SetLocation, TransactionAction, TransactionActionCommitResult, + PendingAction, SetLocation, TransactionAction, TransactionActionCommit, }; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; @@ -73,10 +74,7 @@ impl<'a> Transaction<'a> { Ok(()) } - fn apply_commit_result( - &mut self, - mut tx_commit_res: TransactionActionCommitResult, - ) -> Result<()> { + fn apply_commit_result(&mut self, mut tx_commit_res: TransactionActionCommit) -> Result<()> { if let Some(action) = tx_commit_res.take_action() { self.actions.push(action); return self.apply( @@ -214,8 +212,6 @@ impl<'a> Transaction<'a> { /// Commit transaction. pub async fn commit(&mut self, catalog: &dyn Catalog) -> Result
{ - let base_table_identifier = self.base_table.identifier().to_owned(); - if self.actions.is_empty() || (self.base_table.metadata() == self.current_table.metadata() && self.base_table.metadata_location() == self.current_table.metadata_location()) @@ -224,36 +220,58 @@ impl<'a> Transaction<'a> { return Ok(self.current_table.clone()); } - // TODO use an actual retry - for _attempt in 0..3 { - let refreshed = catalog - .load_table(&base_table_identifier.clone()) - .await - .expect(format!("Failed to refresh table {}", base_table_identifier).as_str()); + // TODO revisit the retry config + let retry_strategy = ExponentialBackoff::from_millis(10) + .take(3); // limit to 3 retries - if self.base_table.metadata() != refreshed.metadata() - || self.base_table.metadata_location() != refreshed.metadata_location() - { - // current base is stale, use refreshed as base and re-apply transaction actions - self.base_table = &refreshed.clone(); - self.current_table = refreshed.clone(); - - for action in self.actions { - self.apply_commit_result(action.commit()?) - .expect("Failed to apply updates!"); - } - } + Retry::spawn(retry_strategy, + || { + let catalog = catalog.clone(); + self.do_commit(catalog) + } ).await + } - let table_commit = TableCommit::builder() - .ident(base_table_identifier.clone()) - .updates(self.updates.clone()) - .requirements(self.requirements.clone()) - .build(); + async fn do_commit(&mut self, catalog: &dyn Catalog) -> std::result::Result> { + let base_table_identifier = self.base_table.identifier().to_owned(); + + let refreshed = catalog + .load_table(&base_table_identifier.clone()) + .await + .expect(format!("Failed to refresh table {}", base_table_identifier).as_str()); + + if self.base_table.metadata() != refreshed.metadata() + || self.base_table.metadata_location() != refreshed.metadata_location() + { + // current base is stale, use refreshed as base and re-apply transaction actions + self.base_table = &refreshed.clone(); + self.current_table = refreshed.clone(); - return catalog.update_table(table_commit).await; + let pending_actions = take(&mut self.actions); + + for action in pending_actions { + self.apply_commit_result(action.commit()?) + .expect("Failed to apply updates!"); + } } - Err(Error::new(ErrorKind::DataInvalid, "Failed to commit!")) + let table_commit = TableCommit::builder() + .ident(base_table_identifier.clone()) + .updates(self.updates.clone()) + .requirements(self.requirements.clone()) + .build(); + + let result = catalog + .update_table(table_commit) + .await; + + match result { + Ok(table) => Ok(table), + // TODO revisit the error kind + Err(e) if e.kind() == ErrorKind::DataInvalid => { + Err(RetryError::transient(e)) + } + Err(e) => Err(RetryError::permanent(e)) + } } } From 2bfc5009cc3778fde24f9e10bd159e65812dd38f Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 11:24:01 -0700 Subject: [PATCH 10/25] Update crates/iceberg/src/transaction/action/mod.rs Co-authored-by: Renjie Liu --- crates/iceberg/src/transaction/action/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 11024763d..bc1063056 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -19,7 +19,7 @@ use std::mem::take; use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; -pub type PendingAction = Box; +pub type BoxedTransactionAction = Arc; pub(crate) trait TransactionAction: Sync { /// Commit the changes and apply the changes to the transaction, From e881811e21b5335f998e301a36e3f002b84edeca Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 11:24:36 -0700 Subject: [PATCH 11/25] Update crates/iceberg/src/transaction/action/mod.rs Co-authored-by: Renjie Liu --- crates/iceberg/src/transaction/action/mod.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index bc1063056..2b6b87324 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -21,10 +21,11 @@ use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; pub type BoxedTransactionAction = Arc; -pub(crate) trait TransactionAction: Sync { +#[async_trait] +pub(crate) trait TransactionAction: Sync + Send { /// Commit the changes and apply the changes to the transaction, /// return the transaction with the updated current_table - fn commit(self: Box) -> Result; + fn commit(self: Arc, tx: &Transaction) -> Result; } pub struct TransactionActionCommit { From c2baf4eb822c8970591593e76070aaa7a92a5088 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 11:50:50 -0700 Subject: [PATCH 12/25] Have tx own base table, mark tx_commit_res as not used, need to fix retry --- crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/transaction/action/mod.rs | 23 +++++------ crates/iceberg/src/transaction/append.rs | 24 ++++++------ crates/iceberg/src/transaction/mod.rs | 40 ++++++++++---------- crates/iceberg/src/transaction/snapshot.rs | 10 ++--- crates/iceberg/src/transaction/sort_order.rs | 10 ++--- 6 files changed, 54 insertions(+), 54 deletions(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 3324a4e9e..9d28c0d4a 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -55,6 +55,7 @@ arrow-select = { workspace = true } arrow-string = { workspace = true } async-std = { workspace = true, optional = true, features = ["attributes"] } async-trait = { workspace = true } +tokio-retry2 = { version = "0.5.7" } base64 = { workspace = true } bimap = { workspace = true } bytes = { workspace = true } diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 2b6b87324..68739d8ee 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -16,8 +16,10 @@ // under the License. use std::mem::take; - +use std::sync::Arc; +use async_trait::async_trait; use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; +use crate::transaction::Transaction; pub type BoxedTransactionAction = Arc; @@ -25,17 +27,18 @@ pub type BoxedTransactionAction = Arc; pub(crate) trait TransactionAction: Sync + Send { /// Commit the changes and apply the changes to the transaction, /// return the transaction with the updated current_table - fn commit(self: Arc, tx: &Transaction) -> Result; + fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } pub struct TransactionActionCommit { - action: Option, + action: Option, updates: Vec, requirements: Vec, } +// TODO probably we don't need this? impl TransactionActionCommit { - pub fn take_action(&mut self) -> Option { + pub fn take_action(&mut self) -> Option { take(&mut self.action) } @@ -64,7 +67,7 @@ impl SetLocation { } impl TransactionAction for SetLocation { - fn commit(self: Box) -> Result { + fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { let updates: Vec; let requirements: Vec; if let Some(location) = self.location.clone() { @@ -76,11 +79,9 @@ impl TransactionAction for SetLocation { "Location is not set for SetLocation!", )); } - - Ok(TransactionActionCommit { - action: Some(self), - updates, - requirements, - }) + + tx.actions.push(self); + + tx.apply(updates, requirements) } } diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index d3b3cb2e3..abeff71f9 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -31,15 +31,15 @@ use crate::writer::file_writer::ParquetWriter; use crate::{Error, ErrorKind}; /// FastAppendAction is a transaction action for fast append data files to the table. -pub struct FastAppendAction<'a> { - snapshot_produce_action: SnapshotProduceAction<'a>, +pub struct FastAppendAction { + snapshot_produce_action: SnapshotProduceAction, check_duplicate: bool, } -impl<'a> FastAppendAction<'a> { +impl FastAppendAction { #[allow(clippy::too_many_arguments)] pub(crate) fn new( - tx: Transaction<'a>, + tx: Transaction, snapshot_id: i64, commit_uuid: Uuid, key_metadata: Vec, @@ -89,7 +89,7 @@ impl<'a> FastAppendAction<'a> { /// Specifically, schema compatibility checks and support for adding to partitioned tables /// have not yet been implemented. #[allow(dead_code)] - async fn add_parquet_files(mut self, file_path: Vec) -> Result> { + async fn add_parquet_files(mut self, file_path: Vec) -> Result { if !self .snapshot_produce_action .tx @@ -119,7 +119,7 @@ impl<'a> FastAppendAction<'a> { } /// Finished building the action and apply it to the transaction. - pub async fn apply(self) -> Result> { + pub async fn apply(self) -> Result { // Checks duplicate files if self.check_duplicate { let new_files: HashSet<&str> = self @@ -185,14 +185,14 @@ impl SnapshotProduceOperation for FastAppendOperation { async fn delete_entries( &self, - _snapshot_produce: &SnapshotProduceAction<'_>, + _snapshot_produce: &SnapshotProduceAction, ) -> Result> { Ok(vec![]) } async fn existing_manifest( &self, - snapshot_produce: &SnapshotProduceAction<'_>, + snapshot_produce: &SnapshotProduceAction, ) -> Result> { let Some(snapshot) = snapshot_produce .tx @@ -234,7 +234,7 @@ mod tests { #[tokio::test] async fn test_empty_data_append_action() { let table = make_v2_minimal_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut action = tx.fast_append(None, vec![]).unwrap(); action.add_data_files(vec![]).unwrap(); assert!(action.apply().await.is_err()); @@ -243,7 +243,7 @@ mod tests { #[tokio::test] async fn test_set_snapshot_properties() { let table = make_v2_minimal_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table.clone()); let mut action = tx.fast_append(None, vec![]).unwrap(); let mut snapshot_properties = HashMap::new(); @@ -281,7 +281,7 @@ mod tests { #[tokio::test] async fn test_fast_append_action() { let table = make_v2_minimal_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table.clone()); let mut action = tx.fast_append(None, vec![]).unwrap(); // check add data file with incompatible partition value @@ -367,7 +367,7 @@ mod tests { async fn test_add_existing_parquet_files_to_unpartitioned_table() { let mut fixture = TableTestFixture::new_unpartitioned(); fixture.setup_unpartitioned_manifest_files().await; - let tx = crate::transaction::Transaction::new(&fixture.table); + let tx = crate::transaction::Transaction::new(fixture.table.clone()); let file_paths = vec![ format!("{}/1.parquet", &fixture.table_location), diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index e9ef03477..0dbd3f993 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -35,26 +35,26 @@ use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; use crate::transaction::action::{ - PendingAction, SetLocation, TransactionAction, TransactionActionCommit, + BoxedTransactionAction, SetLocation, TransactionAction, TransactionActionCommit, }; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; /// Table transaction. -pub struct Transaction<'a> { - base_table: &'a Table, +pub struct Transaction { + base_table: Table, current_table: Table, - actions: Vec, + actions: Vec, updates: Vec, requirements: Vec, } -impl<'a> Transaction<'a> { +impl Transaction { /// Creates a new transaction. - pub fn new(table: &'a Table) -> Self { + pub fn new(table: Table) -> Self { Self { - base_table: table, + base_table: table.clone(), current_table: table.clone(), actions: vec![], updates: vec![], @@ -174,7 +174,7 @@ impl<'a> Transaction<'a> { self, commit_uuid: Option, key_metadata: Vec, - ) -> Result> { + ) -> Result { let snapshot_id = self.generate_unique_snapshot_id(); FastAppendAction::new( self, @@ -186,7 +186,7 @@ impl<'a> Transaction<'a> { } /// Creates replace sort order action. - pub fn replace_sort_order(self) -> ReplaceSortOrderAction<'a> { + pub fn replace_sort_order(self) -> ReplaceSortOrderAction { ReplaceSortOrderAction { tx: self, sort_fields: vec![], @@ -205,8 +205,7 @@ impl<'a> Transaction<'a> { /// Set the location of table pub fn set_location(mut self, location: String) -> Result { let set_location = SetLocation::new().set_location(location); - self.apply_commit_result(Box::new(set_location).commit()?) - .expect("Some error msg"); + Arc::new(set_location).commit(&mut self)?; Ok(self) } @@ -243,14 +242,13 @@ impl<'a> Transaction<'a> { || self.base_table.metadata_location() != refreshed.metadata_location() { // current base is stale, use refreshed as base and re-apply transaction actions - self.base_table = &refreshed.clone(); + self.base_table = refreshed.clone(); self.current_table = refreshed.clone(); let pending_actions = take(&mut self.actions); for action in pending_actions { - self.apply_commit_result(action.commit()?) - .expect("Failed to apply updates!"); + action.commit(self).expect("Failed to apply updates!"); } } @@ -347,7 +345,7 @@ mod tests { #[test] fn test_upgrade_table_version_v1_to_v2() { let table = make_v1_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx.upgrade_table_version(FormatVersion::V2).unwrap(); assert_eq!( @@ -361,7 +359,7 @@ mod tests { #[test] fn test_upgrade_table_version_v2_to_v2() { let table = make_v2_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx.upgrade_table_version(FormatVersion::V2).unwrap(); assert!( @@ -377,7 +375,7 @@ mod tests { #[test] fn test_downgrade_table_version() { let table = make_v2_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx.upgrade_table_version(FormatVersion::V1); assert!(tx.is_err(), "Downgrade table version should fail!"); @@ -386,7 +384,7 @@ mod tests { #[test] fn test_set_table_property() { let table = make_v2_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx .set_properties(HashMap::from([("a".to_string(), "b".to_string())])) .unwrap(); @@ -402,7 +400,7 @@ mod tests { #[test] fn test_remove_property() { let table = make_v2_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx .remove_properties(vec!["a".to_string(), "b".to_string()]) .unwrap(); @@ -418,7 +416,7 @@ mod tests { #[test] fn test_set_location() { let table = make_v2_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx .set_location(String::from("s3://bucket/prefix/new_table")) .unwrap(); @@ -434,7 +432,7 @@ mod tests { #[tokio::test] async fn test_transaction_apply_upgrade() { let table = make_v1_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); // Upgrade v1 to v1, do nothing. let tx = tx.upgrade_table_version(FormatVersion::V1).unwrap(); // Upgrade v1 to v2, success. diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 012fb52bb..3d3fd4570 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -59,8 +59,8 @@ pub(crate) trait ManifestProcess: Send + Sync { fn process_manifests(&self, manifests: Vec) -> Vec; } -pub(crate) struct SnapshotProduceAction<'a> { - pub tx: Transaction<'a>, +pub(crate) struct SnapshotProduceAction { + pub tx: Transaction, snapshot_id: i64, key_metadata: Vec, commit_uuid: Uuid, @@ -72,9 +72,9 @@ pub(crate) struct SnapshotProduceAction<'a> { manifest_counter: RangeFrom, } -impl<'a> SnapshotProduceAction<'a> { +impl SnapshotProduceAction { pub(crate) fn new( - tx: Transaction<'a>, + tx: Transaction, snapshot_id: i64, key_metadata: Vec, commit_uuid: Uuid, @@ -310,7 +310,7 @@ impl<'a> SnapshotProduceAction<'a> { mut self, snapshot_produce_operation: OP, process: MP, - ) -> Result> { + ) -> Result { let new_manifests = self .manifest_file(&snapshot_produce_operation, &process) .await?; diff --git a/crates/iceberg/src/transaction/sort_order.rs b/crates/iceberg/src/transaction/sort_order.rs index f925e602a..7564445e4 100644 --- a/crates/iceberg/src/transaction/sort_order.rs +++ b/crates/iceberg/src/transaction/sort_order.rs @@ -21,12 +21,12 @@ use crate::transaction::Transaction; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; /// Transaction action for replacing sort order. -pub struct ReplaceSortOrderAction<'a> { - pub tx: Transaction<'a>, +pub struct ReplaceSortOrderAction { + pub tx: Transaction, pub sort_fields: Vec, } -impl<'a> ReplaceSortOrderAction<'a> { +impl ReplaceSortOrderAction { /// Adds a field for sorting in ascending order. pub fn asc(self, name: &str, null_order: NullOrder) -> Result { self.add_sort_field(name, SortDirection::Ascending, null_order) @@ -38,7 +38,7 @@ impl<'a> ReplaceSortOrderAction<'a> { } /// Finished building the action and apply it to the transaction. - pub fn apply(mut self) -> Result> { + pub fn apply(mut self) -> Result { let unbound_sort_order = SortOrder::builder() .with_fields(self.sort_fields) .build_unbound()?; @@ -114,7 +114,7 @@ mod tests { #[test] fn test_replace_sort_order() { let table = make_v2_table(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let tx = tx.replace_sort_order().apply().unwrap(); assert_eq!( From 1c40b861eafd61f0ab017375848d50f09a41ff77 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 20:29:30 -0700 Subject: [PATCH 13/25] use backon, code cleanup --- Cargo.lock | 2 + crates/catalog/memory/src/catalog.rs | 104 ++++++--------- crates/catalog/memory/src/namespace_state.rs | 4 +- crates/catalog/rest/src/catalog.rs | 14 +- .../catalog/rest/tests/rest_catalog_test.rs | 6 +- crates/iceberg/Cargo.toml | 2 + crates/iceberg/src/catalog/mod.rs | 2 - crates/iceberg/src/spec/table_metadata.rs | 26 ++++ crates/iceberg/src/transaction/action/mod.rs | 30 +---- crates/iceberg/src/transaction/mod.rs | 121 +++++++++++------- crates/iceberg/src/transaction/snapshot.rs | 47 +++---- .../shared_tests/append_data_file_test.rs | 12 +- .../append_partition_data_file_test.rs | 10 +- .../shared_tests/conflict_commit_test.rs | 8 +- .../tests/shared_tests/scan_all_type.rs | 6 +- 15 files changed, 206 insertions(+), 188 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4e9bd47c9..56d950be0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3490,6 +3490,7 @@ dependencies = [ "arrow-string", "async-std", "async-trait", + "backon", "base64 0.22.1", "bimap", "bytes", @@ -3510,6 +3511,7 @@ dependencies = [ "parquet", "pretty_assertions", "rand 0.8.5", + "regex", "reqwest", "roaring", "rust_decimal", diff --git a/crates/catalog/memory/src/catalog.rs b/crates/catalog/memory/src/catalog.rs index 11f82d8b6..370749dcf 100644 --- a/crates/catalog/memory/src/catalog.rs +++ b/crates/catalog/memory/src/catalog.rs @@ -26,10 +26,9 @@ use iceberg::spec::{TableMetadata, TableMetadataBuilder}; use iceberg::table::Table; use iceberg::{ Catalog, Error, ErrorKind, Namespace, NamespaceIdent, Result, TableCommit, TableCreation, - TableIdent, TableRequirement, TableUpdate, + TableIdent, }; use itertools::Itertools; -use regex::Regex; use uuid::Uuid; use crate::namespace_state::NamespaceState; @@ -278,75 +277,50 @@ impl Catalog for MemoryCatalog { } /// Update a table to the catalog. - async fn update_table(&self, commit: TableCommit) -> Result
{ + async fn update_table(&self, _commit: TableCommit) -> Result
{ Err(Error::new( ErrorKind::FeatureUnsupported, "MemoryCatalog does not currently support updating tables.", )) } - async fn commit_table(&self, base: &Table, current: Table) -> Result
{ - if base.metadata() == current.metadata() { - // no change - return Ok(current); - } - - let mut root_namespace_state = self.root_namespace_state.lock().await; - // TODO: caller needs to retry on the error below - let _ = root_namespace_state - .check_metadata_location(base.identifier(), base.metadata_location())?; - - let next_metadata_version = if let Some(base_metadata_location) = base.metadata_location() { - self.parse_metadata_version(base_metadata_location) + 1 - } else { - 0 - }; - - // write metadata - let metadata_location = format!( - "{}/metadata/{}-{}.metadata.json", - current.metadata().location(), - next_metadata_version, - Uuid::new_v4() - ); - - // TODO instead of using current.metadata(), build a new metadata with some properties like last_updated_ms updated - self.file_io - .new_output(&metadata_location)? - .write(serde_json::to_vec(current.metadata())?.into()) - .await?; - - root_namespace_state - .update_existing_table_location(current.identifier(), current.metadata_location())?; - - // TODO same here, need to update the metadata location - Ok(current) - } -} - -// todo move this to metadata? -fn parse_metadata_version(metadata_location: &str) -> Result { - let pattern = r"(\d+)-([\w-]{36})(?:\.\w+)?\.metadata\.json"; // todo make this constant - - if let Some(metadata_file_name) = metadata_location.split('/').last() { - let re = Regex::new(pattern).expect("Failed to parse regex for metadata file!"); - if let Some(caps) = re.captures(metadata_file_name) { - let metadata_version_str = &caps[1]; - let uuid_str = &caps[2]; - - let metadata_version = metadata_version_str - .parse() - .expect(format!("Invalid metadata version: {metadata_version_str}").as_str()); - let uuid = Uuid::parse_str(uuid_str)?; - - return Ok(metadata_version); - } - } - - Err(Error::new( - ErrorKind::Unexpected, - format!("Unrecognizable metadata location: {metadata_location}"), - )) + // async fn commit_table(&self, base: &Table, current: Table) -> Result
{ + // if base.metadata() == current.metadata() { + // // no change + // return Ok(current); + // } + // + // let mut root_namespace_state = self.root_namespace_state.lock().await; + // // TODO: caller needs to retry on the error below + // let _ = root_namespace_state + // .check_metadata_location(base.identifier(), base.metadata_location())?; + // + // let next_metadata_version = if let Some(base_metadata_location) = base.metadata_location() { + // self.parse_metadata_version(base_metadata_location) + 1 + // } else { + // 0 + // }; + // + // // write metadata + // let metadata_location = format!( + // "{}/metadata/{}-{}.metadata.json", + // current.metadata().location(), + // next_metadata_version, + // Uuid::new_v4() + // ); + // + // // TODO instead of using current.metadata(), build a new metadata with some properties like last_updated_ms updated + // self.file_io + // .new_output(&metadata_location)? + // .write(serde_json::to_vec(current.metadata())?.into()) + // .await?; + // + // root_namespace_state + // .update_existing_table_location(current.identifier(), current.metadata_location())?; + // + // // TODO same here, need to update the metadata location + // Ok(current) + // } } #[cfg(test)] diff --git a/crates/catalog/memory/src/namespace_state.rs b/crates/catalog/memory/src/namespace_state.rs index c406b6c0a..547c6bcd0 100644 --- a/crates/catalog/memory/src/namespace_state.rs +++ b/crates/catalog/memory/src/namespace_state.rs @@ -293,11 +293,11 @@ impl NamespaceState { return Ok(()); } - let mut namespace = self.get_mut_namespace(table_ident.namespace())?; + let namespace = self.get_mut_namespace(table_ident.namespace())?; namespace .table_metadata_locations .entry(table_ident.name().to_string()) - .insert_entry(new_metadata_location.unwrap().into_string()); + .insert_entry(new_metadata_location.unwrap().to_string()); Ok(()) } diff --git a/crates/catalog/rest/src/catalog.rs b/crates/catalog/rest/src/catalog.rs index 851819069..398e8a4ee 100644 --- a/crates/catalog/rest/src/catalog.rs +++ b/crates/catalog/rest/src/catalog.rs @@ -2104,7 +2104,9 @@ mod tests { .create_async() .await; - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()); + let catalog = Arc::new(RestCatalog::new( + RestCatalogConfig::builder().uri(server.url()).build(), + )); let table1 = { let file = File::open(format!( @@ -2125,10 +2127,10 @@ mod tests { .unwrap() }; - let table = Transaction::new(&table1) + let table = Transaction::new(table1) .upgrade_table_version(FormatVersion::V2) .unwrap() - .commit(&catalog) + .commit(catalog) .await .unwrap(); @@ -2229,7 +2231,9 @@ mod tests { .create_async() .await; - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()); + let catalog = Arc::new(RestCatalog::new( + RestCatalogConfig::builder().uri(server.url()).build(), + )); let table1 = { let file = File::open(format!( @@ -2253,7 +2257,7 @@ mod tests { let table_result = Transaction::new(&table1) .upgrade_table_version(FormatVersion::V2) .unwrap() - .commit(&catalog) + .commit(catalog) .await; assert!(table_result.is_err()); diff --git a/crates/catalog/rest/tests/rest_catalog_test.rs b/crates/catalog/rest/tests/rest_catalog_test.rs index ab7ea3d62..499fdc45d 100644 --- a/crates/catalog/rest/tests/rest_catalog_test.rs +++ b/crates/catalog/rest/tests/rest_catalog_test.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::net::SocketAddr; -use std::sync::RwLock; +use std::sync::{Arc, RwLock}; use ctor::{ctor, dtor}; use iceberg::spec::{FormatVersion, NestedField, PrimitiveType, Schema, Type}; @@ -347,10 +347,10 @@ async fn test_update_table() { ); // Update table by committing transaction - let table2 = Transaction::new(&table) + let table2 = Transaction::new(table) .set_properties(HashMap::from([("prop1".to_string(), "v1".to_string())])) .unwrap() - .commit(&catalog) + .commit(Arc::new(catalog)) .await .unwrap(); diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 9d28c0d4a..90f4a864d 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -56,6 +56,7 @@ arrow-string = { workspace = true } async-std = { workspace = true, optional = true, features = ["attributes"] } async-trait = { workspace = true } tokio-retry2 = { version = "0.5.7" } +backon = { version = "1.5.1"} base64 = { workspace = true } bimap = { workspace = true } bytes = { workspace = true } @@ -87,6 +88,7 @@ typed-builder = { workspace = true } url = { workspace = true } uuid = { workspace = true } zstd = { workspace = true } +regex = "1.11.1" [dev-dependencies] ctor = { workspace = true } diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index bf5059421..7c30d29aa 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -95,8 +95,6 @@ pub trait Catalog: Debug + Sync + Send { /// Update a table to the catalog. async fn update_table(&self, commit: TableCommit) -> Result
; - - async fn commit_table(&self, base: &Table, current: Table) -> Result
; } /// NamespaceIdent represents the identifier of a namespace in the catalog. diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index fcd510920..68aac252c 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -26,6 +26,7 @@ use std::sync::Arc; use _serde::TableMetadataEnum; use chrono::{DateTime, Utc}; +use regex::Regex; use serde::{Deserialize, Serialize}; use serde_repr::{Deserialize_repr, Serialize_repr}; use uuid::Uuid; @@ -40,6 +41,7 @@ use crate::error::{Result, timestamp_ms_to_utc}; use crate::{Error, ErrorKind}; static MAIN_BRANCH: &str = "main"; +const TABLE_METADATA_FILE_NAME_REGEX: &str = r"(\d+)-([\w-]{36})(?:\.\w+)?\.metadata\.json"; pub(crate) static ONE_MINUTE_MS: i64 = 60_000; pub(crate) static EMPTY_SNAPSHOT_ID: i64 = -1; @@ -432,6 +434,30 @@ impl TableMetadata { self.encryption_keys.get(key_id) } + /// Parse metadata version and uuid from metadata filename + fn parse_metadata_filename(metadata_location: &str) -> Result<(i32, Uuid)> { + if let Some(metadata_file_name) = metadata_location.split('/').last() { + let re = Regex::new(TABLE_METADATA_FILE_NAME_REGEX) + .expect("Failed to parse regex for metadata file!"); + if let Some(caps) = re.captures(metadata_file_name) { + let metadata_version_str = &caps[1]; + let uuid_str = &caps[2]; + + let metadata_version = metadata_version_str + .parse() + .expect(format!("Invalid metadata version: {metadata_version_str}").as_str()); + let uuid = Uuid::parse_str(uuid_str)?; + + return Ok((metadata_version, uuid)); + } + } + + Err(Error::new( + ErrorKind::Unexpected, + format!("Unrecognizable metadata location: {metadata_location}"), + )) + } + /// Normalize this partition spec. /// /// This is an internal method diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 68739d8ee..e9179400e 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. -use std::mem::take; use std::sync::Arc; + use async_trait::async_trait; -use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; + use crate::transaction::Transaction; +use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; pub type BoxedTransactionAction = Arc; @@ -30,27 +31,6 @@ pub(crate) trait TransactionAction: Sync + Send { fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } -pub struct TransactionActionCommit { - action: Option, - updates: Vec, - requirements: Vec, -} - -// TODO probably we don't need this? -impl TransactionActionCommit { - pub fn take_action(&mut self) -> Option { - take(&mut self.action) - } - - pub fn take_updates(&mut self) -> Vec { - take(&mut self.updates) - } - - pub fn take_requirements(&mut self) -> Vec { - take(&mut self.requirements) - } -} - pub struct SetLocation { pub location: Option, } @@ -79,9 +59,9 @@ impl TransactionAction for SetLocation { "Location is not set for SetLocation!", )); } - + tx.actions.push(self); - + tx.apply(updates, requirements) } } diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 0dbd3f993..a62678dfd 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -24,24 +24,26 @@ mod sort_order; use std::cmp::Ordering; use std::collections::HashMap; -use std::mem::{discriminant, take}; +use std::mem::discriminant; use std::sync::Arc; +use std::time::Duration; + +use backon::{BackoffBuilder, ExponentialBuilder, RetryableWithContext}; +// use tokio_retry2::strategy::ExponentialBackoff; +// use tokio_retry2::{Retry, RetryError}; use uuid::Uuid; -use tokio_retry2::{Retry, RetryError}; -use tokio_retry2::strategy::ExponentialBackoff; use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{ - BoxedTransactionAction, SetLocation, TransactionAction, TransactionActionCommit, -}; +use crate::transaction::action::{BoxedTransactionAction, SetLocation, TransactionAction}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; /// Table transaction. +#[derive(Clone)] pub struct Transaction { base_table: Table, current_table: Table, @@ -74,19 +76,8 @@ impl Transaction { Ok(()) } - fn apply_commit_result(&mut self, mut tx_commit_res: TransactionActionCommit) -> Result<()> { - if let Some(action) = tx_commit_res.take_action() { - self.actions.push(action); - return self.apply( - tx_commit_res.take_updates(), - tx_commit_res.take_requirements(), - ); - } - - Err(Error::new(ErrorKind::DataInvalid, "Action cannot be none!")) - } - - fn apply( + /// TODO documentation + pub fn apply( &mut self, updates: Vec, requirements: Vec, @@ -210,7 +201,7 @@ impl Transaction { } /// Commit transaction. - pub async fn commit(&mut self, catalog: &dyn Catalog) -> Result
{ + pub async fn commit(&mut self, catalog: Arc<&dyn Catalog>) -> Result
{ if self.actions.is_empty() || (self.base_table.metadata() == self.current_table.metadata() && self.base_table.metadata_location() == self.current_table.metadata_location()) @@ -219,18 +210,57 @@ impl Transaction { return Ok(self.current_table.clone()); } - // TODO revisit the retry config - let retry_strategy = ExponentialBackoff::from_millis(10) - .take(3); // limit to 3 retries - - Retry::spawn(retry_strategy, - || { - let catalog = catalog.clone(); - self.do_commit(catalog) - } ).await + // let retry_strategy = ExponentialBackoff::from_millis(10) + // .take(3); // limit to 3 retries + + // Retry::spawn(retry_strategy, + // || { + // async { + // Transaction::do_commit( + // &mut self.clone(), + // catalog.clone()).await + // } + // }).await + + let tx = self.clone(); + // TODO try backon: RetryableWithContext + (|mut tx: Transaction| async { + let result = tx.do_commit(catalog.clone()).await; + (tx, result) + }) + .retry( + ExponentialBuilder::new() + // TODO retry strategy should be configurable + .with_min_delay(Duration::from_millis(100)) + .with_max_delay(Duration::from_millis(60 * 1000)) + .with_total_delay(Some(Duration::from_millis(30 * 60 * 1000))) + .with_max_times(4) + .with_factor(2.0) + .build(), + ) + .context(tx) + .sleep(tokio::time::sleep) + .when(|e| { + if let Some(err) = e.downcast_ref::() { + err.kind() == ErrorKind::DataInvalid // TODO add retryable error kind + } else { + false + } + }) + .await + .1 + .map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to commit transaction! caused by: {e}"), + ) + }) } - async fn do_commit(&mut self, catalog: &dyn Catalog) -> std::result::Result> { + async fn do_commit( + &mut self, + catalog: Arc<&dyn Catalog>, + ) -> std::result::Result { let base_table_identifier = self.base_table.identifier().to_owned(); let refreshed = catalog @@ -244,10 +274,10 @@ impl Transaction { // current base is stale, use refreshed as base and re-apply transaction actions self.base_table = refreshed.clone(); self.current_table = refreshed.clone(); + self.updates = vec![]; + self.requirements = vec![]; - let pending_actions = take(&mut self.actions); - - for action in pending_actions { + for action in self.actions.clone() { action.commit(self).expect("Failed to apply updates!"); } } @@ -257,19 +287,20 @@ impl Transaction { .updates(self.updates.clone()) .requirements(self.requirements.clone()) .build(); - - let result = catalog - .update_table(table_commit) - .await; - match result { - Ok(table) => Ok(table), - // TODO revisit the error kind - Err(e) if e.kind() == ErrorKind::DataInvalid => { - Err(RetryError::transient(e)) - } - Err(e) => Err(RetryError::permanent(e)) - } + catalog + .update_table(table_commit) + .await + .map_err(anyhow::Error::from) + + // match result { + // Ok(table) => Ok(table), + // + // Err(e) if e.kind() == ErrorKind::DataInvalid => { + // Err(RetryError::transient(e)) + // } + // Err(e) => Err(RetryError::permanent(e)) + // } } } diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 3d3fd4570..a1fc00b41 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -359,29 +359,30 @@ impl SnapshotProduceAction { .with_timestamp_ms(commit_ts) .build(); - self.tx.apply( - vec![ - TableUpdate::AddSnapshot { - snapshot: new_snapshot, - }, - TableUpdate::SetSnapshotRef { - ref_name: MAIN_BRANCH.to_string(), - reference: SnapshotReference::new( - self.snapshot_id, - SnapshotRetention::branch(None, None, None), - ), - }, - ], - vec![ - TableRequirement::UuidMatch { - uuid: self.tx.current_table.metadata().uuid(), - }, - TableRequirement::RefSnapshotIdMatch { - r#ref: MAIN_BRANCH.to_string(), - snapshot_id: self.tx.current_table.metadata().current_snapshot_id(), - }, - ], - )?; + let updates = vec![ + TableUpdate::AddSnapshot { + snapshot: new_snapshot, + }, + TableUpdate::SetSnapshotRef { + ref_name: MAIN_BRANCH.to_string(), + reference: SnapshotReference::new( + self.snapshot_id, + SnapshotRetention::branch(None, None, None), + ), + }, + ]; + + let requirements = vec![ + TableRequirement::UuidMatch { + uuid: self.tx.current_table.metadata().uuid(), + }, + TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: self.tx.current_table.metadata().current_snapshot_id(), + }, + ]; + + self.tx.apply(updates, requirements)?; Ok(self.tx) } 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 f3ee17c75..a79c78a92 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 @@ -111,11 +111,11 @@ async fn test_append_data_file() { assert_eq!(field_ids, vec![1, 2, 3]); // commit result - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); - let tx = append_action.apply().await.unwrap(); - let table = tx.commit(&rest_catalog).await.unwrap(); + let mut tx = append_action.apply().await.unwrap(); + let table = tx.commit(Arc::new(rest_catalog)).await.unwrap(); // check result let batch_stream = table @@ -131,11 +131,11 @@ async fn test_append_data_file() { assert_eq!(batches[0], batch); // commit result again - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); - let tx = append_action.apply().await.unwrap(); - let table = tx.commit(&rest_catalog).await.unwrap(); + let mut tx = append_action.apply().await.unwrap(); + let table = tx.commit(Arc::new(rest_catalog)).await.unwrap(); // check result again let batch_stream = table diff --git a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs index c5c029a45..3d0903491 100644 --- a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs @@ -119,13 +119,13 @@ async fn test_append_partition_data_file() { let data_file_valid = data_file_writer_valid.close().await.unwrap(); // commit result - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action .add_data_files(data_file_valid.clone()) .unwrap(); - let tx = append_action.apply().await.unwrap(); - let table = tx.commit(&rest_catalog).await.unwrap(); + let mut tx = append_action.apply().await.unwrap(); + let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result let batch_stream = table @@ -179,7 +179,7 @@ async fn test_schema_incompatible_partition_type( data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); if append_action .add_data_files(data_file_invalid.clone()) @@ -219,7 +219,7 @@ async fn test_schema_incompatible_partition_fields( data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); if append_action .add_data_files(data_file_invalid.clone()) diff --git a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs index d277e12e5..4ba1a1bca 100644 --- a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs +++ b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs @@ -89,17 +89,17 @@ async fn test_append_data_file_conflict() { let data_file = data_file_writer.close().await.unwrap(); // start two transaction and commit one of them - let tx1 = Transaction::new(&table); + let tx1 = Transaction::new(table.clone()); let mut append_action = tx1.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); let tx1 = append_action.apply().await.unwrap(); - let tx2 = Transaction::new(&table); + let tx2 = Transaction::new(table.clone()); let mut append_action = tx2.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); - let tx2 = append_action.apply().await.unwrap(); + let mut tx2 = append_action.apply().await.unwrap(); let table = tx2 - .commit(&rest_catalog) + .commit(Arc::new(&rest_catalog)) .await .expect("The first commit should not fail."); diff --git a/crates/integration_tests/tests/shared_tests/scan_all_type.rs b/crates/integration_tests/tests/shared_tests/scan_all_type.rs index 5ff982720..ccf6f4e96 100644 --- a/crates/integration_tests/tests/shared_tests/scan_all_type.rs +++ b/crates/integration_tests/tests/shared_tests/scan_all_type.rs @@ -308,11 +308,11 @@ async fn test_scan_all_type() { let data_file = data_file_writer.close().await.unwrap(); // commit result - let tx = Transaction::new(&table); + let tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); - let tx = append_action.apply().await.unwrap(); - let table = tx.commit(&rest_catalog).await.unwrap(); + let mut tx = append_action.apply().await.unwrap(); + let table = tx.commit(Arc::new(rest_catalog)).await.unwrap(); // check result let batch_stream = table From 014726ee57a41305281bedcfa8444987eae44c28 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 20:30:45 -0700 Subject: [PATCH 14/25] minor --- crates/iceberg/src/transaction/mod.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index a62678dfd..65680b973 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -223,7 +223,6 @@ impl Transaction { // }).await let tx = self.clone(); - // TODO try backon: RetryableWithContext (|mut tx: Transaction| async { let result = tx.do_commit(catalog.clone()).await; (tx, result) From d8a2394a4d951ee6378076371a988e3ad490484b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 20:43:01 -0700 Subject: [PATCH 15/25] fix test build --- crates/catalog/rest/src/catalog.rs | 14 +++++++------- crates/catalog/rest/tests/rest_catalog_test.rs | 2 +- crates/iceberg/src/transaction/action/mod.rs | 8 ++++---- crates/iceberg/src/transaction/mod.rs | 4 ++-- .../tests/shared_tests/append_data_file_test.rs | 4 ++-- .../tests/shared_tests/conflict_commit_test.rs | 4 ++-- .../tests/shared_tests/scan_all_type.rs | 2 +- 7 files changed, 19 insertions(+), 19 deletions(-) diff --git a/crates/catalog/rest/src/catalog.rs b/crates/catalog/rest/src/catalog.rs index 398e8a4ee..8f01f986d 100644 --- a/crates/catalog/rest/src/catalog.rs +++ b/crates/catalog/rest/src/catalog.rs @@ -2104,9 +2104,9 @@ mod tests { .create_async() .await; - let catalog = Arc::new(RestCatalog::new( + let catalog = RestCatalog::new( RestCatalogConfig::builder().uri(server.url()).build(), - )); + ); let table1 = { let file = File::open(format!( @@ -2130,7 +2130,7 @@ mod tests { let table = Transaction::new(table1) .upgrade_table_version(FormatVersion::V2) .unwrap() - .commit(catalog) + .commit(Arc::new(&catalog)) .await .unwrap(); @@ -2231,9 +2231,9 @@ mod tests { .create_async() .await; - let catalog = Arc::new(RestCatalog::new( + let catalog = RestCatalog::new( RestCatalogConfig::builder().uri(server.url()).build(), - )); + ); let table1 = { let file = File::open(format!( @@ -2254,10 +2254,10 @@ mod tests { .unwrap() }; - let table_result = Transaction::new(&table1) + let table_result = Transaction::new(table1) .upgrade_table_version(FormatVersion::V2) .unwrap() - .commit(catalog) + .commit(Arc::new(&catalog)) .await; assert!(table_result.is_err()); diff --git a/crates/catalog/rest/tests/rest_catalog_test.rs b/crates/catalog/rest/tests/rest_catalog_test.rs index 499fdc45d..ddf574d5d 100644 --- a/crates/catalog/rest/tests/rest_catalog_test.rs +++ b/crates/catalog/rest/tests/rest_catalog_test.rs @@ -350,7 +350,7 @@ async fn test_update_table() { let table2 = Transaction::new(table) .set_properties(HashMap::from([("prop1".to_string(), "v1".to_string())])) .unwrap() - .commit(Arc::new(catalog)) + .commit(Arc::new(&catalog)) .await .unwrap(); diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index e9179400e..7a7fd827b 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -31,13 +31,13 @@ pub(crate) trait TransactionAction: Sync + Send { fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } -pub struct SetLocation { +pub struct SetLocationAction { pub location: Option, } -impl SetLocation { +impl SetLocationAction { pub fn new() -> Self { - SetLocation { location: None } + SetLocationAction { location: None } } pub fn set_location(mut self, location: String) -> Self { @@ -46,7 +46,7 @@ impl SetLocation { } } -impl TransactionAction for SetLocation { +impl TransactionAction for SetLocationAction { fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { let updates: Vec; let requirements: Vec; diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 65680b973..8227ba9d8 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -37,7 +37,7 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{BoxedTransactionAction, SetLocation, TransactionAction}; +use crate::transaction::action::{BoxedTransactionAction, SetLocationAction, TransactionAction}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -195,7 +195,7 @@ impl Transaction { /// Set the location of table pub fn set_location(mut self, location: String) -> Result { - let set_location = SetLocation::new().set_location(location); + let set_location = SetLocationAction::new().set_location(location); Arc::new(set_location).commit(&mut self)?; Ok(self) } 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 a79c78a92..ff632925e 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 @@ -115,7 +115,7 @@ async fn test_append_data_file() { let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); let mut tx = append_action.apply().await.unwrap(); - let table = tx.commit(Arc::new(rest_catalog)).await.unwrap(); + let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result let batch_stream = table @@ -135,7 +135,7 @@ async fn test_append_data_file() { let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); let mut tx = append_action.apply().await.unwrap(); - let table = tx.commit(Arc::new(rest_catalog)).await.unwrap(); + let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result again let batch_stream = table diff --git a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs index 4ba1a1bca..d54bad554 100644 --- a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs +++ b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs @@ -92,7 +92,7 @@ async fn test_append_data_file_conflict() { let tx1 = Transaction::new(table.clone()); let mut append_action = tx1.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); - let tx1 = append_action.apply().await.unwrap(); + let mut tx1 = append_action.apply().await.unwrap(); let tx2 = Transaction::new(table.clone()); let mut append_action = tx2.fast_append(None, vec![]).unwrap(); @@ -117,5 +117,5 @@ async fn test_append_data_file_conflict() { assert_eq!(batches[0], batch); // another commit should fail - assert!(tx1.commit(&rest_catalog).await.is_err()); + assert!(tx1.commit(Arc::new(&rest_catalog)).await.is_err()); } diff --git a/crates/integration_tests/tests/shared_tests/scan_all_type.rs b/crates/integration_tests/tests/shared_tests/scan_all_type.rs index ccf6f4e96..a0fac2f5c 100644 --- a/crates/integration_tests/tests/shared_tests/scan_all_type.rs +++ b/crates/integration_tests/tests/shared_tests/scan_all_type.rs @@ -312,7 +312,7 @@ async fn test_scan_all_type() { let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action.add_data_files(data_file.clone()).unwrap(); let mut tx = append_action.apply().await.unwrap(); - let table = tx.commit(Arc::new(rest_catalog)).await.unwrap(); + let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result let batch_stream = table From a6ce0b0a0d08b3a344397e8418a77fb3d4ebf8f2 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 3 Jun 2025 20:45:27 -0700 Subject: [PATCH 16/25] fmt --- crates/catalog/rest/src/catalog.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/crates/catalog/rest/src/catalog.rs b/crates/catalog/rest/src/catalog.rs index 8f01f986d..5d70b5fff 100644 --- a/crates/catalog/rest/src/catalog.rs +++ b/crates/catalog/rest/src/catalog.rs @@ -2104,9 +2104,7 @@ mod tests { .create_async() .await; - let catalog = RestCatalog::new( - RestCatalogConfig::builder().uri(server.url()).build(), - ); + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()); let table1 = { let file = File::open(format!( @@ -2231,9 +2229,7 @@ mod tests { .create_async() .await; - let catalog = RestCatalog::new( - RestCatalogConfig::builder().uri(server.url()).build(), - ); + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()); let table1 = { let file = File::open(format!( From 626715c0b16fd283b27003aae46ca857b48b2366 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 4 Jun 2025 18:24:45 -0700 Subject: [PATCH 17/25] clean up, add apply() --- Cargo.lock | 11 ----- crates/iceberg/Cargo.toml | 1 - crates/iceberg/src/transaction/action/mod.rs | 20 ++++++--- crates/iceberg/src/transaction/mod.rs | 45 ++++++-------------- 4 files changed, 25 insertions(+), 52 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 56d950be0..9dabbaec5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3525,7 +3525,6 @@ dependencies = [ "tera", "thrift", "tokio", - "tokio-retry2", "typed-builder 0.20.1", "url", "uuid", @@ -7015,16 +7014,6 @@ dependencies = [ "syn 2.0.101", ] -[[package]] -name = "tokio-retry2" -version = "0.5.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1264d076dd34560544a2799e40e457bd07c43d30f4a845686b031bcd8455c84f" -dependencies = [ - "pin-project", - "tokio", -] - [[package]] name = "tokio-rustls" version = "0.24.1" diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 90f4a864d..cd0fedc70 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -55,7 +55,6 @@ arrow-select = { workspace = true } arrow-string = { workspace = true } async-std = { workspace = true, optional = true, features = ["attributes"] } async-trait = { workspace = true } -tokio-retry2 = { version = "0.5.7" } backon = { version = "1.5.1"} base64 = { workspace = true } bimap = { workspace = true } diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 7a7fd827b..461153ea4 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; use std::sync::Arc; use async_trait::async_trait; @@ -26,18 +27,19 @@ pub type BoxedTransactionAction = Arc; #[async_trait] pub(crate) trait TransactionAction: Sync + Send { - /// Commit the changes and apply the changes to the transaction, - /// return the transaction with the updated current_table + fn apply(&self) -> Box; + + /// Commit the changes and apply the changes to the transaction fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } -pub struct SetLocationAction { - pub location: Option, +pub struct UpdateLocationAction { + location: Option, } -impl SetLocationAction { +impl UpdateLocationAction { pub fn new() -> Self { - SetLocationAction { location: None } + UpdateLocationAction { location: None } } pub fn set_location(mut self, location: String) -> Self { @@ -46,7 +48,11 @@ impl SetLocationAction { } } -impl TransactionAction for SetLocationAction { +impl TransactionAction for UpdateLocationAction { + fn apply(&self) -> Box { + Box::new(self.location.clone()) + } + fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { let updates: Vec; let requirements: Vec; diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 8227ba9d8..e55dbd4e6 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -29,15 +29,13 @@ use std::sync::Arc; use std::time::Duration; use backon::{BackoffBuilder, ExponentialBuilder, RetryableWithContext}; -// use tokio_retry2::strategy::ExponentialBackoff; -// use tokio_retry2::{Retry, RetryError}; use uuid::Uuid; use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{BoxedTransactionAction, SetLocationAction, TransactionAction}; +use crate::transaction::action::{BoxedTransactionAction, UpdateLocationAction, TransactionAction}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -194,10 +192,8 @@ impl Transaction { } /// Set the location of table - pub fn set_location(mut self, location: String) -> Result { - let set_location = SetLocationAction::new().set_location(location); - Arc::new(set_location).commit(&mut self)?; - Ok(self) + pub fn update_location(&mut self) -> Result { + Ok(UpdateLocationAction::new()) } /// Commit transaction. @@ -210,18 +206,6 @@ impl Transaction { return Ok(self.current_table.clone()); } - // let retry_strategy = ExponentialBackoff::from_millis(10) - // .take(3); // limit to 3 retries - - // Retry::spawn(retry_strategy, - // || { - // async { - // Transaction::do_commit( - // &mut self.clone(), - // catalog.clone()).await - // } - // }).await - let tx = self.clone(); (|mut tx: Transaction| async { let result = tx.do_commit(catalog.clone()).await; @@ -291,15 +275,6 @@ impl Transaction { .update_table(table_commit) .await .map_err(anyhow::Error::from) - - // match result { - // Ok(table) => Ok(table), - // - // Err(e) if e.kind() == ErrorKind::DataInvalid => { - // Err(RetryError::transient(e)) - // } - // Err(e) => Err(RetryError::permanent(e)) - // } } } @@ -308,12 +283,13 @@ mod tests { use std::collections::HashMap; use std::fs::File; use std::io::BufReader; - + use std::sync::Arc; use crate::io::FileIOBuilder; use crate::spec::{FormatVersion, TableMetadata}; use crate::table::Table; use crate::transaction::Transaction; use crate::{TableIdent, TableUpdate}; + use crate::transaction::action::TransactionAction; fn make_v1_table() -> Table { let file = File::open(format!( @@ -446,10 +422,13 @@ mod tests { #[test] fn test_set_location() { let table = make_v2_table(); - let tx = Transaction::new(table); - let tx = tx - .set_location(String::from("s3://bucket/prefix/new_table")) - .unwrap(); + let mut tx = Transaction::new(table); + let update_location_action = tx + .update_location() + .unwrap() + .set_location(String::from("s3://bucket/prefix/new_table")); + + let _res = Arc::new(update_location_action).commit(&mut tx).unwrap(); assert_eq!( vec![TableUpdate::SetLocation { From 11bb9d367c3c6ec7dc35ccdc7e2c1f6741a4ed56 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 5 Jun 2025 12:55:14 -0700 Subject: [PATCH 18/25] remove apply --- crates/iceberg/src/transaction/action/mod.rs | 9 +-------- crates/iceberg/src/transaction/mod.rs | 2 +- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 461153ea4..675074213 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; use std::sync::Arc; use async_trait::async_trait; @@ -27,8 +26,6 @@ pub type BoxedTransactionAction = Arc; #[async_trait] pub(crate) trait TransactionAction: Sync + Send { - fn apply(&self) -> Box; - /// Commit the changes and apply the changes to the transaction fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } @@ -49,11 +46,7 @@ impl UpdateLocationAction { } impl TransactionAction for UpdateLocationAction { - fn apply(&self) -> Box { - Box::new(self.location.clone()) - } - - fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { + async fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { let updates: Vec; let requirements: Vec; if let Some(location) = self.location.clone() { diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index e55dbd4e6..348710507 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -261,7 +261,7 @@ impl Transaction { self.requirements = vec![]; for action in self.actions.clone() { - action.commit(self).expect("Failed to apply updates!"); + action.commit(self)? } } From 8c2192b21ff12c2467bd2bef0052ad7ae780beb2 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 5 Jun 2025 18:37:16 -0700 Subject: [PATCH 19/25] remove tx from FastAppendAction, impl tx action for fast append --- crates/iceberg/src/transaction/action/mod.rs | 3 +- crates/iceberg/src/transaction/append.rs | 111 +++++++++++-------- crates/iceberg/src/transaction/mod.rs | 13 +-- crates/iceberg/src/transaction/snapshot.rs | 80 ++++++------- 4 files changed, 113 insertions(+), 94 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index 675074213..f08fbd5dc 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -27,7 +27,7 @@ pub type BoxedTransactionAction = Arc; #[async_trait] pub(crate) trait TransactionAction: Sync + Send { /// Commit the changes and apply the changes to the transaction - fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; + async fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } pub struct UpdateLocationAction { @@ -45,6 +45,7 @@ impl UpdateLocationAction { } } +#[async_trait] impl TransactionAction for UpdateLocationAction { async fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { let updates: Vec; diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index abeff71f9..906d1b8be 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -16,8 +16,10 @@ // under the License. use std::collections::{HashMap, HashSet}; - +use std::sync::Arc; use arrow_array::StringArray; +use async_trait::async_trait; +use futures::lock::Mutex; use futures::TryStreamExt; use uuid::Uuid; @@ -29,30 +31,29 @@ use crate::transaction::snapshot::{ }; use crate::writer::file_writer::ParquetWriter; use crate::{Error, ErrorKind}; +use crate::transaction::action::TransactionAction; /// FastAppendAction is a transaction action for fast append data files to the table. pub struct FastAppendAction { - snapshot_produce_action: SnapshotProduceAction, + snapshot_produce_action: Mutex, check_duplicate: bool, } impl FastAppendAction { #[allow(clippy::too_many_arguments)] pub(crate) fn new( - tx: Transaction, snapshot_id: i64, commit_uuid: Uuid, key_metadata: Vec, snapshot_properties: HashMap, ) -> Result { Ok(Self { - snapshot_produce_action: SnapshotProduceAction::new( - tx, + snapshot_produce_action: Mutex::new(SnapshotProduceAction::new( snapshot_id, key_metadata, commit_uuid, snapshot_properties, - )?, + )?), check_duplicate: true, }) } @@ -66,9 +67,10 @@ impl FastAppendAction { /// Add data files to the snapshot. pub fn add_data_files( &mut self, + tx: &Transaction, data_files: impl IntoIterator, ) -> Result<&mut Self> { - self.snapshot_produce_action.add_data_files(data_files)?; + self.snapshot_produce_action.get_mut().add_data_files(tx, data_files)?; Ok(self) } @@ -77,7 +79,7 @@ impl FastAppendAction { &mut self, snapshot_properties: HashMap, ) -> Result<&mut Self> { - self.snapshot_produce_action + self.snapshot_produce_action.get_mut() .set_snapshot_properties(snapshot_properties)?; Ok(self) } @@ -89,10 +91,8 @@ impl FastAppendAction { /// Specifically, schema compatibility checks and support for adding to partitioned tables /// have not yet been implemented. #[allow(dead_code)] - async fn add_parquet_files(mut self, file_path: Vec) -> Result { - if !self - .snapshot_produce_action - .tx + async fn add_parquet_files(mut self, tx: &mut Transaction, file_path: Vec) -> Result<()> { + if !tx .current_table .metadata() .default_spec @@ -104,34 +104,45 @@ impl FastAppendAction { )); } - let table_metadata = self.snapshot_produce_action.tx.current_table.metadata(); + let table_metadata = tx.current_table.metadata(); let data_files = ParquetWriter::parquet_files_to_data_files( - self.snapshot_produce_action.tx.current_table.file_io(), + tx.current_table.file_io(), file_path, table_metadata, ) .await?; - self.add_data_files(data_files)?; + self.add_data_files(tx, data_files)?; - self.apply().await + Arc::new(self).commit(tx).await?; + + Ok(()) } + // TODO remove this method /// Finished building the action and apply it to the transaction. - pub async fn apply(self) -> Result { + pub async fn apply(self, tx: Transaction) -> Result { + + Ok(tx) + + } +} + +#[async_trait] +impl TransactionAction for FastAppendAction { + async fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { + let mut action = self.snapshot_produce_action.lock().await; + // Checks duplicate files if self.check_duplicate { - let new_files: HashSet<&str> = self - .snapshot_produce_action + let new_files: HashSet<&str> = action .added_data_files .iter() .map(|df| df.file_path.as_str()) .collect(); - let mut manifest_stream = self - .snapshot_produce_action - .tx + let mut manifest_stream = tx .current_table .inspect() .manifests() @@ -169,10 +180,15 @@ impl FastAppendAction { )); } } - - self.snapshot_produce_action - .apply(FastAppendOperation, DefaultManifestProcess) - .await + + action + .apply(tx, FastAppendOperation, DefaultManifestProcess) + .await?; + + drop(action); + tx.actions.push(self); + + Ok(()) } } @@ -192,10 +208,10 @@ impl SnapshotProduceOperation for FastAppendOperation { async fn existing_manifest( &self, - snapshot_produce: &SnapshotProduceAction, + tx: &Transaction, + _snapshot_produce: &SnapshotProduceAction, ) -> Result> { - let Some(snapshot) = snapshot_produce - .tx + let Some(snapshot) = tx .current_table .metadata() .current_snapshot() @@ -205,8 +221,8 @@ impl SnapshotProduceOperation for FastAppendOperation { let manifest_list = snapshot .load_manifest_list( - snapshot_produce.tx.current_table.file_io(), - &snapshot_produce.tx.current_table.metadata_ref(), + tx.current_table.file_io(), + &tx.current_table.metadata_ref(), ) .await?; @@ -222,7 +238,7 @@ impl SnapshotProduceOperation for FastAppendOperation { #[cfg(test)] mod tests { use std::collections::HashMap; - + use std::sync::Arc; use crate::scan::tests::TableTestFixture; use crate::spec::{ DataContentType, DataFileBuilder, DataFileFormat, Literal, MAIN_BRANCH, Struct, @@ -230,20 +246,21 @@ mod tests { use crate::transaction::Transaction; use crate::transaction::tests::make_v2_minimal_table; use crate::{TableRequirement, TableUpdate}; + use crate::transaction::action::TransactionAction; #[tokio::test] async fn test_empty_data_append_action() { let table = make_v2_minimal_table(); - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut action = tx.fast_append(None, vec![]).unwrap(); - action.add_data_files(vec![]).unwrap(); - assert!(action.apply().await.is_err()); + action.add_data_files(&tx, vec![]).unwrap(); + assert!(Arc::new(action).commit(&mut tx).await.is_err()); } #[tokio::test] async fn test_set_snapshot_properties() { let table = make_v2_minimal_table(); - let tx = Transaction::new(table.clone()); + let mut tx = Transaction::new(table.clone()); let mut action = tx.fast_append(None, vec![]).unwrap(); let mut snapshot_properties = HashMap::new(); @@ -259,8 +276,8 @@ mod tests { .partition(Struct::from_iter([Some(Literal::long(300))])) .build() .unwrap(); - action.add_data_files(vec![data_file]).unwrap(); - let tx = action.apply().await.unwrap(); + action.add_data_files(&tx, vec![data_file]).unwrap(); + Arc::new(action).commit(&mut tx).await.unwrap(); // Check customized properties is contained in snapshot summary properties. let new_snapshot = if let TableUpdate::AddSnapshot { snapshot } = &tx.updates[0] { @@ -281,7 +298,7 @@ mod tests { #[tokio::test] async fn test_fast_append_action() { let table = make_v2_minimal_table(); - let tx = Transaction::new(table.clone()); + let mut tx = Transaction::new(table.clone()); let mut action = tx.fast_append(None, vec![]).unwrap(); // check add data file with incompatible partition value @@ -295,7 +312,7 @@ mod tests { .partition(Struct::from_iter([Some(Literal::string("test"))])) .build() .unwrap(); - assert!(action.add_data_files(vec![data_file.clone()]).is_err()); + assert!(action.add_data_files(&tx, vec![data_file.clone()]).is_err()); let data_file = DataFileBuilder::default() .content(DataContentType::Data) @@ -307,8 +324,8 @@ mod tests { .partition(Struct::from_iter([Some(Literal::long(300))])) .build() .unwrap(); - action.add_data_files(vec![data_file.clone()]).unwrap(); - let tx = action.apply().await.unwrap(); + action.add_data_files(&tx, vec![data_file.clone()]).unwrap(); + Arc::new(action).commit(&mut tx).await.unwrap(); // check updates and requirements assert!( @@ -367,7 +384,7 @@ mod tests { async fn test_add_existing_parquet_files_to_unpartitioned_table() { let mut fixture = TableTestFixture::new_unpartitioned(); fixture.setup_unpartitioned_manifest_files().await; - let tx = crate::transaction::Transaction::new(fixture.table.clone()); + let mut tx = crate::transaction::Transaction::new(fixture.table.clone()); let file_paths = vec![ format!("{}/1.parquet", &fixture.table_location), @@ -378,14 +395,14 @@ mod tests { let fast_append_action = tx.fast_append(None, vec![]).unwrap(); // Attempt to add the existing Parquet files with fast append. - let new_tx = fast_append_action - .add_parquet_files(file_paths.clone()) + fast_append_action + .add_parquet_files(&mut tx, file_paths.clone()) .await .expect("Adding existing Parquet files should succeed"); let mut found_add_snapshot = false; let mut found_set_snapshot_ref = false; - for update in new_tx.updates.iter() { + for update in tx.updates.iter() { match update { TableUpdate::AddSnapshot { .. } => { found_add_snapshot = true; @@ -404,7 +421,7 @@ mod tests { assert!(found_add_snapshot); assert!(found_set_snapshot_ref); - let new_snapshot = if let TableUpdate::AddSnapshot { snapshot } = &new_tx.updates[0] { + let new_snapshot = if let TableUpdate::AddSnapshot { snapshot } = &tx.updates[0] { snapshot } else { panic!("Expected the first update to be an AddSnapshot update"); diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 348710507..5a424edec 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -35,7 +35,7 @@ use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{BoxedTransactionAction, UpdateLocationAction, TransactionAction}; +use crate::transaction::action::{BoxedTransactionAction, UpdateLocationAction}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -160,13 +160,12 @@ impl Transaction { /// Creates a fast append action. pub fn fast_append( - self, + &mut self, commit_uuid: Option, key_metadata: Vec, ) -> Result { let snapshot_id = self.generate_unique_snapshot_id(); FastAppendAction::new( - self, snapshot_id, commit_uuid.unwrap_or_else(Uuid::now_v7), key_metadata, @@ -261,7 +260,7 @@ impl Transaction { self.requirements = vec![]; for action in self.actions.clone() { - action.commit(self)? + action.commit(self).await? } } @@ -419,8 +418,8 @@ mod tests { ); } - #[test] - fn test_set_location() { + #[tokio::test] + async fn test_set_location() { let table = make_v2_table(); let mut tx = Transaction::new(table); let update_location_action = tx @@ -428,7 +427,7 @@ mod tests { .unwrap() .set_location(String::from("s3://bucket/prefix/new_table")); - let _res = Arc::new(update_location_action).commit(&mut tx).unwrap(); + let _res = Arc::new(update_location_action).commit(&mut tx).await; assert_eq!( vec![TableUpdate::SetLocation { diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index a1fc00b41..46b60c78e 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -43,6 +43,7 @@ pub(crate) trait SnapshotProduceOperation: Send + Sync { ) -> impl Future>> + Send; fn existing_manifest( &self, + tx: &Transaction, snapshot_produce: &SnapshotProduceAction, ) -> impl Future>> + Send; } @@ -60,7 +61,6 @@ pub(crate) trait ManifestProcess: Send + Sync { } pub(crate) struct SnapshotProduceAction { - pub tx: Transaction, snapshot_id: i64, key_metadata: Vec, commit_uuid: Uuid, @@ -74,14 +74,12 @@ pub(crate) struct SnapshotProduceAction { impl SnapshotProduceAction { pub(crate) fn new( - tx: Transaction, snapshot_id: i64, key_metadata: Vec, commit_uuid: Uuid, snapshot_properties: HashMap, ) -> Result { Ok(Self { - tx, snapshot_id, commit_uuid, snapshot_properties, @@ -134,6 +132,7 @@ impl SnapshotProduceAction { /// Add data files to the snapshot. pub fn add_data_files( &mut self, + tx: &Transaction, data_files: impl IntoIterator, ) -> Result<&mut Self> { let data_files: Vec = data_files.into_iter().collect(); @@ -145,7 +144,7 @@ impl SnapshotProduceAction { )); } // Check if the data file partition spec id matches the table default partition spec id. - if self.tx.current_table.metadata().default_partition_spec_id() + if tx.current_table.metadata().default_partition_spec_id() != data_file.partition_spec_id { return Err(Error::new( @@ -155,32 +154,32 @@ impl SnapshotProduceAction { } Self::validate_partition_value( data_file.partition(), - self.tx.current_table.metadata().default_partition_type(), + tx.current_table.metadata().default_partition_type(), )?; } self.added_data_files.extend(data_files); Ok(self) } - fn new_manifest_output(&mut self) -> Result { + fn new_manifest_output(&mut self, tx: &Transaction) -> Result { let new_manifest_path = format!( "{}/{}/{}-m{}.{}", - self.tx.current_table.metadata().location(), + tx.current_table.metadata().location(), META_ROOT_PATH, self.commit_uuid, self.manifest_counter.next().unwrap(), DataFileFormat::Avro ); - self.tx + tx .current_table .file_io() .new_output(new_manifest_path) } // 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, tx: &Transaction) -> Result { + // let added_data_files = std::mem::take(&mut self.added_data_files); + if self.added_data_files.is_empty() { return Err(Error::new( ErrorKind::PreconditionFailed, "No added data files found when write a manifest file", @@ -188,8 +187,8 @@ impl SnapshotProduceAction { } let snapshot_id = self.snapshot_id; - let format_version = self.tx.current_table.metadata().format_version(); - let manifest_entries = added_data_files.into_iter().map(|data_file| { + let format_version = tx.current_table.metadata().format_version(); + let manifest_entries = self.added_data_files.clone().into_iter().map(|data_file| { let builder = ManifestEntry::builder() .status(crate::spec::ManifestStatus::Added) .data_file(data_file); @@ -203,18 +202,18 @@ impl SnapshotProduceAction { }); let mut writer = { let builder = ManifestWriterBuilder::new( - self.new_manifest_output()?, + self.new_manifest_output(tx)?, Some(self.snapshot_id), self.key_metadata.clone(), - self.tx.current_table.metadata().current_schema().clone(), - self.tx + tx.current_table.metadata().current_schema().clone(), + tx .current_table .metadata() .default_partition_spec() .as_ref() .clone(), ); - if self.tx.current_table.metadata().format_version() == FormatVersion::V1 { + if tx.current_table.metadata().format_version() == FormatVersion::V1 { builder.build_v1() } else { builder.build_v2_data() @@ -228,11 +227,12 @@ impl SnapshotProduceAction { async fn manifest_file( &mut self, + tx: &Transaction, snapshot_produce_operation: &OP, manifest_process: &MP, ) -> Result> { - let added_manifest = self.write_added_manifest().await?; - let existing_manifests = snapshot_produce_operation.existing_manifest(self).await?; + let added_manifest = self.write_added_manifest(tx).await?; + let existing_manifests = snapshot_produce_operation.existing_manifest(tx, self).await?; // # TODO // Support process delete entries. @@ -245,10 +245,11 @@ impl SnapshotProduceAction { // Returns a `Summary` of the current snapshot fn summary( &self, + tx: &Transaction, snapshot_produce_operation: &OP, ) -> Result { let mut summary_collector = SnapshotSummaryCollector::default(); - let table_metadata = self.tx.current_table.metadata_ref(); + let table_metadata = tx.current_table.metadata_ref(); let partition_summary_limit = if let Some(limit) = table_metadata .properties() @@ -293,10 +294,10 @@ impl SnapshotProduceAction { ) } - fn generate_manifest_list_file_path(&self, attempt: i64) -> String { + fn generate_manifest_list_file_path(&self, tx: &Transaction, attempt: i64) -> String { format!( "{}/{}/snap-{}-{}-{}.{}", - self.tx.current_table.metadata().location(), + tx.current_table.metadata().location(), META_ROOT_PATH, self.snapshot_id, attempt, @@ -307,41 +308,42 @@ impl SnapshotProduceAction { /// Finished building the action and apply it to the transaction. pub async fn apply( - mut self, + &mut self, + tx: &mut Transaction, snapshot_produce_operation: OP, process: MP, - ) -> Result { + ) -> Result<()> { let new_manifests = self - .manifest_file(&snapshot_produce_operation, &process) + .manifest_file(tx, &snapshot_produce_operation, &process) .await?; - let next_seq_num = self.tx.current_table.metadata().next_sequence_number(); + let next_seq_num = tx.current_table.metadata().next_sequence_number(); let summary = self - .summary(&snapshot_produce_operation) + .summary(tx, &snapshot_produce_operation) .map_err(|err| { Error::new(ErrorKind::Unexpected, "Failed to create snapshot summary.") .with_source(err) }) .unwrap(); - let manifest_list_path = self.generate_manifest_list_file_path(0); + let manifest_list_path = self.generate_manifest_list_file_path(tx,0); - let mut manifest_list_writer = match self.tx.current_table.metadata().format_version() { + let mut manifest_list_writer = match tx.current_table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( - self.tx + tx .current_table .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, - self.tx.current_table.metadata().current_snapshot_id(), + tx.current_table.metadata().current_snapshot_id(), ), FormatVersion::V2 => ManifestListWriter::v2( - self.tx + tx .current_table .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, - self.tx.current_table.metadata().current_snapshot_id(), + tx.current_table.metadata().current_snapshot_id(), next_seq_num, ), }; @@ -352,10 +354,10 @@ impl SnapshotProduceAction { let new_snapshot = Snapshot::builder() .with_manifest_list(manifest_list_path) .with_snapshot_id(self.snapshot_id) - .with_parent_snapshot_id(self.tx.current_table.metadata().current_snapshot_id()) + .with_parent_snapshot_id(tx.current_table.metadata().current_snapshot_id()) .with_sequence_number(next_seq_num) .with_summary(summary) - .with_schema_id(self.tx.current_table.metadata().current_schema_id()) + .with_schema_id(tx.current_table.metadata().current_schema_id()) .with_timestamp_ms(commit_ts) .build(); @@ -374,16 +376,16 @@ impl SnapshotProduceAction { let requirements = vec![ TableRequirement::UuidMatch { - uuid: self.tx.current_table.metadata().uuid(), + uuid: tx.current_table.metadata().uuid(), }, TableRequirement::RefSnapshotIdMatch { r#ref: MAIN_BRANCH.to_string(), - snapshot_id: self.tx.current_table.metadata().current_snapshot_id(), + snapshot_id: tx.current_table.metadata().current_snapshot_id(), }, ]; - self.tx.apply(updates, requirements)?; + tx.apply(updates, requirements)?; - Ok(self.tx) + Ok(()) } } From 3ea76bbec9828dce69989e1ce9795c85d0850616 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 5 Jun 2025 20:04:45 -0700 Subject: [PATCH 20/25] fix build and fmt --- crates/iceberg/src/transaction/action/mod.rs | 7 +- crates/iceberg/src/transaction/append.rs | 68 ++++++++----------- crates/iceberg/src/transaction/mod.rs | 8 ++- crates/iceberg/src/transaction/snapshot.rs | 20 +++--- .../shared_tests/append_data_file_test.rs | 17 +++-- .../append_partition_data_file_test.rs | 15 ++-- .../shared_tests/conflict_commit_test.rs | 17 +++-- .../tests/shared_tests/scan_all_type.rs | 9 ++- 8 files changed, 85 insertions(+), 76 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index f08fbd5dc..b12acdc39 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -22,23 +22,28 @@ use async_trait::async_trait; use crate::transaction::Transaction; use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; +/// TODO doc pub type BoxedTransactionAction = Arc; +/// TODO doc #[async_trait] -pub(crate) trait TransactionAction: Sync + Send { +pub trait TransactionAction: Sync + Send { /// Commit the changes and apply the changes to the transaction async fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; } +/// TODO doc pub struct UpdateLocationAction { location: Option, } impl UpdateLocationAction { + /// TODO doc pub fn new() -> Self { UpdateLocationAction { location: None } } + /// TODO doc pub fn set_location(mut self, location: String) -> Self { self.location = Some(location); self diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index 906d1b8be..22e2d26c9 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -17,21 +17,22 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; + use arrow_array::StringArray; use async_trait::async_trait; -use futures::lock::Mutex; use futures::TryStreamExt; +use futures::lock::Mutex; use uuid::Uuid; use crate::error::Result; use crate::spec::{DataFile, ManifestEntry, ManifestFile, Operation}; use crate::transaction::Transaction; +use crate::transaction::action::TransactionAction; use crate::transaction::snapshot::{ DefaultManifestProcess, SnapshotProduceAction, SnapshotProduceOperation, }; use crate::writer::file_writer::ParquetWriter; use crate::{Error, ErrorKind}; -use crate::transaction::action::TransactionAction; /// FastAppendAction is a transaction action for fast append data files to the table. pub struct FastAppendAction { @@ -70,7 +71,9 @@ impl FastAppendAction { tx: &Transaction, data_files: impl IntoIterator, ) -> Result<&mut Self> { - self.snapshot_produce_action.get_mut().add_data_files(tx, data_files)?; + self.snapshot_produce_action + .get_mut() + .add_data_files(tx, data_files)?; Ok(self) } @@ -79,7 +82,8 @@ impl FastAppendAction { &mut self, snapshot_properties: HashMap, ) -> Result<&mut Self> { - self.snapshot_produce_action.get_mut() + self.snapshot_produce_action + .get_mut() .set_snapshot_properties(snapshot_properties)?; Ok(self) } @@ -91,13 +95,12 @@ impl FastAppendAction { /// Specifically, schema compatibility checks and support for adding to partitioned tables /// have not yet been implemented. #[allow(dead_code)] - async fn add_parquet_files(mut self, tx: &mut Transaction, file_path: Vec) -> Result<()> { - if !tx - .current_table - .metadata() - .default_spec - .is_unpartitioned() - { + async fn add_parquet_files( + mut self, + tx: &mut Transaction, + file_path: Vec, + ) -> Result<()> { + if !tx.current_table.metadata().default_spec.is_unpartitioned() { return Err(Error::new( ErrorKind::FeatureUnsupported, "Appending to partitioned tables is not supported", @@ -116,24 +119,24 @@ impl FastAppendAction { self.add_data_files(tx, data_files)?; Arc::new(self).commit(tx).await?; - + Ok(()) } // TODO remove this method - /// Finished building the action and apply it to the transaction. - pub async fn apply(self, tx: Transaction) -> Result { - - Ok(tx) - - } + // /// Finished building the action and apply it to the transaction. + // pub async fn apply(self, tx: Transaction) -> Result { + // + // Ok(tx) + // + // } } #[async_trait] impl TransactionAction for FastAppendAction { async fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { let mut action = self.snapshot_produce_action.lock().await; - + // Checks duplicate files if self.check_duplicate { let new_files: HashSet<&str> = action @@ -142,12 +145,7 @@ impl TransactionAction for FastAppendAction { .map(|df| df.file_path.as_str()) .collect(); - let mut manifest_stream = tx - .current_table - .inspect() - .manifests() - .scan() - .await?; + let mut manifest_stream = tx.current_table.inspect().manifests().scan().await?; let mut referenced_files = Vec::new(); while let Some(batch) = manifest_stream.try_next().await? { @@ -180,14 +178,14 @@ impl TransactionAction for FastAppendAction { )); } } - + action .apply(tx, FastAppendOperation, DefaultManifestProcess) .await?; - + drop(action); tx.actions.push(self); - + Ok(()) } } @@ -211,19 +209,12 @@ impl SnapshotProduceOperation for FastAppendOperation { tx: &Transaction, _snapshot_produce: &SnapshotProduceAction, ) -> Result> { - let Some(snapshot) = tx - .current_table - .metadata() - .current_snapshot() - else { + let Some(snapshot) = tx.current_table.metadata().current_snapshot() else { return Ok(vec![]); }; let manifest_list = snapshot - .load_manifest_list( - tx.current_table.file_io(), - &tx.current_table.metadata_ref(), - ) + .load_manifest_list(tx.current_table.file_io(), &tx.current_table.metadata_ref()) .await?; Ok(manifest_list @@ -239,14 +230,15 @@ impl SnapshotProduceOperation for FastAppendOperation { mod tests { use std::collections::HashMap; use std::sync::Arc; + use crate::scan::tests::TableTestFixture; use crate::spec::{ DataContentType, DataFileBuilder, DataFileFormat, Literal, MAIN_BRANCH, Struct, }; use crate::transaction::Transaction; + use crate::transaction::action::TransactionAction; use crate::transaction::tests::make_v2_minimal_table; use crate::{TableRequirement, TableUpdate}; - use crate::transaction::action::TransactionAction; #[tokio::test] async fn test_empty_data_append_action() { diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 5a424edec..58aa8ea97 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -17,7 +17,8 @@ //! This module contains transaction api. -mod action; +/// TODO doc +pub mod action; mod append; mod snapshot; mod sort_order; @@ -283,12 +284,13 @@ mod tests { use std::fs::File; use std::io::BufReader; use std::sync::Arc; + use crate::io::FileIOBuilder; use crate::spec::{FormatVersion, TableMetadata}; use crate::table::Table; use crate::transaction::Transaction; - use crate::{TableIdent, TableUpdate}; use crate::transaction::action::TransactionAction; + use crate::{TableIdent, TableUpdate}; fn make_v1_table() -> Table { let file = File::open(format!( @@ -426,7 +428,7 @@ mod tests { .update_location() .unwrap() .set_location(String::from("s3://bucket/prefix/new_table")); - + let _res = Arc::new(update_location_action).commit(&mut tx).await; assert_eq!( diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 46b60c78e..e88758a78 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -170,10 +170,7 @@ impl SnapshotProduceAction { self.manifest_counter.next().unwrap(), DataFileFormat::Avro ); - tx - .current_table - .file_io() - .new_output(new_manifest_path) + tx.current_table.file_io().new_output(new_manifest_path) } // Write manifest file for added data files and return the ManifestFile for ManifestList. @@ -206,8 +203,7 @@ impl SnapshotProduceAction { Some(self.snapshot_id), self.key_metadata.clone(), tx.current_table.metadata().current_schema().clone(), - tx - .current_table + tx.current_table .metadata() .default_partition_spec() .as_ref() @@ -232,7 +228,9 @@ impl SnapshotProduceAction { manifest_process: &MP, ) -> Result> { let added_manifest = self.write_added_manifest(tx).await?; - let existing_manifests = snapshot_produce_operation.existing_manifest(tx, self).await?; + let existing_manifests = snapshot_produce_operation + .existing_manifest(tx, self) + .await?; // # TODO // Support process delete entries. @@ -326,20 +324,18 @@ impl SnapshotProduceAction { }) .unwrap(); - let manifest_list_path = self.generate_manifest_list_file_path(tx,0); + let manifest_list_path = self.generate_manifest_list_file_path(tx, 0); let mut manifest_list_writer = match tx.current_table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( - tx - .current_table + tx.current_table .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, tx.current_table.metadata().current_snapshot_id(), ), FormatVersion::V2 => ManifestListWriter::v2( - tx - .current_table + tx.current_table .file_io() .new_output(manifest_list_path.clone())?, self.snapshot_id, 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 ff632925e..aeeda4466 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 @@ -22,6 +22,7 @@ use std::sync::Arc; use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; use futures::TryStreamExt; use iceberg::transaction::Transaction; +use iceberg::transaction::action::TransactionAction; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ @@ -111,10 +112,12 @@ async fn test_append_data_file() { assert_eq!(field_ids, vec![1, 2, 3]); // commit result - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); - append_action.add_data_files(data_file.clone()).unwrap(); - let mut tx = append_action.apply().await.unwrap(); + append_action + .add_data_files(&tx, data_file.clone()) + .unwrap(); + Arc::new(append_action).commit(&mut tx).await.unwrap(); let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result @@ -131,10 +134,12 @@ async fn test_append_data_file() { assert_eq!(batches[0], batch); // commit result again - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); - append_action.add_data_files(data_file.clone()).unwrap(); - let mut tx = append_action.apply().await.unwrap(); + append_action + .add_data_files(&tx, data_file.clone()) + .unwrap(); + Arc::new(append_action).commit(&mut tx).await.unwrap(); let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result again diff --git a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs index 3d0903491..f9e2de10e 100644 --- a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs @@ -24,6 +24,7 @@ use futures::TryStreamExt; use iceberg::spec::{Literal, PrimitiveLiteral, Struct, Transform, UnboundPartitionSpec}; use iceberg::table::Table; use iceberg::transaction::Transaction; +use iceberg::transaction::action::TransactionAction; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ @@ -119,12 +120,12 @@ async fn test_append_partition_data_file() { let data_file_valid = data_file_writer_valid.close().await.unwrap(); // commit result - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); append_action - .add_data_files(data_file_valid.clone()) + .add_data_files(&tx, data_file_valid.clone()) .unwrap(); - let mut tx = append_action.apply().await.unwrap(); + Arc::new(append_action).commit(&mut tx).await.unwrap(); let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result @@ -179,10 +180,10 @@ async fn test_schema_incompatible_partition_type( data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); if append_action - .add_data_files(data_file_invalid.clone()) + .add_data_files(&tx, data_file_invalid.clone()) .is_ok() { panic!("diverging partition info should have returned error"); @@ -219,10 +220,10 @@ async fn test_schema_incompatible_partition_fields( data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); if append_action - .add_data_files(data_file_invalid.clone()) + .add_data_files(&tx, data_file_invalid.clone()) .is_ok() { panic!("passing different number of partition fields should have returned error"); diff --git a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs index d54bad554..0f5302247 100644 --- a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs +++ b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; use futures::TryStreamExt; use iceberg::transaction::Transaction; +use iceberg::transaction::action::TransactionAction; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ @@ -89,15 +90,19 @@ async fn test_append_data_file_conflict() { let data_file = data_file_writer.close().await.unwrap(); // start two transaction and commit one of them - let tx1 = Transaction::new(table.clone()); + let mut tx1 = Transaction::new(table.clone()); let mut append_action = tx1.fast_append(None, vec![]).unwrap(); - append_action.add_data_files(data_file.clone()).unwrap(); - let mut tx1 = append_action.apply().await.unwrap(); + append_action + .add_data_files(&tx1, data_file.clone()) + .unwrap(); + Arc::new(append_action).commit(&mut tx1).await.unwrap(); - let tx2 = Transaction::new(table.clone()); + let mut tx2 = Transaction::new(table.clone()); let mut append_action = tx2.fast_append(None, vec![]).unwrap(); - append_action.add_data_files(data_file.clone()).unwrap(); - let mut tx2 = append_action.apply().await.unwrap(); + append_action + .add_data_files(&tx2, data_file.clone()) + .unwrap(); + Arc::new(append_action).commit(&mut tx2).await.unwrap(); let table = tx2 .commit(Arc::new(&rest_catalog)) .await diff --git a/crates/integration_tests/tests/shared_tests/scan_all_type.rs b/crates/integration_tests/tests/shared_tests/scan_all_type.rs index a0fac2f5c..9f607271c 100644 --- a/crates/integration_tests/tests/shared_tests/scan_all_type.rs +++ b/crates/integration_tests/tests/shared_tests/scan_all_type.rs @@ -34,6 +34,7 @@ use iceberg::spec::{ PrimitiveType, Schema, StructType, Type, }; use iceberg::transaction::Transaction; +use iceberg::transaction::action::TransactionAction; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ @@ -308,10 +309,12 @@ async fn test_scan_all_type() { let data_file = data_file_writer.close().await.unwrap(); // commit result - let tx = Transaction::new(table); + let mut tx = Transaction::new(table); let mut append_action = tx.fast_append(None, vec![]).unwrap(); - append_action.add_data_files(data_file.clone()).unwrap(); - let mut tx = append_action.apply().await.unwrap(); + append_action + .add_data_files(&tx, data_file.clone()) + .unwrap(); + Arc::new(append_action).commit(&mut tx).await.unwrap(); let table = tx.commit(Arc::new(&rest_catalog)).await.unwrap(); // check result From 4d1fff8b439b16842f434073c4f5073e867dbafb Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 6 Jun 2025 17:01:41 -0700 Subject: [PATCH 21/25] impl tx_action for update props, replace sort order, and update format location. removed updates and reqs from tx. Need to push fast append and sort order to tx.actions. --- crates/iceberg/src/transaction/action/mod.rs | 182 ++++++++++++++++++- crates/iceberg/src/transaction/append.rs | 115 ++++++------ crates/iceberg/src/transaction/mod.rs | 84 ++++----- crates/iceberg/src/transaction/snapshot.rs | 90 +++++---- crates/iceberg/src/transaction/sort_order.rs | 85 ++++----- 5 files changed, 336 insertions(+), 220 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index b12acdc39..ca1e2b430 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -15,11 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::cmp::Ordering; +use std::collections::{HashMap, HashSet}; +use std::mem::take; use std::sync::Arc; use async_trait::async_trait; +use tokio::sync::Mutex; -use crate::transaction::Transaction; +use crate::TableUpdate::UpgradeFormatVersion; +use crate::spec::FormatVersion; +use crate::table::Table; use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; /// TODO doc @@ -29,44 +35,200 @@ pub type BoxedTransactionAction = Arc; #[async_trait] pub trait TransactionAction: Sync + Send { /// Commit the changes and apply the changes to the transaction - async fn commit(self: Arc, tx: &mut Transaction) -> Result<()>; + async fn commit(self: Arc, table: &Table) -> Result; +} + +/// TODO doc +pub struct ActionCommit { + updates: Vec, + requirements: Vec, +} + +/// TODO doc +impl ActionCommit { + /// TODO doc + pub fn new(updates: Vec, requirements: Vec) -> Self { + Self { + updates, + requirements, + } + } + + /// TODO doc + pub fn take_updates(&mut self) -> Vec { + take(&mut self.updates) + } + + /// TODO doc + pub fn take_requirements(&mut self) -> Vec { + take(&mut self.requirements) + } } /// TODO doc pub struct UpdateLocationAction { + state: Mutex, +} + +struct UpdateLocationState { location: Option, } +impl UpdateLocationState { + fn new() -> Self { + Self { location: None } + } +} + impl UpdateLocationAction { /// TODO doc pub fn new() -> Self { - UpdateLocationAction { location: None } + UpdateLocationAction { + state: Mutex::new(UpdateLocationState::new()), + } } /// TODO doc - pub fn set_location(mut self, location: String) -> Self { - self.location = Some(location); + pub async fn set_location(self: Arc, location: String) -> Arc { + self.state.lock().await.location = Some(location); self } } #[async_trait] impl TransactionAction for UpdateLocationAction { - async fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { + async fn commit(self: Arc, _table: &Table) -> Result { let updates: Vec; let requirements: Vec; - if let Some(location) = self.location.clone() { + if let Some(location) = self.state.lock().await.location.clone() { updates = vec![TableUpdate::SetLocation { location }]; requirements = vec![]; } else { return Err(Error::new( ErrorKind::DataInvalid, - "Location is not set for SetLocation!", + "Location is not set for UpdateLocationAction!", )); } - tx.actions.push(self); + Ok(ActionCommit::new(updates, requirements)) + } +} + +/// TODO doc +pub struct UpgradeFormatVersionAction { + state: Mutex, +} + +struct UpgradeFormatVersionState { + format_version: Option, +} + +impl UpgradeFormatVersionState { + pub fn new() -> Self { + Self { + format_version: None, + } + } +} + +impl UpgradeFormatVersionAction { + /// TODO doc + pub fn new() -> Self { + UpgradeFormatVersionAction { + state: Mutex::new(UpgradeFormatVersionState::new()), + } + } + + /// TODO doc + pub async fn set_format_version(self: Arc, format_version: FormatVersion) -> Arc { + self.state.lock().await.format_version = Some(format_version); + self + } +} + +#[async_trait] +impl TransactionAction for UpgradeFormatVersionAction { + async fn commit(self: Arc, table: &Table) -> Result { + let current_version = table.metadata().format_version(); + let updates: Vec; + let requirements: Vec; + + if let Some(format_version) = self.state.lock().await.format_version { + match current_version.cmp(&format_version) { + Ordering::Greater => { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot downgrade table version from {} to {}", + current_version, format_version + ), + )); + } + Ordering::Less => { + updates = vec![UpgradeFormatVersion { format_version }]; + requirements = vec![]; + } + Ordering::Equal => { + // do nothing + updates = vec![]; + requirements = vec![]; + } + } + } else { + // error + return Err(Error::new( + ErrorKind::DataInvalid, + "FormatVersion is not set for UpgradeFormatVersionAction!", + )); + } + + Ok(ActionCommit::new(updates, requirements)) + } +} + +/// TODO doc +pub struct UpdatePropertiesAction { + updates: HashMap, + removals: HashSet, +} + +impl UpdatePropertiesAction { + /// TODO doc + pub fn new() -> Self { + UpdatePropertiesAction { + updates: HashMap::default(), + removals: HashSet::default(), + } + } + + /// TODO doc + pub fn set(mut self, key: String, value: String) -> Self { + assert!(!self.removals.contains(&key)); + self.updates.insert(key, value); + self + } + + /// TODO doc + pub fn remove(mut self, key: String) -> Self { + assert!(!self.updates.contains_key(&key)); + self.removals.insert(key); + self + } +} + +#[async_trait] +impl TransactionAction for UpdatePropertiesAction { + async fn commit(self: Arc, _table: &Table) -> Result { + let updates: Vec = vec![ + TableUpdate::SetProperties { + updates: self.updates.clone(), + }, + TableUpdate::RemoveProperties { + removals: self.removals.clone().into_iter().collect::>(), + }, + ]; + let requirements: Vec = vec![]; - tx.apply(updates, requirements) + Ok(ActionCommit::new(updates, requirements)) } } diff --git a/crates/iceberg/src/transaction/append.rs b/crates/iceberg/src/transaction/append.rs index 22e2d26c9..1a930d9ca 100644 --- a/crates/iceberg/src/transaction/append.rs +++ b/crates/iceberg/src/transaction/append.rs @@ -21,13 +21,12 @@ use std::sync::Arc; use arrow_array::StringArray; use async_trait::async_trait; use futures::TryStreamExt; -use futures::lock::Mutex; use uuid::Uuid; use crate::error::Result; use crate::spec::{DataFile, ManifestEntry, ManifestFile, Operation}; -use crate::transaction::Transaction; -use crate::transaction::action::TransactionAction; +use crate::table::Table; +use crate::transaction::action::{ActionCommit, TransactionAction}; use crate::transaction::snapshot::{ DefaultManifestProcess, SnapshotProduceAction, SnapshotProduceOperation, }; @@ -36,8 +35,11 @@ use crate::{Error, ErrorKind}; /// FastAppendAction is a transaction action for fast append data files to the table. pub struct FastAppendAction { - snapshot_produce_action: Mutex, + snapshot_produce_action: SnapshotProduceAction, check_duplicate: bool, + // below are properties used to create SnapshotProduceAction when commit + snapshot_properties: HashMap, + pub added_data_files: Vec, } impl FastAppendAction { @@ -49,13 +51,15 @@ impl FastAppendAction { snapshot_properties: HashMap, ) -> Result { Ok(Self { - snapshot_produce_action: Mutex::new(SnapshotProduceAction::new( + snapshot_produce_action: SnapshotProduceAction::new( snapshot_id, key_metadata, commit_uuid, - snapshot_properties, - )?), + snapshot_properties.clone(), + )?, check_duplicate: true, + snapshot_properties, + added_data_files: vec![], }) } @@ -68,12 +72,19 @@ impl FastAppendAction { /// Add data files to the snapshot. pub fn add_data_files( &mut self, - tx: &Transaction, data_files: impl IntoIterator, ) -> Result<&mut Self> { - self.snapshot_produce_action - .get_mut() - .add_data_files(tx, data_files)?; + let data_files: Vec = data_files.into_iter().collect(); + for data_file in &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", + )); + } + } + self.added_data_files.extend(data_files); + Ok(self) } @@ -82,9 +93,7 @@ impl FastAppendAction { &mut self, snapshot_properties: HashMap, ) -> Result<&mut Self> { - self.snapshot_produce_action - .get_mut() - .set_snapshot_properties(snapshot_properties)?; + self.snapshot_properties = snapshot_properties; Ok(self) } @@ -95,57 +104,44 @@ impl FastAppendAction { /// Specifically, schema compatibility checks and support for adding to partitioned tables /// have not yet been implemented. #[allow(dead_code)] - async fn add_parquet_files( - mut self, - tx: &mut Transaction, - file_path: Vec, - ) -> Result<()> { - if !tx.current_table.metadata().default_spec.is_unpartitioned() { + async fn add_parquet_files(mut self, table: &Table, file_path: Vec) -> Result<()> { + if !table.metadata().default_spec.is_unpartitioned() { return Err(Error::new( ErrorKind::FeatureUnsupported, "Appending to partitioned tables is not supported", )); } - let table_metadata = tx.current_table.metadata(); - - let data_files = ParquetWriter::parquet_files_to_data_files( - tx.current_table.file_io(), - file_path, - table_metadata, - ) - .await?; + let table_metadata = table.metadata(); - self.add_data_files(tx, data_files)?; + let data_files = + ParquetWriter::parquet_files_to_data_files(table.file_io(), file_path, table_metadata) + .await?; - Arc::new(self).commit(tx).await?; + self.add_data_files(data_files)?; Ok(()) } - - // TODO remove this method - // /// Finished building the action and apply it to the transaction. - // pub async fn apply(self, tx: Transaction) -> Result { - // - // Ok(tx) - // - // } } #[async_trait] impl TransactionAction for FastAppendAction { - async fn commit(self: Arc, tx: &mut Transaction) -> Result<()> { - let mut action = self.snapshot_produce_action.lock().await; + async fn commit(self: Arc, table: &Table) -> Result { + let mut snapshot_produce_action = self.snapshot_produce_action.clone(); + + let snapshot_produce_action = snapshot_produce_action + .add_data_files(table, self.added_data_files.clone())? + .set_snapshot_properties(self.snapshot_properties.clone())?; // Checks duplicate files if self.check_duplicate { - let new_files: HashSet<&str> = action + let new_files: HashSet<&str> = snapshot_produce_action .added_data_files .iter() .map(|df| df.file_path.as_str()) .collect(); - let mut manifest_stream = tx.current_table.inspect().manifests().scan().await?; + let mut manifest_stream = table.inspect().manifests().scan().await?; let mut referenced_files = Vec::new(); while let Some(batch) = manifest_stream.try_next().await? { @@ -179,14 +175,9 @@ impl TransactionAction for FastAppendAction { } } - action - .apply(tx, FastAppendOperation, DefaultManifestProcess) - .await?; - - drop(action); - tx.actions.push(self); - - Ok(()) + snapshot_produce_action + .apply(table, FastAppendOperation, DefaultManifestProcess) + .await } } @@ -206,15 +197,15 @@ impl SnapshotProduceOperation for FastAppendOperation { async fn existing_manifest( &self, - tx: &Transaction, + table: &Table, _snapshot_produce: &SnapshotProduceAction, ) -> Result> { - let Some(snapshot) = tx.current_table.metadata().current_snapshot() else { + let Some(snapshot) = table.metadata().current_snapshot() else { return Ok(vec![]); }; let manifest_list = snapshot - .load_manifest_list(tx.current_table.file_io(), &tx.current_table.metadata_ref()) + .load_manifest_list(table.file_io(), &table.metadata_ref()) .await?; Ok(manifest_list @@ -245,8 +236,8 @@ mod tests { let table = make_v2_minimal_table(); let mut tx = Transaction::new(table); let mut action = tx.fast_append(None, vec![]).unwrap(); - action.add_data_files(&tx, vec![]).unwrap(); - assert!(Arc::new(action).commit(&mut tx).await.is_err()); + action.add_data_files(vec![]).unwrap(); + assert!(Arc::new(action).commit(&table).await.is_err()); } #[tokio::test] @@ -268,8 +259,8 @@ mod tests { .partition(Struct::from_iter([Some(Literal::long(300))])) .build() .unwrap(); - action.add_data_files(&tx, vec![data_file]).unwrap(); - Arc::new(action).commit(&mut tx).await.unwrap(); + action.add_data_files(vec![data_file]).unwrap(); + Arc::new(action).commit(&table).await.unwrap(); // Check customized properties is contained in snapshot summary properties. let new_snapshot = if let TableUpdate::AddSnapshot { snapshot } = &tx.updates[0] { @@ -304,7 +295,7 @@ mod tests { .partition(Struct::from_iter([Some(Literal::string("test"))])) .build() .unwrap(); - assert!(action.add_data_files(&tx, vec![data_file.clone()]).is_err()); + assert!(action.add_data_files(vec![data_file.clone()]).is_err()); let data_file = DataFileBuilder::default() .content(DataContentType::Data) @@ -316,8 +307,8 @@ mod tests { .partition(Struct::from_iter([Some(Literal::long(300))])) .build() .unwrap(); - action.add_data_files(&tx, vec![data_file.clone()]).unwrap(); - Arc::new(action).commit(&mut tx).await.unwrap(); + action.add_data_files(vec![data_file.clone()]).unwrap(); + Arc::new(action).commit(&tx.current_table).await.unwrap(); // check updates and requirements assert!( @@ -376,7 +367,7 @@ mod tests { async fn test_add_existing_parquet_files_to_unpartitioned_table() { let mut fixture = TableTestFixture::new_unpartitioned(); fixture.setup_unpartitioned_manifest_files().await; - let mut tx = crate::transaction::Transaction::new(fixture.table.clone()); + let mut tx = Transaction::new(fixture.table.clone()); let file_paths = vec![ format!("{}/1.parquet", &fixture.table_location), @@ -388,7 +379,7 @@ mod tests { // Attempt to add the existing Parquet files with fast append. fast_append_action - .add_parquet_files(&mut tx, file_paths.clone()) + .add_parquet_files(&tx.base_table, file_paths.clone()) .await .expect("Adding existing Parquet files should succeed"); diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 58aa8ea97..0cca1eff1 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -23,7 +23,6 @@ mod append; mod snapshot; mod sort_order; -use std::cmp::Ordering; use std::collections::HashMap; use std::mem::discriminant; use std::sync::Arc; @@ -32,11 +31,12 @@ use std::time::Duration; use backon::{BackoffBuilder, ExponentialBuilder, RetryableWithContext}; use uuid::Uuid; -use crate::TableUpdate::UpgradeFormatVersion; use crate::error::Result; -use crate::spec::FormatVersion; use crate::table::Table; -use crate::transaction::action::{BoxedTransactionAction, UpdateLocationAction}; +use crate::transaction::action::{ + BoxedTransactionAction, UpdateLocationAction, UpdatePropertiesAction, + UpgradeFormatVersionAction, +}; use crate::transaction::append::FastAppendAction; use crate::transaction::sort_order::ReplaceSortOrderAction; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; @@ -47,8 +47,6 @@ pub struct Transaction { base_table: Table, current_table: Table, actions: Vec, - updates: Vec, - requirements: Vec, } impl Transaction { @@ -58,8 +56,6 @@ impl Transaction { base_table: table.clone(), current_table: table.clone(), actions: vec![], - updates: vec![], - requirements: vec![], } } @@ -80,6 +76,8 @@ impl Transaction { &mut self, updates: Vec, requirements: Vec, + existing_updates: &mut Vec, + existing_requirements: &mut Vec, ) -> Result<()> { for requirement in &requirements { requirement.check(Some(self.current_table.metadata()))?; @@ -87,18 +85,17 @@ impl Transaction { self.update_table_metadata(&updates)?; - self.updates.extend(updates); + existing_updates.extend(updates); // For the requirements, it does not make sense to add a requirement more than once // For example, you cannot assert that the current schema has two different IDs for new_requirement in requirements { - if self - .requirements + if existing_requirements .iter() .map(discriminant) .all(|d| d != discriminant(&new_requirement)) { - self.requirements.push(new_requirement); + existing_requirements.push(new_requirement); } } @@ -109,32 +106,13 @@ impl Transaction { } /// Sets table to a new version. - pub fn upgrade_table_version(mut self, format_version: FormatVersion) -> Result { - let current_version = self.current_table.metadata().format_version(); - match current_version.cmp(&format_version) { - Ordering::Greater => { - return Err(Error::new( - ErrorKind::DataInvalid, - format!( - "Cannot downgrade table version from {} to {}", - current_version, format_version - ), - )); - } - Ordering::Less => { - self.apply(vec![UpgradeFormatVersion { format_version }], vec![])?; - } - Ordering::Equal => { - // Do nothing. - } - } - Ok(self) + pub fn upgrade_table_version(&self) -> Result { + Ok(UpgradeFormatVersionAction::new()) } /// Update table's property. - pub fn set_properties(mut self, props: HashMap) -> Result { - self.apply(vec![TableUpdate::SetProperties { updates: props }], vec![])?; - Ok(self) + pub fn update_properties(&self) -> Result { + Ok(UpdatePropertiesAction::new()) } fn generate_unique_snapshot_id(&self) -> i64 { @@ -177,23 +155,15 @@ impl Transaction { /// Creates replace sort order action. pub fn replace_sort_order(self) -> ReplaceSortOrderAction { ReplaceSortOrderAction { - tx: self, sort_fields: vec![], } } - /// Remove properties in table. - pub fn remove_properties(mut self, keys: Vec) -> Result { - self.apply( - vec![TableUpdate::RemoveProperties { removals: keys }], - vec![], - )?; - Ok(self) - } - /// Set the location of table - pub fn update_location(&mut self) -> Result { - Ok(UpdateLocationAction::new()) + pub fn update_location(&mut self) -> Result> { + let update_location_action = Arc::new(UpdateLocationAction::new()); + self.actions.push(update_location_action.clone()); + Ok(update_location_action) } /// Commit transaction. @@ -251,24 +221,32 @@ impl Transaction { .await .expect(format!("Failed to refresh table {}", base_table_identifier).as_str()); + let mut existing_updates: Vec = vec![]; + let mut existing_requirements: Vec = vec![]; + if self.base_table.metadata() != refreshed.metadata() || self.base_table.metadata_location() != refreshed.metadata_location() { // current base is stale, use refreshed as base and re-apply transaction actions self.base_table = refreshed.clone(); - self.current_table = refreshed.clone(); - self.updates = vec![]; - self.requirements = vec![]; + self.current_table = refreshed.clone(); // todo use a temp table for action in self.actions.clone() { - action.commit(self).await? + let mut action_commit = action.commit(&self.current_table).await?; + // apply changes to current_table + self.apply( + action_commit.take_updates(), + action_commit.take_requirements(), + &mut existing_updates, + &mut existing_requirements, + )?; } } let table_commit = TableCommit::builder() .ident(base_table_identifier.clone()) - .updates(self.updates.clone()) - .requirements(self.requirements.clone()) + .updates(existing_updates) + .requirements(existing_requirements) .build(); catalog diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index e88758a78..a9217835b 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -29,7 +29,8 @@ use crate::spec::{ PROPERTY_WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT, Snapshot, SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, StructType, Summary, update_snapshot_summaries, }; -use crate::transaction::Transaction; +use crate::table::Table; +use crate::transaction::action::ActionCommit; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; const META_ROOT_PATH: &str = "metadata"; @@ -43,7 +44,7 @@ pub(crate) trait SnapshotProduceOperation: Send + Sync { ) -> impl Future>> + Send; fn existing_manifest( &self, - tx: &Transaction, + table: &Table, snapshot_produce: &SnapshotProduceAction, ) -> impl Future>> + Send; } @@ -60,6 +61,7 @@ pub(crate) trait ManifestProcess: Send + Sync { fn process_manifests(&self, manifests: Vec) -> Vec; } +#[derive(Clone)] pub(crate) struct SnapshotProduceAction { snapshot_id: i64, key_metadata: Vec, @@ -120,6 +122,7 @@ impl SnapshotProduceAction { Ok(()) } + // todo move this fast_append /// Set snapshot summary properties. pub fn set_snapshot_properties( &mut self, @@ -129,10 +132,11 @@ impl SnapshotProduceAction { Ok(self) } + // TODO moving this to fast append /// Add data files to the snapshot. pub fn add_data_files( &mut self, - tx: &Transaction, + table: &Table, data_files: impl IntoIterator, ) -> Result<&mut Self> { let data_files: Vec = data_files.into_iter().collect(); @@ -144,9 +148,7 @@ impl SnapshotProduceAction { )); } // Check if the data file partition spec id matches the table default partition spec id. - if tx.current_table.metadata().default_partition_spec_id() - != data_file.partition_spec_id - { + if table.metadata().default_partition_spec_id() != data_file.partition_spec_id { return Err(Error::new( ErrorKind::DataInvalid, "Data file partition spec id does not match table default partition spec id", @@ -154,27 +156,27 @@ impl SnapshotProduceAction { } Self::validate_partition_value( data_file.partition(), - tx.current_table.metadata().default_partition_type(), + table.metadata().default_partition_type(), )?; } self.added_data_files.extend(data_files); Ok(self) } - fn new_manifest_output(&mut self, tx: &Transaction) -> Result { + fn new_manifest_output(&mut self, table: &Table) -> Result { let new_manifest_path = format!( "{}/{}/{}-m{}.{}", - tx.current_table.metadata().location(), + table.metadata().location(), META_ROOT_PATH, self.commit_uuid, self.manifest_counter.next().unwrap(), DataFileFormat::Avro ); - tx.current_table.file_io().new_output(new_manifest_path) + table.file_io().new_output(new_manifest_path) } // Write manifest file for added data files and return the ManifestFile for ManifestList. - async fn write_added_manifest(&mut self, tx: &Transaction) -> Result { + async fn write_added_manifest(&mut self, table: &Table) -> Result { // let added_data_files = std::mem::take(&mut self.added_data_files); if self.added_data_files.is_empty() { return Err(Error::new( @@ -184,7 +186,7 @@ impl SnapshotProduceAction { } let snapshot_id = self.snapshot_id; - let format_version = tx.current_table.metadata().format_version(); + let format_version = table.metadata().format_version(); let manifest_entries = self.added_data_files.clone().into_iter().map(|data_file| { let builder = ManifestEntry::builder() .status(crate::spec::ManifestStatus::Added) @@ -199,17 +201,13 @@ impl SnapshotProduceAction { }); let mut writer = { let builder = ManifestWriterBuilder::new( - self.new_manifest_output(tx)?, + self.new_manifest_output(table)?, Some(self.snapshot_id), self.key_metadata.clone(), - tx.current_table.metadata().current_schema().clone(), - tx.current_table - .metadata() - .default_partition_spec() - .as_ref() - .clone(), + table.metadata().current_schema().clone(), + table.metadata().default_partition_spec().as_ref().clone(), ); - if tx.current_table.metadata().format_version() == FormatVersion::V1 { + if table.metadata().format_version() == FormatVersion::V1 { builder.build_v1() } else { builder.build_v2_data() @@ -223,13 +221,13 @@ impl SnapshotProduceAction { async fn manifest_file( &mut self, - tx: &Transaction, + table: &Table, snapshot_produce_operation: &OP, manifest_process: &MP, ) -> Result> { - let added_manifest = self.write_added_manifest(tx).await?; + let added_manifest = self.write_added_manifest(table).await?; let existing_manifests = snapshot_produce_operation - .existing_manifest(tx, self) + .existing_manifest(table, self) .await?; // # TODO // Support process delete entries. @@ -243,11 +241,11 @@ impl SnapshotProduceAction { // Returns a `Summary` of the current snapshot fn summary( &self, - tx: &Transaction, + table: &Table, snapshot_produce_operation: &OP, ) -> Result { let mut summary_collector = SnapshotSummaryCollector::default(); - let table_metadata = tx.current_table.metadata_ref(); + let table_metadata = table.metadata_ref(); let partition_summary_limit = if let Some(limit) = table_metadata .properties() @@ -292,10 +290,10 @@ impl SnapshotProduceAction { ) } - fn generate_manifest_list_file_path(&self, tx: &Transaction, attempt: i64) -> String { + fn generate_manifest_list_file_path(&self, table: &Table, attempt: i64) -> String { format!( "{}/{}/snap-{}-{}-{}.{}", - tx.current_table.metadata().location(), + table.metadata().location(), META_ROOT_PATH, self.snapshot_id, attempt, @@ -307,39 +305,35 @@ impl SnapshotProduceAction { /// Finished building the action and apply it to the transaction. pub async fn apply( &mut self, - tx: &mut Transaction, + table: &Table, snapshot_produce_operation: OP, process: MP, - ) -> Result<()> { + ) -> Result { let new_manifests = self - .manifest_file(tx, &snapshot_produce_operation, &process) + .manifest_file(table, &snapshot_produce_operation, &process) .await?; - let next_seq_num = tx.current_table.metadata().next_sequence_number(); + let next_seq_num = table.metadata().next_sequence_number(); let summary = self - .summary(tx, &snapshot_produce_operation) + .summary(table, &snapshot_produce_operation) .map_err(|err| { Error::new(ErrorKind::Unexpected, "Failed to create snapshot summary.") .with_source(err) }) .unwrap(); - let manifest_list_path = self.generate_manifest_list_file_path(tx, 0); + let manifest_list_path = self.generate_manifest_list_file_path(table, 0); - let mut manifest_list_writer = match tx.current_table.metadata().format_version() { + let mut manifest_list_writer = match table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( - tx.current_table - .file_io() - .new_output(manifest_list_path.clone())?, + table.file_io().new_output(manifest_list_path.clone())?, self.snapshot_id, - tx.current_table.metadata().current_snapshot_id(), + table.metadata().current_snapshot_id(), ), FormatVersion::V2 => ManifestListWriter::v2( - tx.current_table - .file_io() - .new_output(manifest_list_path.clone())?, + table.file_io().new_output(manifest_list_path.clone())?, self.snapshot_id, - tx.current_table.metadata().current_snapshot_id(), + table.metadata().current_snapshot_id(), next_seq_num, ), }; @@ -350,10 +344,10 @@ impl SnapshotProduceAction { let new_snapshot = Snapshot::builder() .with_manifest_list(manifest_list_path) .with_snapshot_id(self.snapshot_id) - .with_parent_snapshot_id(tx.current_table.metadata().current_snapshot_id()) + .with_parent_snapshot_id(table.metadata().current_snapshot_id()) .with_sequence_number(next_seq_num) .with_summary(summary) - .with_schema_id(tx.current_table.metadata().current_schema_id()) + .with_schema_id(table.metadata().current_schema_id()) .with_timestamp_ms(commit_ts) .build(); @@ -372,16 +366,14 @@ impl SnapshotProduceAction { let requirements = vec![ TableRequirement::UuidMatch { - uuid: tx.current_table.metadata().uuid(), + uuid: table.metadata().uuid(), }, TableRequirement::RefSnapshotIdMatch { r#ref: MAIN_BRANCH.to_string(), - snapshot_id: tx.current_table.metadata().current_snapshot_id(), + snapshot_id: table.metadata().current_snapshot_id(), }, ]; - tx.apply(updates, requirements)?; - - Ok(()) + Ok(ActionCommit::new(updates, requirements)) } } diff --git a/crates/iceberg/src/transaction/sort_order.rs b/crates/iceberg/src/transaction/sort_order.rs index 7564445e4..643d7cf6a 100644 --- a/crates/iceberg/src/transaction/sort_order.rs +++ b/crates/iceberg/src/transaction/sort_order.rs @@ -15,74 +15,40 @@ // 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::{NullOrder, SortDirection, SortField, SortOrder, Transform}; -use crate::transaction::Transaction; +use crate::table::Table; +use crate::transaction::action::{ActionCommit, TransactionAction}; use crate::{Error, ErrorKind, TableRequirement, TableUpdate}; /// Transaction action for replacing sort order. pub struct ReplaceSortOrderAction { - pub tx: Transaction, pub sort_fields: Vec, } impl ReplaceSortOrderAction { /// Adds a field for sorting in ascending order. - pub fn asc(self, name: &str, null_order: NullOrder) -> Result { - self.add_sort_field(name, SortDirection::Ascending, null_order) + pub fn asc(self, table: &Table, name: &str, null_order: NullOrder) -> Result { + self.add_sort_field(table, name, SortDirection::Ascending, null_order) } /// Adds a field for sorting in descending order. - pub fn desc(self, name: &str, null_order: NullOrder) -> Result { - self.add_sort_field(name, SortDirection::Descending, null_order) - } - - /// Finished building the action and apply it to the transaction. - pub fn apply(mut self) -> Result { - let unbound_sort_order = SortOrder::builder() - .with_fields(self.sort_fields) - .build_unbound()?; - - let updates = vec![ - TableUpdate::AddSortOrder { - sort_order: unbound_sort_order, - }, - TableUpdate::SetDefaultSortOrder { sort_order_id: -1 }, - ]; - - let requirements = vec![ - TableRequirement::CurrentSchemaIdMatch { - current_schema_id: self - .tx - .current_table - .metadata() - .current_schema() - .schema_id(), - }, - TableRequirement::DefaultSortOrderIdMatch { - default_sort_order_id: self - .tx - .current_table - .metadata() - .default_sort_order() - .order_id, - }, - ]; - - self.tx.apply(updates, requirements)?; - - Ok(self.tx) + pub fn desc(self, table: &Table, name: &str, null_order: NullOrder) -> Result { + self.add_sort_field(table, name, SortDirection::Descending, null_order) } fn add_sort_field( mut self, + table: &Table, name: &str, sort_direction: SortDirection, null_order: NullOrder, ) -> Result { - let field_id = self - .tx - .current_table + let field_id = table .metadata() .current_schema() .field_id_by_name(name) @@ -105,6 +71,33 @@ impl ReplaceSortOrderAction { } } +#[async_trait] +impl TransactionAction for ReplaceSortOrderAction { + async fn commit(self: Arc, table: &Table) -> Result { + let unbound_sort_order = SortOrder::builder() + .with_fields(self.sort_fields.clone()) + .build_unbound()?; + + let updates = vec![ + TableUpdate::AddSortOrder { + sort_order: unbound_sort_order, + }, + TableUpdate::SetDefaultSortOrder { sort_order_id: -1 }, + ]; + + let requirements = vec![ + TableRequirement::CurrentSchemaIdMatch { + current_schema_id: table.metadata().current_schema().schema_id(), + }, + TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id: table.metadata().default_sort_order().order_id, + }, + ]; + + Ok(ActionCommit::new(updates, requirements)) + } +} + #[cfg(test)] mod tests { use crate::transaction::Transaction; From e565f765e3db6c0096260d803b0bbdb3831cbc6c Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 6 Jun 2025 21:29:05 -0700 Subject: [PATCH 22/25] Add ApplyTransactionAction, remove mutex and state, stop adding actions in tx --- crates/iceberg/src/transaction/action/mod.rs | 56 ++++++++------------ crates/iceberg/src/transaction/mod.rs | 16 +++--- 2 files changed, 30 insertions(+), 42 deletions(-) diff --git a/crates/iceberg/src/transaction/action/mod.rs b/crates/iceberg/src/transaction/action/mod.rs index ca1e2b430..cda0c21a7 100644 --- a/crates/iceberg/src/transaction/action/mod.rs +++ b/crates/iceberg/src/transaction/action/mod.rs @@ -21,11 +21,11 @@ use std::mem::take; use std::sync::Arc; use async_trait::async_trait; -use tokio::sync::Mutex; use crate::TableUpdate::UpgradeFormatVersion; use crate::spec::FormatVersion; use crate::table::Table; +use crate::transaction::Transaction; use crate::{Error, ErrorKind, Result, TableRequirement, TableUpdate}; /// TODO doc @@ -38,6 +38,20 @@ pub trait TransactionAction: Sync + Send { async fn commit(self: Arc, table: &Table) -> Result; } +/// TODO doc +pub trait ApplyTransactionAction { + /// TODO doc + fn apply(self, tx: Transaction) -> Result; +} + +impl ApplyTransactionAction for T { + fn apply(self, mut tx: Transaction) -> Result + where Self: Sized { + tx.actions.push(Arc::new(self)); + Ok(tx) + } +} + /// TODO doc pub struct ActionCommit { updates: Vec, @@ -67,30 +81,18 @@ impl ActionCommit { /// TODO doc pub struct UpdateLocationAction { - state: Mutex, -} - -struct UpdateLocationState { location: Option, } -impl UpdateLocationState { - fn new() -> Self { - Self { location: None } - } -} - impl UpdateLocationAction { /// TODO doc pub fn new() -> Self { - UpdateLocationAction { - state: Mutex::new(UpdateLocationState::new()), - } + UpdateLocationAction { location: None } } /// TODO doc - pub async fn set_location(self: Arc, location: String) -> Arc { - self.state.lock().await.location = Some(location); + pub fn set_location(mut self, location: String) -> Self { + self.location = Some(location); self } } @@ -100,7 +102,7 @@ impl TransactionAction for UpdateLocationAction { async fn commit(self: Arc, _table: &Table) -> Result { let updates: Vec; let requirements: Vec; - if let Some(location) = self.state.lock().await.location.clone() { + if let Some(location) = self.location.clone() { updates = vec![TableUpdate::SetLocation { location }]; requirements = vec![]; } else { @@ -116,32 +118,20 @@ impl TransactionAction for UpdateLocationAction { /// TODO doc pub struct UpgradeFormatVersionAction { - state: Mutex, -} - -struct UpgradeFormatVersionState { format_version: Option, } -impl UpgradeFormatVersionState { - pub fn new() -> Self { - Self { - format_version: None, - } - } -} - impl UpgradeFormatVersionAction { /// TODO doc pub fn new() -> Self { UpgradeFormatVersionAction { - state: Mutex::new(UpgradeFormatVersionState::new()), + format_version: None, } } /// TODO doc - pub async fn set_format_version(self: Arc, format_version: FormatVersion) -> Arc { - self.state.lock().await.format_version = Some(format_version); + pub fn set_format_version(mut self, format_version: FormatVersion) -> Self { + self.format_version = Some(format_version); self } } @@ -153,7 +143,7 @@ impl TransactionAction for UpgradeFormatVersionAction { let updates: Vec; let requirements: Vec; - if let Some(format_version) = self.state.lock().await.format_version { + if let Some(format_version) = self.format_version { match current_version.cmp(&format_version) { Ordering::Greater => { return Err(Error::new( diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 0cca1eff1..303e2c3c0 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -34,7 +34,7 @@ use uuid::Uuid; use crate::error::Result; use crate::table::Table; use crate::transaction::action::{ - BoxedTransactionAction, UpdateLocationAction, UpdatePropertiesAction, + ApplyTransactionAction, BoxedTransactionAction, UpdateLocationAction, UpdatePropertiesAction, UpgradeFormatVersionAction, }; use crate::transaction::append::FastAppendAction; @@ -106,13 +106,13 @@ impl Transaction { } /// Sets table to a new version. - pub fn upgrade_table_version(&self) -> Result { - Ok(UpgradeFormatVersionAction::new()) + pub fn upgrade_table_version(&self) -> UpgradeFormatVersionAction { + UpgradeFormatVersionAction::new() } /// Update table's property. - pub fn update_properties(&self) -> Result { - Ok(UpdatePropertiesAction::new()) + pub fn update_properties(&self) -> UpdatePropertiesAction { + UpdatePropertiesAction::new() } fn generate_unique_snapshot_id(&self) -> i64 { @@ -160,10 +160,8 @@ impl Transaction { } /// Set the location of table - pub fn update_location(&mut self) -> Result> { - let update_location_action = Arc::new(UpdateLocationAction::new()); - self.actions.push(update_location_action.clone()); - Ok(update_location_action) + pub fn update_location(&mut self) -> UpdateLocationAction { + UpdateLocationAction::new() } /// Commit transaction. From 3c0bdd7918bcadf920e43458008d7fc350b41ec5 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 6 Jun 2025 21:45:19 -0700 Subject: [PATCH 23/25] removed current_table from tx --- crates/iceberg/src/transaction/mod.rs | 49 +++++++++++++-------------- 1 file changed, 23 insertions(+), 26 deletions(-) diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index 303e2c3c0..e3d58c840 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -45,7 +45,6 @@ use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdat #[derive(Clone)] pub struct Transaction { base_table: Table, - current_table: Table, actions: Vec, } @@ -54,19 +53,17 @@ impl Transaction { pub fn new(table: Table) -> Self { Self { base_table: table.clone(), - current_table: table.clone(), actions: vec![], } } - fn update_table_metadata(&mut self, updates: &[TableUpdate]) -> Result<()> { - let mut metadata_builder = self.current_table.metadata().clone().into_builder(None); + fn update_table_metadata(&mut self, table: &mut Table, updates: &[TableUpdate]) -> Result<()> { + let mut metadata_builder = table.metadata().clone().into_builder(None); for update in updates { metadata_builder = update.clone().apply(metadata_builder)?; } - self.current_table - .with_metadata(Arc::new(metadata_builder.build()?.metadata)); + table.with_metadata(Arc::new(metadata_builder.build()?.metadata)); Ok(()) } @@ -74,16 +71,17 @@ impl Transaction { /// TODO documentation pub fn apply( &mut self, + table: &mut Table, updates: Vec, requirements: Vec, existing_updates: &mut Vec, existing_requirements: &mut Vec, ) -> Result<()> { for requirement in &requirements { - requirement.check(Some(self.current_table.metadata()))?; + requirement.check(Some(table.metadata()))?; } - self.update_table_metadata(&updates)?; + self.update_table_metadata(table, &updates)?; existing_updates.extend(updates); @@ -127,7 +125,7 @@ impl Transaction { }; let mut snapshot_id = generate_random_id(); while self - .current_table + .base_table .metadata() .snapshots() .any(|s| s.snapshot_id() == snapshot_id) @@ -166,12 +164,9 @@ impl Transaction { /// Commit transaction. pub async fn commit(&mut self, catalog: Arc<&dyn Catalog>) -> Result
{ - if self.actions.is_empty() - || (self.base_table.metadata() == self.current_table.metadata() - && self.base_table.metadata_location() == self.current_table.metadata_location()) - { + if self.actions.is_empty() { // nothing to commit - return Ok(self.current_table.clone()); + return Ok(self.base_table.clone()); } let tx = self.clone(); @@ -227,18 +222,20 @@ impl Transaction { { // current base is stale, use refreshed as base and re-apply transaction actions self.base_table = refreshed.clone(); - self.current_table = refreshed.clone(); // todo use a temp table - - for action in self.actions.clone() { - let mut action_commit = action.commit(&self.current_table).await?; - // apply changes to current_table - self.apply( - action_commit.take_updates(), - action_commit.take_requirements(), - &mut existing_updates, - &mut existing_requirements, - )?; - } + } + + let mut current_table = self.base_table.clone(); + + for action in self.actions.clone() { + let mut action_commit = action.commit(¤t_table).await?; + // apply changes to current_table + self.apply( + &mut current_table, + action_commit.take_updates(), + action_commit.take_requirements(), + &mut existing_updates, + &mut existing_requirements, + )?; } let table_commit = TableCommit::builder() From c2dde2358b2ca2d58c36f26a8e3b6a42ae029d88 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 9 Jun 2025 15:59:17 -0700 Subject: [PATCH 24/25] Fix error-handling in retry --- crates/iceberg/src/transaction/mod.rs | 25 ++++--------------------- 1 file changed, 4 insertions(+), 21 deletions(-) diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index e3d58c840..a7f160422 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -186,27 +186,13 @@ impl Transaction { ) .context(tx) .sleep(tokio::time::sleep) - .when(|e| { - if let Some(err) = e.downcast_ref::() { - err.kind() == ErrorKind::DataInvalid // TODO add retryable error kind - } else { - false - } - }) + // todo use a specific commit failure + .when(|e| e.kind() == ErrorKind::DataInvalid) .await .1 - .map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Failed to commit transaction! caused by: {e}"), - ) - }) } - async fn do_commit( - &mut self, - catalog: Arc<&dyn Catalog>, - ) -> std::result::Result { + async fn do_commit(&mut self, catalog: Arc<&dyn Catalog>) -> Result
{ let base_table_identifier = self.base_table.identifier().to_owned(); let refreshed = catalog @@ -244,10 +230,7 @@ impl Transaction { .requirements(existing_requirements) .build(); - catalog - .update_table(table_commit) - .await - .map_err(anyhow::Error::from) + catalog.update_table(table_commit).await } } From 3a8e8f733d71c6a9e16d38b293c42fd788b08413 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 11 Jun 2025 17:48:55 -0700 Subject: [PATCH 25/25] make tx.apply immutable --- crates/iceberg/src/transaction/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/transaction/mod.rs b/crates/iceberg/src/transaction/mod.rs index a7f160422..6a666d51a 100644 --- a/crates/iceberg/src/transaction/mod.rs +++ b/crates/iceberg/src/transaction/mod.rs @@ -57,7 +57,7 @@ impl Transaction { } } - fn update_table_metadata(&mut self, table: &mut Table, updates: &[TableUpdate]) -> Result<()> { + fn update_table_metadata(&self, table: &mut Table, updates: &[TableUpdate]) -> Result<()> { let mut metadata_builder = table.metadata().clone().into_builder(None); for update in updates { metadata_builder = update.clone().apply(metadata_builder)?; @@ -70,7 +70,7 @@ impl Transaction { /// TODO documentation pub fn apply( - &mut self, + &self, table: &mut Table, updates: Vec, requirements: Vec,