From 4b3ccf5934347c923e0b4d24d81f30f158be7cbf Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 4 Jun 2023 11:08:29 +0200 Subject: [PATCH 01/19] chore: remove deprecated create function on DeltaTable --- rust/src/delta.rs | 176 +++++----------------------------- rust/src/writer/test_utils.rs | 56 ++++------- rust/tests/common/mod.rs | 41 ++------ rust/tests/fs_common/mod.rs | 25 ++--- 4 files changed, 64 insertions(+), 234 deletions(-) diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 9a068d839b..636af1f643 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -794,59 +794,6 @@ impl DeltaTable { Ok(version) } - /// Create a DeltaTable with version 0 given the provided MetaData, Protocol, and CommitInfo - #[deprecated( - since = "0.10.0", - note = "use DelaOps from operations module to create a Create operation." - )] - #[allow(deprecated)] - pub async fn create( - &mut self, - metadata: DeltaTableMetaData, - protocol: action::Protocol, - commit_info: Option>, - add_actions: Option>, - ) -> Result<(), DeltaTableError> { - let meta = action::MetaData::try_from(metadata)?; - - // delta-rs commit info will include the delta-rs version and timestamp as of now - let mut enriched_commit_info = commit_info.unwrap_or_default(); - enriched_commit_info.insert( - "delta-rs".to_string(), - Value::String(crate_version().to_string()), - ); - enriched_commit_info.insert( - "timestamp".to_string(), - Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), - ); - - let mut actions = vec![ - Action::commit_info(enriched_commit_info), - Action::protocol(protocol), - Action::metaData(meta), - ]; - if let Some(add_actions) = add_actions { - for add_action in add_actions { - actions.push(Action::add(add_action)); - } - }; - - let mut transaction = self.create_transaction(None); - transaction.add_actions(actions.clone()); - - let prepared_commit = transaction.prepare_commit(None, None).await?; - let committed_version = self.try_commit_transaction(&prepared_commit, 0).await?; - - let new_state = DeltaTableState::from_commit(self, committed_version).await?; - self.state.merge( - new_state, - self.config.require_tombstones, - self.config.require_files, - ); - - Ok(()) - } - /// Time travel Delta table to the latest version that's created at or before provided /// `datetime` argument. /// @@ -1181,16 +1128,16 @@ pub fn crate_version() -> &'static str { #[cfg(test)] mod tests { use super::*; - use crate::action::Protocol; + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] use crate::builder::DeltaTableBuilder; + use crate::operations::create::CreateBuilder; use pretty_assertions::assert_eq; - use std::io::{BufRead, BufReader}; - use std::{collections::HashMap, fs::File, path::Path}; + use std::collections::HashMap; use tempdir::TempDir; #[tokio::test] async fn table_round_trip() { - let (_, _, dt, tmp_dir) = create_test_table().await; + let (dt, tmp_dir) = create_test_table().await; let bytes = serde_json::to_vec(&dt).unwrap(); let actual: DeltaTable = serde_json::from_slice(&bytes).unwrap(); assert_eq!(actual.version(), dt.version()); @@ -1213,107 +1160,32 @@ mod tests { } } - async fn create_test_table() -> (DeltaTableMetaData, Protocol, DeltaTable, TempDir) { - // Setup - let test_schema = Schema::new(vec![ - SchemaField::new( - "Id".to_string(), - SchemaDataType::primitive("integer".to_string()), - true, - HashMap::new(), - ), - SchemaField::new( - "Name".to_string(), - SchemaDataType::primitive("string".to_string()), - true, - HashMap::new(), - ), - ]); - - let delta_md = DeltaTableMetaData::new( - Some("Test Table Create".to_string()), - Some("This table is made to test the create function for a DeltaTable".to_string()), - None, - test_schema, - vec![], - HashMap::new(), - ); - - let protocol = action::Protocol { - min_reader_version: 1, - min_writer_version: 1, - }; - + async fn create_test_table() -> (DeltaTable, TempDir) { let tmp_dir = TempDir::new("create_table_test").unwrap(); let table_dir = tmp_dir.path().join("test_create"); std::fs::create_dir(&table_dir).unwrap(); - let mut dt = DeltaTableBuilder::from_uri(table_dir.to_str().unwrap()) - .build() - .unwrap(); - - let mut commit_info = Map::::new(); - commit_info.insert( - "operation".to_string(), - serde_json::Value::String("CREATE TABLE".to_string()), - ); - commit_info.insert( - "userName".to_string(), - serde_json::Value::String("test user".to_string()), - ); - // Action - #[allow(deprecated)] - dt.create(delta_md.clone(), protocol.clone(), Some(commit_info), None) + let dt = CreateBuilder::new() + .with_location(table_dir.to_str().unwrap()) + .with_table_name("Test Table Create") + .with_comment("This table is made to test the create function for a DeltaTable") + .with_columns(vec![ + SchemaField::new( + "Id".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "Name".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + ]) .await .unwrap(); - (delta_md, protocol, dt, tmp_dir) - } - #[tokio::test] - async fn test_create_delta_table() { - let (delta_md, protocol, dt, tmp_dir) = create_test_table().await; - - // Validation - // assert DeltaTable version is now 0 and no data files have been added - assert_eq!(dt.version(), 0); - assert_eq!(dt.state.files().len(), 0); - - // assert new _delta_log file created in tempDir - let table_uri = dt.table_uri(); - let table_path = Path::new(&table_uri); - assert!(table_path.exists()); - - let delta_log = table_path.join("_delta_log"); - assert!(delta_log.exists()); - - let version_file = delta_log.join("00000000000000000000.json"); - assert!(version_file.exists()); - - // Checking the data written to delta table is the same when read back - let version_data = File::open(version_file).unwrap(); - let lines = BufReader::new(version_data).lines(); - - for line in lines { - let action: Action = serde_json::from_str(line.unwrap().as_str()).unwrap(); - match action { - Action::protocol(action) => { - assert_eq!(action, protocol); - } - Action::metaData(action) => { - assert_eq!(DeltaTableMetaData::try_from(action).unwrap(), delta_md); - } - Action::commitInfo(action) => { - assert_eq!(action.operation, Some("CREATE TABLE".to_string())); - } - _ => (), - } - } - - // assert DeltaTableState metadata matches fields in above DeltaTableMetaData - // assert metadata name - let current_metadata = dt.get_metadata().unwrap(); - assert!(current_metadata.partition_columns.is_empty()); - assert!(current_metadata.configuration.is_empty()); - drop(tmp_dir); + (dt, tmp_dir) } } diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index a0e56e9fc8..d460e61cd9 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -1,19 +1,16 @@ -#![allow(deprecated)] //! Utilities for writing unit tests -use crate::{ - action::Protocol, schema::Schema, DeltaTable, DeltaTableBuilder, DeltaTableMetaData, - SchemaDataType, SchemaField, -}; -use arrow::datatypes::{DataType, Field}; -use arrow::record_batch::RecordBatch; -use arrow::{ - array::{Int32Array, StringArray, UInt32Array}, - compute::take, - datatypes::Schema as ArrowSchema, -}; + use std::collections::HashMap; use std::sync::Arc; +use arrow::compute::take; +use arrow_array::{Int32Array, RecordBatch, StringArray, UInt32Array}; +use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + +use crate::operations::create::CreateBuilder; +use crate::schema::Schema; +use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData, SchemaDataType, SchemaField}; + pub type TestResult = Result<(), Box>; pub fn get_record_batch(part: Option, with_null: bool) -> RecordBatch { @@ -177,35 +174,16 @@ pub fn create_bare_table() -> DeltaTable { } pub async fn create_initialized_table(partition_cols: &[String]) -> DeltaTable { - let mut table = create_bare_table(); let table_schema = get_delta_schema(); + let table_dir = tempfile::tempdir().unwrap(); + let table_path = table_dir.path(); - let mut commit_info = serde_json::Map::::new(); - commit_info.insert( - "operation".to_string(), - serde_json::Value::String("CREATE TABLE".to_string()), - ); - commit_info.insert( - "userName".to_string(), - serde_json::Value::String("test user".to_string()), - ); - - let protocol = Protocol { - min_reader_version: 1, - min_writer_version: 1, - }; - - let metadata = DeltaTableMetaData::new( - None, - None, - None, - table_schema, - partition_cols.to_vec(), - HashMap::new(), - ); - - table - .create(metadata, protocol, Some(commit_info), None) + let table = CreateBuilder::new() + .with_location(table_path.to_str().unwrap()) + .with_table_name("test-table") + .with_comment("A table for running tests") + .with_columns(table_schema.get_fields().clone()) + .with_partition_columns(partition_cols) .await .unwrap(); diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 0bcb7f02a0..6930c6ef36 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -3,10 +3,10 @@ use bytes::Bytes; use deltalake::action::{self, Add, Remove}; use deltalake::builder::DeltaTableBuilder; +use deltalake::operations::create::CreateBuilder; use deltalake::storage::DeltaObjectStore; -use deltalake::{DeltaTable, DeltaTableConfig, DeltaTableMetaData, Schema}; +use deltalake::{DeltaTable, Schema}; use object_store::{path::Path, ObjectStore}; -use serde_json::{Map, Value}; use std::any::Any; use std::collections::HashMap; use std::sync::Arc; @@ -81,36 +81,15 @@ impl TestContext { .iter() .map(|s| s.to_string()) .collect::>(); - let table_meta = DeltaTableMetaData::new( - Some("delta-rs_test_table".to_owned()), - Some("Table created by delta-rs tests".to_owned()), - None, - schema.clone(), - p, - HashMap::new(), - ); - let backend = self.new_storage(); - let mut dt = DeltaTable::new(backend, DeltaTableConfig::default()); - let mut commit_info = Map::::new(); - - let protocol = action::Protocol { - min_reader_version: 1, - min_writer_version: 2, - }; - - commit_info.insert( - "operation".to_string(), - serde_json::Value::String("CREATE TABLE".to_string()), - ); - dt.create( - table_meta.clone(), - protocol.clone(), - Some(commit_info), - None, - ) - .await - .unwrap(); + let dt = CreateBuilder::new() + .with_object_store(backend) + .with_table_name("delta-rs_test_table") + .with_comment("Table created by delta-rs tests") + .with_columns(schema.get_fields().clone()) + .with_partition_columns(p) + .await + .unwrap(); dt } diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index 4243afbe98..cc51ab80d3 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -1,9 +1,8 @@ #![allow(deprecated)] use chrono::Utc; -use deltalake::action::{Action, Add, Protocol, Remove}; -use deltalake::{ - builder::DeltaTableBuilder, DeltaTable, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, -}; +use deltalake::action::{Action, Add, Remove}; +use deltalake::operations::create::CreateBuilder; +use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; use serde_json::Value; use std::collections::HashMap; use std::fs; @@ -44,14 +43,16 @@ pub async fn create_test_table( partition_columns: Vec<&str>, config: HashMap>, ) -> DeltaTable { - let mut table = DeltaTableBuilder::from_uri(path).build().unwrap(); - let partition_columns = partition_columns.iter().map(|s| s.to_string()).collect(); - let md = DeltaTableMetaData::new(None, None, None, schema, partition_columns, config); - let protocol = Protocol { - min_reader_version: 1, - min_writer_version: 2, - }; - table.create(md, protocol, None, None).await.unwrap(); + let table = CreateBuilder::new() + .with_location(path) + .with_table_name("test-table") + .with_comment("A table for running tests") + .with_columns(schema.get_fields().clone()) + .with_partition_columns(partition_columns) + .with_configuration(config) + .await + .unwrap(); + table } From 9a6d87d07bed0e7b58846330055caedde9e08ebf Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 4 Jun 2023 11:12:38 +0200 Subject: [PATCH 02/19] chore: remove deprecated vacuum function on DeltaTable --- rust/src/delta.rs | 26 +------------------------- 1 file changed, 1 insertion(+), 25 deletions(-) diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 636af1f643..c5d3a8cef5 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -10,7 +10,7 @@ use std::io::{BufRead, BufReader, Cursor}; use std::sync::Arc; use std::{cmp::max, cmp::Ordering, collections::HashSet}; -use chrono::{DateTime, Duration, Utc}; +use chrono::{DateTime, Utc}; use futures::StreamExt; use lazy_static::lazy_static; use log::debug; @@ -31,7 +31,6 @@ use super::schema::*; use super::table_state::DeltaTableState; use crate::action::{Add, ProtocolError, Stats}; use crate::errors::DeltaTableError; -use crate::operations::vacuum::VacuumBuilder; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; // TODO re-exports only for transition @@ -724,29 +723,6 @@ impl DeltaTable { .get_configuration()) } - /// Vacuum the delta table. See [`VacuumBuilder`] for more information. - #[deprecated( - since = "0.10.0", - note = "use DelaOps from operations module to create a Vacuum operation." - )] - pub async fn vacuum( - &mut self, - retention_hours: Option, - dry_run: bool, - enforce_retention_duration: bool, - ) -> Result, DeltaTableError> { - let mut plan = VacuumBuilder::new(self.object_store(), self.state.clone()) - .with_dry_run(dry_run) - .with_enforce_retention_duration(enforce_retention_duration); - if let Some(hours) = retention_hours { - plan = plan.with_retention_period(Duration::hours(hours as i64)); - } - - let (table, metrics) = plan.await?; - self.state = table.state; - Ok(metrics.files_deleted) - } - /// Creates a new DeltaTransaction for the DeltaTable. /// The transaction holds a mutable reference to the DeltaTable, preventing other references /// until the transaction is dropped. From 57d0835f36b4b779c46f185fdeb8a7d60df5f0b8 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 4 Jun 2023 14:24:20 +0200 Subject: [PATCH 03/19] chore: remove deprecated commit 1/n --- rust/src/delta.rs | 265 +-------------- .../transaction/conflict_checker.rs | 29 +- rust/src/writer/mod.rs | 22 +- rust/src/writer/record_batch.rs | 30 +- rust/tests/checkpoint_writer.rs | 8 +- rust/tests/command_optimize.rs | 27 +- rust/tests/commit_info_format.rs | 18 +- rust/tests/common/mod.rs | 39 ++- .../_delta_log/00000000000000000000.json | 2 +- rust/tests/fs_common/mod.rs | 33 +- rust/tests/integration_commit.rs | 311 ------------------ rust/tests/integration_concurrent_writes.rs | 52 +-- 12 files changed, 162 insertions(+), 674 deletions(-) delete mode 100644 rust/tests/integration_commit.rs diff --git a/rust/src/delta.rs b/rust/src/delta.rs index c5d3a8cef5..640f17a791 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -1,7 +1,5 @@ //! Delta Table read and write implementation -// Reference: https://github.com/delta-io/delta/blob/master/PROTOCOL.md -// use std::collections::HashMap; use std::convert::TryFrom; use std::fmt; @@ -19,13 +17,9 @@ use regex::Regex; use serde::de::{Error, SeqAccess, Visitor}; use serde::ser::SerializeSeq; use serde::{Deserialize, Deserializer, Serialize, Serializer}; -use serde_json::{Map, Value}; use uuid::Uuid; -use super::action; -use super::action::{ - find_latest_check_point_for_version, get_last_checkpoint, Action, DeltaOperation, -}; +use super::action::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; use super::partitions::PartitionFilter; use super::schema::*; use super::table_state::DeltaTableState; @@ -723,53 +717,6 @@ impl DeltaTable { .get_configuration()) } - /// Creates a new DeltaTransaction for the DeltaTable. - /// The transaction holds a mutable reference to the DeltaTable, preventing other references - /// until the transaction is dropped. - #[deprecated( - since = "0.10.0", - note = "use 'commit' function from operations module to commit to Delta table." - )] - #[allow(deprecated)] - pub fn create_transaction( - &mut self, - options: Option, - ) -> DeltaTransaction { - DeltaTransaction::new(self, options) - } - - /// 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` - /// with retry logic. - #[deprecated( - since = "0.10.0", - note = "use 'commit' function from operations module to commite to Delta table." - )] - #[allow(deprecated)] - pub async fn try_commit_transaction( - &mut self, - commit: &PreparedCommit, - version: i64, - ) -> Result { - // 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, &commit_uri_from_version(version)) - .await - .map_err(|e| match e { - ObjectStoreError::AlreadyExists { .. } => { - DeltaTableError::VersionAlreadyExists(version) - } - _ => DeltaTableError::from(e), - })?; - - self.update().await?; - - Ok(version) - } - /// Time travel Delta table to the latest version that's created at or before provided /// `datetime` argument. /// @@ -839,216 +786,6 @@ impl std::fmt::Debug for DeltaTable { } } -const DEFAULT_DELTA_MAX_RETRY_COMMIT_ATTEMPTS: u32 = 10_000_000; - -/// Options for customizing behavior of a `DeltaTransaction` -#[derive(Debug)] -pub struct DeltaTransactionOptions { - /// number of retry attempts allowed when committing a transaction - max_retry_commit_attempts: u32, -} - -impl DeltaTransactionOptions { - /// Creates a new `DeltaTransactionOptions` - pub fn new(max_retry_commit_attempts: u32) -> Self { - Self { - max_retry_commit_attempts, - } - } -} - -impl Default for DeltaTransactionOptions { - fn default() -> Self { - Self { - max_retry_commit_attempts: DEFAULT_DELTA_MAX_RETRY_COMMIT_ATTEMPTS, - } - } -} - -/// Object representing a delta transaction. -/// Clients that do not need to mutate action content in case a transaction conflict is encountered -/// may use the `commit` method and rely on optimistic concurrency to determine the -/// appropriate Delta version number for a commit. A good example of this type of client is an -/// append only client that does not need to maintain transaction state with external systems. -/// Clients that may need to do conflict resolution if the Delta version changes should use -/// the `prepare_commit` and `try_commit_transaction` methods and manage the Delta version -/// themselves so that they can resolve data conflicts that may occur between Delta versions. -/// -/// Please not that in case of non-retryable error the temporary commit file such as -/// `_delta_log/_commit_.json` will orphaned in storage. -#[derive(Debug)] -#[deprecated( - since = "0.10.0", - note = "use 'commit' function from operations module to commit to Delta table." -)] -pub struct DeltaTransaction<'a> { - delta_table: &'a mut DeltaTable, - actions: Vec, - options: DeltaTransactionOptions, -} - -#[allow(deprecated)] -impl<'a> DeltaTransaction<'a> { - /// Creates a new delta transaction. - /// Holds a mutable reference to the delta table to prevent outside mutation while a transaction commit is in progress. - /// Transaction behavior may be customized by passing an instance of `DeltaTransactionOptions`. - pub fn new(delta_table: &'a mut DeltaTable, options: Option) -> Self { - DeltaTransaction { - delta_table, - actions: vec![], - options: options.unwrap_or_default(), - } - } - - /// Add an arbitrary "action" to the actions associated with this transaction - pub fn add_action(&mut self, action: action::Action) { - self.actions.push(action); - } - - /// Add an arbitrary number of actions to the actions associated with this transaction - pub fn add_actions(&mut self, actions: Vec) { - for action in actions.into_iter() { - self.actions.push(action); - } - } - - /// Commits the given actions to the delta log. - /// 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, - 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 commented 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 - // }; - 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).await?; - - Ok(version) - } - - /// 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 async fn prepare_commit( - &mut self, - operation: Option, - app_metadata: Option>, - ) -> Result { - if !self - .actions - .iter() - .any(|a| matches!(a, action::Action::commitInfo(..))) - { - 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(mut meta) = app_metadata { - extra_info.append(&mut meta) - } - commit_info.info = extra_info; - self.add_action(action::Action::commitInfo(commit_info)); - } - - // Serialize all actions that are part of this log entry. - let log_entry = bytes::Bytes::from( - log_entry_from_actions(&self.actions) - .map_err(|e| DeltaTableError::SerializeLogJson { json_err: e })?, - ); - - // 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. - let token = Uuid::new_v4().to_string(); - let file_name = format!("_commit_{token}.json.tmp"); - let path = Path::from_iter(["_delta_log", &file_name]); - - self.delta_table.storage.put(&path, log_entry).await?; - - Ok(PreparedCommit { uri: path }) - } - - #[allow(deprecated)] - async fn try_commit_loop(&mut self, commit: &PreparedCommit) -> Result { - let mut attempt_number: u32 = 0; - loop { - self.delta_table.update().await?; - - let version = self.delta_table.version() + 1; - - match self - .delta_table - .try_commit_transaction(commit, version) - .await - { - Ok(v) => { - return Ok(v); - } - Err(e) => { - match e { - DeltaTableError::VersionAlreadyExists(_) => { - 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); - } else { - attempt_number += 1; - debug!("Transaction attempt failed. Incrementing attempt number to {} and retrying.", attempt_number); - } - } - // NOTE: Add other retryable errors as needed here - _ => { - return Err(e); - } - } - } - } - } - } -} - -/// Holds the uri to prepared commit temporary file created with `DeltaTransaction.prepare_commit`. -/// Once created, the actual commit could be executed with `DeltaTransaction.try_commit`. -#[derive(Debug)] -#[deprecated( - since = "0.10.0", - note = "use 'commit' function from operations module to commit to Delta table." -)] -pub struct PreparedCommit { - uri: Path, -} - -fn log_entry_from_actions(actions: &[Action]) -> Result { - let mut jsons = Vec::::new(); - - for action in actions { - let json = serde_json::to_string(action)?; - jsons.push(json); - } - - Ok(jsons.join("\n")) -} - /// Creates and loads a DeltaTable from the given path with current metadata. /// Infers the storage backend to use from the scheme in the given table path. pub async fn open_table(table_uri: impl AsRef) -> Result { diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 1058400787..dc35fdb5e6 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -53,8 +53,8 @@ pub enum CommitConflictError { /// 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("Protocol changed since last commit: {0}")] + ProtocolChanged(String), /// Error returned when the table requires an unsupported writer version #[error("Delta-rs does not support writer version {0}")] @@ -392,10 +392,18 @@ 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.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); + let (win_read, curr_read) = ( + p.min_reader_version, + self.txn_info.read_snapshot.min_reader_version(), + ); + let (win_write, curr_write) = ( + p.min_writer_version, + self.txn_info.read_snapshot.min_writer_version(), + ); + if curr_read < win_read || win_write < curr_write { + return Err(CommitConflictError::ProtocolChanged( + format!("reqired read/write {win_read}/{win_write}, current read/write {curr_read}/{curr_write}"), + )); }; } if !self.winning_commit_summary.protocol().is_empty() @@ -405,7 +413,9 @@ impl<'a> ConflictChecker<'a> { .iter() .any(|a| matches!(a, Action::protocol(_))) { - return Err(CommitConflictError::ProtocolChanged); + return Err(CommitConflictError::ProtocolChanged( + "protocol changed".into(), + )); }; Ok(()) } @@ -818,7 +828,10 @@ mod tests { vec![tu::create_protocol_action(None, None)], false, ); - assert!(matches!(result, Err(CommitConflictError::ProtocolChanged))); + assert!(matches!( + result, + Err(CommitConflictError::ProtocolChanged(_)) + )); // taint whole table // `read_whole_table` should disallow any concurrent change, even if the change diff --git a/rust/src/writer/mod.rs b/rust/src/writer/mod.rs index e7dc5a38c5..fde0d808a7 100644 --- a/rust/src/writer/mod.rs +++ b/rust/src/writer/mod.rs @@ -7,8 +7,9 @@ use object_store::Error as ObjectStoreError; use parquet::errors::ParquetError; use serde_json::Value; -use crate::action::{Action, Add, ColumnCountStat}; +use crate::action::{Action, Add, ColumnCountStat, DeltaOperation}; use crate::errors::DeltaTableError; +use crate::operations::transaction::commit; use crate::DeltaTable; pub use json::JsonWriter; @@ -131,11 +132,20 @@ pub trait DeltaWriter { /// Flush the internal write buffers to files in the delta table folder structure. /// and commit the changes to the Delta log, creating a new table version. async fn flush_and_commit(&mut self, table: &mut DeltaTable) -> Result { - let mut adds = self.flush().await?; - #[allow(deprecated)] - let mut tx = table.create_transaction(None); - tx.add_actions(adds.drain(..).map(Action::add).collect()); - let version = tx.commit(None, None).await?; + let adds: Vec<_> = self.flush().await?.drain(..).map(Action::add).collect(); + let partition_cols = table.get_metadata()?.partition_columns.clone(); + let partition_by = if !partition_cols.is_empty() { + Some(partition_cols) + } else { + None + }; + let operation = DeltaOperation::Write { + mode: crate::action::SaveMode::Append, + partition_by, + predicate: None, + }; + let version = commit(table.storage.as_ref(), &adds, operation, &table.state, None).await?; + table.update().await?; Ok(version) } } diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index 78d491fae0..a254b91f78 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -4,37 +4,13 @@ //! Each Parquet file is buffered in-memory and only written once `flush()` is called on //! the writer. Once written, add actions are returned by the writer. It's the users responsibility //! to create the transaction using those actions. -//! -//! # Examples -//! -//! Write to an existing Delta Lake table: -//! ```rust ignore -//! let table = DeltaTable::try_from_uri("../path/to/table") -//! let batch: RecordBatch = ... -//! let mut writer = RecordBatchWriter::for_table(table, /*storage_options=*/ HashMap::new()) -//! writer.write(batch)?; -//! let actions: Vec = writer.flush()?.iter() -//! .map(|add| Action::add(add.into())) -//! .collect(); -//! let mut transaction = table.create_transaction(Some(DeltaTransactionOptions::new(/*max_retry_attempts=*/3))); -//! transaction.add_actions(actions); -//! async { -//! transaction.commit(Some(DeltaOperation::Write { -//! SaveMode::Append, -//! partitionBy: Some(table.get_metadata().partition_columns), -//! predicate: None, -//! })) -//! } -//! ``` + use std::{collections::HashMap, sync::Arc}; -use arrow::array::{Array, UInt32Array}; use arrow::compute::{lexicographical_partition_ranges, take, SortColumn}; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; -use arrow::error::ArrowError; -use arrow::record_batch::RecordBatch; -use arrow_array::ArrayRef; +use arrow_array::{Array, ArrayRef, RecordBatch, UInt32Array}; use arrow_row::{RowConverter, SortField}; +use arrow_schema::{ArrowError, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; use object_store::ObjectStore; use parquet::{arrow::ArrowWriter, errors::ParquetError}; diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index 80067814dd..42667cb165 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -2,6 +2,7 @@ #[cfg(all(feature = "arrow", feature = "parquet"))] mod fs_common; +use deltalake::action::DeltaOperation; // NOTE: The below is a useful external command for inspecting the written checkpoint schema visually: // parquet-tools inspect tests/data/checkpoints/_delta_log/00000000000000000005.checkpoint.parquet @@ -360,8 +361,11 @@ mod checkpoints_with_tombstones { .map(Action::remove) .chain(std::iter::once(Action::add(add.clone()))) .collect(); - - fs_common::commit_actions(table, actions).await; + let operation = DeltaOperation::Optimize { + predicate: None, + target_size: 1000000, + }; + fs_common::commit_actions(table, actions, operation).await; (removes, add) } diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 68eee17d07..0712a9fb77 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -4,16 +4,13 @@ use std::time::SystemTime; use std::time::UNIX_EPOCH; use std::{collections::HashMap, error::Error, sync::Arc}; -use arrow::datatypes::Schema as ArrowSchema; -use arrow::{ - array::{Int32Array, StringArray}, - datatypes::{DataType, Field}, - record_batch::RecordBatch, -}; +use arrow_array::{Int32Array, RecordBatch, StringArray}; +use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use arrow_select::concat::concat_batches; -use deltalake::action::{Action, Remove}; +use deltalake::action::{Action, DeltaOperation, Remove}; use deltalake::errors::DeltaTableError; use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics, OptimizeType}; +use deltalake::operations::transaction::commit; use deltalake::operations::DeltaOps; use deltalake::storage::ObjectStoreRef; use deltalake::writer::{DeltaWriter, RecordBatchWriter}; @@ -196,6 +193,7 @@ async fn write( ) -> Result<(), DeltaTableError> { writer.write(batch).await?; writer.flush_and_commit(table).await?; + table.update().await?; Ok(()) } @@ -280,7 +278,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { )?; let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); - let mut other_dt = deltalake::open_table(uri).await?; + let other_dt = deltalake::open_table(uri).await?; let add = &other_dt.get_state().files()[0]; let remove = Remove { path: add.path.clone(), @@ -297,10 +295,15 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { tags: Some(HashMap::new()), }; - #[allow(deprecated)] - let mut transaction = other_dt.create_transaction(None); - transaction.add_action(Action::remove(remove)); - transaction.commit(None, None).await?; + let operation = DeltaOperation::Delete { predicate: None }; + commit( + other_dt.object_store().as_ref(), + &vec![Action::remove(remove)], + operation, + &other_dt.state, + None, + ) + .await?; let maybe_metrics = plan.execute(dt.object_store(), &dt.state).await; assert!(maybe_metrics.is_err()); diff --git a/rust/tests/commit_info_format.rs b/rust/tests/commit_info_format.rs index 200178f849..c50a40d818 100644 --- a/rust/tests/commit_info_format.rs +++ b/rust/tests/commit_info_format.rs @@ -1,7 +1,8 @@ -#![allow(dead_code, deprecated)] +#![allow(dead_code)] mod fs_common; use deltalake::action::{Action, DeltaOperation, SaveMode}; +use deltalake::operations::transaction::commit; use serde_json::json; use std::error::Error; use tempdir::TempDir; @@ -12,17 +13,22 @@ async fn test_operational_parameters() -> Result<(), Box> { let mut table = fs_common::create_table(path.path().to_str().unwrap(), None).await; let add = fs_common::add(0); - + let actions = vec![Action::add(add)]; let operation = DeltaOperation::Write { mode: SaveMode::Append, partition_by: Some(vec!["some_partition".to_string()]), predicate: None, }; - let mut tx = table.create_transaction(None); - let actions = vec![Action::add(add.clone())]; - tx.add_actions(actions); - tx.commit(Some(operation), None).await.unwrap(); + commit( + table.object_store().as_ref(), + &actions, + operation, + &table.state, + None, + ) + .await?; + table.update().await?; let commit_info = table.history(None).await?; let last_commit = &commit_info[commit_info.len() - 1]; diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 6930c6ef36..3b6b18c35a 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -1,9 +1,10 @@ -#![allow(dead_code, unused_variables, deprecated)] +#![allow(dead_code, unused_variables)] use bytes::Bytes; -use deltalake::action::{self, Add, Remove}; +use deltalake::action::{self, Add, DeltaOperation, Remove, SaveMode}; use deltalake::builder::DeltaTableBuilder; use deltalake::operations::create::CreateBuilder; +use deltalake::operations::transaction::commit; use deltalake::storage::DeltaObjectStore; use deltalake::{DeltaTable, Schema}; use object_store::{path::Path, ObjectStore}; @@ -137,9 +138,22 @@ pub async fn add_file( data_change: true, ..Default::default() }; - let mut transaction = table.create_transaction(None); - transaction.add_action(action::Action::add(add)); - transaction.commit(None, None).await.unwrap(); + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + let actions = vec![action::Action::add(add)]; + commit( + table.object_store().as_ref(), + &actions, + operation, + &table.state, + None, + ) + .await + .unwrap(); + table.update().await.unwrap(); } } @@ -161,7 +175,16 @@ pub async fn remove_file( data_change: true, ..Default::default() }; - let mut transaction = table.create_transaction(None); - transaction.add_action(action::Action::remove(remove)); - transaction.commit(None, None).await.unwrap(); + let operation = DeltaOperation::Delete { predicate: None }; + let actions = vec![action::Action::remove(remove)]; + commit( + table.object_store().as_ref(), + &actions, + operation, + &table.state, + None, + ) + .await + .unwrap(); + table.update().await.unwrap(); } diff --git a/rust/tests/data/concurrent_workers/_delta_log/00000000000000000000.json b/rust/tests/data/concurrent_workers/_delta_log/00000000000000000000.json index 891536001b..36701a9e96 100644 --- a/rust/tests/data/concurrent_workers/_delta_log/00000000000000000000.json +++ b/rust/tests/data/concurrent_workers/_delta_log/00000000000000000000.json @@ -1,3 +1,3 @@ {"commitInfo":{"timestamp":1564524295023,"operation":"CREATE TABLE","operationParameters":{"isManaged":"false","description":null,"partitionBy":"[]","properties":"{}"},"isBlindAppend":true}} -{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} {"metaData":{"id":"22ef18ba-191c-4c36-a606-3dad5cdf3830","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1564524294376}} diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index cc51ab80d3..8e0e953378 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -1,7 +1,8 @@ #![allow(deprecated)] use chrono::Utc; -use deltalake::action::{Action, Add, Remove}; +use deltalake::action::{Action, Add, DeltaOperation, Remove, SaveMode}; use deltalake::operations::create::CreateBuilder; +use deltalake::operations::transaction::commit; use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; use serde_json::Value; use std::collections::HashMap; @@ -89,7 +90,12 @@ pub fn add(offset_millis: i64) -> Add { } pub async fn commit_add(table: &mut DeltaTable, add: &Add) -> i64 { - commit_actions(table, vec![Action::add(add.clone())]).await + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + commit_actions(table, vec![Action::add(add.clone())], operation).await } pub async fn commit_removes(table: &mut DeltaTable, removes: Vec<&Remove>) -> i64 { @@ -97,11 +103,24 @@ pub async fn commit_removes(table: &mut DeltaTable, removes: Vec<&Remove>) -> i6 .iter() .map(|r| Action::remove((*r).clone())) .collect(); - commit_actions(table, vec).await + let operation = DeltaOperation::Delete { predicate: None }; + commit_actions(table, vec, operation).await } -pub async fn commit_actions(table: &mut DeltaTable, actions: Vec) -> i64 { - let mut tx = table.create_transaction(None); - tx.add_actions(actions); - tx.commit(None, None).await.unwrap() +pub async fn commit_actions( + table: &mut DeltaTable, + actions: Vec, + operation: DeltaOperation, +) -> i64 { + let version = commit( + table.object_store().as_ref(), + &actions, + operation, + &table.state, + None, + ) + .await + .unwrap(); + table.update().await.unwrap(); + version } diff --git a/rust/tests/integration_commit.rs b/rust/tests/integration_commit.rs deleted file mode 100644 index e8f6e8bd91..0000000000 --- a/rust/tests/integration_commit.rs +++ /dev/null @@ -1,311 +0,0 @@ -#![cfg(feature = "integration_test")] - -#[allow(dead_code)] -mod fs_common; - -use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; -use deltalake::{action, errors::DeltaTableError, DeltaTableBuilder}; -use serial_test::serial; -use std::collections::HashMap; - -#[tokio::test] -#[serial] -async fn test_commit_tables_local() { - commit_tables(StorageIntegration::Local).await.unwrap(); -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[tokio::test] -#[serial] -async fn test_commit_tables_aws() { - std::env::set_var("AWS_S3_LOCKING_PROVIDER", "dynamodb"); - commit_tables(StorageIntegration::Amazon).await.unwrap(); -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_commit_tables_azure() { - commit_tables(StorageIntegration::Microsoft).await.unwrap(); -} - -#[cfg(feature = "gcs")] -#[tokio::test] -#[serial] -async fn test_commit_tables_gcp() { - commit_tables(StorageIntegration::Google).await.unwrap(); -} - -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn test_commit_tables_hdfs() { - commit_tables(StorageIntegration::Hdfs).await.unwrap(); -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[tokio::test] -#[serial] -async fn test_two_commits_s3_fails_with_no_lock() -> TestResult { - std::env::set_var("AWS_S3_LOCKING_PROVIDER", "none "); - let context = IntegrationContext::new(StorageIntegration::Amazon)?; - context.load_table(TestTables::SimpleCommit).await?; - let table_uri = context.uri_for_table(TestTables::SimpleCommit); - - let result = test_two_commits(&table_uri).await; - assert!(result.is_err()); - - let err_msg = result.err().unwrap().to_string(); - assert!(err_msg.contains("Atomic rename requires a LockClient for S3 backends.")); - - Ok(()) -} - -async fn commit_tables(storage: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(storage)?; - - context - .load_table_with_name(TestTables::SimpleCommit, "simple_commit_1") - .await?; - let table_uri = context.uri_for_table(TestTables::Custom("simple_commit_1".into())); - test_two_commits(&table_uri).await?; - - context - .load_table_with_name(TestTables::SimpleCommit, "simple_commit_2") - .await?; - let table_uri = context.uri_for_table(TestTables::Custom("simple_commit_2".into())); - test_commit_version_succeeds_if_version_does_not_exist(&table_uri).await?; - - Ok(()) -} - -async fn test_commit_version_succeeds_if_version_does_not_exist( - table_path: &str, -) -> Result<(), DeltaTableError> { - let mut table = DeltaTableBuilder::from_uri(table_path) - .with_allow_http(true) - .load() - .await?; - - assert_eq!(0, table.version()); - assert_eq!(0, table.get_files().len()); - - let mut tx1 = table.create_transaction(None); - tx1.add_actions(tx1_actions()); - let commit = tx1.prepare_commit(None, None).await?; - let result = table.try_commit_transaction(&commit, 1).await?; - - assert_eq!(1, result); - assert_eq!(1, table.version()); - assert_eq!(2, table.get_files().len()); - - Ok(()) -} - -mod simple_commit_fs { - use super::*; - - #[tokio::test] - #[serial] - async fn test_commit_version_succeeds_if_version_does_not_exist() { - prepare_fs(); - - let table_path = "./tests/data/simple_commit"; - let mut table = deltalake::open_table(table_path).await.unwrap(); - - assert_eq!(0, table.version()); - assert_eq!(0, table.get_files().len()); - - let mut tx1 = table.create_transaction(None); - tx1.add_actions(tx1_actions()); - let commit = tx1.prepare_commit(None, None).await.unwrap(); - let result = table.try_commit_transaction(&commit, 1).await.unwrap(); - - assert_eq!(1, result); - assert_eq!(1, table.version()); - assert_eq!(2, table.get_files().len()); - - prepare_fs(); - } - - #[tokio::test] - #[serial] - async fn test_commit_version_fails_if_version_exists() { - prepare_fs(); - - let table_path = "./tests/data/simple_commit"; - let mut table = deltalake::open_table(table_path).await.unwrap(); - - assert_eq!(0, table.version()); - assert_eq!(0, table.get_files().len()); - - let mut tx1 = table.create_transaction(None); - tx1.add_actions(tx1_actions()); - let commit = tx1.prepare_commit(None, None).await.unwrap(); - let _ = table.try_commit_transaction(&commit, 1).await.unwrap(); - - let mut tx2 = table.create_transaction(None); - tx2.add_actions(tx2_actions()); - // we already committed version 1 - this should fail and return error for caller to handle. - let commit = tx2.prepare_commit(None, None).await.unwrap(); - let result = table.try_commit_transaction(&commit, 1).await; - - match result { - Err(DeltaTableError::VersionAlreadyExists(_)) => { - assert!(true, "Delta version already exists."); - } - _ => { - assert!(false, "Delta version should already exist."); - } - } - - assert!(result.is_err()); - assert_eq!(1, table.version()); - assert_eq!(2, table.get_files().len()); - - prepare_fs(); - } - - // This test shows an example on how to use low-level transaction API with custom optimistic - // concurrency loop and retry logic. - #[tokio::test] - #[serial] - async fn test_low_level_tx_api() { - prepare_fs(); - - let table_path = "./tests/data/simple_commit"; - let mut table = deltalake::open_table(table_path).await.unwrap(); - - assert_eq!(0, table.version()); - assert_eq!(0, table.get_files().len()); - - let mut attempt = 0; - let prepared_commit = { - let mut tx = table.create_transaction(None); - tx.add_actions(tx1_actions()); - tx.prepare_commit(None, None).await.unwrap() - }; - - loop { - table.update().await.unwrap(); - - let version = table.version() + 1; - match table - .try_commit_transaction(&prepared_commit, version) - .await - { - Ok(_) => { - break; - } - Err(DeltaTableError::VersionAlreadyExists(_)) => { - attempt += 1; - } - Err(e) => { - panic!("{}", e) - } - } - } - - assert_eq!(0, attempt); - assert_eq!(1, table.version()); - assert_eq!(2, table.get_files().len()); - - prepare_fs(); - } - - fn prepare_fs() { - fs_common::cleanup_dir_except( - "./tests/data/simple_commit/_delta_log", - vec!["00000000000000000000.json".to_string()], - ); - } -} - -async fn test_two_commits(table_path: &str) -> Result<(), DeltaTableError> { - let mut table = DeltaTableBuilder::from_uri(table_path) - .with_allow_http(true) - .load() - .await?; - - assert_eq!(0, table.version()); - assert_eq!(0, table.get_files().len()); - - let mut tx1 = table.create_transaction(None); - tx1.add_actions(tx1_actions()); - let version = tx1.commit(None, None).await?; - - assert_eq!(1, version); - assert_eq!(version, table.version()); - assert_eq!(2, table.get_files().len()); - - let mut tx2 = table.create_transaction(None); - tx2.add_actions(tx2_actions()); - let version = tx2.commit(None, None).await.unwrap(); - - assert_eq!(2, version); - assert_eq!(version, table.version()); - assert_eq!(4, table.get_files().len()); - Ok(()) -} - -fn tx1_actions() -> Vec { - vec![ - action::Action::add(action::Add { - path: String::from( - "part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet", - ), - size: 396, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524294000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - }), - action::Action::add(action::Add { - path: String::from( - "part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet", - ), - size: 400, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524294000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - }), - ] -} - -fn tx2_actions() -> Vec { - vec![ - action::Action::add(action::Add { - path: String::from( - "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet", - ), - size: 396, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524296000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - }), - action::Action::add(action::Add { - path: String::from( - "part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet", - ), - size: 400, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524296000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - }), - ] -} diff --git a/rust/tests/integration_concurrent_writes.rs b/rust/tests/integration_concurrent_writes.rs index 739d8bffde..de735a18d4 100644 --- a/rust/tests/integration_concurrent_writes.rs +++ b/rust/tests/integration_concurrent_writes.rs @@ -1,9 +1,10 @@ #![cfg(feature = "integration_test")] +use deltalake::action::{Action, Add, DeltaOperation, SaveMode}; +use deltalake::operations::transaction::commit; +use deltalake::operations::DeltaOps; use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; -use deltalake::{ - action, DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, -}; +use deltalake::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; use std::collections::HashMap; use std::future::Future; use std::iter::FromIterator; @@ -54,28 +55,21 @@ async fn prepare_table( true, HashMap::new(), )]); - let metadata = DeltaTableMetaData::new( - Some("Azure Test Table".to_string()), - None, - None, - schema, - vec![], - HashMap::new(), - ); - let protocol = action::Protocol { - min_reader_version: 1, - min_writer_version: 2, - }; let table_uri = context.uri_for_table(TestTables::Custom("concurrent_workers".into())); - let mut table = DeltaTableBuilder::from_uri(&table_uri) + + let table = DeltaTableBuilder::from_uri(&table_uri) .with_allow_http(true) .build()?; - table.create(metadata, protocol, None, None).await?; + + let table = DeltaOps(table) + .create() + .with_columns(schema.get_fields().clone()) + .await?; assert_eq!(0, table.version()); assert_eq!(1, table.get_min_reader_version()); - assert_eq!(2, table.get_min_writer_version()); + assert_eq!(1, table.get_min_writer_version()); assert_eq!(0, table.get_files().len()); Ok((table, table_uri)) @@ -153,8 +147,12 @@ impl Worker { } async fn commit_file(&mut self, name: &str) -> i64 { - let mut tx = self.table.create_transaction(None); - tx.add_action(action::Action::add(action::Add { + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + let actions = vec![Action::add(Add { path: format!("{}.parquet", name), size: 396, partition_values: HashMap::new(), @@ -164,7 +162,17 @@ impl Worker { stats: None, stats_parsed: None, tags: None, - })); - tx.commit(None, None).await.unwrap() + })]; + let version = commit( + self.table.object_store().as_ref(), + &actions, + operation, + &self.table.state, + None, + ) + .await + .unwrap(); + self.table.update().await.unwrap(); + version } } From 05c51a9c4802a0d534e1fcbba094fcb37d3698c7 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 5 Jun 2023 00:34:37 +0200 Subject: [PATCH 04/19] fix: actually load table in concurrency tests. --- rust/src/operations/transaction/mod.rs | 3 ++- rust/tests/integration_concurrent_writes.rs | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index ba285bd046..d1a70ba7b1 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -164,7 +164,8 @@ pub async fn commit( ) -> DeltaResult { let tmp_commit = prepare_commit(storage, &operation, actions, app_metadata).await?; - let max_attempts = 5; + // TODO make max attempts configurable + let max_attempts = 10; let mut attempt_number = 1; while attempt_number <= max_attempts { diff --git a/rust/tests/integration_concurrent_writes.rs b/rust/tests/integration_concurrent_writes.rs index de735a18d4..91e4345963 100644 --- a/rust/tests/integration_concurrent_writes.rs +++ b/rust/tests/integration_concurrent_writes.rs @@ -129,7 +129,8 @@ impl Worker { std::env::set_var("DYNAMO_LOCK_OWNER_NAME", &name); let table = DeltaTableBuilder::from_uri(path) .with_allow_http(true) - .build() + .load() + .await .unwrap(); Self { table, name } } From e56f003a3768bb3fb3159acd107efa090bb829d1 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 5 Jun 2023 01:15:30 +0200 Subject: [PATCH 05/19] refactor: move table into separate module --- python/src/lib.rs | 2 +- rust/benches/read_checkpoint.rs | 4 ++-- rust/src/action/checkpoints.rs | 2 +- rust/src/action/mod.rs | 2 +- rust/src/delta_datafusion.rs | 2 +- rust/src/lib.rs | 10 ++-------- rust/src/operations/delete.rs | 2 +- rust/src/operations/filesystem_check.rs | 2 +- rust/src/operations/load.rs | 2 +- rust/src/operations/optimize.rs | 2 +- .../operations/transaction/conflict_checker.rs | 2 +- rust/src/operations/transaction/mod.rs | 2 +- rust/src/operations/transaction/state.rs | 2 +- rust/src/operations/transaction/test_utils.rs | 2 +- rust/src/operations/vacuum.rs | 2 +- rust/src/operations/write.rs | 4 ++-- rust/src/{ => table}/builder.rs | 2 +- rust/src/{delta.rs => table/mod.rs} | 16 ++++++++++------ rust/src/{table_state.rs => table/state.rs} | 0 .../state_arrow.rs} | 2 +- rust/tests/common/mod.rs | 2 +- rust/tests/fs_common/mod.rs | 1 - 22 files changed, 32 insertions(+), 35 deletions(-) rename rust/src/{ => table}/builder.rs (99%) rename rust/src/{delta.rs => table/mod.rs} (99%) rename rust/src/{table_state.rs => table/state.rs} (100%) rename rust/src/{table_state_arrow.rs => table/state_arrow.rs} (99%) diff --git a/python/src/lib.rs b/python/src/lib.rs index 8e2c4d282f..8f4d9c17e3 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -19,7 +19,6 @@ use deltalake::action::{ use deltalake::arrow::compute::concat_batches; use deltalake::arrow::record_batch::RecordBatch; use deltalake::arrow::{self, datatypes::Schema as ArrowSchema}; -use deltalake::builder::DeltaTableBuilder; use deltalake::checkpoints::create_checkpoint; use deltalake::datafusion::prelude::SessionContext; use deltalake::delta_datafusion::DeltaDataChecker; @@ -28,6 +27,7 @@ use deltalake::operations::optimize::OptimizeBuilder; use deltalake::operations::transaction::commit; use deltalake::operations::vacuum::VacuumBuilder; use deltalake::partitions::PartitionFilter; +use deltalake::DeltaTableBuilder; use deltalake::{DeltaOps, Invariant, Schema}; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; use pyo3::prelude::*; diff --git a/rust/benches/read_checkpoint.rs b/rust/benches/read_checkpoint.rs index 94c6abff9a..b1019cf17a 100644 --- a/rust/benches/read_checkpoint.rs +++ b/rust/benches/read_checkpoint.rs @@ -1,6 +1,6 @@ use criterion::{criterion_group, criterion_main, Criterion}; -use deltalake::delta::DeltaTableConfig; -use deltalake::table_state::DeltaTableState; +use deltalake::state::DeltaTableState; +use deltalake::DeltaTableConfig; use std::fs::File; use std::io::Read; diff --git a/rust/src/action/checkpoints.rs b/rust/src/action/checkpoints.rs index 499e49e6c7..d453dbd844 100644 --- a/rust/src/action/checkpoints.rs +++ b/rust/src/action/checkpoints.rs @@ -21,7 +21,7 @@ use super::{Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; use crate::delta_arrow::delta_log_schema_for_table; use crate::schema::*; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::{open_table_with_version, time_utils, CheckPoint, DeltaTable}; type SchemaPath = Vec; diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index d2f0004316..ebb9e49b06 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -26,7 +26,7 @@ use std::hash::{Hash, Hasher}; use crate::delta_config::IsolationLevel; use crate::errors::DeltaResult; use crate::storage::ObjectStoreRef; -use crate::{delta::CheckPoint, schema::*, DeltaTableMetaData}; +use crate::{schema::*, table::CheckPoint, DeltaTableMetaData}; /// Error returned when an invalid Delta log action is encountered. #[allow(missing_docs)] diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 7c4fbf8e4c..d7d5b38cd1 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -61,7 +61,7 @@ use crate::action::{self, Add}; use crate::builder::ensure_table_uri; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::{open_table, open_table_with_storage_options, DeltaTable, Invariant, SchemaDataType}; impl From for DataFusionError { diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 5323c34eff..945ebb11b7 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -83,21 +83,16 @@ compile_error!( ); pub mod action; -pub mod builder; pub mod data_catalog; -pub mod delta; pub mod delta_config; pub mod errors; pub mod operations; pub mod partitions; pub mod schema; pub mod storage; -pub mod table_state; +pub mod table; pub mod time_utils; -#[cfg(all(feature = "arrow"))] -pub mod table_state_arrow; - #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod delta_arrow; #[cfg(feature = "datafusion")] @@ -105,12 +100,11 @@ pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; -pub use self::builder::*; pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; -pub use self::delta::*; pub use self::delta_config::*; pub use self::partitions::*; pub use self::schema::*; +pub use self::table::*; pub use errors::*; pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; pub use operations::DeltaOps; diff --git a/rust/src/operations/delete.rs b/rust/src/operations/delete.rs index 064695aa4f..5f139e8b23 100644 --- a/rust/src/operations/delete.rs +++ b/rust/src/operations/delete.rs @@ -61,7 +61,7 @@ use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; use crate::operations::write::write_execution_plan; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; const PATH_COLUMN: &str = "__delta_rs_path"; diff --git a/rust/src/operations/filesystem_check.rs b/rust/src/operations/filesystem_check.rs index 18a387d3db..be2b21ee17 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/rust/src/operations/filesystem_check.rs @@ -28,7 +28,7 @@ use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; /// Audit the Delta Table's active files with the underlying file system. diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs index 9501c18011..7baa59e3e1 100644 --- a/rust/src/operations/load.rs +++ b/rust/src/operations/load.rs @@ -8,7 +8,7 @@ use futures::future::BoxFuture; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; #[derive(Debug, Clone)] diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index a49c6a9940..6793aa65a6 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -46,7 +46,7 @@ use super::writer::{PartitionWriter, PartitionWriterConfig}; use crate::action::{self, Action, DeltaOperation}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; use crate::{crate_version, DeltaTable, ObjectMeta, PartitionFilter}; diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index dc35fdb5e6..4ad044e19e 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -9,7 +9,7 @@ use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::delta_config::IsolationLevel; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; #[cfg(feature = "datafusion")] use super::state::AddContainer; diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index d1a70ba7b1..e3cbc5c70e 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -9,7 +9,7 @@ use crate::action::{Action, CommitInfo, DeltaOperation}; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; mod conflict_checker; #[cfg(feature = "datafusion")] diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index cb1b336c3c..d27713e27c 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -24,7 +24,7 @@ use crate::delta_datafusion::{ get_null_of_arrow_type, logical_expr_to_physical_expr, to_correct_scalar_value, }; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; impl DeltaTableState { /// Get the table schema as an [`ArrowSchemaRef`] diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 92d981d5e1..86e1db6524 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -1,7 +1,7 @@ #![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; +use crate::table::state::DeltaTableState; use crate::{ DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, }; diff --git a/rust/src/operations/vacuum.rs b/rust/src/operations/vacuum.rs index 035f6f7c53..6c9e6dc19e 100644 --- a/rust/src/operations/vacuum.rs +++ b/rust/src/operations/vacuum.rs @@ -32,7 +32,7 @@ use object_store::{path::Path, ObjectStore}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; /// Errors that can occur during vacuum diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index b170315dcf..aa3b90a9db 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -32,14 +32,14 @@ use super::writer::{DeltaWriter, WriterConfig}; use super::MAX_SUPPORTED_WRITER_VERSION; use super::{transaction::commit, CreateBuilder}; use crate::action::{Action, Add, DeltaOperation, Remove, SaveMode}; -use crate::delta::DeltaTable; use crate::delta_datafusion::DeltaDataChecker; use crate::errors::{DeltaResult, DeltaTableError}; use crate::schema::Schema; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::writer::record_batch::divide_by_partition_values; use crate::writer::utils::PartitionPath; +use crate::DeltaTable; #[derive(thiserror::Error, Debug)] enum WriteError { diff --git a/rust/src/builder.rs b/rust/src/table/builder.rs similarity index 99% rename from rust/src/builder.rs rename to rust/src/table/builder.rs index 79dd1faa57..29f8eeaccb 100644 --- a/rust/src/builder.rs +++ b/rust/src/table/builder.rs @@ -9,7 +9,7 @@ use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; use url::Url; -use crate::delta::DeltaTable; +use super::DeltaTable; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::config::StorageOptions; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; diff --git a/rust/src/delta.rs b/rust/src/table/mod.rs similarity index 99% rename from rust/src/delta.rs rename to rust/src/table/mod.rs index 640f17a791..a8724d4102 100644 --- a/rust/src/delta.rs +++ b/rust/src/table/mod.rs @@ -19,16 +19,20 @@ use serde::ser::SerializeSeq; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use uuid::Uuid; -use super::action::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; -use super::partitions::PartitionFilter; -use super::schema::*; -use super::table_state::DeltaTableState; +use self::state::DeltaTableState; +use crate::action::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; use crate::action::{Add, ProtocolError, Stats}; use crate::errors::DeltaTableError; +use crate::partitions::PartitionFilter; +use crate::schema::*; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; -// TODO re-exports only for transition -pub use crate::builder::{DeltaTableBuilder, DeltaTableConfig, DeltaVersion}; +pub(crate) mod builder; +pub mod state; +#[cfg(all(feature = "arrow"))] +pub mod state_arrow; + +pub use builder::*; /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] diff --git a/rust/src/table_state.rs b/rust/src/table/state.rs similarity index 100% rename from rust/src/table_state.rs rename to rust/src/table/state.rs diff --git a/rust/src/table_state_arrow.rs b/rust/src/table/state_arrow.rs similarity index 99% rename from rust/src/table_state_arrow.rs rename to rust/src/table/state_arrow.rs index 48cf8c2de2..19ac8f1957 100644 --- a/rust/src/table_state_arrow.rs +++ b/rust/src/table/state_arrow.rs @@ -16,9 +16,9 @@ use arrow_array::{ use arrow_schema::{DataType, Field, Fields, TimeUnit}; use itertools::Itertools; +use super::state::DeltaTableState; use crate::action::{ColumnCountStat, ColumnValueStat, Stats}; use crate::errors::DeltaTableError; -use crate::table_state::DeltaTableState; use crate::SchemaDataType; use crate::SchemaTypeStruct; diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 3b6b18c35a..fcce3c4b37 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -2,10 +2,10 @@ use bytes::Bytes; use deltalake::action::{self, Add, DeltaOperation, Remove, SaveMode}; -use deltalake::builder::DeltaTableBuilder; use deltalake::operations::create::CreateBuilder; use deltalake::operations::transaction::commit; use deltalake::storage::DeltaObjectStore; +use deltalake::DeltaTableBuilder; use deltalake::{DeltaTable, Schema}; use object_store::{path::Path, ObjectStore}; use std::any::Any; diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index 8e0e953378..7bca9386e8 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -1,4 +1,3 @@ -#![allow(deprecated)] use chrono::Utc; use deltalake::action::{Action, Add, DeltaOperation, Remove, SaveMode}; use deltalake::operations::create::CreateBuilder; From d1e0b321666d62f90d68794f1cc625854588723f Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 5 Jun 2023 01:21:14 +0200 Subject: [PATCH 06/19] refactor: rename arrow module --- rust/src/action/checkpoints.rs | 2 +- rust/src/{delta_arrow.rs => arrow_convert.rs} | 0 rust/src/lib.rs | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) rename rust/src/{delta_arrow.rs => arrow_convert.rs} (100%) diff --git a/rust/src/action/checkpoints.rs b/rust/src/action/checkpoints.rs index d453dbd844..7f62e63627 100644 --- a/rust/src/action/checkpoints.rs +++ b/rust/src/action/checkpoints.rs @@ -18,7 +18,7 @@ use std::iter::Iterator; use std::ops::Add; use super::{Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; -use crate::delta_arrow::delta_log_schema_for_table; +use crate::arrow_convert::delta_log_schema_for_table; use crate::schema::*; use crate::storage::DeltaObjectStore; use crate::table::state::DeltaTableState; diff --git a/rust/src/delta_arrow.rs b/rust/src/arrow_convert.rs similarity index 100% rename from rust/src/delta_arrow.rs rename to rust/src/arrow_convert.rs diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 945ebb11b7..05f7574a90 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -94,7 +94,7 @@ pub mod table; pub mod time_utils; #[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod delta_arrow; +pub mod arrow_convert; #[cfg(feature = "datafusion")] pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] From cf8c1045b40868dfdd816908be162b1ad4098f56 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 5 Jun 2023 01:46:51 +0200 Subject: [PATCH 07/19] fix: test import --- rust/tests/checkpoint_writer.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index 42667cb165..5852e9c6f9 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -1,5 +1,3 @@ -#![deny(warnings)] - #[cfg(all(feature = "arrow", feature = "parquet"))] mod fs_common; use deltalake::action::DeltaOperation; From 17fb81b45edbfbbd8f4430770df2a96621f01883 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 5 Jun 2023 17:30:30 +0200 Subject: [PATCH 08/19] fix: clippy --- rust/src/writer/test_utils.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index d460e61cd9..3eb974a207 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -178,16 +178,14 @@ pub async fn create_initialized_table(partition_cols: &[String]) -> DeltaTable { let table_dir = tempfile::tempdir().unwrap(); let table_path = table_dir.path(); - let table = CreateBuilder::new() + CreateBuilder::new() .with_location(table_path.to_str().unwrap()) .with_table_name("test-table") .with_comment("A table for running tests") .with_columns(table_schema.get_fields().clone()) .with_partition_columns(partition_cols) .await - .unwrap(); - - table + .unwrap() } #[cfg(feature = "datafusion")] From 1fb080cbadb2f6b111849141f3214ea00bf0388d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 07:35:54 +0200 Subject: [PATCH 09/19] fix: docs link --- rust/src/table/state_arrow.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/src/table/state_arrow.rs b/rust/src/table/state_arrow.rs index 19ac8f1957..b286482ce6 100644 --- a/rust/src/table/state_arrow.rs +++ b/rust/src/table/state_arrow.rs @@ -1,6 +1,6 @@ //! Methods to get Delta Table state in Arrow structures //! -//! See [crate::table_state::DeltaTableState]. +//! See [crate::table::DeltaTableState]. use std::borrow::Cow; use std::collections::{HashMap, HashSet, VecDeque}; From 1d5a9d21b0719f8eb45b1ae5781da33a1b34c1bf Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 07:54:39 +0200 Subject: [PATCH 10/19] refactor: move all schema-related into schema module --- rust/src/lib.rs | 5 +-- rust/src/{ => schema}/arrow_convert.rs | 0 rust/src/{schema.rs => schema/mod.rs} | 4 ++ rust/src/{ => schema}/partitions.rs | 54 ++++++++++++-------------- 4 files changed, 30 insertions(+), 33 deletions(-) rename rust/src/{ => schema}/arrow_convert.rs (100%) rename rust/src/{schema.rs => schema/mod.rs} (99%) rename rust/src/{ => schema}/partitions.rs (90%) diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 05f7574a90..04cf198238 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -87,14 +87,11 @@ pub mod data_catalog; pub mod delta_config; pub mod errors; pub mod operations; -pub mod partitions; pub mod schema; pub mod storage; pub mod table; pub mod time_utils; -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod arrow_convert; #[cfg(feature = "datafusion")] pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] @@ -102,7 +99,7 @@ pub mod writer; pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; pub use self::delta_config::*; -pub use self::partitions::*; +pub use self::schema::partitions::*; pub use self::schema::*; pub use self::table::*; pub use errors::*; diff --git a/rust/src/arrow_convert.rs b/rust/src/schema/arrow_convert.rs similarity index 100% rename from rust/src/arrow_convert.rs rename to rust/src/schema/arrow_convert.rs diff --git a/rust/src/schema.rs b/rust/src/schema/mod.rs similarity index 99% rename from rust/src/schema.rs rename to rust/src/schema/mod.rs index 2602c5cd68..a853725fc6 100644 --- a/rust/src/schema.rs +++ b/rust/src/schema/mod.rs @@ -8,6 +8,10 @@ use std::collections::HashMap; use crate::errors::DeltaTableError; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod arrow_convert; +pub mod partitions; + /// Type alias for a string expected to match a GUID/UUID format pub type Guid = String; diff --git a/rust/src/partitions.rs b/rust/src/schema/partitions.rs similarity index 90% rename from rust/src/partitions.rs rename to rust/src/schema/partitions.rs index ed4a5a2eaf..0c1b0f6404 100644 --- a/rust/src/partitions.rs +++ b/rust/src/schema/partitions.rs @@ -2,7 +2,7 @@ use std::convert::TryFrom; -use super::schema::SchemaDataType; +use super::SchemaDataType; use crate::errors::DeltaTableError; use std::cmp::Ordering; use std::collections::HashMap; @@ -189,20 +189,18 @@ pub struct DeltaTablePartition<'a> { pub value: &'a str, } -/** - * Create a DeltaTable partition from a HivePartition string. - * - * A HivePartition string is represented by a "key=value" format. - * - * ```rust - * use deltalake::DeltaTablePartition; - * - * let hive_part = "ds=2023-01-01"; - * let partition = DeltaTablePartition::try_from(hive_part).unwrap(); - * assert_eq!("ds", partition.key); - * assert_eq!("2023-01-01", partition.value); - * ``` - */ +/// Create a DeltaTable partition from a HivePartition string. +/// +/// A HivePartition string is represented by a "key=value" format. +/// +/// ```rust +/// use deltalake::DeltaTablePartition; +/// +/// let hive_part = "ds=2023-01-01"; +/// let partition = DeltaTablePartition::try_from(hive_part).unwrap(); +/// assert_eq!("ds", partition.key); +/// assert_eq!("2023-01-01", partition.value); +/// ``` impl<'a> TryFrom<&'a str> for DeltaTablePartition<'a> { type Error = DeltaTableError; @@ -223,20 +221,18 @@ impl<'a> TryFrom<&'a str> for DeltaTablePartition<'a> { } impl<'a> DeltaTablePartition<'a> { - /** - * Try to create a DeltaTable partition from a partition value kv pair. - * - * ```rust - * use deltalake::DeltaTablePartition; - * - * let value = (&"ds".to_string(), &Some("2023-01-01".to_string())); - * let null_default = "1979-01-01"; - * let partition = DeltaTablePartition::from_partition_value(value, null_default); - * - * assert_eq!("ds", partition.key); - * assert_eq!("2023-01-01", partition.value); - * ``` - */ + /// Try to create a DeltaTable partition from a partition value kv pair. + /// + /// ```rust + /// use deltalake::DeltaTablePartition; + /// + /// let value = (&"ds".to_string(), &Some("2023-01-01".to_string())); + /// let null_default = "1979-01-01"; + /// let partition = DeltaTablePartition::from_partition_value(value, null_default); + /// + /// assert_eq!("ds", partition.key); + /// assert_eq!("2023-01-01", partition.value); + /// ``` pub fn from_partition_value( partition_value: (&'a String, &'a Option), default_for_null: &'a str, From 891754ad175fd1cc5633d15159b271e764695a02 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 08:09:33 +0200 Subject: [PATCH 11/19] refactor: move table config into table module --- rust/src/action/mod.rs | 2 +- rust/src/lib.rs | 3 +-- rust/src/operations/create.rs | 4 ++-- rust/src/operations/transaction/conflict_checker.rs | 2 +- rust/src/{delta_config.rs => table/config.rs} | 0 rust/src/table/mod.rs | 1 + rust/src/table/state.rs | 2 +- rust/tests/checkpoint_writer.rs | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) rename rust/src/{delta_config.rs => table/config.rs} (100%) diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index ebb9e49b06..056b85dc4d 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -23,7 +23,7 @@ use std::borrow::Borrow; use std::collections::HashMap; use std::hash::{Hash, Hasher}; -use crate::delta_config::IsolationLevel; +use crate::config::IsolationLevel; use crate::errors::DeltaResult; use crate::storage::ObjectStoreRef; use crate::{schema::*, table::CheckPoint, DeltaTableMetaData}; diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 04cf198238..c9acd3f537 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -84,7 +84,6 @@ compile_error!( pub mod action; pub mod data_catalog; -pub mod delta_config; pub mod errors; pub mod operations; pub mod schema; @@ -97,8 +96,8 @@ pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; +pub use self::config::*; pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; -pub use self::delta_config::*; pub use self::schema::partitions::*; pub use self::schema::*; pub use self::table::*; diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 697ab3ef1d..b290126618 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -11,7 +11,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::config::DeltaConfigKey; use crate::errors::{DeltaResult, DeltaTableError}; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; use crate::storage::DeltaObjectStore; @@ -322,7 +322,7 @@ impl std::future::IntoFuture for CreateBuilder { #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; - use crate::delta_config::DeltaConfigKey; + use crate::config::DeltaConfigKey; use crate::operations::DeltaOps; use crate::writer::test_utils::get_delta_schema; use tempdir::TempDir; diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 4ad044e19e..5473f53f46 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -6,7 +6,7 @@ use object_store::ObjectStore; use super::CommitInfo; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; -use crate::delta_config::IsolationLevel; +use crate::config::IsolationLevel; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::commit_uri_from_version; use crate::table::state::DeltaTableState; diff --git a/rust/src/delta_config.rs b/rust/src/table/config.rs similarity index 100% rename from rust/src/delta_config.rs rename to rust/src/table/config.rs diff --git a/rust/src/table/mod.rs b/rust/src/table/mod.rs index a8724d4102..abe2c1f22d 100644 --- a/rust/src/table/mod.rs +++ b/rust/src/table/mod.rs @@ -28,6 +28,7 @@ use crate::schema::*; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; pub(crate) mod builder; +pub mod config; pub mod state; #[cfg(all(feature = "arrow"))] pub mod state_arrow; diff --git a/rust/src/table/state.rs b/rust/src/table/state.rs index a84cf36b32..80710348a1 100644 --- a/rust/src/table/state.rs +++ b/rust/src/table/state.rs @@ -10,8 +10,8 @@ use lazy_static::lazy_static; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use serde::{Deserialize, Serialize}; +use super::config::TableConfig; use crate::action::{self, Action, Add, ProtocolError}; -use crate::delta_config::TableConfig; use crate::errors::DeltaTableError; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index 5852e9c6f9..5717255a35 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -90,7 +90,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::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; @@ -212,7 +212,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::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; use parquet::file::reader::{FileReader, SerializedFileReader}; From 08a6a671596738ecd34f0c0feb638f72e61675c6 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 08:11:45 +0200 Subject: [PATCH 12/19] fix: inclerease commit attempts --- 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 e3cbc5c70e..ad74c5b3fe 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -165,7 +165,7 @@ pub async fn commit( let tmp_commit = prepare_commit(storage, &operation, actions, app_metadata).await?; // TODO make max attempts configurable - let max_attempts = 10; + let max_attempts = 25; let mut attempt_number = 1; while attempt_number <= max_attempts { From 00af840e63c23ed93305e2ccdeec1af19d1b06b1 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 12:39:18 +0200 Subject: [PATCH 13/19] refactor: be more deliberate about exports --- rust/benches/read_checkpoint.rs | 2 +- rust/src/action/checkpoints.rs | 3 +- rust/src/action/mod.rs | 5 +- rust/src/data_catalog/storage/mod.rs | 3 +- rust/src/delta_datafusion.rs | 2 +- rust/src/lib.rs | 63 ++++++++++++++++++- rust/src/operations/create.rs | 9 +-- rust/src/operations/mod.rs | 2 +- .../transaction/conflict_checker.rs | 2 +- rust/src/operations/transaction/test_utils.rs | 8 +-- rust/src/storage/config.rs | 2 +- rust/src/storage/s3.rs | 2 +- rust/src/storage/utils.rs | 2 +- rust/src/table/builder.rs | 1 + rust/src/table/config.rs | 2 +- rust/src/table/mod.rs | 59 +---------------- rust/src/table/state.rs | 3 +- rust/src/test_utils.rs | 2 +- rust/src/writer/json.rs | 5 +- rust/src/writer/record_batch.rs | 5 +- rust/src/writer/stats.rs | 2 +- rust/src/writer/test_utils.rs | 3 +- rust/tests/checkpoint_writer.rs | 4 +- rust/tests/common/mod.rs | 6 +- rust/tests/fs_common/mod.rs | 6 +- rust/tests/read_delta_test.rs | 2 +- 26 files changed, 107 insertions(+), 98 deletions(-) diff --git a/rust/benches/read_checkpoint.rs b/rust/benches/read_checkpoint.rs index b1019cf17a..f751aabfe5 100644 --- a/rust/benches/read_checkpoint.rs +++ b/rust/benches/read_checkpoint.rs @@ -1,5 +1,5 @@ use criterion::{criterion_group, criterion_main, Criterion}; -use deltalake::state::DeltaTableState; +use deltalake::table::state::DeltaTableState; use deltalake::DeltaTableConfig; use std::fs::File; use std::io::Read; diff --git a/rust/src/action/checkpoints.rs b/rust/src/action/checkpoints.rs index 7f62e63627..c374e2ce7c 100644 --- a/rust/src/action/checkpoints.rs +++ b/rust/src/action/checkpoints.rs @@ -22,7 +22,8 @@ use crate::arrow_convert::delta_log_schema_for_table; use crate::schema::*; use crate::storage::DeltaObjectStore; use crate::table::state::DeltaTableState; -use crate::{open_table_with_version, time_utils, CheckPoint, DeltaTable}; +use crate::table::CheckPoint; +use crate::{open_table_with_version, time_utils, DeltaTable}; type SchemaPath = Vec; diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 056b85dc4d..42ecdac40a 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -23,10 +23,11 @@ use std::borrow::Borrow; use std::collections::HashMap; use std::hash::{Hash, Hasher}; -use crate::config::IsolationLevel; use crate::errors::DeltaResult; use crate::storage::ObjectStoreRef; -use crate::{schema::*, table::CheckPoint, DeltaTableMetaData}; +use crate::table::config::IsolationLevel; +use crate::table::DeltaTableMetaData; +use crate::{schema::*, table::CheckPoint}; /// Error returned when an invalid Delta log action is encountered. #[allow(missing_docs)] diff --git a/rust/src/data_catalog/storage/mod.rs b/rust/src/data_catalog/storage/mod.rs index 37083411a5..7dceb38f92 100644 --- a/rust/src/data_catalog/storage/mod.rs +++ b/rust/src/data_catalog/storage/mod.rs @@ -13,8 +13,9 @@ use futures::TryStreamExt; use object_store::ObjectStore; use crate::errors::DeltaResult; +use crate::open_table_with_storage_options; use crate::storage::config::{configure_store, StorageOptions}; -use crate::{ensure_table_uri, open_table_with_storage_options}; +use crate::table::builder::ensure_table_uri; const DELTA_LOG_FOLDER: &str = "_delta_log"; diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index d7d5b38cd1..3b5df7de90 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -58,9 +58,9 @@ use object_store::ObjectMeta; use url::Url; use crate::action::{self, Add}; -use crate::builder::ensure_table_uri; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::ObjectStoreRef; +use crate::table::builder::ensure_table_uri; use crate::table::state::DeltaTableState; use crate::{open_table, open_table_with_storage_options, DeltaTable, Invariant, SchemaDataType}; diff --git a/rust/src/lib.rs b/rust/src/lib.rs index c9acd3f537..4cd8cc8e14 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -96,12 +96,17 @@ pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; -pub use self::config::*; +use std::collections::HashMap; + pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; +pub use self::errors::*; pub use self::schema::partitions::*; pub use self::schema::*; -pub use self::table::*; -pub use errors::*; +pub use self::table::builder::{ + DeltaTableBuilder, DeltaTableConfig, DeltaTableLoadOptions, DeltaVersion, +}; +pub use self::table::config::DeltaConfigKey; +pub use self::table::DeltaTable; pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; pub use operations::DeltaOps; @@ -121,3 +126,55 @@ pub use parquet2; // TODO can / should we move this into the test crate? #[cfg(feature = "integration_test")] pub mod test_utils; + +/// Creates and loads a DeltaTable from the given path with current metadata. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table(table_uri: impl AsRef) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri).load().await?; + Ok(table) +} + +/// Same as `open_table`, but also accepts storage options to aid in building the table for a deduced +/// `StorageService`. +pub async fn open_table_with_storage_options( + table_uri: impl AsRef, + storage_options: HashMap, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_storage_options(storage_options) + .load() + .await?; + Ok(table) +} + +/// Creates a DeltaTable from the given path and loads it with the metadata from the given version. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table_with_version( + table_uri: impl AsRef, + version: i64, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_version(version) + .load() + .await?; + Ok(table) +} + +/// Creates a DeltaTable from the given path. +/// Loads metadata from the version appropriate based on the given ISO-8601/RFC-3339 timestamp. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table_with_ds( + table_uri: impl AsRef, + ds: impl AsRef, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_datestring(ds)? + .load() + .await?; + Ok(table) +} + +/// Returns rust crate version, can be use used in language bindings to expose Rust core version +pub fn crate_version() -> &'static str { + env!("CARGO_PKG_VERSION") +} diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index b290126618..5826221a82 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -10,12 +10,13 @@ use serde_json::{Map, Value}; 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::config::DeltaConfigKey; use crate::errors::{DeltaResult, DeltaTableError}; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; use crate::storage::DeltaObjectStore; -use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData}; +use crate::table::builder::ensure_table_uri; +use crate::table::config::DeltaConfigKey; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder}; #[derive(thiserror::Error, Debug)] enum CreateError { @@ -322,8 +323,8 @@ impl std::future::IntoFuture for CreateBuilder { #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; - use crate::config::DeltaConfigKey; use crate::operations::DeltaOps; + use crate::table::config::DeltaConfigKey; use crate::writer::test_utils::get_delta_schema; use tempdir::TempDir; diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 786f455344..d82b0f181a 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -10,8 +10,8 @@ use self::create::CreateBuilder; use self::filesystem_check::FileSystemCheckBuilder; use self::vacuum::VacuumBuilder; -use crate::builder::DeltaTableBuilder; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::table::builder::DeltaTableBuilder; use crate::DeltaTable; pub mod create; diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 5473f53f46..0a90e8f54a 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -6,9 +6,9 @@ use object_store::ObjectStore; use super::CommitInfo; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; -use crate::config::IsolationLevel; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::commit_uri_from_version; +use crate::table::config::IsolationLevel; use crate::table::state::DeltaTableState; #[cfg(feature = "datafusion")] diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 86e1db6524..f955e299fe 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -1,11 +1,11 @@ #![allow(unused)] +use std::collections::HashMap; + use super::{prepare_commit, try_commit_transaction, CommitInfo}; use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; use crate::table::state::DeltaTableState; -use crate::{ - DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, -}; -use std::collections::HashMap; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; pub fn create_add_action( path: impl Into, diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index 873f29dd1b..bc166ba125 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -260,7 +260,7 @@ fn url_prefix_handler( #[cfg(test)] mod test { - use crate::ensure_table_uri; + use crate::table::builder::ensure_table_uri; use super::*; diff --git a/rust/src/storage/s3.rs b/rust/src/storage/s3.rs index cc8c0714d5..1c5404d4f0 100644 --- a/rust/src/storage/s3.rs +++ b/rust/src/storage/s3.rs @@ -1,7 +1,7 @@ //! AWS S3 storage backend. use super::utils::str_is_truthy; -use crate::builder::{s3_storage_options, str_option}; +use crate::table::builder::{s3_storage_options, str_option}; use bytes::Bytes; use dynamodb_lock::{DynamoError, LockClient, LockItem, DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS}; use futures::stream::BoxStream; diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index 30eabb4e70..40f87be8b4 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -9,8 +9,8 @@ use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; use crate::action::Add; -use crate::builder::DeltaTableBuilder; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::table::builder::DeltaTableBuilder; /// Copies the contents from the `from` location into the `to` location pub async fn copy_table( diff --git a/rust/src/table/builder.rs b/rust/src/table/builder.rs index 29f8eeaccb..ffc63efa51 100644 --- a/rust/src/table/builder.rs +++ b/rust/src/table/builder.rs @@ -311,6 +311,7 @@ pub mod s3_storage_options { /// See also . pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; /// Allow http connections - mainly useful for integration tests + #[allow(dead_code)] pub const AWS_STORAGE_ALLOW_HTTP: &str = "AWS_STORAGE_ALLOW_HTTP"; /// If set to "true", allows creating commits without concurrent writer protection. diff --git a/rust/src/table/config.rs b/rust/src/table/config.rs index 1b7f6b7e0f..60498767ab 100644 --- a/rust/src/table/config.rs +++ b/rust/src/table/config.rs @@ -387,7 +387,7 @@ fn parse_int(value: &str) -> Result { #[cfg(test)] mod tests { use super::*; - use crate::DeltaTableMetaData; + use crate::table::DeltaTableMetaData; use crate::Schema; use std::collections::HashMap; diff --git a/rust/src/table/mod.rs b/rust/src/table/mod.rs index abe2c1f22d..2f8ef45ee4 100644 --- a/rust/src/table/mod.rs +++ b/rust/src/table/mod.rs @@ -19,6 +19,7 @@ use serde::ser::SerializeSeq; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use uuid::Uuid; +use self::builder::DeltaTableConfig; use self::state::DeltaTableState; use crate::action::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; use crate::action::{Add, ProtocolError, Stats}; @@ -33,8 +34,6 @@ pub mod state; #[cfg(all(feature = "arrow"))] pub mod state_arrow; -pub use builder::*; - /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] pub struct CheckPoint { @@ -791,64 +790,12 @@ impl std::fmt::Debug for DeltaTable { } } -/// Creates and loads a DeltaTable from the given path with current metadata. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table(table_uri: impl AsRef) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri).load().await?; - Ok(table) -} - -/// Same as `open_table`, but also accepts storage options to aid in building the table for a deduced -/// `StorageService`. -pub async fn open_table_with_storage_options( - table_uri: impl AsRef, - storage_options: HashMap, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_storage_options(storage_options) - .load() - .await?; - Ok(table) -} - -/// Creates a DeltaTable from the given path and loads it with the metadata from the given version. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table_with_version( - table_uri: impl AsRef, - version: i64, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_version(version) - .load() - .await?; - Ok(table) -} - -/// Creates a DeltaTable from the given path. -/// Loads metadata from the version appropriate based on the given ISO-8601/RFC-3339 timestamp. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table_with_ds( - table_uri: impl AsRef, - ds: impl AsRef, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_datestring(ds)? - .load() - .await?; - Ok(table) -} - -/// Returns rust crate version, can be use used in language bindings to expose Rust core version -pub fn crate_version() -> &'static str { - env!("CARGO_PKG_VERSION") -} - #[cfg(test)] mod tests { use super::*; - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - use crate::builder::DeltaTableBuilder; use crate::operations::create::CreateBuilder; + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + use crate::table::builder::DeltaTableBuilder; use pretty_assertions::assert_eq; use std::collections::HashMap; use tempdir::TempDir; diff --git a/rust/src/table/state.rs b/rust/src/table/state.rs index 80710348a1..7df045771f 100644 --- a/rust/src/table/state.rs +++ b/rust/src/table/state.rs @@ -16,8 +16,9 @@ use crate::errors::DeltaTableError; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::schema::SchemaDataType; use crate::storage::commit_uri_from_version; +use crate::table::DeltaTableMetaData; +use crate::DeltaTable; use crate::Schema; -use crate::{DeltaTable, DeltaTableMetaData}; #[cfg(any(feature = "parquet", feature = "parquet2"))] use super::{CheckPoint, DeltaTableConfig}; diff --git a/rust/src/test_utils.rs b/rust/src/test_utils.rs index a11bff8c17..93accab8e6 100644 --- a/rust/src/test_utils.rs +++ b/rust/src/test_utils.rs @@ -334,7 +334,7 @@ pub mod az_cli { /// small wrapper around s3 cli pub mod s3_cli { use super::set_env_if_not_set; - use crate::builder::s3_storage_options; + use crate::table::builder::s3_storage_options; use std::process::{Command, ExitStatus, Stdio}; /// Create a new bucket diff --git a/rust/src/writer/json.rs b/rust/src/writer/json.rs index b8b4e48713..85fdf6ebff 100644 --- a/rust/src/writer/json.rs +++ b/rust/src/writer/json.rs @@ -20,9 +20,10 @@ use super::utils::{ record_batch_without_partitions, stringified_partition_value, }; use super::{DeltaWriter, DeltaWriterError}; -use crate::builder::DeltaTableBuilder; use crate::errors::DeltaTableError; -use crate::{action::Add, DeltaTable, DeltaTableMetaData, Schema}; +use crate::table::builder::DeltaTableBuilder; +use crate::table::DeltaTableMetaData; +use crate::{action::Add, DeltaTable, Schema}; use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; type BadValue = (Value, ParquetError); diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index a254b91f78..6f56d9210d 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -22,9 +22,10 @@ use super::utils::{ stringified_partition_value, PartitionPath, ShareableBuffer, }; use super::{DeltaWriter, DeltaWriterError}; -use crate::builder::DeltaTableBuilder; use crate::errors::DeltaTableError; -use crate::{action::Add, storage::DeltaObjectStore, DeltaTable, DeltaTableMetaData, Schema}; +use crate::table::builder::DeltaTableBuilder; +use crate::table::DeltaTableMetaData; +use crate::{action::Add, storage::DeltaObjectStore, DeltaTable, Schema}; /// Writes messages to a delta lake table. pub struct RecordBatchWriter { diff --git a/rust/src/writer/stats.rs b/rust/src/writer/stats.rs index 2e0878c6d6..5d3be1c4c7 100644 --- a/rust/src/writer/stats.rs +++ b/rust/src/writer/stats.rs @@ -456,8 +456,8 @@ mod tests { use super::*; use crate::{ action::{ColumnCountStat, ColumnValueStat}, - builder::DeltaTableBuilder, errors::DeltaTableError, + table::builder::DeltaTableBuilder, DeltaTable, }; use lazy_static::lazy_static; diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index 3eb974a207..d12587a55d 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -9,7 +9,8 @@ use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use crate::operations::create::CreateBuilder; use crate::schema::Schema; -use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData, SchemaDataType, SchemaField}; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder, SchemaDataType, SchemaField}; pub type TestResult = Result<(), Box>; diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index 5717255a35..b169b7af02 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -90,7 +90,7 @@ mod delete_expired_delta_log_in_checkpoint { use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; - use deltalake::config::DeltaConfigKey; + use deltalake::table::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; @@ -212,7 +212,7 @@ mod checkpoints_with_tombstones { use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; use deltalake::action::*; - use deltalake::config::DeltaConfigKey; + use deltalake::table::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; use parquet::file::reader::{FileReader, SerializedFileReader}; diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index fcce3c4b37..2966b1a911 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -83,16 +83,14 @@ impl TestContext { .map(|s| s.to_string()) .collect::>(); let backend = self.new_storage(); - let dt = CreateBuilder::new() + CreateBuilder::new() .with_object_store(backend) .with_table_name("delta-rs_test_table") .with_comment("Table created by delta-rs tests") .with_columns(schema.get_fields().clone()) .with_partition_columns(p) .await - .unwrap(); - - dt + .unwrap() } } diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index 7bca9386e8..5a4b5b2978 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -43,7 +43,7 @@ pub async fn create_test_table( partition_columns: Vec<&str>, config: HashMap>, ) -> DeltaTable { - let table = CreateBuilder::new() + CreateBuilder::new() .with_location(path) .with_table_name("test-table") .with_comment("A table for running tests") @@ -51,9 +51,7 @@ pub async fn create_test_table( .with_partition_columns(partition_columns) .with_configuration(config) .await - .unwrap(); - - table + .unwrap() } pub async fn create_table( diff --git a/rust/tests/read_delta_test.rs b/rust/tests/read_delta_test.rs index dc70d1fd2d..db6c51ad9f 100644 --- a/rust/tests/read_delta_test.rs +++ b/rust/tests/read_delta_test.rs @@ -1,6 +1,6 @@ use chrono::Utc; +use deltalake::table::PeekCommit; use deltalake::DeltaTableBuilder; -use deltalake::PeekCommit; use object_store::path::Path; use pretty_assertions::assert_eq; use std::collections::HashMap; From 56bbd21b535576dde9281fcfca11fc0fbbe2c2eb Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 12:43:40 +0200 Subject: [PATCH 14/19] refactor: move time_utils into action module --- rust/src/action/checkpoints.rs | 4 ++-- rust/src/action/mod.rs | 1 + rust/src/{ => action}/time_utils.rs | 0 rust/src/lib.rs | 1 - 4 files changed, 3 insertions(+), 3 deletions(-) rename rust/src/{ => action}/time_utils.rs (100%) diff --git a/rust/src/action/checkpoints.rs b/rust/src/action/checkpoints.rs index c374e2ce7c..2db59f29dc 100644 --- a/rust/src/action/checkpoints.rs +++ b/rust/src/action/checkpoints.rs @@ -17,13 +17,13 @@ use std::convert::TryFrom; use std::iter::Iterator; use std::ops::Add; -use super::{Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; +use super::{time_utils, Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; use crate::arrow_convert::delta_log_schema_for_table; use crate::schema::*; use crate::storage::DeltaObjectStore; use crate::table::state::DeltaTableState; use crate::table::CheckPoint; -use crate::{open_table_with_version, time_utils, DeltaTable}; +use crate::{open_table_with_version, DeltaTable}; type SchemaPath = Vec; diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 42ecdac40a..ed9b2dd916 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -8,6 +8,7 @@ pub mod checkpoints; pub mod parquet2_read; #[cfg(feature = "parquet")] mod parquet_read; +mod time_utils; #[cfg(all(feature = "arrow"))] use arrow_schema::ArrowError; diff --git a/rust/src/time_utils.rs b/rust/src/action/time_utils.rs similarity index 100% rename from rust/src/time_utils.rs rename to rust/src/action/time_utils.rs diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 4cd8cc8e14..d5217ee87b 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -89,7 +89,6 @@ pub mod operations; pub mod schema; pub mod storage; pub mod table; -pub mod time_utils; #[cfg(feature = "datafusion")] pub mod delta_datafusion; From be7f9615828fa0828ec9033d2c906dc66e023032 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 12:47:11 +0200 Subject: [PATCH 15/19] fix: docs links --- rust/src/operations/create.rs | 2 +- rust/src/storage/mod.rs | 2 +- rust/src/table/mod.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 5826221a82..396cdad842 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -149,7 +149,7 @@ impl CreateBuilder { /// /// Options may be passed in the HashMap or set as environment variables. /// - /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. + /// [crate::table::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. /// /// If an object store is also passed using `with_object_store()` these options will be ignored. diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index 20ce1bc97d..0fef80cd4c 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -88,7 +88,7 @@ impl DeltaObjectStore { /// # Arguments /// /// * `location` - A url pointing to the root of the delta table. - /// * `options` - Options passed to underlying builders. See [`with_storage_options`](crate::builder::DeltaTableBuilder::with_storage_options) + /// * `options` - Options passed to underlying builders. See [`with_storage_options`](crate::table::builder::DeltaTableBuilder::with_storage_options) pub fn try_new(location: Url, options: impl Into + Clone) -> DeltaResult { let options = options.into(); let storage = config::configure_store(&location, &options)?; diff --git a/rust/src/table/mod.rs b/rust/src/table/mod.rs index 2f8ef45ee4..fa4d058085 100644 --- a/rust/src/table/mod.rs +++ b/rust/src/table/mod.rs @@ -28,7 +28,7 @@ use crate::partitions::PartitionFilter; use crate::schema::*; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; -pub(crate) mod builder; +pub mod builder; pub mod config; pub mod state; #[cfg(all(feature = "arrow"))] From c7853589c5798fd07c6c8fe1cbc54ca30ccb1745 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 12:54:08 +0200 Subject: [PATCH 16/19] reafctor: rename actions to protocol --- python/src/error.rs | 2 +- python/src/lib.rs | 16 +++---- rust/examples/basic_operations.rs | 2 +- rust/src/delta_datafusion.rs | 8 ++-- rust/src/errors.rs | 2 +- rust/src/lib.rs | 6 +-- rust/src/operations/create.rs | 2 +- rust/src/operations/delete.rs | 6 +-- rust/src/operations/filesystem_check.rs | 2 +- rust/src/operations/optimize.rs | 4 +- .../transaction/conflict_checker.rs | 4 +- rust/src/operations/transaction/mod.rs | 2 +- rust/src/operations/transaction/state.rs | 2 +- rust/src/operations/transaction/test_utils.rs | 2 +- rust/src/operations/write.rs | 2 +- rust/src/operations/writer.rs | 2 +- rust/src/{action => protocol}/checkpoints.rs | 0 rust/src/{action => protocol}/mod.rs | 0 .../parquet2_read/boolean.rs | 0 .../parquet2_read/dictionary/binary.rs | 0 .../parquet2_read/dictionary/mod.rs | 0 .../parquet2_read/dictionary/primitive.rs | 0 .../{action => protocol}/parquet2_read/map.rs | 0 .../{action => protocol}/parquet2_read/mod.rs | 0 .../parquet2_read/primitive.rs | 0 .../parquet2_read/stats.rs | 0 .../parquet2_read/string.rs | 0 .../parquet2_read/validity.rs | 0 .../{action => protocol}/parquet_read/mod.rs | 2 +- rust/src/{action => protocol}/time_utils.rs | 0 rust/src/storage/utils.rs | 4 +- rust/src/table/mod.rs | 16 +++---- rust/src/table/state.rs | 46 +++++++++---------- rust/src/table/state_arrow.rs | 2 +- rust/src/writer/json.rs | 2 +- rust/src/writer/mod.rs | 4 +- rust/src/writer/record_batch.rs | 2 +- rust/src/writer/stats.rs | 4 +- rust/tests/checkpoint_writer.rs | 4 +- rust/tests/command_optimize.rs | 2 +- rust/tests/commit_info_format.rs | 2 +- rust/tests/common/mod.rs | 6 +-- rust/tests/datafusion_test.rs | 2 +- rust/tests/fs_common/mod.rs | 2 +- rust/tests/integration_concurrent_writes.rs | 2 +- rust/tests/integration_read.rs | 6 +-- rust/tests/read_delta_partitions_test.rs | 2 +- rust/tests/read_delta_test.rs | 8 ++-- 48 files changed, 91 insertions(+), 91 deletions(-) rename rust/src/{action => protocol}/checkpoints.rs (100%) rename rust/src/{action => protocol}/mod.rs (100%) rename rust/src/{action => protocol}/parquet2_read/boolean.rs (100%) rename rust/src/{action => protocol}/parquet2_read/dictionary/binary.rs (100%) rename rust/src/{action => protocol}/parquet2_read/dictionary/mod.rs (100%) rename rust/src/{action => protocol}/parquet2_read/dictionary/primitive.rs (100%) rename rust/src/{action => protocol}/parquet2_read/map.rs (100%) rename rust/src/{action => protocol}/parquet2_read/mod.rs (100%) rename rust/src/{action => protocol}/parquet2_read/primitive.rs (100%) rename rust/src/{action => protocol}/parquet2_read/stats.rs (100%) rename rust/src/{action => protocol}/parquet2_read/string.rs (100%) rename rust/src/{action => protocol}/parquet2_read/validity.rs (100%) rename rust/src/{action => protocol}/parquet_read/mod.rs (99%) rename rust/src/{action => protocol}/time_utils.rs (100%) diff --git a/python/src/error.rs b/python/src/error.rs index 5148dbced9..aee74c4cee 100644 --- a/python/src/error.rs +++ b/python/src/error.rs @@ -1,5 +1,5 @@ use arrow_schema::ArrowError; -use deltalake::action::ProtocolError; +use deltalake::protocol::ProtocolError; use deltalake::{errors::DeltaTableError, ObjectStoreError}; use pyo3::exceptions::{ PyException, PyFileNotFoundError, PyIOError, PyNotImplementedError, PyValueError, diff --git a/python/src/lib.rs b/python/src/lib.rs index 8f4d9c17e3..496ae4c259 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -13,9 +13,6 @@ use std::time::{SystemTime, UNIX_EPOCH}; use arrow::pyarrow::PyArrowType; use chrono::{DateTime, Duration, FixedOffset, Utc}; -use deltalake::action::{ - self, Action, ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats, -}; use deltalake::arrow::compute::concat_batches; use deltalake::arrow::record_batch::RecordBatch; use deltalake::arrow::{self, datatypes::Schema as ArrowSchema}; @@ -27,6 +24,9 @@ use deltalake::operations::optimize::OptimizeBuilder; use deltalake::operations::transaction::commit; use deltalake::operations::vacuum::VacuumBuilder; use deltalake::partitions::PartitionFilter; +use deltalake::protocol::{ + self, Action, ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats, +}; use deltalake::DeltaTableBuilder; use deltalake::{DeltaOps, Invariant, Schema}; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; @@ -422,7 +422,7 @@ impl RawDeltaTable { let existing_schema = self._table.get_schema().map_err(PythonError::from)?; - let mut actions: Vec = add_actions + let mut actions: Vec = add_actions .iter() .map(|add| Action::add(add.into())) .collect(); @@ -440,7 +440,7 @@ impl RawDeltaTable { .map_err(PythonError::from)?; for old_add in add_actions { - let remove_action = Action::remove(action::Remove { + let remove_action = Action::remove(protocol::Remove { path: old_add.path.clone(), deletion_timestamp: Some(current_timestamp()), data_change: true, @@ -460,7 +460,7 @@ impl RawDeltaTable { .map_err(PythonError::from)? .clone(); metadata.schema = schema; - let metadata_action = action::MetaData::try_from(metadata) + let metadata_action = protocol::MetaData::try_from(metadata) .map_err(|_| PyValueError::new_err("Failed to reparse metadata"))?; actions.push(Action::metaData(metadata_action)); } @@ -686,9 +686,9 @@ pub struct PyAddAction { stats: Option, } -impl From<&PyAddAction> for action::Add { +impl From<&PyAddAction> for protocol::Add { fn from(action: &PyAddAction) -> Self { - action::Add { + protocol::Add { path: action.path.clone(), size: action.size, partition_values: action.partition_values.clone(), diff --git a/rust/examples/basic_operations.rs b/rust/examples/basic_operations.rs index 8aa91a6cb3..c168f6d208 100644 --- a/rust/examples/basic_operations.rs +++ b/rust/examples/basic_operations.rs @@ -4,7 +4,7 @@ use arrow::{ record_batch::RecordBatch, }; use deltalake::operations::collect_sendable_stream; -use deltalake::{action::SaveMode, DeltaOps, SchemaDataType, SchemaField}; +use deltalake::{protocol::SaveMode, DeltaOps, SchemaDataType, SchemaField}; use std::sync::Arc; fn get_table_columns() -> Vec { diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 3b5df7de90..4758cd0505 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -57,8 +57,8 @@ use datafusion_proto::physical_plan::PhysicalExtensionCodec; use object_store::ObjectMeta; use url::Url; -use crate::action::{self, Add}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{self, Add}; use crate::storage::ObjectStoreRef; use crate::table::builder::ensure_table_uri; use crate::table::state::DeltaTableState; @@ -113,7 +113,7 @@ impl DeltaTableState { let acc = acc?; let new_stats = action .get_stats() - .unwrap_or_else(|_| Some(action::Stats::default()))?; + .unwrap_or_else(|_| Some(protocol::Stats::default()))?; Some(Statistics { num_rows: acc .num_rows @@ -614,7 +614,7 @@ pub(crate) fn get_null_of_arrow_type(t: &ArrowDataType) -> DeltaResult PartitionedFile { let partition_values = schema @@ -1152,7 +1152,7 @@ mod tests { let mut partition_values = std::collections::HashMap::new(); partition_values.insert("month".to_string(), Some("1".to_string())); partition_values.insert("year".to_string(), Some("2015".to_string())); - let action = action::Add { + let action = protocol::Add { path: "year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string(), size: 10644, partition_values, diff --git a/rust/src/errors.rs b/rust/src/errors.rs index 7b982b3cc3..f8d9993f0d 100644 --- a/rust/src/errors.rs +++ b/rust/src/errors.rs @@ -1,8 +1,8 @@ //! Exceptions for the deltalake crate use object_store::Error as ObjectStoreError; -use crate::action::ProtocolError; use crate::operations::transaction::TransactionError; +use crate::protocol::ProtocolError; /// A result returned by delta-rs pub type DeltaResult = Result; diff --git a/rust/src/lib.rs b/rust/src/lib.rs index d5217ee87b..67de4118ba 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -82,10 +82,10 @@ compile_error!( "Features s3 and s3-native-tls are mutually exclusive and cannot be enabled together" ); -pub mod action; pub mod data_catalog; pub mod errors; pub mod operations; +pub mod protocol; pub mod schema; pub mod storage; pub mod table; @@ -110,8 +110,6 @@ pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, Object pub use operations::DeltaOps; // convenience exports for consumers to avoid aligning crate versions -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub use action::checkpoints; #[cfg(feature = "arrow")] pub use arrow; #[cfg(feature = "datafusion")] @@ -120,6 +118,8 @@ pub use datafusion; pub use parquet; #[cfg(feature = "parquet2")] pub use parquet2; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub use protocol::checkpoints; // needed only for integration tests // TODO can / should we move this into the test crate? diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 396cdad842..694d95a74e 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -9,8 +9,8 @@ use serde_json::{Map, Value}; use super::transaction::commit; use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; -use crate::action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; use crate::storage::DeltaObjectStore; use crate::table::builder::ensure_table_uri; diff --git a/rust/src/operations/delete.rs b/rust/src/operations/delete.rs index 5f139e8b23..d85ee9625f 100644 --- a/rust/src/operations/delete.rs +++ b/rust/src/operations/delete.rs @@ -22,7 +22,7 @@ use std::pin::Pin; use std::sync::Arc; use std::time::{Instant, SystemTime, UNIX_EPOCH}; -use crate::action::{Action, Add, Remove}; +use crate::protocol::{Action, Add, Remove}; use arrow::array::StringArray; use arrow::datatypes::DataType; use arrow::datatypes::Field; @@ -53,13 +53,13 @@ use parquet::file::properties::WriterProperties; use serde_json::Map; use serde_json::Value; -use crate::action::DeltaOperation; use crate::delta_datafusion::{ parquet_scan_from_actions, partitioned_file_from_action, register_store, }; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; use crate::operations::write::write_execution_plan; +use crate::protocol::DeltaOperation; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; use crate::table::state::DeltaTableState; use crate::DeltaTable; @@ -680,8 +680,8 @@ impl std::future::IntoFuture for DeleteBuilder { #[cfg(test)] mod tests { - use crate::action::*; use crate::operations::DeltaOps; + use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::{get_arrow_schema, get_delta_schema}; use crate::DeltaTable; diff --git a/rust/src/operations/filesystem_check.rs b/rust/src/operations/filesystem_check.rs index be2b21ee17..8b5584291a 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/rust/src/operations/filesystem_check.rs @@ -24,9 +24,9 @@ pub use object_store::path::Path; use object_store::ObjectStore; use url::{ParseError, Url}; -use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; +use crate::protocol::{Action, Add, DeltaOperation, Remove}; use crate::storage::DeltaObjectStore; use crate::table::state::DeltaTableState; use crate::DeltaTable; diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index 6793aa65a6..5c42404ec0 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -43,8 +43,8 @@ use serde_json::Map; use super::transaction::commit; use super::writer::{PartitionWriter, PartitionWriterConfig}; -use crate::action::{self, Action, DeltaOperation}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{self, Action, DeltaOperation}; use crate::storage::ObjectStoreRef; use crate::table::state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; @@ -289,7 +289,7 @@ fn create_remove( let deletion_time = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); let deletion_time = deletion_time.as_millis() as i64; - Ok(Action::remove(action::Remove { + Ok(Action::remove(protocol::Remove { path: path.to_string(), deletion_timestamp: Some(deletion_time), data_change: false, diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 0a90e8f54a..d75e401def 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -5,8 +5,8 @@ use std::io::{BufRead, BufReader, Cursor}; use object_store::ObjectStore; use super::CommitInfo; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::storage::commit_uri_from_version; use crate::table::config::IsolationLevel; use crate::table::state::DeltaTableState; @@ -641,7 +641,7 @@ mod tests { use super::super::test_utils as tu; use super::super::test_utils::init_table_actions; use super::*; - use crate::action::Action; + use crate::protocol::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 ad74c5b3fe..9a358946f6 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -5,9 +5,9 @@ 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::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, CommitInfo, DeltaOperation}; use crate::storage::commit_uri_from_version; use crate::table::state::DeltaTableState; diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index d27713e27c..6ab582c274 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -19,11 +19,11 @@ use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; use sqlparser::tokenizer::Tokenizer; -use crate::action::Add; use crate::delta_datafusion::{ get_null_of_arrow_type, logical_expr_to_physical_expr, to_correct_scalar_value, }; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; use crate::table::state::DeltaTableState; impl DeltaTableState { diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index f955e299fe..cdd98f8d1f 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -2,7 +2,7 @@ use std::collections::HashMap; use super::{prepare_commit, try_commit_transaction, CommitInfo}; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; +use crate::protocol::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; use crate::table::state::DeltaTableState; use crate::table::DeltaTableMetaData; use crate::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index aa3b90a9db..995d996bb1 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -31,9 +31,9 @@ use parquet::file::properties::WriterProperties; use super::writer::{DeltaWriter, WriterConfig}; use super::MAX_SUPPORTED_WRITER_VERSION; use super::{transaction::commit, CreateBuilder}; -use crate::action::{Action, Add, DeltaOperation, Remove, SaveMode}; use crate::delta_datafusion::DeltaDataChecker; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, Add, DeltaOperation, Remove, SaveMode}; use crate::schema::Schema; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; use crate::table::state::DeltaTableState; diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index 893fac7a0e..3beaa70d1d 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -11,9 +11,9 @@ use parquet::arrow::ArrowWriter; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; -use crate::action::Add; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; use crate::storage::ObjectStoreRef; use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; use crate::writer::stats::create_add; diff --git a/rust/src/action/checkpoints.rs b/rust/src/protocol/checkpoints.rs similarity index 100% rename from rust/src/action/checkpoints.rs rename to rust/src/protocol/checkpoints.rs diff --git a/rust/src/action/mod.rs b/rust/src/protocol/mod.rs similarity index 100% rename from rust/src/action/mod.rs rename to rust/src/protocol/mod.rs diff --git a/rust/src/action/parquet2_read/boolean.rs b/rust/src/protocol/parquet2_read/boolean.rs similarity index 100% rename from rust/src/action/parquet2_read/boolean.rs rename to rust/src/protocol/parquet2_read/boolean.rs diff --git a/rust/src/action/parquet2_read/dictionary/binary.rs b/rust/src/protocol/parquet2_read/dictionary/binary.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/binary.rs rename to rust/src/protocol/parquet2_read/dictionary/binary.rs diff --git a/rust/src/action/parquet2_read/dictionary/mod.rs b/rust/src/protocol/parquet2_read/dictionary/mod.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/mod.rs rename to rust/src/protocol/parquet2_read/dictionary/mod.rs diff --git a/rust/src/action/parquet2_read/dictionary/primitive.rs b/rust/src/protocol/parquet2_read/dictionary/primitive.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/primitive.rs rename to rust/src/protocol/parquet2_read/dictionary/primitive.rs diff --git a/rust/src/action/parquet2_read/map.rs b/rust/src/protocol/parquet2_read/map.rs similarity index 100% rename from rust/src/action/parquet2_read/map.rs rename to rust/src/protocol/parquet2_read/map.rs diff --git a/rust/src/action/parquet2_read/mod.rs b/rust/src/protocol/parquet2_read/mod.rs similarity index 100% rename from rust/src/action/parquet2_read/mod.rs rename to rust/src/protocol/parquet2_read/mod.rs diff --git a/rust/src/action/parquet2_read/primitive.rs b/rust/src/protocol/parquet2_read/primitive.rs similarity index 100% rename from rust/src/action/parquet2_read/primitive.rs rename to rust/src/protocol/parquet2_read/primitive.rs diff --git a/rust/src/action/parquet2_read/stats.rs b/rust/src/protocol/parquet2_read/stats.rs similarity index 100% rename from rust/src/action/parquet2_read/stats.rs rename to rust/src/protocol/parquet2_read/stats.rs diff --git a/rust/src/action/parquet2_read/string.rs b/rust/src/protocol/parquet2_read/string.rs similarity index 100% rename from rust/src/action/parquet2_read/string.rs rename to rust/src/protocol/parquet2_read/string.rs diff --git a/rust/src/action/parquet2_read/validity.rs b/rust/src/protocol/parquet2_read/validity.rs similarity index 100% rename from rust/src/action/parquet2_read/validity.rs rename to rust/src/protocol/parquet2_read/validity.rs diff --git a/rust/src/action/parquet_read/mod.rs b/rust/src/protocol/parquet_read/mod.rs similarity index 99% rename from rust/src/action/parquet_read/mod.rs rename to rust/src/protocol/parquet_read/mod.rs index 15737a1761..f57b36b846 100644 --- a/rust/src/action/parquet_read/mod.rs +++ b/rust/src/protocol/parquet_read/mod.rs @@ -6,7 +6,7 @@ use num_traits::cast::ToPrimitive; use parquet::record::{Field, ListAccessor, MapAccessor, RowAccessor}; use serde_json::json; -use crate::action::{ +use crate::protocol::{ Action, Add, AddCDCFile, ColumnCountStat, ColumnValueStat, MetaData, Protocol, ProtocolError, Remove, Stats, Txn, }; diff --git a/rust/src/action/time_utils.rs b/rust/src/protocol/time_utils.rs similarity index 100% rename from rust/src/action/time_utils.rs rename to rust/src/protocol/time_utils.rs diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index 40f87be8b4..6699c0af2a 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -8,8 +8,8 @@ use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; -use crate::action::Add; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; use crate::table::builder::DeltaTableBuilder; /// Copies the contents from the `from` location into the `to` location @@ -82,7 +82,7 @@ impl TryFrom<&Add> for ObjectMeta { fn try_from(value: &Add) -> DeltaResult { let last_modified = DateTime::::from_utc( NaiveDateTime::from_timestamp_millis(value.modification_time).ok_or( - DeltaTableError::from(crate::action::ProtocolError::InvalidField(format!( + DeltaTableError::from(crate::protocol::ProtocolError::InvalidField(format!( "invalid modification_time: {:?}", value.modification_time ))), diff --git a/rust/src/table/mod.rs b/rust/src/table/mod.rs index fa4d058085..ada9bdcbcc 100644 --- a/rust/src/table/mod.rs +++ b/rust/src/table/mod.rs @@ -21,10 +21,10 @@ use uuid::Uuid; use self::builder::DeltaTableConfig; use self::state::DeltaTableState; -use crate::action::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; -use crate::action::{Add, ProtocolError, Stats}; use crate::errors::DeltaTableError; use crate::partitions::PartitionFilter; +use crate::protocol::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; +use crate::protocol::{Add, ProtocolError, Stats}; use crate::schema::*; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; @@ -72,7 +72,7 @@ pub struct DeltaTableMetaData { /// User-provided description for this table pub description: Option, /// Specification of the encoding for the files stored in the table - pub format: action::Format, + pub format: protocol::Format, /// Schema of the table pub schema: Schema, /// An array containing the names of columns by which the data should be partitioned @@ -88,7 +88,7 @@ impl DeltaTableMetaData { pub fn new( name: Option, description: Option, - format: Option, + format: Option, schema: Schema, partition_columns: Vec, configuration: HashMap>, @@ -144,10 +144,10 @@ impl fmt::Display for DeltaTableMetaData { } } -impl TryFrom for DeltaTableMetaData { +impl TryFrom for DeltaTableMetaData { type Error = ProtocolError; - fn try_from(action_metadata: action::MetaData) -> Result { + fn try_from(action_metadata: protocol::MetaData) -> Result { let schema = action_metadata.get_schema()?; Ok(Self { id: action_metadata.id, @@ -552,7 +552,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?, @@ -679,7 +679,7 @@ impl DeltaTable { } /// Returns a vector of active tombstones (i.e. `Remove` actions present in the current delta log). - pub fn get_tombstones(&self) -> impl Iterator { + pub fn get_tombstones(&self) -> impl Iterator { self.state.unexpired_tombstones() } diff --git a/rust/src/table/state.rs b/rust/src/table/state.rs index 7df045771f..67a790470d 100644 --- a/rust/src/table/state.rs +++ b/rust/src/table/state.rs @@ -11,9 +11,9 @@ use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use serde::{Deserialize, Serialize}; use super::config::TableConfig; -use crate::action::{self, Action, Add, ProtocolError}; use crate::errors::DeltaTableError; use crate::partitions::{DeltaTablePartition, PartitionFilter}; +use crate::protocol::{self, Action, Add, ProtocolError}; use crate::schema::SchemaDataType; use crate::storage::commit_uri_from_version; use crate::table::DeltaTableMetaData; @@ -31,11 +31,11 @@ pub struct DeltaTableState { version: i64, // A remove action should remain in the state of the table as a tombstone until it has expired. // A tombstone expires when the creation timestamp of the delta file exceeds the expiration - tombstones: HashSet, + tombstones: HashSet, // active files for table state - files: Vec, + 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, @@ -74,7 +74,7 @@ impl DeltaTableState { let mut new_state = DeltaTableState::with_version(version); for line in reader.lines() { - let action: action::Action = serde_json::from_str(line?.as_str())?; + let action: protocol::Action = serde_json::from_str(line?.as_str())?; new_state.process_action( action, table.config.require_tombstones, @@ -108,13 +108,13 @@ impl DeltaTableState { let preader = SerializedFileReader::new(data)?; let schema = preader.metadata().file_metadata().schema(); if !schema.is_group() { - return Err(DeltaTableError::from(action::ProtocolError::Generic( + return Err(DeltaTableError::from(protocol::ProtocolError::Generic( "Action record in checkpoint should be a struct".to_string(), ))); } for record in preader.get_row_iter(None)? { self.process_action( - action::Action::from_parquet_record(schema, &record)?, + protocol::Action::from_parquet_record(schema, &record)?, table_config.require_tombstones, table_config.require_files, )?; @@ -123,7 +123,7 @@ impl DeltaTableState { #[cfg(feature = "parquet2")] { - use crate::action::parquet2_read::actions_from_row_group; + use crate::protocol::parquet2_read::actions_from_row_group; use parquet2::read::read_metadata; let mut reader = std::io::Cursor::new(data); @@ -131,10 +131,10 @@ impl DeltaTableState { for row_group in metadata.row_groups { for action in actions_from_row_group(row_group, &mut reader) - .map_err(action::ProtocolError::from)? + .map_err(protocol::ProtocolError::from)? { self.process_action( - action, + protocol, table_config.require_tombstones, table_config.require_files, )?; @@ -163,7 +163,7 @@ impl DeltaTableState { } /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec { + pub fn commit_infos(&self) -> &Vec { &self.commit_infos } @@ -183,13 +183,13 @@ impl DeltaTableState { } /// Full list of tombstones (remove actions) representing files removed from table state). - pub fn all_tombstones(&self) -> &HashSet { + pub fn all_tombstones(&self) -> &HashSet { &self.tombstones } /// List of unexpired tombstones (remove actions) representing files removed from table state. /// The retention period is set by `deletedFileRetentionDuration` with default value of 1 week. - pub fn unexpired_tombstones(&self) -> impl Iterator { + pub fn unexpired_tombstones(&self) -> impl Iterator { let retention_timestamp = Utc::now().timestamp_millis() - self.tombstone_retention_millis; self.tombstones .iter() @@ -198,7 +198,7 @@ impl DeltaTableState { /// Full list of add actions representing all parquet files that are part of the current /// delta table state. - pub fn files(&self) -> &Vec { + pub fn files(&self) -> &Vec { self.files.as_ref() } @@ -314,29 +314,29 @@ impl DeltaTableState { /// Process given action by updating current state. fn process_action( &mut self, - action: action::Action, + action: protocol::Action, require_tombstones: bool, require_files: bool, ) -> Result<(), ProtocolError> { match action { // TODO: optionally load CDC into TableState - action::Action::cdc(_v) => {} - action::Action::add(v) => { + protocol::Action::cdc(_v) => {} + protocol::Action::add(v) => { if require_files { self.files.push(v.path_decoded()?); } } - action::Action::remove(v) => { + protocol::Action::remove(v) => { if require_tombstones && require_files { let v = v.path_decoded()?; self.tombstones.insert(v); } } - action::Action::protocol(v) => { + protocol::Action::protocol(v) => { self.min_reader_version = v.min_reader_version; self.min_writer_version = v.min_writer_version; } - action::Action::metaData(v) => { + protocol::Action::metaData(v) => { let md = DeltaTableMetaData::try_from(v)?; let table_config = TableConfig(&md.configuration); self.tombstone_retention_millis = @@ -346,13 +346,13 @@ impl DeltaTableState { self.enable_expired_log_cleanup = table_config.enable_expired_log_cleanup(); self.current_metadata = Some(md); } - action::Action::txn(v) => { + protocol::Action::txn(v) => { *self .app_transaction_version .entry(v.app_id) .or_insert(v.version) = v.version; } - action::Action::commitInfo(v) => { + protocol::Action::commitInfo(v) => { self.commit_infos.push(v); } } @@ -443,7 +443,7 @@ mod tests { enable_expired_log_cleanup: true, }; - let txn_action = action::Action::txn(action::Txn { + let txn_action = protocol::Action::txn(protocol::Txn { app_id: "abc".to_string(), version: 2, last_updated: Some(0), diff --git a/rust/src/table/state_arrow.rs b/rust/src/table/state_arrow.rs index b286482ce6..a86535037a 100644 --- a/rust/src/table/state_arrow.rs +++ b/rust/src/table/state_arrow.rs @@ -17,8 +17,8 @@ use arrow_schema::{DataType, Field, Fields, TimeUnit}; use itertools::Itertools; use super::state::DeltaTableState; -use crate::action::{ColumnCountStat, ColumnValueStat, Stats}; use crate::errors::DeltaTableError; +use crate::protocol::{ColumnCountStat, ColumnValueStat, Stats}; use crate::SchemaDataType; use crate::SchemaTypeStruct; diff --git a/rust/src/writer/json.rs b/rust/src/writer/json.rs index 85fdf6ebff..e20f9038f9 100644 --- a/rust/src/writer/json.rs +++ b/rust/src/writer/json.rs @@ -23,7 +23,7 @@ use super::{DeltaWriter, DeltaWriterError}; use crate::errors::DeltaTableError; use crate::table::builder::DeltaTableBuilder; use crate::table::DeltaTableMetaData; -use crate::{action::Add, DeltaTable, Schema}; +use crate::{protocol::Add, DeltaTable, Schema}; use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; type BadValue = (Value, ParquetError); diff --git a/rust/src/writer/mod.rs b/rust/src/writer/mod.rs index fde0d808a7..f0bcabde97 100644 --- a/rust/src/writer/mod.rs +++ b/rust/src/writer/mod.rs @@ -7,9 +7,9 @@ use object_store::Error as ObjectStoreError; use parquet::errors::ParquetError; use serde_json::Value; -use crate::action::{Action, Add, ColumnCountStat, DeltaOperation}; use crate::errors::DeltaTableError; use crate::operations::transaction::commit; +use crate::protocol::{Action, Add, ColumnCountStat, DeltaOperation}; use crate::DeltaTable; pub use json::JsonWriter; @@ -140,7 +140,7 @@ pub trait DeltaWriter { None }; let operation = DeltaOperation::Write { - mode: crate::action::SaveMode::Append, + mode: crate::protocol::SaveMode::Append, partition_by, predicate: None, }; diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index 6f56d9210d..e3eb90da5e 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -25,7 +25,7 @@ use super::{DeltaWriter, DeltaWriterError}; use crate::errors::DeltaTableError; use crate::table::builder::DeltaTableBuilder; use crate::table::DeltaTableMetaData; -use crate::{action::Add, storage::DeltaObjectStore, DeltaTable, Schema}; +use crate::{protocol::Add, storage::DeltaObjectStore, DeltaTable, Schema}; /// Writes messages to a delta lake table. pub struct RecordBatchWriter { diff --git a/rust/src/writer/stats.rs b/rust/src/writer/stats.rs index 5d3be1c4c7..00c3deb95e 100644 --- a/rust/src/writer/stats.rs +++ b/rust/src/writer/stats.rs @@ -11,7 +11,7 @@ use parquet::{ }; use super::*; -use crate::action::{Add, ColumnValueStat, Stats}; +use crate::protocol::{Add, ColumnValueStat, Stats}; pub(crate) fn create_add( partition_values: &HashMap>, @@ -455,8 +455,8 @@ mod tests { use super::utils::record_batch_from_message; use super::*; use crate::{ - action::{ColumnCountStat, ColumnValueStat}, errors::DeltaTableError, + protocol::{ColumnCountStat, ColumnValueStat}, table::builder::DeltaTableBuilder, DeltaTable, }; diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index b169b7af02..b230dfad0a 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -1,6 +1,6 @@ #[cfg(all(feature = "arrow", feature = "parquet"))] mod fs_common; -use deltalake::action::DeltaOperation; +use deltalake::protocol::DeltaOperation; // NOTE: The below is a useful external command for inspecting the written checkpoint schema visually: // parquet-tools inspect tests/data/checkpoints/_delta_log/00000000000000000005.checkpoint.parquet @@ -211,7 +211,7 @@ mod checkpoints_with_tombstones { use super::*; use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; - use deltalake::action::*; + use deltalake::protocol::*; use deltalake::table::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 0712a9fb77..2f80e1009f 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -7,11 +7,11 @@ use std::{collections::HashMap, error::Error, sync::Arc}; use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use arrow_select::concat::concat_batches; -use deltalake::action::{Action, DeltaOperation, Remove}; use deltalake::errors::DeltaTableError; use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics, OptimizeType}; use deltalake::operations::transaction::commit; use deltalake::operations::DeltaOps; +use deltalake::protocol::{Action, DeltaOperation, Remove}; use deltalake::storage::ObjectStoreRef; use deltalake::writer::{DeltaWriter, RecordBatchWriter}; use deltalake::{DeltaTable, PartitionFilter, Path, SchemaDataType, SchemaField}; diff --git a/rust/tests/commit_info_format.rs b/rust/tests/commit_info_format.rs index c50a40d818..fdb1f89d92 100644 --- a/rust/tests/commit_info_format.rs +++ b/rust/tests/commit_info_format.rs @@ -1,8 +1,8 @@ #![allow(dead_code)] mod fs_common; -use deltalake::action::{Action, DeltaOperation, SaveMode}; use deltalake::operations::transaction::commit; +use deltalake::protocol::{Action, DeltaOperation, SaveMode}; use serde_json::json; use std::error::Error; use tempdir::TempDir; diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 2966b1a911..ecadb5f647 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -1,9 +1,9 @@ #![allow(dead_code, unused_variables)] use bytes::Bytes; -use deltalake::action::{self, Add, DeltaOperation, Remove, SaveMode}; use deltalake::operations::create::CreateBuilder; use deltalake::operations::transaction::commit; +use deltalake::protocol::{self, Add, DeltaOperation, Remove, SaveMode}; use deltalake::storage::DeltaObjectStore; use deltalake::DeltaTableBuilder; use deltalake::{DeltaTable, Schema}; @@ -141,7 +141,7 @@ pub async fn add_file( partition_by: None, predicate: None, }; - let actions = vec![action::Action::add(add)]; + let actions = vec![protocol::Action::add(add)]; commit( table.object_store().as_ref(), &actions, @@ -174,7 +174,7 @@ pub async fn remove_file( ..Default::default() }; let operation = DeltaOperation::Delete { predicate: None }; - let actions = vec![action::Action::remove(remove)]; + let actions = vec![protocol::Action::remove(remove)]; commit( table.object_store().as_ref(), &actions, diff --git a/rust/tests/datafusion_test.rs b/rust/tests/datafusion_test.rs index 5d3e07f003..c7fb377990 100644 --- a/rust/tests/datafusion_test.rs +++ b/rust/tests/datafusion_test.rs @@ -25,9 +25,9 @@ use datafusion_proto::bytes::{ }; use url::Url; -use deltalake::action::SaveMode; use deltalake::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; use deltalake::operations::create::CreateBuilder; +use deltalake::protocol::SaveMode; use deltalake::storage::DeltaObjectStore; use deltalake::{ operations::{write::WriteBuilder, DeltaOps}, diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index 5a4b5b2978..8e7e687aff 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -1,7 +1,7 @@ use chrono::Utc; -use deltalake::action::{Action, Add, DeltaOperation, Remove, SaveMode}; use deltalake::operations::create::CreateBuilder; use deltalake::operations::transaction::commit; +use deltalake::protocol::{Action, Add, DeltaOperation, Remove, SaveMode}; use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; use serde_json::Value; use std::collections::HashMap; diff --git a/rust/tests/integration_concurrent_writes.rs b/rust/tests/integration_concurrent_writes.rs index 91e4345963..9056816146 100644 --- a/rust/tests/integration_concurrent_writes.rs +++ b/rust/tests/integration_concurrent_writes.rs @@ -1,8 +1,8 @@ #![cfg(feature = "integration_test")] -use deltalake::action::{Action, Add, DeltaOperation, SaveMode}; use deltalake::operations::transaction::commit; use deltalake::operations::DeltaOps; +use deltalake::protocol::{Action, Add, DeltaOperation, SaveMode}; use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; use deltalake::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; use std::collections::HashMap; diff --git a/rust/tests/integration_read.rs b/rust/tests/integration_read.rs index 67d135d084..c49ab7945a 100644 --- a/rust/tests/integration_read.rs +++ b/rust/tests/integration_read.rs @@ -152,7 +152,7 @@ async fn read_simple_table(integration: &IntegrationContext) -> TestResult { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 31); - assert!(tombstones.contains(&deltalake::action::Remove { + assert!(tombstones.contains(&deltalake::protocol::Remove { path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1587968596250), data_change: true, @@ -188,7 +188,7 @@ async fn read_simple_table_with_version(integration: &IntegrationContext) -> Tes ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 29); - assert!(tombstones.contains(&deltalake::action::Remove { + assert!(tombstones.contains(&deltalake::protocol::Remove { path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1587968596250), data_change: true, @@ -251,7 +251,7 @@ mod gcs { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 31); - assert!(tombstones.contains(&deltalake::action::Remove { + assert!(tombstones.contains(&deltalake::protocol::Remove { path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1587968596250), data_change: true, diff --git a/rust/tests/read_delta_partitions_test.rs b/rust/tests/read_delta_partitions_test.rs index 522c06646d..8ee68777cc 100644 --- a/rust/tests/read_delta_partitions_test.rs +++ b/rust/tests/read_delta_partitions_test.rs @@ -101,7 +101,7 @@ fn test_match_filters() { #[cfg(all(feature = "arrow", feature = "parquet"))] #[tokio::test] async fn read_null_partitions_from_checkpoint() { - use deltalake::action::Add; + use deltalake::protocol::Add; use maplit::hashmap; use serde_json::json; diff --git a/rust/tests/read_delta_test.rs b/rust/tests/read_delta_test.rs index db6c51ad9f..a70b241f95 100644 --- a/rust/tests/read_delta_test.rs +++ b/rust/tests/read_delta_test.rs @@ -26,7 +26,7 @@ async fn read_delta_2_0_table_without_version() { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 4); - assert!(tombstones.contains(&deltalake::action::Remove { + assert!(tombstones.contains(&deltalake::protocol::Remove { path: "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1564524298213), data_change: false, @@ -192,7 +192,7 @@ async fn read_delta_8_0_table_without_version() { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 1); - assert!(tombstones.contains(&deltalake::action::Remove { + assert!(tombstones.contains(&deltalake::protocol::Remove { path: "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1615043776198), data_change: true, @@ -416,7 +416,7 @@ async fn read_delta_1_2_1_struct_stats_table() { fn get_stats_for_file( table: &deltalake::DeltaTable, file_name: &str, - ) -> deltalake::action::Stats { + ) -> deltalake::protocol::Stats { table .get_file_uris() .zip(table.get_stats()) @@ -450,7 +450,7 @@ async fn test_action_reconciliation() { assert_eq!(table.get_files(), vec![Path::from(a.path.clone())]); // Remove added file. - let r = deltalake::action::Remove { + let r = deltalake::protocol::Remove { path: a.path.clone(), deletion_timestamp: Some(Utc::now().timestamp_millis()), data_change: false, From e5c323a7500ec9d51985c81402b6b0825eae5c00 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 6 Jun 2023 13:00:40 +0200 Subject: [PATCH 17/19] fix: parquet2 imports --- rust/src/protocol/parquet2_read/boolean.rs | 2 +- rust/src/protocol/parquet2_read/map.rs | 2 +- rust/src/protocol/parquet2_read/mod.rs | 6 +++--- rust/src/protocol/parquet2_read/primitive.rs | 2 +- rust/src/protocol/parquet2_read/stats.rs | 2 +- rust/src/protocol/parquet2_read/string.rs | 2 +- rust/src/table/state.rs | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/rust/src/protocol/parquet2_read/boolean.rs b/rust/src/protocol/parquet2_read/boolean.rs index 553ba36709..474a61a153 100644 --- a/rust/src/protocol/parquet2_read/boolean.rs +++ b/rust/src/protocol/parquet2_read/boolean.rs @@ -4,7 +4,7 @@ use parquet2::page::DataPage; use super::validity::ValidityRowIndexIter; use super::{split_page, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; /// Parquet dictionary primitive value reader pub struct SomeBooleanValueIter<'a> { diff --git a/rust/src/protocol/parquet2_read/map.rs b/rust/src/protocol/parquet2_read/map.rs index ed730e383b..0739feae2d 100644 --- a/rust/src/protocol/parquet2_read/map.rs +++ b/rust/src/protocol/parquet2_read/map.rs @@ -3,7 +3,7 @@ use parquet2::page::{DataPage, DictPage}; use super::string::for_each_repeated_string_field_value_with_idx; use super::{ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; #[derive(Default)] pub struct MapState { diff --git a/rust/src/protocol/parquet2_read/mod.rs b/rust/src/protocol/parquet2_read/mod.rs index afa6065279..28908fe6bd 100644 --- a/rust/src/protocol/parquet2_read/mod.rs +++ b/rust/src/protocol/parquet2_read/mod.rs @@ -11,7 +11,7 @@ use parquet2::read::get_page_iterator; use parquet2::read::levels::get_bit_width; use super::ProtocolError; -use crate::action::{Action, Add, CommitInfo, MetaData, Protocol, Remove, Txn}; +use crate::protocol::{Action, Add, CommitInfo, MetaData, Protocol, Remove, Txn}; use crate::schema::Guid; use boolean::for_each_boolean_field_value; use map::for_each_map_field_value; @@ -751,7 +751,7 @@ mod tests { assert_eq!(meta_data.description, None); assert_eq!( meta_data.format, - crate::action::Format::new("parquet".to_string(), None), + crate::protocol::Format::new("parquet".to_string(), None), ); assert_eq!(meta_data.schema_string, "{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"); assert_eq!(meta_data.partition_columns.len(), 0); @@ -826,7 +826,7 @@ mod tests { assert_eq!(meta_data.description, None); assert_eq!( meta_data.format, - crate::action::Format::new("parquet".to_string(), None), + crate::protocol::Format::new("parquet".to_string(), None), ); assert_eq!( meta_data.schema_string, diff --git a/rust/src/protocol/parquet2_read/primitive.rs b/rust/src/protocol/parquet2_read/primitive.rs index 4f262f7aaa..29147ea8ca 100644 --- a/rust/src/protocol/parquet2_read/primitive.rs +++ b/rust/src/protocol/parquet2_read/primitive.rs @@ -12,7 +12,7 @@ use parquet2::types::NativeType; use super::dictionary; use super::validity::ValidityRowIndexIter; use super::{split_page, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; struct ExactChunksIter<'a, T: NativeType> { chunks: std::slice::ChunksExact<'a, u8>, diff --git a/rust/src/protocol/parquet2_read/stats.rs b/rust/src/protocol/parquet2_read/stats.rs index c9bb2f9bdc..689dfea6c0 100644 --- a/rust/src/protocol/parquet2_read/stats.rs +++ b/rust/src/protocol/parquet2_read/stats.rs @@ -1,4 +1,4 @@ -use crate::action::{Add, ProtocolError, Stats}; +use crate::protocol::{Add, ProtocolError, Stats}; impl Add { /// Returns the composite HashMap representation of stats contained in the action if present. diff --git a/rust/src/protocol/parquet2_read/string.rs b/rust/src/protocol/parquet2_read/string.rs index 1a851aec3b..fc0ec574e0 100644 --- a/rust/src/protocol/parquet2_read/string.rs +++ b/rust/src/protocol/parquet2_read/string.rs @@ -9,7 +9,7 @@ use super::dictionary; use super::dictionary::binary::BinaryPageDict; use super::validity::{ValidityRepeatedRowIndexIter, ValidityRowIndexIter}; use super::{split_page, split_page_nested, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; pub trait StringValueIter<'a>: Iterator> { fn try_from_encoded_values( diff --git a/rust/src/table/state.rs b/rust/src/table/state.rs index 67a790470d..2d818deb9a 100644 --- a/rust/src/table/state.rs +++ b/rust/src/table/state.rs @@ -134,7 +134,7 @@ impl DeltaTableState { .map_err(protocol::ProtocolError::from)? { self.process_action( - protocol, + action, table_config.require_tombstones, table_config.require_files, )?; From c9bd37ccddc75bf5feb1c966bed1f80a0380ea47 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Mon, 17 Jul 2023 22:36:45 +0200 Subject: [PATCH 18/19] fix: imports after merge --- rust/examples/basic_operations.rs | 2 +- rust/src/operations/restore.rs | 12 ++++++------ rust/src/writer/test_utils.rs | 4 ++-- rust/tests/command_restore.rs | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/rust/examples/basic_operations.rs b/rust/examples/basic_operations.rs index 5c1fb46e86..d75123bdfd 100644 --- a/rust/examples/basic_operations.rs +++ b/rust/examples/basic_operations.rs @@ -4,7 +4,7 @@ use arrow::{ record_batch::RecordBatch, }; use deltalake::operations::collect_sendable_stream; -use deltalake::{action::SaveMode, DeltaOps, SchemaDataType, SchemaField}; +use deltalake::{protocol::SaveMode, DeltaOps, SchemaDataType, SchemaField}; use parquet::{ basic::{Compression, ZstdLevel}, file::properties::WriterProperties, diff --git a/rust/src/operations/restore.rs b/rust/src/operations/restore.rs index d1e8a378aa..2d6d5754de 100644 --- a/rust/src/operations/restore.rs +++ b/rust/src/operations/restore.rs @@ -29,11 +29,11 @@ use futures::future::BoxFuture; use object_store::path::Path; use object_store::ObjectStore; -use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::operations::transaction::{prepare_commit, try_commit_transaction, TransactionError}; +use crate::protocol::{Action, Add, DeltaOperation, Protocol, Remove}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; -use crate::{action, DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; +use crate::table::state::DeltaTableState; +use crate::{DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; /// Errors that can occur during restore #[derive(thiserror::Error, Debug)] @@ -199,12 +199,12 @@ async fn execute( let mut actions = vec![]; let protocol = if protocol_downgrade_allowed { - action::Protocol { + Protocol { min_reader_version: table.get_min_reader_version(), min_writer_version: table.get_min_writer_version(), } } else { - action::Protocol { + Protocol { min_reader_version: max( table.get_min_reader_version(), snapshot.min_reader_version(), @@ -246,7 +246,7 @@ async fn execute( async fn check_files_available( object_store: &dyn ObjectStore, - files: &Vec, + files: &Vec, ) -> DeltaResult<()> { for file in files { let file_path = Path::from(file.path.clone()); diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index 032f03eff1..d6c79bee94 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -4,11 +4,11 @@ use std::collections::HashMap; use std::sync::Arc; use arrow::compute::take; -use arrow_array::{Int32Array, RecordBatch, StringArray, UInt32Array}; +use arrow_array::{Int32Array, Int64Array, RecordBatch, StringArray, StructArray, UInt32Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use crate::operations::create::CreateBuilder; -use crate::schema::Schema; +use crate::schema::{Schema, SchemaTypeStruct}; use crate::table::DeltaTableMetaData; use crate::{DeltaTable, DeltaTableBuilder, SchemaDataType, SchemaField}; diff --git a/rust/tests/command_restore.rs b/rust/tests/command_restore.rs index 4902d85a28..8ab6d8ee44 100644 --- a/rust/tests/command_restore.rs +++ b/rust/tests/command_restore.rs @@ -4,7 +4,7 @@ use arrow::datatypes::Schema as ArrowSchema; use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType, Field}; use chrono::{DateTime, NaiveDateTime, Utc}; -use deltalake::action::SaveMode; +use deltalake::protocol::SaveMode; use deltalake::{DeltaOps, DeltaTable, SchemaDataType, SchemaField}; use rand::Rng; use std::collections::HashMap; From 2a425d71dfaddab65c170700d822f9912c56eb2f Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 13:32:53 +0200 Subject: [PATCH 19/19] chore: various fixes and cleanup after merge --- rust/src/operations/merge.rs | 33 ++++++++------------- rust/src/operations/mod.rs | 2 +- rust/src/protocol/parquet_read/mod.rs | 8 ++--- rust/src/storage/utils.rs | 2 +- rust/src/table/mod.rs | 2 +- rust/src/table/state.rs | 2 +- rust/src/writer/record_batch.rs | 2 -- rust/tests/command_filesystem_check.rs | 14 ++++----- rust/tests/integration_concurrent_writes.rs | 4 +-- rust/tests/integration_datafusion.rs | 14 ++++----- rust/tests/integration_object_store.rs | 4 +-- rust/tests/integration_read.rs | 2 +- rust/tests/read_delta_log_test.rs | 2 +- rust/tests/repair_s3_rename_test.rs | 8 ++--- 14 files changed, 43 insertions(+), 56 deletions(-) diff --git a/rust/src/operations/merge.rs b/rust/src/operations/merge.rs index 29c5ffaa27..d088fbd3b7 100644 --- a/rust/src/operations/merge.rs +++ b/rust/src/operations/merge.rs @@ -32,6 +32,10 @@ //! .await? //! ```` +use std::collections::HashMap; +use std::sync::Arc; +use std::time::{Instant, SystemTime, UNIX_EPOCH}; + use arrow_schema::SchemaRef; use datafusion::error::Result as DataFusionResult; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -55,27 +59,16 @@ use datafusion_physical_expr::{create_physical_expr, expressions, PhysicalExpr}; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; use serde_json::{Map, Value}; -use std::{ - collections::HashMap, - sync::Arc, - time::{Instant, SystemTime, UNIX_EPOCH}, -}; -use crate::action::MergePredicate; +use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; +use super::transaction::commit; +use crate::delta_datafusion::{parquet_scan_from_actions, register_store}; use crate::operations::datafusion_utils::MetricObserverExec; -use crate::{ - action::{Action, DeltaOperation, Remove}, - delta_datafusion::{parquet_scan_from_actions, register_store}, - operations::write::write_execution_plan, - storage::{DeltaObjectStore, ObjectStoreRef}, - table_state::DeltaTableState, - DeltaResult, DeltaTable, DeltaTableError, -}; - -use super::{ - datafusion_utils::{into_expr, maybe_into_expr, Expression}, - transaction::commit, -}; +use crate::operations::write::write_execution_plan; +use crate::protocol::{Action, DeltaOperation, MergePredicate, Remove}; +use crate::storage::{DeltaObjectStore, ObjectStoreRef}; +use crate::table::state::DeltaTableState; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; const OPERATION_COLUMN: &str = "__delta_rs_operation"; const DELETE_COLUMN: &str = "__delta_rs_delete"; @@ -1105,8 +1098,8 @@ impl std::future::IntoFuture for MergeBuilder { #[cfg(test)] mod tests { - use crate::action::*; use crate::operations::DeltaOps; + use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::get_arrow_schema; use crate::writer::test_utils::get_delta_schema; diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 0bc7ce55a3..7b6cb27ace 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -213,7 +213,7 @@ mod datafusion_utils { use datafusion_expr::Expr; use futures::{Stream, StreamExt}; - use crate::{table_state::DeltaTableState, DeltaResult}; + use crate::{table::state::DeltaTableState, DeltaResult}; /// Used to represent user input of either a Datafusion expression or string expression pub enum Expression { diff --git a/rust/src/protocol/parquet_read/mod.rs b/rust/src/protocol/parquet_read/mod.rs index 3e0c22557f..93fdc4c2df 100644 --- a/rust/src/protocol/parquet_read/mod.rs +++ b/rust/src/protocol/parquet_read/mod.rs @@ -6,7 +6,7 @@ use num_traits::cast::ToPrimitive; use parquet::record::{Field, ListAccessor, MapAccessor, RowAccessor}; use serde_json::json; -use crate::action::{ +use crate::protocol::{ Action, Add, AddCDCFile, ColumnCountStat, ColumnValueStat, DeletionVector, MetaData, Protocol, ProtocolError, Remove, Stats, Txn, }; @@ -217,7 +217,7 @@ impl Add { "minValues" => if let Ok(row) = record.get_group(i) { for (name, field) in row.get_column_iter() { if !matches!(field, Field::Null) { - if let Some(values) = field_to_value_stat(&field, name) { + if let Some(values) = field_to_value_stat(field, name) { stats.min_values.insert(name.clone(), values); } } @@ -228,7 +228,7 @@ impl Add { "maxValues" => if let Ok(row) = record.get_group(i) { for (name, field) in row.get_column_iter() { if !matches!(field, Field::Null) { - if let Some(values) = field_to_value_stat(&field, name) { + if let Some(values) = field_to_value_stat(field, name) { stats.max_values.insert(name.clone(), values); } } @@ -239,7 +239,7 @@ impl Add { "nullCount" => if let Ok(row) = record.get_group(i) { for (name, field) in row.get_column_iter() { if !matches!(field, Field::Null) { - if let Some(count) = field_to_count_stat(&field, name) { + if let Some(count) = field_to_count_stat(field, name) { stats.null_count.insert(name.clone(), count); } } diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index c99992763a..80710efd9b 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -82,7 +82,7 @@ impl TryFrom<&Add> for ObjectMeta { fn try_from(value: &Add) -> DeltaResult { let last_modified = Utc.from_utc_datetime( &NaiveDateTime::from_timestamp_millis(value.modification_time).ok_or( - DeltaTableError::from(crate::action::ProtocolError::InvalidField(format!( + DeltaTableError::from(crate::protocol::ProtocolError::InvalidField(format!( "invalid modification_time: {:?}", value.modification_time ))), diff --git a/rust/src/table/mod.rs b/rust/src/table/mod.rs index f7b40d391c..667b2239aa 100644 --- a/rust/src/table/mod.rs +++ b/rust/src/table/mod.rs @@ -31,7 +31,7 @@ use crate::storage::{commit_uri_from_version, ObjectStoreRef}; pub mod builder; pub mod config; pub mod state; -#[cfg(all(feature = "arrow"))] +#[cfg(feature = "arrow")] pub mod state_arrow; /// Metadata for a checkpoint file diff --git a/rust/src/table/state.rs b/rust/src/table/state.rs index 52ff5383c7..77feabdc8c 100644 --- a/rust/src/table/state.rs +++ b/rust/src/table/state.rs @@ -360,7 +360,7 @@ impl DeltaTableState { protocol::Action::commitInfo(v) => { self.commit_infos.push(v); } - action::Action::domainMetadata(v) => { + protocol::Action::domainMetadata(v) => { self.domain_metadatas.push(v); } } diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index 2beb38515c..e6495b6539 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -9,8 +9,6 @@ use std::{collections::HashMap, sync::Arc}; use arrow::array::{Array, UInt32Array}; use arrow::compute::{partition, take}; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; -use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use arrow_array::ArrayRef; use arrow_row::{RowConverter, SortField}; diff --git a/rust/tests/command_filesystem_check.rs b/rust/tests/command_filesystem_check.rs index ced317d990..86fcd8f52e 100644 --- a/rust/tests/command_filesystem_check.rs +++ b/rust/tests/command_filesystem_check.rs @@ -12,7 +12,7 @@ mod common; #[tokio::test] #[serial] async fn test_filesystem_check_local() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Local).await?) + test_filesystem_check(StorageIntegration::Local).await } #[cfg(any(feature = "s3", feature = "s3-native-tls"))] @@ -21,21 +21,21 @@ async fn test_filesystem_check_local() -> TestResult { async fn test_filesystem_check_aws() -> TestResult { set_env_if_not_set("AWS_S3_ALLOW_UNSAFE_RENAME", "true"); set_env_if_not_set("AWS_S3_LOCKING_PROVIDER", "none"); - Ok(test_filesystem_check(StorageIntegration::Amazon).await?) + test_filesystem_check(StorageIntegration::Amazon).await } #[cfg(feature = "azure")] #[tokio::test] #[serial] async fn test_filesystem_check_azure() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Microsoft).await?) + test_filesystem_check(StorageIntegration::Microsoft).await } #[cfg(feature = "gcs")] #[tokio::test] #[serial] async fn test_filesystem_check_gcp() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Google).await?) + test_filesystem_check(StorageIntegration::Google).await } #[cfg(feature = "hdfs")] @@ -73,7 +73,7 @@ async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table.state.all_tombstones().get(file).unwrap(); - assert_eq!(remove.data_change, true); + assert!(remove.data_change); // An additonal run should return an empty list of orphaned actions let op = DeltaOps::from(table); @@ -114,7 +114,7 @@ async fn test_filesystem_check_partitioned() -> TestResult { assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table.state.all_tombstones().get(file).unwrap(); - assert_eq!(remove.data_change, true); + assert!(remove.data_change); Ok(()) } @@ -170,7 +170,7 @@ async fn test_filesystem_check_outdated() -> TestResult { if let Err(DeltaTableError::VersionAlreadyExists(version)) = res { assert!(version == 3); } else { - assert!(false); + panic!(); } Ok(()) diff --git a/rust/tests/integration_concurrent_writes.rs b/rust/tests/integration_concurrent_writes.rs index 2494e8c472..f34feac6e0 100644 --- a/rust/tests/integration_concurrent_writes.rs +++ b/rust/tests/integration_concurrent_writes.rs @@ -103,12 +103,12 @@ where assert_eq!(map.len() as i64, WORKERS * COMMITS); // check that we have unique and ascending versions committed - let mut versions = Vec::from_iter(map.keys().map(|x| x.clone())); + let mut versions = Vec::from_iter(map.keys().copied()); versions.sort(); assert_eq!(versions, Vec::from_iter(1i64..=WORKERS * COMMITS)); // check that each file for each worker is committed as expected - let mut files = Vec::from_iter(map.values().map(|x| x.clone())); + let mut files = Vec::from_iter(map.values().cloned()); files.sort(); let mut expected = Vec::new(); for w in 0..WORKERS { diff --git a/rust/tests/integration_datafusion.rs b/rust/tests/integration_datafusion.rs index 2489ab9371..1524305759 100644 --- a/rust/tests/integration_datafusion.rs +++ b/rust/tests/integration_datafusion.rs @@ -31,9 +31,9 @@ use datafusion_proto::bytes::{ }; use url::Url; -use deltalake::action::SaveMode; use deltalake::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; use deltalake::operations::create::CreateBuilder; +use deltalake::protocol::SaveMode; use deltalake::storage::DeltaObjectStore; use deltalake::writer::{DeltaWriter, RecordBatchWriter}; use deltalake::{ @@ -51,7 +51,7 @@ mod local { #[tokio::test] #[serial] async fn test_datafusion_local() -> TestResult { - Ok(test_datafusion(StorageIntegration::Local).await?) + test_datafusion(StorageIntegration::Local).await } fn get_scanned_files(node: &dyn ExecutionPlan) -> HashSet