From aedecde3420c590b4f30d7a21e371b65d8d1039e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 28 Jun 2022 00:19:49 +0200 Subject: [PATCH 01/67] feat: initial conflict checker methods --- rust/src/conflict_checker.rs | 431 +++++++++++++++++++++++++++++++++++ rust/src/delta.rs | 14 +- rust/src/lib.rs | 33 +-- 3 files changed, 455 insertions(+), 23 deletions(-) create mode 100644 rust/src/conflict_checker.rs diff --git a/rust/src/conflict_checker.rs b/rust/src/conflict_checker.rs new file mode 100644 index 0000000000..90f75c6ff4 --- /dev/null +++ b/rust/src/conflict_checker.rs @@ -0,0 +1,431 @@ +//! Helper module to check if a transaction can be committed in case of conflicting commits. +use crate::action::{Action, Add, MetaData, Protocol, Remove}; +use crate::{ + table_state::DeltaTableState, DeltaDataTypeTimestamp, DeltaDataTypeVersion, DeltaTable, + DeltaTableError, DeltaTableMetaData, +}; +use serde::{Deserialize, Serialize}; +use serde_json::{Map, Value}; +use std::collections::{HashMap, HashSet}; + +/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. +/// However the reference implementation as well as delta-rs store useful information that may for instance +/// allow us to be more permissive in commit conflict resolution. +#[derive(Serialize, Deserialize, Debug, Clone)] +#[serde(rename_all = "camelCase")] +pub struct CommitInfo { + pub version: Option, + pub timestamp: DeltaDataTypeTimestamp, + pub user_id: Option, + pub user_name: Option, + pub operation: String, + pub operation_parameters: HashMap, + pub read_version: Option, + pub isolation_level: Option, + pub is_blind_append: Option, +} + +/// Exceptions raised during commit conflict resolution +#[derive(thiserror::Error, Debug)] +pub enum CommitConflictError { + /// This exception occurs when a concurrent operation adds files in the same partition + /// (or anywhere in an unpartitioned table) that your operation reads. The file additions + /// can be caused by INSERT, DELETE, UPDATE, or MERGE operations. + #[error("Concurrent append failed.")] + ConcurrentAppend, + + /// This exception occurs when a concurrent operation deleted a file that your operation read. + /// Common causes are a DELETE, UPDATE, or MERGE operation that rewrites files. + #[error("Concurrent delete-read failed.")] + ConcurrentDeleteRead, + + /// This exception occurs when a concurrent operation deleted a file that your operation also deletes. + /// This could be caused by two concurrent compaction operations rewriting the same files. + #[error("Concurrent delete-delete failed.")] + ConcurrentDeleteDelete, + + /// This exception occurs when a concurrent transaction updates the metadata of a Delta table. + /// Common causes are ALTER TABLE operations or writes to your Delta table that update the schema of the table. + #[error("Metadata changed since last commit.")] + MetadataChanged, + + /// If a streaming query using the same checkpoint location is started multiple times concurrently + /// and tries to write to the Delta table at the same time. You should never have two streaming + /// queries use the same checkpoint location and run at the same time. + #[error("Concurrent transaction failed.")] + ConcurrentTransaction, + + /// This exception can occur in the following cases: + /// - When your Delta table is upgraded to a new version. For future operations to succeed + /// you may need to upgrade your Delta Lake version. + /// - When multiple writers are creating or replacing a table at the same time. + /// - When multiple writers are writing to an empty path at the same time. + #[error("Protocol changed since last commit.")] + ProtocolChanged, + + /// Error returned when the table requires an unsupported writer version + #[error("Delta-rs does not support writer version {0}")] + UnsupportedWriterVersion(i32), + + /// Error returned when the table requires an unsupported writer version + #[error("Delta-rs does not support reader version {0}")] + UnsupportedReaderVersion(i32), +} + +#[derive(Serialize, Deserialize, Debug, Clone)] +pub enum IsolationLevel { + /// The strongest isolation level. It ensures that committed write operations + /// and all reads are Serializable. Operations are allowed as long as there + /// exists a serial sequence of executing them one-at-a-time that generates + /// the same outcome as that seen in the table. For the write operations, + /// the serial sequence is exactly the same as that seen in the table’s history. + Serializable, + + /// A weaker isolation level than Serializable. It ensures only that the write + /// operations (that is, not reads) are serializable. However, this is still stronger + /// than Snapshot isolation. WriteSerializable is the default isolation level because + /// it provides great balance of data consistency and availability for most common operations. + WriteSerializable, + + SnapshotIsolation, +} + +/// A struct representing different attributes of current transaction needed for conflict detection. +pub(crate) struct CurrentTransactionInfo { + txn_id: String, + /// partition predicates by which files have been queried by the transaction + read_predicates: Vec, + /// files that have been seen by the transaction + read_files: HashSet, + /// whether the whole table was read during the transaction + read_whole_table: bool, + /// appIds that have been seen by the transaction + read_app_ids: HashSet, + /// table metadata for the transaction + metadata: DeltaTableMetaData, + /// delta log actions that the transaction wants to commit + actions: Vec, + /// read [[Snapshot]] used for the transaction + // read_snapshot: Snapshot, + /// [CommitInfo] for the commit + commit_info: Option>, +} + +impl CurrentTransactionInfo { + pub fn new() -> Self { + todo!() + } + + pub fn metadata_changed(&self) -> bool { + todo!() + } + + pub fn final_actions_to_commit(&self) -> Vec { + todo!() + } +} + +/// Summary of the Winning commit against which we want to check the conflict +pub(crate) struct WinningCommitSummary { + pub actions: Vec, + pub commit_info: Option, +} + +impl WinningCommitSummary { + pub fn new(actions: Vec, version: DeltaDataTypeVersion) -> Self { + let commit_info = actions + .iter() + .find(|action| matches!(action, Action::commitInfo(_))) + .map(|action| match action { + Action::commitInfo(info) => { + // TODO remove panic + let mut ci = serde_json::from_value::(serde_json::Value::Object( + info.clone(), + )) + .unwrap(); + ci.version = Some(version); + ci + } + _ => unreachable!(), + }); + Self { + actions, + commit_info, + } + } + + pub fn metadata_updates(&self) -> Vec { + self.actions + .iter() + .cloned() + .filter_map(|action| match action { + Action::metaData(metadata) => Some(metadata), + _ => None, + }) + .collect() + } + + pub fn app_level_transactions(&self) -> HashSet { + self.actions + .iter() + .cloned() + .filter_map(|action| match action { + Action::txn(txn) => Some(txn.app_id), + _ => None, + }) + .collect() + } + + pub fn protocol(&self) -> Vec { + self.actions + .iter() + .cloned() + .filter_map(|action| match action { + Action::protocol(protocol) => Some(protocol), + _ => None, + }) + .collect() + } + + pub fn removed_files(&self) -> Vec { + self.actions + .iter() + .cloned() + .filter_map(|action| match action { + Action::remove(remove) => Some(remove), + _ => None, + }) + .collect() + } + + pub fn added_files(&self) -> Vec { + self.actions + .iter() + .cloned() + .filter_map(|action| match action { + Action::add(add) => Some(add), + _ => None, + }) + .collect() + } + + pub fn blind_append_added_files(&self) -> Vec { + if self.is_blind_append().unwrap_or(false) { + self.added_files() + } else { + vec![] + } + } + + pub fn changed_data_added_files(&self) -> Vec { + if self.is_blind_append().unwrap_or(false) { + vec![] + } else { + self.added_files() + } + } + + pub fn only_add_files(&self) -> bool { + !self + .actions + .iter() + .any(|action| matches!(action, Action::remove(_))) + } + + pub fn is_blind_append(&self) -> Option { + self.commit_info + .as_ref() + .map(|opt| opt.is_blind_append.unwrap_or(false)) + } +} + +pub(crate) struct ConflictChecker<'a> { + delta_table: &'a mut DeltaTable, + initial_current_transaction_info: CurrentTransactionInfo, + winning_commit_version: DeltaDataTypeVersion, + /// Summary of the transaction, that has been committed ahead of the current transaction + winning_commit_summary: WinningCommitSummary, + /// isolation level for the current transaction + isolation_level: IsolationLevel, + /// The state of the delta table at the base version from the current (not winning) commit + state: DeltaTableState, +} + +impl<'a> ConflictChecker<'a> { + pub async fn try_new( + table: &'a mut DeltaTable, + winning_commit_version: DeltaDataTypeVersion, + ) -> Result, DeltaTableError> { + // TODO raise proper error here + assert!(winning_commit_version == table.version() + 1); + let next_state = DeltaTableState::from_commit(table, winning_commit_version).await?; + let mut new_state = table.get_state().clone(); + new_state.merge(next_state, true, true); + todo!() + // Self {} + } + + fn current_transaction_info(&self) -> &CurrentTransactionInfo { + // TODO figure out when we need to update this + &self.initial_current_transaction_info + } + /// This function checks conflict of the `initial_current_transaction_info` against the + /// `winning_commit_version` and returns an updated [CurrentTransactionInfo] that represents + /// the transaction as if it had started while reading the `winning_commit_version`. + pub fn check_conflicts(&self) -> Result { + self.check_protocol_compatibility()?; + self.check_no_metadata_updates()?; + self.check_for_added_files_that_should_have_been_read_by_current_txn()?; + self.check_for_deleted_files_against_current_txn_read_files()?; + self.check_for_deleted_files_against_current_txn_deleted_files()?; + self.check_for_updated_application_transaction_ids_that_current_txn_depends_on()?; + todo!() + } + + /// Asserts that the client is up to date with the protocol and is allowed + /// to read and write against the protocol set by the committed transaction. + fn check_protocol_compatibility(&self) -> Result<(), CommitConflictError> { + for p in self.winning_commit_summary.protocol() { + if self.state.min_reader_version() < p.min_reader_version + || self.state.min_writer_version() < p.min_writer_version + { + return Err(CommitConflictError::ProtocolChanged); + }; + } + if !self.winning_commit_summary.protocol().is_empty() + && self + .current_transaction_info() + .actions + .iter() + .any(|a| matches!(a, Action::protocol(_))) + { + return Err(CommitConflictError::ProtocolChanged); + }; + Ok(()) + } + + /// Check if the committed transaction has changed metadata. + fn check_no_metadata_updates(&self) -> Result<(), CommitConflictError> { + // Fail if the metadata is different than what the txn read. + if !self.winning_commit_summary.metadata_updates().is_empty() { + Err(CommitConflictError::MetadataChanged) + } else { + Ok(()) + } + } + + /// Check if the new files added by the already committed transactions + /// should have been read by the current transaction. + fn check_for_added_files_that_should_have_been_read_by_current_txn( + &self, + ) -> Result<(), CommitConflictError> { + // Fail if new files have been added that the txn should have read. + let added_files_to_check = match self.isolation_level { + IsolationLevel::WriteSerializable + if !self.current_transaction_info().metadata_changed() => + { + // don't conflict with blind appends + self.winning_commit_summary.changed_data_added_files() + } + IsolationLevel::Serializable | IsolationLevel::WriteSerializable => { + let mut files = self.winning_commit_summary.changed_data_added_files(); + files.extend(self.winning_commit_summary.blind_append_added_files()); + files + } + IsolationLevel::SnapshotIsolation => vec![], + }; + // TODO here we need to check if the current transaction would have read the + // added files. for this we need to be able to evaluate predicates. Err on the safe side is + // to assume all files match + let added_files_matching_predicates = added_files_to_check; + if !added_files_matching_predicates.is_empty() { + Err(CommitConflictError::ConcurrentAppend) + } else { + Ok(()) + } + } + + /// Check if [Remove] actions added by already committed transactions + /// conflicts with files read by the current transaction. + fn check_for_deleted_files_against_current_txn_read_files( + &self, + ) -> Result<(), CommitConflictError> { + // Fail if files have been deleted that the txn read. + let read_file_path: HashSet = self + .current_transaction_info() + .read_files + .iter() + .map(|f| f.path.clone()) + .collect(); + let deleted_read_overlap = self + .winning_commit_summary + .removed_files() + .iter() + .cloned() + .find(|f| read_file_path.contains(&f.path)); + if deleted_read_overlap.is_some() + || (!self.winning_commit_summary.removed_files().is_empty() + && self.current_transaction_info().read_whole_table) + { + Err(CommitConflictError::ConcurrentDeleteRead) + } else { + Ok(()) + } + } + + /// Check if [Remove] actions added by already committed transactions conflicts + /// with [Remove] actions this transaction is trying to add. + fn check_for_deleted_files_against_current_txn_deleted_files( + &self, + ) -> Result<(), CommitConflictError> { + // Fail if a file is deleted twice. + let txn_deleted_files: HashSet = self + .current_transaction_info() + .actions + .iter() + .cloned() + .filter_map(|action| match action { + Action::remove(remove) => Some(remove.path), + _ => None, + }) + .collect(); + let winning_deleted_files: HashSet = self + .winning_commit_summary + .removed_files() + .iter() + .cloned() + .map(|r| r.path) + .collect(); + let intersection: HashSet<&String> = txn_deleted_files + .intersection(&winning_deleted_files) + .collect(); + if !intersection.is_empty() { + Err(CommitConflictError::ConcurrentDeleteDelete) + } else { + Ok(()) + } + } + + /// Checks if the winning transaction corresponds to some AppId on which + /// current transaction also depends. + fn check_for_updated_application_transaction_ids_that_current_txn_depends_on( + &self, + ) -> Result<(), CommitConflictError> { + // Fail if the appIds seen by the current transaction has been updated by the winning + // transaction i.e. the winning transaction have [Txn] corresponding to + // some appId on which current transaction depends on. Example - This can happen when + // multiple instances of the same streaming query are running at the same time. + let winning_txns = self.winning_commit_summary.app_level_transactions(); + let txn_overlap: HashSet<&String> = winning_txns + .intersection(&self.current_transaction_info().read_app_ids) + .collect(); + if !txn_overlap.is_empty() { + Err(CommitConflictError::ConcurrentTransaction) + } else { + Ok(()) + } + } +} + +#[cfg(test)] +mod tests {} diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 3fe3e8f2ba..8c231a8aad 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -28,6 +28,7 @@ use super::schema::*; use super::storage; use super::storage::{StorageBackend, StorageError, UriError}; use super::table_state::DeltaTableState; +use crate::conflict_checker::{CommitConflictError, ConflictChecker}; use crate::delta_config::DeltaConfigError; const MILLIS_IN_HOUR: i64 = 3600000; @@ -198,6 +199,13 @@ pub enum DeltaTableError { /// Error returned when user attempts to commit actions that don't belong to the next version. #[error("Delta transaction failed, version {0} does not follow {1}")] VersionMismatch(DeltaDataTypeVersion, DeltaDataTypeVersion), + /// Error returned for non resolvable commit conflicts + #[error("Delta transaction failed: {source}")] + CommitConflict { + /// underlying error returned from conflict checker + #[from] + source: CommitConflictError, + }, /// Generic Delta Table error #[error("Generic DeltaTable error: {0}")] Generic(String), @@ -816,7 +824,7 @@ impl DeltaTable { Ok(version) } - /// Currently loaded evrsion of the table + /// Currently loaded version of the table pub fn version(&self) -> DeltaDataTypeVersion { self.state.version } @@ -1658,6 +1666,10 @@ impl<'a> DeltaTransaction<'a> { Err(e) => { match e { DeltaTableError::VersionAlreadyExists(_) => { + let checker = + ConflictChecker::try_new(self.delta_table, version).await?; + let result = checker.check_conflicts()?; + // TODO update prepared commit in case transaction info got updated. if attempt_number > self.options.max_retry_commit_attempts + 1 { debug!("Transaction attempt failed. Attempts exhausted beyond max_retry_commit_attempts of {} so failing.", self.options.max_retry_commit_attempts); return Err(e); diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 28d5ced635..a1dccb483c 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -74,25 +74,17 @@ #![deny(warnings)] #![deny(missing_docs)] -extern crate log; - -pub use arrow; -extern crate chrono; -extern crate lazy_static; -extern crate parquet; -extern crate regex; -extern crate serde; -#[cfg(test)] -#[macro_use] -extern crate serde_json; -extern crate thiserror; - pub mod action; pub mod checkpoints; +mod conflict_checker; pub mod data_catalog; mod delta; pub mod delta_arrow; pub mod delta_config; +#[cfg(feature = "rust-dataframe-ext")] +mod delta_dataframe; +#[cfg(feature = "datafusion-ext")] +pub mod delta_datafusion; #[cfg(feature = "datafusion-ext")] pub mod operations; pub mod optimize; @@ -103,20 +95,17 @@ mod table_state; pub mod time_utils; pub mod writer; -#[cfg(feature = "datafusion-ext")] -pub mod delta_datafusion; - -#[cfg(feature = "rust-dataframe-ext")] -mod delta_dataframe; - pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; pub use self::delta::*; pub use self::partitions::*; pub use self::schema::*; +#[cfg(feature = "azure")] +pub use self::storage::azure::azure_storage_options; +#[cfg(feature = "s3")] +pub use self::storage::s3::s3_storage_options; pub use self::storage::{ get_backend_for_uri, get_backend_for_uri_with_options, parse_uri, StorageBackend, StorageError, Uri, UriError, }; - -#[cfg(feature = "s3")] -pub use self::storage::s3::s3_storage_options; +pub use arrow; +pub use parquet; From 3243ee5c8110edac95a962af9f666d11f57c7bed Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 28 Jun 2022 23:49:22 +0200 Subject: [PATCH 02/67] initialize conflict checker --- rust/src/action.rs | 1 + rust/src/conflict_checker.rs | 43 ++++++++++++++++++++++++++---------- rust/src/delta.rs | 22 ++++++++++-------- rust/src/table_state.rs | 2 ++ 4 files changed, 47 insertions(+), 21 deletions(-) diff --git a/rust/src/action.rs b/rust/src/action.rs index 3562ef3c05..fd0176f1d2 100644 --- a/rust/src/action.rs +++ b/rust/src/action.rs @@ -874,6 +874,7 @@ pub enum DeltaOperation { /// The predicate used during the write. predicate: Option, }, + /// Represents a Delta `StreamingUpdate` operation. #[serde(rename_all = "camelCase")] StreamingUpdate { diff --git a/rust/src/conflict_checker.rs b/rust/src/conflict_checker.rs index 90f75c6ff4..a4722edd09 100644 --- a/rust/src/conflict_checker.rs +++ b/rust/src/conflict_checker.rs @@ -1,5 +1,5 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. -use crate::action::{Action, Add, MetaData, Protocol, Remove}; +use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::{ table_state::DeltaTableState, DeltaDataTypeTimestamp, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, @@ -7,6 +7,7 @@ use crate::{ use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; use std::collections::{HashMap, HashSet}; +use std::io::{BufRead, BufReader, Cursor}; /// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. /// However the reference implementation as well as delta-rs store useful information that may for instance @@ -112,7 +113,7 @@ pub(crate) struct CurrentTransactionInfo { } impl CurrentTransactionInfo { - pub fn new() -> Self { + pub fn try_new(table: &DeltaTable, operation: DeltaOperation) -> Result { todo!() } @@ -239,8 +240,8 @@ impl WinningCommitSummary { } } -pub(crate) struct ConflictChecker<'a> { - delta_table: &'a mut DeltaTable, +pub(crate) struct ConflictChecker { + /// transaction information for current transaction at start of check initial_current_transaction_info: CurrentTransactionInfo, winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction @@ -251,18 +252,36 @@ pub(crate) struct ConflictChecker<'a> { state: DeltaTableState, } -impl<'a> ConflictChecker<'a> { +impl ConflictChecker { pub async fn try_new( - table: &'a mut DeltaTable, + table: &DeltaTable, winning_commit_version: DeltaDataTypeVersion, - ) -> Result, DeltaTableError> { + operation: DeltaOperation, + ) -> Result { // TODO raise proper error here assert!(winning_commit_version == table.version() + 1); - let next_state = DeltaTableState::from_commit(table, winning_commit_version).await?; - let mut new_state = table.get_state().clone(); - new_state.merge(next_state, true, true); - todo!() - // Self {} + + // create winning commit summary + let commit_uri = table.commit_uri_from_version(winning_commit_version); + let commit_log_bytes = table.storage.get_obj(&commit_uri).await?; + let reader = BufReader::new(Cursor::new(commit_log_bytes)); + let mut commit_actions = Vec::new(); + for line in reader.lines() { + commit_actions.push(serde_json::from_str::(line?.as_str())?); + } + let winning_commit_summary = + WinningCommitSummary::new(commit_actions, winning_commit_version); + + let initial_current_transaction_info = CurrentTransactionInfo::try_new(table, operation)?; + + Ok(Self { + initial_current_transaction_info, + winning_commit_summary, + winning_commit_version, + isolation_level: IsolationLevel::Serializable, + // TODO cloning the state is probably a bad idea, since it can be very large... + state: table.state.clone(), + }) } fn current_transaction_info(&self) -> &CurrentTransactionInfo { diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 8c231a8aad..a8aae5717c 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -1566,10 +1566,6 @@ impl<'a> DeltaTransaction<'a> { operation: Option, app_metadata: Option>, ) -> Result { - // TODO: stubbing `operation` parameter (which will be necessary for writing the CommitInfo action), - // but leaving it unused for now. `CommitInfo` is a fairly dynamic data structure so we should work - // out the data structure approach separately. - // TODO: calculate isolation level to use when checking for conflicts. // Leaving conflict checking unimplemented for now to get the "single writer" implementation off the ground. // Leaving some commmented code in place as a guidepost for the future. @@ -1585,10 +1581,13 @@ impl<'a> DeltaTransaction<'a> { // IsolationLevel::Serializable // }; - let prepared_commit = self.prepare_commit(operation, app_metadata).await?; + // TODO make operation a required parameter + let op = operation.unwrap(); + + let prepared_commit = self.prepare_commit(Some(op.clone()), app_metadata).await?; // try to commit in a loop in case other writers write the next version first - let version = self.try_commit_loop(&prepared_commit).await?; + let version = self.try_commit_loop(&prepared_commit, op).await?; Ok(version) } @@ -1648,6 +1647,7 @@ impl<'a> DeltaTransaction<'a> { async fn try_commit_loop( &mut self, commit: &PreparedCommit, + operation: DeltaOperation, ) -> Result { let mut attempt_number: u32 = 0; loop { @@ -1666,9 +1666,13 @@ impl<'a> DeltaTransaction<'a> { Err(e) => { match e { DeltaTableError::VersionAlreadyExists(_) => { - let checker = - ConflictChecker::try_new(self.delta_table, version).await?; - let result = checker.check_conflicts()?; + let checker = ConflictChecker::try_new( + self.delta_table, + version, + operation.clone(), + ) + .await?; + let _result = checker.check_conflicts()?; // TODO update prepared commit in case transaction info got updated. if attempt_number > self.options.max_retry_commit_attempts + 1 { debug!("Transaction attempt failed. Attempts exhausted beyond max_retry_commit_attempts of {} so failing.", self.options.max_retry_commit_attempts); diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 93f378d526..505f6c9b1c 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -37,12 +37,14 @@ pub struct DeltaTableState { } impl DeltaTableState { + /// Create a new table state instance with default values and version specified pub fn with_version(version: DeltaDataTypeVersion) -> Self { Self { version, ..Self::default() } } + /// Construct a delta table state object from commit version. pub async fn from_commit( table: &DeltaTable, From bf5522686a9fbb4eca9ff3567b2fb613422e91d9 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 30 Jun 2022 00:02:26 +0200 Subject: [PATCH 03/67] feat: collect more transaction infos --- rust/src/action.rs | 55 ++++++++++++++----- rust/src/conflict_checker.rs | 8 +++ rust/src/delta.rs | 102 +++++++++++++++++++++++++++++------ 3 files changed, 135 insertions(+), 30 deletions(-) diff --git a/rust/src/action.rs b/rust/src/action.rs index fd0176f1d2..080a7d4526 100644 --- a/rust/src/action.rs +++ b/rust/src/action.rs @@ -894,33 +894,60 @@ pub enum DeltaOperation { predicate: Option, /// Target optimize size target_size: DeltaDataTypeLong, - }, // TODO: Add more operations + }, + // TODO: Add more operations } impl DeltaOperation { - /// Retrieve basic commit information to be added to Delta commits - pub fn get_commit_info(&self) -> Map { - let mut commit_info = Map::::new(); - let operation = match &self { + /// Name of the operation + pub fn name(&self) -> String { + match &self { DeltaOperation::Create { .. } => "delta-rs.Create", DeltaOperation::Write { .. } => "delta-rs.Write", DeltaOperation::StreamingUpdate { .. } => "delta-rs.StreamingUpdate", DeltaOperation::Optimize { .. } => "delta-rs.Optimize", - }; + } + .into() + } + + pub fn operation_parameters(&self) -> Option> { + if let Ok(serde_json::Value::Object(map)) = serde_json::to_value(self) { + Some(map.values().next().unwrap().clone()) + } else { + None + } + } + + /// Retrieve basic commit information to be added to Delta commits + pub fn get_commit_info(&self) -> Map { + let mut commit_info = Map::::new(); commit_info.insert( "operation".to_string(), - serde_json::Value::String(operation.into()), + serde_json::Value::String(self.name()), ); - - if let Ok(serde_json::Value::Object(map)) = serde_json::to_value(self) { - commit_info.insert( - "operationParameters".to_string(), - map.values().next().unwrap().clone(), - ); + if let Some(params) = self.operation_parameters() { + commit_info.insert("operationParameters".to_string(), params); }; - commit_info } + + /// Denotes whether the operation will change the data contained in the table + pub fn changes_data(&self) -> bool { + // TODO update accordingly when new operations are added + // Operations that change data: + // - Write + // - StreamingUpdate + // - Delete + // - Truncate + // - Convert + // - Merge + // - Update + // - ReplaceTable + !matches!( + self, + DeltaOperation::Create { .. } | DeltaOperation::Optimize { .. } + ) + } } /// The SaveMode used when performing a DeltaOperation diff --git a/rust/src/conflict_checker.rs b/rust/src/conflict_checker.rs index a4722edd09..70b8fe70ee 100644 --- a/rust/src/conflict_checker.rs +++ b/rust/src/conflict_checker.rs @@ -91,6 +91,14 @@ pub enum IsolationLevel { SnapshotIsolation, } +impl IsolationLevel { + /// The default isolation level to use, analogous to reference implementation + pub fn default_level() -> Self { + // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 + IsolationLevel::Serializable + } +} + /// A struct representing different attributes of current transaction needed for conflict detection. pub(crate) struct CurrentTransactionInfo { txn_id: String, diff --git a/rust/src/delta.rs b/rust/src/delta.rs index a8aae5717c..0cd31f0727 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -28,7 +28,7 @@ use super::schema::*; use super::storage; use super::storage::{StorageBackend, StorageError, UriError}; use super::table_state::DeltaTableState; -use crate::conflict_checker::{CommitConflictError, ConflictChecker}; +use crate::conflict_checker::{CommitConflictError, CommitInfo, ConflictChecker, IsolationLevel}; use crate::delta_config::DeltaConfigError; const MILLIS_IN_HOUR: i64 = 3600000; @@ -1533,6 +1533,7 @@ pub struct DeltaTransaction<'a> { delta_table: &'a mut DeltaTable, actions: Vec, options: DeltaTransactionOptions, + txn_id: String, } impl<'a> DeltaTransaction<'a> { @@ -1544,6 +1545,7 @@ impl<'a> DeltaTransaction<'a> { delta_table, actions: vec![], options: options.unwrap_or_default(), + txn_id: Uuid::new_v4().into(), } } @@ -1566,24 +1568,48 @@ impl<'a> DeltaTransaction<'a> { operation: Option, app_metadata: Option>, ) -> Result { - // TODO: calculate isolation level to use when checking for conflicts. - // Leaving conflict checking unimplemented for now to get the "single writer" implementation off the ground. - // Leaving some commmented code in place as a guidepost for the future. - - // let no_data_changed = actions.iter().all(|a| match a { - // Action::add(x) => !x.dataChange, - // Action::remove(x) => !x.dataChange, - // _ => false, - // }); - // let isolation_level = if no_data_changed { - // IsolationLevel::SnapshotIsolation - // } else { - // IsolationLevel::Serializable - // }; - // TODO make operation a required parameter let op = operation.unwrap(); + // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, + // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... + // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 + let isolation_level = if self.can_downgrade_to_snapshot_isolation(&op) { + IsolationLevel::SnapshotIsolation + } else { + IsolationLevel::default_level() + }; + + let only_add_files = self + .actions + .iter() + .any(|action| !matches!(action, Action::add(_))); + + // readPredicates.nonEmpty || readFiles.nonEmpty + // TODO revise logic if files are read + let depends_on_files = match op { + DeltaOperation::Create { .. } | DeltaOperation::StreamingUpdate { .. } => false, + DeltaOperation::Optimize { .. } => true, + DeltaOperation::Write { + predicate: Some(_), .. + } => true, + _ => false, + }; + + let is_blind_append = only_add_files && !depends_on_files; + + let commit_info = CommitInfo { + version: None, + timestamp: chrono::Utc::now().timestamp(), + read_version: Some(self.delta_table.version()), + isolation_level: Some(isolation_level), + operation, + operation_parameters: op.operation_parameters(), + user_id: None, + user_name: None, + is_blind_append: Some(is_blind_append), + }; + let prepared_commit = self.prepare_commit(Some(op.clone()), app_metadata).await?; // try to commit in a loop in case other writers write the next version first @@ -1691,6 +1717,50 @@ impl<'a> DeltaTransaction<'a> { } } } + + fn can_downgrade_to_snapshot_isolation(&self, operation: &DeltaOperation) -> bool { + let mut data_changed = false; + let mut has_non_file_actions = false; + for action in &self.actions { + match action { + Action::add(act) if act.data_change => data_changed = true, + Action::remove(rem) if rem.data_change => data_changed = true, + _ => has_non_file_actions = true, + } + } + + if has_non_file_actions { + // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation + // level to SnapshotIsolation. + return false; + } + + let default_isolation_level = IsolationLevel::default_level(); + // Note-1: For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can + // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict + // detection by skipping the + // [[ConflictChecker.checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn]] check i.e. + // don't worry about concurrent appends. + // Note-2: + // We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: + // t0 - Initial state of table + // t1 - Q1, Q2 starts + // t2 - Q1 commits + // t3 - Q2 is empty and wants to commit. + // In this scenario, we can always allow Q2 to commit without worrying about new files + // generated by Q1. + // The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 + // Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has + // a Metadata update (say schema change/identity column high watermark update), then Q2 can't + // be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, + // then Q1 should see the updates from Q2 - which actually didn't happen. + + match default_isolation_level { + IsolationLevel::Serializable => !data_changed, + IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), + _ => false, // This case should never happen + } + } } /// Holds the uri to prepared commit temporary file created with `DeltaTransaction.prepare_commit`. From 595e91b17481ece5b5736c86a590fb31c9491eb4 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 4 Jul 2022 22:58:27 +0200 Subject: [PATCH 04/67] docs: add some comments --- rust/src/action.rs | 1 + rust/src/conflict_checker.rs | 21 +++++++++++---------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/rust/src/action.rs b/rust/src/action.rs index 080a7d4526..f5914033f4 100644 --- a/rust/src/action.rs +++ b/rust/src/action.rs @@ -912,6 +912,7 @@ impl DeltaOperation { pub fn operation_parameters(&self) -> Option> { if let Ok(serde_json::Value::Object(map)) = serde_json::to_value(self) { + // serializing self will always yield something like {"": {...}} so unwrapping is safe Some(map.values().next().unwrap().clone()) } else { None diff --git a/rust/src/conflict_checker.rs b/rust/src/conflict_checker.rs index 70b8fe70ee..f9072ced66 100644 --- a/rust/src/conflict_checker.rs +++ b/rust/src/conflict_checker.rs @@ -101,23 +101,23 @@ impl IsolationLevel { /// A struct representing different attributes of current transaction needed for conflict detection. pub(crate) struct CurrentTransactionInfo { - txn_id: String, + pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction - read_predicates: Vec, + pub(crate) read_predicates: Vec, /// files that have been seen by the transaction - read_files: HashSet, + pub(crate) read_files: HashSet, /// whether the whole table was read during the transaction - read_whole_table: bool, + pub(crate) read_whole_table: bool, /// appIds that have been seen by the transaction - read_app_ids: HashSet, + pub(crate) read_app_ids: HashSet, /// table metadata for the transaction - metadata: DeltaTableMetaData, + pub(crate) metadata: DeltaTableMetaData, /// delta log actions that the transaction wants to commit - actions: Vec, - /// read [[Snapshot]] used for the transaction - // read_snapshot: Snapshot, + pub(crate) actions: Vec, + /// read [DeltaTableState] used for the transaction + pub(crate) read_snapshot: DeltaTableState, /// [CommitInfo] for the commit - commit_info: Option>, + pub(crate) commit_info: Option>, } impl CurrentTransactionInfo { @@ -296,6 +296,7 @@ impl ConflictChecker { // TODO figure out when we need to update this &self.initial_current_transaction_info } + /// This function checks conflict of the `initial_current_transaction_info` against the /// `winning_commit_version` and returns an updated [CurrentTransactionInfo] that represents /// the transaction as if it had started while reading the `winning_commit_version`. From fcc7c901d33151d79c6a04ff71786c2b3a7baac5 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 10 Jul 2022 22:40:02 +0200 Subject: [PATCH 05/67] chore: small fixes --- rust/src/action.rs | 5 ++++- rust/src/delta.rs | 17 ++++++----------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/rust/src/action.rs b/rust/src/action.rs index 037468cbdf..d2a5adebdf 100644 --- a/rust/src/action.rs +++ b/rust/src/action.rs @@ -995,7 +995,10 @@ impl DeltaOperation { serde_json::Value::String(self.name()), ); if let Some(params) = self.operation_parameters() { - commit_info.insert("operationParameters".to_string(), params); + commit_info.insert( + "operationParameters".to_string(), + serde_json::Value::Object(params), + ); }; commit_info } diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 5862925c92..95efce1f5f 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -1452,7 +1452,7 @@ impl<'a> DeltaTransaction<'a> { delta_table, actions: vec![], options: options.unwrap_or_default(), - txn_id: Uuid::new_v4().into(), + txn_id: Uuid::new_v4().to_string(), } } @@ -1472,12 +1472,9 @@ impl<'a> DeltaTransaction<'a> { /// This method will retry the transaction commit based on the value of `max_retry_commit_attempts` set in `DeltaTransactionOptions`. pub async fn commit( &mut self, - operation: Option, + operation: DeltaOperation, app_metadata: Option>, ) -> Result { - // TODO make operation a required parameter - let op = operation.unwrap(); - // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 @@ -1494,7 +1491,7 @@ impl<'a> DeltaTransaction<'a> { // readPredicates.nonEmpty || readFiles.nonEmpty // TODO revise logic if files are read - let depends_on_files = match op { + let depends_on_files = match operation { DeltaOperation::Create { .. } | DeltaOperation::StreamingUpdate { .. } => false, DeltaOperation::Optimize { .. } => true, DeltaOperation::Write { @@ -1510,7 +1507,7 @@ impl<'a> DeltaTransaction<'a> { timestamp: chrono::Utc::now().timestamp(), read_version: Some(self.delta_table.version()), isolation_level: Some(isolation_level), - operation, + operation: operation.name(), operation_parameters: op.operation_parameters(), user_id: None, user_name: None, @@ -1530,7 +1527,7 @@ impl<'a> DeltaTransaction<'a> { /// with `DeltaTable.try_commit_transaction`. pub async fn prepare_commit( &mut self, - operation: Option, + operation: &DeltaOperation, app_metadata: Option>, ) -> Result { if !self @@ -1547,10 +1544,8 @@ impl<'a> DeltaTransaction<'a> { "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); + commit_info.append(&mut operation.get_commit_info()); - if let Some(op) = &operation { - commit_info.append(&mut op.get_commit_info()) - } if let Some(mut meta) = app_metadata { commit_info.append(&mut meta) } From d3f10e6cbe0b2c8a01ef659513930e46519c1735 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 05:08:10 +0100 Subject: [PATCH 06/67] chore: clippy fix --- rust/examples/recordbatch-writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/examples/recordbatch-writer.rs b/rust/examples/recordbatch-writer.rs index c3fcb905a4..4943daa8e4 100644 --- a/rust/examples/recordbatch-writer.rs +++ b/rust/examples/recordbatch-writer.rs @@ -125,7 +125,7 @@ fn fetch_readings() -> Vec { for i in 1..=5 { let mut wx = WeatherRecord::default(); - wx.temp = wx.temp - i; + wx.temp -= i; readings.push(wx); } readings From 9fea909fb7a42c0a48ea288a4941a1bd8a18447d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 05:37:03 +0100 Subject: [PATCH 07/67] chore: bump datqafusion and arrow --- python/Cargo.toml | 4 ++-- rust/Cargo.toml | 17 +++++++---------- rust/src/lib.rs | 2 -- rust/src/operations/mod.rs | 8 ++++++++ rust/src/{ => operations}/optimize.rs | 4 +++- rust/src/writer/json.rs | 22 +++++++++++++--------- rust/tests/command_optimize.rs | 10 +++++----- 7 files changed, 38 insertions(+), 29 deletions(-) rename rust/src/{ => operations}/optimize.rs (98%) diff --git a/python/Cargo.toml b/python/Cargo.toml index af0ca53c62..fc0db7970c 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -18,7 +18,7 @@ doc = false name = "deltalake._internal" [dependencies] -arrow-schema = { version = "31", features = ["serde"] } +arrow-schema = { version = "32", features = ["serde"] } chrono = "0" env_logger = "0" futures = "0.3" @@ -33,7 +33,7 @@ tokio = { version = "1", features = ["rt-multi-thread"] } reqwest = { version = "*", features = ["native-tls-vendored"] } [dependencies.pyo3] -version = "0.17" +version = "0.18" features = ["extension-module", "abi3", "abi3-py37"] [dependencies.deltalake] diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 377702d874..3ca01deb06 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -13,7 +13,7 @@ readme = "README.md" edition = "2021" [dependencies] -arrow = { version = "31", optional = true } +arrow = { version = "32", optional = true } async-trait = "0.1" bytes = "1" chrono = { version = "0.4.22", default-features = false, features = ["clock"] } @@ -29,7 +29,7 @@ num-traits = "0.2.15" object_store = "0.5.3" once_cell = "1.16.0" parking_lot = "0.12" -parquet = { version = "31", features = ["async"], optional = true } +parquet = { version = "32", features = ["async"], optional = true } parquet2 = { version = "0.17", optional = true } percent-encoding = "2" serde = { version = "1", features = ["derive"] } @@ -50,10 +50,10 @@ rusoto_dynamodb = { version = "0.47", default-features = false, optional = true rusoto_glue = { version = "0.47", default-features = false, optional = true } # Datafusion -datafusion = { version = "17", optional = true } -datafusion-expr = { version = "17", optional = true } -datafusion-common = { version = "17", optional = true } -datafusion-proto = { version = "17", optional = true } +datafusion = { version = "18", optional = true } +datafusion-expr = { version = "18", optional = true } +datafusion-common = { version = "18", optional = true } +datafusion-proto = { version = "18", optional = true } # NOTE dependencies only for integration tests fs_extra = { version = "1.2.0", optional = true } @@ -113,10 +113,7 @@ s3 = [ "object_store/aws_profile", ] glue-native-tls = ["s3-native-tls", "rusoto_glue"] -glue = [ - "s3", - "rusoto_glue/rustls" -] +glue = ["s3", "rusoto_glue/rustls"] python = ["arrow/pyarrow"] # used only for integration testing diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 63211dec71..27ef2841b7 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -105,8 +105,6 @@ pub mod delta_arrow; #[cfg(feature = "datafusion")] pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod optimize; -#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; pub use self::builder::*; diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 212bf34414..66e75cf92a 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -15,6 +15,8 @@ use crate::{DeltaResult, DeltaTable, DeltaTableError}; pub mod create; pub mod filesystem_check; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod optimize; pub mod transaction; pub mod vacuum; @@ -123,6 +125,12 @@ impl DeltaOps { pub fn filesystem_check(self) -> FileSystemCheckBuilder { FileSystemCheckBuilder::new(self.0.object_store(), self.0.state) } + + /// Audit active files with files present on the filesystem + #[must_use] + pub fn optimize(self) -> FileSystemCheckBuilder { + FileSystemCheckBuilder::new(self.0.object_store(), self.0.state) + } } impl From for DeltaOps { diff --git a/rust/src/optimize.rs b/rust/src/operations/optimize.rs similarity index 98% rename from rust/src/optimize.rs rename to rust/src/operations/optimize.rs index 5be72cdd93..36ca3f117d 100644 --- a/rust/src/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -22,6 +22,7 @@ use crate::action::DeltaOperation; use crate::action::{self, Action}; +use crate::table_properties; use crate::writer::utils::PartitionPath; use crate::writer::{DeltaWriter, RecordBatchWriter}; use crate::{DeltaDataTypeLong, DeltaTable, DeltaTableError, PartitionFilter}; @@ -175,6 +176,7 @@ fn create_remove( partitions: &HashMap>, size: DeltaDataTypeLong, ) -> Result { + // NOTE unwrap is safe since UNIX_EPOCH will always be earlier then now. let deletion_time = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); let deletion_time = deletion_time.as_millis() as i64; @@ -301,7 +303,7 @@ fn get_target_file_size(table: &DeltaTable) -> DeltaDataTypeLong { let config = table.get_configurations(); let mut target_size = 268_435_456; if let Ok(config) = config { - let config_str = config.get("delta.targetFileSize"); + let config_str = config.get(table_properties::TARGET_FILE_SIZE); if let Some(s) = config_str { if let Some(s) = s { let r = s.parse::(); diff --git a/rust/src/writer/json.rs b/rust/src/writer/json.rs index c9fee9db77..5b0add1de5 100644 --- a/rust/src/writer/json.rs +++ b/rust/src/writer/json.rs @@ -339,16 +339,20 @@ impl DeltaWriter> for JsonWriter { } if !partial_writes.is_empty() { - let sample = partial_writes.first().map(|t| t.to_owned()); - if let Some((_, e)) = sample { - return Err(DeltaWriterError::PartialParquetWrite { - skipped_values: partial_writes, - sample_error: e, - } - .into()); - } else { - unreachable!() + return Err(DeltaWriterError::PartialParquetWrite { + sample_error: match &partial_writes[0].1 { + ParquetError::General(msg) => ParquetError::General(msg.to_owned()), + ParquetError::ArrowError(msg) => ParquetError::ArrowError(msg.to_owned()), + ParquetError::EOF(msg) => ParquetError::EOF(msg.to_owned()), + ParquetError::External(err) => ParquetError::General(err.to_string()), + ParquetError::IndexOutOfBound(u, v) => { + ParquetError::IndexOutOfBound(u.to_owned(), v.to_owned()) + } + ParquetError::NYI(msg) => ParquetError::NYI(msg.to_owned()), + }, + skipped_values: partial_writes, } + .into()); } Ok(()) diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 7f656fa513..ed07a31c7e 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -6,17 +6,17 @@ use arrow::{ datatypes::{DataType, Field}, record_batch::RecordBatch, }; -use deltalake::optimize::{MetricDetails, Metrics}; -use deltalake::DeltaTableError; +use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics, Optimize}; use deltalake::{ action, action::Remove, builder::DeltaTableBuilder, - optimize::{create_merge_plan, Optimize}, writer::{DeltaWriter, RecordBatchWriter}, - DeltaTableMetaData, PartitionFilter, }; -use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; +use deltalake::{ + DeltaTable, DeltaTableError, DeltaTableMetaData, PartitionFilter, Schema, SchemaDataType, + SchemaField, +}; use rand::prelude::*; use serde_json::{json, Map, Value}; use std::time::SystemTime; From 40611aabc7c9b60ef7b35e4f7e844886b71a683f Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 06:01:44 +0100 Subject: [PATCH 08/67] chore!: update pyo3 function signatures --- python/deltalake/table.py | 2 +- python/src/filesystem.rs | 20 +++++++++++--------- python/src/lib.rs | 7 +++++-- python/src/schema.rs | 6 +++--- 4 files changed, 20 insertions(+), 15 deletions(-) diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 54928568a9..422421a70b 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -366,7 +366,7 @@ def to_pyarrow_dataset( partition_expression=part_expression, ) for file, part_expression in self._table.dataset_partitions( - partitions, self.schema().to_pyarrow() + self.schema().to_pyarrow(), partitions ) ] diff --git a/python/src/filesystem.rs b/python/src/filesystem.rs index 587c004f47..41d8b4ec49 100644 --- a/python/src/filesystem.rs +++ b/python/src/filesystem.rs @@ -30,7 +30,7 @@ pub struct DeltaFileSystemHandler { #[pymethods] impl DeltaFileSystemHandler { #[new] - #[args(options = "None")] + #[pyo3(signature = (table_uri, options = None))] fn new(table_uri: &str, options: Option>) -> PyResult { let storage = DeltaTableBuilder::from_uri(table_uri) .with_storage_options(options.clone().unwrap_or_default()) @@ -144,7 +144,7 @@ impl DeltaFileSystemHandler { Ok(infos) } - #[args(allow_not_found = "false", recursive = "false")] + #[pyo3(signature = (base_dir, allow_not_found = false, recursive = false))] fn get_file_info_selector<'py>( &self, base_dir: String, @@ -237,7 +237,7 @@ impl DeltaFileSystemHandler { Ok(file) } - #[args(metadata = "None")] + #[pyo3(signature = (path, metadata = None))] fn open_output_stream( &self, path: String, @@ -358,7 +358,7 @@ impl ObjectInputFile { Ok(self.content_length) } - #[args(whence = "0")] + #[pyo3(signature = (offset, whence = 0))] fn seek(&mut self, offset: i64, whence: i64) -> PyResult { self.check_closed()?; self.check_position(offset, "seek")?; @@ -384,7 +384,7 @@ impl ObjectInputFile { Ok(self.pos) } - #[args(nbytes = "None")] + #[pyo3(signature = (nbytes = None))] fn read(&mut self, nbytes: Option) -> PyResult> { self.check_closed()?; let range = match nbytes { @@ -516,14 +516,16 @@ impl ObjectOutputStream { Err(PyNotImplementedError::new_err("'size' not implemented")) } - #[args(whence = "0")] - fn seek(&mut self, _offset: i64, _whence: i64) -> PyResult { + #[allow(unused_variables)] + #[pyo3(signature = (offset, whence = 0))] + fn seek(&mut self, offset: i64, whence: i64) -> PyResult { self.check_closed()?; Err(PyNotImplementedError::new_err("'seek' not implemented")) } - #[args(nbytes = "None")] - fn read(&mut self, _nbytes: Option) -> PyResult<()> { + #[allow(unused_variables)] + #[pyo3(signature = (nbytes = None))] + fn read(&mut self, nbytes: Option) -> PyResult<()> { self.check_closed()?; Err(PyNotImplementedError::new_err("'read' not implemented")) } diff --git a/python/src/lib.rs b/python/src/lib.rs index 919672eab6..0e5659555d 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -108,6 +108,7 @@ struct RawDeltaTableMetaData { #[pymethods] impl RawDeltaTable { #[new] + #[pyo3(signature = (table_uri, version = None, storage_options = None, without_files = false))] fn new( table_uri: &str, version: Option, @@ -282,7 +283,9 @@ impl RawDeltaTable { schema_to_pyobject(schema, py) } - /// Run the Vacuum command on the Delta Table: list and delete files no longer referenced by the Delta table and are older than the retention threshold. + /// Run the Vacuum command on the Delta Table: list and delete files no longer referenced + /// by the Delta table and are older than the retention threshold. + #[pyo3(signature = (dry_run, retention_hours = None, enforce_retention_duration = true))] pub fn vacuum( &mut self, dry_run: bool, @@ -334,8 +337,8 @@ impl RawDeltaTable { pub fn dataset_partitions<'py>( &mut self, py: Python<'py>, - partition_filters: Option>, schema: PyArrowType, + partition_filters: Option>, ) -> PyResult)>> { let path_set = match partition_filters { Some(filters) => Some(HashSet::<_>::from_iter( diff --git a/python/src/schema.rs b/python/src/schema.rs index 97aefdd541..db5df88746 100644 --- a/python/src/schema.rs +++ b/python/src/schema.rs @@ -279,7 +279,7 @@ impl TryFrom for ArrayType { #[pymethods] impl ArrayType { #[new] - #[args(contains_null = true)] + #[pyo3(signature = (element_type, contains_null = true))] fn new(element_type: PyObject, contains_null: bool, py: Python) -> PyResult { let inner_type = SchemaTypeArray::new( Box::new(python_type_to_schema(element_type, py)?), @@ -444,7 +444,7 @@ impl TryFrom for MapType { #[pymethods] impl MapType { #[new] - #[args(value_contains_null = true)] + #[pyo3(signature = (key_type, value_type, value_contains_null = true))] fn new( key_type: PyObject, value_type: PyObject, @@ -608,7 +608,7 @@ pub struct Field { #[pymethods] impl Field { #[new] - #[args(nullable = true)] + #[pyo3(signature = (name, ty, nullable = true, metadata = None))] fn new( name: String, ty: PyObject, From 03ec7bcf45d8867a10dda7aca467597f149b4b02 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 19:28:05 +0100 Subject: [PATCH 09/67] feat: move and update optimize command --- rust/Cargo.toml | 5 +- rust/src/operations/mod.rs | 12 +- rust/src/operations/optimize.rs | 221 +++++++++++++++++++++++--------- rust/src/operations/writer.rs | 11 +- rust/src/storage/mod.rs | 1 + rust/src/storage/utils.rs | 38 +++++- rust/tests/command_optimize.rs | 68 ++++++---- 7 files changed, 257 insertions(+), 99 deletions(-) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 3ca01deb06..f0b1fff124 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -29,7 +29,10 @@ num-traits = "0.2.15" object_store = "0.5.3" once_cell = "1.16.0" parking_lot = "0.12" -parquet = { version = "32", features = ["async"], optional = true } +parquet = { version = "32", features = [ + "async", + "object_store", +], optional = true } parquet2 = { version = "0.17", optional = true } percent-encoding = "2" serde = { version = "1", features = ["derive"] } diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 66e75cf92a..f7a222948a 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -26,15 +26,15 @@ use self::{load::LoadBuilder, write::WriteBuilder}; use arrow::record_batch::RecordBatch; #[cfg(feature = "datafusion")] pub use datafusion::physical_plan::common::collect as collect_sendable_stream; +#[cfg(all(feature = "arrow", feature = "parquet"))] +use optimize::OptimizeBuilder; #[cfg(feature = "datafusion")] mod load; #[cfg(feature = "datafusion")] pub mod write; -// TODO the writer module does not actually depend on datafusion, -// eventually we should consolidate with the record batch writer -#[cfg(feature = "datafusion")] -mod writer; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod writer; /// Maximum supported writer version pub const MAX_SUPPORTED_WRITER_VERSION: i32 = 1; @@ -128,8 +128,8 @@ impl DeltaOps { /// Audit active files with files present on the filesystem #[must_use] - pub fn optimize(self) -> FileSystemCheckBuilder { - FileSystemCheckBuilder::new(self.0.object_store(), self.0.state) + pub fn optimize<'a>(self) -> OptimizeBuilder<'a> { + OptimizeBuilder::new(self.0.object_store(), self.0.state) } } diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 36ca3f117d..1ebc4a0da5 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -11,27 +11,31 @@ //! optimized files. Optimize does not delete files from storage. To delete //! files that were removed, call `vacuum` on [`DeltaTable`]. //! -//! See [`Optimize`] for configuration. +//! See [`OptimizeBuilder`] for configuration. //! //! # Example //! ```rust ignore //! let table = open_table("../path/to/table")?; //! let metrics = Optimize::default().execute(table).await?; //! ```` -#![allow(deprecated)] -use crate::action::DeltaOperation; -use crate::action::{self, Action}; -use crate::table_properties; +use super::transaction::commit; +use super::writer::{PartitionWriter, PartitionWriterConfig}; +use crate::action::{self, Action, DeltaOperation}; +use crate::storage::ObjectStoreRef; +use crate::table_state::DeltaTableState; +use crate::writer::utils::arrow_schema_without_partitions; use crate::writer::utils::PartitionPath; -use crate::writer::{DeltaWriter, RecordBatchWriter}; -use crate::{DeltaDataTypeLong, DeltaTable, DeltaTableError, PartitionFilter}; -use log::debug; -use log::error; -use object_store::{path::Path, ObjectStore}; -use parquet::arrow::{ArrowReader, ParquetFileArrowReader}; -use parquet::file::reader::FileReader; -use parquet::file::serialized_reader::SerializedFileReader; +use crate::{table_properties, DeltaDataTypeVersion}; +use crate::{ + DeltaDataTypeLong, DeltaResult, DeltaTable, DeltaTableError, ObjectMeta, PartitionFilter, +}; +use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; +use futures::future::BoxFuture; +use futures::StreamExt; +use log::{debug, error}; +use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; +use parquet::file::properties::WriterProperties; use serde::{Deserialize, Serialize}; use serde_json::Map; use std::collections::HashMap; @@ -95,30 +99,85 @@ impl Default for MetricDetails { /// /// If a target file size is not provided then `delta.targetFileSize` from the /// table's configuration is read. Otherwise a default value is used. -#[derive(Default)] -pub struct Optimize<'a> { +#[derive(Debug)] +pub struct OptimizeBuilder<'a> { + /// A snapshot of the to-be-optimized table's state + snapshot: DeltaTableState, + /// Delta object store for handling data files + store: ObjectStoreRef, + /// Filters to select specific table partitions to be optimized filters: &'a [PartitionFilter<'a, &'a str>], + /// Desired file size after bin-packing files target_size: Option, + /// Properties passed to underlying parquet writer + writer_properties: Option, + /// Additional metadata to be added to commit + app_metadata: Option>, } -impl<'a> Optimize<'a> { +impl<'a> OptimizeBuilder<'a> { + /// Create a new [`OptimizeBuilder`] + pub fn new(store: ObjectStoreRef, snapshot: DeltaTableState) -> Self { + Self { + snapshot, + store, + filters: &[], + target_size: None, + writer_properties: None, + app_metadata: None, + } + } + /// Only optimize files that return true for the specified partition filter - pub fn filter(mut self, filters: &'a [PartitionFilter<'a, &'a str>]) -> Self { + pub fn with_filters(mut self, filters: &'a [PartitionFilter<'a, &'a str>]) -> Self { self.filters = filters; self } /// Set the target file size - pub fn target_size(mut self, target: DeltaDataTypeLong) -> Self { + pub fn with_target_size(mut self, target: DeltaDataTypeLong) -> Self { self.target_size = Some(target); self } - /// Perform the optimization. On completion, a summary of how many files were added and removed is returned - pub async fn execute(&self, table: &mut DeltaTable) -> Result { - let plan = create_merge_plan(table, self.filters, self.target_size.to_owned())?; - let metrics = plan.execute(table).await?; - Ok(metrics) + /// Writer properties passed to parquet writer + pub fn with_writer_properties(mut self, writer_properties: WriterProperties) -> Self { + self.writer_properties = Some(writer_properties); + self + } + + /// Additional metadata to be addedds to commit info + pub fn with_metadata( + mut self, + metadata: impl IntoIterator, + ) -> Self { + self.app_metadata = Some(HashMap::from_iter(metadata)); + self + } +} + +impl<'a> std::future::IntoFuture for OptimizeBuilder<'a> { + type Output = DeltaResult<(DeltaTable, Metrics)>; + type IntoFuture = BoxFuture<'a, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let this = self; + + Box::pin(async move { + let writer_properties = this + .writer_properties + .unwrap_or_else(|| WriterProperties::builder().build()); + let plan = create_merge_plan( + &this.snapshot, + this.filters, + this.target_size.to_owned(), + writer_properties, + )?; + let metrics = plan.execute(this.store.clone()).await?; + let mut table = DeltaTable::new_with_state(this.store, this.snapshot); + table.update().await?; + Ok((table, metrics)) + }) } } @@ -139,7 +198,7 @@ impl From for DeltaOperation { /// A collection of bins for a particular partition #[derive(Debug)] struct MergeBin { - files: Vec, + files: Vec, size_bytes: DeltaDataTypeLong, } @@ -165,9 +224,9 @@ impl MergeBin { self.files.len() } - fn add(&mut self, file_path: Path, size: i64) { - self.files.push(file_path); - self.size_bytes += size; + fn add(&mut self, meta: ObjectMeta) { + self.size_bytes += meta.size as i64; + self.files.push(meta); } } @@ -195,17 +254,27 @@ fn create_remove( /// Encapsulates the operations required to optimize a Delta Table pub struct MergePlan { operations: HashMap, + /// metrics collected during operation metrics: Metrics, + /// parameters passed to optimize operation input_parameters: OptimizeInput, + /// Schema of written files + file_schema: ArrowSchemaRef, + /// Coulumn names the table si partitioned by + partition_columns: Vec, + /// Properties passed to parquet writer + writer_properties: WriterProperties, + /// read table version + read_table_version: DeltaDataTypeVersion, } impl MergePlan { /// Peform the operations outlined in the plan. - pub async fn execute(self, table: &mut DeltaTable) -> Result { - // Read files into memory and write into memory. Once a file is complete write to underlying storage. + pub async fn execute(self, object_store: ObjectStoreRef) -> Result { let mut actions = vec![]; let mut metrics = self.metrics; + // TODO since we are now async in read and write, should we parallelize this? for (_partition_path, merge_partition) in self.operations.iter() { let partition_values = &merge_partition.partition_values; let bins = &merge_partition.bins; @@ -213,35 +282,44 @@ impl MergePlan { debug!("{:?}", _partition_path); for bin in bins { - let mut writer = RecordBatchWriter::for_table(table)?; - - for path in &bin.files { - //Read the file into memory and append it to the writer. - let data = table.storage.get(path).await?.bytes().await?; - let size: DeltaDataTypeLong = data.len().try_into().unwrap(); - let reader = SerializedFileReader::new(data)?; - let records = reader.metadata().file_metadata().num_rows(); - - let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(reader)); - - let batch_reader = - arrow_reader.get_record_reader(records.try_into().unwrap())?; - for batch in batch_reader { + let config = PartitionWriterConfig::try_new( + self.file_schema.clone(), + partition_values.clone(), + self.partition_columns.clone(), + Some(self.writer_properties.clone()), + Some(self.input_parameters.target_size as usize), + None, + )?; + let mut writer = PartitionWriter::try_with_config(object_store.clone(), config)?; + + for file_meta in &bin.files { + let file_reader = + ParquetObjectReader::new(object_store.clone(), file_meta.clone()); + let mut batch_stream = ParquetRecordBatchStreamBuilder::new(file_reader) + .await? + .build()?; + + while let Some(batch) = batch_stream.next().await { let batch = batch?; - writer.write_partition(batch, partition_values).await?; + writer.write(&batch).await?; } - actions.push(create_remove(path.as_ref(), partition_values, size)?); + let size = file_meta.size as i64; + actions.push(create_remove( + file_meta.location.as_ref(), + partition_values, + size, + )?); metrics.num_files_removed += 1; metrics.files_removed.total_files += 1; - metrics.files_removed.total_size += size; + metrics.files_removed.total_size += file_meta.size as i64; metrics.files_removed.max = std::cmp::max(metrics.files_removed.max, size); metrics.files_removed.min = std::cmp::min(metrics.files_removed.min, size); } // Save the file to storage and create corresponding add and remove actions. Do not commit yet. - let add_actions = writer.flush().await?; + let add_actions = writer.close().await?; if add_actions.len() != 1 { // Ensure we don't deviate from the merge plan which may result in idempotency being violated return Err(DeltaTableError::Generic( @@ -282,28 +360,30 @@ impl MergePlan { // optimized partition was updated then abort the commit. Requires (#593). if !actions.is_empty() { let mut metadata = Map::new(); - metadata.insert("readVersion".to_owned(), table.version().into()); + metadata.insert("readVersion".to_owned(), self.read_table_version.into()); let maybe_map_metrics = serde_json::to_value(metrics.clone()); if let Ok(map) = maybe_map_metrics { metadata.insert("operationMetrics".to_owned(), map); } - let mut dtx = table.create_transaction(None); - dtx.add_actions(actions); - // TODO: Add predicate information when we can convert the filter to a string representation - dtx.commit(Some(self.input_parameters.into()), Some(metadata)) - .await?; + commit( + object_store.as_ref(), + self.read_table_version + 1, + actions, + self.input_parameters.into(), + Some(metadata), + ) + .await?; } Ok(metrics) } } -fn get_target_file_size(table: &DeltaTable) -> DeltaDataTypeLong { - let config = table.get_configurations(); +fn get_target_file_size(snapshot: &DeltaTableState) -> DeltaDataTypeLong { let mut target_size = 268_435_456; - if let Ok(config) = config { - let config_str = config.get(table_properties::TARGET_FILE_SIZE); + if let Some(meta) = snapshot.current_metadata() { + let config_str = meta.configuration.get(table_properties::TARGET_FILE_SIZE); if let Some(s) = config_str { if let Some(s) = s { let r = s.parse::(); @@ -317,24 +397,27 @@ fn get_target_file_size(table: &DeltaTable) -> DeltaDataTypeLong { } } } - target_size } /// Build a Plan on which files to merge together. See [`Optimize`] pub fn create_merge_plan( - table: &mut DeltaTable, + snapshot: &DeltaTableState, filters: &[PartitionFilter<'_, &str>], target_size: Option, + writer_properties: WriterProperties, ) -> Result { - let target_size = target_size.unwrap_or_else(|| get_target_file_size(table)); + let target_size = target_size.unwrap_or_else(|| get_target_file_size(snapshot)); let mut candidates = HashMap::new(); let mut operations: HashMap = HashMap::new(); let mut metrics = Metrics::default(); - let partitions_keys = &table.get_metadata()?.partition_columns; + let partitions_keys = &snapshot + .current_metadata() + .ok_or(DeltaTableError::NoMetadata)? + .partition_columns; //Place each add action into a bucket determined by the file's partition - for add in table.get_active_add_actions_by_partitions(filters)? { + for add in snapshot.get_active_add_actions_by_partitions(filters)? { let path = PartitionPath::from_hashmap(partitions_keys, &add.partition_values)?; let v = candidates .entry(path) @@ -360,7 +443,7 @@ pub fn create_merge_plan( } if file.size + curr_bin.get_total_file_size() < target_size { - curr_bin.add(Path::from(file.path.as_str()), file.size); + curr_bin.add((*file).try_into()?); } else { if curr_bin.get_num_files() > 1 { bins.push(curr_bin); @@ -368,7 +451,7 @@ pub fn create_merge_plan( metrics.total_files_skipped += curr_bin.get_num_files(); } curr_bin = MergeBin::new(); - curr_bin.add(Path::from(file.path.as_str()), file.size); + curr_bin.add((*file).try_into()?); } } @@ -390,10 +473,20 @@ pub fn create_merge_plan( } let input_parameters = OptimizeInput { target_size }; + let file_schema = arrow_schema_without_partitions( + &Arc::new(>::try_from( + &snapshot.current_metadata().unwrap().schema, + )?), + partitions_keys, + ); Ok(MergePlan { operations, metrics, input_parameters, + writer_properties, + file_schema, + partition_columns: partitions_keys.clone(), + read_table_version: snapshot.version(), }) } diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index 4a98102199..5188853c8d 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -1,3 +1,5 @@ +//! Abstractions and implementations for writing data to delta tables + use std::collections::HashMap; use crate::action::Add; @@ -66,7 +68,8 @@ impl From for DeltaTableError { } } -pub(crate) struct WriterConfig { +/// Configuration to write data into Delta tables +pub struct WriterConfig { /// Schema of the delta table table_schema: ArrowSchemaRef, /// Column names for columns the table is partitioned by @@ -81,6 +84,7 @@ pub(crate) struct WriterConfig { } impl WriterConfig { + /// create a new instalce of [`WriterConfig`] pub fn new( table_schema: ArrowSchemaRef, partition_columns: Vec, @@ -105,13 +109,14 @@ impl WriterConfig { } } + /// Schema of files written to disk pub fn file_schema(&self) -> ArrowSchemaRef { arrow_schema_without_partitions(&self.table_schema, &self.partition_columns) } } /// A parquet writer implementation tailored to the needs of writing data to a delta table. -pub(crate) struct DeltaWriter { +pub struct DeltaWriter { /// An object store pointing at Delta table root object_store: ObjectStoreRef, /// configuration for the writers @@ -121,6 +126,7 @@ pub(crate) struct DeltaWriter { } impl DeltaWriter { + /// Create a new instance of [`DeltaWriter`] pub fn new(object_store: ObjectStoreRef, config: WriterConfig) -> Self { Self { object_store, @@ -193,6 +199,7 @@ impl DeltaWriter { Ok(()) } + /// close the writer pub async fn close(mut self) -> DeltaResult> { let writers = std::mem::take(&mut self.partition_writers); let mut actions = Vec::new(); diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index 37ec6ae899..360c241a84 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -31,6 +31,7 @@ pub use object_store::{ DynObjectStore, Error as ObjectStoreError, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, Result as ObjectStoreResult, }; +pub use utils::*; lazy_static! { static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index 0ccc6b480e..8c29e842db 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -2,11 +2,13 @@ use std::collections::HashMap; +use crate::action::Add; use crate::builder::DeltaTableBuilder; -use crate::DeltaTableError; +use crate::{DeltaResult, DeltaTableError}; +use chrono::{DateTime, NaiveDateTime, Utc}; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; -use object_store::{DynObjectStore, Result as ObjectStoreResult}; +use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; use std::sync::Arc; /// Copies the contents from the `from` location into the `to` location @@ -64,3 +66,35 @@ pub(crate) fn str_is_truthy(val: &str) -> bool { | val.eq_ignore_ascii_case("yes") | val.eq_ignore_ascii_case("y") } + +impl TryFrom for ObjectMeta { + type Error = DeltaTableError; + + fn try_from(value: Add) -> DeltaResult { + (&value).try_into() + } +} + +impl TryFrom<&Add> for ObjectMeta { + type Error = DeltaTableError; + + fn try_from(value: &Add) -> DeltaResult { + let last_modified = DateTime::::from_utc( + NaiveDateTime::from_timestamp_millis(value.modification_time).ok_or( + DeltaTableError::InvalidAction { + source: crate::action::ActionError::InvalidField(format!( + "invalid modification_time: {:?}", + value.modification_time + )), + }, + )?, + Utc, + ); + Ok(Self { + // TODO this won't work for absoute paths, since Paths are always relative to store. + location: Path::from(value.path.as_str()), + last_modified, + size: value.size as usize, + }) + } +} diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index ed07a31c7e..5006e4a209 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -6,17 +6,16 @@ use arrow::{ datatypes::{DataType, Field}, record_batch::RecordBatch, }; -use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics, Optimize}; -use deltalake::{ - action, - action::Remove, - builder::DeltaTableBuilder, - writer::{DeltaWriter, RecordBatchWriter}, -}; +use deltalake::action::{Action, Protocol, Remove}; +use deltalake::builder::DeltaTableBuilder; +use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics}; +use deltalake::operations::DeltaOps; +use deltalake::writer::{DeltaWriter, RecordBatchWriter}; use deltalake::{ DeltaTable, DeltaTableError, DeltaTableMetaData, PartitionFilter, Schema, SchemaDataType, SchemaField, }; +use parquet::file::properties::WriterProperties; use rand::prelude::*; use serde_json::{json, Map, Value}; use std::time::SystemTime; @@ -72,7 +71,7 @@ async fn setup_test(partitioned: bool) -> Result> { let mut commit_info = Map::::new(); - let protocol = action::Protocol { + let protocol = Protocol { min_reader_version: 1, min_writer_version: 2, }; @@ -196,8 +195,8 @@ async fn test_optimize_non_partitioned_table() -> Result<(), Box> { let version = dt.version(); assert_eq!(dt.get_state().files().len(), 5); - let optimize = Optimize::default().target_size(2_000_000); - let metrics = optimize.execute(&mut dt).await?; + let optimize = DeltaOps(dt).optimize().with_target_size(2_000_000); + let (dt, metrics) = optimize.await?; assert_eq!(version + 1, dt.version()); assert_eq!(metrics.num_files_added, 1); @@ -253,8 +252,8 @@ async fn test_optimize_with_partitions() -> Result<(), Box> { let version = dt.version(); let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; - let optimize = Optimize::default().filter(&filter); - let metrics = optimize.execute(&mut dt).await?; + let optimize = DeltaOps(dt).optimize().with_filters(&filter); + let (dt, metrics) = optimize.await?; assert_eq!(version + 1, dt.version()); assert_eq!(metrics.num_files_added, 1); @@ -290,7 +289,12 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { //create the merge plan, remove a file, and execute the plan. let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; - let plan = create_merge_plan(&mut dt, &filter, None)?; + let plan = create_merge_plan( + &dt.state, + &filter, + None, + WriterProperties::builder().build(), + )?; let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let mut other_dt = deltalake::open_table(uri).await?; @@ -311,15 +315,16 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { }; let mut transaction = other_dt.create_transaction(None); - transaction.add_action(action::Action::remove(remove)); + transaction.add_action(Action::remove(remove)); transaction.commit(None, None).await?; - let maybe_metrics = plan.execute(&mut dt).await; + let maybe_metrics = plan.execute(dt.object_store()).await; assert!(maybe_metrics.is_err()); assert_eq!(dt.version(), version + 1); Ok(()) } +#[ignore = "we do not yet re-try in operations commits."] #[tokio::test] /// Validate that optimize succeeds when only add actions occur for a optimized partition async fn test_no_conflict_for_append_actions() -> Result<(), Box> { @@ -345,7 +350,12 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { //create the merge plan, remove a file, and execute the plan. let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; - let plan = create_merge_plan(&mut dt, &filter, None)?; + let plan = create_merge_plan( + &dt.state, + &filter, + None, + WriterProperties::builder().build(), + )?; let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let mut other_dt = deltalake::open_table(uri).await?; @@ -357,7 +367,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { ) .await?; - let metrics = plan.execute(&mut dt).await?; + let metrics = plan.execute(dt.object_store()).await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); assert_eq!(dt.version(), version + 2); @@ -397,13 +407,20 @@ async fn test_idempotent() -> Result<(), Box> { let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; - let optimize = Optimize::default().filter(&filter).target_size(10_000_000); - let metrics = optimize.execute(&mut dt).await?; + let optimize = DeltaOps(dt) + .optimize() + .with_filters(&filter) + .with_target_size(10_000_000); + let (dt, metrics) = optimize.await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); assert_eq!(dt.version(), version + 1); - let metrics = optimize.execute(&mut dt).await?; + let optimize = DeltaOps(dt) + .optimize() + .with_filters(&filter) + .with_target_size(10_000_000); + let (dt, metrics) = optimize.await?; assert_eq!(metrics.num_files_added, 0); assert_eq!(metrics.num_files_removed, 0); @@ -427,8 +444,8 @@ async fn test_idempotent_metrics() -> Result<(), Box> { .await?; let version = dt.version(); - let optimize = Optimize::default().target_size(10_000_000); - let metrics = optimize.execute(&mut dt).await?; + let optimize = DeltaOps(dt).optimize().with_target_size(10_000_000); + let (dt, metrics) = optimize.await?; let expected_metric_details = MetricDetails { min: 0, @@ -480,8 +497,11 @@ async fn test_commit_info() -> Result<(), Box> { let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; - let optimize = Optimize::default().target_size(2_000_000).filter(&filter); - let metrics = optimize.execute(&mut dt).await?; + let optimize = DeltaOps(dt) + .optimize() + .with_target_size(2_000_000) + .with_filters(&filter); + let (mut dt, metrics) = optimize.await?; let commit_info = dt.history(None).await?; let last_commit = &commit_info[commit_info.len() - 1]; From dbb8be91233068557337e97406404cb8ceb3d133 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 19:51:24 +0100 Subject: [PATCH 10/67] fix: add missing feature cfg --- rust/src/operations/mod.rs | 1 + rust/src/operations/optimize.rs | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index f7a222948a..7a554809a5 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -127,6 +127,7 @@ impl DeltaOps { } /// Audit active files with files present on the filesystem + #[cfg(all(feature = "arrow", feature = "parquet"))] #[must_use] pub fn optimize<'a>(self) -> OptimizeBuilder<'a> { OptimizeBuilder::new(self.0.object_store(), self.0.state) diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 1ebc4a0da5..05c9a38a31 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -400,7 +400,7 @@ fn get_target_file_size(snapshot: &DeltaTableState) -> DeltaDataTypeLong { target_size } -/// Build a Plan on which files to merge together. See [`Optimize`] +/// Build a Plan on which files to merge together. See [`OptimizeBuilder`] pub fn create_merge_plan( snapshot: &DeltaTableState, filters: &[PartitionFilter<'_, &str>], From 9622ef5ab1f2e209fc18fc594d0765ae3bb77b74 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 20:26:21 +0100 Subject: [PATCH 11/67] fix: remove unwrap --- rust/src/operations/optimize.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 05c9a38a31..5e56da30f2 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -475,7 +475,10 @@ pub fn create_merge_plan( let input_parameters = OptimizeInput { target_size }; let file_schema = arrow_schema_without_partitions( &Arc::new(>::try_from( - &snapshot.current_metadata().unwrap().schema, + &snapshot + .current_metadata() + .ok_or(DeltaTableError::NoMetadata)? + .schema, )?), partitions_keys, ); From 3bdcd135ba63bd9b871875d0d5cf910f5cec7fb5 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Feb 2023 20:43:51 +0100 Subject: [PATCH 12/67] docs: fix optimize documentation --- rust/src/operations/optimize.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 5e56da30f2..cda565f233 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -16,7 +16,7 @@ //! # Example //! ```rust ignore //! let table = open_table("../path/to/table")?; -//! let metrics = Optimize::default().execute(table).await?; +//! let (table, metrics) = OptimizeBuilder::new(table.object_store(). table.state).await?; //! ```` use super::transaction::commit; From 2cfc391342f995202213a6638bdfb845d8e155df Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 20 Feb 2023 08:35:21 +0100 Subject: [PATCH 13/67] chore: move commit_uri_from_version --- rust/src/delta.rs | 137 ++++++++------------------------------ rust/src/storage/utils.rs | 10 ++- rust/src/table_state.rs | 3 +- 3 files changed, 37 insertions(+), 113 deletions(-) diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 260acc6bc7..9888ae47d0 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -18,7 +18,7 @@ use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; use crate::delta_config::DeltaConfigError; use crate::operations::vacuum::VacuumBuilder; -use crate::storage::ObjectStoreRef; +use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use chrono::{DateTime, Duration, Utc}; use futures::StreamExt; @@ -560,12 +560,6 @@ impl DeltaTable { self.storage.root_uri() } - /// Return the uri of commit version. - pub fn commit_uri_from_version(&self, version: DeltaDataTypeVersion) -> Path { - let version = format!("{version:020}.json"); - Path::from_iter(["_delta_log", &version]) - } - /// Return the list of paths of given checkpoint. pub fn get_checkpoint_data_paths(&self, check_point: &CheckPoint) -> Vec { let checkpoint_prefix = format!("{:020}", check_point.version); @@ -727,11 +721,7 @@ impl DeltaTable { // scan logs after checkpoint loop { - match self - .storage - .head(&self.commit_uri_from_version(version)) - .await - { + match self.storage.head(&commit_uri_from_version(version)).await { Ok(meta) => { // also cache timestamp for version self.version_timestamp @@ -778,7 +768,7 @@ impl DeltaTable { current_version: DeltaDataTypeVersion, ) -> Result { let next_version = current_version + 1; - let commit_uri = self.commit_uri_from_version(next_version); + let commit_uri = commit_uri_from_version(next_version); let commit_log_bytes = self.storage.get(&commit_uri).await; let commit_log_bytes = match commit_log_bytes { Err(ObjectStoreError::NotFound { .. }) => return Ok(PeekCommit::UpToDate), @@ -873,7 +863,7 @@ impl DeltaTable { version: DeltaDataTypeVersion, ) -> Result<(), DeltaTableError> { // check if version is valid - let commit_uri = self.commit_uri_from_version(version); + let commit_uri = commit_uri_from_version(version); match self.storage.head(&commit_uri).await { Ok(_) => {} Err(ObjectStoreError::NotFound { .. }) => { @@ -910,10 +900,7 @@ impl DeltaTable { match self.version_timestamp.get(&version) { Some(ts) => Ok(*ts), None => { - let meta = self - .storage - .head(&self.commit_uri_from_version(version)) - .await?; + let meta = self.storage.head(&commit_uri_from_version(version)).await?; let ts = meta.last_modified.timestamp(); // also cache timestamp for version self.version_timestamp.insert(version, ts); @@ -1145,7 +1132,7 @@ impl DeltaTable { // move temporary commit file to delta log directory // rely on storage to fail if the file already exists - self.storage - .rename_if_not_exists(&commit.uri, &self.commit_uri_from_version(version)) + .rename_if_not_exists(&commit.uri, &commit_uri_from_version(version)) .await .map_err(|e| match e { ObjectStoreError::AlreadyExists { .. } => { @@ -1318,7 +1305,6 @@ pub struct DeltaTransaction<'a> { delta_table: &'a mut DeltaTable, actions: Vec, options: DeltaTransactionOptions, - txn_id: String, } impl<'a> DeltaTransaction<'a> { @@ -1330,7 +1316,6 @@ impl<'a> DeltaTransaction<'a> { delta_table, actions: vec![], options: options.unwrap_or_default(), - txn_id: Uuid::new_v4().to_string(), } } @@ -1350,51 +1335,32 @@ impl<'a> DeltaTransaction<'a> { /// This method will retry the transaction commit based on the value of `max_retry_commit_attempts` set in `DeltaTransactionOptions`. pub async fn commit( &mut self, - operation: DeltaOperation, + operation: Option, app_metadata: Option>, ) -> Result { - // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, - // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... - // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 - let isolation_level = if self.can_downgrade_to_snapshot_isolation(&op) { - IsolationLevel::SnapshotIsolation - } else { - IsolationLevel::default_level() - }; + // TODO: stubbing `operation` parameter (which will be necessary for writing the CommitInfo action), + // but leaving it unused for now. `CommitInfo` is a fairly dynamic data structure so we should work + // out the data structure approach separately. // TODO: calculate isolation level to use when checking for conflicts. // Leaving conflict checking unimplemented for now to get the "single writer" implementation off the ground. // Leaving some commented code in place as a guidepost for the future. - // readPredicates.nonEmpty || readFiles.nonEmpty - // TODO revise logic if files are read - let depends_on_files = match operation { - DeltaOperation::Create { .. } | DeltaOperation::StreamingUpdate { .. } => false, - DeltaOperation::Optimize { .. } => true, - DeltaOperation::Write { - predicate: Some(_), .. - } => true, - _ => false, - }; + // let no_data_changed = actions.iter().all(|a| match a { + // Action::add(x) => !x.dataChange, + // Action::remove(x) => !x.dataChange, + // _ => false, + // }); + // let isolation_level = if no_data_changed { + // IsolationLevel::SnapshotIsolation + // } else { + // IsolationLevel::Serializable + // }; - let is_blind_append = only_add_files && !depends_on_files; - - let commit_info = CommitInfo { - version: None, - timestamp: chrono::Utc::now().timestamp(), - read_version: Some(self.delta_table.version()), - isolation_level: Some(isolation_level), - operation: operation.name(), - operation_parameters: op.operation_parameters(), - user_id: None, - user_name: None, - is_blind_append: Some(is_blind_append), - }; - - let prepared_commit = self.prepare_commit(Some(op.clone()), app_metadata).await?; + let prepared_commit = self.prepare_commit(operation, app_metadata).await?; // try to commit in a loop in case other writers write the next version first - let version = self.try_commit_loop(&prepared_commit, op).await?; + let version = self.try_commit_loop(&prepared_commit).await?; Ok(version) } @@ -1404,7 +1370,7 @@ impl<'a> DeltaTransaction<'a> { /// with `DeltaTable.try_commit_transaction`. pub async fn prepare_commit( &mut self, - operation: &DeltaOperation, + operation: Option, app_metadata: Option>, ) -> Result { if !self @@ -1421,8 +1387,10 @@ impl<'a> DeltaTransaction<'a> { "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - commit_info.append(&mut operation.get_commit_info()); + if let Some(op) = &operation { + commit_info.append(&mut op.get_commit_info()) + } if let Some(mut meta) = app_metadata { commit_info.append(&mut meta) } @@ -1449,7 +1417,6 @@ impl<'a> DeltaTransaction<'a> { async fn try_commit_loop( &mut self, commit: &PreparedCommit, - operation: DeltaOperation, ) -> Result { let mut attempt_number: u32 = 0; loop { @@ -1468,14 +1435,6 @@ impl<'a> DeltaTransaction<'a> { Err(e) => { match e { DeltaTableError::VersionAlreadyExists(_) => { - let checker = ConflictChecker::try_new( - self.delta_table, - version, - operation.clone(), - ) - .await?; - let _result = checker.check_conflicts()?; - // TODO update prepared commit in case transaction info got updated. if attempt_number > self.options.max_retry_commit_attempts + 1 { debug!("Transaction attempt failed. Attempts exhausted beyond max_retry_commit_attempts of {} so failing.", self.options.max_retry_commit_attempts); return Err(e); @@ -1493,50 +1452,6 @@ impl<'a> DeltaTransaction<'a> { } } } - - fn can_downgrade_to_snapshot_isolation(&self, operation: &DeltaOperation) -> bool { - let mut data_changed = false; - let mut has_non_file_actions = false; - for action in &self.actions { - match action { - Action::add(act) if act.data_change => data_changed = true, - Action::remove(rem) if rem.data_change => data_changed = true, - _ => has_non_file_actions = true, - } - } - - if has_non_file_actions { - // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation - // level to SnapshotIsolation. - return false; - } - - let default_isolation_level = IsolationLevel::default_level(); - // Note-1: For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can - // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict - // detection by skipping the - // [[ConflictChecker.checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn]] check i.e. - // don't worry about concurrent appends. - // Note-2: - // We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: - // t0 - Initial state of table - // t1 - Q1, Q2 starts - // t2 - Q1 commits - // t3 - Q2 is empty and wants to commit. - // In this scenario, we can always allow Q2 to commit without worrying about new files - // generated by Q1. - // The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 - // Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has - // a Metadata update (say schema change/identity column high watermark update), then Q2 can't - // be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, - // then Q1 should see the updates from Q2 - which actually didn't happen. - - match default_isolation_level { - IsolationLevel::Serializable => !data_changed, - IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), - _ => false, // This case should never happen - } - } } /// Holds the uri to prepared commit temporary file created with `DeltaTransaction.prepare_commit`. diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index 8c29e842db..27607deee6 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -4,13 +4,21 @@ use std::collections::HashMap; use crate::action::Add; use crate::builder::DeltaTableBuilder; -use crate::{DeltaResult, DeltaTableError}; +use crate::{DeltaDataTypeVersion, DeltaResult, DeltaTableError}; use chrono::{DateTime, NaiveDateTime, Utc}; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; use std::sync::Arc; +const DELTA_LOG_FOLDER: &str = "_delta_log"; + +/// Return the uri of commit version. +pub(crate) fn commit_uri_from_version(version: DeltaDataTypeVersion) -> Path { + let version = format!("{version:020}.json"); + Path::from_iter([DELTA_LOG_FOLDER, &version]) +} + /// Copies the contents from the `from` location into the `to` location pub async fn copy_table( from: impl AsRef, diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 1eaf862398..1ede850996 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -4,6 +4,7 @@ use crate::action::{self, Action, Add}; use crate::delta_config; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; +use crate::storage::commit_uri_from_version; use crate::{ ApplyLogError, DeltaDataTypeLong, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, @@ -64,7 +65,7 @@ impl DeltaTableState { table: &DeltaTable, version: DeltaDataTypeVersion, ) -> Result { - let commit_uri = table.commit_uri_from_version(version); + let commit_uri = commit_uri_from_version(version); let commit_log_bytes = table.storage.get(&commit_uri).await?.bytes().await?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); From e1e0b0b1b99cbecc2a38f1b6fef91966971cc6a4 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 20 Feb 2023 13:07:12 +0100 Subject: [PATCH 14/67] chore: build with ConflictChecker --- rust/src/action/mod.rs | 56 +++++---- rust/src/operations/create.rs | 1 + rust/src/operations/filesystem_check.rs | 2 + rust/src/operations/mod.rs | 1 + rust/src/operations/optimize.rs | 1 + .../transaction}/conflict_checker.rs | 74 ++++-------- .../{transaction.rs => transaction/mod.rs} | 109 ++++++++++++++++-- rust/src/operations/transaction/types.rs | 71 ++++++++++++ rust/src/operations/write.rs | 2 + 9 files changed, 235 insertions(+), 82 deletions(-) rename rust/src/{ => operations/transaction}/conflict_checker.rs (85%) rename rust/src/operations/{transaction.rs => transaction/mod.rs} (63%) create mode 100644 rust/src/operations/transaction/types.rs diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 978a485836..8055c89a45 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -8,7 +8,7 @@ mod parquet_read; #[cfg(feature = "parquet2")] pub mod parquet2_read; -use crate::{schema::*, DeltaTableError, DeltaTableMetaData}; +use crate::{schema::*, DeltaResult, DeltaTableError, DeltaTableMetaData}; use percent_encoding::percent_decode; use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; @@ -517,44 +517,60 @@ pub enum DeltaOperation { } impl DeltaOperation { - /// Retrieve basic commit information to be added to Delta commits - pub fn get_commit_info(&self) -> Map { - let mut commit_info = Map::::new(); - let operation = match &self { + /// A human readable name for the operation + pub fn name(&self) -> &str { + match &self { DeltaOperation::Create { .. } => "delta-rs.Create", DeltaOperation::Write { .. } => "delta-rs.Write", DeltaOperation::StreamingUpdate { .. } => "delta-rs.StreamingUpdate", DeltaOperation::Optimize { .. } => "delta-rs.Optimize", DeltaOperation::FileSystemCheck { .. } => "delta-rs.FileSystemCheck", - }; - commit_info.insert( - "operation".to_string(), - serde_json::Value::String(operation.into()), - ); + } + } - if let Ok(serde_json::Value::Object(map)) = serde_json::to_value(self) { - let all_operation_fields = map.values().next().unwrap().as_object().unwrap(); - let converted_operation_fields: Map = all_operation_fields - .iter() + /// Paraemters configured for operation. + pub fn operation_parameters(&self) -> DeltaResult> { + // TODO remove unwrap + let map2 = serde_json::to_value(self).unwrap(); + if let serde_json::Value::Object(map) = map2 { + let all_operation_fields = map.values().next().unwrap().as_object().unwrap().clone(); + Ok(all_operation_fields + .into_iter() .filter(|item| !item.1.is_null()) .map(|(k, v)| { ( - k.clone(), + k, serde_json::Value::String(if v.is_string() { String::from(v.as_str().unwrap()) } else { v.to_string() }), ) - }) - .collect(); + })) + } else { + todo!() + } + } + /// Denotes if the operation changes the data contained in the table + pub fn changes_data(&self) -> bool { + todo!() + } + + /// Retrieve basic commit information to be added to Delta commits + pub fn get_commit_info(&self) -> Map { + let mut commit_info = Map::::new(); + commit_info.insert( + "operation".to_string(), + serde_json::Value::String(self.name().into()), + ); + if let Ok(fields) = self.operation_parameters() { + let operation_parameters: Map = fields.collect(); commit_info.insert( "operationParameters".to_string(), - serde_json::Value::Object(converted_operation_fields), + serde_json::Value::Object(operation_parameters), ); - }; - + } commit_info } } diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index f10fb50bf5..9b47d2c1d6 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -298,6 +298,7 @@ impl std::future::IntoFuture for CreateBuilder { 0, actions, operation, + None, metadata, ) .await?; diff --git a/rust/src/operations/filesystem_check.rs b/rust/src/operations/filesystem_check.rs index 279748c19d..4144945bba 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/rust/src/operations/filesystem_check.rs @@ -158,6 +158,8 @@ impl FileSystemCheckPlan { version + 1, actions, DeltaOperation::FileSystemCheck {}, + Some(version), + // TODO pass through metadata None, ) .await?; diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 7a554809a5..18e86c45d2 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -17,6 +17,7 @@ pub mod create; pub mod filesystem_check; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod optimize; +#[allow(missing_docs, unused)] pub mod transaction; pub mod vacuum; diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index cda565f233..e249356e21 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -371,6 +371,7 @@ impl MergePlan { self.read_table_version + 1, actions, self.input_parameters.into(), + Some(self.read_table_version), Some(metadata), ) .await?; diff --git a/rust/src/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs similarity index 85% rename from rust/src/conflict_checker.rs rename to rust/src/operations/transaction/conflict_checker.rs index f9072ced66..518e39a290 100644 --- a/rust/src/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,31 +1,16 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. +use super::{CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; +use crate::storage::commit_uri_from_version; use crate::{ - table_state::DeltaTableState, DeltaDataTypeTimestamp, DeltaDataTypeVersion, DeltaTable, - DeltaTableError, DeltaTableMetaData, + table_state::DeltaTableState, DeltaDataTypeVersion, DeltaTable, DeltaTableError, + DeltaTableMetaData, }; -use serde::{Deserialize, Serialize}; +use object_store::ObjectStore; use serde_json::{Map, Value}; -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::io::{BufRead, BufReader, Cursor}; -/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. -/// However the reference implementation as well as delta-rs store useful information that may for instance -/// allow us to be more permissive in commit conflict resolution. -#[derive(Serialize, Deserialize, Debug, Clone)] -#[serde(rename_all = "camelCase")] -pub struct CommitInfo { - pub version: Option, - pub timestamp: DeltaDataTypeTimestamp, - pub user_id: Option, - pub user_name: Option, - pub operation: String, - pub operation_parameters: HashMap, - pub read_version: Option, - pub isolation_level: Option, - pub is_blind_append: Option, -} - /// Exceptions raised during commit conflict resolution #[derive(thiserror::Error, Debug)] pub enum CommitConflictError { @@ -73,32 +58,6 @@ pub enum CommitConflictError { UnsupportedReaderVersion(i32), } -#[derive(Serialize, Deserialize, Debug, Clone)] -pub enum IsolationLevel { - /// The strongest isolation level. It ensures that committed write operations - /// and all reads are Serializable. Operations are allowed as long as there - /// exists a serial sequence of executing them one-at-a-time that generates - /// the same outcome as that seen in the table. For the write operations, - /// the serial sequence is exactly the same as that seen in the table’s history. - Serializable, - - /// A weaker isolation level than Serializable. It ensures only that the write - /// operations (that is, not reads) are serializable. However, this is still stronger - /// than Snapshot isolation. WriteSerializable is the default isolation level because - /// it provides great balance of data consistency and availability for most common operations. - WriteSerializable, - - SnapshotIsolation, -} - -impl IsolationLevel { - /// The default isolation level to use, analogous to reference implementation - pub fn default_level() -> Self { - // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 - IsolationLevel::Serializable - } -} - /// A struct representing different attributes of current transaction needed for conflict detection. pub(crate) struct CurrentTransactionInfo { pub(crate) txn_id: String, @@ -121,7 +80,10 @@ pub(crate) struct CurrentTransactionInfo { } impl CurrentTransactionInfo { - pub fn try_new(table: &DeltaTable, operation: DeltaOperation) -> Result { + pub fn try_new( + _table: &DeltaTable, + _operation: DeltaOperation, + ) -> Result { todo!() } @@ -270,12 +232,20 @@ impl ConflictChecker { assert!(winning_commit_version == table.version() + 1); // create winning commit summary - let commit_uri = table.commit_uri_from_version(winning_commit_version); - let commit_log_bytes = table.storage.get_obj(&commit_uri).await?; + let commit_uri = commit_uri_from_version(winning_commit_version); + let commit_log_bytes = table.storage.get(&commit_uri).await?.bytes().await?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); let mut commit_actions = Vec::new(); - for line in reader.lines() { - commit_actions.push(serde_json::from_str::(line?.as_str())?); + + for maybe_line in reader.lines() { + let line = maybe_line?; + commit_actions.push(serde_json::from_str::(line.as_str()).map_err(|e| { + DeltaTableError::InvalidJsonLog { + json_err: e, + version: winning_commit_version, + line, + } + })?); } let winning_commit_summary = WinningCommitSummary::new(commit_actions, winning_commit_version); diff --git a/rust/src/operations/transaction.rs b/rust/src/operations/transaction/mod.rs similarity index 63% rename from rust/src/operations/transaction.rs rename to rust/src/operations/transaction/mod.rs index 974eaeeedc..bd241465e7 100644 --- a/rust/src/operations/transaction.rs +++ b/rust/src/operations/transaction/mod.rs @@ -1,13 +1,17 @@ //! Delta transactions use crate::action::{Action, DeltaOperation}; -use crate::storage::DeltaObjectStore; +use crate::storage::{commit_uri_from_version, DeltaObjectStore}; use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; - use chrono::Utc; use object_store::path::Path; use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::{Map, Value}; +mod conflict_checker; +mod types; + +pub use types::*; + const DELTA_LOG_FOLDER: &str = "_delta_log"; #[derive(thiserror::Error, Debug)] @@ -45,12 +49,6 @@ impl From for DeltaTableError { } } -/// Return the uri of commit version. -fn commit_uri_from_version(version: DeltaDataTypeVersion) -> Path { - let version = format!("{version:020}.json"); - Path::from_iter([DELTA_LOG_FOLDER, &version]) -} - // Convert actions to their json representation fn log_entry_from_actions(actions: &[Action]) -> Result { let mut jsons = Vec::::new(); @@ -130,8 +128,52 @@ pub(crate) async fn commit( version: DeltaDataTypeVersion, actions: Vec, operation: DeltaOperation, + read_version: Option, app_metadata: Option>, ) -> DeltaResult { + // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, + // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... + // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 + let isolation_level = if can_downgrade_to_snapshot_isolation(&actions, &operation) { + IsolationLevel::SnapshotIsolation + } else { + IsolationLevel::default_level() + }; + + // TODO: calculate isolation level to use when checking for conflicts. + // Leaving conflict checking unimplemented for now to get the "single writer" implementation off the ground. + // Leaving some commented code in place as a guidepost for the future. + + // readPredicates.nonEmpty || readFiles.nonEmpty + // TODO revise logic if files are read + let depends_on_files = match operation { + DeltaOperation::Create { .. } | DeltaOperation::StreamingUpdate { .. } => false, + DeltaOperation::Optimize { .. } => true, + DeltaOperation::Write { + predicate: Some(_), .. + } => true, + _ => false, + }; + + // TODO actually get the prop from commit infos... + let only_add_files = false; + let is_blind_append = only_add_files && !depends_on_files; + + let commit_info = CommitInfo { + version: None, + timestamp: chrono::Utc::now().timestamp(), + read_version, + isolation_level: Some(isolation_level), + operation: operation.name().to_string(), + operation_parameters: operation + .operation_parameters()? + .map(|(k, v)| (k, v.to_string())) + .collect(), + user_id: None, + user_name: None, + is_blind_append: Some(is_blind_append), + }; + let tmp_commit = prepare_commit(storage, operation, actions, app_metadata).await?; match try_commit_transaction(storage, &tmp_commit, version).await { Ok(version) => Ok(version), @@ -143,6 +185,53 @@ pub(crate) async fn commit( } } +fn can_downgrade_to_snapshot_isolation<'a>( + actions: impl IntoIterator, + operation: &DeltaOperation, +) -> bool { + let mut data_changed = false; + let mut has_non_file_actions = false; + for action in actions { + match action { + Action::add(act) if act.data_change => data_changed = true, + Action::remove(rem) if rem.data_change => data_changed = true, + _ => has_non_file_actions = true, + } + } + + if has_non_file_actions { + // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation + // level to SnapshotIsolation. + return false; + } + + let default_isolation_level = IsolationLevel::default_level(); + // Note-1: For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can + // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict + // detection by skipping the + // [[ConflictChecker.checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn]] check i.e. + // don't worry about concurrent appends. + // Note-2: + // We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: + // t0 - Initial state of table + // t1 - Q1, Q2 starts + // t2 - Q1 commits + // t3 - Q2 is empty and wants to commit. + // In this scenario, we can always allow Q2 to commit without worrying about new files + // generated by Q1. + // The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 + // Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has + // a Metadata update (say schema change/identity column high watermark update), then Q2 can't + // be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, + // then Q1 should see the updates from Q2 - which actually didn't happen. + + match default_isolation_level { + IsolationLevel::Serializable => !data_changed, + IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), + _ => false, // This case should never happen + } +} + #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; @@ -178,7 +267,7 @@ mod tests { .unwrap(); // successfully write in clean location - commit(storage.as_ref(), 0, vec![], operation.clone(), None) + commit(storage.as_ref(), 0, vec![], operation.clone(), None, None) .await .unwrap(); let head = storage.head(&commit_path).await; @@ -186,7 +275,7 @@ mod tests { assert_eq!(head.as_ref().unwrap().location, commit_path); // fail on overwriting - let failed_commit = commit(storage.as_ref(), 0, vec![], operation, None).await; + let failed_commit = commit(storage.as_ref(), 0, vec![], operation, None, None).await; assert!(failed_commit.is_err()); assert!(matches!( failed_commit.unwrap_err(), diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs new file mode 100644 index 0000000000..1a275f39c6 --- /dev/null +++ b/rust/src/operations/transaction/types.rs @@ -0,0 +1,71 @@ +//! Types and structs used when commitind operations to a delta table +use crate::{DeltaDataTypeTimestamp, DeltaDataTypeVersion, DeltaTableError}; +use serde::{Deserialize, Serialize}; +use std::collections::HashMap; +use std::str::FromStr; + +/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. +/// However the reference implementation as well as delta-rs store useful information that may for instance +/// allow us to be more permissive in commit conflict resolution. +#[derive(Serialize, Deserialize, Debug, Clone)] +#[serde(rename_all = "camelCase")] +pub struct CommitInfo { + pub version: Option, + pub timestamp: DeltaDataTypeTimestamp, + pub user_id: Option, + pub user_name: Option, + pub operation: String, + pub operation_parameters: HashMap, + pub read_version: Option, + pub isolation_level: Option, + pub is_blind_append: Option, +} + +#[derive(Serialize, Deserialize, Debug, Clone)] +pub enum IsolationLevel { + /// The strongest isolation level. It ensures that committed write operations + /// and all reads are Serializable. Operations are allowed as long as there + /// exists a serial sequence of executing them one-at-a-time that generates + /// the same outcome as that seen in the table. For the write operations, + /// the serial sequence is exactly the same as that seen in the table’s history. + Serializable, + + /// A weaker isolation level than Serializable. It ensures only that the write + /// operations (that is, not reads) are serializable. However, this is still stronger + /// than Snapshot isolation. WriteSerializable is the default isolation level because + /// it provides great balance of data consistency and availability for most common operations. + WriteSerializable, + + SnapshotIsolation, +} + +impl IsolationLevel { + /// The default isolation level to use, analogous to reference implementation + pub fn default_level() -> Self { + // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 + IsolationLevel::Serializable + } +} + +impl AsRef for IsolationLevel { + fn as_ref(&self) -> &str { + match self { + Self::Serializable => "Serializable", + Self::WriteSerializable => "WriteSerializable", + Self::SnapshotIsolation => "SnapshotIsolation", + } + } +} + +impl FromStr for IsolationLevel { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "serializable" => Ok(Self::Serializable), + "writeserializable" | "write_serializable" => Ok(Self::WriteSerializable), + "snapshotisolation" | "snapshot_isolation" => Ok(Self::SnapshotIsolation), + _ => todo!(), + } + } +} diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index 15ca82d6fd..fdfce81d88 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -437,6 +437,8 @@ impl std::future::IntoFuture for WriteBuilder { table.version() + 1, actions, operation, + Some(table.version()), + // TODO pass through metadata None, ) .await?; From dc96f19a73ef2e0242777ea539e1ebafa5f29c5e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 20 Feb 2023 19:14:43 +0100 Subject: [PATCH 15/67] feat: typed CommitInfo --- rust/src/action/mod.rs | 132 ++++++++++++++++-- rust/src/delta.rs | 24 ++-- .../transaction/conflict_checker.rs | 127 +++++++++++++---- rust/src/operations/transaction/mod.rs | 27 ++-- rust/src/operations/transaction/types.rs | 24 +--- rust/src/table_state.rs | 5 +- rust/tests/command_optimize.rs | 12 +- rust/tests/commit_info_format.rs | 16 +-- 8 files changed, 257 insertions(+), 110 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 8055c89a45..2c2a004bb5 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -8,6 +8,7 @@ mod parquet_read; #[cfg(feature = "parquet2")] pub mod parquet2_read; +use crate::operations::transaction::IsolationLevel; use crate::{schema::*, DeltaResult, DeltaTableError, DeltaTableMetaData}; use percent_encoding::percent_decode; use serde::{Deserialize, Serialize}; @@ -435,7 +436,46 @@ pub struct Protocol { pub min_writer_version: DeltaDataTypeInt, } -type CommitInfo = Map; +/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. +/// However the reference implementation as well as delta-rs store useful information that may for instance +/// allow us to be more permissive in commit conflict resolution. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq)] +#[serde(rename_all = "camelCase")] +pub struct CommitInfo { + /// Version number the commit corresponds to + #[serde(skip_serializing_if = "Option::is_none")] + pub version: Option, + /// Timestamp in millis when the commit was created + #[serde(skip_serializing_if = "Option::is_none")] + pub timestamp: Option, + /// Id of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_id: Option, + /// Name of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_name: Option, + /// The operation performed during the + #[serde(skip_serializing_if = "Option::is_none")] + pub operation: Option, + /// Parameters used for table operation + #[serde(skip_serializing_if = "Option::is_none")] + pub operation_parameters: Option>, + /// Version of the table when the operation was started + #[serde(skip_serializing_if = "Option::is_none")] + pub read_version: Option, + /// The isolation level of the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub isolation_level: Option, + /// TODO + #[serde(skip_serializing_if = "Option::is_none")] + pub is_blind_append: Option, + /// Delta engine which created the commit. + #[serde(skip_serializing_if = "Option::is_none")] + pub engine_info: Option, + /// Additional provenance information for the commit + #[serde(flatten, default)] + pub info: Map, +} /// Represents an action in the Delta log. The Delta log is an aggregate of all actions performed /// on the table, so the full list of actions is required to properly read a table. @@ -459,6 +499,16 @@ pub enum Action { commitInfo(CommitInfo), } +impl Action { + /// Create a commit info from a map + pub fn commit_info(info: Map) -> Self { + Self::commitInfo(CommitInfo { + info, + ..Default::default() + }) + } +} + /// Operation performed when creating a new log entry with one or more actions. /// This is a key element of the `CommitInfo` action. #[allow(clippy::large_enum_variant)] @@ -558,20 +608,13 @@ impl DeltaOperation { } /// Retrieve basic commit information to be added to Delta commits - pub fn get_commit_info(&self) -> Map { - let mut commit_info = Map::::new(); - commit_info.insert( - "operation".to_string(), - serde_json::Value::String(self.name().into()), - ); - if let Ok(fields) = self.operation_parameters() { - let operation_parameters: Map = fields.collect(); - commit_info.insert( - "operationParameters".to_string(), - serde_json::Value::Object(operation_parameters), - ); + pub fn get_commit_info(&self) -> CommitInfo { + // TODO infer additional info from operation parameters ... + CommitInfo { + operation: Some(self.name().into()), + operation_parameters: self.operation_parameters().ok().map(|iter| iter.collect()), + ..Default::default() } - commit_info } } @@ -670,4 +713,65 @@ mod tests { 1 ); } + + #[test] + fn test_read_commit_info() { + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c" + }"#; + + let info = serde_json::from_str::(raw); + assert!(info.is_ok()); + + println!("{:?}", info); + + // assert that commit info has no required filelds + let raw = "{}"; + let info = serde_json::from_str::(raw); + assert!(info.is_ok()); + + // arbitrary field data may be added to commit + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c", + "additionalField": "more data", + "additionalStruct": { + "key": "value", + "otherKey": 123 + } + }"#; + + let info = serde_json::from_str::(raw).expect("should parse"); + assert!(info.info.contains_key("additionalField")); + assert!(info.info.contains_key("additionalStruct")); + } } diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 9888ae47d0..c0b85f044e 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -917,7 +917,7 @@ impl DeltaTable { pub async fn history( &mut self, limit: Option, - ) -> Result>, DeltaTableError> { + ) -> Result, DeltaTableError> { let mut version = match limit { Some(l) => max(self.version() - l as i64 + 1, 0), None => self.get_earliest_delta_log_version().await?, @@ -1168,7 +1168,7 @@ impl DeltaTable { ); let mut actions = vec![ - Action::commitInfo(enriched_commit_info), + Action::commit_info(enriched_commit_info), Action::protocol(protocol), Action::metaData(meta), ]; @@ -1378,22 +1378,17 @@ impl<'a> DeltaTransaction<'a> { .iter() .any(|a| matches!(a, action::Action::commitInfo(..))) { - let mut commit_info = Map::::new(); - commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - commit_info.insert( + let mut extra_info = Map::::new(); + let mut commit_info = operation.map(|op| op.get_commit_info()).unwrap_or_default(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - - if let Some(op) = &operation { - commit_info.append(&mut op.get_commit_info()) - } if let Some(mut meta) = app_metadata { - commit_info.append(&mut meta) + extra_info.append(&mut meta) } + commit_info.info = extra_info; self.add_action(action::Action::commitInfo(commit_info)); } @@ -1615,6 +1610,7 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_create_delta_table() { let (delta_md, protocol, dt, tmp_dir) = create_test_table().await; @@ -1648,7 +1644,7 @@ mod tests { assert_eq!(DeltaTableMetaData::try_from(action).unwrap(), delta_md); } Action::commitInfo(action) => { - let mut modified_action = action; + let mut modified_action = action.info; let timestamp = serde_json::Number::from(0i64); modified_action["timestamp"] = Value::Number(serde_json::Number::from(0i64)); let mut expected = Map::::new(); diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 518e39a290..5852674c47 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,7 +1,7 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. use super::{CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; -use crate::storage::commit_uri_from_version; +use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use crate::{ table_state::DeltaTableState, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, @@ -59,7 +59,7 @@ pub enum CommitConflictError { } /// A struct representing different attributes of current transaction needed for conflict detection. -pub(crate) struct CurrentTransactionInfo { +pub(crate) struct TransactionInfo { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction pub(crate) read_predicates: Vec, @@ -79,9 +79,9 @@ pub(crate) struct CurrentTransactionInfo { pub(crate) commit_info: Option>, } -impl CurrentTransactionInfo { +impl TransactionInfo { pub fn try_new( - _table: &DeltaTable, + _snapshot: &DeltaTableState, _operation: DeltaOperation, ) -> Result { todo!() @@ -109,13 +109,9 @@ impl WinningCommitSummary { .find(|action| matches!(action, Action::commitInfo(_))) .map(|action| match action { Action::commitInfo(info) => { - // TODO remove panic - let mut ci = serde_json::from_value::(serde_json::Value::Object( - info.clone(), - )) - .unwrap(); - ci.version = Some(version); - ci + let mut updated = info.clone(); + updated.version = Some(version); + updated } _ => unreachable!(), }); @@ -210,30 +206,31 @@ impl WinningCommitSummary { } } -pub(crate) struct ConflictChecker { +pub(crate) struct ConflictChecker<'a> { /// transaction information for current transaction at start of check - initial_current_transaction_info: CurrentTransactionInfo, + initial_current_transaction_info: TransactionInfo, winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction winning_commit_summary: WinningCommitSummary, /// isolation level for the current transaction isolation_level: IsolationLevel, /// The state of the delta table at the base version from the current (not winning) commit - state: DeltaTableState, + snapshot: &'a DeltaTableState, } -impl ConflictChecker { +impl<'a> ConflictChecker<'a> { pub async fn try_new( - table: &DeltaTable, + snapshot: &'a DeltaTableState, + object_store: ObjectStoreRef, winning_commit_version: DeltaDataTypeVersion, operation: DeltaOperation, - ) -> Result { + ) -> Result, DeltaTableError> { // TODO raise proper error here - assert!(winning_commit_version == table.version() + 1); + assert!(winning_commit_version == snapshot.version() + 1); // create winning commit summary let commit_uri = commit_uri_from_version(winning_commit_version); - let commit_log_bytes = table.storage.get(&commit_uri).await?.bytes().await?; + let commit_log_bytes = object_store.get(&commit_uri).await?.bytes().await?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); let mut commit_actions = Vec::new(); @@ -250,27 +247,26 @@ impl ConflictChecker { let winning_commit_summary = WinningCommitSummary::new(commit_actions, winning_commit_version); - let initial_current_transaction_info = CurrentTransactionInfo::try_new(table, operation)?; + let initial_current_transaction_info = TransactionInfo::try_new(snapshot, operation)?; Ok(Self { initial_current_transaction_info, winning_commit_summary, winning_commit_version, isolation_level: IsolationLevel::Serializable, - // TODO cloning the state is probably a bad idea, since it can be very large... - state: table.state.clone(), + snapshot, }) } - fn current_transaction_info(&self) -> &CurrentTransactionInfo { + fn current_transaction_info(&self) -> &TransactionInfo { // TODO figure out when we need to update this &self.initial_current_transaction_info } /// This function checks conflict of the `initial_current_transaction_info` against the - /// `winning_commit_version` and returns an updated [CurrentTransactionInfo] that represents + /// `winning_commit_version` and returns an updated [`TransactionInfo`] that represents /// the transaction as if it had started while reading the `winning_commit_version`. - pub fn check_conflicts(&self) -> Result { + pub fn check_conflicts(&self) -> Result { self.check_protocol_compatibility()?; self.check_no_metadata_updates()?; self.check_for_added_files_that_should_have_been_read_by_current_txn()?; @@ -284,8 +280,8 @@ impl ConflictChecker { /// to read and write against the protocol set by the committed transaction. fn check_protocol_compatibility(&self) -> Result<(), CommitConflictError> { for p in self.winning_commit_summary.protocol() { - if self.state.min_reader_version() < p.min_reader_version - || self.state.min_writer_version() < p.min_writer_version + if self.snapshot.min_reader_version() < p.min_reader_version + || self.snapshot.min_writer_version() < p.min_writer_version { return Err(CommitConflictError::ProtocolChanged); }; @@ -426,4 +422,79 @@ impl ConflictChecker { } #[cfg(test)] -mod tests {} +mod tests { + use super::*; + use crate::action::{MetaData, Protocol}; + use crate::schema::Schema; + use crate::table_state::DeltaTableState; + use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData, SchemaDataType, SchemaField}; + use std::collections::HashMap; + + fn get_table_actions() -> Vec { + let protocol = Protocol { + min_reader_version: 1, + min_writer_version: 1, + }; + let table_schema = Schema::new(vec![ + SchemaField::new( + "id".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "value".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "modified".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + ]); + let metadata = + DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c" + }"#; + + let commit_info = serde_json::from_str::(raw).unwrap(); + + vec![ + Action::commitInfo(commit_info), + Action::protocol(protocol), + Action::metaData(MetaData::try_from(metadata).unwrap()), + ] + } + + async fn create_initialized_table(partition_cols: &[String]) -> DeltaTable { + let storage = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap(); + let state = DeltaTableState::from_actions(get_table_actions(), 0).unwrap(); + DeltaTable::new_with_state(storage, state) + } + + #[test] + fn test_append_only_commits() { + () + } +} diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index bd241465e7..e1c0324d7a 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -1,5 +1,5 @@ //! Delta transactions -use crate::action::{Action, DeltaOperation}; +use crate::action::{Action, CommitInfo, DeltaOperation}; use crate::storage::{commit_uri_from_version, DeltaObjectStore}; use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; use chrono::Utc; @@ -70,19 +70,17 @@ async fn prepare_commit( app_metadata: Option>, ) -> Result { if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { - let mut commit_info = Map::::new(); - commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - commit_info.insert( + let mut extra_info = Map::::new(); + let mut commit_info = operation.get_commit_info(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - commit_info.append(&mut operation.get_commit_info()); if let Some(mut meta) = app_metadata { - commit_info.append(&mut meta) + extra_info.append(&mut meta) } + commit_info.info = extra_info; actions.push(Action::commitInfo(commit_info)); } @@ -161,17 +159,16 @@ pub(crate) async fn commit( let commit_info = CommitInfo { version: None, - timestamp: chrono::Utc::now().timestamp(), + timestamp: Some(chrono::Utc::now().timestamp()), read_version, isolation_level: Some(isolation_level), - operation: operation.name().to_string(), - operation_parameters: operation - .operation_parameters()? - .map(|(k, v)| (k, v.to_string())) - .collect(), + operation: Some(operation.name().to_string()), + operation_parameters: Some(operation.operation_parameters()?.collect()), user_id: None, user_name: None, is_blind_append: Some(is_blind_append), + engine_info: Some(format!("Delta-RS/{}", crate_version())), + ..Default::default() }; let tmp_commit = prepare_commit(storage, operation, actions, app_metadata).await?; diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs index 1a275f39c6..637bfccbc2 100644 --- a/rust/src/operations/transaction/types.rs +++ b/rust/src/operations/transaction/types.rs @@ -4,24 +4,7 @@ use serde::{Deserialize, Serialize}; use std::collections::HashMap; use std::str::FromStr; -/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. -/// However the reference implementation as well as delta-rs store useful information that may for instance -/// allow us to be more permissive in commit conflict resolution. -#[derive(Serialize, Deserialize, Debug, Clone)] -#[serde(rename_all = "camelCase")] -pub struct CommitInfo { - pub version: Option, - pub timestamp: DeltaDataTypeTimestamp, - pub user_id: Option, - pub user_name: Option, - pub operation: String, - pub operation_parameters: HashMap, - pub read_version: Option, - pub isolation_level: Option, - pub is_blind_append: Option, -} - -#[derive(Serialize, Deserialize, Debug, Clone)] +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] pub enum IsolationLevel { /// The strongest isolation level. It ensures that committed write operations /// and all reads are Serializable. Operations are allowed as long as there @@ -69,3 +52,8 @@ impl FromStr for IsolationLevel { } } } + +#[cfg(test)] +mod tests { + use super::*; +} diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 1ede850996..d87e9d33ce 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -12,7 +12,6 @@ use crate::{ use chrono::Utc; use object_store::{path::Path, ObjectStore}; use serde::{Deserialize, Serialize}; -use serde_json::{Map, Value}; use std::collections::HashMap; use std::collections::HashSet; use std::convert::TryFrom; @@ -33,7 +32,7 @@ pub struct DeltaTableState { // active files for table state files: Vec, // Information added to individual commits - commit_infos: Vec>, + commit_infos: Vec, app_transaction_version: HashMap, min_reader_version: i32, min_writer_version: i32, @@ -163,7 +162,7 @@ impl DeltaTableState { } /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec> { + pub fn commit_infos(&self) -> &Vec { &self.commit_infos } diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 5006e4a209..9952c00825 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -507,16 +507,14 @@ async fn test_commit_info() -> Result<(), Box> { let last_commit = &commit_info[commit_info.len() - 1]; let commit_metrics = - serde_json::from_value::(last_commit["operationMetrics"].clone())?; + serde_json::from_value::(last_commit.info["operationMetrics"].clone())?; assert_eq!(commit_metrics, metrics); - assert_eq!(last_commit["readVersion"], json!(version)); - assert_eq!( - last_commit["operationParameters"]["targetSize"], - json!("2000000") - ); + assert_eq!(last_commit.read_version, Some(version)); + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["targetSize"], json!("2000000")); // TODO: Requires a string representation for PartitionFilter - assert_eq!(last_commit["operationParameters"]["predicate"], Value::Null); + // assert_eq!(parameters["predicate"], None); Ok(()) } diff --git a/rust/tests/commit_info_format.rs b/rust/tests/commit_info_format.rs index 8a04702579..60a8bf5ee4 100644 --- a/rust/tests/commit_info_format.rs +++ b/rust/tests/commit_info_format.rs @@ -2,8 +2,7 @@ mod fs_common; use deltalake::action::{Action, DeltaOperation, SaveMode}; - -use serde_json::{json, Value}; +use serde_json::json; use std::error::Error; use tempdir::TempDir; @@ -27,15 +26,10 @@ async fn test_operational_parameters() -> Result<(), Box> { let commit_info = table.history(None).await?; let last_commit = &commit_info[commit_info.len() - 1]; - - assert_eq!(last_commit["operationParameters"]["mode"], json!("Append")); - - assert_eq!( - last_commit["operationParameters"]["partitionBy"], - json!("[\"some_partition\"]") - ); - - assert_eq!(last_commit["operationParameters"]["predicate"], Value::Null); + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["mode"], json!("Append")); + assert_eq!(parameters["partitionBy"], json!("[\"some_partition\"]")); + // assert_eq!(parameters["predicate"], None); Ok(()) } From 8b65628410d981ccfe54b23965a3a9d22dee6cac Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 21 Feb 2023 03:48:55 +0100 Subject: [PATCH 16/67] feat: implement PruningStatistics for DeltaTableState --- rust/src/delta.rs | 2 +- rust/src/delta_datafusion.rs | 102 ++--------- rust/src/operations/mod.rs | 2 +- .../transaction/conflict_checker.rs | 133 ++++++++++++-- rust/src/operations/transaction/mod.rs | 4 +- rust/src/operations/transaction/state.rs | 166 ++++++++++++++++++ rust/src/operations/transaction/types.rs | 12 +- rust/src/table_state.rs | 6 + 8 files changed, 313 insertions(+), 114 deletions(-) create mode 100644 rust/src/operations/transaction/state.rs diff --git a/rust/src/delta.rs b/rust/src/delta.rs index c0b85f044e..f71b86dc62 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -1072,7 +1072,7 @@ impl DeltaTable { /// Return table schema parsed from transaction log. Return None if table hasn't been loaded or /// no metadata was found in the log. pub fn schema(&self) -> Option<&Schema> { - self.state.current_metadata().map(|m| &m.schema) + self.state.schema() } /// Return table schema parsed from transaction log. Return `DeltaTableError` if table hasn't diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 29ec01d121..9bc4f32508 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -20,13 +20,11 @@ //! }; //! ``` -use std::any::Any; -use std::collections::HashMap; -use std::convert::TryFrom; -use std::fmt::Debug; -use std::sync::Arc; - -use arrow::array::ArrayRef; +use crate::builder::ensure_table_uri; +use crate::Invariant; +use crate::{action, open_table, open_table_with_storage_options}; +use crate::{schema, DeltaTableBuilder}; +use crate::{DeltaTable, DeltaTableError}; use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, SchemaRef, TimeUnit}; use arrow::error::ArrowError; @@ -42,26 +40,25 @@ use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::optimizer::utils::conjunction; use datafusion::physical_expr::PhysicalSortExpr; -use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::file_format::{partition_type_wrap, FileScanConfig}; use datafusion::physical_plan::{ ColumnStatistics, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; use datafusion_common::scalar::ScalarValue; -use datafusion_common::{Column, DataFusionError, Result as DataFusionResult}; +use datafusion_common::{DataFusionError, Result as DataFusionResult}; use datafusion_expr::logical_plan::CreateExternalTable; use datafusion_expr::{Expr, Extension, LogicalPlan, TableProviderFilterPushDown}; use datafusion_proto::logical_plan::LogicalExtensionCodec; use datafusion_proto::physical_plan::PhysicalExtensionCodec; use object_store::{path::Path, ObjectMeta}; +use std::any::Any; +use std::collections::HashMap; +use std::convert::TryFrom; +use std::fmt::Debug; +use std::sync::Arc; use url::Url; -use crate::builder::ensure_table_uri; -use crate::Invariant; -use crate::{action, open_table, open_table_with_storage_options}; -use crate::{schema, DeltaTableBuilder}; -use crate::{DeltaTable, DeltaTableError}; - impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { match err { @@ -236,77 +233,6 @@ impl DeltaTable { } } -impl PruningStatistics for DeltaTable { - /// return the minimum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows - fn min_values(&self, column: &Column) -> Option { - let field = self - .get_schema() - .ok() - .map(|s| s.get_field_with_name(&column.name).ok())??; - let data_type = field.get_type().try_into().ok()?; - let values = self.get_state().files().iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - statistics - .min_values - .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) - .unwrap_or(ScalarValue::Null) - } else { - ScalarValue::Null - } - }); - ScalarValue::iter_to_array(values).ok() - } - - /// return the maximum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows. - fn max_values(&self, column: &Column) -> Option { - let field = self - .get_schema() - .ok() - .map(|s| s.get_field_with_name(&column.name).ok())??; - let data_type = field.get_type().try_into().ok()?; - let values = self.get_state().files().iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - statistics - .max_values - .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) - .unwrap_or(ScalarValue::Null) - } else { - ScalarValue::Null - } - }); - ScalarValue::iter_to_array(values).ok() - } - - /// return the number of containers (e.g. row groups) being - /// pruned with these statistics - fn num_containers(&self) -> usize { - self.get_state().files().len() - } - - /// return the number of null values for the named column as an - /// `Option`. - /// - /// Note: the returned array must contain `num_containers()` rows. - fn null_counts(&self, column: &Column) -> Option { - let values = self.get_state().files().iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - statistics - .null_count - .get(&column.name) - .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) - .unwrap_or(ScalarValue::UInt64(None)) - } else { - ScalarValue::UInt64(None) - } - }); - ScalarValue::iter_to_array(values).ok() - } -} - // each delta table must register a specific object store, since paths are internally // handled relative to the table root. fn register_store(table: &DeltaTable, env: Arc) { @@ -365,7 +291,7 @@ impl TableProvider for DeltaTable { (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) { let pruning_predicate = PruningPredicate::try_new(predicate, schema.clone())?; - let files_to_prune = pruning_predicate.prune(self)?; + let files_to_prune = pruning_predicate.prune(&self.state)?; self.get_state() .files() .iter() @@ -563,7 +489,7 @@ fn to_scalar_value(stat_val: &serde_json::Value) -> Option { } } -fn to_correct_scalar_value( +pub(crate) fn to_correct_scalar_value( stat_val: &serde_json::Value, field_dt: &ArrowDataType, ) -> Option { diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 18e86c45d2..7839eef49d 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -17,7 +17,7 @@ pub mod create; pub mod filesystem_check; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod optimize; -#[allow(missing_docs, unused)] +#[allow(missing_docs)] pub mod transaction; pub mod vacuum; diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 5852674c47..7e36e2c9c5 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,4 +1,5 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. +#![allow(unused)] use super::{CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; @@ -59,7 +60,7 @@ pub enum CommitConflictError { } /// A struct representing different attributes of current transaction needed for conflict detection. -pub(crate) struct TransactionInfo { +pub(crate) struct TransactionInfo<'a> { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction pub(crate) read_predicates: Vec, @@ -74,17 +75,27 @@ pub(crate) struct TransactionInfo { /// delta log actions that the transaction wants to commit pub(crate) actions: Vec, /// read [DeltaTableState] used for the transaction - pub(crate) read_snapshot: DeltaTableState, + pub(crate) read_snapshot: &'a DeltaTableState, /// [CommitInfo] for the commit - pub(crate) commit_info: Option>, + pub(crate) commit_info: Option, } -impl TransactionInfo { +impl<'a> TransactionInfo<'a> { pub fn try_new( - _snapshot: &DeltaTableState, - _operation: DeltaOperation, + snapshot: &'a DeltaTableState, + operation: DeltaOperation, ) -> Result { - todo!() + Ok(Self { + txn_id: "".into(), + read_predicates: vec![], + read_files: Default::default(), + read_whole_table: true, + read_app_ids: Default::default(), + metadata: snapshot.current_metadata().unwrap().clone(), + actions: vec![], + read_snapshot: snapshot, + commit_info: Some(operation.get_commit_info()), + }) } pub fn metadata_changed(&self) -> bool { @@ -208,7 +219,8 @@ impl WinningCommitSummary { pub(crate) struct ConflictChecker<'a> { /// transaction information for current transaction at start of check - initial_current_transaction_info: TransactionInfo, + initial_current_transaction_info: TransactionInfo<'a>, + /// Version numbe of commit, that has been committed ahead of the current transaction winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction winning_commit_summary: WinningCommitSummary, @@ -266,14 +278,14 @@ impl<'a> ConflictChecker<'a> { /// This function checks conflict of the `initial_current_transaction_info` against the /// `winning_commit_version` and returns an updated [`TransactionInfo`] that represents /// the transaction as if it had started while reading the `winning_commit_version`. - pub fn check_conflicts(&self) -> Result { + pub fn check_conflicts(&self) -> Result<(), CommitConflictError> { self.check_protocol_compatibility()?; self.check_no_metadata_updates()?; self.check_for_added_files_that_should_have_been_read_by_current_txn()?; self.check_for_deleted_files_against_current_txn_read_files()?; self.check_for_deleted_files_against_current_txn_deleted_files()?; self.check_for_updated_application_transaction_ids_that_current_txn_depends_on()?; - todo!() + Ok(()) } /// Asserts that the client is up to date with the protocol and is allowed @@ -333,7 +345,8 @@ impl<'a> ConflictChecker<'a> { // to assume all files match let added_files_matching_predicates = added_files_to_check; if !added_files_matching_predicates.is_empty() { - Err(CommitConflictError::ConcurrentAppend) + // Err(CommitConflictError::ConcurrentAppend) + Ok(()) } else { Ok(()) } @@ -424,12 +437,22 @@ impl<'a> ConflictChecker<'a> { #[cfg(test)] mod tests { use super::*; - use crate::action::{MetaData, Protocol}; + use crate::action::{Action, Add, MetaData, Protocol, SaveMode}; + use crate::operations::transaction::commit; use crate::schema::Schema; use crate::table_state::DeltaTableState; use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData, SchemaDataType, SchemaField}; use std::collections::HashMap; + fn create_add_action(path: impl Into, data_change: bool) -> Add { + Add { + path: path.into(), + size: 100, + data_change, + ..Default::default() + } + } + fn get_table_actions() -> Vec { let protocol = Protocol { min_reader_version: 1, @@ -489,12 +512,92 @@ mod tests { let storage = DeltaTableBuilder::from_uri("memory://") .build_storage() .unwrap(); + let table_schema = Schema::new(vec![ + SchemaField::new( + "id".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "value".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "modified".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + ]); let state = DeltaTableState::from_actions(get_table_actions(), 0).unwrap(); + let operation = DeltaOperation::Create { + mode: SaveMode::ErrorIfExists, + location: "location".into(), + protocol: Protocol { + min_reader_version: 1, + min_writer_version: 1, + }, + metadata: DeltaTableMetaData::new( + None, + None, + None, + table_schema, + vec![], + HashMap::new(), + ), + }; + commit( + storage.as_ref(), + 0, + get_table_actions(), + operation, + None, + None, + ) + .await + .unwrap(); DeltaTable::new_with_state(storage, state) } - #[test] - fn test_append_only_commits() { - () + #[tokio::test] + async fn test_append_only_commits() { + let table = create_initialized_table(&[]).await; + + let commit_info = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: Default::default(), + predicate: None, + } + .get_commit_info(); + + let add = create_add_action("file-path", true); + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: Default::default(), + predicate: None, + }; + + commit( + table.object_store().as_ref(), + 1, + vec![Action::add(add)], + operation.clone(), + Some(0), + None, + ) + .await + .unwrap(); + + let checker = ConflictChecker::try_new(&table.state, table.object_store(), 1, operation) + .await + .unwrap(); + + println!("actions: {:?}", checker.winning_commit_summary.actions); + + let result = checker.check_conflicts(); + println!("result: {:?}", result); } } diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index e1c0324d7a..5fcde973d7 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -8,6 +8,8 @@ use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::{Map, Value}; mod conflict_checker; +#[cfg(feature = "datafusion")] +mod state; mod types; pub use types::*; @@ -157,7 +159,7 @@ pub(crate) async fn commit( let only_add_files = false; let is_blind_append = only_add_files && !depends_on_files; - let commit_info = CommitInfo { + let _commit_info = CommitInfo { version: None, timestamp: Some(chrono::Utc::now().timestamp()), read_version, diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs new file mode 100644 index 0000000000..89145d32c4 --- /dev/null +++ b/rust/src/operations/transaction/state.rs @@ -0,0 +1,166 @@ +use crate::action::Add; +use crate::delta_datafusion::to_correct_scalar_value; +use crate::schema; +use crate::table_state::DeltaTableState; +use crate::DeltaResult; +use arrow::array::ArrayRef; +use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; +use datafusion::optimizer::utils::conjunction; +use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use datafusion_common::scalar::ScalarValue; +use datafusion_common::Column; +use datafusion_expr::Expr; +use itertools::Either; +use std::convert::TryFrom; +use std::sync::Arc; + +impl DeltaTableState { + pub fn files_matching_predicate( + &self, + filters: &[Expr], + ) -> DeltaResult> { + if let Some(Some(predicate)) = + (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) + { + let arrow_schema = Arc::new(>::try_from( + self.schema().unwrap(), + )?); + + let pruning_predicate = PruningPredicate::try_new(predicate, arrow_schema)?; + let files_to_prune = pruning_predicate.prune(self)?; + Ok(Either::Left( + self.files() + .iter() + .zip(files_to_prune.into_iter()) + .filter_map( + |(action, keep_file)| { + if keep_file { + Some(action) + } else { + None + } + }, + ), + )) + } else { + Ok(Either::Right(self.files().iter())) + } + } +} + +pub struct AddContainer<'a> { + inner: &'a Vec, + schema: ArrowSchemaRef, +} + +impl<'a> AddContainer<'a> { + pub fn new(adds: &'a Vec, schema: ArrowSchemaRef) -> Self { + Self { + inner: adds, + schema, + } + } +} + +impl<'a> PruningStatistics for AddContainer<'a> { + /// return the minimum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows + fn min_values(&self, column: &Column) -> Option { + let data_type = self.schema.field_with_name(&column.name).ok()?.data_type(); + let values = self.inner.iter().map(|add| { + if let Ok(Some(statistics)) = add.get_stats() { + statistics + .min_values + .get(&column.name) + .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) + .unwrap_or(ScalarValue::Null) + } else { + ScalarValue::Null + } + }); + ScalarValue::iter_to_array(values).ok() + } + + /// return the maximum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows. + fn max_values(&self, column: &Column) -> Option { + let data_type = self.schema.field_with_name(&column.name).ok()?.data_type(); + let values = self.inner.iter().map(|add| { + if let Ok(Some(statistics)) = add.get_stats() { + statistics + .max_values + .get(&column.name) + .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) + .unwrap_or(ScalarValue::Null) + } else { + ScalarValue::Null + } + }); + ScalarValue::iter_to_array(values).ok() + } + + /// return the number of containers (e.g. row groups) being + /// pruned with these statistics + fn num_containers(&self) -> usize { + self.inner.len() + } + + /// return the number of null values for the named column as an + /// `Option`. + /// + /// Note: the returned array must contain `num_containers()` rows. + fn null_counts(&self, column: &Column) -> Option { + let values = self.inner.iter().map(|add| { + if let Ok(Some(statistics)) = add.get_stats() { + statistics + .null_count + .get(&column.name) + .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) + .unwrap_or(ScalarValue::UInt64(None)) + } else { + ScalarValue::UInt64(None) + } + }); + ScalarValue::iter_to_array(values).ok() + } +} + +impl PruningStatistics for DeltaTableState { + /// return the minimum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows + fn min_values(&self, column: &Column) -> Option { + let arrow_schema = Arc::new( + >::try_from(self.schema().unwrap()).ok()?, + ); + let container = AddContainer::new(self.files(), arrow_schema); + container.min_values(column) + } + + /// return the maximum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows. + fn max_values(&self, column: &Column) -> Option { + let arrow_schema = Arc::new( + >::try_from(self.schema().unwrap()).ok()?, + ); + let container = AddContainer::new(self.files(), arrow_schema); + container.max_values(column) + } + + /// return the number of containers (e.g. row groups) being + /// pruned with these statistics + fn num_containers(&self) -> usize { + self.files().len() + } + + /// return the number of null values for the named column as an + /// `Option`. + /// + /// Note: the returned array must contain `num_containers()` rows. + fn null_counts(&self, column: &Column) -> Option { + let arrow_schema = Arc::new( + >::try_from(self.schema().unwrap()).ok()?, + ); + let container = AddContainer::new(self.files(), arrow_schema); + container.null_counts(column) + } +} diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs index 637bfccbc2..af8604c34d 100644 --- a/rust/src/operations/transaction/types.rs +++ b/rust/src/operations/transaction/types.rs @@ -1,7 +1,6 @@ //! Types and structs used when commitind operations to a delta table -use crate::{DeltaDataTypeTimestamp, DeltaDataTypeVersion, DeltaTableError}; +use crate::DeltaTableError; use serde::{Deserialize, Serialize}; -use std::collections::HashMap; use std::str::FromStr; #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] @@ -48,12 +47,9 @@ impl FromStr for IsolationLevel { "serializable" => Ok(Self::Serializable), "writeserializable" | "write_serializable" => Ok(Self::WriteSerializable), "snapshotisolation" | "snapshot_isolation" => Ok(Self::SnapshotIsolation), - _ => todo!(), + _ => Err(DeltaTableError::Generic( + "Invalid string for IsolationLevel".into(), + )), } } } - -#[cfg(test)] -mod tests { - use super::*; -} diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index d87e9d33ce..744cc6a571 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -5,6 +5,7 @@ use crate::delta_config; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; use crate::storage::commit_uri_from_version; +use crate::Schema; use crate::{ ApplyLogError, DeltaDataTypeLong, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, @@ -228,6 +229,11 @@ impl DeltaTableState { self.current_metadata.as_ref() } + /// The table schema + pub fn schema(&self) -> Option<&Schema> { + self.current_metadata.as_ref().map(|m| &m.schema) + } + /// Merges new state information into our state /// /// The DeltaTableState also carries the version information for the given state, From 59df7b07851c2b10ee84fe68cfc969c38bfe590d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 21 Feb 2023 13:03:50 +0100 Subject: [PATCH 17/67] feat: basic expression parsing --- .../transaction/conflict_checker.rs | 139 ++---------------- rust/src/operations/transaction/mod.rs | 2 + rust/src/operations/transaction/state.rs | 106 ++++++++++--- rust/src/operations/transaction/test_utils.rs | 127 ++++++++++++++++ 4 files changed, 228 insertions(+), 146 deletions(-) create mode 100644 rust/src/operations/transaction/test_utils.rs diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 7e36e2c9c5..3e1cef0689 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -343,7 +343,14 @@ impl<'a> ConflictChecker<'a> { // TODO here we need to check if the current transaction would have read the // added files. for this we need to be able to evaluate predicates. Err on the safe side is // to assume all files match - let added_files_matching_predicates = added_files_to_check; + cfg_if::cfg_if! { + if #[cfg(feature = "datafusion")] { + let added_files_matching_predicates = added_files_to_check; + } else { + let added_files_matching_predicates = added_files_to_check; + } + } + if !added_files_matching_predicates.is_empty() { // Err(CommitConflictError::ConcurrentAppend) Ok(()) @@ -368,6 +375,7 @@ impl<'a> ConflictChecker<'a> { .winning_commit_summary .removed_files() .iter() + // TODO remove cloned .cloned() .find(|f| read_file_path.contains(&f.path)); if deleted_read_overlap.is_some() @@ -436,135 +444,16 @@ impl<'a> ConflictChecker<'a> { #[cfg(test)] mod tests { + use super::super::test_utils::{ + create_add_action, create_initialized_table, init_table_actions, + }; use super::*; - use crate::action::{Action, Add, MetaData, Protocol, SaveMode}; + use crate::action::{Action, SaveMode}; use crate::operations::transaction::commit; - use crate::schema::Schema; - use crate::table_state::DeltaTableState; - use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData, SchemaDataType, SchemaField}; - use std::collections::HashMap; - - fn create_add_action(path: impl Into, data_change: bool) -> Add { - Add { - path: path.into(), - size: 100, - data_change, - ..Default::default() - } - } - - fn get_table_actions() -> Vec { - let protocol = Protocol { - min_reader_version: 1, - min_writer_version: 1, - }; - let table_schema = Schema::new(vec![ - SchemaField::new( - "id".to_string(), - SchemaDataType::primitive("string".to_string()), - true, - HashMap::new(), - ), - SchemaField::new( - "value".to_string(), - SchemaDataType::primitive("integer".to_string()), - true, - HashMap::new(), - ), - SchemaField::new( - "modified".to_string(), - SchemaDataType::primitive("string".to_string()), - true, - HashMap::new(), - ), - ]); - let metadata = - DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); - let raw = r#" - { - "timestamp": 1670892998177, - "operation": "WRITE", - "operationParameters": { - "mode": "Append", - "partitionBy": "[\"c1\",\"c2\"]" - }, - "isolationLevel": "Serializable", - "isBlindAppend": true, - "operationMetrics": { - "numFiles": "3", - "numOutputRows": "3", - "numOutputBytes": "1356" - }, - "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", - "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c" - }"#; - - let commit_info = serde_json::from_str::(raw).unwrap(); - - vec![ - Action::commitInfo(commit_info), - Action::protocol(protocol), - Action::metaData(MetaData::try_from(metadata).unwrap()), - ] - } - - async fn create_initialized_table(partition_cols: &[String]) -> DeltaTable { - let storage = DeltaTableBuilder::from_uri("memory://") - .build_storage() - .unwrap(); - let table_schema = Schema::new(vec![ - SchemaField::new( - "id".to_string(), - SchemaDataType::primitive("string".to_string()), - true, - HashMap::new(), - ), - SchemaField::new( - "value".to_string(), - SchemaDataType::primitive("integer".to_string()), - true, - HashMap::new(), - ), - SchemaField::new( - "modified".to_string(), - SchemaDataType::primitive("string".to_string()), - true, - HashMap::new(), - ), - ]); - let state = DeltaTableState::from_actions(get_table_actions(), 0).unwrap(); - let operation = DeltaOperation::Create { - mode: SaveMode::ErrorIfExists, - location: "location".into(), - protocol: Protocol { - min_reader_version: 1, - min_writer_version: 1, - }, - metadata: DeltaTableMetaData::new( - None, - None, - None, - table_schema, - vec![], - HashMap::new(), - ), - }; - commit( - storage.as_ref(), - 0, - get_table_actions(), - operation, - None, - None, - ) - .await - .unwrap(); - DeltaTable::new_with_state(storage, state) - } #[tokio::test] async fn test_append_only_commits() { - let table = create_initialized_table(&[]).await; + let table = create_initialized_table(&[], None).await; let commit_info = DeltaOperation::Write { mode: SaveMode::Append, diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 5fcde973d7..cdd2922c01 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -10,6 +10,8 @@ use serde_json::{Map, Value}; mod conflict_checker; #[cfg(feature = "datafusion")] mod state; +#[cfg(test)] +pub(crate) mod test_utils; mod types; pub use types::*; diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 89145d32c4..7cd708d49c 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -1,20 +1,33 @@ use crate::action::Add; use crate::delta_datafusion::to_correct_scalar_value; -use crate::schema; use crate::table_state::DeltaTableState; use crate::DeltaResult; +use crate::{schema, DeltaTableError}; use arrow::array::ArrayRef; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; +use arrow::datatypes::{DataType, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use datafusion::optimizer::utils::conjunction; use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use datafusion_common::config::ConfigOptions; use datafusion_common::scalar::ScalarValue; -use datafusion_common::Column; -use datafusion_expr::Expr; +use datafusion_common::{Column, DFSchema, Result as DFResult, TableReference}; +use datafusion_expr::{AggregateUDF, Expr, ScalarUDF, TableSource}; +use datafusion_sql::planner::{ContextProvider, SqlToRel}; use itertools::Either; +use sqlparser::dialect::GenericDialect; +use sqlparser::parser::Parser; +use sqlparser::tokenizer::Tokenizer; use std::convert::TryFrom; use std::sync::Arc; impl DeltaTableState { + pub fn arrow_schema(&self) -> DeltaResult { + Ok(Arc::new( + >::try_from( + self.schema().ok_or(DeltaTableError::NoMetadata)?, + )?, + )) + } + pub fn files_matching_predicate( &self, filters: &[Expr], @@ -22,11 +35,7 @@ impl DeltaTableState { if let Some(Some(predicate)) = (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) { - let arrow_schema = Arc::new(>::try_from( - self.schema().unwrap(), - )?); - - let pruning_predicate = PruningPredicate::try_new(predicate, arrow_schema)?; + let pruning_predicate = PruningPredicate::try_new(predicate, self.arrow_schema()?)?; let files_to_prune = pruning_predicate.prune(self)?; Ok(Either::Left( self.files() @@ -46,6 +55,29 @@ impl DeltaTableState { Ok(Either::Right(self.files().iter())) } } + + pub fn parse_predicate_expression(&self, expr: impl AsRef) -> DeltaResult { + let dialect = &GenericDialect {}; + let mut tokenizer = Tokenizer::new(dialect, expr.as_ref()); + let tokens = tokenizer + .tokenize() + .map_err(|err| DeltaTableError::GenericError { + source: Box::new(err), + })?; + let sql = Parser::new(dialect) + .with_tokens(tokens) + .parse_expr() + .map_err(|err| DeltaTableError::GenericError { + source: Box::new(err), + })?; + + // TODO should we add the table name as qualifier when available? + let df_schema = DFSchema::try_from_qualified_schema("", self.arrow_schema()?.as_ref())?; + let context_provider = DummyContextProvider::default(); + let sql_to_rel = SqlToRel::new(&context_provider); + + Ok(sql_to_rel.sql_to_expr(sql, &df_schema, &mut Default::default())?) + } } pub struct AddContainer<'a> { @@ -129,20 +161,14 @@ impl PruningStatistics for DeltaTableState { /// return the minimum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows fn min_values(&self, column: &Column) -> Option { - let arrow_schema = Arc::new( - >::try_from(self.schema().unwrap()).ok()?, - ); - let container = AddContainer::new(self.files(), arrow_schema); + let container = AddContainer::new(self.files(), self.arrow_schema().ok()?); container.min_values(column) } /// return the maximum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows. fn max_values(&self, column: &Column) -> Option { - let arrow_schema = Arc::new( - >::try_from(self.schema().unwrap()).ok()?, - ); - let container = AddContainer::new(self.files(), arrow_schema); + let container = AddContainer::new(self.files(), self.arrow_schema().ok()?); container.max_values(column) } @@ -157,10 +183,48 @@ impl PruningStatistics for DeltaTableState { /// /// Note: the returned array must contain `num_containers()` rows. fn null_counts(&self, column: &Column) -> Option { - let arrow_schema = Arc::new( - >::try_from(self.schema().unwrap()).ok()?, - ); - let container = AddContainer::new(self.files(), arrow_schema); + let container = AddContainer::new(self.files(), self.arrow_schema().ok()?); container.null_counts(column) } } + +#[derive(Default)] +struct DummyContextProvider { + options: ConfigOptions, +} + +impl ContextProvider for DummyContextProvider { + fn get_table_provider(&self, _name: TableReference) -> DFResult> { + unimplemented!() + } + + fn get_function_meta(&self, _name: &str) -> Option> { + unimplemented!() + } + + fn get_aggregate_meta(&self, _name: &str) -> Option> { + unimplemented!() + } + + fn get_variable_type(&self, _: &[String]) -> Option { + unimplemented!() + } + + fn options(&self) -> &ConfigOptions { + &self.options + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::operations::transaction::test_utils::init_table_actions; + + #[test] + fn test_parse_expression() { + let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); + let parsed = state.parse_predicate_expression("value > 10").unwrap(); + + println!("{:?}", parsed) + } +} diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs new file mode 100644 index 0000000000..146192f03d --- /dev/null +++ b/rust/src/operations/transaction/test_utils.rs @@ -0,0 +1,127 @@ +use super::{commit, CommitInfo}; +use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, SaveMode}; +use crate::table_state::DeltaTableState; +use crate::{ + DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, +}; +use std::collections::HashMap; + +pub fn create_add_action(path: impl Into, data_change: bool) -> Add { + Add { + path: path.into(), + size: 100, + data_change, + ..Default::default() + } +} + +pub fn init_table_actions() -> Vec { + let protocol = Protocol { + min_reader_version: 1, + min_writer_version: 1, + }; + let table_schema = Schema::new(vec![ + SchemaField::new( + "id".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "value".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "modified".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + ]); + let metadata = DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c" + }"#; + + let commit_info = serde_json::from_str::(raw).unwrap(); + + vec![ + Action::commitInfo(commit_info), + Action::protocol(protocol), + Action::metaData(MetaData::try_from(metadata).unwrap()), + ] +} + +pub async fn create_initialized_table( + partition_cols: &[String], + configuration: Option>>, +) -> DeltaTable { + let storage = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap(); + let table_schema = Schema::new(vec![ + SchemaField::new( + "id".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "value".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "modified".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + ]); + let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); + let operation = DeltaOperation::Create { + mode: SaveMode::ErrorIfExists, + location: "location".into(), + protocol: Protocol { + min_reader_version: 1, + min_writer_version: 1, + }, + metadata: DeltaTableMetaData::new( + None, + None, + None, + table_schema, + partition_cols.to_vec(), + configuration.unwrap_or_default(), + ), + }; + commit( + storage.as_ref(), + 0, + init_table_actions(), + operation, + None, + None, + ) + .await + .unwrap(); + DeltaTable::new_with_state(storage, state) +} From 466999ec4f0c3672783164c0a4a8898c6d78a421 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 21 Feb 2023 13:29:33 +0100 Subject: [PATCH 18/67] test: add basic parser tests --- rust/src/operations/transaction/state.rs | 21 +++++++++++++++++-- rust/src/operations/transaction/test_utils.rs | 4 ++-- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 7cd708d49c..d3ff5b4036 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -36,11 +36,10 @@ impl DeltaTableState { (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) { let pruning_predicate = PruningPredicate::try_new(predicate, self.arrow_schema()?)?; - let files_to_prune = pruning_predicate.prune(self)?; Ok(Either::Left( self.files() .iter() - .zip(files_to_prune.into_iter()) + .zip(pruning_predicate.prune(self)?.into_iter()) .filter_map( |(action, keep_file)| { if keep_file { @@ -219,11 +218,29 @@ impl ContextProvider for DummyContextProvider { mod tests { use super::*; use crate::operations::transaction::test_utils::init_table_actions; + use datafusion_expr::{col, lit}; #[test] fn test_parse_expression() { let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); + + // parses simple expression let parsed = state.parse_predicate_expression("value > 10").unwrap(); + let expected = col("value").gt(lit::(10)); + assert_eq!(parsed, expected); + + // fails for unknown column + let parsed = state.parse_predicate_expression("non_existent > 10"); + assert!(parsed.is_err()); + + // parses complex expression + let parsed = state + .parse_predicate_expression("value > 10 OR value <= 0") + .unwrap(); + let expected = col("value") + .gt(lit::(10)) + .or(col("value").lt_eq(lit::(0))); + assert_eq!(parsed, expected); println!("{:?}", parsed) } diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 146192f03d..45e320de24 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -17,8 +17,8 @@ pub fn create_add_action(path: impl Into, data_change: bool) -> Add { pub fn init_table_actions() -> Vec { let protocol = Protocol { - min_reader_version: 1, - min_writer_version: 1, + min_reader_version: crate::operations::MAX_SUPPORTED_READER_VERSION, + min_writer_version: crate::operations::MAX_SUPPORTED_WRITER_VERSION, }; let table_schema = Schema::new(vec![ SchemaField::new( From c8034b0eb8ad5a073582789dd61630df08844149 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 21 Feb 2023 19:15:07 +0100 Subject: [PATCH 19/67] test: test files with predicate --- .../transaction/conflict_checker.rs | 31 ++++++++++++----- rust/src/operations/transaction/state.rs | 33 +++++++++++++++++-- rust/src/operations/transaction/test_utils.rs | 7 +++- 3 files changed, 60 insertions(+), 11 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 3e1cef0689..193edbed7f 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,12 +1,13 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. #![allow(unused)] -use super::{CommitInfo, IsolationLevel}; +use super::{state::AddContainer, CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use crate::{ table_state::DeltaTableState, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, }; +use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use object_store::ObjectStore; use serde_json::{Map, Value}; use std::collections::HashSet; @@ -70,13 +71,11 @@ pub(crate) struct TransactionInfo<'a> { pub(crate) read_whole_table: bool, /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, - /// table metadata for the transaction - pub(crate) metadata: DeltaTableMetaData, /// delta log actions that the transaction wants to commit pub(crate) actions: Vec, - /// read [DeltaTableState] used for the transaction + /// read [`DeltaTableState`] used for the transaction pub(crate) read_snapshot: &'a DeltaTableState, - /// [CommitInfo] for the commit + /// [`CommitInfo`] for the commit pub(crate) commit_info: Option, } @@ -91,13 +90,16 @@ impl<'a> TransactionInfo<'a> { read_files: Default::default(), read_whole_table: true, read_app_ids: Default::default(), - metadata: snapshot.current_metadata().unwrap().clone(), actions: vec![], read_snapshot: snapshot, commit_info: Some(operation.get_commit_info()), }) } + pub fn metadata(&self) -> Option<&DeltaTableMetaData> { + self.read_snapshot.current_metadata() + } + pub fn metadata_changed(&self) -> bool { todo!() } @@ -237,7 +239,7 @@ impl<'a> ConflictChecker<'a> { winning_commit_version: DeltaDataTypeVersion, operation: DeltaOperation, ) -> Result, DeltaTableError> { - // TODO raise proper error here + // TODO raise proper error here or should we collect up so some max versions? assert!(winning_commit_version == snapshot.version() + 1); // create winning commit summary @@ -343,8 +345,21 @@ impl<'a> ConflictChecker<'a> { // TODO here we need to check if the current transaction would have read the // added files. for this we need to be able to evaluate predicates. Err on the safe side is // to assume all files match + // let container = AddContainer::new( + // self.initial_current_transaction_info.read_snapshot.files(), + // self.initial_current_transaction_info + // .read_snapshot + // .arrow_schema() + // .unwrap(), + // ); + // let pruning_predicate = PruningPredicate::try_new(predicate, self.arrow_schema()?)?; + // let added_files_matching_predicates = added_files_to_check; cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { + // let container = AddContainer::new( + // self.initial_current_transaction_info.read_snapshot.files(), + // self.initial_current_transaction_info.read_snapshot.arrow_schema().unwrap() + // ); let added_files_matching_predicates = added_files_to_check; } else { let added_files_matching_predicates = added_files_to_check; @@ -462,7 +477,7 @@ mod tests { } .get_commit_info(); - let add = create_add_action("file-path", true); + let add = create_add_action("file-path", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":10},\"nullCount\":{\"value\":0}}")); let operation = DeltaOperation::Write { mode: SaveMode::Append, partition_by: Default::default(), diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index d3ff5b4036..278b1e5eef 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -20,6 +20,7 @@ use std::convert::TryFrom; use std::sync::Arc; impl DeltaTableState { + /// Get the table schema as an [`ArrowSchemaRef`] pub fn arrow_schema(&self) -> DeltaResult { Ok(Arc::new( >::try_from( @@ -55,6 +56,7 @@ impl DeltaTableState { } } + /// Parse an expression string into a datafusion [`Expr`] pub fn parse_predicate_expression(&self, expr: impl AsRef) -> DeltaResult { let dialect = &GenericDialect {}; let mut tokenizer = Tokenizer::new(dialect, expr.as_ref()); @@ -217,11 +219,12 @@ impl ContextProvider for DummyContextProvider { #[cfg(test)] mod tests { use super::*; - use crate::operations::transaction::test_utils::init_table_actions; + use crate::action::Action; + use crate::operations::transaction::test_utils::{create_add_action, init_table_actions}; use datafusion_expr::{col, lit}; #[test] - fn test_parse_expression() { + fn test_parse_predicate_expression() { let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); // parses simple expression @@ -244,4 +247,30 @@ mod tests { println!("{:?}", parsed) } + + #[test] + fn test_files_matching_predicate() { + let mut actions = init_table_actions(); + actions.push(Action::add(create_add_action("excluded", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":10},\"nullCount\":{\"value\":0}}")))); + actions.push(Action::add(create_add_action("included-1", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":100},\"nullCount\":{\"value\":0}}")))); + actions.push(Action::add(create_add_action("included-2", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":-10},\"maxValues\":{\"value\":3},\"nullCount\":{\"value\":0}}")))); + + let state = DeltaTableState::from_actions(actions, 0).unwrap(); + let files = state + .files_matching_predicate(&[]) + .unwrap() + .collect::>(); + assert_eq!(files.len(), 3); + + let predictate = col("value") + .gt(lit::(10)) + .or(col("value").lt_eq(lit::(0))); + + let files = state + .files_matching_predicate(&[predictate]) + .unwrap() + .collect::>(); + assert_eq!(files.len(), 2); + assert!(files.iter().all(|add| add.path.contains("included"))); + } } diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 45e320de24..183c2b4da6 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -6,11 +6,16 @@ use crate::{ }; use std::collections::HashMap; -pub fn create_add_action(path: impl Into, data_change: bool) -> Add { +pub fn create_add_action( + path: impl Into, + data_change: bool, + stats: Option>, +) -> Add { Add { path: path.into(), size: 100, data_change, + stats: stats.map(|s| s.into()), ..Default::default() } } From a048d67ca89ce59900c68e26ccee72c5d310a2b5 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 21 Feb 2023 22:24:12 +0100 Subject: [PATCH 20/67] chore: some error handling --- rust/Cargo.toml | 4 ++ .../transaction/conflict_checker.rs | 50 ++++++++++++++----- 2 files changed, 41 insertions(+), 13 deletions(-) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index f0b1fff124..38f996e990 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -57,6 +57,8 @@ datafusion = { version = "18", optional = true } datafusion-expr = { version = "18", optional = true } datafusion-common = { version = "18", optional = true } datafusion-proto = { version = "18", optional = true } +datafusion-sql = { version = "18", optional = true } +sqlparser = { version = "0.30", optional = true } # NOTE dependencies only for integration tests fs_extra = { version = "1.2.0", optional = true } @@ -91,6 +93,8 @@ datafusion = [ "datafusion-expr", "datafusion-common", "datafusion-proto", + "datafusion-sql", + "sqlparser", "arrow", "parquet", ] diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 193edbed7f..82ed6a865c 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -2,6 +2,7 @@ #![allow(unused)] use super::{state::AddContainer, CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; +use crate::operations::transaction::TransactionError; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use crate::{ table_state::DeltaTableState, DeltaDataTypeVersion, DeltaTable, DeltaTableError, @@ -58,6 +59,20 @@ pub enum CommitConflictError { /// Error returned when the table requires an unsupported writer version #[error("Delta-rs does not support reader version {0}")] UnsupportedReaderVersion(i32), + + /// Error returned when the snapshot has missing or corrupted data + #[error("Snapshot is corrupted: {source}")] + CorruptedState { + /// Source error + source: Box, + }, + + /// Error returned when evaluating predicate + #[error("Error evaluating predicate: {source}")] + Predicate { + /// Source error + source: Box, + }, } /// A struct representing different attributes of current transaction needed for conflict detection. @@ -221,8 +236,8 @@ impl WinningCommitSummary { pub(crate) struct ConflictChecker<'a> { /// transaction information for current transaction at start of check - initial_current_transaction_info: TransactionInfo<'a>, - /// Version numbe of commit, that has been committed ahead of the current transaction + transaction_info: TransactionInfo<'a>, + /// Version number of commit, that has been committed ahead of the current transaction winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction winning_commit_summary: WinningCommitSummary, @@ -261,10 +276,10 @@ impl<'a> ConflictChecker<'a> { let winning_commit_summary = WinningCommitSummary::new(commit_actions, winning_commit_version); - let initial_current_transaction_info = TransactionInfo::try_new(snapshot, operation)?; + let transaction_info = TransactionInfo::try_new(snapshot, operation)?; Ok(Self { - initial_current_transaction_info, + transaction_info, winning_commit_summary, winning_commit_version, isolation_level: IsolationLevel::Serializable, @@ -274,7 +289,7 @@ impl<'a> ConflictChecker<'a> { fn current_transaction_info(&self) -> &TransactionInfo { // TODO figure out when we need to update this - &self.initial_current_transaction_info + &self.transaction_info } /// This function checks conflict of the `initial_current_transaction_info` against the @@ -345,14 +360,23 @@ impl<'a> ConflictChecker<'a> { // TODO here we need to check if the current transaction would have read the // added files. for this we need to be able to evaluate predicates. Err on the safe side is // to assume all files match - // let container = AddContainer::new( - // self.initial_current_transaction_info.read_snapshot.files(), - // self.initial_current_transaction_info - // .read_snapshot - // .arrow_schema() - // .unwrap(), - // ); - // let pruning_predicate = PruningPredicate::try_new(predicate, self.arrow_schema()?)?; + let arrow_schema = self + .transaction_info + .read_snapshot + .arrow_schema() + .map_err(|err| CommitConflictError::CorruptedState { + source: Box::new(err), + })?; + let _container = AddContainer::new( + self.transaction_info.read_snapshot.files(), + arrow_schema.clone(), + ); + // let pruning_predicate = + // PruningPredicate::try_new(predicate, arrow_schema).map_err(|err| { + // CommitConflictError::Predicate { + // source: Box::new(err), + // } + // })?; // let added_files_matching_predicates = added_files_to_check; cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { From 1f923acadc7cb55998e264eee1fbf2785a1142cb Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 21 Feb 2023 23:47:32 +0100 Subject: [PATCH 21/67] chore: going green --- rust/src/action/mod.rs | 9 +++ .../transaction/conflict_checker.rs | 63 ++++++++++--------- rust/src/operations/transaction/state.rs | 34 +++++++++- 3 files changed, 76 insertions(+), 30 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 2c2a004bb5..f37aa5c178 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -616,6 +616,15 @@ impl DeltaOperation { ..Default::default() } } + + /// Get predicate expression applien when the operation reads data from the table. + pub fn read_predicate(&self) -> Option { + match self { + // TODO add more operations + Self::Write { predicate, .. } => predicate.clone(), + _ => None, + } + } } /// The SaveMode used when performing a DeltaOperation diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 82ed6a865c..df597aea8d 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -97,7 +97,7 @@ pub(crate) struct TransactionInfo<'a> { impl<'a> TransactionInfo<'a> { pub fn try_new( snapshot: &'a DeltaTableState, - operation: DeltaOperation, + operation: &DeltaOperation, ) -> Result { Ok(Self { txn_id: "".into(), @@ -245,6 +245,8 @@ pub(crate) struct ConflictChecker<'a> { isolation_level: IsolationLevel, /// The state of the delta table at the base version from the current (not winning) commit snapshot: &'a DeltaTableState, + /// The state of the delta table at the base version from the current (not winning) commit + operation: DeltaOperation, } impl<'a> ConflictChecker<'a> { @@ -276,7 +278,7 @@ impl<'a> ConflictChecker<'a> { let winning_commit_summary = WinningCommitSummary::new(commit_actions, winning_commit_version); - let transaction_info = TransactionInfo::try_new(snapshot, operation)?; + let transaction_info = TransactionInfo::try_new(snapshot, &operation)?; Ok(Self { transaction_info, @@ -284,6 +286,7 @@ impl<'a> ConflictChecker<'a> { winning_commit_version, isolation_level: IsolationLevel::Serializable, snapshot, + operation, }) } @@ -357,42 +360,46 @@ impl<'a> ConflictChecker<'a> { } IsolationLevel::SnapshotIsolation => vec![], }; - // TODO here we need to check if the current transaction would have read the + + // Here we need to check if the current transaction would have read the // added files. for this we need to be able to evaluate predicates. Err on the safe side is // to assume all files match - let arrow_schema = self - .transaction_info - .read_snapshot - .arrow_schema() - .map_err(|err| CommitConflictError::CorruptedState { - source: Box::new(err), - })?; - let _container = AddContainer::new( - self.transaction_info.read_snapshot.files(), - arrow_schema.clone(), - ); - // let pruning_predicate = - // PruningPredicate::try_new(predicate, arrow_schema).map_err(|err| { - // CommitConflictError::Predicate { - // source: Box::new(err), - // } - // })?; - // let added_files_matching_predicates = added_files_to_check; cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { - // let container = AddContainer::new( - // self.initial_current_transaction_info.read_snapshot.files(), - // self.initial_current_transaction_info.read_snapshot.arrow_schema().unwrap() - // ); - let added_files_matching_predicates = added_files_to_check; + let added_files_matching_predicates = + if let Some(predicate_str) = self.operation.read_predicate() { + let arrow_schema = + self.transaction_info + .read_snapshot + .arrow_schema() + .map_err(|err| CommitConflictError::CorruptedState { + source: Box::new(err), + })?; + let predicate = self + .transaction_info + .read_snapshot + .parse_predicate_expression(predicate_str) + .map_err(|err| CommitConflictError::Predicate { + source: Box::new(err), + })?; + let container = AddContainer::new(&added_files_to_check, arrow_schema.clone()); + AddContainer::new(&added_files_to_check, arrow_schema) + .predicate_matches(&[predicate]) + .map_err(|err| CommitConflictError::Predicate { + source: Box::new(err), + })? + .cloned() + .collect::>() + } else { + added_files_to_check + }; } else { let added_files_matching_predicates = added_files_to_check; } } if !added_files_matching_predicates.is_empty() { - // Err(CommitConflictError::ConcurrentAppend) - Ok(()) + Err(CommitConflictError::ConcurrentAppend) } else { Ok(()) } diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 278b1e5eef..42bebb12dd 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -87,12 +87,42 @@ pub struct AddContainer<'a> { } impl<'a> AddContainer<'a> { + /// Create a new instance of [`AddContainer`] pub fn new(adds: &'a Vec, schema: ArrowSchemaRef) -> Self { Self { inner: adds, schema, } } + + /// Get an iterator of add actions / files, that MAY containtain data mathcing the predicate. + /// + /// Expressions are evaluated for file statistics, essentially column-wise min max bounds, + /// so evalutaing expressions is inexact. However exluded files are guaranteed (for a correct log) + /// to not contain matches by the predicate expression. + pub fn predicate_matches(&self, filters: &[Expr]) -> DeltaResult> { + if let Some(Some(predicate)) = + (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) + { + let pruning_predicate = PruningPredicate::try_new(predicate, self.schema.clone())?; + Ok(Either::Left( + self.inner + .iter() + .zip(pruning_predicate.prune(self)?.into_iter()) + .filter_map( + |(action, keep_file)| { + if keep_file { + Some(action) + } else { + None + } + }, + ), + )) + } else { + Ok(Either::Right(self.inner.iter())) + } + } } impl<'a> PruningStatistics for AddContainer<'a> { @@ -105,7 +135,7 @@ impl<'a> PruningStatistics for AddContainer<'a> { statistics .min_values .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) + .and_then(|f| to_correct_scalar_value(f.as_value()?, data_type)) .unwrap_or(ScalarValue::Null) } else { ScalarValue::Null @@ -123,7 +153,7 @@ impl<'a> PruningStatistics for AddContainer<'a> { statistics .max_values .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) + .and_then(|f| to_correct_scalar_value(f.as_value()?, data_type)) .unwrap_or(ScalarValue::Null) } else { ScalarValue::Null From 58f23ea9618075f7a1c8a2004010d7a557863163 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 22 Feb 2023 00:06:55 +0100 Subject: [PATCH 22/67] fix: feature-gate df import --- rust/src/operations/transaction/conflict_checker.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index df597aea8d..77a47ef33a 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -8,17 +8,19 @@ use crate::{ table_state::DeltaTableState, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, }; -use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use object_store::ObjectStore; use serde_json::{Map, Value}; use std::collections::HashSet; use std::io::{BufRead, BufReader, Cursor}; +#[cfg(feature = "datafusion")] +use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; + /// Exceptions raised during commit conflict resolution #[derive(thiserror::Error, Debug)] pub enum CommitConflictError { /// This exception occurs when a concurrent operation adds files in the same partition - /// (or anywhere in an unpartitioned table) that your operation reads. The file additions + /// (or anywhere in an un-partitioned table) that your operation reads. The file additions /// can be caused by INSERT, DELETE, UPDATE, or MERGE operations. #[error("Concurrent append failed.")] ConcurrentAppend, From a831d1a4c0bc6e2448b806bcba0af72ecb296316 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 22 Feb 2023 00:34:17 +0100 Subject: [PATCH 23/67] fix: parquet2 tests --- rust/src/action/parquet2_read/mod.rs | 2 +- rust/src/operations/transaction/conflict_checker.rs | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/rust/src/action/parquet2_read/mod.rs b/rust/src/action/parquet2_read/mod.rs index 34988d5954..ddfd2a9e55 100644 --- a/rust/src/action/parquet2_read/mod.rs +++ b/rust/src/action/parquet2_read/mod.rs @@ -223,7 +223,7 @@ impl ActionVariant for CommitInfo { type Variant = CommitInfo; fn default_action() -> Action { - Action::commitInfo(CommitInfo::new()) + Action::commitInfo(CommitInfo::default()) } fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 77a47ef33a..4f161c17b3 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,6 +1,6 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. #![allow(unused)] -use super::{state::AddContainer, CommitInfo, IsolationLevel}; +use super::{CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::operations::transaction::TransactionError; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; @@ -13,6 +13,8 @@ use serde_json::{Map, Value}; use std::collections::HashSet; use std::io::{BufRead, BufReader, Cursor}; +#[cfg(feature = "datafusion")] +use super::state::AddContainer; #[cfg(feature = "datafusion")] use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; @@ -384,7 +386,6 @@ impl<'a> ConflictChecker<'a> { .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), })?; - let container = AddContainer::new(&added_files_to_check, arrow_schema.clone()); AddContainer::new(&added_files_to_check, arrow_schema) .predicate_matches(&[predicate]) .map_err(|err| CommitConflictError::Predicate { From fbb008372a025ebef2aba44860183f5a69576f1b Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 22 Feb 2023 23:44:37 +0100 Subject: [PATCH 24/67] feat: initial re-try loop --- rust/src/operations/create.rs | 12 +- rust/src/operations/filesystem_check.rs | 16 +- rust/src/operations/optimize.rs | 13 +- .../transaction/conflict_checker.rs | 175 ++++++++++----- rust/src/operations/transaction/mod.rs | 202 ++++++------------ rust/src/operations/transaction/test_utils.rs | 19 +- rust/src/operations/write.rs | 5 +- rust/tests/command_optimize.rs | 4 +- 8 files changed, 210 insertions(+), 236 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 9b47d2c1d6..975110dfbc 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -278,7 +278,6 @@ impl std::future::IntoFuture for CreateBuilder { Box::pin(async move { let mode = this.mode.clone(); - let metadata = this.metadata.clone(); let (mut table, actions, operation) = this.into_table_and_actions()?; if table.object_store().is_delta_table_location().await? { match mode { @@ -293,15 +292,8 @@ impl std::future::IntoFuture for CreateBuilder { } } } - let version = commit( - table.object_store().as_ref(), - 0, - actions, - operation, - None, - metadata, - ) - .await?; + let version = + commit(table.object_store(), actions, operation, &table.state, None).await?; table.load_version(version).await?; Ok(table) diff --git a/rust/src/operations/filesystem_check.rs b/rust/src/operations/filesystem_check.rs index 4144945bba..f40225886e 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/rust/src/operations/filesystem_check.rs @@ -15,7 +15,6 @@ use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::operations::transaction::commit; use crate::storage::DeltaObjectStore; use crate::table_state::DeltaTableState; -use crate::DeltaDataTypeVersion; use crate::{DeltaDataTypeLong, DeltaResult, DeltaTable, DeltaTableError}; use futures::future::BoxFuture; use futures::StreamExt; @@ -50,8 +49,6 @@ pub struct FileSystemCheckMetrics { } struct FileSystemCheckPlan { - /// Version of the snapshot provided - version: DeltaDataTypeVersion, /// Delta object store for handling data files store: Arc, /// Files that no longer exists in undlying ObjectStore but have active add actions @@ -88,7 +85,6 @@ impl FileSystemCheckBuilder { async fn create_fsck_plan(&self) -> DeltaResult { let mut files_relative: HashMap<&str, &Add> = HashMap::with_capacity(self.state.files().len()); - let version = self.state.version(); let store = self.store.clone(); for active in self.state.files() { @@ -118,14 +114,13 @@ impl FileSystemCheckBuilder { Ok(FileSystemCheckPlan { files_to_remove, - version, store, }) } } impl FileSystemCheckPlan { - pub async fn execute(self) -> DeltaResult { + pub async fn execute(self, snapshot: &DeltaTableState) -> DeltaResult { if self.files_to_remove.is_empty() { return Ok(FileSystemCheckMetrics { dry_run: false, @@ -135,8 +130,6 @@ impl FileSystemCheckPlan { let mut actions = Vec::with_capacity(self.files_to_remove.len()); let mut removed_file_paths = Vec::with_capacity(self.files_to_remove.len()); - let version = self.version; - let store = &self.store; for file in self.files_to_remove { let deletion_time = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); @@ -154,11 +147,10 @@ impl FileSystemCheckPlan { } commit( - store, - version + 1, + self.store, actions, DeltaOperation::FileSystemCheck {}, - Some(version), + snapshot, // TODO pass through metadata None, ) @@ -190,7 +182,7 @@ impl std::future::IntoFuture for FileSystemCheckBuilder { )); } - let metrics = plan.execute().await?; + let metrics = plan.execute(&this.state).await?; let mut table = DeltaTable::new_with_state(this.store, this.state); table.update().await?; Ok((table, metrics)) diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index e249356e21..fb7aca3731 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -173,7 +173,7 @@ impl<'a> std::future::IntoFuture for OptimizeBuilder<'a> { this.target_size.to_owned(), writer_properties, )?; - let metrics = plan.execute(this.store.clone()).await?; + let metrics = plan.execute(this.store.clone(), &this.snapshot).await?; let mut table = DeltaTable::new_with_state(this.store, this.snapshot); table.update().await?; Ok((table, metrics)) @@ -270,7 +270,11 @@ pub struct MergePlan { impl MergePlan { /// Peform the operations outlined in the plan. - pub async fn execute(self, object_store: ObjectStoreRef) -> Result { + pub async fn execute( + self, + object_store: ObjectStoreRef, + snapshot: &DeltaTableState, + ) -> Result { let mut actions = vec![]; let mut metrics = self.metrics; @@ -367,11 +371,10 @@ impl MergePlan { } commit( - object_store.as_ref(), - self.read_table_version + 1, + object_store, actions, self.input_parameters.into(), - Some(self.read_table_version), + snapshot, Some(metadata), ) .await?; diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 4f161c17b3..71bba52faf 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -3,9 +3,9 @@ use super::{CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::operations::transaction::TransactionError; -use crate::storage::{commit_uri_from_version, ObjectStoreRef}; +use crate::storage::{commit_uri_from_version, DeltaObjectStore, ObjectStoreRef}; use crate::{ - table_state::DeltaTableState, DeltaDataTypeVersion, DeltaTable, DeltaTableError, + table_state::DeltaTableState, DeltaDataTypeVersion, DeltaResult, DeltaTable, DeltaTableError, DeltaTableMetaData, }; use object_store::ObjectStore; @@ -80,7 +80,7 @@ pub enum CommitConflictError { } /// A struct representing different attributes of current transaction needed for conflict detection. -pub(crate) struct TransactionInfo<'a> { +pub(crate) struct TransactionInfo<'a, 'b> { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction pub(crate) read_predicates: Vec, @@ -91,17 +91,18 @@ pub(crate) struct TransactionInfo<'a> { /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit - pub(crate) actions: Vec, + pub(crate) actions: &'b Vec, /// read [`DeltaTableState`] used for the transaction pub(crate) read_snapshot: &'a DeltaTableState, /// [`CommitInfo`] for the commit pub(crate) commit_info: Option, } -impl<'a> TransactionInfo<'a> { +impl<'a, 'b> TransactionInfo<'a, 'b> { pub fn try_new( snapshot: &'a DeltaTableState, operation: &DeltaOperation, + actions: &'b Vec, ) -> Result { Ok(Self { txn_id: "".into(), @@ -109,7 +110,7 @@ impl<'a> TransactionInfo<'a> { read_files: Default::default(), read_whole_table: true, read_app_ids: Default::default(), - actions: vec![], + actions, read_snapshot: snapshot, commit_info: Some(operation.get_commit_info()), }) @@ -135,22 +136,48 @@ pub(crate) struct WinningCommitSummary { } impl WinningCommitSummary { - pub fn new(actions: Vec, version: DeltaDataTypeVersion) -> Self { + pub async fn try_new( + object_store: &DeltaObjectStore, + read_version: DeltaDataTypeVersion, + winning_commit_version: DeltaDataTypeVersion, + ) -> DeltaResult { + let mut actions = Vec::new(); + let mut version_to_read = read_version + 1; + + while version_to_read <= winning_commit_version { + let commit_uri = commit_uri_from_version(winning_commit_version); + let commit_log_bytes = object_store.get(&commit_uri).await?.bytes().await?; + + let reader = BufReader::new(Cursor::new(commit_log_bytes)); + for maybe_line in reader.lines() { + let line = maybe_line?; + actions.push(serde_json::from_str::(line.as_str()).map_err(|e| { + DeltaTableError::InvalidJsonLog { + json_err: e, + version: winning_commit_version, + line, + } + })?); + } + version_to_read += 1; + } + + // TODO how to handle commit info for multiple read commits? let commit_info = actions .iter() .find(|action| matches!(action, Action::commitInfo(_))) .map(|action| match action { Action::commitInfo(info) => { - let mut updated = info.clone(); - updated.version = Some(version); - updated + // let mut updated = info.clone(); + // updated.version = Some(version_to_read); + info.clone() } _ => unreachable!(), }); - Self { + Ok(Self { actions, commit_info, - } + }) } pub fn metadata_updates(&self) -> Vec { @@ -238,57 +265,40 @@ impl WinningCommitSummary { } } -pub(crate) struct ConflictChecker<'a> { +pub(crate) struct ConflictChecker<'a, 'b> { /// transaction information for current transaction at start of check - transaction_info: TransactionInfo<'a>, + transaction_info: TransactionInfo<'a, 'b>, /// Version number of commit, that has been committed ahead of the current transaction winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction winning_commit_summary: WinningCommitSummary, - /// isolation level for the current transaction - isolation_level: IsolationLevel, /// The state of the delta table at the base version from the current (not winning) commit snapshot: &'a DeltaTableState, /// The state of the delta table at the base version from the current (not winning) commit operation: DeltaOperation, } -impl<'a> ConflictChecker<'a> { +impl<'a, 'b> ConflictChecker<'a, 'b> { pub async fn try_new( snapshot: &'a DeltaTableState, object_store: ObjectStoreRef, winning_commit_version: DeltaDataTypeVersion, operation: DeltaOperation, - ) -> Result, DeltaTableError> { - // TODO raise proper error here or should we collect up so some max versions? - assert!(winning_commit_version == snapshot.version() + 1); - - // create winning commit summary - let commit_uri = commit_uri_from_version(winning_commit_version); - let commit_log_bytes = object_store.get(&commit_uri).await?.bytes().await?; - let reader = BufReader::new(Cursor::new(commit_log_bytes)); - let mut commit_actions = Vec::new(); - - for maybe_line in reader.lines() { - let line = maybe_line?; - commit_actions.push(serde_json::from_str::(line.as_str()).map_err(|e| { - DeltaTableError::InvalidJsonLog { - json_err: e, - version: winning_commit_version, - line, - } - })?); - } - let winning_commit_summary = - WinningCommitSummary::new(commit_actions, winning_commit_version); + actions: &'b Vec, + ) -> Result, DeltaTableError> { + let winning_commit_summary = WinningCommitSummary::try_new( + object_store.as_ref(), + snapshot.version(), + winning_commit_version, + ) + .await?; - let transaction_info = TransactionInfo::try_new(snapshot, &operation)?; + let transaction_info = TransactionInfo::try_new(snapshot, &operation, actions)?; Ok(Self { transaction_info, winning_commit_summary, winning_commit_version, - isolation_level: IsolationLevel::Serializable, snapshot, operation, }) @@ -349,8 +359,20 @@ impl<'a> ConflictChecker<'a> { fn check_for_added_files_that_should_have_been_read_by_current_txn( &self, ) -> Result<(), CommitConflictError> { + // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, + // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... + // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 + let isolation_level = if can_downgrade_to_snapshot_isolation( + &self.winning_commit_summary.actions, + &self.operation, + ) { + IsolationLevel::SnapshotIsolation + } else { + IsolationLevel::default_level() + }; + // Fail if new files have been added that the txn should have read. - let added_files_to_check = match self.isolation_level { + let added_files_to_check = match isolation_level { IsolationLevel::WriteSerializable if !self.current_transaction_info().metadata_changed() => { @@ -373,15 +395,13 @@ impl<'a> ConflictChecker<'a> { let added_files_matching_predicates = if let Some(predicate_str) = self.operation.read_predicate() { let arrow_schema = - self.transaction_info - .read_snapshot + self.snapshot .arrow_schema() .map_err(|err| CommitConflictError::CorruptedState { source: Box::new(err), })?; let predicate = self - .transaction_info - .read_snapshot + .snapshot .parse_predicate_expression(predicate_str) .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), @@ -491,6 +511,53 @@ impl<'a> ConflictChecker<'a> { } } +fn can_downgrade_to_snapshot_isolation<'a>( + actions: impl IntoIterator, + operation: &DeltaOperation, +) -> bool { + let mut data_changed = false; + let mut has_non_file_actions = false; + for action in actions { + match action { + Action::add(act) if act.data_change => data_changed = true, + Action::remove(rem) if rem.data_change => data_changed = true, + _ => has_non_file_actions = true, + } + } + + if has_non_file_actions { + // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation + // level to SnapshotIsolation. + return false; + } + + let default_isolation_level = IsolationLevel::default_level(); + // Note-1: For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can + // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict + // detection by skipping the + // [[ConflictChecker.checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn]] check i.e. + // don't worry about concurrent appends. + // Note-2: + // We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: + // t0 - Initial state of table + // t1 - Q1, Q2 starts + // t2 - Q1 commits + // t3 - Q2 is empty and wants to commit. + // In this scenario, we can always allow Q2 to commit without worrying about new files + // generated by Q1. + // The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 + // Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has + // a Metadata update (say schema change/identity column high watermark update), then Q2 can't + // be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, + // then Q1 should see the updates from Q2 - which actually didn't happen. + + match default_isolation_level { + IsolationLevel::Serializable => !data_changed, + IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), + _ => false, // This case should never happen + } +} + #[cfg(test)] mod tests { use super::super::test_utils::{ @@ -517,21 +584,21 @@ mod tests { partition_by: Default::default(), predicate: None, }; - + let actions = vec![Action::add(add)]; commit( - table.object_store().as_ref(), - 1, - vec![Action::add(add)], + table.object_store().clone(), + actions.clone(), operation.clone(), - Some(0), + &table.state, None, ) .await .unwrap(); - let checker = ConflictChecker::try_new(&table.state, table.object_store(), 1, operation) - .await - .unwrap(); + let checker = + ConflictChecker::try_new(&table.state, table.object_store(), 1, operation, &actions) + .await + .unwrap(); println!("actions: {:?}", checker.winning_commit_summary.actions); diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index cdd2922c01..eb773ba44c 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -1,8 +1,10 @@ //! Delta transactions use crate::action::{Action, CommitInfo, DeltaOperation}; -use crate::storage::{commit_uri_from_version, DeltaObjectStore}; +use crate::storage::{commit_uri_from_version, DeltaObjectStore, ObjectStoreRef}; +use crate::table_state::DeltaTableState; use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; use chrono::Utc; +use conflict_checker::ConflictChecker; use object_store::path::Path; use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::{Map, Value}; @@ -19,7 +21,7 @@ pub use types::*; const DELTA_LOG_FOLDER: &str = "_delta_log"; #[derive(thiserror::Error, Debug)] -enum TransactionError { +pub(crate) enum TransactionError { #[error("Tried committing existing table version: {0}")] VersionAlreadyExists(DeltaDataTypeVersion), @@ -67,10 +69,10 @@ fn log_entry_from_actions(actions: &[Action]) -> Result, + operation: &DeltaOperation, + actions: &mut Vec, app_metadata: Option>, ) -> Result { if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { @@ -126,26 +128,14 @@ async fn try_commit_transaction( } pub(crate) async fn commit( - storage: &DeltaObjectStore, - version: DeltaDataTypeVersion, - actions: Vec, + storage: ObjectStoreRef, + // version: DeltaDataTypeVersion, + mut actions: Vec, operation: DeltaOperation, - read_version: Option, + // read_version: Option, + read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> DeltaResult { - // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, - // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... - // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 - let isolation_level = if can_downgrade_to_snapshot_isolation(&actions, &operation) { - IsolationLevel::SnapshotIsolation - } else { - IsolationLevel::default_level() - }; - - // TODO: calculate isolation level to use when checking for conflicts. - // Leaving conflict checking unimplemented for now to get the "single writer" implementation off the ground. - // Leaving some commented code in place as a guidepost for the future. - // readPredicates.nonEmpty || readFiles.nonEmpty // TODO revise logic if files are read let depends_on_files = match operation { @@ -159,87 +149,65 @@ pub(crate) async fn commit( // TODO actually get the prop from commit infos... let only_add_files = false; - let is_blind_append = only_add_files && !depends_on_files; - - let _commit_info = CommitInfo { - version: None, - timestamp: Some(chrono::Utc::now().timestamp()), - read_version, - isolation_level: Some(isolation_level), - operation: Some(operation.name().to_string()), - operation_parameters: Some(operation.operation_parameters()?.collect()), - user_id: None, - user_name: None, - is_blind_append: Some(is_blind_append), - engine_info: Some(format!("Delta-RS/{}", crate_version())), - ..Default::default() - }; - - let tmp_commit = prepare_commit(storage, operation, actions, app_metadata).await?; - match try_commit_transaction(storage, &tmp_commit, version).await { - Ok(version) => Ok(version), - Err(TransactionError::VersionAlreadyExists(version)) => { - storage.delete(&tmp_commit).await?; - Err(DeltaTableError::VersionAlreadyExists(version)) - } - Err(err) => Err(err.into()), - } -} - -fn can_downgrade_to_snapshot_isolation<'a>( - actions: impl IntoIterator, - operation: &DeltaOperation, -) -> bool { - let mut data_changed = false; - let mut has_non_file_actions = false; - for action in actions { - match action { - Action::add(act) if act.data_change => data_changed = true, - Action::remove(rem) if rem.data_change => data_changed = true, - _ => has_non_file_actions = true, + let _is_blind_append = only_add_files && !depends_on_files; + + // let _commit_info = CommitInfo { + // version: None, + // timestamp: Some(chrono::Utc::now().timestamp()), + // read_version, + // isolation_level: Some(IsolationLevel::default_level()), + // operation: Some(operation.name().to_string()), + // operation_parameters: Some(operation.operation_parameters()?.collect()), + // user_id: None, + // user_name: None, + // is_blind_append: Some(is_blind_append), + // engine_info: Some(format!("Delta-RS/{}", crate_version())), + // ..Default::default() + // }; + + let tmp_commit = + prepare_commit(storage.as_ref(), &operation, &mut actions, app_metadata).await?; + + let max_attempts = 5; + let mut attempt_number = 1; + + while attempt_number <= max_attempts { + let version = read_snapshot.version() + attempt_number; + match try_commit_transaction(storage.as_ref(), &tmp_commit, version).await { + Ok(version) => return Ok(version), + Err(TransactionError::VersionAlreadyExists(version)) => { + let conflict_checker = ConflictChecker::try_new( + read_snapshot, + storage.clone(), + version, + operation.clone(), + &actions, + ) + .await?; + match conflict_checker.check_conflicts() { + Ok(_) => { + attempt_number += 1; + } + Err(_err) => { + storage.delete(&tmp_commit).await?; + return Err(DeltaTableError::VersionAlreadyExists(version)); + } + }; + } + Err(err) => { + storage.delete(&tmp_commit).await?; + return Err(err.into()); + } } } - if has_non_file_actions { - // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation - // level to SnapshotIsolation. - return false; - } - - let default_isolation_level = IsolationLevel::default_level(); - // Note-1: For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can - // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict - // detection by skipping the - // [[ConflictChecker.checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn]] check i.e. - // don't worry about concurrent appends. - // Note-2: - // We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: - // t0 - Initial state of table - // t1 - Q1, Q2 starts - // t2 - Q1 commits - // t3 - Q2 is empty and wants to commit. - // In this scenario, we can always allow Q2 to commit without worrying about new files - // generated by Q1. - // The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 - // Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has - // a Metadata update (say schema change/identity column high watermark update), then Q2 can't - // be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, - // then Q1 should see the updates from Q2 - which actually didn't happen. - - match default_isolation_level { - IsolationLevel::Serializable => !data_changed, - IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), - _ => false, // This case should never happen - } + // TODO max attempts error + Err(DeltaTableError::VersionAlreadyExists(-1)) } #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; - use crate::action::{DeltaOperation, Protocol, SaveMode}; - use crate::storage::utils::flatten_list_stream; - use crate::writer::test_utils::get_delta_metadata; - use crate::{DeltaTable, DeltaTableBuilder}; #[test] fn test_commit_version() { @@ -248,48 +216,4 @@ mod tests { let version = commit_uri_from_version(123); assert_eq!(version, Path::from("_delta_log/00000000000000000123.json")) } - - #[tokio::test] - async fn test_commits_writes_file() { - let metadata = get_delta_metadata(&[]); - let operation = DeltaOperation::Create { - mode: SaveMode::Append, - location: "memory://".into(), - protocol: Protocol { - min_reader_version: 1, - min_writer_version: 1, - }, - metadata, - }; - - let commit_path = Path::from("_delta_log/00000000000000000000.json"); - let storage = DeltaTableBuilder::from_uri("memory://") - .build_storage() - .unwrap(); - - // successfully write in clean location - commit(storage.as_ref(), 0, vec![], operation.clone(), None, None) - .await - .unwrap(); - let head = storage.head(&commit_path).await; - assert!(head.is_ok()); - assert_eq!(head.as_ref().unwrap().location, commit_path); - - // fail on overwriting - let failed_commit = commit(storage.as_ref(), 0, vec![], operation, None, None).await; - assert!(failed_commit.is_err()); - assert!(matches!( - failed_commit.unwrap_err(), - DeltaTableError::VersionAlreadyExists(_) - )); - - // check we clean up after ourselves - let objects = flatten_list_stream(storage.as_ref(), None).await.unwrap(); - assert_eq!(objects.len(), 1); - - // table can be loaded - let mut table = DeltaTable::new(storage, Default::default()); - table.load().await.unwrap(); - assert_eq!(table.version(), 0) - } } diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 183c2b4da6..9e5be64ceb 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -1,4 +1,4 @@ -use super::{commit, CommitInfo}; +use super::{prepare_commit, try_commit_transaction, CommitInfo}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::table_state::DeltaTableState; use crate::{ @@ -118,15 +118,12 @@ pub async fn create_initialized_table( configuration.unwrap_or_default(), ), }; - commit( - storage.as_ref(), - 0, - init_table_actions(), - operation, - None, - None, - ) - .await - .unwrap(); + let mut actions = init_table_actions(); + let prepared_commit = prepare_commit(storage.as_ref(), &operation, &mut actions, None) + .await + .unwrap(); + try_commit_transaction(storage.as_ref(), &prepared_commit, 0) + .await + .unwrap(); DeltaTable::new_with_state(storage, state) } diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index fdfce81d88..0b9f6c1410 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -433,11 +433,10 @@ impl std::future::IntoFuture for WriteBuilder { predicate: this.predicate, }; let _version = commit( - &table.storage, - table.version() + 1, + table.storage.clone(), actions, operation, - Some(table.version()), + &table.state, // TODO pass through metadata None, ) diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 9952c00825..0d768c3feb 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -318,7 +318,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { transaction.add_action(Action::remove(remove)); transaction.commit(None, None).await?; - let maybe_metrics = plan.execute(dt.object_store()).await; + let maybe_metrics = plan.execute(dt.object_store(), &dt.state).await; assert!(maybe_metrics.is_err()); assert_eq!(dt.version(), version + 1); Ok(()) @@ -367,7 +367,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { ) .await?; - let metrics = plan.execute(dt.object_store()).await?; + let metrics = plan.execute(dt.object_store(), &dt.state).await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); assert_eq!(dt.version(), version + 2); From 5ac369aebc70d923e13f76bf99da975a983240e7 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 22 Feb 2023 23:50:51 +0100 Subject: [PATCH 25/67] fix: clippy --- rust/src/operations/transaction/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index eb773ba44c..d9b12b0a0a 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -91,7 +91,7 @@ pub(crate) async fn prepare_commit( } // Serialize all actions that are part of this log entry. - let log_entry = bytes::Bytes::from(log_entry_from_actions(&actions)?); + let log_entry = bytes::Bytes::from(log_entry_from_actions(actions)?); // Write delta log entry as temporary file to storage. For the actual commit, // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. From a792dbeb04b4c913c2bb3bb26bdf04b9b8594cc0 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 23 Feb 2023 07:54:47 +0100 Subject: [PATCH 26/67] chore: simplify lifetimes --- .../transaction/conflict_checker.rs | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 71bba52faf..1b770c0d6a 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -80,7 +80,7 @@ pub enum CommitConflictError { } /// A struct representing different attributes of current transaction needed for conflict detection. -pub(crate) struct TransactionInfo<'a, 'b> { +pub(crate) struct TransactionInfo<'a> { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction pub(crate) read_predicates: Vec, @@ -91,18 +91,18 @@ pub(crate) struct TransactionInfo<'a, 'b> { /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit - pub(crate) actions: &'b Vec, + pub(crate) actions: &'a Vec, /// read [`DeltaTableState`] used for the transaction pub(crate) read_snapshot: &'a DeltaTableState, /// [`CommitInfo`] for the commit pub(crate) commit_info: Option, } -impl<'a, 'b> TransactionInfo<'a, 'b> { +impl<'a> TransactionInfo<'a> { pub fn try_new( snapshot: &'a DeltaTableState, operation: &DeltaOperation, - actions: &'b Vec, + actions: &'a Vec, ) -> Result { Ok(Self { txn_id: "".into(), @@ -265,9 +265,10 @@ impl WinningCommitSummary { } } -pub(crate) struct ConflictChecker<'a, 'b> { +/// Checks if a failed commit may be committed after a conflicting winning commit +pub(crate) struct ConflictChecker<'a> { /// transaction information for current transaction at start of check - transaction_info: TransactionInfo<'a, 'b>, + transaction_info: TransactionInfo<'a>, /// Version number of commit, that has been committed ahead of the current transaction winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction @@ -278,14 +279,14 @@ pub(crate) struct ConflictChecker<'a, 'b> { operation: DeltaOperation, } -impl<'a, 'b> ConflictChecker<'a, 'b> { +impl<'a> ConflictChecker<'a> { pub async fn try_new( snapshot: &'a DeltaTableState, object_store: ObjectStoreRef, winning_commit_version: DeltaDataTypeVersion, operation: DeltaOperation, - actions: &'b Vec, - ) -> Result, DeltaTableError> { + actions: &'a Vec, + ) -> Result, DeltaTableError> { let winning_commit_summary = WinningCommitSummary::try_new( object_store.as_ref(), snapshot.version(), From c1bd4eada6f38d03b3596f200e317141826eb608 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 23 Feb 2023 22:22:17 +0100 Subject: [PATCH 27/67] chore: flatten --- .../transaction/conflict_checker.rs | 2 +- rust/src/operations/transaction/state.rs | 35 ++++++++----------- 2 files changed, 15 insertions(+), 22 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 1b770c0d6a..01697beeef 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -408,7 +408,7 @@ impl<'a> ConflictChecker<'a> { source: Box::new(err), })?; AddContainer::new(&added_files_to_check, arrow_schema) - .predicate_matches(&[predicate]) + .predicate_matches(predicate) .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), })? diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 42bebb12dd..f40156ab22 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -100,28 +100,21 @@ impl<'a> AddContainer<'a> { /// Expressions are evaluated for file statistics, essentially column-wise min max bounds, /// so evalutaing expressions is inexact. However exluded files are guaranteed (for a correct log) /// to not contain matches by the predicate expression. - pub fn predicate_matches(&self, filters: &[Expr]) -> DeltaResult> { - if let Some(Some(predicate)) = - (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) - { - let pruning_predicate = PruningPredicate::try_new(predicate, self.schema.clone())?; - Ok(Either::Left( - self.inner - .iter() - .zip(pruning_predicate.prune(self)?.into_iter()) - .filter_map( - |(action, keep_file)| { - if keep_file { - Some(action) - } else { - None - } - }, - ), + pub fn predicate_matches(&self, predicate: Expr) -> DeltaResult> { + let pruning_predicate = PruningPredicate::try_new(predicate, self.schema.clone())?; + Ok(self + .inner + .iter() + .zip(pruning_predicate.prune(self)?.into_iter()) + .filter_map( + |(action, keep_file)| { + if keep_file { + Some(action) + } else { + None + } + }, )) - } else { - Ok(Either::Right(self.inner.iter())) - } } } From adbfc09030f1c48547b956677af95471e4134ef6 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 24 Feb 2023 00:08:51 +0100 Subject: [PATCH 28/67] chore: cleanup --- .../transaction/conflict_checker.rs | 29 +++++++++++++------ rust/src/operations/transaction/mod.rs | 18 +----------- 2 files changed, 21 insertions(+), 26 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 01697beeef..70c5ec4527 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -567,25 +567,36 @@ mod tests { use super::*; use crate::action::{Action, SaveMode}; use crate::operations::transaction::commit; + use serde_json::json; + + fn get_stats(min: i64, max: i64) -> Option { + let data = json!({ + "numRecords": 18, + "minValues": { + "value": min + }, + "maxValues": { + "value": max + }, + "nullCount": { + "value": 0 + } + }); + Some(data.to_string()) + } #[tokio::test] async fn test_append_only_commits() { let table = create_initialized_table(&[], None).await; - let commit_info = DeltaOperation::Write { - mode: SaveMode::Append, - partition_by: Default::default(), - predicate: None, - } - .get_commit_info(); - - let add = create_add_action("file-path", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":10},\"nullCount\":{\"value\":0}}")); + let add = create_add_action("file-path", true, get_stats(1, 10)); + let actions = vec![Action::add(add)]; let operation = DeltaOperation::Write { mode: SaveMode::Append, partition_by: Default::default(), predicate: None, }; - let actions = vec![Action::add(add)]; + commit( table.object_store().clone(), actions.clone(), diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index d9b12b0a0a..58dec2ec9b 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -103,7 +103,7 @@ pub(crate) async fn prepare_commit( Ok(path) } -/// Tries to commit a prepared commit file. Returns [`DeltaTableError::VersionAlreadyExists`] +/// Tries to commit a prepared commit file. Returns [DeltaTableError::VersionAlreadyExists] /// if the given `version` already exists. The caller should handle the retry logic itself. /// This is low-level transaction API. If user does not want to maintain the commit loop then /// the `DeltaTransaction.commit` is desired to be used as it handles `try_commit_transaction` @@ -129,10 +129,8 @@ async fn try_commit_transaction( pub(crate) async fn commit( storage: ObjectStoreRef, - // version: DeltaDataTypeVersion, mut actions: Vec, operation: DeltaOperation, - // read_version: Option, read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> DeltaResult { @@ -151,20 +149,6 @@ pub(crate) async fn commit( let only_add_files = false; let _is_blind_append = only_add_files && !depends_on_files; - // let _commit_info = CommitInfo { - // version: None, - // timestamp: Some(chrono::Utc::now().timestamp()), - // read_version, - // isolation_level: Some(IsolationLevel::default_level()), - // operation: Some(operation.name().to_string()), - // operation_parameters: Some(operation.operation_parameters()?.collect()), - // user_id: None, - // user_name: None, - // is_blind_append: Some(is_blind_append), - // engine_info: Some(format!("Delta-RS/{}", crate_version())), - // ..Default::default() - // }; - let tmp_commit = prepare_commit(storage.as_ref(), &operation, &mut actions, app_metadata).await?; From a638b50d8369c98898190c3142ed354af9d3d6d3 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 06:39:41 +0100 Subject: [PATCH 29/67] test: separate generating commit bytes --- rust/src/operations/transaction/mod.rs | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 58dec2ec9b..785fdddcc0 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -66,15 +66,11 @@ fn log_entry_from_actions(actions: &[Action]) -> Result, app_metadata: Option>, -) -> Result { +) -> Result { if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { let mut extra_info = Map::::new(); let mut commit_info = operation.get_commit_info(); @@ -91,7 +87,20 @@ pub(crate) async fn prepare_commit( } // Serialize all actions that are part of this log entry. - let log_entry = bytes::Bytes::from(log_entry_from_actions(actions)?); + Ok(bytes::Bytes::from(log_entry_from_actions(actions)?)) +} + +/// Low-level transaction API. Creates a temporary commit file. Once created, +/// the transaction object could be dropped and the actual commit could be executed +/// with `DeltaTable.try_commit_transaction`. +pub(crate) async fn prepare_commit( + storage: &DeltaObjectStore, + operation: &DeltaOperation, + actions: &mut Vec, + app_metadata: Option>, +) -> Result { + // Serialize all actions that are part of this log entry. + let log_entry = get_commit_bytes(operation, actions, app_metadata)?; // Write delta log entry as temporary file to storage. For the actual commit, // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. From 0562e17e298abe7a3928886064fe0885550947a0 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 06:55:11 +0100 Subject: [PATCH 30/67] test: try_commit_transaction --- rust/src/operations/transaction/mod.rs | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 785fdddcc0..7a462a67e9 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -1,6 +1,6 @@ //! Delta transactions use crate::action::{Action, CommitInfo, DeltaOperation}; -use crate::storage::{commit_uri_from_version, DeltaObjectStore, ObjectStoreRef}; +use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use crate::table_state::DeltaTableState; use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; use chrono::Utc; @@ -94,7 +94,7 @@ pub(crate) fn get_commit_bytes( /// the transaction object could be dropped and the actual commit could be executed /// with `DeltaTable.try_commit_transaction`. pub(crate) async fn prepare_commit( - storage: &DeltaObjectStore, + storage: &dyn ObjectStore, operation: &DeltaOperation, actions: &mut Vec, app_metadata: Option>, @@ -118,7 +118,7 @@ pub(crate) async fn prepare_commit( /// the `DeltaTransaction.commit` is desired to be used as it handles `try_commit_transaction` /// with retry logic. async fn try_commit_transaction( - storage: &DeltaObjectStore, + storage: &dyn ObjectStore, tmp_commit: &Path, version: DeltaDataTypeVersion, ) -> Result { @@ -201,6 +201,7 @@ pub(crate) async fn commit( #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; + use object_store::memory::InMemory; #[test] fn test_commit_version() { @@ -209,4 +210,21 @@ mod tests { let version = commit_uri_from_version(123); assert_eq!(version, Path::from("_delta_log/00000000000000000123.json")) } + + #[tokio::test] + async fn test_try_commit_transaction() { + let store = InMemory::new(); + let tmp_path = Path::from("_delta_log/tmp"); + let version_path = Path::from("_delta_log/00000000000000000000.json"); + store.put(&tmp_path, bytes::Bytes::new()).await.unwrap(); + store.put(&version_path, bytes::Bytes::new()).await.unwrap(); + + // fails if file version already exists + let res = try_commit_transaction(&store, &tmp_path, 0).await; + assert!(res.is_err()); + + // succeeds for next version + let res = try_commit_transaction(&store, &tmp_path, 1).await.unwrap(); + assert_eq!(res, 1); + } } From 2a2c34f1e934751dfe05f433120748d99474470a Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 07:19:14 +0100 Subject: [PATCH 31/67] test: add action serilaization test --- rust/src/operations/mod.rs | 1 - rust/src/operations/transaction/mod.rs | 12 +++++++- rust/src/operations/transaction/state.rs | 5 ++-- rust/src/operations/transaction/types.rs | 35 ++++++++++++++++++++++++ 4 files changed, 48 insertions(+), 5 deletions(-) diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 7839eef49d..7a554809a5 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -17,7 +17,6 @@ pub mod create; pub mod filesystem_check; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod optimize; -#[allow(missing_docs)] pub mod transaction; pub mod vacuum; diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 7a462a67e9..3da887f7e5 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -200,17 +200,27 @@ pub(crate) async fn commit( #[cfg(all(test, feature = "parquet"))] mod tests { + use self::test_utils::init_table_actions; use super::*; use object_store::memory::InMemory; #[test] - fn test_commit_version() { + fn test_commit_uri_from_version() { let version = commit_uri_from_version(0); assert_eq!(version, Path::from("_delta_log/00000000000000000000.json")); let version = commit_uri_from_version(123); assert_eq!(version, Path::from("_delta_log/00000000000000000123.json")) } + #[test] + fn test_log_entry_from_actions() { + let actions = init_table_actions(); + let entry = log_entry_from_actions(&actions).unwrap(); + let lines: Vec<_> = entry.lines().collect(); + // writes every action to a line + assert_eq!(actions.len(), lines.len()) + } + #[tokio::test] async fn test_try_commit_transaction() { let store = InMemory::new(); diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index f40156ab22..314e1fcb71 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -29,6 +29,7 @@ impl DeltaTableState { )) } + /// Iterate over all files in the log matching a predicate pub fn files_matching_predicate( &self, filters: &[Expr], @@ -266,9 +267,7 @@ mod tests { let expected = col("value") .gt(lit::(10)) .or(col("value").lt_eq(lit::(0))); - assert_eq!(parsed, expected); - - println!("{:?}", parsed) + assert_eq!(parsed, expected) } #[test] diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs index af8604c34d..ce5a8788a1 100644 --- a/rust/src/operations/transaction/types.rs +++ b/rust/src/operations/transaction/types.rs @@ -4,6 +4,7 @@ use serde::{Deserialize, Serialize}; use std::str::FromStr; #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] +/// The isolation level applied during transaction pub enum IsolationLevel { /// The strongest isolation level. It ensures that committed write operations /// and all reads are Serializable. Operations are allowed as long as there @@ -18,6 +19,7 @@ pub enum IsolationLevel { /// it provides great balance of data consistency and availability for most common operations. WriteSerializable, + /// SnapshotIsolation SnapshotIsolation, } @@ -53,3 +55,36 @@ impl FromStr for IsolationLevel { } } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_roundtrip_isolation_level() { + assert!(matches!( + "Serializable".parse().unwrap(), + IsolationLevel::Serializable + )); + assert!(matches!( + "WriteSerializable".parse().unwrap(), + IsolationLevel::WriteSerializable + )); + assert!(matches!( + "SnapshotIsolation".parse().unwrap(), + IsolationLevel::SnapshotIsolation + )); + assert!(matches!( + IsolationLevel::Serializable.as_ref().parse().unwrap(), + IsolationLevel::Serializable + )); + assert!(matches!( + IsolationLevel::WriteSerializable.as_ref().parse().unwrap(), + IsolationLevel::WriteSerializable + )); + assert!(matches!( + IsolationLevel::SnapshotIsolation.as_ref().parse().unwrap(), + IsolationLevel::SnapshotIsolation + )) + } +} From 0c1e5af78773718e9d62bace0b999f550a2675b0 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 17:55:24 +0100 Subject: [PATCH 32/67] feat: improve table config access --- rust/src/delta_config.rs | 212 +++++++++++++++++- .../transaction/conflict_checker.rs | 71 ++---- rust/src/operations/transaction/types.rs | 80 ++++++- rust/src/table_state.rs | 9 +- 4 files changed, 310 insertions(+), 62 deletions(-) diff --git a/rust/src/delta_config.rs b/rust/src/delta_config.rs index 4540dc587e..bc5de76a06 100644 --- a/rust/src/delta_config.rs +++ b/rust/src/delta_config.rs @@ -1,8 +1,11 @@ //! Delta Table configuration +use std::time::Duration; +use std::{collections::HashMap, str::FromStr}; -use crate::{DeltaDataTypeInt, DeltaDataTypeLong, DeltaTableMetaData}; use lazy_static::lazy_static; -use std::time::Duration; + +use crate::operations::transaction::IsolationLevel; +use crate::{DeltaDataTypeInt, DeltaDataTypeLong, DeltaTableError, DeltaTableMetaData}; lazy_static! { /// How often to checkpoint the delta log. @@ -28,6 +31,149 @@ lazy_static! { pub static ref ENABLE_EXPIRED_LOG_CLEANUP: DeltaConfig = DeltaConfig::new("enableExpiredLogCleanup", "true"); } +/// Typed property keys that can be defined on a delta table +/// +/// +enum DeltaConfigKey { + /// true for this Delta table to be append-only. If append-only, + /// existing records cannot be deleted, and existing values cannot be updated. + AppendOnly, + + /// true for Delta Lake to automatically optimize the layout of the files for this Delta table. + AutoOptimizeAutoCompact, + + /// true for Delta Lake to automatically optimize the layout of the files for this Delta table during writes. + AutoOptimizeOptimizeWrite, + + /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. + CheckpointWriteStatsAsJson, + + /// true for Delta Lake to write file statistics to checkpoints in struct format for the + /// stats_parsed column and to write partition values as a struct for partitionValues_parsed. + CheckpointWriteStatsAsStruct, + + /// Whether column mapping is enabled for Delta table columns and the corresponding + /// Parquet columns that use different names. + ColumnMappingMode, + + /// The number of columns for Delta Lake to collect statistics about for data skipping. + /// A value of -1 means to collect statistics for all columns. Updating this property does + /// not automatically collect statistics again; instead, it redefines the statistics schema + /// of the Delta table. Specifically, it changes the behavior of future statistics collection + /// (such as during appends and optimizations) as well as data skipping (such as ignoring column + /// statistics beyond this number, even when such statistics exist). + DataSkippingNumIndexedCols, + + /// The shortest duration for Delta Lake to keep logically deleted data files before deleting + /// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. + /// + /// This value should be large enough to ensure that: + /// + /// * It is larger than the longest possible duration of a job if you run VACUUM when there are + /// concurrent readers or writers accessing the Delta table. + /// * If you run a streaming query that reads from the table, that query does not stop for longer + /// than this value. Otherwise, the query may not be able to restart, as it must still read old files. + DeletedFileRetentionDuration, + + /// true to enable change data feed. + EnableChangeDataFeed, + + /// The degree to which a transaction must be isolated from modifications made by concurrent transactions. + /// + /// Valid values are `Serializable` and `WriteSerializable`. + IsolationLevel, + + /// How long the history for a Delta table is kept. + /// + /// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older + /// than the retention interval. If you set this property to a large enough value, many log + /// entries are retained. This should not impact performance as operations against the log are + /// constant time. Operations on history are parallel but will become more expensive as the log size increases. + LogRetentionDuration, + + /// The minimum required protocol reader version for a reader that allows to read from this Delta table. + MinReaderVersion, + + /// The minimum required protocol writer version for a writer that allows to write to this Delta table. + MinWriterVersion, + + /// true for Delta Lake to generate a random prefix for a file path instead of partition information. + /// + /// For example, this ma + /// y improve Amazon S3 performance when Delta Lake needs to send very high volumes + /// of Amazon S3 calls to better partition across S3 servers. + RandomizeFilePrefixes, + + /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta Lake generates for random prefixes. + RandomPrefixLength, + + /// The shortest duration within which new snapshots will retain transaction identifiers (for example, SetTransactions). + /// When a new snapshot sees a transaction identifier older than or equal to the duration specified by this property, + /// the snapshot considers it expired and ignores it. The SetTransaction identifier is used when making the writes idempotent. + SetTransactionRetentionDuration, + + /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. + TargetFileSize, + + /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. + TuneFileSizesForRewrites, +} + +impl AsRef for DeltaConfigKey { + fn as_ref(&self) -> &str { + match self { + Self::AppendOnly => "delta.appendOnly", + Self::AutoOptimizeAutoCompact => "delta.autoOptimize.autoCompact", + Self::AutoOptimizeOptimizeWrite => "delta.autoOptimize.optimizeWrite", + Self::CheckpointWriteStatsAsJson => "delta.checkpoint.writeStatsAsJson", + Self::CheckpointWriteStatsAsStruct => "delta.checkpoint.writeStatsAsStruct", + Self::ColumnMappingMode => "delta.columnMapping.mode", + Self::DataSkippingNumIndexedCols => "delta.dataSkippingNumIndexedCols", + Self::DeletedFileRetentionDuration => "delta.deletedFileRetentionDuration", + Self::EnableChangeDataFeed => "delta.enableChangeDataFeed", + Self::IsolationLevel => "delta.isolationLevel", + Self::LogRetentionDuration => "delta.logRetentionDuration", + Self::MinReaderVersion => "delta.minReaderVersion", + Self::MinWriterVersion => "delta.minWriterVersion", + Self::RandomizeFilePrefixes => "delta.randomizeFilePrefixes", + Self::RandomPrefixLength => "delta.randomPrefixLength", + Self::SetTransactionRetentionDuration => "delta.setTransactionRetentionDuration", + Self::TargetFileSize => "delta.targetFileSize", + Self::TuneFileSizesForRewrites => "delta.tuneFileSizesForRewrites", + } + } +} + +impl FromStr for DeltaConfigKey { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s { + "delta.appendOnly" => Ok(Self::AppendOnly), + "delta.autoOptimize.autoCompact" => Ok(Self::AutoOptimizeAutoCompact), + "delta.autoOptimize.optimizeWrite" => Ok(Self::AutoOptimizeOptimizeWrite), + "delta.checkpoint.writeStatsAsJson" => Ok(Self::CheckpointWriteStatsAsJson), + "delta.checkpoint.writeStatsAsStruct" => Ok(Self::CheckpointWriteStatsAsStruct), + "delta.columnMapping.mode" => Ok(Self::ColumnMappingMode), + "delta.dataSkippingNumIndexedCols" => Ok(Self::DataSkippingNumIndexedCols), + "delta.deletedFileRetentionDuration" | "deletedFileRetentionDuration" => { + Ok(Self::DeletedFileRetentionDuration) + } + "delta.enableChangeDataFeed" => Ok(Self::EnableChangeDataFeed), + "delta.isolationLevel" => Ok(Self::IsolationLevel), + "delta.logRetentionDuration" | "logRetentionDuration" => Ok(Self::LogRetentionDuration), + "delta.minReaderVersion" => Ok(Self::MinReaderVersion), + "delta.minWriterVersion" => Ok(Self::MinWriterVersion), + "delta.randomizeFilePrefixes" => Ok(Self::RandomizeFilePrefixes), + "delta.randomPrefixLength" => Ok(Self::RandomPrefixLength), + "delta.setTransactionRetentionDuration" => Ok(Self::SetTransactionRetentionDuration), + "delta.targetFileSize" => Ok(Self::TargetFileSize), + "delta.tuneFileSizesForRewrites" => Ok(Self::TuneFileSizesForRewrites), + _ => Err(DeltaTableError::Generic("unknown config key".into())), + } + } +} + /// Delta configuration error #[derive(thiserror::Error, Debug, PartialEq, Eq)] pub enum DeltaConfigError { @@ -36,6 +182,68 @@ pub enum DeltaConfigError { Validation(String), } +macro_rules! table_config { + ($(($key:expr, $name:ident, $ret:ty, $default:literal),)*) => { + $( + /// read property $key + pub fn $name(&self) -> $ret { + self.0 + .get($key.as_ref()) + .and_then(|opt| opt.as_ref().and_then(|value| value.parse().ok())) + .unwrap_or($default) + } + )* + } +} + +/// Well known delta table configuration +pub struct TableConfig<'a>(pub(crate) &'a HashMap>); + +impl<'a> TableConfig<'a> { + table_config!( + (DeltaConfigKey::AppendOnly, append_only, bool, false), + ( + DeltaConfigKey::CheckpointWriteStatsAsJson, + write_stats_as_json, + bool, + true + ), + ( + DeltaConfigKey::CheckpointWriteStatsAsStruct, + write_stats_as_struct, + bool, + true + ), + ( + DeltaConfigKey::TargetFileSize, + target_file_size, + i64, + // Databricks / spark defaults to 104857600 (bytes) or 100mb + 104857600 + ), + ( + DeltaConfigKey::EnableChangeDataFeed, + enable_change_data_feed, + bool, + false + ), + ( + DeltaConfigKey::DataSkippingNumIndexedCols, + num_indexed_cols, + i32, + 32 + ), + ); + + /// Get the configured or default isolation level + pub fn isolation_level(&self) -> IsolationLevel { + self.0 + .get(DeltaConfigKey::IsolationLevel.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) + .unwrap_or_default() + } +} + /// Delta table's `metadata.configuration` entry. #[derive(Debug)] pub struct DeltaConfig { diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 70c5ec4527..605818df41 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,6 +1,12 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. #![allow(unused)] -use super::{CommitInfo, IsolationLevel}; +use std::collections::HashSet; +use std::io::{BufRead, BufReader, Cursor}; + +use object_store::ObjectStore; +use serde_json::{Map, Value}; + +use super::{can_downgrade_to_snapshot_isolation, CommitInfo, IsolationLevel}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::operations::transaction::TransactionError; use crate::storage::{commit_uri_from_version, DeltaObjectStore, ObjectStoreRef}; @@ -8,10 +14,6 @@ use crate::{ table_state::DeltaTableState, DeltaDataTypeVersion, DeltaResult, DeltaTable, DeltaTableError, DeltaTableMetaData, }; -use object_store::ObjectStore; -use serde_json::{Map, Value}; -use std::collections::HashSet; -use std::io::{BufRead, BufReader, Cursor}; #[cfg(feature = "datafusion")] use super::state::AddContainer; @@ -360,16 +362,20 @@ impl<'a> ConflictChecker<'a> { fn check_for_added_files_that_should_have_been_read_by_current_txn( &self, ) -> Result<(), CommitConflictError> { - // TODO(roeap) in the reference implementation this logic is implemented, which seems somewhat strange, - // as it seems we will never have "WriteSerializable" as level - probably need to check the table config ... - // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L964 + let defaault_isolation_level = self + .snapshot + .table_config() + .map(|config| config.isolation_level()) + .unwrap_or_default(); + let isolation_level = if can_downgrade_to_snapshot_isolation( &self.winning_commit_summary.actions, &self.operation, + &defaault_isolation_level, ) { IsolationLevel::SnapshotIsolation } else { - IsolationLevel::default_level() + defaault_isolation_level }; // Fail if new files have been added that the txn should have read. @@ -512,53 +518,6 @@ impl<'a> ConflictChecker<'a> { } } -fn can_downgrade_to_snapshot_isolation<'a>( - actions: impl IntoIterator, - operation: &DeltaOperation, -) -> bool { - let mut data_changed = false; - let mut has_non_file_actions = false; - for action in actions { - match action { - Action::add(act) if act.data_change => data_changed = true, - Action::remove(rem) if rem.data_change => data_changed = true, - _ => has_non_file_actions = true, - } - } - - if has_non_file_actions { - // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation - // level to SnapshotIsolation. - return false; - } - - let default_isolation_level = IsolationLevel::default_level(); - // Note-1: For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can - // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict - // detection by skipping the - // [[ConflictChecker.checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn]] check i.e. - // don't worry about concurrent appends. - // Note-2: - // We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: - // t0 - Initial state of table - // t1 - Q1, Q2 starts - // t2 - Q1 commits - // t3 - Q2 is empty and wants to commit. - // In this scenario, we can always allow Q2 to commit without worrying about new files - // generated by Q1. - // The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 - // Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has - // a Metadata update (say schema change/identity column high watermark update), then Q2 can't - // be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, - // then Q1 should see the updates from Q2 - which actually didn't happen. - - match default_isolation_level { - IsolationLevel::Serializable => !data_changed, - IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), - _ => false, // This case should never happen - } -} - #[cfg(test)] mod tests { use super::super::test_utils::{ diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs index ce5a8788a1..0366ea5267 100644 --- a/rust/src/operations/transaction/types.rs +++ b/rust/src/operations/transaction/types.rs @@ -1,8 +1,11 @@ //! Types and structs used when commitind operations to a delta table -use crate::DeltaTableError; -use serde::{Deserialize, Serialize}; use std::str::FromStr; +use serde::{Deserialize, Serialize}; + +use crate::action::{Action, DeltaOperation}; +use crate::DeltaTableError; + #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] /// The isolation level applied during transaction pub enum IsolationLevel { @@ -19,7 +22,10 @@ pub enum IsolationLevel { /// it provides great balance of data consistency and availability for most common operations. WriteSerializable, - /// SnapshotIsolation + /// SnapshotIsolation is a guarantee that all reads made in a transaction will see a consistent + /// snapshot of the database (in practice it reads the last committed values that existed at the + /// time it started), and the transaction itself will successfully commit only if no updates + /// it has made conflict with any concurrent updates made since that snapshot. SnapshotIsolation, } @@ -31,6 +37,14 @@ impl IsolationLevel { } } +// Spark assumes Serializable as default isolation level +// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 +impl Default for IsolationLevel { + fn default() -> Self { + Self::Serializable + } +} + impl AsRef for IsolationLevel { fn as_ref(&self) -> &str { match self { @@ -56,6 +70,53 @@ impl FromStr for IsolationLevel { } } +// For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can +// change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict +// detection by skipping the +// [ConflictChecker::check_for_added_files_that_should_have_been_read_by_current_txn] check i.e. +// don't worry about concurrent appends. +// +// We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: +// t0 - Initial state of table +// t1 - Q1, Q2 starts +// t2 - Q1 commits +// t3 - Q2 is empty and wants to commit. +// In this scenario, we can always allow Q2 to commit without worrying about new files +// generated by Q1. +// +// The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 +// Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has +// a Metadata update (say schema change/identity column high watermark update), then Q2 can't +// be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, +// then Q1 should see the updates from Q2 - which actually didn't happen. +pub(super) fn can_downgrade_to_snapshot_isolation<'a>( + actions: impl IntoIterator, + operation: &DeltaOperation, + isolation_level: &IsolationLevel, +) -> bool { + let mut data_changed = false; + let mut has_non_file_actions = false; + for action in actions { + match action { + Action::add(act) if act.data_change => data_changed = true, + Action::remove(rem) if rem.data_change => data_changed = true, + _ => has_non_file_actions = true, + } + } + + if has_non_file_actions { + // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation + // level to SnapshotIsolation. + return false; + } + + match isolation_level { + IsolationLevel::Serializable => !data_changed, + IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), + IsolationLevel::SnapshotIsolation => true, + } +} + #[cfg(test)] mod tests { use super::*; @@ -87,4 +148,17 @@ mod tests { IsolationLevel::SnapshotIsolation )) } + + #[test] + fn test_default_isolation_level() { + assert!(matches!( + IsolationLevel::default(), + IsolationLevel::Serializable + )) + } + + #[test] + fn test_can_downgrade_to_snapshot_isolation() { + assert!(true) + } } diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 744cc6a571..d7562fac69 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -1,7 +1,7 @@ //! The module for delta table state. use crate::action::{self, Action, Add}; -use crate::delta_config; +use crate::delta_config::{self, TableConfig}; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; use crate::storage::commit_uri_from_version; @@ -234,6 +234,13 @@ impl DeltaTableState { self.current_metadata.as_ref().map(|m| &m.schema) } + /// Well known table configuration + pub fn table_config(&self) -> Option> { + self.current_metadata + .as_ref() + .map(|meta| TableConfig(&meta.configuration)) + } + /// Merges new state information into our state /// /// The DeltaTableState also carries the version information for the given state, From b8d40a7037455f983bf277f74cfca5c9abe4e815 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 18:40:06 +0100 Subject: [PATCH 33/67] chore: use table_config --- rust/src/delta_config.rs | 2 +- rust/src/lib.rs | 1 - rust/src/operations/create.rs | 6 +-- rust/src/operations/optimize.rs | 31 ++++----------- rust/src/table_properties.rs | 69 --------------------------------- 5 files changed, 12 insertions(+), 97 deletions(-) delete mode 100644 rust/src/table_properties.rs diff --git a/rust/src/delta_config.rs b/rust/src/delta_config.rs index bc5de76a06..07941e5827 100644 --- a/rust/src/delta_config.rs +++ b/rust/src/delta_config.rs @@ -34,7 +34,7 @@ lazy_static! { /// Typed property keys that can be defined on a delta table /// /// -enum DeltaConfigKey { +pub enum DeltaConfigKey { /// true for this Delta table to be append-only. If append-only, /// existing records cannot be deleted, and existing values cannot be updated. AppendOnly, diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 9aece98ff2..43a3277ae6 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -91,7 +91,6 @@ pub mod operations; pub mod partitions; pub mod schema; pub mod storage; -pub mod table_properties; pub mod table_state; pub mod time_utils; diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 975110dfbc..5c52d833ea 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -304,8 +304,8 @@ impl std::future::IntoFuture for CreateBuilder { #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; + use crate::delta_config::DeltaConfigKey; use crate::operations::DeltaOps; - use crate::table_properties::APPEND_ONLY; use crate::writer::test_utils::get_delta_schema; use tempdir::TempDir; @@ -389,14 +389,14 @@ mod tests { let table = CreateBuilder::new() .with_location("memory://") .with_columns(schema.get_fields().clone()) - .with_configuration_property(APPEND_ONLY, Some("true")) + .with_configuration_property(DeltaConfigKey::AppendOnly.as_ref(), Some("true")) .await .unwrap(); let append = table .get_metadata() .unwrap() .configuration - .get(APPEND_ONLY) + .get(DeltaConfigKey::AppendOnly.as_ref()) .unwrap() .as_ref() .unwrap() diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 8e53cd2035..eb48300786 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -26,14 +26,14 @@ use crate::storage::ObjectStoreRef; use crate::table_state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; use crate::writer::utils::PartitionPath; -use crate::{table_properties, DeltaDataTypeVersion}; +use crate::DeltaDataTypeVersion; use crate::{ DeltaDataTypeLong, DeltaResult, DeltaTable, DeltaTableError, ObjectMeta, PartitionFilter, }; use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use futures::future::BoxFuture; use futures::StreamExt; -use log::{debug, error}; +use log::debug; use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; use parquet::file::properties::WriterProperties; use serde::{Deserialize, Serialize}; @@ -384,26 +384,6 @@ impl MergePlan { } } -fn get_target_file_size(snapshot: &DeltaTableState) -> DeltaDataTypeLong { - let mut target_size = 268_435_456; - if let Some(meta) = snapshot.current_metadata() { - let config_str = meta.configuration.get(table_properties::TARGET_FILE_SIZE); - if let Some(s) = config_str { - if let Some(s) = s { - let r = s.parse::(); - if let Ok(size) = r { - target_size = size; - } else { - error!("Unable to parse value of 'delta.targetFileSize'. Using default value"); - } - } else { - error!("Check your configuration of 'delta.targetFileSize'. Using default value"); - } - } - } - target_size -} - /// Build a Plan on which files to merge together. See [OptimizeBuilder] pub fn create_merge_plan( snapshot: &DeltaTableState, @@ -411,7 +391,12 @@ pub fn create_merge_plan( target_size: Option, writer_properties: WriterProperties, ) -> Result { - let target_size = target_size.unwrap_or_else(|| get_target_file_size(snapshot)); + let target_size = target_size.unwrap_or_else(|| { + snapshot + .table_config() + .map(|conf| conf.target_file_size()) + .unwrap_or(268_435_456) + }); let mut candidates = HashMap::new(); let mut operations: HashMap = HashMap::new(); let mut metrics = Metrics::default(); diff --git a/rust/src/table_properties.rs b/rust/src/table_properties.rs deleted file mode 100644 index 1c69d423fd..0000000000 --- a/rust/src/table_properties.rs +++ /dev/null @@ -1,69 +0,0 @@ -//! properties defined on Delta Tables -//! - -/// true for this Delta table to be append-only. If append-only, -/// existing records cannot be deleted, and existing values cannot be updated. -pub const APPEND_ONLY: &str = "delta.appendOnly"; -/// true for Delta Lake to automatically optimize the layout of the files for this Delta table. -pub const AUTO_OPTIMIZE_AUTO_COMPACT: &str = "delta.autoOptimize.autoCompact"; -/// true for Delta Lake to automatically optimize the layout of the files for this Delta table during writes. -pub const AUTO_OPTIMIZE_OPTIMIZE_WRITE: &str = "delta.autoOptimize.optimizeWrite"; -/// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. -pub const CHECKPOINT_WRITE_STATS_AS_JSON: &str = "delta.checkpoint.writeStatsAsJson"; -/// true for Delta Lake to write file statistics to checkpoints in struct format for the -/// stats_parsed column and to write partition values as a struct for partitionValues_parsed. -pub const CHECKPOINT_WRITE_STATS_AS_STRUCT: &str = "delta.checkpoint.writeStatsAsStruct"; -/// Whether column mapping is enabled for Delta table columns and the corresponding Parquet columns that use different names. -pub const COLUMN_MAPPING_MODE: &str = "delta.columnMapping.mode"; -/// Whether column mapping is enabled for Delta table columns and the corresponding Parquet columns that use different names. -pub const COMPATIBILITY_SYMLINK_FORMAT_MANIFEST_ENABLED: &str = - "delta.compatibility.symlinkFormatManifest.enabled"; -/// The number of columns for Delta Lake to collect statistics about for data skipping. -/// A value of -1 means to collect statistics for all columns. Updating this property does -/// not automatically collect statistics again; instead, it redefines the statistics schema -/// of the Delta table. Specifically, it changes the behavior of future statistics collection -/// (such as during appends and optimizations) as well as data skipping (such as ignoring column -/// statistics beyond this number, even when such statistics exist). -pub const DATA_SKIPPING_NUM_INDEXED_COLS: &str = "delta.dataSkippingNumIndexedCols"; -/// The shortest duration for Delta Lake to keep logically deleted data files before deleting -/// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. -/// -/// This value should be large enough to ensure that: -/// -/// * It is larger than the longest possible duration of a job if you run VACUUM when there are -/// concurrent readers or writers accessing the Delta table. -/// * If you run a streaming query that reads from the table, that query does not stop for longer -/// than this value. Otherwise, the query may not be able to restart, as it must still read old files. -pub const DELETED_FILE_RETENTION_DURATION: &str = "delta.deletedFileRetentionDuration"; -/// true to enable change data feed. -pub const ENABLE_CHANGE_DATA_FEED: &str = "delta.enableChangeDataFeed"; -/// The degree to which a transaction must be isolated from modifications made by concurrent transactions. -/// -/// Valid values are `Serializable` and `WriteSerializable`. -pub const ISOLATION_LEVEL: &str = "delta.isolationLevel"; -/// How long the history for a Delta table is kept. -/// -/// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older -/// than the retention interval. If you set this property to a large enough value, many log -/// entries are retained. This should not impact performance as operations against the log are -/// constant time. Operations on history are parallel but will become more expensive as the log size increases. -pub const LOG_RETENTION_DURATION: &str = "delta.logRetentionDuration"; -/// The minimum required protocol reader version for a reader that allows to read from this Delta table. -pub const MIN_READER_VERSION: &str = "delta.minReaderVersion"; -/// The minimum required protocol writer version for a writer that allows to write to this Delta table. -pub const MIN_WRITER_VERSION: &str = "delta.minWriterVersion"; -/// true for Delta Lake to generate a random prefix for a file path instead of partition information. -/// -/// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high volumes -/// of Amazon S3 calls to better partition across S3 servers. -pub const RANDOMIZE_FILE_PREFIXES: &str = "delta.randomizeFilePrefixes"; -/// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta Lake generates for random prefixes. -pub const RANDOM_PREFIX_LENGTH: &str = "delta.randomPrefixLength"; -/// The shortest duration within which new snapshots will retain transaction identifiers (for example, SetTransactions). -/// When a new snapshot sees a transaction identifier older than or equal to the duration specified by this property, -/// the snapshot considers it expired and ignores it. The SetTransaction identifier is used when making the writes idempotent. -pub const SET_TRANSACTION_RETENTION_DURATION: &str = "delta.setTransactionRetentionDuration"; -/// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. -pub const TARGET_FILE_SIZE: &str = "delta.targetFileSize"; -/// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. -pub const TUNE_FILE_SIZES_FOR_REWRITES: &str = "delta.tuneFileSizesForRewrites"; From 1f0f27f908651ba2d15f19f8599cea02c3d98ba7 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 18:53:55 +0100 Subject: [PATCH 34/67] feat: assure table config --- rust/src/operations/optimize.rs | 7 +------ rust/src/operations/transaction/conflict_checker.rs | 6 +----- rust/src/table_state.rs | 7 ++++++- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index eb48300786..7d9f2be404 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -391,12 +391,7 @@ pub fn create_merge_plan( target_size: Option, writer_properties: WriterProperties, ) -> Result { - let target_size = target_size.unwrap_or_else(|| { - snapshot - .table_config() - .map(|conf| conf.target_file_size()) - .unwrap_or(268_435_456) - }); + let target_size = target_size.unwrap_or_else(|| snapshot.table_config().target_file_size()); let mut candidates = HashMap::new(); let mut operations: HashMap = HashMap::new(); let mut metrics = Metrics::default(); diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 605818df41..11e0343da9 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -362,11 +362,7 @@ impl<'a> ConflictChecker<'a> { fn check_for_added_files_that_should_have_been_read_by_current_txn( &self, ) -> Result<(), CommitConflictError> { - let defaault_isolation_level = self - .snapshot - .table_config() - .map(|config| config.isolation_level()) - .unwrap_or_default(); + let defaault_isolation_level = self.snapshot.table_config().isolation_level(); let isolation_level = if can_downgrade_to_snapshot_isolation( &self.winning_commit_summary.actions, diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index d7562fac69..8a7b24b056 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -11,6 +11,7 @@ use crate::{ DeltaTableMetaData, }; use chrono::Utc; +use lazy_static::lazy_static; use object_store::{path::Path, ObjectStore}; use serde::{Deserialize, Serialize}; use std::collections::HashMap; @@ -235,10 +236,14 @@ impl DeltaTableState { } /// Well known table configuration - pub fn table_config(&self) -> Option> { + pub fn table_config(&self) -> TableConfig<'_> { + lazy_static! { + static ref DUMMY_CONF: HashMap> = HashMap::new(); + } self.current_metadata .as_ref() .map(|meta| TableConfig(&meta.configuration)) + .unwrap_or_else(|| TableConfig(&DUMMY_CONF)) } /// Merges new state information into our state From 58b25aa723e2555adb70452200d0ae93df76b0a1 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 19:38:20 +0100 Subject: [PATCH 35/67] test: add sanity check for isolation level downgrade --- rust/src/action/mod.rs | 5 ++++- .../operations/transaction/conflict_checker.rs | 2 +- rust/src/operations/transaction/state.rs | 7 +++---- rust/src/operations/transaction/test_utils.rs | 8 ++++---- rust/src/operations/transaction/types.rs | 15 +++++++++++++-- 5 files changed, 25 insertions(+), 12 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index f37aa5c178..7a7308b298 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -604,7 +604,10 @@ impl DeltaOperation { /// Denotes if the operation changes the data contained in the table pub fn changes_data(&self) -> bool { - todo!() + match self { + Self::Optimize { .. } => false, + _ => true, + } } /// Retrieve basic commit information to be added to Delta commits diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 11e0343da9..dd93a98183 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -545,7 +545,7 @@ mod tests { let table = create_initialized_table(&[], None).await; let add = create_add_action("file-path", true, get_stats(1, 10)); - let actions = vec![Action::add(add)]; + let actions = vec![add]; let operation = DeltaOperation::Write { mode: SaveMode::Append, partition_by: Default::default(), diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 314e1fcb71..898dba23aa 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -243,7 +243,6 @@ impl ContextProvider for DummyContextProvider { #[cfg(test)] mod tests { use super::*; - use crate::action::Action; use crate::operations::transaction::test_utils::{create_add_action, init_table_actions}; use datafusion_expr::{col, lit}; @@ -273,9 +272,9 @@ mod tests { #[test] fn test_files_matching_predicate() { let mut actions = init_table_actions(); - actions.push(Action::add(create_add_action("excluded", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":10},\"nullCount\":{\"value\":0}}")))); - actions.push(Action::add(create_add_action("included-1", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":100},\"nullCount\":{\"value\":0}}")))); - actions.push(Action::add(create_add_action("included-2", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":-10},\"maxValues\":{\"value\":3},\"nullCount\":{\"value\":0}}")))); + actions.push(create_add_action("excluded", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":10},\"nullCount\":{\"value\":0}}".into()))); + actions.push(create_add_action("included-1", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":100},\"nullCount\":{\"value\":0}}".into()))); + actions.push(create_add_action("included-2", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":-10},\"maxValues\":{\"value\":3},\"nullCount\":{\"value\":0}}".into()))); let state = DeltaTableState::from_actions(actions, 0).unwrap(); let files = state diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 9e5be64ceb..0d1e65af38 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -9,15 +9,15 @@ use std::collections::HashMap; pub fn create_add_action( path: impl Into, data_change: bool, - stats: Option>, -) -> Add { - Add { + stats: Option, +) -> Action { + Action::add(Add { path: path.into(), size: 100, data_change, stats: stats.map(|s| s.into()), ..Default::default() - } + }) } pub fn init_table_actions() -> Vec { diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs index 0366ea5267..0cbba12a58 100644 --- a/rust/src/operations/transaction/types.rs +++ b/rust/src/operations/transaction/types.rs @@ -70,6 +70,9 @@ impl FromStr for IsolationLevel { } } +// implementation and comments adopted from +// https://github.com/delta-io/delta/blob/1c18c1d972e37d314711b3a485e6fb7c98fce96d/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1268 +// // For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can // change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict // detection by skipping the @@ -113,12 +116,13 @@ pub(super) fn can_downgrade_to_snapshot_isolation<'a>( match isolation_level { IsolationLevel::Serializable => !data_changed, IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), - IsolationLevel::SnapshotIsolation => true, + IsolationLevel::SnapshotIsolation => false, // this case should never happen, since spanpshot isolation canot be configured on table } } #[cfg(test)] mod tests { + use super::super::test_utils; use super::*; #[test] @@ -159,6 +163,13 @@ mod tests { #[test] fn test_can_downgrade_to_snapshot_isolation() { - assert!(true) + let isolation = IsolationLevel::WriteSerializable; + let operation = DeltaOperation::Optimize { + predicate: None, + target_size: 0, + }; + let add = test_utils::create_add_action("p", false, None); + let res = can_downgrade_to_snapshot_isolation(&[add], &operation, &isolation); + assert!(!res) } } From 10486c578595e46f3da0a84bdce95e161530b656 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 20:02:33 +0100 Subject: [PATCH 36/67] refactor: move io out of conflict checker --- .../transaction/conflict_checker.rs | 29 +++++++------------ rust/src/operations/transaction/mod.rs | 13 +++++---- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index dd93a98183..174c7dbfdd 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -271,8 +271,6 @@ impl WinningCommitSummary { pub(crate) struct ConflictChecker<'a> { /// transaction information for current transaction at start of check transaction_info: TransactionInfo<'a>, - /// Version number of commit, that has been committed ahead of the current transaction - winning_commit_version: DeltaDataTypeVersion, /// Summary of the transaction, that has been committed ahead of the current transaction winning_commit_summary: WinningCommitSummary, /// The state of the delta table at the base version from the current (not winning) commit @@ -284,24 +282,14 @@ pub(crate) struct ConflictChecker<'a> { impl<'a> ConflictChecker<'a> { pub async fn try_new( snapshot: &'a DeltaTableState, - object_store: ObjectStoreRef, - winning_commit_version: DeltaDataTypeVersion, + winning_commit_summary: WinningCommitSummary, operation: DeltaOperation, actions: &'a Vec, ) -> Result, DeltaTableError> { - let winning_commit_summary = WinningCommitSummary::try_new( - object_store.as_ref(), - snapshot.version(), - winning_commit_version, - ) - .await?; - let transaction_info = TransactionInfo::try_new(snapshot, &operation, actions)?; - Ok(Self { transaction_info, winning_commit_summary, - winning_commit_version, snapshot, operation, }) @@ -541,7 +529,8 @@ mod tests { } #[tokio::test] - async fn test_append_only_commits() { + // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 + async fn test_allowed_concurrent_actions() { let table = create_initialized_table(&[], None).await; let add = create_add_action("file-path", true, get_stats(1, 10)); @@ -562,10 +551,14 @@ mod tests { .await .unwrap(); - let checker = - ConflictChecker::try_new(&table.state, table.object_store(), 1, operation, &actions) - .await - .unwrap(); + let summary = WinningCommitSummary { + actions: vec![], + commit_info: None, + }; + + let checker = ConflictChecker::try_new(&table.state, summary, operation, &actions) + .await + .unwrap(); println!("actions: {:?}", checker.winning_commit_summary.actions); diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 3da887f7e5..f70b18e5ce 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -18,6 +18,8 @@ mod types; pub use types::*; +use self::conflict_checker::WinningCommitSummary; + const DELTA_LOG_FOLDER: &str = "_delta_log"; #[derive(thiserror::Error, Debug)] @@ -169,14 +171,15 @@ pub(crate) async fn commit( match try_commit_transaction(storage.as_ref(), &tmp_commit, version).await { Ok(version) => return Ok(version), Err(TransactionError::VersionAlreadyExists(version)) => { - let conflict_checker = ConflictChecker::try_new( - read_snapshot, - storage.clone(), + let summary = WinningCommitSummary::try_new( + storage.as_ref(), + read_snapshot.version(), version, - operation.clone(), - &actions, ) .await?; + let conflict_checker = + ConflictChecker::try_new(read_snapshot, summary, operation.clone(), &actions) + .await?; match conflict_checker.check_conflicts() { Ok(_) => { attempt_number += 1; From 24cc38a302ba356c0c74e879736614bab9e73160 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 2 Mar 2023 21:07:05 +0100 Subject: [PATCH 37/67] test: start scenario tests --- rust/src/action/mod.rs | 5 +- .../transaction/conflict_checker.rs | 53 ++++++++++--------- 2 files changed, 28 insertions(+), 30 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 7a7308b298..7da37d202a 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -604,10 +604,7 @@ impl DeltaOperation { /// Denotes if the operation changes the data contained in the table pub fn changes_data(&self) -> bool { - match self { - Self::Optimize { .. } => false, - _ => true, - } + !matches!(self, Self::Optimize { .. }) } /// Retrieve basic commit information to be added to Delta commits diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 174c7dbfdd..3ae89d9b6b 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -123,7 +123,9 @@ impl<'a> TransactionInfo<'a> { } pub fn metadata_changed(&self) -> bool { - todo!() + self.actions + .iter() + .any(|a| matches!(a, Action::metaData(_))) } pub fn final_actions_to_commit(&self) -> Vec { @@ -504,9 +506,8 @@ impl<'a> ConflictChecker<'a> { #[cfg(test)] mod tests { - use super::super::test_utils::{ - create_add_action, create_initialized_table, init_table_actions, - }; + use super::super::test_utils as tu; + use super::super::test_utils::{create_initialized_table, init_table_actions}; use super::*; use crate::action::{Action, SaveMode}; use crate::operations::transaction::commit; @@ -531,38 +532,38 @@ mod tests { #[tokio::test] // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 async fn test_allowed_concurrent_actions() { - let table = create_initialized_table(&[], None).await; + // check( + // "append / append", + // conflicts = false, + // reads = Seq( + // t => t.metadata + // ), + // concurrentWrites = Seq( + // addA), + // actions = Seq( + // addB)) + let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); + + let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); + let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - let add = create_add_action("file-path", true, get_stats(1, 10)); - let actions = vec![add]; + let summary = WinningCommitSummary { + actions: vec![file1], + commit_info: None, + }; let operation = DeltaOperation::Write { mode: SaveMode::Append, partition_by: Default::default(), predicate: None, }; - commit( - table.object_store().clone(), - actions.clone(), - operation.clone(), - &table.state, - None, - ) - .await - .unwrap(); - - let summary = WinningCommitSummary { - actions: vec![], - commit_info: None, - }; - - let checker = ConflictChecker::try_new(&table.state, summary, operation, &actions) + let actions = vec![file2]; + let checker = ConflictChecker::try_new(&state, summary, operation, &actions) .await .unwrap(); - println!("actions: {:?}", checker.winning_commit_summary.actions); - let result = checker.check_conflicts(); - println!("result: {:?}", result); + println!("{:?}", result); + assert!(result.is_ok()); } } From 1825a23bd52847a3dd3032b7d26bc340af2bdc0d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 3 Mar 2023 07:44:04 +0100 Subject: [PATCH 38/67] fix: load command for local tables --- rust/src/operations/load.rs | 31 ++++++++++++++++++++++++++++--- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs index ddcd14174a..6574bcedb3 100644 --- a/rust/src/operations/load.rs +++ b/rust/src/operations/load.rs @@ -2,7 +2,7 @@ use std::collections::HashMap; use std::sync::Arc; use crate::storage::DeltaObjectStore; -use crate::{DeltaResult, DeltaTable, DeltaTableError}; +use crate::{builder::ensure_table_uri, DeltaResult, DeltaTable}; use datafusion::datasource::TableProvider; use datafusion::execution::context::{SessionContext, TaskContext}; @@ -77,8 +77,7 @@ impl std::future::IntoFuture for LoadBuilder { Box::pin(async move { let object_store = this.object_store.unwrap(); - let url = url::Url::parse(&object_store.root_uri()) - .map_err(|_| DeltaTableError::InvalidTableLocation(object_store.root_uri()))?; + let url = ensure_table_uri(&object_store.root_uri())?; let store = object_store.storage_backend().clone(); let mut table = DeltaTable::new(object_store, Default::default()); table.load().await?; @@ -100,8 +99,34 @@ impl std::future::IntoFuture for LoadBuilder { mod tests { use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::writer::test_utils::{get_record_batch, TestResult}; + use crate::DeltaTableBuilder; use datafusion::assert_batches_sorted_eq; + #[tokio::test] + async fn test_load_local() -> TestResult { + let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + .load() + .await + .unwrap(); + + let (_table, stream) = DeltaOps(table).load().await?; + let data = collect_sendable_stream(stream).await?; + + let expected = vec![ + "+-------+", + "| value |", + "+-------+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 4 |", + "+-------+", + ]; + + assert_batches_sorted_eq!(&expected, &data); + Ok(()) + } + #[tokio::test] async fn test_write_load() -> TestResult { let batch = get_record_batch(None, false); From 7821ea2acdde04dde8fd542125e17a44881fe69a Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 3 Mar 2023 07:44:04 +0100 Subject: [PATCH 39/67] chore: clippy --- rust/src/operations/load.rs | 31 ++++++++++++++++++++++++++++--- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs index ddcd14174a..ea4c5e49dd 100644 --- a/rust/src/operations/load.rs +++ b/rust/src/operations/load.rs @@ -2,7 +2,7 @@ use std::collections::HashMap; use std::sync::Arc; use crate::storage::DeltaObjectStore; -use crate::{DeltaResult, DeltaTable, DeltaTableError}; +use crate::{builder::ensure_table_uri, DeltaResult, DeltaTable}; use datafusion::datasource::TableProvider; use datafusion::execution::context::{SessionContext, TaskContext}; @@ -77,8 +77,7 @@ impl std::future::IntoFuture for LoadBuilder { Box::pin(async move { let object_store = this.object_store.unwrap(); - let url = url::Url::parse(&object_store.root_uri()) - .map_err(|_| DeltaTableError::InvalidTableLocation(object_store.root_uri()))?; + let url = ensure_table_uri(object_store.root_uri())?; let store = object_store.storage_backend().clone(); let mut table = DeltaTable::new(object_store, Default::default()); table.load().await?; @@ -100,8 +99,34 @@ impl std::future::IntoFuture for LoadBuilder { mod tests { use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::writer::test_utils::{get_record_batch, TestResult}; + use crate::DeltaTableBuilder; use datafusion::assert_batches_sorted_eq; + #[tokio::test] + async fn test_load_local() -> TestResult { + let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + .load() + .await + .unwrap(); + + let (_table, stream) = DeltaOps(table).load().await?; + let data = collect_sendable_stream(stream).await?; + + let expected = vec![ + "+-------+", + "| value |", + "+-------+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 4 |", + "+-------+", + ]; + + assert_batches_sorted_eq!(&expected, &data); + Ok(()) + } + #[tokio::test] async fn test_write_load() -> TestResult { let batch = get_record_batch(None, false); From cad586535b5c1476db9c80f7b44da24d2d7ea2a4 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 3 Mar 2023 21:41:06 +0100 Subject: [PATCH 40/67] feat: extend configuration handling --- rust/src/delta_config.rs | 446 ++++++++++++++++++++++++-------- rust/src/lib.rs | 1 - rust/src/operations/create.rs | 21 +- rust/src/operations/optimize.rs | 26 +- rust/src/table_properties.rs | 69 ----- rust/src/table_state.rs | 34 ++- rust/tests/checkpoint_writer.rs | 12 +- 7 files changed, 380 insertions(+), 229 deletions(-) delete mode 100644 rust/src/table_properties.rs diff --git a/rust/src/delta_config.rs b/rust/src/delta_config.rs index 4540dc587e..df0a13b455 100644 --- a/rust/src/delta_config.rs +++ b/rust/src/delta_config.rs @@ -1,31 +1,165 @@ //! Delta Table configuration +use std::time::Duration; +use std::{collections::HashMap, str::FromStr}; -use crate::{DeltaDataTypeInt, DeltaDataTypeLong, DeltaTableMetaData}; use lazy_static::lazy_static; -use std::time::Duration; +use serde::{Deserialize, Serialize}; + +use crate::DeltaTableError; + +/// Typed property keys that can be defined on a delta table +/// +/// +pub enum DeltaConfigKey { + /// true for this Delta table to be append-only. If append-only, + /// existing records cannot be deleted, and existing values cannot be updated. + AppendOnly, + + /// true for Delta Lake to automatically optimize the layout of the files for this Delta table. + AutoOptimizeAutoCompact, + + /// true for Delta Lake to automatically optimize the layout of the files for this Delta table during writes. + AutoOptimizeOptimizeWrite, + + /// Interval (number of commits) after which a new checkpoint should be created + CheckpointInterval, + + /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. + CheckpointWriteStatsAsJson, + + /// true for Delta Lake to write file statistics to checkpoints in struct format for the + /// stats_parsed column and to write partition values as a struct for partitionValues_parsed. + CheckpointWriteStatsAsStruct, + + /// Whether column mapping is enabled for Delta table columns and the corresponding + /// Parquet columns that use different names. + ColumnMappingMode, + + /// The number of columns for Delta Lake to collect statistics about for data skipping. + /// A value of -1 means to collect statistics for all columns. Updating this property does + /// not automatically collect statistics again; instead, it redefines the statistics schema + /// of the Delta table. Specifically, it changes the behavior of future statistics collection + /// (such as during appends and optimizations) as well as data skipping (such as ignoring column + /// statistics beyond this number, even when such statistics exist). + DataSkippingNumIndexedCols, + + /// The shortest duration for Delta Lake to keep logically deleted data files before deleting + /// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. + /// + /// This value should be large enough to ensure that: + /// + /// * It is larger than the longest possible duration of a job if you run VACUUM when there are + /// concurrent readers or writers accessing the Delta table. + /// * If you run a streaming query that reads from the table, that query does not stop for longer + /// than this value. Otherwise, the query may not be able to restart, as it must still read old files. + DeletedFileRetentionDuration, + + /// true to enable change data feed. + EnableChangeDataFeed, + + /// The degree to which a transaction must be isolated from modifications made by concurrent transactions. + /// + /// Valid values are `Serializable` and `WriteSerializable`. + IsolationLevel, + + /// How long the history for a Delta table is kept. + /// + /// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older + /// than the retention interval. If you set this property to a large enough value, many log + /// entries are retained. This should not impact performance as operations against the log are + /// constant time. Operations on history are parallel but will become more expensive as the log size increases. + LogRetentionDuration, + + /// TODO I could not find this property in the documentation, but was defined here and makes sense..? + EnableExpiredLogCleanup, + + /// The minimum required protocol reader version for a reader that allows to read from this Delta table. + MinReaderVersion, + + /// The minimum required protocol writer version for a writer that allows to write to this Delta table. + MinWriterVersion, + + /// true for Delta Lake to generate a random prefix for a file path instead of partition information. + /// + /// For example, this ma + /// y improve Amazon S3 performance when Delta Lake needs to send very high volumes + /// of Amazon S3 calls to better partition across S3 servers. + RandomizeFilePrefixes, + + /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta Lake generates for random prefixes. + RandomPrefixLength, + + /// The shortest duration within which new snapshots will retain transaction identifiers (for example, SetTransactions). + /// When a new snapshot sees a transaction identifier older than or equal to the duration specified by this property, + /// the snapshot considers it expired and ignores it. The SetTransaction identifier is used when making the writes idempotent. + SetTransactionRetentionDuration, + + /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. + TargetFileSize, + + /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. + TuneFileSizesForRewrites, +} + +impl AsRef for DeltaConfigKey { + fn as_ref(&self) -> &str { + match self { + Self::AppendOnly => "delta.appendOnly", + Self::CheckpointInterval => "delta.checkpointInterval", + Self::AutoOptimizeAutoCompact => "delta.autoOptimize.autoCompact", + Self::AutoOptimizeOptimizeWrite => "delta.autoOptimize.optimizeWrite", + Self::CheckpointWriteStatsAsJson => "delta.checkpoint.writeStatsAsJson", + Self::CheckpointWriteStatsAsStruct => "delta.checkpoint.writeStatsAsStruct", + Self::ColumnMappingMode => "delta.columnMapping.mode", + Self::DataSkippingNumIndexedCols => "delta.dataSkippingNumIndexedCols", + Self::DeletedFileRetentionDuration => "delta.deletedFileRetentionDuration", + Self::EnableChangeDataFeed => "delta.enableChangeDataFeed", + Self::IsolationLevel => "delta.isolationLevel", + Self::LogRetentionDuration => "delta.logRetentionDuration", + Self::EnableExpiredLogCleanup => "delta.enableExpiredLogCleanup", + Self::MinReaderVersion => "delta.minReaderVersion", + Self::MinWriterVersion => "delta.minWriterVersion", + Self::RandomizeFilePrefixes => "delta.randomizeFilePrefixes", + Self::RandomPrefixLength => "delta.randomPrefixLength", + Self::SetTransactionRetentionDuration => "delta.setTransactionRetentionDuration", + Self::TargetFileSize => "delta.targetFileSize", + Self::TuneFileSizesForRewrites => "delta.tuneFileSizesForRewrites", + } + } +} -lazy_static! { - /// How often to checkpoint the delta log. - pub static ref CHECKPOINT_INTERVAL: DeltaConfig = DeltaConfig::new("checkpointInterval", "10"); - - /// The shortest duration we have to keep logically deleted data files around before deleting - /// them physically. - /// Note: this value should be large enough: - /// - It should be larger than the longest possible duration of a job if you decide to run "VACUUM" - /// when there are concurrent readers or writers accessing the table. - ///- If you are running a streaming query reading from the table, you should make sure the query - /// doesn't stop longer than this value. Otherwise, the query may not be able to restart as it - /// still needs to read old files. - pub static ref TOMBSTONE_RETENTION: DeltaConfig = - DeltaConfig::new("deletedFileRetentionDuration", "interval 1 week"); - - /// The shortest duration we have to keep delta files around before deleting them. We can only - /// delete delta files that are before a compaction. We may keep files beyond this duration until - /// the next calendar day. - pub static ref LOG_RETENTION: DeltaConfig = DeltaConfig::new("logRetentionDuration", "interval 30 day"); - - /// Whether to clean up expired checkpoints and delta logs. - pub static ref ENABLE_EXPIRED_LOG_CLEANUP: DeltaConfig = DeltaConfig::new("enableExpiredLogCleanup", "true"); +impl FromStr for DeltaConfigKey { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s { + "delta.appendOnly" => Ok(Self::AppendOnly), + "delta.checkpointInterval" => Ok(Self::CheckpointInterval), + "delta.autoOptimize.autoCompact" => Ok(Self::AutoOptimizeAutoCompact), + "delta.autoOptimize.optimizeWrite" => Ok(Self::AutoOptimizeOptimizeWrite), + "delta.checkpoint.writeStatsAsJson" => Ok(Self::CheckpointWriteStatsAsJson), + "delta.checkpoint.writeStatsAsStruct" => Ok(Self::CheckpointWriteStatsAsStruct), + "delta.columnMapping.mode" => Ok(Self::ColumnMappingMode), + "delta.dataSkippingNumIndexedCols" => Ok(Self::DataSkippingNumIndexedCols), + "delta.deletedFileRetentionDuration" | "deletedFileRetentionDuration" => { + Ok(Self::DeletedFileRetentionDuration) + } + "delta.enableChangeDataFeed" => Ok(Self::EnableChangeDataFeed), + "delta.isolationLevel" => Ok(Self::IsolationLevel), + "delta.logRetentionDuration" | "logRetentionDuration" => Ok(Self::LogRetentionDuration), + "delta.enableExpiredLogCleanup" | "enableExpiredLogCleanup" => { + Ok(Self::EnableExpiredLogCleanup) + } + "delta.minReaderVersion" => Ok(Self::MinReaderVersion), + "delta.minWriterVersion" => Ok(Self::MinWriterVersion), + "delta.randomizeFilePrefixes" => Ok(Self::RandomizeFilePrefixes), + "delta.randomPrefixLength" => Ok(Self::RandomPrefixLength), + "delta.setTransactionRetentionDuration" => Ok(Self::SetTransactionRetentionDuration), + "delta.targetFileSize" => Ok(Self::TargetFileSize), + "delta.tuneFileSizesForRewrites" => Ok(Self::TuneFileSizesForRewrites), + _ => Err(DeltaTableError::Generic("unknown config key".into())), + } + } } /// Delta configuration error @@ -36,70 +170,170 @@ pub enum DeltaConfigError { Validation(String), } -/// Delta table's `metadata.configuration` entry. -#[derive(Debug)] -pub struct DeltaConfig { - /// The configuration name - pub key: String, - /// The default value if `key` is not set in `metadata.configuration`. - pub default: String, +macro_rules! table_config { + ($(($key:expr, $name:ident, $ret:ty, $default:literal),)*) => { + $( + /// read property $key + pub fn $name(&self) -> $ret { + self.0 + .get($key.as_ref()) + .and_then(|opt| opt.as_ref().and_then(|value| value.parse().ok())) + .unwrap_or($default) + } + )* + } } -impl DeltaConfig { - fn new(key: &str, default: &str) -> Self { - Self { - key: key.to_string(), - default: default.to_string(), +/// Well known delta table configuration +pub struct TableConfig<'a>(pub(crate) &'a HashMap>); + +impl<'a> TableConfig<'a> { + table_config!( + (DeltaConfigKey::AppendOnly, append_only, bool, false), + ( + DeltaConfigKey::CheckpointWriteStatsAsJson, + write_stats_as_json, + bool, + true + ), + ( + DeltaConfigKey::CheckpointWriteStatsAsStruct, + write_stats_as_struct, + bool, + true + ), + ( + DeltaConfigKey::TargetFileSize, + target_file_size, + i64, + // Databricks / spark defaults to 104857600 (bytes) or 100mb + 104857600 + ), + ( + DeltaConfigKey::EnableChangeDataFeed, + enable_change_data_feed, + bool, + false + ), + ( + DeltaConfigKey::DataSkippingNumIndexedCols, + num_indexed_cols, + i32, + 32 + ), + ( + DeltaConfigKey::EnableExpiredLogCleanup, + enable_expired_log_cleanup, + bool, + true + ), + ( + DeltaConfigKey::CheckpointInterval, + checkpoint_interval, + i32, + 10 + ), + ); + + /// The shortest duration for Delta Lake to keep logically deleted data files before deleting + /// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. + /// + /// This value should be large enough to ensure that: + /// + /// * It is larger than the longest possible duration of a job if you run VACUUM when there are + /// concurrent readers or writers accessing the Delta table. + /// * If you run a streaming query that reads from the table, that query does not stop for longer + /// than this value. Otherwise, the query may not be able to restart, as it must still read old files. + pub fn deleted_file_retention_duration(&self) -> Duration { + lazy_static! { + static ref DEFAULT_DURATION: Duration = parse_interval("interval 1 week").unwrap(); } + self.0 + .get(DeltaConfigKey::DeletedFileRetentionDuration.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| parse_interval(v).ok())) + .unwrap_or_else(|| DEFAULT_DURATION.to_owned()) } - /// Returns the value from `metadata.configuration` for `self.key` as DeltaDataTypeInt. - /// If it's missing in metadata then the `self.default` is used. - #[allow(dead_code)] - pub fn get_int_from_metadata( - &self, - metadata: &DeltaTableMetaData, - ) -> Result { - Ok(parse_int(&self.get_raw_from_metadata(metadata))? as i32) + /// How long the history for a Delta table is kept. + /// + /// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older + /// than the retention interval. If you set this property to a large enough value, many log + /// entries are retained. This should not impact performance as operations against the log are + /// constant time. Operations on history are parallel but will become more expensive as the log size increases. + pub fn log_retention_duration(&self) -> Duration { + lazy_static! { + static ref DEFAULT_DURATION: Duration = parse_interval("interval 30 day").unwrap(); + } + self.0 + .get(DeltaConfigKey::LogRetentionDuration.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| parse_interval(v).ok())) + .unwrap_or_else(|| DEFAULT_DURATION.to_owned()) } - /// Returns the value from `metadata.configuration` for `self.key` as DeltaDataTypeLong. - /// If it's missing in metadata then the `self.default` is used. - #[allow(dead_code)] - pub fn get_long_from_metadata( - &self, - metadata: &DeltaTableMetaData, - ) -> Result { - parse_int(&self.get_raw_from_metadata(metadata)) + /// The degree to which a transaction must be isolated from modifications made by concurrent transactions. + /// + /// Valid values are `Serializable` and `WriteSerializable`. + pub fn isolation_level(&self) -> IsolationLevel { + self.0 + .get(DeltaConfigKey::IsolationLevel.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) + .unwrap_or_default() } +} - /// Returns the value from `metadata.configuration` for `self.key` as Duration type for the interval. - /// The string value of this config has to have the following format: interval . - /// Where is either week, day, hour, second, millisecond, microsecond or nanosecond. - /// If it's missing in metadata then the `self.default` is used. - pub fn get_interval_from_metadata( - &self, - metadata: &DeltaTableMetaData, - ) -> Result { - parse_interval(&self.get_raw_from_metadata(metadata)) +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] +/// The isolation level applied during transaction +pub enum IsolationLevel { + /// The strongest isolation level. It ensures that committed write operations + /// and all reads are Serializable. Operations are allowed as long as there + /// exists a serial sequence of executing them one-at-a-time that generates + /// the same outcome as that seen in the table. For the write operations, + /// the serial sequence is exactly the same as that seen in the table’s history. + Serializable, + + /// A weaker isolation level than Serializable. It ensures only that the write + /// operations (that is, not reads) are serializable. However, this is still stronger + /// than Snapshot isolation. WriteSerializable is the default isolation level because + /// it provides great balance of data consistency and availability for most common operations. + WriteSerializable, + + /// SnapshotIsolation is a guarantee that all reads made in a transaction will see a consistent + /// snapshot of the database (in practice it reads the last committed values that existed at the + /// time it started), and the transaction itself will successfully commit only if no updates + /// it has made conflict with any concurrent updates made since that snapshot. + SnapshotIsolation, +} + +// Spark assumes Serializable as default isolation level +// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 +impl Default for IsolationLevel { + fn default() -> Self { + Self::Serializable } +} - /// Returns the value from `metadata.configuration` for `self.key` as bool. - /// If it's missing in metadata then the `self.default` is used. - pub fn get_boolean_from_metadata( - &self, - metadata: &DeltaTableMetaData, - ) -> Result { - parse_bool(&self.get_raw_from_metadata(metadata)) +impl AsRef for IsolationLevel { + fn as_ref(&self) -> &str { + match self { + Self::Serializable => "Serializable", + Self::WriteSerializable => "WriteSerializable", + Self::SnapshotIsolation => "SnapshotIsolation", + } } +} - fn get_raw_from_metadata(&self, metadata: &DeltaTableMetaData) -> String { - metadata - .configuration - .get(&self.key) - .and_then(|opt| opt.as_deref()) - .unwrap_or(self.default.as_str()) - .to_string() +impl FromStr for IsolationLevel { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "serializable" => Ok(Self::Serializable), + "writeserializable" | "write_serializable" => Ok(Self::WriteSerializable), + "snapshotisolation" | "snapshot_isolation" => Ok(Self::SnapshotIsolation), + _ => Err(DeltaTableError::Generic( + "Invalid string for IsolationLevel".into(), + )), + } } } @@ -149,15 +383,10 @@ fn parse_int(value: &str) -> Result { }) } -fn parse_bool(value: &str) -> Result { - value - .parse() - .map_err(|e| DeltaConfigError::Validation(format!("Cannot parse '{value}' as bool: {e}"))) -} - #[cfg(test)] mod tests { use super::*; + use crate::DeltaTableMetaData; use crate::Schema; use std::collections::HashMap; @@ -168,68 +397,55 @@ mod tests { #[test] fn get_interval_from_metadata_test() { - let mut md = dummy_metadata(); + let md = dummy_metadata(); + let config = TableConfig(&md.configuration); // default 1 week assert_eq!( - TOMBSTONE_RETENTION - .get_interval_from_metadata(&md) - .unwrap() - .as_secs(), + config.deleted_file_retention_duration().as_secs(), SECONDS_PER_WEEK, ); // change to 2 day + let mut md = dummy_metadata(); md.configuration.insert( - TOMBSTONE_RETENTION.key.to_string(), + DeltaConfigKey::DeletedFileRetentionDuration + .as_ref() + .to_string(), Some("interval 2 day".to_string()), ); + let config = TableConfig(&md.configuration); + assert_eq!( - TOMBSTONE_RETENTION - .get_interval_from_metadata(&md) - .unwrap() - .as_secs(), + config.deleted_file_retention_duration().as_secs(), 2 * SECONDS_PER_DAY, ); } #[test] fn get_long_from_metadata_test() { - assert_eq!( - CHECKPOINT_INTERVAL - .get_long_from_metadata(&dummy_metadata()) - .unwrap(), - 10, - ) - } - - #[test] - fn get_int_from_metadata_test() { - assert_eq!( - CHECKPOINT_INTERVAL - .get_int_from_metadata(&dummy_metadata()) - .unwrap(), - 10, - ) + let md = dummy_metadata(); + let config = TableConfig(&md.configuration); + assert_eq!(config.checkpoint_interval(), 10,) } #[test] fn get_boolean_from_metadata_test() { - let mut md = dummy_metadata(); + let md = dummy_metadata(); + let config = TableConfig(&md.configuration); // default value is true - assert!(ENABLE_EXPIRED_LOG_CLEANUP - .get_boolean_from_metadata(&md) - .unwrap(),); + assert!(config.enable_expired_log_cleanup()); // change to false + let mut md = dummy_metadata(); md.configuration.insert( - ENABLE_EXPIRED_LOG_CLEANUP.key.to_string(), + DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into(), Some("false".to_string()), ); - assert!(!ENABLE_EXPIRED_LOG_CLEANUP - .get_boolean_from_metadata(&md) - .unwrap()); + let config = TableConfig(&md.configuration); + + assert!(!config.enable_expired_log_cleanup()); } #[test] diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 9aece98ff2..43a3277ae6 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -91,7 +91,6 @@ pub mod operations; pub mod partitions; pub mod schema; pub mod storage; -pub mod table_properties; pub mod table_state; pub mod time_utils; diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index f10fb50bf5..864ef0164d 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -4,6 +4,7 @@ use super::transaction::commit; use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; use crate::action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::builder::ensure_table_uri; +use crate::delta_config::DeltaConfigKey; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; use crate::storage::DeltaObjectStore; use crate::{DeltaResult, DeltaTable, DeltaTableError}; @@ -152,19 +153,25 @@ impl CreateBuilder { } /// Set configuration on created table - pub fn with_configuration(mut self, configuration: HashMap>) -> Self { - self.configuration = configuration; + pub fn with_configuration( + mut self, + configuration: HashMap>>, + ) -> Self { + self.configuration = configuration + .into_iter() + .map(|(k, v)| (k.as_ref().into(), v.map(|s| s.into()))) + .collect(); self } /// Specify a table property in the table configuration pub fn with_configuration_property( mut self, - key: impl Into, + key: DeltaConfigKey, value: Option>, ) -> Self { self.configuration - .insert(key.into(), value.map(|v| v.into())); + .insert(key.as_ref().into(), value.map(|v| v.into())); self } @@ -311,8 +318,8 @@ impl std::future::IntoFuture for CreateBuilder { #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; + use crate::delta_config::DeltaConfigKey; use crate::operations::DeltaOps; - use crate::table_properties::APPEND_ONLY; use crate::writer::test_utils::get_delta_schema; use tempdir::TempDir; @@ -396,14 +403,14 @@ mod tests { let table = CreateBuilder::new() .with_location("memory://") .with_columns(schema.get_fields().clone()) - .with_configuration_property(APPEND_ONLY, Some("true")) + .with_configuration_property(DeltaConfigKey::AppendOnly, Some("true")) .await .unwrap(); let append = table .get_metadata() .unwrap() .configuration - .get(APPEND_ONLY) + .get(DeltaConfigKey::AppendOnly.as_ref()) .unwrap() .as_ref() .unwrap() diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index a3a89ad998..d1cfaa7721 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -26,14 +26,14 @@ use crate::storage::ObjectStoreRef; use crate::table_state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; use crate::writer::utils::PartitionPath; -use crate::{table_properties, DeltaDataTypeVersion}; +use crate::DeltaDataTypeVersion; use crate::{ DeltaDataTypeLong, DeltaResult, DeltaTable, DeltaTableError, ObjectMeta, PartitionFilter, }; use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use futures::future::BoxFuture; use futures::StreamExt; -use log::{debug, error}; +use log::debug; use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; use parquet::file::properties::WriterProperties; use serde::{Deserialize, Serialize}; @@ -380,26 +380,6 @@ impl MergePlan { } } -fn get_target_file_size(snapshot: &DeltaTableState) -> DeltaDataTypeLong { - let mut target_size = 268_435_456; - if let Some(meta) = snapshot.current_metadata() { - let config_str = meta.configuration.get(table_properties::TARGET_FILE_SIZE); - if let Some(s) = config_str { - if let Some(s) = s { - let r = s.parse::(); - if let Ok(size) = r { - target_size = size; - } else { - error!("Unable to parse value of 'delta.targetFileSize'. Using default value"); - } - } else { - error!("Check your configuration of 'delta.targetFileSize'. Using default value"); - } - } - } - target_size -} - /// Build a Plan on which files to merge together. See [OptimizeBuilder] pub fn create_merge_plan( snapshot: &DeltaTableState, @@ -407,7 +387,7 @@ pub fn create_merge_plan( target_size: Option, writer_properties: WriterProperties, ) -> Result { - let target_size = target_size.unwrap_or_else(|| get_target_file_size(snapshot)); + let target_size = target_size.unwrap_or_else(|| snapshot.table_config().target_file_size()); let mut candidates = HashMap::new(); let mut operations: HashMap = HashMap::new(); let mut metrics = Metrics::default(); diff --git a/rust/src/table_properties.rs b/rust/src/table_properties.rs deleted file mode 100644 index 1c69d423fd..0000000000 --- a/rust/src/table_properties.rs +++ /dev/null @@ -1,69 +0,0 @@ -//! properties defined on Delta Tables -//! - -/// true for this Delta table to be append-only. If append-only, -/// existing records cannot be deleted, and existing values cannot be updated. -pub const APPEND_ONLY: &str = "delta.appendOnly"; -/// true for Delta Lake to automatically optimize the layout of the files for this Delta table. -pub const AUTO_OPTIMIZE_AUTO_COMPACT: &str = "delta.autoOptimize.autoCompact"; -/// true for Delta Lake to automatically optimize the layout of the files for this Delta table during writes. -pub const AUTO_OPTIMIZE_OPTIMIZE_WRITE: &str = "delta.autoOptimize.optimizeWrite"; -/// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. -pub const CHECKPOINT_WRITE_STATS_AS_JSON: &str = "delta.checkpoint.writeStatsAsJson"; -/// true for Delta Lake to write file statistics to checkpoints in struct format for the -/// stats_parsed column and to write partition values as a struct for partitionValues_parsed. -pub const CHECKPOINT_WRITE_STATS_AS_STRUCT: &str = "delta.checkpoint.writeStatsAsStruct"; -/// Whether column mapping is enabled for Delta table columns and the corresponding Parquet columns that use different names. -pub const COLUMN_MAPPING_MODE: &str = "delta.columnMapping.mode"; -/// Whether column mapping is enabled for Delta table columns and the corresponding Parquet columns that use different names. -pub const COMPATIBILITY_SYMLINK_FORMAT_MANIFEST_ENABLED: &str = - "delta.compatibility.symlinkFormatManifest.enabled"; -/// The number of columns for Delta Lake to collect statistics about for data skipping. -/// A value of -1 means to collect statistics for all columns. Updating this property does -/// not automatically collect statistics again; instead, it redefines the statistics schema -/// of the Delta table. Specifically, it changes the behavior of future statistics collection -/// (such as during appends and optimizations) as well as data skipping (such as ignoring column -/// statistics beyond this number, even when such statistics exist). -pub const DATA_SKIPPING_NUM_INDEXED_COLS: &str = "delta.dataSkippingNumIndexedCols"; -/// The shortest duration for Delta Lake to keep logically deleted data files before deleting -/// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. -/// -/// This value should be large enough to ensure that: -/// -/// * It is larger than the longest possible duration of a job if you run VACUUM when there are -/// concurrent readers or writers accessing the Delta table. -/// * If you run a streaming query that reads from the table, that query does not stop for longer -/// than this value. Otherwise, the query may not be able to restart, as it must still read old files. -pub const DELETED_FILE_RETENTION_DURATION: &str = "delta.deletedFileRetentionDuration"; -/// true to enable change data feed. -pub const ENABLE_CHANGE_DATA_FEED: &str = "delta.enableChangeDataFeed"; -/// The degree to which a transaction must be isolated from modifications made by concurrent transactions. -/// -/// Valid values are `Serializable` and `WriteSerializable`. -pub const ISOLATION_LEVEL: &str = "delta.isolationLevel"; -/// How long the history for a Delta table is kept. -/// -/// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older -/// than the retention interval. If you set this property to a large enough value, many log -/// entries are retained. This should not impact performance as operations against the log are -/// constant time. Operations on history are parallel but will become more expensive as the log size increases. -pub const LOG_RETENTION_DURATION: &str = "delta.logRetentionDuration"; -/// The minimum required protocol reader version for a reader that allows to read from this Delta table. -pub const MIN_READER_VERSION: &str = "delta.minReaderVersion"; -/// The minimum required protocol writer version for a writer that allows to write to this Delta table. -pub const MIN_WRITER_VERSION: &str = "delta.minWriterVersion"; -/// true for Delta Lake to generate a random prefix for a file path instead of partition information. -/// -/// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high volumes -/// of Amazon S3 calls to better partition across S3 servers. -pub const RANDOMIZE_FILE_PREFIXES: &str = "delta.randomizeFilePrefixes"; -/// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta Lake generates for random prefixes. -pub const RANDOM_PREFIX_LENGTH: &str = "delta.randomPrefixLength"; -/// The shortest duration within which new snapshots will retain transaction identifiers (for example, SetTransactions). -/// When a new snapshot sees a transaction identifier older than or equal to the duration specified by this property, -/// the snapshot considers it expired and ignores it. The SetTransaction identifier is used when making the writes idempotent. -pub const SET_TRANSACTION_RETENTION_DURATION: &str = "delta.setTransactionRetentionDuration"; -/// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. -pub const TARGET_FILE_SIZE: &str = "delta.targetFileSize"; -/// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. -pub const TUNE_FILE_SIZES_FOR_REWRITES: &str = "delta.tuneFileSizesForRewrites"; diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 1eaf862398..341e3d8f75 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -1,14 +1,16 @@ //! The module for delta table state. use crate::action::{self, Action, Add}; -use crate::delta_config; +use crate::delta_config::TableConfig; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; +use crate::Schema; use crate::{ ApplyLogError, DeltaDataTypeLong, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, }; use chrono::Utc; +use lazy_static::lazy_static; use object_store::{path::Path, ObjectStore}; use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; @@ -228,6 +230,22 @@ impl DeltaTableState { self.current_metadata.as_ref() } + /// The table schema + pub fn schema(&self) -> Option<&Schema> { + self.current_metadata.as_ref().map(|m| &m.schema) + } + + /// Well known table configuration + pub fn table_config(&self) -> TableConfig<'_> { + lazy_static! { + static ref DUMMY_CONF: HashMap> = HashMap::new(); + } + self.current_metadata + .as_ref() + .map(|meta| TableConfig(&meta.configuration)) + .unwrap_or_else(|| TableConfig(&DUMMY_CONF)) + } + /// Merges new state information into our state /// /// The DeltaTableState also carries the version information for the given state, @@ -322,14 +340,12 @@ impl DeltaTableState { action::Action::metaData(v) => { let md = DeltaTableMetaData::try_from(v) .map_err(|e| ApplyLogError::InvalidJson { source: e })?; - self.tombstone_retention_millis = delta_config::TOMBSTONE_RETENTION - .get_interval_from_metadata(&md)? - .as_millis() as i64; - self.log_retention_millis = delta_config::LOG_RETENTION - .get_interval_from_metadata(&md)? - .as_millis() as i64; - self.enable_expired_log_cleanup = - delta_config::ENABLE_EXPIRED_LOG_CLEANUP.get_boolean_from_metadata(&md)?; + let table_config = TableConfig(&md.configuration); + self.tombstone_retention_millis = + table_config.deleted_file_retention_duration().as_millis() as i64; + self.log_retention_millis = + table_config.log_retention_duration().as_millis() as i64; + self.enable_expired_log_cleanup = table_config.enable_expired_log_cleanup(); self.current_metadata = Some(md); } action::Action::txn(v) => { diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index a494f238e2..80067814dd 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -91,6 +91,7 @@ mod delete_expired_delta_log_in_checkpoint { use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; + use deltalake::delta_config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; @@ -99,8 +100,8 @@ mod delete_expired_delta_log_in_checkpoint { let mut table = fs_common::create_table( "./tests/data/checkpoints_with_expired_logs/expired", Some(hashmap! { - delta_config::LOG_RETENTION.key.clone() => Some("interval 10 minute".to_string()), - delta_config::ENABLE_EXPIRED_LOG_CLEANUP.key.clone() => Some("true".to_string()) + DeltaConfigKey::LogRetentionDuration.as_ref().into() => Some("interval 10 minute".to_string()), + DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into() => Some("true".to_string()) }), ) .await; @@ -163,8 +164,8 @@ mod delete_expired_delta_log_in_checkpoint { let mut table = fs_common::create_table( "./tests/data/checkpoints_with_expired_logs/not_delete_expired", Some(hashmap! { - delta_config::LOG_RETENTION.key.clone() => Some("interval 1 second".to_string()), - delta_config::ENABLE_EXPIRED_LOG_CLEANUP.key.clone() => Some("false".to_string()) + DeltaConfigKey::LogRetentionDuration.as_ref().into() => Some("interval 1 second".to_string()), + DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into() => Some("false".to_string()) }), ) .await; @@ -212,6 +213,7 @@ mod checkpoints_with_tombstones { use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; use deltalake::action::*; + use deltalake::delta_config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; use parquet::file::reader::{FileReader, SerializedFileReader}; @@ -237,7 +239,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_expired_tombstones() { let mut table = fs_common::create_table("./tests/data/checkpoints_tombstones/expired", Some(hashmap! { - delta_config::TOMBSTONE_RETENTION.key.clone() => Some("interval 1 minute".to_string()) + DeltaConfigKey::DeletedFileRetentionDuration.as_ref().into() => Some("interval 1 minute".to_string()) })).await; let a1 = fs_common::add(3 * 60 * 1000); // 3 mins ago, From ad8141c4fcd77ad3224f36716b83dd3aa039e3b0 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 4 Mar 2023 21:11:45 +0100 Subject: [PATCH 41/67] chore: clippy fix --- rust/src/operations/transaction/test_utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 0d1e65af38..45ee1efc01 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -15,7 +15,7 @@ pub fn create_add_action( path: path.into(), size: 100, data_change, - stats: stats.map(|s| s.into()), + stats, ..Default::default() }) } From a844e79a3fe9c477acd4e6fee9def02e481c3665 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 4 Mar 2023 22:03:21 +0100 Subject: [PATCH 42/67] feat: typed commit info --- rust/src/action/mod.rs | 202 +++++++++++++++++++++++---- rust/src/action/parquet2_read/mod.rs | 2 +- rust/src/delta.rs | 72 +++------- rust/src/operations/transaction.rs | 47 ++++--- rust/src/storage/mod.rs | 8 +- rust/src/table_state.rs | 8 +- rust/tests/command_optimize.rs | 12 +- rust/tests/commit_info_format.rs | 16 +-- 8 files changed, 239 insertions(+), 128 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 978a485836..4e98242029 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -8,7 +8,8 @@ mod parquet_read; #[cfg(feature = "parquet2")] pub mod parquet2_read; -use crate::{schema::*, DeltaTableError, DeltaTableMetaData}; +use crate::delta_config::IsolationLevel; +use crate::{schema::*, DeltaResult, DeltaTableError, DeltaTableMetaData}; use percent_encoding::percent_decode; use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; @@ -44,6 +45,13 @@ pub enum ActionError { #[from] source: parquet::errors::ParquetError, }, + /// Faild to serialize operation + #[error("Failed to serialize operation: {source}")] + SerializeOperation { + #[from] + /// The source error + source: serde_json::Error, + }, } fn decode_path(raw_path: &str) -> Result { @@ -435,7 +443,46 @@ pub struct Protocol { pub min_writer_version: DeltaDataTypeInt, } -type CommitInfo = Map; +/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. +/// However the reference implementation as well as delta-rs store useful information that may for instance +/// allow us to be more permissive in commit conflict resolution. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq)] +#[serde(rename_all = "camelCase")] +pub struct CommitInfo { + /// Version number the commit corresponds to + #[serde(skip_serializing_if = "Option::is_none")] + pub version: Option, + /// Timestamp in millis when the commit was created + #[serde(skip_serializing_if = "Option::is_none")] + pub timestamp: Option, + /// Id of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_id: Option, + /// Name of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_name: Option, + /// The operation performed during the + #[serde(skip_serializing_if = "Option::is_none")] + pub operation: Option, + /// Parameters used for table operation + #[serde(skip_serializing_if = "Option::is_none")] + pub operation_parameters: Option>, + /// Version of the table when the operation was started + #[serde(skip_serializing_if = "Option::is_none")] + pub read_version: Option, + /// The isolation level of the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub isolation_level: Option, + /// TODO + #[serde(skip_serializing_if = "Option::is_none")] + pub is_blind_append: Option, + /// Delta engine which created the commit. + #[serde(skip_serializing_if = "Option::is_none")] + pub engine_info: Option, + /// Additional provenance information for the commit + #[serde(flatten, default)] + pub info: Map, +} /// Represents an action in the Delta log. The Delta log is an aggregate of all actions performed /// on the table, so the full list of actions is required to properly read a table. @@ -459,6 +506,16 @@ pub enum Action { commitInfo(CommitInfo), } +impl Action { + /// Create a commit info from a map + pub fn commit_info(info: Map) -> Self { + Self::commitInfo(CommitInfo { + info, + ..Default::default() + }) + } +} + /// Operation performed when creating a new log entry with one or more actions. /// This is a key element of the `CommitInfo` action. #[allow(clippy::large_enum_variant)] @@ -517,45 +574,71 @@ pub enum DeltaOperation { } impl DeltaOperation { - /// Retrieve basic commit information to be added to Delta commits - pub fn get_commit_info(&self) -> Map { - let mut commit_info = Map::::new(); - let operation = match &self { - DeltaOperation::Create { .. } => "delta-rs.Create", - DeltaOperation::Write { .. } => "delta-rs.Write", - DeltaOperation::StreamingUpdate { .. } => "delta-rs.StreamingUpdate", - DeltaOperation::Optimize { .. } => "delta-rs.Optimize", - DeltaOperation::FileSystemCheck { .. } => "delta-rs.FileSystemCheck", - }; - commit_info.insert( - "operation".to_string(), - serde_json::Value::String(operation.into()), - ); + /// A human readable name for the operation + pub fn name(&self) -> &str { + // operation names taken from https://learn.microsoft.com/en-us/azure/databricks/delta/history#--operation-metrics-keys + match &self { + DeltaOperation::Create { mode, .. } if matches!(mode, SaveMode::Overwrite) => { + "CREATE OR REPLACE TABLE" + } + DeltaOperation::Create { .. } => "CREATE TABLE", + DeltaOperation::Write { .. } => "WRITE", + DeltaOperation::StreamingUpdate { .. } => "STREAMING UPDATE", + DeltaOperation::Optimize { .. } => "OPTIMIZE", + DeltaOperation::FileSystemCheck { .. } => "FSCK", + } + } - if let Ok(serde_json::Value::Object(map)) = serde_json::to_value(self) { - let all_operation_fields = map.values().next().unwrap().as_object().unwrap(); - let converted_operation_fields: Map = all_operation_fields - .iter() + /// Paraemters configured for operation. + pub fn operation_parameters(&self) -> DeltaResult> { + // TODO remove unwrap + let serialized = serde_json::to_value(self) + .map_err(|err| ActionError::SerializeOperation { source: err })?; + if let serde_json::Value::Object(map) = serialized { + let all_operation_fields = map.values().next().unwrap().as_object().unwrap().clone(); + Ok(all_operation_fields + .into_iter() .filter(|item| !item.1.is_null()) .map(|(k, v)| { ( - k.clone(), + k, serde_json::Value::String(if v.is_string() { String::from(v.as_str().unwrap()) } else { v.to_string() }), ) - }) - .collect(); + })) + } else { + Err(ActionError::Generic( + "operation parameetrs serialized into unexpected shape".into(), + ) + .into()) + } + } - commit_info.insert( - "operationParameters".to_string(), - serde_json::Value::Object(converted_operation_fields), - ); - }; + /// Denotes if the operation changes the data contained in the table + pub fn changes_data(&self) -> bool { + !matches!(self, Self::Optimize { .. }) + } + + /// Retrieve basic commit information to be added to Delta commits + pub fn get_commit_info(&self) -> CommitInfo { + // TODO infer additional info from operation parameters ... + CommitInfo { + operation: Some(self.name().into()), + operation_parameters: self.operation_parameters().ok().map(|iter| iter.collect()), + ..Default::default() + } + } - commit_info + /// Get predicate expression applien when the operation reads data from the table. + pub fn read_predicate(&self) -> Option { + match self { + // TODO add more operations + Self::Write { predicate, .. } => predicate.clone(), + _ => None, + } } } @@ -654,4 +737,65 @@ mod tests { 1 ); } + + #[test] + fn test_read_commit_info() { + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c" + }"#; + + let info = serde_json::from_str::(raw); + assert!(info.is_ok()); + + println!("{:?}", info); + + // assert that commit info has no required filelds + let raw = "{}"; + let info = serde_json::from_str::(raw); + assert!(info.is_ok()); + + // arbitrary field data may be added to commit + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c", + "additionalField": "more data", + "additionalStruct": { + "key": "value", + "otherKey": 123 + } + }"#; + + let info = serde_json::from_str::(raw).expect("should parse"); + assert!(info.info.contains_key("additionalField")); + assert!(info.info.contains_key("additionalStruct")); + } } diff --git a/rust/src/action/parquet2_read/mod.rs b/rust/src/action/parquet2_read/mod.rs index 34988d5954..ddfd2a9e55 100644 --- a/rust/src/action/parquet2_read/mod.rs +++ b/rust/src/action/parquet2_read/mod.rs @@ -223,7 +223,7 @@ impl ActionVariant for CommitInfo { type Variant = CommitInfo; fn default_action() -> Action { - Action::commitInfo(CommitInfo::new()) + Action::commitInfo(CommitInfo::default()) } fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 783cca2026..ca51bafb6e 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -18,7 +18,7 @@ use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; use crate::delta_config::DeltaConfigError; use crate::operations::vacuum::VacuumBuilder; -use crate::storage::ObjectStoreRef; +use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use chrono::{DateTime, Duration, Utc}; use futures::StreamExt; @@ -560,12 +560,6 @@ impl DeltaTable { self.storage.root_uri() } - /// Return the uri of commit version. - pub fn commit_uri_from_version(&self, version: DeltaDataTypeVersion) -> Path { - let version = format!("{version:020}.json"); - Path::from_iter(["_delta_log", &version]) - } - /// Return the list of paths of given checkpoint. pub fn get_checkpoint_data_paths(&self, check_point: &CheckPoint) -> Vec { let checkpoint_prefix = format!("{:020}", check_point.version); @@ -727,11 +721,7 @@ impl DeltaTable { // scan logs after checkpoint loop { - match self - .storage - .head(&self.commit_uri_from_version(version)) - .await - { + match self.storage.head(&commit_uri_from_version(version)).await { Ok(meta) => { // also cache timestamp for version self.version_timestamp @@ -760,7 +750,7 @@ impl DeltaTable { Ok(version) } - /// Currently loaded evrsion of the table + /// Currently loaded version of the table pub fn version(&self) -> DeltaDataTypeVersion { self.state.version() } @@ -778,7 +768,7 @@ impl DeltaTable { current_version: DeltaDataTypeVersion, ) -> Result { let next_version = current_version + 1; - let commit_uri = self.commit_uri_from_version(next_version); + let commit_uri = commit_uri_from_version(next_version); let commit_log_bytes = self.storage.get(&commit_uri).await; let commit_log_bytes = match commit_log_bytes { Err(ObjectStoreError::NotFound { .. }) => return Ok(PeekCommit::UpToDate), @@ -873,7 +863,7 @@ impl DeltaTable { version: DeltaDataTypeVersion, ) -> Result<(), DeltaTableError> { // check if version is valid - let commit_uri = self.commit_uri_from_version(version); + let commit_uri = commit_uri_from_version(version); match self.storage.head(&commit_uri).await { Ok(_) => {} Err(ObjectStoreError::NotFound { .. }) => { @@ -910,10 +900,7 @@ impl DeltaTable { match self.version_timestamp.get(&version) { Some(ts) => Ok(*ts), None => { - let meta = self - .storage - .head(&self.commit_uri_from_version(version)) - .await?; + let meta = self.storage.head(&commit_uri_from_version(version)).await?; let ts = meta.last_modified.timestamp(); // also cache timestamp for version self.version_timestamp.insert(version, ts); @@ -930,7 +917,7 @@ impl DeltaTable { pub async fn history( &mut self, limit: Option, - ) -> Result>, DeltaTableError> { + ) -> Result, DeltaTableError> { let mut version = match limit { Some(l) => max(self.version() - l as i64 + 1, 0), None => self.get_earliest_delta_log_version().await?, @@ -1085,7 +1072,7 @@ impl DeltaTable { /// Return table schema parsed from transaction log. Return None if table hasn't been loaded or /// no metadata was found in the log. pub fn schema(&self) -> Option<&Schema> { - self.state.current_metadata().map(|m| &m.schema) + self.state.schema() } /// Return table schema parsed from transaction log. Return `DeltaTableError` if table hasn't @@ -1145,7 +1132,7 @@ impl DeltaTable { // move temporary commit file to delta log directory // rely on storage to fail if the file already exists - self.storage - .rename_if_not_exists(&commit.uri, &self.commit_uri_from_version(version)) + .rename_if_not_exists(&commit.uri, &commit_uri_from_version(version)) .await .map_err(|e| match e { ObjectStoreError::AlreadyExists { .. } => { @@ -1181,7 +1168,7 @@ impl DeltaTable { ); let mut actions = vec![ - Action::commitInfo(enriched_commit_info), + Action::commit_info(enriched_commit_info), Action::protocol(protocol), Action::metaData(meta), ]; @@ -1391,22 +1378,17 @@ impl<'a> DeltaTransaction<'a> { .iter() .any(|a| matches!(a, action::Action::commitInfo(..))) { - let mut commit_info = Map::::new(); - commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - commit_info.insert( + let mut extra_info = Map::::new(); + let mut commit_info = operation.map(|op| op.get_commit_info()).unwrap_or_default(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - - if let Some(op) = &operation { - commit_info.append(&mut op.get_commit_info()) - } if let Some(mut meta) = app_metadata { - commit_info.append(&mut meta) + extra_info.append(&mut meta) } + commit_info.info = extra_info; self.add_action(action::Action::commitInfo(commit_info)); } @@ -1661,27 +1643,7 @@ mod tests { assert_eq!(DeltaTableMetaData::try_from(action).unwrap(), delta_md); } Action::commitInfo(action) => { - let mut modified_action = action; - let timestamp = serde_json::Number::from(0i64); - modified_action["timestamp"] = Value::Number(serde_json::Number::from(0i64)); - let mut expected = Map::::new(); - expected.insert( - "operation".to_string(), - serde_json::Value::String("CREATE TABLE".to_string()), - ); - expected.insert( - "userName".to_string(), - serde_json::Value::String("test user".to_string()), - ); - expected.insert( - "delta-rs".to_string(), - serde_json::Value::String(crate_version().to_string()), - ); - expected.insert( - "timestamp".to_string(), - serde_json::Value::Number(timestamp), - ); - assert_eq!(modified_action, expected) + assert_eq!(action.operation, Some("CREATE TABLE".to_string())); } _ => (), } diff --git a/rust/src/operations/transaction.rs b/rust/src/operations/transaction.rs index 974eaeeedc..9198a960b4 100644 --- a/rust/src/operations/transaction.rs +++ b/rust/src/operations/transaction.rs @@ -11,7 +11,7 @@ use serde_json::{Map, Value}; const DELTA_LOG_FOLDER: &str = "_delta_log"; #[derive(thiserror::Error, Debug)] -enum TransactionError { +pub(crate) enum TransactionError { #[error("Tried committing existing table version: {0}")] VersionAlreadyExists(DeltaDataTypeVersion), @@ -62,34 +62,41 @@ fn log_entry_from_actions(actions: &[Action]) -> Result, +pub(crate) fn get_commit_bytes( + operation: &DeltaOperation, + actions: &mut Vec, app_metadata: Option>, -) -> Result { +) -> Result { if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { - let mut commit_info = Map::::new(); - commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - commit_info.insert( + let mut extra_info = Map::::new(); + let mut commit_info = operation.get_commit_info(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - commit_info.append(&mut operation.get_commit_info()); if let Some(mut meta) = app_metadata { - commit_info.append(&mut meta) + extra_info.append(&mut meta) } + commit_info.info = extra_info; actions.push(Action::commitInfo(commit_info)); } // Serialize all actions that are part of this log entry. - let log_entry = bytes::Bytes::from(log_entry_from_actions(&actions)?); + Ok(bytes::Bytes::from(log_entry_from_actions(actions)?)) +} + +/// Low-level transaction API. Creates a temporary commit file. Once created, +/// the transaction object could be dropped and the actual commit could be executed +/// with `DeltaTable.try_commit_transaction`. +pub(crate) async fn prepare_commit( + storage: &dyn ObjectStore, + operation: &DeltaOperation, + actions: &mut Vec, + app_metadata: Option>, +) -> Result { + // Serialize all actions that are part of this log entry. + let log_entry = get_commit_bytes(operation, actions, app_metadata)?; // Write delta log entry as temporary file to storage. For the actual commit, // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. @@ -128,11 +135,11 @@ async fn try_commit_transaction( pub(crate) async fn commit( storage: &DeltaObjectStore, version: DeltaDataTypeVersion, - actions: Vec, + mut actions: Vec, operation: DeltaOperation, app_metadata: Option>, ) -> DeltaResult { - let tmp_commit = prepare_commit(storage, operation, actions, app_metadata).await?; + let tmp_commit = prepare_commit(storage, &operation, &mut actions, app_metadata).await?; match try_commit_transaction(storage, &tmp_commit, version).await { Ok(version) => Ok(version), Err(TransactionError::VersionAlreadyExists(version)) => { diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index 360c241a84..99b1ac98a1 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -5,7 +5,7 @@ pub mod file; pub mod utils; use self::config::{ObjectStoreKind, StorageOptions}; -use crate::DeltaResult; +use crate::{DeltaDataTypeVersion, DeltaResult}; use bytes::Bytes; use futures::{stream::BoxStream, StreamExt}; @@ -37,6 +37,12 @@ lazy_static! { static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); } +/// Return the uri of commit version. +pub(crate) fn commit_uri_from_version(version: DeltaDataTypeVersion) -> Path { + let version = format!("{version:020}.json"); + DELTA_LOG_PATH.child(version.as_str()) +} + /// Sharable reference to [`DeltaObjectStore`] pub type ObjectStoreRef = Arc; diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 341e3d8f75..b1808f9452 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -4,6 +4,7 @@ use crate::action::{self, Action, Add}; use crate::delta_config::TableConfig; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; +use crate::storage::commit_uri_from_version; use crate::Schema; use crate::{ ApplyLogError, DeltaDataTypeLong, DeltaDataTypeVersion, DeltaTable, DeltaTableError, @@ -13,7 +14,6 @@ use chrono::Utc; use lazy_static::lazy_static; use object_store::{path::Path, ObjectStore}; use serde::{Deserialize, Serialize}; -use serde_json::{Map, Value}; use std::collections::HashMap; use std::collections::HashSet; use std::convert::TryFrom; @@ -34,7 +34,7 @@ pub struct DeltaTableState { // active files for table state files: Vec, // Information added to individual commits - commit_infos: Vec>, + commit_infos: Vec, app_transaction_version: HashMap, min_reader_version: i32, min_writer_version: i32, @@ -66,7 +66,7 @@ impl DeltaTableState { table: &DeltaTable, version: DeltaDataTypeVersion, ) -> Result { - let commit_uri = table.commit_uri_from_version(version); + let commit_uri = commit_uri_from_version(version); let commit_log_bytes = table.storage.get(&commit_uri).await?.bytes().await?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); @@ -164,7 +164,7 @@ impl DeltaTableState { } /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec> { + pub fn commit_infos(&self) -> &Vec { &self.commit_infos } diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 5006e4a209..9952c00825 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -507,16 +507,14 @@ async fn test_commit_info() -> Result<(), Box> { let last_commit = &commit_info[commit_info.len() - 1]; let commit_metrics = - serde_json::from_value::(last_commit["operationMetrics"].clone())?; + serde_json::from_value::(last_commit.info["operationMetrics"].clone())?; assert_eq!(commit_metrics, metrics); - assert_eq!(last_commit["readVersion"], json!(version)); - assert_eq!( - last_commit["operationParameters"]["targetSize"], - json!("2000000") - ); + assert_eq!(last_commit.read_version, Some(version)); + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["targetSize"], json!("2000000")); // TODO: Requires a string representation for PartitionFilter - assert_eq!(last_commit["operationParameters"]["predicate"], Value::Null); + // assert_eq!(parameters["predicate"], None); Ok(()) } diff --git a/rust/tests/commit_info_format.rs b/rust/tests/commit_info_format.rs index 8a04702579..60a8bf5ee4 100644 --- a/rust/tests/commit_info_format.rs +++ b/rust/tests/commit_info_format.rs @@ -2,8 +2,7 @@ mod fs_common; use deltalake::action::{Action, DeltaOperation, SaveMode}; - -use serde_json::{json, Value}; +use serde_json::json; use std::error::Error; use tempdir::TempDir; @@ -27,15 +26,10 @@ async fn test_operational_parameters() -> Result<(), Box> { let commit_info = table.history(None).await?; let last_commit = &commit_info[commit_info.len() - 1]; - - assert_eq!(last_commit["operationParameters"]["mode"], json!("Append")); - - assert_eq!( - last_commit["operationParameters"]["partitionBy"], - json!("[\"some_partition\"]") - ); - - assert_eq!(last_commit["operationParameters"]["predicate"], Value::Null); + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["mode"], json!("Append")); + assert_eq!(parameters["partitionBy"], json!("[\"some_partition\"]")); + // assert_eq!(parameters["predicate"], None); Ok(()) } From f69bfc7a6ebb03b9523e5a9352cbb6c29b46e3f4 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 4 Mar 2023 22:59:18 +0100 Subject: [PATCH 43/67] chore: cleanup after merge --- .../transaction/conflict_checker.rs | 65 ++++++- rust/src/operations/transaction/mod.rs | 3 - rust/src/operations/transaction/types.rs | 175 ------------------ rust/src/storage/utils.rs | 10 +- 4 files changed, 65 insertions(+), 188 deletions(-) delete mode 100644 rust/src/operations/transaction/types.rs diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 3ae89d9b6b..aad87c86b3 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -6,8 +6,9 @@ use std::io::{BufRead, BufReader, Cursor}; use object_store::ObjectStore; use serde_json::{Map, Value}; -use super::{can_downgrade_to_snapshot_isolation, CommitInfo, IsolationLevel}; +use super::CommitInfo; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; +use crate::delta_config::IsolationLevel; use crate::operations::transaction::TransactionError; use crate::storage::{commit_uri_from_version, DeltaObjectStore, ObjectStoreRef}; use crate::{ @@ -504,6 +505,56 @@ impl<'a> ConflictChecker<'a> { } } +// implementation and comments adopted from +// https://github.com/delta-io/delta/blob/1c18c1d972e37d314711b3a485e6fb7c98fce96d/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1268 +// +// For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can +// change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict +// detection by skipping the +// [ConflictChecker::check_for_added_files_that_should_have_been_read_by_current_txn] check i.e. +// don't worry about concurrent appends. +// +// We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: +// t0 - Initial state of table +// t1 - Q1, Q2 starts +// t2 - Q1 commits +// t3 - Q2 is empty and wants to commit. +// In this scenario, we can always allow Q2 to commit without worrying about new files +// generated by Q1. +// +// The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 +// Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has +// a Metadata update (say schema change/identity column high watermark update), then Q2 can't +// be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, +// then Q1 should see the updates from Q2 - which actually didn't happen. +pub(super) fn can_downgrade_to_snapshot_isolation<'a>( + actions: impl IntoIterator, + operation: &DeltaOperation, + isolation_level: &IsolationLevel, +) -> bool { + let mut data_changed = false; + let mut has_non_file_actions = false; + for action in actions { + match action { + Action::add(act) if act.data_change => data_changed = true, + Action::remove(rem) if rem.data_change => data_changed = true, + _ => has_non_file_actions = true, + } + } + + if has_non_file_actions { + // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation + // level to SnapshotIsolation. + return false; + } + + match isolation_level { + IsolationLevel::Serializable => !data_changed, + IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), + IsolationLevel::SnapshotIsolation => false, // this case should never happen, since spanpshot isolation canot be configured on table + } +} + #[cfg(test)] mod tests { use super::super::test_utils as tu; @@ -529,6 +580,18 @@ mod tests { Some(data.to_string()) } + #[test] + fn test_can_downgrade_to_snapshot_isolation() { + let isolation = IsolationLevel::WriteSerializable; + let operation = DeltaOperation::Optimize { + predicate: None, + target_size: 0, + }; + let add = tu::create_add_action("p", false, None); + let res = can_downgrade_to_snapshot_isolation(&[add], &operation, &isolation); + assert!(!res) + } + #[tokio::test] // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 async fn test_allowed_concurrent_actions() { diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index f70b18e5ce..30b441ed31 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -14,9 +14,6 @@ mod conflict_checker; mod state; #[cfg(test)] pub(crate) mod test_utils; -mod types; - -pub use types::*; use self::conflict_checker::WinningCommitSummary; diff --git a/rust/src/operations/transaction/types.rs b/rust/src/operations/transaction/types.rs deleted file mode 100644 index 0cbba12a58..0000000000 --- a/rust/src/operations/transaction/types.rs +++ /dev/null @@ -1,175 +0,0 @@ -//! Types and structs used when commitind operations to a delta table -use std::str::FromStr; - -use serde::{Deserialize, Serialize}; - -use crate::action::{Action, DeltaOperation}; -use crate::DeltaTableError; - -#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] -/// The isolation level applied during transaction -pub enum IsolationLevel { - /// The strongest isolation level. It ensures that committed write operations - /// and all reads are Serializable. Operations are allowed as long as there - /// exists a serial sequence of executing them one-at-a-time that generates - /// the same outcome as that seen in the table. For the write operations, - /// the serial sequence is exactly the same as that seen in the table’s history. - Serializable, - - /// A weaker isolation level than Serializable. It ensures only that the write - /// operations (that is, not reads) are serializable. However, this is still stronger - /// than Snapshot isolation. WriteSerializable is the default isolation level because - /// it provides great balance of data consistency and availability for most common operations. - WriteSerializable, - - /// SnapshotIsolation is a guarantee that all reads made in a transaction will see a consistent - /// snapshot of the database (in practice it reads the last committed values that existed at the - /// time it started), and the transaction itself will successfully commit only if no updates - /// it has made conflict with any concurrent updates made since that snapshot. - SnapshotIsolation, -} - -impl IsolationLevel { - /// The default isolation level to use, analogous to reference implementation - pub fn default_level() -> Self { - // https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 - IsolationLevel::Serializable - } -} - -// Spark assumes Serializable as default isolation level -// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 -impl Default for IsolationLevel { - fn default() -> Self { - Self::Serializable - } -} - -impl AsRef for IsolationLevel { - fn as_ref(&self) -> &str { - match self { - Self::Serializable => "Serializable", - Self::WriteSerializable => "WriteSerializable", - Self::SnapshotIsolation => "SnapshotIsolation", - } - } -} - -impl FromStr for IsolationLevel { - type Err = DeltaTableError; - - fn from_str(s: &str) -> Result { - match s.to_ascii_lowercase().as_str() { - "serializable" => Ok(Self::Serializable), - "writeserializable" | "write_serializable" => Ok(Self::WriteSerializable), - "snapshotisolation" | "snapshot_isolation" => Ok(Self::SnapshotIsolation), - _ => Err(DeltaTableError::Generic( - "Invalid string for IsolationLevel".into(), - )), - } - } -} - -// implementation and comments adopted from -// https://github.com/delta-io/delta/blob/1c18c1d972e37d314711b3a485e6fb7c98fce96d/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1268 -// -// For no-data-change transactions such as OPTIMIZE/Auto Compaction/ZorderBY, we can -// change the isolation level to SnapshotIsolation. SnapshotIsolation allows reduced conflict -// detection by skipping the -// [ConflictChecker::check_for_added_files_that_should_have_been_read_by_current_txn] check i.e. -// don't worry about concurrent appends. -// -// We can also use SnapshotIsolation for empty transactions. e.g. consider a commit: -// t0 - Initial state of table -// t1 - Q1, Q2 starts -// t2 - Q1 commits -// t3 - Q2 is empty and wants to commit. -// In this scenario, we can always allow Q2 to commit without worrying about new files -// generated by Q1. -// -// The final order which satisfies both Serializability and WriteSerializability is: Q2, Q1 -// Note that Metadata only update transactions shouldn't be considered empty. If Q2 above has -// a Metadata update (say schema change/identity column high watermark update), then Q2 can't -// be moved above Q1 in the final SERIALIZABLE order. This is because if Q2 is moved above Q1, -// then Q1 should see the updates from Q2 - which actually didn't happen. -pub(super) fn can_downgrade_to_snapshot_isolation<'a>( - actions: impl IntoIterator, - operation: &DeltaOperation, - isolation_level: &IsolationLevel, -) -> bool { - let mut data_changed = false; - let mut has_non_file_actions = false; - for action in actions { - match action { - Action::add(act) if act.data_change => data_changed = true, - Action::remove(rem) if rem.data_change => data_changed = true, - _ => has_non_file_actions = true, - } - } - - if has_non_file_actions { - // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation - // level to SnapshotIsolation. - return false; - } - - match isolation_level { - IsolationLevel::Serializable => !data_changed, - IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), - IsolationLevel::SnapshotIsolation => false, // this case should never happen, since spanpshot isolation canot be configured on table - } -} - -#[cfg(test)] -mod tests { - use super::super::test_utils; - use super::*; - - #[test] - fn test_roundtrip_isolation_level() { - assert!(matches!( - "Serializable".parse().unwrap(), - IsolationLevel::Serializable - )); - assert!(matches!( - "WriteSerializable".parse().unwrap(), - IsolationLevel::WriteSerializable - )); - assert!(matches!( - "SnapshotIsolation".parse().unwrap(), - IsolationLevel::SnapshotIsolation - )); - assert!(matches!( - IsolationLevel::Serializable.as_ref().parse().unwrap(), - IsolationLevel::Serializable - )); - assert!(matches!( - IsolationLevel::WriteSerializable.as_ref().parse().unwrap(), - IsolationLevel::WriteSerializable - )); - assert!(matches!( - IsolationLevel::SnapshotIsolation.as_ref().parse().unwrap(), - IsolationLevel::SnapshotIsolation - )) - } - - #[test] - fn test_default_isolation_level() { - assert!(matches!( - IsolationLevel::default(), - IsolationLevel::Serializable - )) - } - - #[test] - fn test_can_downgrade_to_snapshot_isolation() { - let isolation = IsolationLevel::WriteSerializable; - let operation = DeltaOperation::Optimize { - predicate: None, - target_size: 0, - }; - let add = test_utils::create_add_action("p", false, None); - let res = can_downgrade_to_snapshot_isolation(&[add], &operation, &isolation); - assert!(!res) - } -} diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index 27607deee6..8c29e842db 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -4,21 +4,13 @@ use std::collections::HashMap; use crate::action::Add; use crate::builder::DeltaTableBuilder; -use crate::{DeltaDataTypeVersion, DeltaResult, DeltaTableError}; +use crate::{DeltaResult, DeltaTableError}; use chrono::{DateTime, NaiveDateTime, Utc}; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; use std::sync::Arc; -const DELTA_LOG_FOLDER: &str = "_delta_log"; - -/// Return the uri of commit version. -pub(crate) fn commit_uri_from_version(version: DeltaDataTypeVersion) -> Path { - let version = format!("{version:020}.json"); - Path::from_iter([DELTA_LOG_FOLDER, &version]) -} - /// Copies the contents from the `from` location into the `to` location pub async fn copy_table( from: impl AsRef, From 93dcfaf1e7f46426552f9b4b92ee585113928660 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 4 Mar 2023 23:22:15 +0100 Subject: [PATCH 44/67] feat: propagate errors --- rust/src/operations/transaction/mod.rs | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 30b441ed31..e2da601ab1 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -15,7 +15,7 @@ mod state; #[cfg(test)] pub(crate) mod test_utils; -use self::conflict_checker::WinningCommitSummary; +use self::conflict_checker::{CommitConflictError, WinningCommitSummary}; const DELTA_LOG_FOLDER: &str = "_delta_log"; @@ -38,6 +38,12 @@ pub(crate) enum TransactionError { #[from] source: ObjectStoreError, }, + /// Error returned when a commit conflict ocurred + #[error("Failed to commit transaction: {0}")] + CommitConflict(#[from] CommitConflictError), + /// Error returned when maximum number of commit trioals is exceeded + #[error("Failed to commit transaction: {0}")] + MaxCommitAttempts(i32), } impl From for DeltaTableError { @@ -50,6 +56,9 @@ impl From for DeltaTableError { DeltaTableError::SerializeLogJson { json_err } } TransactionError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, + other => DeltaTableError::GenericError { + source: Box::new(other), + }, } } } @@ -181,9 +190,9 @@ pub(crate) async fn commit( Ok(_) => { attempt_number += 1; } - Err(_err) => { + Err(err) => { storage.delete(&tmp_commit).await?; - return Err(DeltaTableError::VersionAlreadyExists(version)); + return Err(TransactionError::CommitConflict(err).into()); } }; } @@ -194,8 +203,7 @@ pub(crate) async fn commit( } } - // TODO max attempts error - Err(DeltaTableError::VersionAlreadyExists(-1)) + Err(TransactionError::MaxCommitAttempts(max_attempts as i32).into()) } #[cfg(all(test, feature = "parquet"))] From cee283f2cd5c95e35ba28a3b16ccb7f83c960b6d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 5 Mar 2023 00:52:35 +0100 Subject: [PATCH 45/67] test: start conflict checker tests --- rust/src/action/mod.rs | 12 +++- .../transaction/conflict_checker.rs | 67 +++++++++++++------ 2 files changed, 57 insertions(+), 22 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 4e98242029..869b07a9c7 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -632,7 +632,7 @@ impl DeltaOperation { } } - /// Get predicate expression applien when the operation reads data from the table. + /// Get predicate expression applied when the operation reads data from the table. pub fn read_predicate(&self) -> Option { match self { // TODO add more operations @@ -640,6 +640,16 @@ impl DeltaOperation { _ => None, } } + + /// Denotes if the operation reads the entire table + pub fn read_whole_table(&self) -> bool { + match self { + // TODO just adding one operation example, as currently none of the + // implemented operations scan the entire table. + Self::Write { predicate, .. } if predicate.is_none() => false, + _ => false, + } + } } /// The SaveMode used when performing a DeltaOperation diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index aad87c86b3..3ee74dae5f 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -365,6 +365,11 @@ impl<'a> ConflictChecker<'a> { defaault_isolation_level }; + // Skip check, if the operation can be downgraded to snapshot isolation + if matches!(isolation_level, IsolationLevel::SnapshotIsolation) { + return Ok(()); + } + // Fail if new files have been added that the txn should have read. let added_files_to_check = match isolation_level { IsolationLevel::WriteSerializable @@ -387,7 +392,7 @@ impl<'a> ConflictChecker<'a> { cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { let added_files_matching_predicates = - if let Some(predicate_str) = self.operation.read_predicate() { + if let (Some(predicate_str), false) = (self.operation.read_predicate(), self.operation.read_whole_table()) { let arrow_schema = self.snapshot .arrow_schema() @@ -408,10 +413,19 @@ impl<'a> ConflictChecker<'a> { .cloned() .collect::>() } else { - added_files_to_check + if self.operation.read_whole_table() { + added_files_to_check + } else { + vec![] + } + }; } else { - let added_files_matching_predicates = added_files_to_check; + let added_files_matching_predicates = if self.operation.read_whole_table() { + added_files_to_check + } else { + vec![] + }; } } @@ -592,41 +606,52 @@ mod tests { assert!(!res) } + // Check whether the test transaction conflict with the concurrent writes by executing the + // given params in the following order: + // - setup (including setting table isolation level + // - reads + // - concurrentWrites + // - actions + fn prepare_test( + setup: Option>, + reads: Vec, + concurrent: Vec, + ) -> (DeltaTableState, WinningCommitSummary) { + let setup_actions = setup.unwrap_or_else(|| init_table_actions()); + let mut state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); + state.merge(DeltaTableState::from_actions(reads, 1).unwrap(), true, true); + let summary = WinningCommitSummary { + actions: concurrent, + commit_info: None, + }; + (state, summary) + } + #[tokio::test] // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 async fn test_allowed_concurrent_actions() { - // check( - // "append / append", - // conflicts = false, - // reads = Seq( - // t => t.metadata - // ), - // concurrentWrites = Seq( - // addA), - // actions = Seq( - // addB)) - let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); - + // append - append + // append file to table while a concurrent writer also appends a file let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - let summary = WinningCommitSummary { - actions: vec![file1], - commit_info: None, - }; + let (state, summary) = prepare_test(None, vec![], vec![file1]); let operation = DeltaOperation::Write { mode: SaveMode::Append, partition_by: Default::default(), predicate: None, }; - let actions = vec![file2]; let checker = ConflictChecker::try_new(&state, summary, operation, &actions) .await .unwrap(); let result = checker.check_conflicts(); - println!("{:?}", result); assert!(result.is_ok()); + + // disjoint delete - read + // the concurrent transaction deletes a file that the current transaction did NOT read + + // TODO disjoint transactions } } From 711c59d6b34bc6c642df2283df2ad7f431bc251a Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 5 Mar 2023 01:10:18 +0100 Subject: [PATCH 46/67] chore: cleanup --- .../transaction/conflict_checker.rs | 84 ++++++++++--------- 1 file changed, 45 insertions(+), 39 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 3ee74dae5f..28b65cd957 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -97,14 +97,14 @@ pub(crate) struct TransactionInfo<'a> { pub(crate) actions: &'a Vec, /// read [`DeltaTableState`] used for the transaction pub(crate) read_snapshot: &'a DeltaTableState, - /// [`CommitInfo`] for the commit - pub(crate) commit_info: Option, + /// operation during commit + pub(crate) operation: DeltaOperation, } impl<'a> TransactionInfo<'a> { pub fn try_new( snapshot: &'a DeltaTableState, - operation: &DeltaOperation, + operation: DeltaOperation, actions: &'a Vec, ) -> Result { Ok(Self { @@ -115,7 +115,7 @@ impl<'a> TransactionInfo<'a> { read_app_ids: Default::default(), actions, read_snapshot: snapshot, - commit_info: Some(operation.get_commit_info()), + operation, }) } @@ -132,6 +132,16 @@ impl<'a> TransactionInfo<'a> { pub fn final_actions_to_commit(&self) -> Vec { todo!() } + + /// Denotes if the operation reads the entire table + pub fn read_whole_table(&self) -> bool { + match &self.operation { + // TODO just adding one operation example, as currently none of the + // implemented operations scan the entire table. + DeltaOperation::Write { predicate, .. } if predicate.is_none() => false, + _ => false, + } + } } /// Summary of the Winning commit against which we want to check the conflict @@ -278,8 +288,6 @@ pub(crate) struct ConflictChecker<'a> { winning_commit_summary: WinningCommitSummary, /// The state of the delta table at the base version from the current (not winning) commit snapshot: &'a DeltaTableState, - /// The state of the delta table at the base version from the current (not winning) commit - operation: DeltaOperation, } impl<'a> ConflictChecker<'a> { @@ -289,12 +297,11 @@ impl<'a> ConflictChecker<'a> { operation: DeltaOperation, actions: &'a Vec, ) -> Result, DeltaTableError> { - let transaction_info = TransactionInfo::try_new(snapshot, &operation, actions)?; + let transaction_info = TransactionInfo::try_new(snapshot, operation, actions)?; Ok(Self { transaction_info, winning_commit_summary, snapshot, - operation, }) } @@ -357,7 +364,7 @@ impl<'a> ConflictChecker<'a> { let isolation_level = if can_downgrade_to_snapshot_isolation( &self.winning_commit_summary.actions, - &self.operation, + &self.transaction_info.operation, &defaault_isolation_level, ) { IsolationLevel::SnapshotIsolation @@ -391,37 +398,36 @@ impl<'a> ConflictChecker<'a> { // to assume all files match cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { - let added_files_matching_predicates = - if let (Some(predicate_str), false) = (self.operation.read_predicate(), self.operation.read_whole_table()) { - let arrow_schema = - self.snapshot - .arrow_schema() - .map_err(|err| CommitConflictError::CorruptedState { - source: Box::new(err), - })?; - let predicate = self - .snapshot - .parse_predicate_expression(predicate_str) - .map_err(|err| CommitConflictError::Predicate { - source: Box::new(err), - })?; - AddContainer::new(&added_files_to_check, arrow_schema) - .predicate_matches(predicate) - .map_err(|err| CommitConflictError::Predicate { - source: Box::new(err), - })? - .cloned() - .collect::>() - } else { - if self.operation.read_whole_table() { - added_files_to_check - } else { - vec![] + let added_files_matching_predicates = if let (Some(predicate_str), false) = ( + self.transaction_info.operation.read_predicate(), + self.transaction_info.operation.read_whole_table(), + ) { + let arrow_schema = self.snapshot.arrow_schema().map_err(|err| { + CommitConflictError::CorruptedState { + source: Box::new(err), } - - }; + })?; + let predicate = self + .snapshot + .parse_predicate_expression(predicate_str) + .map_err(|err| CommitConflictError::Predicate { + source: Box::new(err), + })?; + AddContainer::new(&added_files_to_check, arrow_schema) + .predicate_matches(predicate) + .map_err(|err| CommitConflictError::Predicate { + source: Box::new(err), + })? + .cloned() + .collect::>() + } else if self.transaction_info.operation.read_whole_table() { + added_files_to_check + } else { + vec![] + }; } else { - let added_files_matching_predicates = if self.operation.read_whole_table() { + let added_files_matching_predicates = if self.transaction_info.operation.read_whole_table() + { added_files_to_check } else { vec![] @@ -617,7 +623,7 @@ mod tests { reads: Vec, concurrent: Vec, ) -> (DeltaTableState, WinningCommitSummary) { - let setup_actions = setup.unwrap_or_else(|| init_table_actions()); + let setup_actions = setup.unwrap_or_else(init_table_actions); let mut state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); state.merge(DeltaTableState::from_actions(reads, 1).unwrap(), true, true); let summary = WinningCommitSummary { From c209fc9303be9f348ebb69aac32effbc10ac5a07 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 4 Mar 2023 22:03:21 +0100 Subject: [PATCH 47/67] feat: typed commit info --- rust/src/action/mod.rs | 202 +++++++++++++++++++++++---- rust/src/action/parquet2_read/mod.rs | 2 +- rust/src/delta.rs | 72 +++------- rust/src/operations/transaction.rs | 47 ++++--- rust/src/storage/mod.rs | 8 +- rust/src/table_state.rs | 8 +- rust/tests/command_optimize.rs | 12 +- rust/tests/commit_info_format.rs | 16 +-- 8 files changed, 239 insertions(+), 128 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 978a485836..4e98242029 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -8,7 +8,8 @@ mod parquet_read; #[cfg(feature = "parquet2")] pub mod parquet2_read; -use crate::{schema::*, DeltaTableError, DeltaTableMetaData}; +use crate::delta_config::IsolationLevel; +use crate::{schema::*, DeltaResult, DeltaTableError, DeltaTableMetaData}; use percent_encoding::percent_decode; use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; @@ -44,6 +45,13 @@ pub enum ActionError { #[from] source: parquet::errors::ParquetError, }, + /// Faild to serialize operation + #[error("Failed to serialize operation: {source}")] + SerializeOperation { + #[from] + /// The source error + source: serde_json::Error, + }, } fn decode_path(raw_path: &str) -> Result { @@ -435,7 +443,46 @@ pub struct Protocol { pub min_writer_version: DeltaDataTypeInt, } -type CommitInfo = Map; +/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. +/// However the reference implementation as well as delta-rs store useful information that may for instance +/// allow us to be more permissive in commit conflict resolution. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq)] +#[serde(rename_all = "camelCase")] +pub struct CommitInfo { + /// Version number the commit corresponds to + #[serde(skip_serializing_if = "Option::is_none")] + pub version: Option, + /// Timestamp in millis when the commit was created + #[serde(skip_serializing_if = "Option::is_none")] + pub timestamp: Option, + /// Id of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_id: Option, + /// Name of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_name: Option, + /// The operation performed during the + #[serde(skip_serializing_if = "Option::is_none")] + pub operation: Option, + /// Parameters used for table operation + #[serde(skip_serializing_if = "Option::is_none")] + pub operation_parameters: Option>, + /// Version of the table when the operation was started + #[serde(skip_serializing_if = "Option::is_none")] + pub read_version: Option, + /// The isolation level of the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub isolation_level: Option, + /// TODO + #[serde(skip_serializing_if = "Option::is_none")] + pub is_blind_append: Option, + /// Delta engine which created the commit. + #[serde(skip_serializing_if = "Option::is_none")] + pub engine_info: Option, + /// Additional provenance information for the commit + #[serde(flatten, default)] + pub info: Map, +} /// Represents an action in the Delta log. The Delta log is an aggregate of all actions performed /// on the table, so the full list of actions is required to properly read a table. @@ -459,6 +506,16 @@ pub enum Action { commitInfo(CommitInfo), } +impl Action { + /// Create a commit info from a map + pub fn commit_info(info: Map) -> Self { + Self::commitInfo(CommitInfo { + info, + ..Default::default() + }) + } +} + /// Operation performed when creating a new log entry with one or more actions. /// This is a key element of the `CommitInfo` action. #[allow(clippy::large_enum_variant)] @@ -517,45 +574,71 @@ pub enum DeltaOperation { } impl DeltaOperation { - /// Retrieve basic commit information to be added to Delta commits - pub fn get_commit_info(&self) -> Map { - let mut commit_info = Map::::new(); - let operation = match &self { - DeltaOperation::Create { .. } => "delta-rs.Create", - DeltaOperation::Write { .. } => "delta-rs.Write", - DeltaOperation::StreamingUpdate { .. } => "delta-rs.StreamingUpdate", - DeltaOperation::Optimize { .. } => "delta-rs.Optimize", - DeltaOperation::FileSystemCheck { .. } => "delta-rs.FileSystemCheck", - }; - commit_info.insert( - "operation".to_string(), - serde_json::Value::String(operation.into()), - ); + /// A human readable name for the operation + pub fn name(&self) -> &str { + // operation names taken from https://learn.microsoft.com/en-us/azure/databricks/delta/history#--operation-metrics-keys + match &self { + DeltaOperation::Create { mode, .. } if matches!(mode, SaveMode::Overwrite) => { + "CREATE OR REPLACE TABLE" + } + DeltaOperation::Create { .. } => "CREATE TABLE", + DeltaOperation::Write { .. } => "WRITE", + DeltaOperation::StreamingUpdate { .. } => "STREAMING UPDATE", + DeltaOperation::Optimize { .. } => "OPTIMIZE", + DeltaOperation::FileSystemCheck { .. } => "FSCK", + } + } - if let Ok(serde_json::Value::Object(map)) = serde_json::to_value(self) { - let all_operation_fields = map.values().next().unwrap().as_object().unwrap(); - let converted_operation_fields: Map = all_operation_fields - .iter() + /// Paraemters configured for operation. + pub fn operation_parameters(&self) -> DeltaResult> { + // TODO remove unwrap + let serialized = serde_json::to_value(self) + .map_err(|err| ActionError::SerializeOperation { source: err })?; + if let serde_json::Value::Object(map) = serialized { + let all_operation_fields = map.values().next().unwrap().as_object().unwrap().clone(); + Ok(all_operation_fields + .into_iter() .filter(|item| !item.1.is_null()) .map(|(k, v)| { ( - k.clone(), + k, serde_json::Value::String(if v.is_string() { String::from(v.as_str().unwrap()) } else { v.to_string() }), ) - }) - .collect(); + })) + } else { + Err(ActionError::Generic( + "operation parameetrs serialized into unexpected shape".into(), + ) + .into()) + } + } - commit_info.insert( - "operationParameters".to_string(), - serde_json::Value::Object(converted_operation_fields), - ); - }; + /// Denotes if the operation changes the data contained in the table + pub fn changes_data(&self) -> bool { + !matches!(self, Self::Optimize { .. }) + } + + /// Retrieve basic commit information to be added to Delta commits + pub fn get_commit_info(&self) -> CommitInfo { + // TODO infer additional info from operation parameters ... + CommitInfo { + operation: Some(self.name().into()), + operation_parameters: self.operation_parameters().ok().map(|iter| iter.collect()), + ..Default::default() + } + } - commit_info + /// Get predicate expression applien when the operation reads data from the table. + pub fn read_predicate(&self) -> Option { + match self { + // TODO add more operations + Self::Write { predicate, .. } => predicate.clone(), + _ => None, + } } } @@ -654,4 +737,65 @@ mod tests { 1 ); } + + #[test] + fn test_read_commit_info() { + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c" + }"#; + + let info = serde_json::from_str::(raw); + assert!(info.is_ok()); + + println!("{:?}", info); + + // assert that commit info has no required filelds + let raw = "{}"; + let info = serde_json::from_str::(raw); + assert!(info.is_ok()); + + // arbitrary field data may be added to commit + let raw = r#" + { + "timestamp": 1670892998177, + "operation": "WRITE", + "operationParameters": { + "mode": "Append", + "partitionBy": "[\"c1\",\"c2\"]" + }, + "isolationLevel": "Serializable", + "isBlindAppend": true, + "operationMetrics": { + "numFiles": "3", + "numOutputRows": "3", + "numOutputBytes": "1356" + }, + "engineInfo": "Apache-Spark/3.3.1 Delta-Lake/2.2.0", + "txnId": "046a258f-45e3-4657-b0bf-abfb0f76681c", + "additionalField": "more data", + "additionalStruct": { + "key": "value", + "otherKey": 123 + } + }"#; + + let info = serde_json::from_str::(raw).expect("should parse"); + assert!(info.info.contains_key("additionalField")); + assert!(info.info.contains_key("additionalStruct")); + } } diff --git a/rust/src/action/parquet2_read/mod.rs b/rust/src/action/parquet2_read/mod.rs index 34988d5954..ddfd2a9e55 100644 --- a/rust/src/action/parquet2_read/mod.rs +++ b/rust/src/action/parquet2_read/mod.rs @@ -223,7 +223,7 @@ impl ActionVariant for CommitInfo { type Variant = CommitInfo; fn default_action() -> Action { - Action::commitInfo(CommitInfo::new()) + Action::commitInfo(CommitInfo::default()) } fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 783cca2026..ca51bafb6e 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -18,7 +18,7 @@ use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; use crate::delta_config::DeltaConfigError; use crate::operations::vacuum::VacuumBuilder; -use crate::storage::ObjectStoreRef; +use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use chrono::{DateTime, Duration, Utc}; use futures::StreamExt; @@ -560,12 +560,6 @@ impl DeltaTable { self.storage.root_uri() } - /// Return the uri of commit version. - pub fn commit_uri_from_version(&self, version: DeltaDataTypeVersion) -> Path { - let version = format!("{version:020}.json"); - Path::from_iter(["_delta_log", &version]) - } - /// Return the list of paths of given checkpoint. pub fn get_checkpoint_data_paths(&self, check_point: &CheckPoint) -> Vec { let checkpoint_prefix = format!("{:020}", check_point.version); @@ -727,11 +721,7 @@ impl DeltaTable { // scan logs after checkpoint loop { - match self - .storage - .head(&self.commit_uri_from_version(version)) - .await - { + match self.storage.head(&commit_uri_from_version(version)).await { Ok(meta) => { // also cache timestamp for version self.version_timestamp @@ -760,7 +750,7 @@ impl DeltaTable { Ok(version) } - /// Currently loaded evrsion of the table + /// Currently loaded version of the table pub fn version(&self) -> DeltaDataTypeVersion { self.state.version() } @@ -778,7 +768,7 @@ impl DeltaTable { current_version: DeltaDataTypeVersion, ) -> Result { let next_version = current_version + 1; - let commit_uri = self.commit_uri_from_version(next_version); + let commit_uri = commit_uri_from_version(next_version); let commit_log_bytes = self.storage.get(&commit_uri).await; let commit_log_bytes = match commit_log_bytes { Err(ObjectStoreError::NotFound { .. }) => return Ok(PeekCommit::UpToDate), @@ -873,7 +863,7 @@ impl DeltaTable { version: DeltaDataTypeVersion, ) -> Result<(), DeltaTableError> { // check if version is valid - let commit_uri = self.commit_uri_from_version(version); + let commit_uri = commit_uri_from_version(version); match self.storage.head(&commit_uri).await { Ok(_) => {} Err(ObjectStoreError::NotFound { .. }) => { @@ -910,10 +900,7 @@ impl DeltaTable { match self.version_timestamp.get(&version) { Some(ts) => Ok(*ts), None => { - let meta = self - .storage - .head(&self.commit_uri_from_version(version)) - .await?; + let meta = self.storage.head(&commit_uri_from_version(version)).await?; let ts = meta.last_modified.timestamp(); // also cache timestamp for version self.version_timestamp.insert(version, ts); @@ -930,7 +917,7 @@ impl DeltaTable { pub async fn history( &mut self, limit: Option, - ) -> Result>, DeltaTableError> { + ) -> Result, DeltaTableError> { let mut version = match limit { Some(l) => max(self.version() - l as i64 + 1, 0), None => self.get_earliest_delta_log_version().await?, @@ -1085,7 +1072,7 @@ impl DeltaTable { /// Return table schema parsed from transaction log. Return None if table hasn't been loaded or /// no metadata was found in the log. pub fn schema(&self) -> Option<&Schema> { - self.state.current_metadata().map(|m| &m.schema) + self.state.schema() } /// Return table schema parsed from transaction log. Return `DeltaTableError` if table hasn't @@ -1145,7 +1132,7 @@ impl DeltaTable { // move temporary commit file to delta log directory // rely on storage to fail if the file already exists - self.storage - .rename_if_not_exists(&commit.uri, &self.commit_uri_from_version(version)) + .rename_if_not_exists(&commit.uri, &commit_uri_from_version(version)) .await .map_err(|e| match e { ObjectStoreError::AlreadyExists { .. } => { @@ -1181,7 +1168,7 @@ impl DeltaTable { ); let mut actions = vec![ - Action::commitInfo(enriched_commit_info), + Action::commit_info(enriched_commit_info), Action::protocol(protocol), Action::metaData(meta), ]; @@ -1391,22 +1378,17 @@ impl<'a> DeltaTransaction<'a> { .iter() .any(|a| matches!(a, action::Action::commitInfo(..))) { - let mut commit_info = Map::::new(); - commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - commit_info.insert( + let mut extra_info = Map::::new(); + let mut commit_info = operation.map(|op| op.get_commit_info()).unwrap_or_default(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - - if let Some(op) = &operation { - commit_info.append(&mut op.get_commit_info()) - } if let Some(mut meta) = app_metadata { - commit_info.append(&mut meta) + extra_info.append(&mut meta) } + commit_info.info = extra_info; self.add_action(action::Action::commitInfo(commit_info)); } @@ -1661,27 +1643,7 @@ mod tests { assert_eq!(DeltaTableMetaData::try_from(action).unwrap(), delta_md); } Action::commitInfo(action) => { - let mut modified_action = action; - let timestamp = serde_json::Number::from(0i64); - modified_action["timestamp"] = Value::Number(serde_json::Number::from(0i64)); - let mut expected = Map::::new(); - expected.insert( - "operation".to_string(), - serde_json::Value::String("CREATE TABLE".to_string()), - ); - expected.insert( - "userName".to_string(), - serde_json::Value::String("test user".to_string()), - ); - expected.insert( - "delta-rs".to_string(), - serde_json::Value::String(crate_version().to_string()), - ); - expected.insert( - "timestamp".to_string(), - serde_json::Value::Number(timestamp), - ); - assert_eq!(modified_action, expected) + assert_eq!(action.operation, Some("CREATE TABLE".to_string())); } _ => (), } diff --git a/rust/src/operations/transaction.rs b/rust/src/operations/transaction.rs index 974eaeeedc..9198a960b4 100644 --- a/rust/src/operations/transaction.rs +++ b/rust/src/operations/transaction.rs @@ -11,7 +11,7 @@ use serde_json::{Map, Value}; const DELTA_LOG_FOLDER: &str = "_delta_log"; #[derive(thiserror::Error, Debug)] -enum TransactionError { +pub(crate) enum TransactionError { #[error("Tried committing existing table version: {0}")] VersionAlreadyExists(DeltaDataTypeVersion), @@ -62,34 +62,41 @@ fn log_entry_from_actions(actions: &[Action]) -> Result, +pub(crate) fn get_commit_bytes( + operation: &DeltaOperation, + actions: &mut Vec, app_metadata: Option>, -) -> Result { +) -> Result { if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { - let mut commit_info = Map::::new(); - commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - commit_info.insert( + let mut extra_info = Map::::new(); + let mut commit_info = operation.get_commit_info(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( "clientVersion".to_string(), Value::String(format!("delta-rs.{}", crate_version())), ); - commit_info.append(&mut operation.get_commit_info()); if let Some(mut meta) = app_metadata { - commit_info.append(&mut meta) + extra_info.append(&mut meta) } + commit_info.info = extra_info; actions.push(Action::commitInfo(commit_info)); } // Serialize all actions that are part of this log entry. - let log_entry = bytes::Bytes::from(log_entry_from_actions(&actions)?); + Ok(bytes::Bytes::from(log_entry_from_actions(actions)?)) +} + +/// Low-level transaction API. Creates a temporary commit file. Once created, +/// the transaction object could be dropped and the actual commit could be executed +/// with `DeltaTable.try_commit_transaction`. +pub(crate) async fn prepare_commit( + storage: &dyn ObjectStore, + operation: &DeltaOperation, + actions: &mut Vec, + app_metadata: Option>, +) -> Result { + // Serialize all actions that are part of this log entry. + let log_entry = get_commit_bytes(operation, actions, app_metadata)?; // Write delta log entry as temporary file to storage. For the actual commit, // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. @@ -128,11 +135,11 @@ async fn try_commit_transaction( pub(crate) async fn commit( storage: &DeltaObjectStore, version: DeltaDataTypeVersion, - actions: Vec, + mut actions: Vec, operation: DeltaOperation, app_metadata: Option>, ) -> DeltaResult { - let tmp_commit = prepare_commit(storage, operation, actions, app_metadata).await?; + let tmp_commit = prepare_commit(storage, &operation, &mut actions, app_metadata).await?; match try_commit_transaction(storage, &tmp_commit, version).await { Ok(version) => Ok(version), Err(TransactionError::VersionAlreadyExists(version)) => { diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index 360c241a84..99b1ac98a1 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -5,7 +5,7 @@ pub mod file; pub mod utils; use self::config::{ObjectStoreKind, StorageOptions}; -use crate::DeltaResult; +use crate::{DeltaDataTypeVersion, DeltaResult}; use bytes::Bytes; use futures::{stream::BoxStream, StreamExt}; @@ -37,6 +37,12 @@ lazy_static! { static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); } +/// Return the uri of commit version. +pub(crate) fn commit_uri_from_version(version: DeltaDataTypeVersion) -> Path { + let version = format!("{version:020}.json"); + DELTA_LOG_PATH.child(version.as_str()) +} + /// Sharable reference to [`DeltaObjectStore`] pub type ObjectStoreRef = Arc; diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 341e3d8f75..b1808f9452 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -4,6 +4,7 @@ use crate::action::{self, Action, Add}; use crate::delta_config::TableConfig; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; +use crate::storage::commit_uri_from_version; use crate::Schema; use crate::{ ApplyLogError, DeltaDataTypeLong, DeltaDataTypeVersion, DeltaTable, DeltaTableError, @@ -13,7 +14,6 @@ use chrono::Utc; use lazy_static::lazy_static; use object_store::{path::Path, ObjectStore}; use serde::{Deserialize, Serialize}; -use serde_json::{Map, Value}; use std::collections::HashMap; use std::collections::HashSet; use std::convert::TryFrom; @@ -34,7 +34,7 @@ pub struct DeltaTableState { // active files for table state files: Vec, // Information added to individual commits - commit_infos: Vec>, + commit_infos: Vec, app_transaction_version: HashMap, min_reader_version: i32, min_writer_version: i32, @@ -66,7 +66,7 @@ impl DeltaTableState { table: &DeltaTable, version: DeltaDataTypeVersion, ) -> Result { - let commit_uri = table.commit_uri_from_version(version); + let commit_uri = commit_uri_from_version(version); let commit_log_bytes = table.storage.get(&commit_uri).await?.bytes().await?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); @@ -164,7 +164,7 @@ impl DeltaTableState { } /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec> { + pub fn commit_infos(&self) -> &Vec { &self.commit_infos } diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 5006e4a209..9952c00825 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -507,16 +507,14 @@ async fn test_commit_info() -> Result<(), Box> { let last_commit = &commit_info[commit_info.len() - 1]; let commit_metrics = - serde_json::from_value::(last_commit["operationMetrics"].clone())?; + serde_json::from_value::(last_commit.info["operationMetrics"].clone())?; assert_eq!(commit_metrics, metrics); - assert_eq!(last_commit["readVersion"], json!(version)); - assert_eq!( - last_commit["operationParameters"]["targetSize"], - json!("2000000") - ); + assert_eq!(last_commit.read_version, Some(version)); + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["targetSize"], json!("2000000")); // TODO: Requires a string representation for PartitionFilter - assert_eq!(last_commit["operationParameters"]["predicate"], Value::Null); + // assert_eq!(parameters["predicate"], None); Ok(()) } diff --git a/rust/tests/commit_info_format.rs b/rust/tests/commit_info_format.rs index 8a04702579..60a8bf5ee4 100644 --- a/rust/tests/commit_info_format.rs +++ b/rust/tests/commit_info_format.rs @@ -2,8 +2,7 @@ mod fs_common; use deltalake::action::{Action, DeltaOperation, SaveMode}; - -use serde_json::{json, Value}; +use serde_json::json; use std::error::Error; use tempdir::TempDir; @@ -27,15 +26,10 @@ async fn test_operational_parameters() -> Result<(), Box> { let commit_info = table.history(None).await?; let last_commit = &commit_info[commit_info.len() - 1]; - - assert_eq!(last_commit["operationParameters"]["mode"], json!("Append")); - - assert_eq!( - last_commit["operationParameters"]["partitionBy"], - json!("[\"some_partition\"]") - ); - - assert_eq!(last_commit["operationParameters"]["predicate"], Value::Null); + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["mode"], json!("Append")); + assert_eq!(parameters["partitionBy"], json!("[\"some_partition\"]")); + // assert_eq!(parameters["predicate"], None); Ok(()) } From 3bf44f7dad11b348fb13e13bc122ba1e8a2792b0 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 7 Mar 2023 18:21:01 +0100 Subject: [PATCH 48/67] refactor: pass objectore as a ref to commit --- rust/src/operations/create.rs | 2 +- rust/src/operations/filesystem_check.rs | 2 +- rust/src/operations/optimize.rs | 2 +- .../operations/transaction/conflict_checker.rs | 2 +- rust/src/operations/transaction/mod.rs | 17 +++++++---------- rust/src/operations/transaction/test_utils.rs | 4 ++-- rust/src/operations/write.rs | 2 +- 7 files changed, 14 insertions(+), 17 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index ad47cff517..9384aa4767 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -300,7 +300,7 @@ impl std::future::IntoFuture for CreateBuilder { } } let version = commit( - table.object_store(), + table.object_store().as_ref(), &actions, operation, &table.state, diff --git a/rust/src/operations/filesystem_check.rs b/rust/src/operations/filesystem_check.rs index ea3e08b4e7..26be9e9113 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/rust/src/operations/filesystem_check.rs @@ -147,7 +147,7 @@ impl FileSystemCheckPlan { } commit( - self.store, + self.store.as_ref(), &actions, DeltaOperation::FileSystemCheck {}, snapshot, diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 26479bdb6a..f3a5c5f68d 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -371,7 +371,7 @@ impl MergePlan { } commit( - object_store, + object_store.as_ref(), &actions, self.input_parameters.into(), snapshot, diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 28b65cd957..b83b3f5b7d 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -152,7 +152,7 @@ pub(crate) struct WinningCommitSummary { impl WinningCommitSummary { pub async fn try_new( - object_store: &DeltaObjectStore, + object_store: &dyn ObjectStore, read_version: DeltaDataTypeVersion, winning_commit_version: DeltaDataTypeVersion, ) -> DeltaResult { diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 2996f6d667..b45a8c989c 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -1,6 +1,6 @@ //! Delta transactions use crate::action::{Action, CommitInfo, DeltaOperation}; -use crate::storage::{commit_uri_from_version, ObjectStoreRef}; +use crate::storage::commit_uri_from_version; use crate::table_state::DeltaTableState; use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; use chrono::Utc; @@ -150,7 +150,7 @@ async fn try_commit_transaction( } pub(crate) async fn commit( - storage: ObjectStoreRef, + storage: &dyn ObjectStore, actions: &Vec, operation: DeltaOperation, read_snapshot: &DeltaTableState, @@ -171,22 +171,19 @@ pub(crate) async fn commit( let only_add_files = false; let _is_blind_append = only_add_files && !depends_on_files; - let tmp_commit = prepare_commit(storage.as_ref(), &operation, actions, app_metadata).await?; + let tmp_commit = prepare_commit(storage, &operation, actions, app_metadata).await?; let max_attempts = 5; let mut attempt_number = 1; while attempt_number <= max_attempts { let version = read_snapshot.version() + attempt_number; - match try_commit_transaction(storage.as_ref(), &tmp_commit, version).await { + match try_commit_transaction(storage, &tmp_commit, version).await { Ok(version) => return Ok(version), Err(TransactionError::VersionAlreadyExists(version)) => { - let summary = WinningCommitSummary::try_new( - storage.as_ref(), - read_snapshot.version(), - version, - ) - .await?; + let summary = + WinningCommitSummary::try_new(storage, read_snapshot.version(), version) + .await?; let conflict_checker = ConflictChecker::try_new(read_snapshot, summary, operation.clone(), actions) .await?; diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 45ee1efc01..db981e1c29 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -118,8 +118,8 @@ pub async fn create_initialized_table( configuration.unwrap_or_default(), ), }; - let mut actions = init_table_actions(); - let prepared_commit = prepare_commit(storage.as_ref(), &operation, &mut actions, None) + let actions = init_table_actions(); + let prepared_commit = prepare_commit(storage.as_ref(), &operation, &actions, None) .await .unwrap(); try_commit_transaction(storage.as_ref(), &prepared_commit, 0) diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index c46500c079..203e0f53f9 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -446,7 +446,7 @@ impl std::future::IntoFuture for WriteBuilder { predicate: this.predicate, }; let _version = commit( - table.storage.clone(), + table.storage.as_ref(), &actions, operation, &table.state, From 7cfdde38f8917b61128b723710021009197087d4 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 10 Mar 2023 00:13:10 +0100 Subject: [PATCH 49/67] test: check concurrent delete in fsck --- rust/src/delta.rs | 7 +++++ .../transaction/conflict_checker.rs | 8 ++--- rust/src/operations/transaction/mod.rs | 8 ++--- rust/tests/command_filesystem_check.rs | 30 ++++++++++++++++++- 4 files changed, 43 insertions(+), 10 deletions(-) diff --git a/rust/src/delta.rs b/rust/src/delta.rs index ca51bafb6e..f1ad7315fd 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -17,6 +17,7 @@ use super::schema::*; use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; use crate::delta_config::DeltaConfigError; +use crate::operations::transaction::TransactionError; use crate::operations::vacuum::VacuumBuilder; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; @@ -210,6 +211,12 @@ pub enum DeltaTableError { #[from] source: std::io::Error, }, + /// Error raised while commititng transaction + #[error("Transaction failed: {source}")] + Transaction { + /// The source error + source: TransactionError, + }, /// Error returned when transaction is failed to be committed because given version already exists. #[error("Delta transaction failed, version {0} already exists.")] VersionAlreadyExists(DeltaDataTypeVersion), diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index b83b3f5b7d..c40881dcec 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -89,8 +89,6 @@ pub(crate) struct TransactionInfo<'a> { pub(crate) read_predicates: Vec, /// files that have been seen by the transaction pub(crate) read_files: HashSet, - /// whether the whole table was read during the transaction - pub(crate) read_whole_table: bool, /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit @@ -111,7 +109,6 @@ impl<'a> TransactionInfo<'a> { txn_id: "".into(), read_predicates: vec![], read_files: Default::default(), - read_whole_table: true, read_app_ids: Default::default(), actions, read_snapshot: snapshot, @@ -133,7 +130,7 @@ impl<'a> TransactionInfo<'a> { todo!() } - /// Denotes if the operation reads the entire table + /// Whether the whole table was read during the transaction pub fn read_whole_table(&self) -> bool { match &self.operation { // TODO just adding one operation example, as currently none of the @@ -463,7 +460,7 @@ impl<'a> ConflictChecker<'a> { .find(|f| read_file_path.contains(&f.path)); if deleted_read_overlap.is_some() || (!self.winning_commit_summary.removed_files().is_empty() - && self.current_transaction_info().read_whole_table) + && self.current_transaction_info().read_whole_table()) { Err(CommitConflictError::ConcurrentDeleteRead) } else { @@ -497,6 +494,7 @@ impl<'a> ConflictChecker<'a> { let intersection: HashSet<&String> = txn_deleted_files .intersection(&winning_deleted_files) .collect(); + if !intersection.is_empty() { Err(CommitConflictError::ConcurrentDeleteDelete) } else { diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index b45a8c989c..eba3308e39 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -19,8 +19,10 @@ use self::conflict_checker::{CommitConflictError, WinningCommitSummary}; const DELTA_LOG_FOLDER: &str = "_delta_log"; +/// Error raised while commititng transaction #[derive(thiserror::Error, Debug)] -pub(crate) enum TransactionError { +pub enum TransactionError { + /// Version already exists #[error("Tried committing existing table version: {0}")] VersionAlreadyExists(DeltaDataTypeVersion), @@ -56,9 +58,7 @@ impl From for DeltaTableError { DeltaTableError::SerializeLogJson { json_err } } TransactionError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, - other => DeltaTableError::GenericError { - source: Box::new(other), - }, + other => DeltaTableError::Transaction { source: other }, } } } diff --git a/rust/tests/command_filesystem_check.rs b/rust/tests/command_filesystem_check.rs index afc6297640..06584cc077 100644 --- a/rust/tests/command_filesystem_check.rs +++ b/rust/tests/command_filesystem_check.rs @@ -113,7 +113,7 @@ async fn test_filesystem_check_partitioned() -> TestResult { #[tokio::test] #[serial] -async fn test_filesystem_check_outdated() -> TestResult { +async fn test_filesystem_check_fails_for_concurrent_delete() -> TestResult { // Validate failure when a non dry only executes on the latest version let context = IntegrationContext::new(StorageIntegration::Local)?; context.load_table(TestTables::Simple).await?; @@ -132,6 +132,34 @@ async fn test_filesystem_check_outdated() -> TestResult { let op = DeltaOps::from(table); let res = op.filesystem_check().with_dry_run(false).await; + // TODO check more specific error + assert!(matches!(res, Err(DeltaTableError::Transaction { .. }))); + + Ok(()) +} + +#[tokio::test] +#[serial] +#[ignore = "should this actually fail? with conflcit resolution, we are re-trying again."] +async fn test_filesystem_check_outdated() -> TestResult { + // Validate failure when a non dry only executes on the latest version + let context = IntegrationContext::new(StorageIntegration::Local)?; + context.load_table(TestTables::Simple).await?; + let file = "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"; + let path = Path::from_iter([&TestTables::Simple.as_name(), file]); + + // Delete an active file from underlying storage without an update to the log to simulate an external fault + context.object_store().delete(&path).await?; + + let table = context + .table_builder(TestTables::Simple) + .with_version(2) + .load() + .await?; + + let op = DeltaOps::from(table); + let res = op.filesystem_check().with_dry_run(false).await; + println!("{:?}", res); if let Err(DeltaTableError::VersionAlreadyExists(version)) = res { assert!(version == 3); } else { From b4dc95eafc9a30c39bc6e8639a36d1b21e99b83e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 10 Mar 2023 00:22:42 +0100 Subject: [PATCH 50/67] chore: clippy --- rust/src/builder.rs | 8 +++----- rust/src/writer/record_batch.rs | 1 - 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/rust/src/builder.rs b/rust/src/builder.rs index 072403e687..70d276b22a 100644 --- a/rust/src/builder.rs +++ b/rust/src/builder.rs @@ -38,8 +38,10 @@ impl From for DeltaTableError { /// possible version specifications for loading a delta table #[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Default)] pub enum DeltaVersion { /// load the newest version + #[default] Newest, /// specify the version to load Version(DeltaDataTypeVersion), @@ -47,11 +49,7 @@ pub enum DeltaVersion { Timestamp(DateTime), } -impl Default for DeltaVersion { - fn default() -> Self { - DeltaVersion::Newest - } -} + /// Configuration options for delta table #[derive(Debug, Serialize, Deserialize)] diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index d5a0c1a320..4e0bd38833 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -382,7 +382,6 @@ pub(crate) fn divide_by_partition_values( // get row indices for current partition let idx: UInt32Array = (range.start..range.end) .map(|i| Some(indices.value(i))) - .into_iter() .collect(); let partition_key_iter = sorted_partition_columns.iter().map(|c| { From 906fa7ca3878bdc04bb44fc138e368c4cea0862d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 10 Mar 2023 00:24:53 +0100 Subject: [PATCH 51/67] chore: fmt --- rust/src/builder.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/rust/src/builder.rs b/rust/src/builder.rs index 70d276b22a..ed2f207ddf 100644 --- a/rust/src/builder.rs +++ b/rust/src/builder.rs @@ -37,8 +37,7 @@ impl From for DeltaTableError { } /// possible version specifications for loading a delta table -#[derive(Debug, Clone, PartialEq, Eq)] -#[derive(Default)] +#[derive(Debug, Clone, PartialEq, Eq, Default)] pub enum DeltaVersion { /// load the newest version #[default] @@ -49,8 +48,6 @@ pub enum DeltaVersion { Timestamp(DateTime), } - - /// Configuration options for delta table #[derive(Debug, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] From 2378123122567755a86e8165979e33014eaaa24c Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 10 Mar 2023 07:05:45 +0100 Subject: [PATCH 52/67] style: import order --- rust/src/delta_datafusion.rs | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index bf16f69da3..47ca4b50e9 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -20,11 +20,12 @@ //! }; //! ``` -use crate::builder::ensure_table_uri; -use crate::Invariant; -use crate::{action, open_table, open_table_with_storage_options}; -use crate::{schema, DeltaResult, DeltaTableBuilder}; -use crate::{DeltaTable, DeltaTableError, SchemaDataType}; +use std::any::Any; +use std::collections::HashMap; +use std::convert::TryFrom; +use std::fmt::Debug; +use std::sync::Arc; + use arrow::array::ArrayRef; use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, SchemaRef, TimeUnit}; @@ -54,13 +55,14 @@ use datafusion_expr::{Expr, Extension, LogicalPlan, TableProviderFilterPushDown} use datafusion_proto::logical_plan::LogicalExtensionCodec; use datafusion_proto::physical_plan::PhysicalExtensionCodec; use object_store::{path::Path, ObjectMeta}; -use std::any::Any; -use std::collections::HashMap; -use std::convert::TryFrom; -use std::fmt::Debug; -use std::sync::Arc; use url::Url; +use crate::builder::ensure_table_uri; +use crate::Invariant; +use crate::{action, open_table, open_table_with_storage_options}; +use crate::{schema, DeltaResult, DeltaTableBuilder}; +use crate::{DeltaTable, DeltaTableError, SchemaDataType}; + impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { match err { From f7ab7494ec8c25e9fce8a7e65019e3da80355a70 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 10 Mar 2023 07:48:20 +0100 Subject: [PATCH 53/67] chore: adopt fixed pruning approach --- rust/src/delta_datafusion.rs | 11 +- .../transaction/conflict_checker.rs | 40 +++++- rust/src/operations/transaction/mod.rs | 9 +- rust/src/operations/transaction/state.rs | 122 ++++++++++++------ 4 files changed, 129 insertions(+), 53 deletions(-) diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 47ca4b50e9..91f4ba5e00 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -42,8 +42,7 @@ use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::optimizer::utils::conjunction; use datafusion::physical_expr::PhysicalSortExpr; -use datafusion::physical_optimizer::pruning::PruningPredicate; -use datafusion::physical_optimizer::pruning::PruningStatistics; +use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion::physical_plan::file_format::{partition_type_wrap, FileScanConfig}; use datafusion::physical_plan::{ ColumnStatistics, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -58,10 +57,10 @@ use object_store::{path::Path, ObjectMeta}; use url::Url; use crate::builder::ensure_table_uri; -use crate::Invariant; -use crate::{action, open_table, open_table_with_storage_options}; -use crate::{schema, DeltaResult, DeltaTableBuilder}; -use crate::{DeltaTable, DeltaTableError, SchemaDataType}; +use crate::{ + action, open_table, open_table_with_storage_options, schema, DeltaResult, DeltaTable, + DeltaTableBuilder, DeltaTableError, Invariant, SchemaDataType, +}; impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index c40881dcec..92462f0642 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -80,6 +80,10 @@ pub enum CommitConflictError { /// Source error source: Box, }, + + /// Error returned when no metadata was found in the DeltaTable. + #[error("No metadata found, please make sure table is loaded.")] + NoMetadata, } /// A struct representing different attributes of current transaction needed for conflict detection. @@ -410,7 +414,13 @@ impl<'a> ConflictChecker<'a> { .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), })?; - AddContainer::new(&added_files_to_check, arrow_schema) + // TODO remove unwrap + let partition_columns = &self + .snapshot + .current_metadata() + .ok_or(CommitConflictError::NoMetadata)? + .partition_columns; + AddContainer::new(&added_files_to_check, partition_columns, arrow_schema) .predicate_matches(predicate) .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), @@ -658,4 +668,32 @@ mod tests { // TODO disjoint transactions } + + #[tokio::test] + // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 + async fn test_disallowed_concurrent_actions() { + // delete - delete + // append file to table while a concurrent writer also appends a file + let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); + let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); + + let (state, summary) = prepare_test(None, vec![], vec![file1]); + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: Default::default(), + predicate: None, + }; + let actions = vec![file2]; + let checker = ConflictChecker::try_new(&state, summary, operation, &actions) + .await + .unwrap(); + + let result = checker.check_conflicts(); + assert!(result.is_ok()); + + // disjoint delete - read + // the concurrent transaction deletes a file that the current transaction did NOT read + + // TODO disjoint transactions + } } diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index eba3308e39..cd0087aae8 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -1,14 +1,15 @@ //! Delta transactions -use crate::action::{Action, CommitInfo, DeltaOperation}; -use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; -use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; use chrono::Utc; use conflict_checker::ConflictChecker; use object_store::path::Path; use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::{Map, Value}; +use crate::action::{Action, CommitInfo, DeltaOperation}; +use crate::storage::commit_uri_from_version; +use crate::table_state::DeltaTableState; +use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; + mod conflict_checker; #[cfg(feature = "datafusion")] mod state; diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 898dba23aa..76c39816f8 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -1,8 +1,6 @@ -use crate::action::Add; -use crate::delta_datafusion::to_correct_scalar_value; -use crate::table_state::DeltaTableState; -use crate::DeltaResult; -use crate::{schema, DeltaTableError}; +use std::convert::TryFrom; +use std::sync::Arc; + use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use datafusion::optimizer::utils::conjunction; @@ -16,8 +14,12 @@ use itertools::Either; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; use sqlparser::tokenizer::Tokenizer; -use std::convert::TryFrom; -use std::sync::Arc; + +use crate::action::Add; +use crate::delta_datafusion::to_correct_scalar_value; +use crate::table_state::DeltaTableState; +use crate::DeltaResult; +use crate::{schema, DeltaTableError}; impl DeltaTableState { /// Get the table schema as an [`ArrowSchemaRef`] @@ -84,18 +86,58 @@ impl DeltaTableState { pub struct AddContainer<'a> { inner: &'a Vec, + partition_columns: &'a Vec, schema: ArrowSchemaRef, } impl<'a> AddContainer<'a> { /// Create a new instance of [`AddContainer`] - pub fn new(adds: &'a Vec, schema: ArrowSchemaRef) -> Self { + pub fn new( + adds: &'a Vec, + partition_columns: &'a Vec, + schema: ArrowSchemaRef, + ) -> Self { Self { inner: adds, + partition_columns, schema, } } + pub fn get_prune_stats(&self, column: &Column, get_max: bool) -> Option { + let (_, field) = self.schema.column_with_name(&column.name)?; + + // See issue 1214. Binary type does not support natural order which is required for Datafusion to prune + if field.data_type() == &DataType::Binary { + return None; + } + + let values = self.inner.iter().map(|add| { + if self.partition_columns.contains(&column.name) { + let value = add.partition_values.get(&column.name).unwrap(); + let value = match value { + Some(v) => serde_json::Value::String(v.to_string()), + None => serde_json::Value::Null, + }; + to_correct_scalar_value(&value, field.data_type()).unwrap_or(ScalarValue::Null) + } else if let Ok(Some(statistics)) = add.get_stats() { + let values = if get_max { + statistics.max_values + } else { + statistics.min_values + }; + + values + .get(&column.name) + .and_then(|f| to_correct_scalar_value(f.as_value()?, field.data_type())) + .unwrap_or(ScalarValue::Null) + } else { + ScalarValue::Null + } + }); + ScalarValue::iter_to_array(values).ok() + } + /// Get an iterator of add actions / files, that MAY containtain data mathcing the predicate. /// /// Expressions are evaluated for file statistics, essentially column-wise min max bounds, @@ -123,37 +165,13 @@ impl<'a> PruningStatistics for AddContainer<'a> { /// return the minimum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows fn min_values(&self, column: &Column) -> Option { - let data_type = self.schema.field_with_name(&column.name).ok()?.data_type(); - let values = self.inner.iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - statistics - .min_values - .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, data_type)) - .unwrap_or(ScalarValue::Null) - } else { - ScalarValue::Null - } - }); - ScalarValue::iter_to_array(values).ok() + self.get_prune_stats(column, false) } /// return the maximum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows. fn max_values(&self, column: &Column) -> Option { - let data_type = self.schema.field_with_name(&column.name).ok()?.data_type(); - let values = self.inner.iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - statistics - .max_values - .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, data_type)) - .unwrap_or(ScalarValue::Null) - } else { - ScalarValue::Null - } - }); - ScalarValue::iter_to_array(values).ok() + self.get_prune_stats(column, true) } /// return the number of containers (e.g. row groups) being @@ -169,11 +187,25 @@ impl<'a> PruningStatistics for AddContainer<'a> { fn null_counts(&self, column: &Column) -> Option { let values = self.inner.iter().map(|add| { if let Ok(Some(statistics)) = add.get_stats() { - statistics - .null_count - .get(&column.name) - .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) - .unwrap_or(ScalarValue::UInt64(None)) + if self.partition_columns.contains(&column.name) { + let value = add.partition_values.get(&column.name).unwrap(); + match value { + Some(_) => ScalarValue::UInt64(Some(0)), + None => ScalarValue::UInt64(Some(statistics.num_records as u64)), + } + } else { + statistics + .null_count + .get(&column.name) + .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) + .unwrap_or(ScalarValue::UInt64(None)) + } + } else if self.partition_columns.contains(&column.name) { + let value = add.partition_values.get(&column.name).unwrap(); + match value { + Some(_) => ScalarValue::UInt64(Some(0)), + None => ScalarValue::UInt64(None), + } } else { ScalarValue::UInt64(None) } @@ -186,14 +218,18 @@ impl PruningStatistics for DeltaTableState { /// return the minimum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows fn min_values(&self, column: &Column) -> Option { - let container = AddContainer::new(self.files(), self.arrow_schema().ok()?); + let partition_columns = &self.current_metadata()?.partition_columns; + let container = + AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); container.min_values(column) } /// return the maximum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows. fn max_values(&self, column: &Column) -> Option { - let container = AddContainer::new(self.files(), self.arrow_schema().ok()?); + let partition_columns = &self.current_metadata()?.partition_columns; + let container = + AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); container.max_values(column) } @@ -208,7 +244,9 @@ impl PruningStatistics for DeltaTableState { /// /// Note: the returned array must contain `num_containers()` rows. fn null_counts(&self, column: &Column) -> Option { - let container = AddContainer::new(self.files(), self.arrow_schema().ok()?); + let partition_columns = &self.current_metadata()?.partition_columns; + let container = + AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); container.null_counts(column) } } From 4c61b1eb9c4c460344833200027ad6ec4ee5eda1 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 10 Mar 2023 08:29:30 +0100 Subject: [PATCH 54/67] test: add delete-delete test --- rust/src/action/mod.rs | 12 ++++++++- .../transaction/conflict_checker.rs | 27 +++++++------------ rust/src/operations/transaction/test_utils.rs | 10 ++++++- 3 files changed, 29 insertions(+), 20 deletions(-) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index ccfa32a902..732b2c24c2 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -544,6 +544,13 @@ pub enum DeltaOperation { /// The predicate used during the write. predicate: Option, }, + + /// Delete data matching predicate from delta table + Delete { + /// The condition the to be deleted data must match + predicate: Option, + }, + /// Represents a Delta `StreamingUpdate` operation. #[serde(rename_all = "camelCase")] StreamingUpdate { @@ -580,6 +587,7 @@ impl DeltaOperation { } DeltaOperation::Create { .. } => "CREATE TABLE", DeltaOperation::Write { .. } => "WRITE", + DeltaOperation::Delete { .. } => "DELETE", DeltaOperation::StreamingUpdate { .. } => "STREAMING UPDATE", DeltaOperation::Optimize { .. } => "OPTIMIZE", DeltaOperation::FileSystemCheck { .. } => "FSCK", @@ -622,7 +630,8 @@ impl DeltaOperation { Self::Create { .. } | Self::FileSystemCheck {} | Self::StreamingUpdate { .. } - | Self::Write { .. } => true, + | Self::Write { .. } + | Self::Delete { .. } => true, } } @@ -641,6 +650,7 @@ impl DeltaOperation { match self { // TODO add more operations Self::Write { predicate, .. } => predicate.clone(), + Self::Delete { predicate, .. } => predicate.clone(), _ => None, } } diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 92462f0642..d44c937305 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -673,27 +673,18 @@ mod tests { // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 async fn test_disallowed_concurrent_actions() { // delete - delete - // append file to table while a concurrent writer also appends a file - let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); - let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - - let (state, summary) = prepare_test(None, vec![], vec![file1]); - let operation = DeltaOperation::Write { - mode: SaveMode::Append, - partition_by: Default::default(), - predicate: None, - }; - let actions = vec![file2]; + // remove file from table that has previously been removed + let removed_file = tu::create_remove_action("removed_file", true); + let (state, summary) = prepare_test(None, vec![], vec![removed_file.clone()]); + let operation = DeltaOperation::Delete { predicate: None }; + let actions = vec![removed_file]; let checker = ConflictChecker::try_new(&state, summary, operation, &actions) .await .unwrap(); - let result = checker.check_conflicts(); - assert!(result.is_ok()); - - // disjoint delete - read - // the concurrent transaction deletes a file that the current transaction did NOT read - - // TODO disjoint transactions + assert!(matches!( + checker.check_conflicts(), + Err(CommitConflictError::ConcurrentDeleteDelete) + )); } } diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index db981e1c29..a22de38aa8 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -1,5 +1,5 @@ use super::{prepare_commit, try_commit_transaction, CommitInfo}; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, SaveMode}; +use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; use crate::table_state::DeltaTableState; use crate::{ DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, @@ -20,6 +20,14 @@ pub fn create_add_action( }) } +pub fn create_remove_action(path: impl Into, data_change: bool) -> Action { + Action::remove(Remove { + path: path.into(), + data_change, + ..Default::default() + }) +} + pub fn init_table_actions() -> Vec { let protocol = Protocol { min_reader_version: crate::operations::MAX_SUPPORTED_READER_VERSION, From 88072e9f6f37543f3eac6cddc74d81dfce69003f Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 11 Mar 2023 10:13:52 +0100 Subject: [PATCH 55/67] refactor: make ConflictChecker independent of operation --- .../transaction/conflict_checker.rs | 164 ++++++++++-------- rust/src/operations/transaction/mod.rs | 11 +- 2 files changed, 99 insertions(+), 76 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index d44c937305..f9861e5382 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -3,6 +3,7 @@ use std::collections::HashSet; use std::io::{BufRead, BufReader, Cursor}; +use itertools::Either; use object_store::ObjectStore; use serde_json::{Map, Value}; @@ -86,62 +87,86 @@ pub enum CommitConflictError { NoMetadata, } +fn read_whole_table(operation: &DeltaOperation) -> bool { + match operation { + // TODO just adding one operation example, as currently none of the + // implemented operations scan the entire table. + DeltaOperation::Write { predicate, .. } if predicate.is_none() => false, + _ => false, + } +} + /// A struct representing different attributes of current transaction needed for conflict detection. pub(crate) struct TransactionInfo<'a> { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction - pub(crate) read_predicates: Vec, - /// files that have been seen by the transaction - pub(crate) read_files: HashSet, + pub(crate) read_predicates: Option, /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit pub(crate) actions: &'a Vec, /// read [`DeltaTableState`] used for the transaction pub(crate) read_snapshot: &'a DeltaTableState, - /// operation during commit - pub(crate) operation: DeltaOperation, } impl<'a> TransactionInfo<'a> { - pub fn try_new( + pub fn new( snapshot: &'a DeltaTableState, - operation: DeltaOperation, + read_predicates: Option, actions: &'a Vec, - ) -> Result { - Ok(Self { + ) -> Self { + Self { txn_id: "".into(), - read_predicates: vec![], - read_files: Default::default(), + read_predicates, read_app_ids: Default::default(), actions, read_snapshot: snapshot, - operation, - }) + } } pub fn metadata(&self) -> Option<&DeltaTableMetaData> { self.read_snapshot.current_metadata() } + /// Whether the transaction changed the tables metadatas pub fn metadata_changed(&self) -> bool { self.actions .iter() .any(|a| matches!(a, Action::metaData(_))) } - pub fn final_actions_to_commit(&self) -> Vec { - todo!() + #[cfg(feature = "datafusion")] + /// Files read by the transaction + pub fn read_files(&self) -> Result, CommitConflictError> { + if let Some(predicate) = &self.read_predicates { + let pred = self + .read_snapshot + .parse_predicate_expression(predicate) + .map_err(|err| CommitConflictError::Predicate { + source: Box::new(err), + })?; + Ok(Either::Left( + self.read_snapshot + .files_matching_predicate(&[pred]) + .map_err(|err| CommitConflictError::Predicate { + source: Box::new(err), + })?, + )) + } else { + Ok(Either::Right(self.read_snapshot.files().iter())) + } + } + + #[cfg(not(feature = "datafusion"))] + /// Files read by the transaction + pub fn read_files(&self) -> Result, CommitConflictError> { + Ok(self.read_snapshot.files().iter()) } /// Whether the whole table was read during the transaction pub fn read_whole_table(&self) -> bool { - match &self.operation { - // TODO just adding one operation example, as currently none of the - // implemented operations scan the entire table. - DeltaOperation::Write { predicate, .. } if predicate.is_none() => false, - _ => false, - } + // TODO actually check + self.read_predicates.is_some() } } @@ -284,31 +309,46 @@ impl WinningCommitSummary { /// Checks if a failed commit may be committed after a conflicting winning commit pub(crate) struct ConflictChecker<'a> { /// transaction information for current transaction at start of check - transaction_info: TransactionInfo<'a>, + txn_info: TransactionInfo<'a>, /// Summary of the transaction, that has been committed ahead of the current transaction winning_commit_summary: WinningCommitSummary, - /// The state of the delta table at the base version from the current (not winning) commit - snapshot: &'a DeltaTableState, + /// Isolation level for the current transaction + isolation_level: IsolationLevel, } impl<'a> ConflictChecker<'a> { - pub async fn try_new( - snapshot: &'a DeltaTableState, + pub fn new( + read_snapshot: &'a DeltaTableState, winning_commit_summary: WinningCommitSummary, - operation: DeltaOperation, + read_predicates: Option, actions: &'a Vec, - ) -> Result, DeltaTableError> { - let transaction_info = TransactionInfo::try_new(snapshot, operation, actions)?; - Ok(Self { - transaction_info, + operation: Option<&DeltaOperation>, + ) -> ConflictChecker<'a> { + let transaction_info = TransactionInfo::new(read_snapshot, read_predicates, actions); + let isolation_level = operation + .and_then(|op| { + if can_downgrade_to_snapshot_isolation( + &winning_commit_summary.actions, + op, + &read_snapshot.table_config().isolation_level(), + ) { + Some(IsolationLevel::SnapshotIsolation) + } else { + None + } + }) + .unwrap_or_else(|| read_snapshot.table_config().isolation_level()); + + Self { + txn_info: transaction_info, winning_commit_summary, - snapshot, - }) + isolation_level, + } } fn current_transaction_info(&self) -> &TransactionInfo { // TODO figure out when we need to update this - &self.transaction_info + &self.txn_info } /// This function checks conflict of the `initial_current_transaction_info` against the @@ -328,8 +368,8 @@ impl<'a> ConflictChecker<'a> { /// to read and write against the protocol set by the committed transaction. fn check_protocol_compatibility(&self) -> Result<(), CommitConflictError> { for p in self.winning_commit_summary.protocol() { - if self.snapshot.min_reader_version() < p.min_reader_version - || self.snapshot.min_writer_version() < p.min_writer_version + if self.txn_info.read_snapshot.min_reader_version() < p.min_reader_version + || self.txn_info.read_snapshot.min_writer_version() < p.min_writer_version { return Err(CommitConflictError::ProtocolChanged); }; @@ -361,25 +401,13 @@ impl<'a> ConflictChecker<'a> { fn check_for_added_files_that_should_have_been_read_by_current_txn( &self, ) -> Result<(), CommitConflictError> { - let defaault_isolation_level = self.snapshot.table_config().isolation_level(); - - let isolation_level = if can_downgrade_to_snapshot_isolation( - &self.winning_commit_summary.actions, - &self.transaction_info.operation, - &defaault_isolation_level, - ) { - IsolationLevel::SnapshotIsolation - } else { - defaault_isolation_level - }; - // Skip check, if the operation can be downgraded to snapshot isolation - if matches!(isolation_level, IsolationLevel::SnapshotIsolation) { + if matches!(self.isolation_level, IsolationLevel::SnapshotIsolation) { return Ok(()); } // Fail if new files have been added that the txn should have read. - let added_files_to_check = match isolation_level { + let added_files_to_check = match self.isolation_level { IsolationLevel::WriteSerializable if !self.current_transaction_info().metadata_changed() => { @@ -400,23 +428,25 @@ impl<'a> ConflictChecker<'a> { cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { let added_files_matching_predicates = if let (Some(predicate_str), false) = ( - self.transaction_info.operation.read_predicate(), - self.transaction_info.operation.read_whole_table(), + &self.txn_info.read_predicates, + self.txn_info.read_whole_table(), ) { - let arrow_schema = self.snapshot.arrow_schema().map_err(|err| { + let arrow_schema = self.txn_info.read_snapshot.arrow_schema().map_err(|err| { CommitConflictError::CorruptedState { source: Box::new(err), } })?; let predicate = self - .snapshot + .txn_info + .read_snapshot .parse_predicate_expression(predicate_str) .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), })?; // TODO remove unwrap let partition_columns = &self - .snapshot + .txn_info + .read_snapshot .current_metadata() .ok_or(CommitConflictError::NoMetadata)? .partition_columns; @@ -427,13 +457,13 @@ impl<'a> ConflictChecker<'a> { })? .cloned() .collect::>() - } else if self.transaction_info.operation.read_whole_table() { + } else if self.txn_info.read_whole_table() { added_files_to_check } else { vec![] }; } else { - let added_files_matching_predicates = if self.transaction_info.operation.read_whole_table() + let added_files_matching_predicates = if self.txn_info.read_whole_table() { added_files_to_check } else { @@ -457,8 +487,7 @@ impl<'a> ConflictChecker<'a> { // Fail if files have been deleted that the txn read. let read_file_path: HashSet = self .current_transaction_info() - .read_files - .iter() + .read_files()? .map(|f| f.path.clone()) .collect(); let deleted_read_overlap = self @@ -571,8 +600,7 @@ pub(super) fn can_downgrade_to_snapshot_isolation<'a>( } if has_non_file_actions { - // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation - // level to SnapshotIsolation. + // if Non-file-actions are present (e.g. METADATA etc.), then don't downgrade the isolation level. return false; } @@ -650,15 +678,8 @@ mod tests { let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); let (state, summary) = prepare_test(None, vec![], vec![file1]); - let operation = DeltaOperation::Write { - mode: SaveMode::Append, - partition_by: Default::default(), - predicate: None, - }; let actions = vec![file2]; - let checker = ConflictChecker::try_new(&state, summary, operation, &actions) - .await - .unwrap(); + let checker = ConflictChecker::new(&state, summary, None, &actions, None); let result = checker.check_conflicts(); assert!(result.is_ok()); @@ -676,11 +697,8 @@ mod tests { // remove file from table that has previously been removed let removed_file = tu::create_remove_action("removed_file", true); let (state, summary) = prepare_test(None, vec![], vec![removed_file.clone()]); - let operation = DeltaOperation::Delete { predicate: None }; let actions = vec![removed_file]; - let checker = ConflictChecker::try_new(&state, summary, operation, &actions) - .await - .unwrap(); + let checker = ConflictChecker::new(&state, summary, None, &actions, None); assert!(matches!( checker.check_conflicts(), diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index cd0087aae8..001a6d52ee 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -185,9 +185,14 @@ pub(crate) async fn commit( let summary = WinningCommitSummary::try_new(storage, read_snapshot.version(), version) .await?; - let conflict_checker = - ConflictChecker::try_new(read_snapshot, summary, operation.clone(), actions) - .await?; + let read_redicate = operation.read_predicate(); + let conflict_checker = ConflictChecker::new( + read_snapshot, + summary, + read_redicate, + actions, + Some(&operation), + ); match conflict_checker.check_conflicts() { Ok(_) => { attempt_number += 1; From 576f5dbfd5d2ed8a5b29eb1828a93bbb1d00c05b Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 11 Mar 2023 11:11:19 +0100 Subject: [PATCH 56/67] refactor: move transaction info creation out of conflict checker --- .../transaction/conflict_checker.rs | 109 +++++++++++------- rust/src/operations/transaction/mod.rs | 14 +-- 2 files changed, 72 insertions(+), 51 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index f9861e5382..d59e0c1dd3 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -21,6 +21,8 @@ use crate::{ use super::state::AddContainer; #[cfg(feature = "datafusion")] use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +#[cfg(feature = "datafusion")] +use datafusion_expr::Expr; /// Exceptions raised during commit conflict resolution #[derive(thiserror::Error, Debug)] @@ -100,7 +102,11 @@ fn read_whole_table(operation: &DeltaOperation) -> bool { pub(crate) struct TransactionInfo<'a> { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction + #[cfg(not(feature = "datafusion"))] pub(crate) read_predicates: Option, + /// partition predicates by which files have been queried by the transaction + #[cfg(feature = "datafusion")] + pub(crate) read_predicates: Option, /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit @@ -110,20 +116,54 @@ pub(crate) struct TransactionInfo<'a> { } impl<'a> TransactionInfo<'a> { - pub fn new( - snapshot: &'a DeltaTableState, + #[cfg(feature = "datafusion")] + pub fn try_new( + read_snapshot: &'a DeltaTableState, read_predicates: Option, actions: &'a Vec, + ) -> DeltaResult { + let read_predicates = read_predicates + .map(|pred| read_snapshot.parse_predicate_expression(&pred)) + .transpose()?; + Ok(Self { + txn_id: "".into(), + read_predicates, + read_app_ids: Default::default(), + actions, + read_snapshot, + }) + } + + #[cfg(feature = "datafusion")] + pub fn new( + read_snapshot: &'a DeltaTableState, + read_predicates: Option, + actions: &'a Vec, ) -> Self { Self { txn_id: "".into(), read_predicates, read_app_ids: Default::default(), actions, - read_snapshot: snapshot, + read_snapshot, } } + #[cfg(not(feature = "datafusion"))] + pub fn try_new( + read_snapshot: &'a DeltaTableState, + read_predicates: Option, + actions: &'a Vec, + ) -> DeltaResult { + Ok(Self { + txn_id: "".into(), + read_predicates, + read_app_ids: Default::default(), + actions, + read_snapshot, + }) + } + pub fn metadata(&self) -> Option<&DeltaTableMetaData> { self.read_snapshot.current_metadata() } @@ -139,15 +179,9 @@ impl<'a> TransactionInfo<'a> { /// Files read by the transaction pub fn read_files(&self) -> Result, CommitConflictError> { if let Some(predicate) = &self.read_predicates { - let pred = self - .read_snapshot - .parse_predicate_expression(predicate) - .map_err(|err| CommitConflictError::Predicate { - source: Box::new(err), - })?; Ok(Either::Left( self.read_snapshot - .files_matching_predicate(&[pred]) + .files_matching_predicate(&[predicate.clone()]) .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), })?, @@ -318,26 +352,31 @@ pub(crate) struct ConflictChecker<'a> { impl<'a> ConflictChecker<'a> { pub fn new( - read_snapshot: &'a DeltaTableState, + transaction_info: TransactionInfo<'a>, winning_commit_summary: WinningCommitSummary, - read_predicates: Option, - actions: &'a Vec, operation: Option<&DeltaOperation>, ) -> ConflictChecker<'a> { - let transaction_info = TransactionInfo::new(read_snapshot, read_predicates, actions); let isolation_level = operation .and_then(|op| { if can_downgrade_to_snapshot_isolation( &winning_commit_summary.actions, op, - &read_snapshot.table_config().isolation_level(), + &transaction_info + .read_snapshot + .table_config() + .isolation_level(), ) { Some(IsolationLevel::SnapshotIsolation) } else { None } }) - .unwrap_or_else(|| read_snapshot.table_config().isolation_level()); + .unwrap_or_else(|| { + transaction_info + .read_snapshot + .table_config() + .isolation_level() + }); Self { txn_info: transaction_info, @@ -427,7 +466,7 @@ impl<'a> ConflictChecker<'a> { // to assume all files match cfg_if::cfg_if! { if #[cfg(feature = "datafusion")] { - let added_files_matching_predicates = if let (Some(predicate_str), false) = ( + let added_files_matching_predicates = if let (Some(predicate), false) = ( &self.txn_info.read_predicates, self.txn_info.read_whole_table(), ) { @@ -436,14 +475,6 @@ impl<'a> ConflictChecker<'a> { source: Box::new(err), } })?; - let predicate = self - .txn_info - .read_snapshot - .parse_predicate_expression(predicate_str) - .map_err(|err| CommitConflictError::Predicate { - source: Box::new(err), - })?; - // TODO remove unwrap let partition_columns = &self .txn_info .read_snapshot @@ -451,7 +482,7 @@ impl<'a> ConflictChecker<'a> { .ok_or(CommitConflictError::NoMetadata)? .partition_columns; AddContainer::new(&added_files_to_check, partition_columns, arrow_schema) - .predicate_matches(predicate) + .predicate_matches(predicate.clone()) .map_err(|err| CommitConflictError::Predicate { source: Box::new(err), })? @@ -656,17 +687,19 @@ mod tests { // - actions fn prepare_test( setup: Option>, - reads: Vec, + reads: Option, concurrent: Vec, - ) -> (DeltaTableState, WinningCommitSummary) { + actions: Vec, + ) -> Result<(), CommitConflictError> { let setup_actions = setup.unwrap_or_else(init_table_actions); - let mut state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); - state.merge(DeltaTableState::from_actions(reads, 1).unwrap(), true, true); + let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); + let transaction_info = TransactionInfo::try_new(&state, reads, &actions).unwrap(); let summary = WinningCommitSummary { actions: concurrent, commit_info: None, }; - (state, summary) + let checker = ConflictChecker::new(transaction_info, summary, None); + checker.check_conflicts() } #[tokio::test] @@ -676,12 +709,7 @@ mod tests { // append file to table while a concurrent writer also appends a file let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - - let (state, summary) = prepare_test(None, vec![], vec![file1]); - let actions = vec![file2]; - let checker = ConflictChecker::new(&state, summary, None, &actions, None); - - let result = checker.check_conflicts(); + let result = prepare_test(None, None, vec![file1], vec![file2]); assert!(result.is_ok()); // disjoint delete - read @@ -696,12 +724,9 @@ mod tests { // delete - delete // remove file from table that has previously been removed let removed_file = tu::create_remove_action("removed_file", true); - let (state, summary) = prepare_test(None, vec![], vec![removed_file.clone()]); - let actions = vec![removed_file]; - let checker = ConflictChecker::new(&state, summary, None, &actions, None); - + let result = prepare_test(None, None, vec![removed_file.clone()], vec![removed_file]); assert!(matches!( - checker.check_conflicts(), + result, Err(CommitConflictError::ConcurrentDeleteDelete) )); } diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 001a6d52ee..0ced16d760 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -16,7 +16,7 @@ mod state; #[cfg(test)] pub(crate) mod test_utils; -use self::conflict_checker::{CommitConflictError, WinningCommitSummary}; +use self::conflict_checker::{CommitConflictError, TransactionInfo, WinningCommitSummary}; const DELTA_LOG_FOLDER: &str = "_delta_log"; @@ -185,14 +185,10 @@ pub(crate) async fn commit( let summary = WinningCommitSummary::try_new(storage, read_snapshot.version(), version) .await?; - let read_redicate = operation.read_predicate(); - let conflict_checker = ConflictChecker::new( - read_snapshot, - summary, - read_redicate, - actions, - Some(&operation), - ); + let transaction_info = + TransactionInfo::try_new(read_snapshot, operation.read_predicate(), actions)?; + let conflict_checker = + ConflictChecker::new(transaction_info, summary, Some(&operation)); match conflict_checker.check_conflicts() { Ok(_) => { attempt_number += 1; From 1ad09ad289dc5de9fb1f059ef6006f9b5411a839 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 11 Mar 2023 15:48:04 +0100 Subject: [PATCH 57/67] test: disjoint delete-read --- .../transaction/conflict_checker.rs | 26 ++++++++++++++++--- rust/src/operations/transaction/test_utils.rs | 1 + 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index d59e0c1dd3..8647ac7a70 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -123,7 +123,7 @@ impl<'a> TransactionInfo<'a> { actions: &'a Vec, ) -> DeltaResult { let read_predicates = read_predicates - .map(|pred| read_snapshot.parse_predicate_expression(&pred)) + .map(|pred| read_snapshot.parse_predicate_expression(pred)) .transpose()?; Ok(Self { txn_id: "".into(), @@ -200,7 +200,8 @@ impl<'a> TransactionInfo<'a> { /// Whether the whole table was read during the transaction pub fn read_whole_table(&self) -> bool { // TODO actually check - self.read_predicates.is_some() + // self.read_predicates.is_none() + false } } @@ -649,6 +650,8 @@ mod tests { use super::*; use crate::action::{Action, SaveMode}; use crate::operations::transaction::commit; + #[cfg(feature = "datafusion")] + use datafusion_expr::{col, lit}; use serde_json::json; fn get_stats(min: i64, max: i64) -> Option { @@ -685,15 +688,16 @@ mod tests { // - reads // - concurrentWrites // - actions + #[cfg(feature = "datafusion")] fn prepare_test( setup: Option>, - reads: Option, + reads: Option, concurrent: Vec, actions: Vec, ) -> Result<(), CommitConflictError> { let setup_actions = setup.unwrap_or_else(init_table_actions); let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); - let transaction_info = TransactionInfo::try_new(&state, reads, &actions).unwrap(); + let transaction_info = TransactionInfo::new(&state, reads, &actions); let summary = WinningCommitSummary { actions: concurrent, commit_info: None, @@ -703,6 +707,7 @@ mod tests { } #[tokio::test] + #[cfg(feature = "datafusion")] // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 async fn test_allowed_concurrent_actions() { // append - append @@ -714,11 +719,24 @@ mod tests { // disjoint delete - read // the concurrent transaction deletes a file that the current transaction did NOT read + let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); + let file2 = tu::create_add_action("file2", true, get_stats(100, 10000)); + let mut setup_actions = init_table_actions(); + setup_actions.push(file1); + setup_actions.push(file2); + let result = prepare_test( + Some(setup_actions), + Some(col("value").gt(lit::(10))), + vec![tu::create_remove_action("file1", true)], + vec![], + ); + assert!(result.is_ok()); // TODO disjoint transactions } #[tokio::test] + #[cfg(feature = "datafusion")] // tests adopted from https://github.com/delta-io/delta/blob/24c025128612a4ae02d0ad958621f928cda9a3ec/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala#L40-L94 async fn test_disallowed_concurrent_actions() { // delete - delete diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index a22de38aa8..04f38014e9 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -1,3 +1,4 @@ +#![allow(unused)] use super::{prepare_commit, try_commit_transaction, CommitInfo}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; use crate::table_state::DeltaTableState; From fd9da638be49d95e11f60c3d21a058a505e83316 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 11 Mar 2023 18:03:43 +0100 Subject: [PATCH 58/67] test: add disallowed transaction tests --- .../transaction/conflict_checker.rs | 159 ++++++++++++++++-- rust/src/operations/transaction/mod.rs | 9 +- rust/src/operations/transaction/test_utils.rs | 31 +++- 3 files changed, 173 insertions(+), 26 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 8647ac7a70..663794563b 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -103,16 +103,18 @@ pub(crate) struct TransactionInfo<'a> { pub(crate) txn_id: String, /// partition predicates by which files have been queried by the transaction #[cfg(not(feature = "datafusion"))] - pub(crate) read_predicates: Option, + read_predicates: Option, /// partition predicates by which files have been queried by the transaction #[cfg(feature = "datafusion")] - pub(crate) read_predicates: Option, + read_predicates: Option, /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit - pub(crate) actions: &'a Vec, + actions: &'a Vec, /// read [`DeltaTableState`] used for the transaction pub(crate) read_snapshot: &'a DeltaTableState, + /// Whether the transaction tainted the whole table + read_whole_table: bool, } impl<'a> TransactionInfo<'a> { @@ -121,6 +123,7 @@ impl<'a> TransactionInfo<'a> { read_snapshot: &'a DeltaTableState, read_predicates: Option, actions: &'a Vec, + read_whole_table: bool, ) -> DeltaResult { let read_predicates = read_predicates .map(|pred| read_snapshot.parse_predicate_expression(pred)) @@ -131,6 +134,7 @@ impl<'a> TransactionInfo<'a> { read_app_ids: Default::default(), actions, read_snapshot, + read_whole_table, }) } @@ -139,6 +143,7 @@ impl<'a> TransactionInfo<'a> { read_snapshot: &'a DeltaTableState, read_predicates: Option, actions: &'a Vec, + read_whole_table: bool, ) -> Self { Self { txn_id: "".into(), @@ -146,6 +151,7 @@ impl<'a> TransactionInfo<'a> { read_app_ids: Default::default(), actions, read_snapshot, + read_whole_table, } } @@ -154,6 +160,7 @@ impl<'a> TransactionInfo<'a> { read_snapshot: &'a DeltaTableState, read_predicates: Option, actions: &'a Vec, + read_whole_table: bool, ) -> DeltaResult { Ok(Self { txn_id: "".into(), @@ -161,6 +168,7 @@ impl<'a> TransactionInfo<'a> { read_app_ids: Default::default(), actions, read_snapshot, + read_whole_table, }) } @@ -187,7 +195,7 @@ impl<'a> TransactionInfo<'a> { })?, )) } else { - Ok(Either::Right(self.read_snapshot.files().iter())) + Ok(Either::Right(std::iter::empty())) } } @@ -199,9 +207,7 @@ impl<'a> TransactionInfo<'a> { /// Whether the whole table was read during the transaction pub fn read_whole_table(&self) -> bool { - // TODO actually check - // self.read_predicates.is_none() - false + self.read_whole_table } } @@ -689,15 +695,16 @@ mod tests { // - concurrentWrites // - actions #[cfg(feature = "datafusion")] - fn prepare_test( + fn execute_test( setup: Option>, reads: Option, concurrent: Vec, actions: Vec, + read_whole_table: bool, ) -> Result<(), CommitConflictError> { let setup_actions = setup.unwrap_or_else(init_table_actions); let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); - let transaction_info = TransactionInfo::new(&state, reads, &actions); + let transaction_info = TransactionInfo::new(&state, reads, &actions, read_whole_table); let summary = WinningCommitSummary { actions: concurrent, commit_info: None, @@ -714,24 +721,42 @@ mod tests { // append file to table while a concurrent writer also appends a file let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - let result = prepare_test(None, None, vec![file1], vec![file2]); + let result = execute_test(None, None, vec![file1], vec![file2], false); assert!(result.is_ok()); // disjoint delete - read // the concurrent transaction deletes a file that the current transaction did NOT read - let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); - let file2 = tu::create_add_action("file2", true, get_stats(100, 10000)); + let file_not_read = tu::create_add_action("file_not_read", true, get_stats(1, 10)); + let file_read = tu::create_add_action("file_read", true, get_stats(100, 10000)); + let mut setup_actions = init_table_actions(); + setup_actions.push(file_not_read); + setup_actions.push(file_read); + let result = execute_test( + Some(setup_actions), + Some(col("value").gt(lit::(10))), + vec![tu::create_remove_action("file_not_read", true)], + vec![], + false, + ); + assert!(result.is_ok()); + + // disjoint add - read + // concurrently add file, that the current transaction would not have read + let file_added = tu::create_add_action("file_added", true, get_stats(1, 10)); + let file_read = tu::create_add_action("file_read", true, get_stats(100, 10000)); let mut setup_actions = init_table_actions(); - setup_actions.push(file1); - setup_actions.push(file2); - let result = prepare_test( + setup_actions.push(file_read); + let result = execute_test( Some(setup_actions), Some(col("value").gt(lit::(10))), - vec![tu::create_remove_action("file1", true)], + vec![file_added], vec![], + false, ); assert!(result.is_ok()); + // TODO add / read + no write + // TODO disjoint transactions } @@ -742,10 +767,110 @@ mod tests { // delete - delete // remove file from table that has previously been removed let removed_file = tu::create_remove_action("removed_file", true); - let result = prepare_test(None, None, vec![removed_file.clone()], vec![removed_file]); + let result = execute_test( + None, + None, + vec![removed_file.clone()], + vec![removed_file], + false, + ); assert!(matches!( result, Err(CommitConflictError::ConcurrentDeleteDelete) )); + + // add / read + write + // a file is concurrently added that should have been read by the current transaction + let file_added = tu::create_add_action("file_added", true, get_stats(1, 10)); + let file_should_have_read = + tu::create_add_action("file_should_have_read", true, get_stats(1, 10)); + let result = execute_test( + None, + Some(col("value").lt_eq(lit::(10))), + vec![file_should_have_read], + vec![file_added], + false, + ); + assert!(matches!(result, Err(CommitConflictError::ConcurrentAppend))); + + // delete / read + // transaction reads a file that is removed by concurrent transaction + let file_read = tu::create_add_action("file_read", true, get_stats(1, 10)); + let mut setup_actions = init_table_actions(); + setup_actions.push(file_read); + let result = execute_test( + Some(setup_actions), + Some(col("value").lt_eq(lit::(10))), + vec![tu::create_remove_action("file_read", true)], + vec![], + false, + ); + assert!(matches!( + result, + Err(CommitConflictError::ConcurrentDeleteRead) + )); + + // schema change + // concurrent transactions changes table metadata + let result = execute_test( + None, + None, + vec![tu::create_metadata_action(None, None)], + vec![], + false, + ); + assert!(matches!(result, Err(CommitConflictError::MetadataChanged))); + + // upgrade / upgrade + // current and concurrent transactions chnage the protocol version + let result = execute_test( + None, + None, + vec![tu::create_protocol_action(None, None)], + vec![tu::create_protocol_action(None, None)], + false, + ); + assert!(matches!(result, Err(CommitConflictError::ProtocolChanged))); + + // taint whole table + // `read_whole_table` should disallow any concurrent change, even if the change + // is disjoint with the earlier filter + let file_part1 = tu::create_add_action("file_part1", true, get_stats(1, 10)); + let file_part2 = tu::create_add_action("file_part2", true, get_stats(11, 100)); + let file_part3 = tu::create_add_action("file_part3", true, get_stats(101, 1000)); + let mut setup_actions = init_table_actions(); + setup_actions.push(file_part1); + let result = execute_test( + Some(setup_actions), + // filter matches neither exisiting nor added files + Some(col("value").lt(lit::(0))), + vec![file_part2], + vec![file_part3], + true, + ); + assert!(matches!(result, Err(CommitConflictError::ConcurrentAppend))); + + // taint whole table + concurrent remove + // `read_whole_table` should disallow any concurrent remove actions + let file_part1 = tu::create_add_action("file_part1", true, get_stats(1, 10)); + let file_part2 = tu::create_add_action("file_part2", true, get_stats(11, 100)); + let mut setup_actions = init_table_actions(); + setup_actions.push(file_part1); + let result = execute_test( + Some(setup_actions), + None, + vec![tu::create_remove_action("file_part1", true)], + vec![file_part2], + true, + ); + println!("result: {:?}", result); + assert!(matches!( + result, + Err(CommitConflictError::ConcurrentDeleteRead) + )); + + // TODO "add in part=2 / read from part=1,2 and write to part=1" + + // TODO conflicting txns } } diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 0ced16d760..f6fea18693 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -185,8 +185,13 @@ pub(crate) async fn commit( let summary = WinningCommitSummary::try_new(storage, read_snapshot.version(), version) .await?; - let transaction_info = - TransactionInfo::try_new(read_snapshot, operation.read_predicate(), actions)?; + let transaction_info = TransactionInfo::try_new( + read_snapshot, + operation.read_predicate(), + actions, + // TODO allow tainting whole table + false, + )?; let conflict_checker = ConflictChecker::new(transaction_info, summary, Some(&operation)); match conflict_checker.check_conflicts() { diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 04f38014e9..92d981d5e1 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -29,11 +29,18 @@ pub fn create_remove_action(path: impl Into, data_change: bool) -> Actio }) } -pub fn init_table_actions() -> Vec { +pub fn create_protocol_action(max_reader: Option, max_writer: Option) -> Action { let protocol = Protocol { - min_reader_version: crate::operations::MAX_SUPPORTED_READER_VERSION, - min_writer_version: crate::operations::MAX_SUPPORTED_WRITER_VERSION, + min_reader_version: max_reader.unwrap_or(crate::operations::MAX_SUPPORTED_READER_VERSION), + min_writer_version: max_writer.unwrap_or(crate::operations::MAX_SUPPORTED_WRITER_VERSION), }; + Action::protocol(protocol) +} + +pub fn create_metadata_action( + parttiton_columns: Option>, + configuration: Option>>, +) -> Action { let table_schema = Schema::new(vec![ SchemaField::new( "id".to_string(), @@ -54,7 +61,18 @@ pub fn init_table_actions() -> Vec { HashMap::new(), ), ]); - let metadata = DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); + let metadata = DeltaTableMetaData::new( + None, + None, + None, + table_schema, + parttiton_columns.unwrap_or_default(), + configuration.unwrap_or_default(), + ); + Action::metaData(MetaData::try_from(metadata).unwrap()) +} + +pub fn init_table_actions() -> Vec { let raw = r#" { "timestamp": 1670892998177, @@ -75,11 +93,10 @@ pub fn init_table_actions() -> Vec { }"#; let commit_info = serde_json::from_str::(raw).unwrap(); - vec![ Action::commitInfo(commit_info), - Action::protocol(protocol), - Action::metaData(MetaData::try_from(metadata).unwrap()), + create_protocol_action(None, None), + create_metadata_action(None, None), ] } From 0dda8cb05c1b53bd5597a5d6d568787f9f1fd7c4 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 11 Mar 2023 20:08:07 +0100 Subject: [PATCH 59/67] chore: simplify tests --- .../transaction/conflict_checker.rs | 37 +++++++------------ 1 file changed, 14 insertions(+), 23 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 663794563b..89537398fa 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -701,7 +701,8 @@ mod tests { concurrent: Vec, actions: Vec, read_whole_table: bool, - ) -> Result<(), CommitConflictError> { + expected: Result<(), CommitConflictError>, + ) { let setup_actions = setup.unwrap_or_else(init_table_actions); let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); let transaction_info = TransactionInfo::new(&state, reads, &actions, read_whole_table); @@ -710,7 +711,8 @@ mod tests { commit_info: None, }; let checker = ConflictChecker::new(transaction_info, summary, None); - checker.check_conflicts() + let result = checker.check_conflicts(); + assert!(matches!(result, expected)); } #[tokio::test] @@ -721,8 +723,7 @@ mod tests { // append file to table while a concurrent writer also appends a file let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - let result = execute_test(None, None, vec![file1], vec![file2], false); - assert!(result.is_ok()); + let result = execute_test(None, None, vec![file1], vec![file2], false, Ok(())); // disjoint delete - read // the concurrent transaction deletes a file that the current transaction did NOT read @@ -737,8 +738,8 @@ mod tests { vec![tu::create_remove_action("file_not_read", true)], vec![], false, + Ok(()), ); - assert!(result.is_ok()); // disjoint add - read // concurrently add file, that the current transaction would not have read @@ -752,8 +753,8 @@ mod tests { vec![file_added], vec![], false, + Ok(()), ); - assert!(result.is_ok()); // TODO add / read + no write @@ -773,11 +774,8 @@ mod tests { vec![removed_file.clone()], vec![removed_file], false, + Err(CommitConflictError::ConcurrentDeleteDelete), ); - assert!(matches!( - result, - Err(CommitConflictError::ConcurrentDeleteDelete) - )); // add / read + write // a file is concurrently added that should have been read by the current transaction @@ -790,8 +788,8 @@ mod tests { vec![file_should_have_read], vec![file_added], false, + Err(CommitConflictError::ConcurrentAppend), ); - assert!(matches!(result, Err(CommitConflictError::ConcurrentAppend))); // delete / read // transaction reads a file that is removed by concurrent transaction @@ -804,11 +802,8 @@ mod tests { vec![tu::create_remove_action("file_read", true)], vec![], false, + Err(CommitConflictError::ConcurrentDeleteRead), ); - assert!(matches!( - result, - Err(CommitConflictError::ConcurrentDeleteRead) - )); // schema change // concurrent transactions changes table metadata @@ -818,8 +813,8 @@ mod tests { vec![tu::create_metadata_action(None, None)], vec![], false, + Err(CommitConflictError::MetadataChanged), ); - assert!(matches!(result, Err(CommitConflictError::MetadataChanged))); // upgrade / upgrade // current and concurrent transactions chnage the protocol version @@ -829,8 +824,8 @@ mod tests { vec![tu::create_protocol_action(None, None)], vec![tu::create_protocol_action(None, None)], false, + Err(CommitConflictError::ProtocolChanged), ); - assert!(matches!(result, Err(CommitConflictError::ProtocolChanged))); // taint whole table // `read_whole_table` should disallow any concurrent change, even if the change @@ -847,8 +842,8 @@ mod tests { vec![file_part2], vec![file_part3], true, + Err(CommitConflictError::ConcurrentAppend), ); - assert!(matches!(result, Err(CommitConflictError::ConcurrentAppend))); // taint whole table + concurrent remove // `read_whole_table` should disallow any concurrent remove actions @@ -862,12 +857,8 @@ mod tests { vec![tu::create_remove_action("file_part1", true)], vec![file_part2], true, + Err(CommitConflictError::ConcurrentDeleteRead), ); - println!("result: {:?}", result); - assert!(matches!( - result, - Err(CommitConflictError::ConcurrentDeleteRead) - )); // TODO "add in part=2 / read from part=1,2 and write to part=1" From 97d3405a4f533748b68478bc03a02532ca416319 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 11 Mar 2023 21:27:29 +0100 Subject: [PATCH 60/67] chore: cleanup --- .../transaction/conflict_checker.rs | 52 +++++++++++++------ 1 file changed, 36 insertions(+), 16 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 89537398fa..621f3dbc42 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -701,8 +701,7 @@ mod tests { concurrent: Vec, actions: Vec, read_whole_table: bool, - expected: Result<(), CommitConflictError>, - ) { + ) -> Result<(), CommitConflictError> { let setup_actions = setup.unwrap_or_else(init_table_actions); let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); let transaction_info = TransactionInfo::new(&state, reads, &actions, read_whole_table); @@ -711,8 +710,7 @@ mod tests { commit_info: None, }; let checker = ConflictChecker::new(transaction_info, summary, None); - let result = checker.check_conflicts(); - assert!(matches!(result, expected)); + checker.check_conflicts() } #[tokio::test] @@ -723,7 +721,8 @@ mod tests { // append file to table while a concurrent writer also appends a file let file1 = tu::create_add_action("file1", true, get_stats(1, 10)); let file2 = tu::create_add_action("file2", true, get_stats(1, 10)); - let result = execute_test(None, None, vec![file1], vec![file2], false, Ok(())); + let result = execute_test(None, None, vec![file1], vec![file2], false); + assert!(result.is_ok()); // disjoint delete - read // the concurrent transaction deletes a file that the current transaction did NOT read @@ -738,8 +737,8 @@ mod tests { vec![tu::create_remove_action("file_not_read", true)], vec![], false, - Ok(()), ); + assert!(result.is_ok()); // disjoint add - read // concurrently add file, that the current transaction would not have read @@ -753,10 +752,22 @@ mod tests { vec![file_added], vec![], false, - Ok(()), ); - - // TODO add / read + no write + assert!(result.is_ok()); + + // TODO enable test once we have isolation level downcast + // add / read + no write + // transaction would have read file added by concurrent txn, but does not write data, + // so no real conflicting change even though data was added + // let file_added = tu::create_add_action("file_added", true, get_stats(1, 10)); + // let result = execute_test( + // None, + // Some(col("value").gt(lit::(5))), + // vec![file_added], + // vec![], + // false, + // ); + // assert!(result.is_ok()); // TODO disjoint transactions } @@ -774,8 +785,11 @@ mod tests { vec![removed_file.clone()], vec![removed_file], false, - Err(CommitConflictError::ConcurrentDeleteDelete), ); + assert!(matches!( + result, + Err(CommitConflictError::ConcurrentDeleteDelete) + )); // add / read + write // a file is concurrently added that should have been read by the current transaction @@ -788,8 +802,8 @@ mod tests { vec![file_should_have_read], vec![file_added], false, - Err(CommitConflictError::ConcurrentAppend), ); + assert!(matches!(result, Err(CommitConflictError::ConcurrentAppend))); // delete / read // transaction reads a file that is removed by concurrent transaction @@ -802,8 +816,11 @@ mod tests { vec![tu::create_remove_action("file_read", true)], vec![], false, - Err(CommitConflictError::ConcurrentDeleteRead), ); + assert!(matches!( + result, + Err(CommitConflictError::ConcurrentDeleteRead) + )); // schema change // concurrent transactions changes table metadata @@ -813,8 +830,8 @@ mod tests { vec![tu::create_metadata_action(None, None)], vec![], false, - Err(CommitConflictError::MetadataChanged), ); + assert!(matches!(result, Err(CommitConflictError::MetadataChanged))); // upgrade / upgrade // current and concurrent transactions chnage the protocol version @@ -824,8 +841,8 @@ mod tests { vec![tu::create_protocol_action(None, None)], vec![tu::create_protocol_action(None, None)], false, - Err(CommitConflictError::ProtocolChanged), ); + assert!(matches!(result, Err(CommitConflictError::ProtocolChanged))); // taint whole table // `read_whole_table` should disallow any concurrent change, even if the change @@ -842,8 +859,8 @@ mod tests { vec![file_part2], vec![file_part3], true, - Err(CommitConflictError::ConcurrentAppend), ); + assert!(matches!(result, Err(CommitConflictError::ConcurrentAppend))); // taint whole table + concurrent remove // `read_whole_table` should disallow any concurrent remove actions @@ -857,8 +874,11 @@ mod tests { vec![tu::create_remove_action("file_part1", true)], vec![file_part2], true, - Err(CommitConflictError::ConcurrentDeleteRead), ); + assert!(matches!( + result, + Err(CommitConflictError::ConcurrentDeleteRead) + )); // TODO "add in part=2 / read from part=1,2 and write to part=1" From 54b1a85976bfe6784d9865137c46df7b3522f243 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Mar 2023 18:33:09 +0100 Subject: [PATCH 61/67] chore: cleanup --- .../transaction/conflict_checker.rs | 65 ++++++------------- rust/src/operations/transaction/mod.rs | 15 ----- 2 files changed, 19 insertions(+), 61 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 621f3dbc42..ac3144a97e 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -1,27 +1,19 @@ //! Helper module to check if a transaction can be committed in case of conflicting commits. -#![allow(unused)] use std::collections::HashSet; use std::io::{BufRead, BufReader, Cursor}; use itertools::Either; use object_store::ObjectStore; -use serde_json::{Map, Value}; use super::CommitInfo; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::delta_config::IsolationLevel; -use crate::operations::transaction::TransactionError; -use crate::storage::{commit_uri_from_version, DeltaObjectStore, ObjectStoreRef}; -use crate::{ - table_state::DeltaTableState, DeltaDataTypeVersion, DeltaResult, DeltaTable, DeltaTableError, - DeltaTableMetaData, -}; +use crate::storage::commit_uri_from_version; +use crate::{table_state::DeltaTableState, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; #[cfg(feature = "datafusion")] use super::state::AddContainer; #[cfg(feature = "datafusion")] -use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; -#[cfg(feature = "datafusion")] use datafusion_expr::Expr; /// Exceptions raised during commit conflict resolution @@ -89,18 +81,10 @@ pub enum CommitConflictError { NoMetadata, } -fn read_whole_table(operation: &DeltaOperation) -> bool { - match operation { - // TODO just adding one operation example, as currently none of the - // implemented operations scan the entire table. - DeltaOperation::Write { predicate, .. } if predicate.is_none() => false, - _ => false, - } -} - /// A struct representing different attributes of current transaction needed for conflict detection. pub(crate) struct TransactionInfo<'a> { - pub(crate) txn_id: String, + #[allow(unused)] + txn_id: String, /// partition predicates by which files have been queried by the transaction #[cfg(not(feature = "datafusion"))] read_predicates: Option, @@ -139,6 +123,7 @@ impl<'a> TransactionInfo<'a> { } #[cfg(feature = "datafusion")] + #[allow(unused)] pub fn new( read_snapshot: &'a DeltaTableState, read_predicates: Option, @@ -172,10 +157,6 @@ impl<'a> TransactionInfo<'a> { }) } - pub fn metadata(&self) -> Option<&DeltaTableMetaData> { - self.read_snapshot.current_metadata() - } - /// Whether the transaction changed the tables metadatas pub fn metadata_changed(&self) -> bool { self.actions @@ -333,12 +314,12 @@ impl WinningCommitSummary { } } - pub fn only_add_files(&self) -> bool { - !self - .actions - .iter() - .any(|action| matches!(action, Action::remove(_))) - } + // pub fn only_add_files(&self) -> bool { + // !self + // .actions + // .iter() + // .any(|action| matches!(action, Action::remove(_))) + // } pub fn is_blind_append(&self) -> Option { self.commit_info @@ -392,11 +373,6 @@ impl<'a> ConflictChecker<'a> { } } - fn current_transaction_info(&self) -> &TransactionInfo { - // TODO figure out when we need to update this - &self.txn_info - } - /// This function checks conflict of the `initial_current_transaction_info` against the /// `winning_commit_version` and returns an updated [`TransactionInfo`] that represents /// the transaction as if it had started while reading the `winning_commit_version`. @@ -422,7 +398,7 @@ impl<'a> ConflictChecker<'a> { } if !self.winning_commit_summary.protocol().is_empty() && self - .current_transaction_info() + .txn_info .actions .iter() .any(|a| matches!(a, Action::protocol(_))) @@ -454,9 +430,7 @@ impl<'a> ConflictChecker<'a> { // Fail if new files have been added that the txn should have read. let added_files_to_check = match self.isolation_level { - IsolationLevel::WriteSerializable - if !self.current_transaction_info().metadata_changed() => - { + IsolationLevel::WriteSerializable if !self.txn_info.metadata_changed() => { // don't conflict with blind appends self.winning_commit_summary.changed_data_added_files() } @@ -524,7 +498,7 @@ impl<'a> ConflictChecker<'a> { ) -> Result<(), CommitConflictError> { // Fail if files have been deleted that the txn read. let read_file_path: HashSet = self - .current_transaction_info() + .txn_info .read_files()? .map(|f| f.path.clone()) .collect(); @@ -537,7 +511,7 @@ impl<'a> ConflictChecker<'a> { .find(|f| read_file_path.contains(&f.path)); if deleted_read_overlap.is_some() || (!self.winning_commit_summary.removed_files().is_empty() - && self.current_transaction_info().read_whole_table()) + && self.txn_info.read_whole_table()) { Err(CommitConflictError::ConcurrentDeleteRead) } else { @@ -552,7 +526,7 @@ impl<'a> ConflictChecker<'a> { ) -> Result<(), CommitConflictError> { // Fail if a file is deleted twice. let txn_deleted_files: HashSet = self - .current_transaction_info() + .txn_info .actions .iter() .cloned() @@ -590,7 +564,7 @@ impl<'a> ConflictChecker<'a> { // multiple instances of the same streaming query are running at the same time. let winning_txns = self.winning_commit_summary.app_level_transactions(); let txn_overlap: HashSet<&String> = winning_txns - .intersection(&self.current_transaction_info().read_app_ids) + .intersection(&self.txn_info.read_app_ids) .collect(); if !txn_overlap.is_empty() { Err(CommitConflictError::ConcurrentTransaction) @@ -652,10 +626,9 @@ pub(super) fn can_downgrade_to_snapshot_isolation<'a>( #[cfg(test)] mod tests { use super::super::test_utils as tu; - use super::super::test_utils::{create_initialized_table, init_table_actions}; + use super::super::test_utils::init_table_actions; use super::*; - use crate::action::{Action, SaveMode}; - use crate::operations::transaction::commit; + use crate::action::Action; #[cfg(feature = "datafusion")] use datafusion_expr::{col, lit}; use serde_json::json; diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index f6fea18693..6a27f4efc1 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -157,21 +157,6 @@ pub(crate) async fn commit( read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> DeltaResult { - // readPredicates.nonEmpty || readFiles.nonEmpty - // TODO revise logic if files are read - let depends_on_files = match operation { - DeltaOperation::Create { .. } | DeltaOperation::StreamingUpdate { .. } => false, - DeltaOperation::Optimize { .. } => true, - DeltaOperation::Write { - predicate: Some(_), .. - } => true, - _ => false, - }; - - // TODO actually get the prop from commit infos... - let only_add_files = false; - let _is_blind_append = only_add_files && !depends_on_files; - let tmp_commit = prepare_commit(storage, &operation, actions, app_metadata).await?; let max_attempts = 5; From 3eb23b3613aa4f7e97eb81d832579547a9cacde2 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Mar 2023 18:42:13 +0100 Subject: [PATCH 62/67] chore: clippy --- rust/src/operations/transaction/conflict_checker.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index ac3144a97e..90eccb6cf4 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -2,7 +2,6 @@ use std::collections::HashSet; use std::io::{BufRead, BufReader, Cursor}; -use itertools::Either; use object_store::ObjectStore; use super::CommitInfo; @@ -15,6 +14,8 @@ use crate::{table_state::DeltaTableState, DeltaDataTypeVersion, DeltaResult, Del use super::state::AddContainer; #[cfg(feature = "datafusion")] use datafusion_expr::Expr; +#[cfg(feature = "datafusion")] +use itertools::Either; /// Exceptions raised during commit conflict resolution #[derive(thiserror::Error, Debug)] @@ -82,8 +83,8 @@ pub enum CommitConflictError { } /// A struct representing different attributes of current transaction needed for conflict detection. +#[allow(unused)] pub(crate) struct TransactionInfo<'a> { - #[allow(unused)] txn_id: String, /// partition predicates by which files have been queried by the transaction #[cfg(not(feature = "datafusion"))] From d39088424311f21c5736e010bb99e5ecc08d9d72 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 17 Mar 2023 19:03:12 +0100 Subject: [PATCH 63/67] chore: parquet2 clippy --- rust/src/operations/transaction/conflict_checker.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 90eccb6cf4..e1e42c4c20 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -625,6 +625,7 @@ pub(super) fn can_downgrade_to_snapshot_isolation<'a>( } #[cfg(test)] +#[allow(unused)] mod tests { use super::super::test_utils as tu; use super::super::test_utils::init_table_actions; From 68f1b6ef22b53dce5b5704538605e8d896ce6976 Mon Sep 17 00:00:00 2001 From: Robert Pack <42610831+roeap@users.noreply.github.com> Date: Sun, 19 Mar 2023 08:56:42 +0100 Subject: [PATCH 64/67] Apply suggestions from code review Co-authored-by: Will Jones --- rust/src/operations/transaction/conflict_checker.rs | 9 ++++++--- rust/src/operations/transaction/state.rs | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index e1e42c4c20..91d24cc23c 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -23,12 +23,12 @@ pub enum CommitConflictError { /// This exception occurs when a concurrent operation adds files in the same partition /// (or anywhere in an un-partitioned table) that your operation reads. The file additions /// can be caused by INSERT, DELETE, UPDATE, or MERGE operations. - #[error("Concurrent append failed.")] + #[error("Commit failed: a concurrent transactions added new files.\nHelp: This transaction's query must be rerun to include the new data. Also, if you don't care to require this check to pass in the future, the isolation level can be set to Snapshot Isolation.")] ConcurrentAppend, /// This exception occurs when a concurrent operation deleted a file that your operation read. /// Common causes are a DELETE, UPDATE, or MERGE operation that rewrites files. - #[error("Concurrent delete-read failed.")] + #[error("Commit failed: a concurrent transaction deleted a file this operation read.\nHelp: This transaction's query must be rerun to exclude the removed data. Also, if you don't care to require this check to pass in the future, the isolation level can be set to Snapshot Isolation.")] ConcurrentDeleteRead, /// This exception occurs when a concurrent operation deleted a file that your operation also deletes. @@ -87,6 +87,9 @@ pub enum CommitConflictError { pub(crate) struct TransactionInfo<'a> { txn_id: String, /// partition predicates by which files have been queried by the transaction + /// + /// If any new data files or removed data files match this predicate, the + /// transaction should fail. #[cfg(not(feature = "datafusion"))] read_predicates: Option, /// partition predicates by which files have been queried by the transaction @@ -809,7 +812,7 @@ mod tests { assert!(matches!(result, Err(CommitConflictError::MetadataChanged))); // upgrade / upgrade - // current and concurrent transactions chnage the protocol version + // current and concurrent transactions change the protocol version let result = execute_test( None, None, diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 76c39816f8..f521ac2062 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -138,10 +138,10 @@ impl<'a> AddContainer<'a> { ScalarValue::iter_to_array(values).ok() } - /// Get an iterator of add actions / files, that MAY containtain data mathcing the predicate. + /// Get an iterator of add actions / files, that MAY contain data matching the predicate. /// /// Expressions are evaluated for file statistics, essentially column-wise min max bounds, - /// so evalutaing expressions is inexact. However exluded files are guaranteed (for a correct log) + /// so evaluating expressions is inexact. However, excluded files are guaranteed (for a correct log) /// to not contain matches by the predicate expression. pub fn predicate_matches(&self, predicate: Expr) -> DeltaResult> { let pruning_predicate = PruningPredicate::try_new(predicate, self.schema.clone())?; From 740eeff015629c111354da737f6f094baa6b21dc Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 28 Mar 2023 03:55:41 +0200 Subject: [PATCH 65/67] fix: chack conflicts against next commit only --- .../transaction/conflict_checker.rs | 35 +++++++++---------- rust/src/operations/transaction/mod.rs | 4 +-- 2 files changed, 17 insertions(+), 22 deletions(-) diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 91d24cc23c..a8403f1e67 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -88,7 +88,7 @@ pub(crate) struct TransactionInfo<'a> { txn_id: String, /// partition predicates by which files have been queried by the transaction /// - /// If any new data files or removed data files match this predicate, the + /// If any new data files or removed data files match this predicate, the /// transaction should fail. #[cfg(not(feature = "datafusion"))] read_predicates: Option, @@ -208,39 +208,36 @@ impl WinningCommitSummary { read_version: DeltaDataTypeVersion, winning_commit_version: DeltaDataTypeVersion, ) -> DeltaResult { - let mut actions = Vec::new(); - let mut version_to_read = read_version + 1; + // NOTE using asser, since a wrong version would right now mean a bug in our code. + assert_eq!(winning_commit_version, read_version + 1); - while version_to_read <= winning_commit_version { - let commit_uri = commit_uri_from_version(winning_commit_version); - let commit_log_bytes = object_store.get(&commit_uri).await?.bytes().await?; + let commit_uri = commit_uri_from_version(winning_commit_version); + let commit_log_bytes = object_store.get(&commit_uri).await?.bytes().await?; - let reader = BufReader::new(Cursor::new(commit_log_bytes)); - for maybe_line in reader.lines() { + let reader = BufReader::new(Cursor::new(commit_log_bytes)); + + let actions = reader + .lines() + .map(|maybe_line| { let line = maybe_line?; - actions.push(serde_json::from_str::(line.as_str()).map_err(|e| { + serde_json::from_str::(line.as_str()).map_err(|e| { DeltaTableError::InvalidJsonLog { json_err: e, version: winning_commit_version, line, } - })?); - } - version_to_read += 1; - } + }) + }) + .collect::, _>>()?; - // TODO how to handle commit info for multiple read commits? let commit_info = actions .iter() .find(|action| matches!(action, Action::commitInfo(_))) .map(|action| match action { - Action::commitInfo(info) => { - // let mut updated = info.clone(); - // updated.version = Some(version_to_read); - info.clone() - } + Action::commitInfo(info) => info.clone(), _ => unreachable!(), }); + Ok(Self { actions, commit_info, diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index 6a27f4efc1..47ea68fae2 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -167,9 +167,7 @@ pub(crate) async fn commit( match try_commit_transaction(storage, &tmp_commit, version).await { Ok(version) => return Ok(version), Err(TransactionError::VersionAlreadyExists(version)) => { - let summary = - WinningCommitSummary::try_new(storage, read_snapshot.version(), version) - .await?; + let summary = WinningCommitSummary::try_new(storage, version - 1, version).await?; let transaction_info = TransactionInfo::try_new( read_snapshot, operation.read_predicate(), From bfcb393495c58fc0e00d4e7d6db7cc4d14273dea Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 28 Mar 2023 05:03:55 +0200 Subject: [PATCH 66/67] test: re-enable ignored optimize test --- rust/tests/command_optimize.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 0d768c3feb..81a292867a 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -324,7 +324,6 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { Ok(()) } -#[ignore = "we do not yet re-try in operations commits."] #[tokio::test] /// Validate that optimize succeeds when only add actions occur for a optimized partition async fn test_no_conflict_for_append_actions() -> Result<(), Box> { @@ -370,6 +369,8 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { let metrics = plan.execute(dt.object_store(), &dt.state).await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); + + dt.update().await.unwrap(); assert_eq!(dt.version(), version + 2); Ok(()) } From 4cbd15833e63c8012fd6fa70bc04bdc68ef75a0c Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 4 Apr 2023 07:43:49 +0200 Subject: [PATCH 67/67] chore: rename PrefixObjectStore imports --- rust/Cargo.toml | 2 +- rust/src/storage/config.rs | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index fa7b8cabae..817e3c1d38 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -26,7 +26,7 @@ log = "0" libc = ">=0.2.90, <1" num-bigint = "0.4" num-traits = "0.2.15" -object_store = "0.5.3" +object_store = "0.5.6" once_cell = "1.16.0" parking_lot = "0.12" parquet = { version = "33", features = [ diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index 1f4eb999a1..e447bc687a 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -4,7 +4,7 @@ use super::utils::str_is_truthy; use crate::{DeltaResult, DeltaTableError}; use object_store::memory::InMemory; use object_store::path::Path; -use object_store::prefix::PrefixObjectStore; +use object_store::prefix::PrefixStore; use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; use std::collections::HashMap; @@ -111,14 +111,14 @@ pub(crate) enum ObjectStoreImpl { impl ObjectStoreImpl { pub(crate) fn into_prefix(self, prefix: Path) -> Arc { match self { - ObjectStoreImpl::Local(store) => Arc::new(PrefixObjectStore::new(store, prefix)), - ObjectStoreImpl::InMemory(store) => Arc::new(PrefixObjectStore::new(store, prefix)), + ObjectStoreImpl::Local(store) => Arc::new(PrefixStore::new(store, prefix)), + ObjectStoreImpl::InMemory(store) => Arc::new(PrefixStore::new(store, prefix)), #[cfg(feature = "azure")] - ObjectStoreImpl::Azure(store) => Arc::new(PrefixObjectStore::new(store, prefix)), + ObjectStoreImpl::Azure(store) => Arc::new(PrefixStore::new(store, prefix)), #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - ObjectStoreImpl::S3(store) => Arc::new(PrefixObjectStore::new(store, prefix)), + ObjectStoreImpl::S3(store) => Arc::new(PrefixStore::new(store, prefix)), #[cfg(feature = "gcs")] - ObjectStoreImpl::Google(store) => Arc::new(PrefixObjectStore::new(store, prefix)), + ObjectStoreImpl::Google(store) => Arc::new(PrefixStore::new(store, prefix)), } }