From 8485aa9628387e208a1ba7cb6d1ee09e633cd74b Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 01:28:28 -0500 Subject: [PATCH 01/10] Extract log store into its own crate Signed-off-by: Abhi Agarwal --- crates/aws/Cargo.toml | 2 +- crates/aws/src/credentials.rs | 12 +- crates/aws/src/lib.rs | 8 +- crates/aws/src/logstore/default_logstore.rs | 19 +- crates/aws/src/logstore/dynamodb_logstore.rs | 52 +- crates/aws/src/storage.rs | 29 +- crates/aws/tests/repair_s3_rename_test.rs | 4 +- crates/azure/Cargo.toml | 2 +- crates/azure/src/config.rs | 6 +- crates/azure/src/error.rs | 22 - crates/azure/src/lib.rs | 19 +- crates/catalog-unity/Cargo.toml | 2 +- crates/core/Cargo.toml | 2 + crates/core/src/delta_datafusion/mod.rs | 27 + .../src/delta_datafusion/table_provider.rs | 5 +- crates/core/src/errors.rs | 3 + crates/core/src/kernel/mod.rs | 1 + crates/core/src/kernel/models/actions.rs | 2 +- crates/core/src/kernel/models/mod.rs | 83 +-- crates/core/src/kernel/schema/partitions.rs | 2 +- crates/core/src/kernel/schema/schema.rs | 2 +- .../kernel/transaction/conflict_checker.rs | 3 +- crates/core/src/kernel/transaction/mod.rs | 3 +- crates/core/src/kernel/transaction/state.rs | 1 + crates/core/src/lib.rs | 11 +- .../core/src/operations/convert_to_delta.rs | 2 + crates/core/src/operations/load_cdf.rs | 10 +- crates/core/src/operations/merge/mod.rs | 6 +- crates/core/src/operations/optimize.rs | 4 +- crates/core/src/operations/restore.rs | 5 +- crates/core/src/operations/write/mod.rs | 2 +- crates/core/src/protocol/mod.rs | 82 +-- crates/core/src/table/builder.rs | 8 +- crates/core/src/table/mod.rs | 18 +- crates/core/src/writer/stats.rs | 1 + crates/derive/src/lib.rs | 4 +- crates/gcp/Cargo.toml | 2 +- crates/gcp/src/lib.rs | 14 +- crates/gcp/src/storage.rs | 2 +- crates/hdfs/Cargo.toml | 2 +- crates/hdfs/src/lib.rs | 12 +- crates/logstore/Cargo.toml | 62 ++ .../src/logstore => logstore/src}/config.rs | 62 +- .../src}/default_logstore.rs | 19 +- crates/logstore/src/error.rs | 85 +++ .../logstore => logstore/src}/factories.rs | 20 +- crates/logstore/src/lib.rs | 41 ++ .../mod.rs => logstore/src/logstore_impl.rs} | 210 +++--- .../logstore => logstore/src}/storage/mod.rs | 25 +- .../src}/storage/retry_ext.rs | 7 +- .../src}/storage/runtime.rs | 0 .../src}/storage/utils.rs | 38 +- crates/types/Cargo.toml | 27 + crates/types/src/actions.rs | 644 ++++++++++++++++++ crates/types/src/lib.rs | 13 + crates/types/src/serde_path.rs | 59 ++ 56 files changed, 1304 insertions(+), 504 deletions(-) delete mode 100644 crates/azure/src/error.rs create mode 100644 crates/logstore/Cargo.toml rename crates/{core/src/logstore => logstore/src}/config.rs (87%) rename crates/{core/src/logstore => logstore/src}/default_logstore.rs (87%) create mode 100644 crates/logstore/src/error.rs rename crates/{core/src/logstore => logstore/src}/factories.rs (89%) create mode 100644 crates/logstore/src/lib.rs rename crates/{core/src/logstore/mod.rs => logstore/src/logstore_impl.rs} (84%) rename crates/{core/src/logstore => logstore/src}/storage/mod.rs (84%) rename crates/{core/src/logstore => logstore/src}/storage/retry_ext.rs (95%) rename crates/{core/src/logstore => logstore/src}/storage/runtime.rs (100%) rename crates/{core/src/logstore => logstore/src}/storage/utils.rs (58%) create mode 100644 crates/types/Cargo.toml create mode 100644 crates/types/src/actions.rs create mode 100644 crates/types/src/lib.rs create mode 100644 crates/types/src/serde_path.rs diff --git a/crates/aws/Cargo.toml b/crates/aws/Cargo.toml index f6d1bf37cf..d6e326ee21 100644 --- a/crates/aws/Cargo.toml +++ b/crates/aws/Cargo.toml @@ -13,7 +13,7 @@ rust-version.workspace = true [dependencies] # path dependencies -deltalake-core = { version = "0.29.0", path = "../core" , features = ["cloud"]} +deltalake-logstore = { version = "0.29.0", path = "../logstore" , features = ["cloud"]} # workspace dependencies async-trait = { workspace = true } diff --git a/crates/aws/src/credentials.rs b/crates/aws/src/credentials.rs index 065f00ac7b..3834551418 100644 --- a/crates/aws/src/credentials.rs +++ b/crates/aws/src/credentials.rs @@ -14,11 +14,11 @@ use aws_credential_types::provider::error::CredentialsError; use aws_credential_types::provider::{future, ProvideCredentials}; use aws_credential_types::Credentials; -use deltalake_core::logstore::object_store::aws::{AmazonS3ConfigKey, AwsCredential}; -use deltalake_core::logstore::object_store::{ +use deltalake_logstore::object_store::aws::{AmazonS3ConfigKey, AwsCredential}; +use deltalake_logstore::object_store::{ CredentialProvider, Error as ObjectStoreError, Result as ObjectStoreResult, }; -use deltalake_core::DeltaResult; +use deltalake_logstore::LogStoreResult; use tokio::sync::Mutex; use tracing::log::*; @@ -255,7 +255,7 @@ fn assume_session_name(options: &HashMap) -> String { /// Take a set of [StorageOptions] and produce an appropriate AWS SDK [SdkConfig] /// for use with various AWS SDK APIs, such as in our [S3DynamoDbLogStore](crate::logstore::S3DynamoDbLogStore) -pub async fn resolve_credentials(options: &HashMap) -> DeltaResult { +pub async fn resolve_credentials(options: &HashMap) -> LogStoreResult { let default_provider = DefaultCredentialsChain::builder().build().await; let credentials_provider = match assume_role_arn(options) { @@ -373,7 +373,7 @@ mod tests { #[tokio::test] #[serial] - async fn test_object_store_credential_provider() -> DeltaResult<()> { + async fn test_object_store_credential_provider() -> LogStoreResult<()> { let options = HashMap::from([ ( constants::AWS_ACCESS_KEY_ID.to_string(), @@ -403,7 +403,7 @@ mod tests { /// operations. #[tokio::test] #[serial] - async fn test_object_store_credential_provider_consistency() -> DeltaResult<()> { + async fn test_object_store_credential_provider_consistency() -> LogStoreResult<()> { let options = HashMap::from([ ( constants::AWS_ACCESS_KEY_ID.to_string(), diff --git a/crates/aws/src/lib.rs b/crates/aws/src/lib.rs index f5d2f47404..c7bf739576 100644 --- a/crates/aws/src/lib.rs +++ b/crates/aws/src/lib.rs @@ -26,12 +26,12 @@ use aws_sdk_dynamodb::{ }, Client, }; -use deltalake_core::logstore::object_store::aws::AmazonS3ConfigKey; -use deltalake_core::logstore::{ +use deltalake_logstore::object_store::aws::AmazonS3ConfigKey; +use deltalake_logstore::{ default_logstore, logstore_factories, object_store_factories, LogStore, LogStoreFactory, ObjectStoreRef, StorageConfig, }; -use deltalake_core::{DeltaResult, Path}; +use deltalake_logstore::{LogStoreResult, Path}; use errors::{DynamoDbConfigError, LockClientError}; use regex::Regex; use std::{ @@ -58,7 +58,7 @@ impl LogStoreFactory for S3LogStoreFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { let s3_options = self.with_env_s3(&options.raw.clone()); if s3_options.keys().any(|key| { let key = key.to_ascii_lowercase(); diff --git a/crates/aws/src/logstore/default_logstore.rs b/crates/aws/src/logstore/default_logstore.rs index 75c6d75a22..4568017798 100644 --- a/crates/aws/src/logstore/default_logstore.rs +++ b/crates/aws/src/logstore/default_logstore.rs @@ -3,10 +3,7 @@ use std::sync::Arc; use bytes::Bytes; -use deltalake_core::logstore::*; -use deltalake_core::{ - kernel::transaction::TransactionError, logstore::ObjectStoreRef, DeltaResult, -}; +use deltalake_logstore::*; use object_store::{Error as ObjectStoreError, ObjectStore}; use url::Url; use uuid::Uuid; @@ -65,7 +62,7 @@ impl LogStore for S3LogStore { "S3LogStore".into() } - async fn read_commit_entry(&self, version: i64) -> DeltaResult> { + async fn read_commit_entry(&self, version: i64) -> LogStoreResult> { read_commit_entry(self.object_store(None).as_ref(), version).await } @@ -79,7 +76,7 @@ impl LogStore for S3LogStore { version: i64, commit_or_bytes: CommitOrBytes, _operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { match commit_or_bytes { CommitOrBytes::TmpCommit(tmp_commit) => { Ok( @@ -89,12 +86,12 @@ impl LogStore for S3LogStore { } _ => unreachable!(), // S3 Log Store should never receive bytes } - .map_err(|err| -> TransactionError { + .map_err(|err| -> LogStoreError { match err { ObjectStoreError::AlreadyExists { .. } => { - TransactionError::VersionAlreadyExists(version) + LogStoreError::VersionAlreadyExists(version) } - _ => TransactionError::from(err), + _ => LogStoreError::from(err), } })?; Ok(()) @@ -105,7 +102,7 @@ impl LogStore for S3LogStore { version: i64, commit_or_bytes: CommitOrBytes, _operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { match &commit_or_bytes { CommitOrBytes::TmpCommit(tmp_commit) => { abort_commit_entry(self.object_store(None).as_ref(), version, tmp_commit).await @@ -114,7 +111,7 @@ impl LogStore for S3LogStore { } } - async fn get_latest_version(&self, current_version: i64) -> DeltaResult { + async fn get_latest_version(&self, current_version: i64) -> LogStoreResult { get_latest_version(self, current_version).await } diff --git a/crates/aws/src/logstore/dynamodb_logstore.rs b/crates/aws/src/logstore/dynamodb_logstore.rs index d43c280e50..8113fc8f66 100644 --- a/crates/aws/src/logstore/dynamodb_logstore.rs +++ b/crates/aws/src/logstore/dynamodb_logstore.rs @@ -8,14 +8,11 @@ use crate::storage::S3StorageOptions; use crate::{constants, CommitEntry, DynamoDbLockClient, UpdateLogEntryResult}; use bytes::Bytes; -use deltalake_core::{ObjectStoreError, Path}; +use deltalake_logstore::{ObjectStoreError, Path}; use tracing::{debug, error, warn}; use url::Url; -use deltalake_core::logstore::*; -use deltalake_core::{ - kernel::transaction::TransactionError, logstore::ObjectStoreRef, DeltaResult, DeltaTableError, -}; +use deltalake_logstore::*; use uuid::Uuid; const STORE_NAME: &str = "DeltaS3ObjectStore"; @@ -44,7 +41,7 @@ impl S3DynamoDbLogStore { s3_options: &S3StorageOptions, prefixed_store: ObjectStoreRef, root_store: ObjectStoreRef, - ) -> DeltaResult { + ) -> LogStoreResult { let lock_client = DynamoDbLockClient::try_new( &s3_options.sdk_config.clone().unwrap(), s3_options @@ -65,7 +62,7 @@ impl S3DynamoDbLogStore { s3_options.dynamodb_secret_access_key.clone(), s3_options.dynamodb_session_token.clone(), ) - .map_err(|err| DeltaTableError::ObjectStore { + .map_err(|err| LogStoreError::ObjectStore { source: ObjectStoreError::Generic { store: STORE_NAME, source: Box::new(err), @@ -86,10 +83,7 @@ impl S3DynamoDbLogStore { /// Attempt to repair an incomplete log entry by moving the temporary commit file /// to `N.json` and update the associated log entry to mark it as completed. - pub async fn repair_entry( - &self, - entry: &CommitEntry, - ) -> Result { + pub async fn repair_entry(&self, entry: &CommitEntry) -> LogStoreResult { // java does this, do we need it? if entry.complete { return Ok(RepairLogEntryResult::AlreadyCompleted); @@ -107,13 +101,13 @@ impl S3DynamoDbLogStore { return self.try_complete_entry(entry, true).await; } // `N.json` has already been moved, complete the entry in DynamoDb just in case - Err(TransactionError::ObjectStore { + Err(LogStoreError::ObjectStore { source: ObjectStoreError::NotFound { .. }, }) => { warn!("It looks like the {}.json has already been moved, we got 404 from ObjectStorage.", entry.version); return self.try_complete_entry(entry, false).await; } - Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), + Err(err) if retry == MAX_REPAIR_RETRIES => return Err(LogStoreError::from(err)), Err(err) => { debug!("retry #{retry} on log entry {entry:?} failed to move commit: '{err}'") } @@ -127,14 +121,14 @@ impl S3DynamoDbLogStore { &self, entry: &CommitEntry, copy_performed: bool, - ) -> Result { + ) -> LogStoreResult { debug!("try_complete_entry for {entry:?}, {copy_performed}"); for retry in 0..=MAX_REPAIR_RETRIES { match self .lock_client .update_commit_entry(entry.version, &self.table_path) .await - .map_err(|err| TransactionError::LogStoreError { + .map_err(|err| LogStoreError::LogStoreError { msg: format!( "unable to complete entry for '{}': failure to write to DynamoDb", entry.version @@ -142,7 +136,7 @@ impl S3DynamoDbLogStore { source: Box::new(err), }) { Ok(x) => return Ok(Self::map_retry_result(x, copy_performed)), - Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), + Err(err) if retry == MAX_REPAIR_RETRIES => return Err(LogStoreError::from(err)), Err(err) => error!( "retry #{retry} on log entry {entry:?} failed to update lock db: '{err}'" ), @@ -178,12 +172,12 @@ impl LogStore for S3DynamoDbLogStore { self.table_path.clone() } - async fn refresh(&self) -> DeltaResult<()> { + async fn refresh(&self) -> LogStoreResult<()> { let entry = self .lock_client .get_latest_entry(&self.table_path) .await - .map_err(|err| DeltaTableError::GenericError { + .map_err(|err| LogStoreError::Generic { source: Box::new(err), })?; if let Some(entry) = entry { @@ -192,7 +186,7 @@ impl LogStore for S3DynamoDbLogStore { Ok(()) } - async fn read_commit_entry(&self, version: i64) -> DeltaResult> { + async fn read_commit_entry(&self, version: i64) -> LogStoreResult> { let entry = self .lock_client .get_commit_entry(&self.table_path, version) @@ -203,7 +197,7 @@ impl LogStore for S3DynamoDbLogStore { read_commit_entry(self.object_store(None).as_ref(), version).await } - /// Tries to commit a prepared commit file. Returns [DeltaTableError::VersionAlreadyExists] + /// Tries to commit a prepared commit file. Returns [LogStoreError::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` @@ -213,7 +207,7 @@ impl LogStore for S3DynamoDbLogStore { version: i64, commit_or_bytes: CommitOrBytes, _operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { let tmp_commit = match commit_or_bytes { CommitOrBytes::TmpCommit(tmp_commit) => tmp_commit, _ => unreachable!(), // S3DynamoDBLogstore should never get Bytes @@ -227,7 +221,7 @@ impl LogStore for S3DynamoDbLogStore { .map_err(|err| match err { LockClientError::VersionAlreadyExists { version, .. } => { warn!("LockClientError::VersionAlreadyExists({version})"); - TransactionError::VersionAlreadyExists(version) + LogStoreError::VersionAlreadyExists(version) } LockClientError::ProvisionedThroughputExceeded => todo!( "deltalake-aws does not yet handle DynamoDB provisioned throughput errors" @@ -235,14 +229,14 @@ impl LogStore for S3DynamoDbLogStore { LockClientError::LockTableNotFound => { let table_name = self.lock_client.get_lock_table_name(); error!("Lock table '{table_name}' not found"); - TransactionError::LogStoreError { + LogStoreError::LogStoreError { msg: format!("lock table '{table_name}' not found"), source: Box::new(err), } } err => { error!("dynamodb client failed to write log entry: {err:?}"); - TransactionError::LogStoreError { + LogStoreError::LogStoreError { msg: "dynamodb client failed to write log entry".to_owned(), source: Box::new(err), } @@ -263,7 +257,7 @@ impl LogStore for S3DynamoDbLogStore { version: i64, commit_or_bytes: CommitOrBytes, _operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { let tmp_commit = match commit_or_bytes { CommitOrBytes::TmpCommit(tmp_commit) => tmp_commit, _ => unreachable!(), // S3DynamoDBLogstore should never get Bytes @@ -277,12 +271,12 @@ impl LogStore for S3DynamoDbLogStore { ), LockClientError::VersionAlreadyCompleted { version, .. } => { error!("Trying to abort a completed commit"); - TransactionError::LogStoreError { + LogStoreError::LogStoreError { msg: format!("trying to abort a completed log entry: {version}"), source: Box::new(err), } } - err => TransactionError::LogStoreError { + err => LogStoreError::LogStoreError { msg: "dynamodb client failed to delete log entry".to_owned(), source: Box::new(err), }, @@ -292,13 +286,13 @@ impl LogStore for S3DynamoDbLogStore { Ok(()) } - async fn get_latest_version(&self, current_version: i64) -> DeltaResult { + async fn get_latest_version(&self, current_version: i64) -> LogStoreResult { debug!("Retrieving latest version of {self:?} at v{current_version}"); let entry = self .lock_client .get_latest_entry(&self.table_path) .await - .map_err(|err| DeltaTableError::GenericError { + .map_err(|err| LogStoreError::Generic { source: Box::new(err), })?; // when there is a latest entry in DynamoDb, we can avoid the file listing in S3. diff --git a/crates/aws/src/storage.rs b/crates/aws/src/storage.rs index c009ba4afe..d26172ae60 100644 --- a/crates/aws/src/storage.rs +++ b/crates/aws/src/storage.rs @@ -8,15 +8,15 @@ use std::time::Duration; use aws_config::{Region, SdkConfig}; use bytes::Bytes; -use deltalake_core::logstore::object_store::aws::{AmazonS3Builder, AmazonS3ConfigKey}; -use deltalake_core::logstore::object_store::{ +use deltalake_logstore::object_store::aws::{AmazonS3Builder, AmazonS3ConfigKey}; +use deltalake_logstore::object_store::{ GetOptions, GetResult, ListResult, MultipartUpload, ObjectMeta, ObjectStore, ObjectStoreScheme, PutMultipartOptions, PutOptions, PutPayload, PutResult, Result as ObjectStoreResult, }; -use deltalake_core::logstore::{ +use deltalake_logstore::{ config::str_is_truthy, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; -use deltalake_core::{DeltaResult, DeltaTableError, ObjectStoreError, Path}; +use deltalake_logstore::{LogStore, LogStoreError, LogStoreResult, ObjectStoreError, Path}; use futures::stream::BoxStream; use futures::Future; use object_store::aws::AmazonS3; @@ -40,7 +40,7 @@ impl ObjectStoreFactory for S3ObjectStoreFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let options = self.with_env_s3(&config.raw); // All S3-likes should start their builder the same way @@ -65,11 +65,10 @@ impl ObjectStoreFactory for S3ObjectStoreFactory { builder.with_credentials(Arc::new(AWSForObjectStore::new(sdk_config.clone()))); } - let (_, path) = - ObjectStoreScheme::parse(url).map_err(|e| DeltaTableError::GenericError { - source: Box::new(e), - })?; - let prefix = Path::parse(path)?; + let (_, path) = ObjectStoreScheme::parse(url) + .map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; + let prefix = + Path::parse(path).map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; let store = aws_storage_handler(builder.build()?, &s3_options)?; debug!("Initialized the object store: {store:?}"); @@ -81,7 +80,7 @@ impl ObjectStoreFactory for S3ObjectStoreFactory { fn aws_storage_handler( store: AmazonS3, s3_options: &S3StorageOptions, -) -> DeltaResult { +) -> LogStoreResult { // Nearly all S3 Object stores support conditional put, so we change the default to always returning an S3 Object store // unless explicitly passing a locking provider key or allow_unsafe_rename. Then we will pass it to the old S3StorageBackend. if s3_options.locking_provider.as_deref() == Some("dynamodb") || s3_options.allow_unsafe_rename @@ -161,7 +160,7 @@ impl PartialEq for S3StorageOptions { impl S3StorageOptions { /// Creates an instance of [`S3StorageOptions`] from the given HashMap. - pub fn from_map(options: &HashMap) -> DeltaResult { + pub fn from_map(options: &HashMap) -> LogStoreResult { let extra_opts: HashMap = options .iter() .filter(|(k, _)| !constants::S3_OPTS.contains(&k.as_str())) @@ -251,12 +250,12 @@ impl S3StorageOptions { } } - pub fn try_default() -> DeltaResult { + pub fn try_default() -> LogStoreResult { Self::from_map(&HashMap::new()) } } -fn execute_sdk_future(future: F) -> DeltaResult +fn execute_sdk_future(future: F) -> LogStoreResult where T: Send, F: Future + Send, @@ -273,7 +272,7 @@ where cfg = Some(handle.block_on(future)); }); }); - cfg.ok_or(DeltaTableError::ObjectStore { + cfg.ok_or(LogStoreError::ObjectStore { source: ObjectStoreError::Generic { store: STORE_NAME, source: Box::new(DynamoDbConfigError::InitializationError), diff --git a/crates/aws/tests/repair_s3_rename_test.rs b/crates/aws/tests/repair_s3_rename_test.rs index d9cddb97d7..4dfe6e054d 100644 --- a/crates/aws/tests/repair_s3_rename_test.rs +++ b/crates/aws/tests/repair_s3_rename_test.rs @@ -2,11 +2,11 @@ use bytes::Bytes; use deltalake_aws::storage::S3StorageBackend; -use deltalake_core::logstore::object_store::{ +use deltalake_core::{DeltaTableBuilder, ObjectStore, Path}; +use deltalake_logstore::object_store::{ DynObjectStore, Error as ObjectStoreError, GetOptions, GetResult, ListResult, ObjectMeta, PutOptions, PutResult, Result as ObjectStoreResult, }; -use deltalake_core::{DeltaTableBuilder, ObjectStore, Path}; use deltalake_test::utils::IntegrationContext; use futures::stream::BoxStream; use object_store::{MultipartUpload, PutMultipartOptions, PutPayload}; diff --git a/crates/azure/Cargo.toml b/crates/azure/Cargo.toml index 28c775037d..f32424965b 100644 --- a/crates/azure/Cargo.toml +++ b/crates/azure/Cargo.toml @@ -12,7 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.29.0", path = "../core", features = ["cloud"] } +deltalake-logstore = { version = "0.29.0", path = "../logstore", features = ["cloud"] } # workspace depenndecies bytes = { workspace = true } diff --git a/crates/azure/src/config.rs b/crates/azure/src/config.rs index 17bfdfad11..3fb0bd24ee 100644 --- a/crates/azure/src/config.rs +++ b/crates/azure/src/config.rs @@ -11,7 +11,7 @@ use std::sync::LazyLock; use object_store::azure::AzureConfigKey; use object_store::Error as ObjectStoreError; -use crate::error::Result; +use deltalake_logstore::LogStoreResult; static CREDENTIAL_KEYS: LazyLock> = LazyLock::new(|| { Vec::from_iter([ @@ -79,7 +79,7 @@ impl AzureConfigHelper { /// Create a new [`ConfigHelper`] pub fn try_new( config: impl IntoIterator, impl Into)>, - ) -> Result { + ) -> LogStoreResult { let mut env_config = HashMap::new(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { @@ -125,7 +125,7 @@ impl AzureConfigHelper { } /// Generate a configuration augmented with options from the environment - pub fn build(mut self) -> Result> { + pub fn build(mut self) -> LogStoreResult> { let mut has_credential = false; if self.config.contains_key(&AzureConfigKey::UseAzureCli) { diff --git a/crates/azure/src/error.rs b/crates/azure/src/error.rs deleted file mode 100644 index acc18f67f9..0000000000 --- a/crates/azure/src/error.rs +++ /dev/null @@ -1,22 +0,0 @@ -use deltalake_core::errors::DeltaTableError; - -pub(crate) type Result = std::result::Result; - -#[derive(thiserror::Error, Debug)] -pub(crate) enum Error { - #[allow(dead_code)] - #[error("failed to parse config: {0}")] - Parse(String), - - #[error(transparent)] - ObjectStore(#[from] object_store::Error), -} - -impl From for DeltaTableError { - fn from(e: Error) -> Self { - match e { - Error::Parse(msg) => DeltaTableError::Generic(msg), - Error::ObjectStore(e) => DeltaTableError::ObjectStore { source: e }, - } - } -} diff --git a/crates/azure/src/lib.rs b/crates/azure/src/lib.rs index 55b21f1358..af43eafb49 100644 --- a/crates/azure/src/lib.rs +++ b/crates/azure/src/lib.rs @@ -2,18 +2,17 @@ use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; -use deltalake_core::logstore::{ +use deltalake_logstore::{ default_logstore, logstore_factories, object_store_factories, LogStore, LogStoreFactory, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; -use deltalake_core::{DeltaResult, DeltaTableError, Path}; +use deltalake_logstore::{LogStoreError, LogStoreResult, Path}; use object_store::azure::{AzureConfigKey, MicrosoftAzureBuilder}; use object_store::client::SpawnedReqwestConnector; use object_store::ObjectStoreScheme; use url::Url; mod config; -pub mod error; trait AzureOptions { fn as_azure_options(&self) -> HashMap; @@ -40,7 +39,7 @@ impl ObjectStoreFactory for AzureFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let mut builder = MicrosoftAzureBuilder::new() .with_url(url.to_string()) .with_retry(config.retry.clone()); @@ -56,11 +55,11 @@ impl ObjectStoreFactory for AzureFactory { } let store = builder.build()?; - let (_, path) = - ObjectStoreScheme::parse(url).map_err(|e| DeltaTableError::GenericError { - source: Box::new(e), - })?; - let prefix = Path::parse(path)?; + let (_, path) = ObjectStoreScheme::parse(url).map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })?; + let prefix = + Path::parse(path).map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; Ok((Arc::new(store), prefix)) } @@ -73,7 +72,7 @@ impl LogStoreFactory for AzureFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { Ok(default_logstore( prefixed_store, root_store, diff --git a/crates/catalog-unity/Cargo.toml b/crates/catalog-unity/Cargo.toml index 23afd3dc46..326a456319 100644 --- a/crates/catalog-unity/Cargo.toml +++ b/crates/catalog-unity/Cargo.toml @@ -20,7 +20,7 @@ thiserror.workspace = true futures.workspace = true chrono.workspace = true tracing.workspace = true -deltalake-core = { version = "0.29.0", path = "../core" } +deltalake-core = { version = "0.29.2", path = "../core" } deltalake-aws = { version = "0.12.0", path = "../aws", optional = true } deltalake-azure = { version = "0.12.0", path = "../azure", optional = true } deltalake-gcp = { version = "0.13.0", path = "../gcp", optional = true } diff --git a/crates/core/Cargo.toml b/crates/core/Cargo.toml index e6da9e42a3..83e3ecc097 100644 --- a/crates/core/Cargo.toml +++ b/crates/core/Cargo.toml @@ -16,6 +16,8 @@ features = ["datafusion", "json"] [dependencies] deltalake-derive = { version = "0.29.0", path = "../derive" } +deltalake-logstore = { version = "0.29.2", path = "../logstore" } +deltalake-types = { version = "0.29.2", path = "../types" } delta_kernel.workspace = true diff --git a/crates/core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs index 7bb7157360..79bfb206d6 100644 --- a/crates/core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -42,6 +42,7 @@ use datafusion::common::{ use datafusion::datasource::physical_plan::wrap_partition_type_in_dict; use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::context::SessionContext; +use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::logical_plan::CreateExternalTable; @@ -114,6 +115,32 @@ impl From for DeltaTableError { } } +pub trait DatafusionLogStore { + /// Generate a unique enough url to identify the store in datafusion. + /// The DF object store registry only cares about the scheme and the host of the url for + /// registering/fetching. In our case the scheme is hard-coded to "delta-rs", so to get a unique + /// host we convert the location from this `LogStore` to a valid name, combining the + /// original scheme, host and path with invalid characters replaced. + fn object_store_url(&self) -> ObjectStoreUrl; +} + +impl DatafusionLogStore for T { + fn object_store_url(&self) -> ObjectStoreUrl { + object_store_url_internal(&self.config().location) + } +} + +pub(crate) fn object_store_url_internal(location: &Url) -> ObjectStoreUrl { + use object_store::path::DELIMITER; + ObjectStoreUrl::parse(format!( + "delta-rs://{}-{}{}", + location.scheme(), + location.host_str().unwrap_or("-"), + location.path().replace(DELIMITER, "-").replace(':', "-") + )) + .expect("Invalid object store url.") +} + /// Convenience trait for calling common methods on snapshot hierarchies pub trait DataFusionMixins { /// The physical datafusion schema of a table diff --git a/crates/core/src/delta_datafusion/table_provider.rs b/crates/core/src/delta_datafusion/table_provider.rs index 7a1ac88781..5d7d08c50a 100644 --- a/crates/core/src/delta_datafusion/table_provider.rs +++ b/crates/core/src/delta_datafusion/table_provider.rs @@ -52,13 +52,14 @@ use serde::{Deserialize, Serialize}; use crate::delta_datafusion::schema_adapter::DeltaSchemaAdapterFactory; use crate::delta_datafusion::{ get_null_of_arrow_type, register_store, to_correct_scalar_value, DataFusionMixins as _, - LogDataHandler, + DatafusionLogStore, LogDataHandler, }; use crate::kernel::schema::cast::cast_record_batch; use crate::kernel::transaction::{CommitBuilder, PROTOCOL}; use crate::kernel::{Action, Add, EagerSnapshot, Remove}; use crate::operations::write::writer::{DeltaWriter, WriterConfig}; use crate::operations::write::WriterStatsConfig; +use crate::protocol::AddStatsExt; use crate::protocol::{DeltaOperation, SaveMode}; use crate::{ensure_table_uri, DeltaTable}; use crate::{logstore::LogStoreRef, DeltaResult, DeltaTableError}; @@ -1145,7 +1146,7 @@ fn partitioned_file_from_action( PartitionedFile { object_meta: ObjectMeta { last_modified, - ..action.try_into().unwrap() + ..action.try_into_object_meta().unwrap() }, partition_values, range: None, diff --git a/crates/core/src/errors.rs b/crates/core/src/errors.rs index c633861c73..669498ec97 100644 --- a/crates/core/src/errors.rs +++ b/crates/core/src/errors.rs @@ -14,6 +14,9 @@ pub enum DeltaTableError { #[error("Kernel error: {0}")] KernelError(#[from] delta_kernel::error::Error), + #[error("LogStore error: {0}")] + LogStore(#[from] crate::logstore::error::LogStoreError), + /// Error returned when reading the delta log object failed. #[error("Failed to read delta log object: {}", .source)] ObjectStore { diff --git a/crates/core/src/kernel/mod.rs b/crates/core/src/kernel/mod.rs index 7fea67efec..32594638b7 100644 --- a/crates/core/src/kernel/mod.rs +++ b/crates/core/src/kernel/mod.rs @@ -18,6 +18,7 @@ pub mod transaction; pub use arrow::engine_ext::StructDataExt; pub use delta_kernel::engine; +pub use deltalake_logstore::get_actions; pub use error::*; pub use models::*; pub use schema::*; diff --git a/crates/core/src/kernel/models/actions.rs b/crates/core/src/kernel/models/actions.rs index 328c211b01..b1f48df120 100644 --- a/crates/core/src/kernel/models/actions.rs +++ b/crates/core/src/kernel/models/actions.rs @@ -827,7 +827,7 @@ pub struct Add { } /// Represents a tombstone (deleted file) in the Delta log. -#[derive(Serialize, Deserialize, Debug, Clone, Eq, Default)] +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, Default)] #[serde(rename_all = "camelCase")] pub struct Remove { /// A relative path to a data file from the root of the table or an absolute path to a file diff --git a/crates/core/src/kernel/models/mod.rs b/crates/core/src/kernel/models/mod.rs index e5bc1db1c5..515ebeb0a8 100644 --- a/crates/core/src/kernel/models/mod.rs +++ b/crates/core/src/kernel/models/mod.rs @@ -3,84 +3,13 @@ //! Delta table in JSON format. The log is a time series of actions that represent all the changes //! made to a table. -use std::collections::HashMap; - -use serde::{Deserialize, Serialize}; - pub(crate) mod actions; pub(crate) mod fields; pub use actions::*; - -#[derive(Debug, PartialEq, Eq, Clone, Deserialize, Serialize)] -#[serde(rename_all = "camelCase")] -#[allow(missing_docs)] -pub enum Action { - #[serde(rename = "metaData")] - Metadata(Metadata), - Protocol(Protocol), - Add(Add), - Remove(Remove), - Cdc(AddCDCFile), - Txn(Transaction), - CommitInfo(CommitInfo), - DomainMetadata(DomainMetadata), -} - -impl Action { - /// Create a commit info from a map - pub fn commit_info(info: HashMap) -> Self { - Self::CommitInfo(CommitInfo { - info, - ..Default::default() - }) - } -} - -impl From for Action { - fn from(a: Add) -> Self { - Self::Add(a) - } -} - -impl From for Action { - fn from(a: Remove) -> Self { - Self::Remove(a) - } -} - -impl From for Action { - fn from(a: AddCDCFile) -> Self { - Self::Cdc(a) - } -} - -impl From for Action { - fn from(a: Metadata) -> Self { - Self::Metadata(a) - } -} - -impl From for Action { - fn from(a: Protocol) -> Self { - Self::Protocol(a) - } -} - -impl From for Action { - fn from(a: Transaction) -> Self { - Self::Txn(a) - } -} - -impl From for Action { - fn from(a: CommitInfo) -> Self { - Self::CommitInfo(a) - } -} - -impl From for Action { - fn from(a: DomainMetadata) -> Self { - Self::DomainMetadata(a) - } -} +// Re-export Action and action types from deltalake-types +pub use delta_kernel::actions::{Metadata, Protocol}; +pub use deltalake_types::{ + Action, Add, AddCDCFile, CheckpointMetadata, CommitInfo, DeletionVectorDescriptor, + DomainMetadata, IsolationLevel, Remove, Sidecar, StorageType, TableFeatures, Transaction, +}; diff --git a/crates/core/src/kernel/schema/partitions.rs b/crates/core/src/kernel/schema/partitions.rs index 7a392b4276..b0d11a28d8 100644 --- a/crates/core/src/kernel/schema/partitions.rs +++ b/crates/core/src/kernel/schema/partitions.rs @@ -184,7 +184,7 @@ impl TryFrom<&str> for DeltaTablePartition { } #[allow(unused)] // TODO: remove once we use this in kernel log replay -pub(crate) fn to_kernel_predicate( +pub fn to_kernel_predicate( filters: &[PartitionFilter], table_schema: &StructType, ) -> DeltaResult { diff --git a/crates/core/src/kernel/schema/schema.rs b/crates/core/src/kernel/schema/schema.rs index 0450124aed..cc586caf6a 100644 --- a/crates/core/src/kernel/schema/schema.rs +++ b/crates/core/src/kernel/schema/schema.rs @@ -10,7 +10,7 @@ pub use delta_kernel::schema::{ use serde_json::Value; use crate::kernel::error::Error; -use crate::schema::DataCheck; +use crate::kernel::schema::DataCheck; use crate::table::GeneratedColumn; /// Type alias for a top level schema diff --git a/crates/core/src/kernel/transaction/conflict_checker.rs b/crates/core/src/kernel/transaction/conflict_checker.rs index 4a4868e92e..cbf4215394 100644 --- a/crates/core/src/kernel/transaction/conflict_checker.rs +++ b/crates/core/src/kernel/transaction/conflict_checker.rs @@ -7,8 +7,9 @@ use super::CommitInfo; #[cfg(feature = "datafusion")] use crate::delta_datafusion::DataFusionMixins; use crate::errors::DeltaResult; +use crate::kernel::get_actions; use crate::kernel::{Action, Add, LogDataHandler, Metadata, Protocol, Remove, Transaction}; -use crate::logstore::{get_actions, LogStore}; +use crate::logstore::LogStore; use crate::protocol::DeltaOperation; use crate::table::config::TablePropertiesExt as _; use crate::DeltaTableError; diff --git a/crates/core/src/kernel/transaction/mod.rs b/crates/core/src/kernel/transaction/mod.rs index ccd011a496..29a4baf54f 100644 --- a/crates/core/src/kernel/transaction/mod.rs +++ b/crates/core/src/kernel/transaction/mod.rs @@ -80,6 +80,7 @@ use bytes::Bytes; use chrono::Utc; use conflict_checker::ConflictChecker; use delta_kernel::table_properties::TableProperties; +use deltalake_logstore::LogStoreError; use futures::future::BoxFuture; use object_store::path::Path; use object_store::Error as ObjectStoreError; @@ -758,7 +759,7 @@ impl<'a> std::future::IntoFuture for PreparedCommit<'a> { }, }); } - Err(TransactionError::VersionAlreadyExists(version)) => { + Err(LogStoreError::VersionAlreadyExists(version)) => { warn!( version = version, attempt = attempt_number, diff --git a/crates/core/src/kernel/transaction/state.rs b/crates/core/src/kernel/transaction/state.rs index f4170baeb7..8d9133ecd4 100644 --- a/crates/core/src/kernel/transaction/state.rs +++ b/crates/core/src/kernel/transaction/state.rs @@ -11,6 +11,7 @@ use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistic use crate::delta_datafusion::{get_null_of_arrow_type, to_correct_scalar_value}; use crate::errors::DeltaResult; use crate::kernel::{Add, EagerSnapshot}; +use crate::protocol::AddStatsExt; use crate::table::state::DeltaTableState; pub struct AddContainer<'a> { diff --git a/crates/core/src/lib.rs b/crates/core/src/lib.rs index e6599dc1d5..81452d1fcd 100644 --- a/crates/core/src/lib.rs +++ b/crates/core/src/lib.rs @@ -81,11 +81,19 @@ pub mod data_catalog; pub mod errors; pub mod kernel; -pub mod logstore; pub mod operations; pub mod protocol; pub use kernel::schema; pub mod table; +pub mod logstore { + pub use deltalake_logstore::*; + pub mod default_logstore { + pub use deltalake_logstore::DefaultLogStore; + } +} + +// Compatibility shim for DeltaConfig derive macro which expects crate::DeltaResult +pub type DeltaResult = crate::errors::DeltaResult; #[cfg(any(test, feature = "integration_test"))] pub mod test_utils; @@ -111,6 +119,7 @@ pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, Object pub use operations::DeltaOps; pub use protocol::checkpoints; +pub use protocol::AddStatsExt; // convenience exports for consumers to avoid aligning crate versions pub use arrow; diff --git a/crates/core/src/operations/convert_to_delta.rs b/crates/core/src/operations/convert_to_delta.rs index fb7a843873..82b033713d 100644 --- a/crates/core/src/operations/convert_to_delta.rs +++ b/crates/core/src/operations/convert_to_delta.rs @@ -107,6 +107,8 @@ enum Error { MissingLocation, #[error("The location provided must be a valid URL")] InvalidLocation(#[from] url::ParseError), + #[error("LogStore error: {0}")] + LogStore(#[from] crate::logstore::error::LogStoreError), } impl From for DeltaTableError { diff --git a/crates/core/src/operations/load_cdf.rs b/crates/core/src/operations/load_cdf.rs index 4ae0dea874..3183a9fc6b 100644 --- a/crates/core/src/operations/load_cdf.rs +++ b/crates/core/src/operations/load_cdf.rs @@ -28,13 +28,15 @@ use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::union::UnionExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; +use deltalake_logstore::LogStoreError; use tracing::log; -use crate::delta_datafusion::{register_store, DataFusionMixins}; +use crate::delta_datafusion::{register_store, DataFusionMixins, DatafusionLogStore}; use crate::errors::DeltaResult; +use crate::kernel::get_actions; use crate::kernel::transaction::PROTOCOL; use crate::kernel::{resolve_snapshot, Action, Add, AddCDCFile, CommitInfo, EagerSnapshot}; -use crate::logstore::{get_actions, LogStoreRef}; +use crate::logstore::LogStoreRef; use crate::DeltaTableError; use crate::{delta_datafusion::cdf::*, kernel::Remove}; @@ -170,10 +172,10 @@ impl CdfLoadBuilder { // Start from 0 since if start > latest commit, the returned commit is not a valid commit let latest_version = match self.log_store.get_latest_version(start).await { Ok(latest_version) => latest_version, - Err(DeltaTableError::InvalidVersion(_)) if self.allow_out_of_range => { + Err(LogStoreError::InvalidVersion(_)) if self.allow_out_of_range => { return Ok((change_files, add_files, remove_files)); } - Err(e) => return Err(e), + Err(e) => return Err(e.into()), }; let mut end = self.ending_version.unwrap_or(latest_version); diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index 2674b32ca4..38dfbebffb 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -2006,7 +2006,7 @@ mod tests { .await .unwrap() .expect("failed to get snapshot bytes"); - let actions = crate::logstore::get_actions(2, &snapshot_bytes).unwrap(); + let actions = crate::kernel::get_actions(2, &snapshot_bytes).unwrap(); let schema_actions = actions .iter() @@ -2080,7 +2080,7 @@ mod tests { .await .unwrap() .expect("failed to get snapshot bytes"); - let actions = crate::logstore::get_actions(2, &snapshot_bytes).unwrap(); + let actions = crate::kernel::get_actions(2, &snapshot_bytes).unwrap(); let schema_actions = actions .iter() @@ -2189,7 +2189,7 @@ mod tests { .await .unwrap() .expect("failed to get snapshot bytes"); - let actions = crate::logstore::get_actions(2, &snapshot_bytes).unwrap(); + let actions = crate::kernel::get_actions(2, &snapshot_bytes).unwrap(); let schema_actions = actions .iter() diff --git a/crates/core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs index c57bd55905..e511820cda 100644 --- a/crates/core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -61,7 +61,7 @@ use crate::protocol::DeltaOperation; use crate::table::config::TablePropertiesExt as _; use crate::table::state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; -use crate::{crate_version, DeltaTable, ObjectMeta, PartitionFilter}; +use crate::{crate_version, AddStatsExt, DeltaTable, ObjectMeta, PartitionFilter}; /// Metrics from Optimize #[derive(Default, Debug, PartialEq, Clone, Serialize, Deserialize)] @@ -664,7 +664,7 @@ impl MergePlan { let batch_stream = futures::stream::iter(files.clone()) .then(move |file| { let object_store_ref = object_store_ref.clone(); - let meta = ObjectMeta::try_from(file).unwrap(); + let meta = file.try_into_object_meta().unwrap(); async move { let file_reader = ParquetObjectReader::new(object_store_ref, meta.location) diff --git a/crates/core/src/operations/restore.rs b/crates/core/src/operations/restore.rs index c235304a03..ca04a418fe 100644 --- a/crates/core/src/operations/restore.rs +++ b/crates/core/src/operations/restore.rs @@ -27,6 +27,7 @@ use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; use chrono::{DateTime, Utc}; +use deltalake_logstore::LogStoreError; use futures::future::BoxFuture; use futures::TryStreamExt; use object_store::path::Path; @@ -307,8 +308,8 @@ async fn execute( .await { Ok(_) => {} - Err(err @ TransactionError::VersionAlreadyExists(_)) => { - return Err(err.into()); + Err(LogStoreError::VersionAlreadyExists(version)) => { + return Err(DeltaTableError::VersionAlreadyExists(version)); } Err(err) => { log_store diff --git a/crates/core/src/operations/write/mod.rs b/crates/core/src/operations/write/mod.rs index e652b2272d..c578d48b73 100644 --- a/crates/core/src/operations/write/mod.rs +++ b/crates/core/src/operations/write/mod.rs @@ -757,8 +757,8 @@ impl std::future::IntoFuture for WriteBuilder { mod tests { use super::*; use crate::ensure_table_uri; + use crate::kernel::get_actions; use crate::kernel::CommitInfo; - use crate::logstore::get_actions; use crate::operations::load_cdf::collect_batches; use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::protocol::SaveMode; diff --git a/crates/core/src/protocol/mod.rs b/crates/core/src/protocol/mod.rs index fb3dcfdff4..94b3e8338e 100644 --- a/crates/core/src/protocol/mod.rs +++ b/crates/core/src/protocol/mod.rs @@ -3,18 +3,19 @@ #![allow(non_camel_case_types)] use crate::table::Constraint; -use std::borrow::Borrow; use std::collections::HashMap; -use std::hash::{Hash, Hasher}; use std::mem::take; use std::str::FromStr; +use chrono::DateTime; +use object_store::path::Path; +use object_store::ObjectMeta; use serde::{Deserialize, Serialize}; use serde_json::Value; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Add, CommitInfo, Metadata, Protocol, Remove, StructField, TableFeatures}; +use crate::kernel::{Add, CommitInfo, Metadata, Protocol, StructField, TableFeatures}; pub mod checkpoints; @@ -140,67 +141,46 @@ pub struct StatsParsed { pub null_count: HashMap, } -impl Hash for Add { - fn hash(&self, state: &mut H) { - self.path.hash(state); - } -} +/// Extension trait for Add actions to provide stats functionality +pub trait AddStatsExt { + /// Get whatever stats are available. Uses (parquet struct) parsed_stats if present falling back to json stats. + fn get_stats(&self) -> Result, serde_json::error::Error>; -impl PartialEq for Add { - fn eq(&self, other: &Self) -> bool { - self.path == other.path - && self.size == other.size - && self.partition_values == other.partition_values - && self.modification_time == other.modification_time - && self.data_change == other.data_change - && self.stats == other.stats - && self.tags == other.tags - && self.deletion_vector == other.deletion_vector - } -} + /// Returns the serde_json representation of stats contained in the action if present. + /// Since stats are defined as optional in the protocol, this may be None. + fn get_json_stats(&self) -> Result, serde_json::error::Error>; -impl Eq for Add {} + fn try_into_object_meta(&self) -> Result; +} -impl Add { - /// Get whatever stats are available. Uses (parquet struct) parsed_stats if present falling back to json stats. - pub fn get_stats(&self) -> Result, serde_json::error::Error> { +impl AddStatsExt for Add { + fn get_stats(&self) -> Result, serde_json::error::Error> { self.get_json_stats() } - /// Returns the serde_json representation of stats contained in the action if present. - /// Since stats are defined as optional in the protocol, this may be None. fn get_json_stats(&self) -> Result, serde_json::error::Error> { self.stats .as_ref() .map(|stats| serde_json::from_str(stats).map(|mut ps: PartialStats| ps.as_stats())) .transpose() } -} - -impl Hash for Remove { - fn hash(&self, state: &mut H) { - self.path.hash(state); - } -} - -/// Borrow `Remove` as str so we can look at tombstones hashset in `DeltaTableState` by using -/// a path from action `Add`. -impl Borrow for Remove { - fn borrow(&self) -> &str { - self.path.as_ref() - } -} -impl PartialEq for Remove { - fn eq(&self, other: &Self) -> bool { - self.path == other.path - && self.deletion_timestamp == other.deletion_timestamp - && self.data_change == other.data_change - && self.extended_file_metadata == other.extended_file_metadata - && self.partition_values == other.partition_values - && self.size == other.size - && self.tags == other.tags - && self.deletion_vector == other.deletion_vector + fn try_into_object_meta(&self) -> Result { + let last_modified = DateTime::from_timestamp_millis(self.modification_time).ok_or( + DeltaTableError::MetadataError(format!( + "invalid modification_time: {:?}", + self.modification_time + )), + )?; + + Ok(ObjectMeta { + // TODO this won't work for absolute paths, since Paths are always relative to store. + location: Path::parse(self.path.as_str())?, + last_modified, + size: self.size as u64, + e_tag: None, + version: None, + }) } } diff --git a/crates/core/src/table/builder.rs b/crates/core/src/table/builder.rs index 1a4c8940b7..2e2ffc3f05 100644 --- a/crates/core/src/table/builder.rs +++ b/crates/core/src/table/builder.rs @@ -11,7 +11,7 @@ use serde::{Deserialize, Serialize}; use tracing::debug; use url::Url; -use crate::logstore::storage::IORuntime; +use crate::logstore::IORuntime; use crate::logstore::{object_store_factories, LogStoreRef, StorageConfig}; use crate::{DeltaResult, DeltaTable, DeltaTableError}; @@ -248,11 +248,13 @@ impl DeltaTableBuilder { if let Some((store, _url)) = self.storage_backend.as_ref() { debug!("Loading a logstore with a custom store: {store:?}"); - crate::logstore::logstore_with(store.clone(), location, storage_config) + let result = crate::logstore::logstore_with(store.clone(), location, storage_config)?; + Ok(result) } else { // If there has been no backend defined just default to the normal logstore look up debug!("Loading a logstore based off the location: {location:?}"); - crate::logstore::logstore_for(location, storage_config) + let result = crate::logstore::logstore_for(location, storage_config)?; + Ok(result) } } diff --git a/crates/core/src/table/mod.rs b/crates/core/src/table/mod.rs index 82ab33f53f..3c170e84a7 100644 --- a/crates/core/src/table/mod.rs +++ b/crates/core/src/table/mod.rs @@ -14,12 +14,12 @@ use serde::{Deserialize, Deserializer, Serialize, Serializer}; use self::builder::DeltaTableConfig; use self::state::DeltaTableState; +use crate::kernel::schema::partitions::PartitionFilter; use crate::kernel::{CommitInfo, DataCheck, LogicalFileView}; use crate::logstore::{ commit_uri_from_version, extract_version_from_filename, LogStoreConfig, LogStoreExt, LogStoreRef, ObjectStoreRef, }; -use crate::partitions::PartitionFilter; use crate::{DeltaResult, DeltaTableError}; // NOTE: this use can go away when peek_next_commit is removed off of [DeltaTable] @@ -141,7 +141,10 @@ impl DeltaTable { /// Check if the [`DeltaTable`] exists pub async fn verify_deltatable_existence(&self) -> DeltaResult { - self.log_store.is_delta_table_location().await + self.log_store + .is_delta_table_location() + .await + .map_err(Into::into) } /// The URI of the underlying data @@ -159,6 +162,7 @@ impl DeltaTable { self.log_store .get_latest_version(self.version().unwrap_or(0)) .await + .map_err(Into::into) } /// Currently loaded version of the table - if any. @@ -365,15 +369,9 @@ impl DeltaTable { let mut max_version = match self .log_store .get_latest_version(self.version().unwrap_or(min_version)) - .await + .await? { - Ok(version) => version, - Err(DeltaTableError::InvalidVersion(_)) => { - return Err(DeltaTableError::NotATable( - log_store.table_root_url().to_string(), - )) - } - Err(e) => return Err(e), + version => version, }; let mut version = min_version; let lowest_table_version = min_version; diff --git a/crates/core/src/writer/stats.rs b/crates/core/src/writer/stats.rs index 34ffdcdc46..557dc184fb 100644 --- a/crates/core/src/writer/stats.rs +++ b/crates/core/src/writer/stats.rs @@ -21,6 +21,7 @@ use tracing::warn; use super::*; use crate::kernel::{scalars::ScalarExt, Add}; +use crate::protocol::AddStatsExt; use crate::protocol::{ColumnValueStat, Stats}; /// Creates an [`Add`] log action struct. diff --git a/crates/derive/src/lib.rs b/crates/derive/src/lib.rs index c4d4afac4d..5303a9199f 100644 --- a/crates/derive/src/lib.rs +++ b/crates/derive/src/lib.rs @@ -165,7 +165,7 @@ fn generate_try_update_key(name: &Ident, fields: &[&Field]) -> Result crate::DeltaResult> { + fn try_update_key(&mut self, key: &str, v: &str) -> crate::logstore::error::LogStoreResult> { match key { #(#match_arms)* _ => return Ok(None), @@ -173,7 +173,7 @@ fn generate_try_update_key(name: &Ident, fields: &[&Field]) -> Result crate::DeltaResult<()> { + fn load_from_environment(&mut self) -> crate::logstore::error::LogStoreResult<()> { let default_values = Self::default(); #(#env_setters)* Ok(()) diff --git a/crates/gcp/Cargo.toml b/crates/gcp/Cargo.toml index 975faa71fb..1f1da670e3 100644 --- a/crates/gcp/Cargo.toml +++ b/crates/gcp/Cargo.toml @@ -12,7 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.29.0", path = "../core", features = ["cloud"] } +deltalake-logstore = { version = "0.29.0", path = "../logstore", features = ["cloud"] } # workspace depenndecies async-trait = { workspace = true } diff --git a/crates/gcp/src/lib.rs b/crates/gcp/src/lib.rs index bf18ec8378..ac4052ba84 100644 --- a/crates/gcp/src/lib.rs +++ b/crates/gcp/src/lib.rs @@ -2,13 +2,13 @@ use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; -use deltalake_core::logstore::object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; -use deltalake_core::logstore::object_store::ObjectStoreScheme; -use deltalake_core::logstore::{default_logstore, logstore_factories, LogStore, LogStoreFactory}; -use deltalake_core::logstore::{ +use deltalake_logstore::object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; +use deltalake_logstore::object_store::ObjectStoreScheme; +use deltalake_logstore::{default_logstore, logstore_factories, LogStore, LogStoreFactory}; +use deltalake_logstore::{ object_store_factories, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; -use deltalake_core::{DeltaResult, DeltaTableError, Path}; +use deltalake_logstore::{LogStoreError, LogStoreResult, Path}; use object_store::client::SpawnedReqwestConnector; use url::Url; @@ -41,7 +41,7 @@ impl ObjectStoreFactory for GcpFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let mut builder = GoogleCloudStorageBuilder::new().with_url(url.to_string()); builder = builder.with_retry(config.retry.clone()); @@ -74,7 +74,7 @@ impl LogStoreFactory for GcpFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { Ok(default_logstore( prefixed_store, root_store, diff --git a/crates/gcp/src/storage.rs b/crates/gcp/src/storage.rs index 784b5cf529..cc845b65fb 100644 --- a/crates/gcp/src/storage.rs +++ b/crates/gcp/src/storage.rs @@ -7,7 +7,7 @@ use futures::stream::BoxStream; use object_store::{MultipartUpload, PutMultipartOptions, PutPayload}; use std::ops::Range; -use deltalake_core::logstore::object_store::{ +use deltalake_logstore::object_store::{ GetOptions, GetResult, ListResult, ObjectMeta, ObjectStore, PutOptions, PutResult, Result as ObjectStoreResult, }; diff --git a/crates/hdfs/Cargo.toml b/crates/hdfs/Cargo.toml index 7b1efcc9fc..f239475feb 100644 --- a/crates/hdfs/Cargo.toml +++ b/crates/hdfs/Cargo.toml @@ -12,7 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.29.0", path = "../core"} +deltalake-logstore = { version = "0.29.0", path = "../logstore"} hdfs-native-object-store = "0.15" # workspace dependencies diff --git a/crates/hdfs/src/lib.rs b/crates/hdfs/src/lib.rs index 3eaa8a844d..889ed89f11 100644 --- a/crates/hdfs/src/lib.rs +++ b/crates/hdfs/src/lib.rs @@ -1,10 +1,10 @@ use std::sync::Arc; -use deltalake_core::logstore::{ +use deltalake_logstore::{ default_logstore, logstore_factories, LogStore, LogStoreFactory, StorageConfig, }; -use deltalake_core::logstore::{object_store_factories, ObjectStoreFactory, ObjectStoreRef}; -use deltalake_core::{DeltaResult, Path}; +use deltalake_logstore::{object_store_factories, ObjectStoreFactory, ObjectStoreRef}; +use deltalake_logstore::{LogStoreError, LogStoreResult, Path}; use hdfs_native_object_store::HdfsObjectStoreBuilder; use url::Url; @@ -16,7 +16,7 @@ impl ObjectStoreFactory for HdfsFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let mut builder = HdfsObjectStoreBuilder::new() .with_url(url.as_str()) .with_config(&config.raw); @@ -39,7 +39,7 @@ impl LogStoreFactory for HdfsFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { Ok(default_logstore( prefixed_store, root_store, @@ -64,7 +64,7 @@ mod tests { use super::*; #[test] - fn test_parse_url_opts() -> DeltaResult<()> { + fn test_parse_url_opts() -> LogStoreResult<()> { let factory = HdfsFactory::default(); let _ = factory.parse_url_opts( &Url::parse("hdfs://localhost:9000").expect("Failed to parse hdfs://"), diff --git a/crates/logstore/Cargo.toml b/crates/logstore/Cargo.toml new file mode 100644 index 0000000000..c53bcd1b5e --- /dev/null +++ b/crates/logstore/Cargo.toml @@ -0,0 +1,62 @@ +[package] +name = "deltalake-logstore" +version = "0.29.2" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description = "Delta Lake logstore abstraction" +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +deltalake-derive = { version = "0.29.0", path = "../derive" } +deltalake-types = { version = "0.29.2", path = "../types" } + +delta_kernel.workspace = true + +# arrow +arrow = { workspace = true } +bytes = { workspace = true } +chrono = { workspace = true, default-features = false, features = ["clock"] } + +# serde +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true } + +# runtime +async-trait = { workspace = true } +futures = { workspace = true } +tokio = { workspace = true, features = [ + "macros", + "rt", + "rt-multi-thread", + "sync", +] } + +# other deps +dashmap = "6" +object_store = { workspace = true } +regex = { workspace = true } +thiserror = { workspace = true } +tracing = { workspace = true } +url = { workspace = true, features = ["serde"] } +uuid = { workspace = true, features = ["serde", "v4"] } +humantime = { version = "2.1.0", optional = true } + +[dev-dependencies] +deltalake-test = { path = "../test" } +tokio = { version = "1", features = ["macros", "rt-multi-thread"] } +tracing-subscriber = { version = "0.3", features = ["env-filter"] } + +[features] +default = ["rustls"] +rustls = ["delta_kernel/default-engine-rustls"] +native-tls = ["delta_kernel/default-engine-native-tls"] +cloud = ["object_store/cloud", "dep:humantime"] + +[package.metadata.docs.rs] +features = ["cloud"] + diff --git a/crates/core/src/logstore/config.rs b/crates/logstore/src/config.rs similarity index 87% rename from crates/core/src/logstore/config.rs rename to crates/logstore/src/config.rs index 9969599a91..f0ff6ca12b 100644 --- a/crates/core/src/logstore/config.rs +++ b/crates/logstore/src/config.rs @@ -13,7 +13,7 @@ use std::collections::HashMap; use super::storage::LimitConfig; use super::{storage::runtime::RuntimeConfig, IORuntime}; -use crate::{DeltaResult, DeltaTableError}; +use crate::error::{LogStoreError, LogStoreResult}; pub trait TryUpdateKey: Default { /// Update an internal field in the configuration. @@ -23,14 +23,14 @@ pub trait TryUpdateKey: Default { /// - `Ok(None)` if the key was not found and no internal field was updated. /// - `Err(_)` if the update failed. Failed updates may include finding a known key, /// but failing to parse the value into the expected type. - fn try_update_key(&mut self, key: &str, value: &str) -> DeltaResult>; + fn try_update_key(&mut self, key: &str, value: &str) -> LogStoreResult>; /// Load configuration values from environment variables /// /// For Option fields, this will only set values that are None /// For non-optional fields, environment variables will update the /// value if the current value corresponds to the default value. - fn load_from_environment(&mut self) -> DeltaResult<()>; + fn load_from_environment(&mut self) -> LogStoreResult<()>; } #[derive(Debug)] @@ -47,12 +47,13 @@ pub struct ParseResult { } impl ParseResult { - pub fn raise_errors(&self) -> DeltaResult<()> { + pub fn raise_errors(&self) -> LogStoreResult<()> { if !self.errors.is_empty() { - return Err(DeltaTableError::Generic(format!( - "Failed to parse config: {:?}", - self.errors - ))); + todo!() + // let error: Box = format!("Failed to parse config: {:?}", self.errors).into(); + // return Err(LogStoreError::Generic { + // source: Box::new(error), + // }); } Ok(()) } @@ -125,7 +126,7 @@ impl StorageConfig { &self, store: T, table_root: &url::Url, - ) -> DeltaResult> { + ) -> LogStoreResult> { let inner = Self::decorate_prefix(store, table_root)?; Ok(inner) } @@ -133,8 +134,8 @@ impl StorageConfig { pub(crate) fn decorate_prefix( store: T, table_root: &url::Url, - ) -> DeltaResult> { - let prefix = super::object_store_path(table_root)?; + ) -> LogStoreResult> { + let prefix = crate::object_store_path(table_root)?; Ok(if prefix != Path::from("/") { Box::new(PrefixStore::new(store, prefix)) as Box } else { @@ -193,7 +194,7 @@ impl StorageConfig { /// # Raises /// /// Raises a `DeltaError` if any of the options are invalid - i.e. cannot be parsed into target type. - pub fn parse_options(options: I) -> DeltaResult + pub fn parse_options(options: I) -> LogStoreResult where I: IntoIterator, K: AsRef + Into, @@ -236,43 +237,50 @@ impl StorageConfig { } } -pub(super) fn try_parse_impl( - options: I, -) -> DeltaResult<(T, HashMap)> +pub(super) fn try_parse_impl(options: I) -> LogStoreResult<(T, HashMap)> where I: IntoIterator, K: AsRef + Into, V: AsRef + Into, - T: TryUpdateKey, + T: TryUpdateKey + std::fmt::Debug, { let result = ParseResult::from_iter(options); result.raise_errors()?; Ok((result.config, result.unparsed)) } -pub fn parse_usize(value: &str) -> DeltaResult { +pub fn parse_usize(value: &str) -> LogStoreResult { value .parse::() - .map_err(|_| DeltaTableError::Generic(format!("failed to parse \"{value}\" as usize"))) + .map_err(|e| LogStoreError::ParseError { + value: value.to_string(), + type_name: "usize".to_string(), + source: Box::new(e), + }) } -pub fn parse_f64(value: &str) -> DeltaResult { - value - .parse::() - .map_err(|_| DeltaTableError::Generic(format!("failed to parse \"{value}\" as f64"))) +pub fn parse_f64(value: &str) -> LogStoreResult { + value.parse::().map_err(|e| LogStoreError::ParseError { + value: value.to_string(), + type_name: "f64".to_string(), + source: Box::new(e), + }) } #[cfg(feature = "cloud")] -pub fn parse_duration(value: &str) -> DeltaResult { - humantime::parse_duration(value) - .map_err(|_| DeltaTableError::Generic(format!("failed to parse \"{value}\" as Duration"))) +pub fn parse_duration(value: &str) -> LogStoreResult { + humantime::parse_duration(value).map_err(|e| LogStoreError::ParseError { + value: value.to_string(), + type_name: "Duration".to_string(), + source: Box::new(e), + }) } -pub fn parse_bool(value: &str) -> DeltaResult { +pub fn parse_bool(value: &str) -> LogStoreResult { Ok(str_is_truthy(value)) } -pub fn parse_string(value: &str) -> DeltaResult { +pub fn parse_string(value: &str) -> LogStoreResult { Ok(value.to_string()) } diff --git a/crates/core/src/logstore/default_logstore.rs b/crates/logstore/src/default_logstore.rs similarity index 87% rename from crates/core/src/logstore/default_logstore.rs rename to crates/logstore/src/default_logstore.rs index ed9d38fae2..a7d2d91677 100644 --- a/crates/core/src/logstore/default_logstore.rs +++ b/crates/logstore/src/default_logstore.rs @@ -8,8 +8,7 @@ use uuid::Uuid; use super::storage::{utils::commit_uri_from_version, ObjectStoreRef}; use super::{CommitOrBytes, LogStore, LogStoreConfig}; -use crate::kernel::transaction::TransactionError; -use crate::DeltaResult; +use crate::error::{LogStoreError, LogStoreResult}; fn put_options() -> &'static PutOptions { static PUT_OPTS: OnceLock = OnceLock::new(); @@ -58,11 +57,11 @@ impl LogStore for DefaultLogStore { "DefaultLogStore".into() } - async fn read_commit_entry(&self, version: i64) -> DeltaResult> { + async fn read_commit_entry(&self, version: i64) -> LogStoreResult> { super::read_commit_entry(self.object_store(None).as_ref(), version).await } - /// Tries to commit a prepared commit file. Returns [`TransactionError`] + /// Tries to commit a prepared commit file. Returns [`LogStoreError`] /// 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` @@ -72,7 +71,7 @@ impl LogStore for DefaultLogStore { version: i64, commit_or_bytes: CommitOrBytes, _: Uuid, - ) -> Result<(), TransactionError> { + ) -> Result<(), LogStoreError> { match commit_or_bytes { CommitOrBytes::LogBytes(log_bytes) => self .object_store(None) @@ -82,12 +81,12 @@ impl LogStore for DefaultLogStore { put_options().clone(), ) .await - .map_err(|err| -> TransactionError { + .map_err(|err| -> LogStoreError { match err { ObjectStoreError::AlreadyExists { .. } => { - TransactionError::VersionAlreadyExists(version) + LogStoreError::VersionAlreadyExists(version) } - _ => TransactionError::from(err), + _ => LogStoreError::from(err), } })?, _ => unreachable!(), // Default log store should never get a tmp_commit, since this is for conditional put stores @@ -100,14 +99,14 @@ impl LogStore for DefaultLogStore { _version: i64, commit_or_bytes: CommitOrBytes, _: Uuid, - ) -> Result<(), TransactionError> { + ) -> Result<(), LogStoreError> { match &commit_or_bytes { CommitOrBytes::LogBytes(_) => Ok(()), _ => unreachable!(), // Default log store should never get a tmp_commit, since this is for conditional put stores } } - async fn get_latest_version(&self, current_version: i64) -> DeltaResult { + async fn get_latest_version(&self, current_version: i64) -> LogStoreResult { super::get_latest_version(self, current_version).await } diff --git a/crates/logstore/src/error.rs b/crates/logstore/src/error.rs new file mode 100644 index 0000000000..b1425b3355 --- /dev/null +++ b/crates/logstore/src/error.rs @@ -0,0 +1,85 @@ +//! Error types for logstore operations + +use object_store::Error as ObjectStoreError; +use thiserror::Error; + +/// Error raised during logstore operations +#[derive(Error, Debug)] +pub enum LogStoreError { + /// Version already exists + #[error("Tried committing existing table version: {0}")] + VersionAlreadyExists(i64), + + /// Error returned when reading the delta log object failed. + #[error("Log storage error: {}", .source)] + ObjectStore { + /// Storage error details when reading the delta log object failed. + #[from] + source: ObjectStoreError, + }, + + /// The transaction failed to commit due to an error in an implementation-specific layer. + #[error("Transaction failed: {msg}")] + LogStoreError { + /// Detailed message for the commit failure. + msg: String, + /// underlying error in the log store transactional layer. + source: Box, + }, + + /// Invalid table location + #[error("Invalid table location: {0}")] + InvalidTableLocation(String), + + /// Error returned when the log record has an invalid JSON. + #[error("Invalid JSON in log record, version={}, line=`{}`, err=`{}`", .version, .line, .json_err)] + InvalidJsonLog { + /// JSON error details returned when parsing the record JSON. + json_err: serde_json::error::Error, + /// invalid log entry content. + line: String, + /// corresponding table version for the log file. + version: i64, + }, + + /// Error returned when parsing a path failed. + #[error("Failed to parse path: {path} - {source}")] + InvalidPath { + /// The path that failed to parse. + path: String, + /// Source error details. + source: Box, + }, + + /// Error returned when the DeltaTable has an invalid version. + #[error("Invalid table version: {0}")] + InvalidVersion(i64), + + /// Error returned when parsing a configuration value failed. + #[error("Failed to parse \"{value}\" as {type_name}: {source}")] + ParseError { + /// The value that failed to parse. + value: String, + /// The type name that was expected. + type_name: String, + /// Source error details. + source: Box, + }, + + /// Error returned when an object store is not found for a URL. + #[error("No suitable object store found for '{url}'")] + ObjectStoreNotFound { + /// The URL for which no object store was found. + url: String, + }, + + /// Generic error + #[error("Logstore error: {source}")] + Generic { + /// Source error + source: Box, + }, +} + +/// Result type for logstore operations +pub type LogStoreResult = Result; diff --git a/crates/core/src/logstore/factories.rs b/crates/logstore/src/factories.rs similarity index 89% rename from crates/core/src/logstore/factories.rs rename to crates/logstore/src/factories.rs index 6a000a0f2b..2eb33fd0f3 100644 --- a/crates/core/src/logstore/factories.rs +++ b/crates/logstore/src/factories.rs @@ -8,7 +8,7 @@ use object_store::{path::Path, DynObjectStore}; use url::Url; use super::{default_logstore, DeltaIOStorageBackend, LogStore, ObjectStoreRef, StorageConfig}; -use crate::{DeltaResult, DeltaTableError}; +use crate::error::{LogStoreError, LogStoreResult}; /// Factory registry to manage [`ObjectStoreFactory`] instances pub type ObjectStoreFactoryRegistry = Arc>>; @@ -27,7 +27,7 @@ pub trait ObjectStoreFactory: Send + Sync { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)>; + ) -> LogStoreResult<(ObjectStoreRef, Path)>; } #[derive(Clone, Debug, Default)] @@ -38,7 +38,7 @@ impl ObjectStoreFactory for DefaultObjectStoreFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let (mut store, path) = default_parse_url_opts(url, &config.raw)?; if let Some(runtime) = &config.runtime { @@ -52,14 +52,14 @@ impl ObjectStoreFactory for DefaultObjectStoreFactory { fn default_parse_url_opts( url: &Url, options: &HashMap, -) -> DeltaResult<(ObjectStoreRef, Path)> { +) -> LogStoreResult<(ObjectStoreRef, Path)> { match url.scheme() { "memory" | "file" => { let (store, path) = object_store::parse_url_opts(url, options)?; tracing::debug!("building store with:\n\tParsed URL: {url}\n\tPath in store: {path}"); Ok((Arc::new(store), path)) } - _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), + _ => Err(LogStoreError::InvalidTableLocation(url.to_string())), } } @@ -78,7 +78,7 @@ pub fn object_store_factories() -> ObjectStoreFactoryRegistry { } /// Simpler access pattern for the [ObjectStoreFactoryRegistry] to get a single store -pub fn store_for(url: &Url, options: I) -> DeltaResult +pub fn store_for(url: &Url, options: I) -> LogStoreResult where I: IntoIterator, K: AsRef + Into, @@ -91,7 +91,7 @@ where let store = storage_config.decorate_store(store, url)?; Ok(Arc::new(store)) } else { - Err(DeltaTableError::InvalidTableLocation(url.clone().into())) + Err(LogStoreError::InvalidTableLocation(url.to_string())) } } @@ -109,14 +109,14 @@ pub trait LogStoreFactory: Send + Sync { /// - `location`: A reference to the URL of the location. /// - `options`: A reference to the storage configuration options. /// - /// It returns a [DeltaResult] containing an [Arc] to the newly created [LogStore] implementation. + /// It returns a [LogStoreResult] containing an [Arc] to the newly created [LogStore] implementation. fn with_options( &self, prefixed_store: ObjectStoreRef, root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult>; + ) -> LogStoreResult>; } #[derive(Clone, Debug, Default)] @@ -129,7 +129,7 @@ impl LogStoreFactory for DefaultLogStoreFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { Ok(default_logstore( prefixed_store, root_store, diff --git a/crates/logstore/src/lib.rs b/crates/logstore/src/lib.rs new file mode 100644 index 0000000000..637707cf59 --- /dev/null +++ b/crates/logstore/src/lib.rs @@ -0,0 +1,41 @@ +//! # DeltaLake storage system +//! +//! Interacting with storage systems is a crucial part of any table format. +//! On one had the storage abstractions need to provide certain guarantees +//! (e.g. atomic rename, ...) and meet certain assumptions (e.g. sorted list results) +//! on the other hand can we exploit our knowledge about the general file layout +//! and access patterns to optimize our operations in terms of cost and performance. + +pub mod config; +pub(crate) mod default_logstore; +pub mod error; +pub(crate) mod factories; +mod logstore_impl; +pub(crate) mod storage; + +// Compatibility shim for DeltaConfig derive macro which expects crate::logstore::config +pub mod logstore { + pub mod config { + pub use crate::config::*; + } + pub mod error { + pub use crate::error::LogStoreResult; + } +} + +pub use self::config::StorageConfig; +pub use self::default_logstore::DefaultLogStore; +pub use self::error::{LogStoreError, LogStoreResult}; +pub use self::factories::{ + logstore_factories, object_store_factories, store_for, LogStoreFactory, + LogStoreFactoryRegistry, ObjectStoreFactory, ObjectStoreFactoryRegistry, +}; +pub use self::storage::utils::commit_uri_from_version; +pub use self::storage::{ + DefaultObjectStoreRegistry, DeltaIOStorageBackend, IORuntime, ObjectStoreRef, + ObjectStoreRegistry, ObjectStoreRetryExt, +}; +pub use ::object_store; +pub use logstore_impl::{DELTA_LOG_PATH, DELTA_LOG_REGEX, *}; +pub use object_store::path::Path; +pub use object_store::Error as ObjectStoreError; diff --git a/crates/core/src/logstore/mod.rs b/crates/logstore/src/logstore_impl.rs similarity index 84% rename from crates/core/src/logstore/mod.rs rename to crates/logstore/src/logstore_impl.rs index 08b65e8419..9d8920fb78 100644 --- a/crates/core/src/logstore/mod.rs +++ b/crates/logstore/src/logstore_impl.rs @@ -51,8 +51,6 @@ use std::collections::HashMap; use std::sync::{Arc, LazyLock}; use bytes::Bytes; -#[cfg(feature = "datafusion")] -use datafusion::datasource::object_store::ObjectStoreUrl; use delta_kernel::engine::default::executor::tokio::{ TokioBackgroundExecutor, TokioMultiThreadExecutor, }; @@ -60,6 +58,7 @@ use delta_kernel::engine::default::DefaultEngine; use delta_kernel::log_segment::LogSegment; use delta_kernel::path::{LogPathFileType, ParsedLogPath}; use delta_kernel::{AsAny, Engine}; +use deltalake_types::Action; use futures::StreamExt; use object_store::ObjectStoreScheme; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; @@ -73,27 +72,40 @@ use tracing::*; use url::Url; use uuid::Uuid; -use crate::kernel::transaction::TransactionError; -use crate::kernel::{spawn_blocking_with_span, Action}; -use crate::{DeltaResult, DeltaTableError}; +use tokio::task::spawn_blocking; +use tracing::dispatcher; +use tracing::Span; -pub use self::config::StorageConfig; -pub use self::factories::{ +use crate::config::StorageConfig; +use crate::default_logstore::DefaultLogStore; +use crate::error::{LogStoreError, LogStoreResult}; +use crate::factories::{ logstore_factories, object_store_factories, store_for, LogStoreFactory, LogStoreFactoryRegistry, ObjectStoreFactory, ObjectStoreFactoryRegistry, }; -pub use self::storage::utils::commit_uri_from_version; -pub use self::storage::{ +use crate::storage::utils::commit_uri_from_version; +use crate::storage::{ DefaultObjectStoreRegistry, DeltaIOStorageBackend, IORuntime, ObjectStoreRef, ObjectStoreRegistry, ObjectStoreRetryExt, }; -/// Convenience re-export of the object store crate -pub use ::object_store; -pub mod config; -pub(crate) mod default_logstore; -pub(crate) mod factories; -pub(crate) mod storage; +/// Helper function to spawn blocking tasks with tracing span context +pub(crate) fn spawn_blocking_with_span(f: F) -> tokio::task::JoinHandle +where + F: FnOnce() -> R + Send + 'static, + R: Send + 'static, +{ + // Capture the current dispatcher and span + let dispatch = dispatcher::get_default(|d| d.clone()); + let span = Span::current(); + + spawn_blocking(move || { + dispatcher::with_default(&dispatch, || { + let _enter = span.enter(); + f() + }) + }) +} /// Internal trait to handle object store configuration and initialization. trait LogStoreFactoryExt { @@ -110,7 +122,7 @@ trait LogStoreFactoryExt { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult; + ) -> LogStoreResult; } impl LogStoreFactoryExt for T { @@ -119,7 +131,7 @@ impl LogStoreFactoryExt for T { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult { + ) -> LogStoreResult { let prefixed_store = options.decorate_store(root_store.clone(), location)?; let log_store = self.with_options(Arc::new(prefixed_store), root_store, location, options)?; @@ -133,7 +145,7 @@ impl LogStoreFactoryExt for Arc { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult { + ) -> LogStoreResult { T::with_options_internal(self, root_store, location, options) } } @@ -145,7 +157,7 @@ pub fn default_logstore( location: &Url, options: &StorageConfig, ) -> Arc { - Arc::new(default_logstore::DefaultLogStore::new( + Arc::new(DefaultLogStore::new( prefixed_store, root_store, LogStoreConfig { @@ -158,9 +170,9 @@ pub fn default_logstore( /// Sharable reference to [`LogStore`] pub type LogStoreRef = Arc; -static DELTA_LOG_PATH: LazyLock = LazyLock::new(|| Path::from("_delta_log")); +pub static DELTA_LOG_PATH: LazyLock = LazyLock::new(|| Path::from("_delta_log")); -pub(crate) static DELTA_LOG_REGEX: LazyLock = +pub static DELTA_LOG_REGEX: LazyLock = LazyLock::new(|| Regex::new(r"(\d{20})\.(json|checkpoint(\.\d+)?\.parquet)$").unwrap()); /// Return the [LogStoreRef] for the provided [Url] location @@ -175,10 +187,10 @@ pub(crate) static DELTA_LOG_REGEX: LazyLock = /// let storage_config = StorageConfig::default(); /// let logstore = logstore_for(location, storage_config).expect("Failed to get a logstore"); /// ``` -pub fn logstore_for(location: Url, storage_config: StorageConfig) -> DeltaResult { +pub fn logstore_for(location: Url, storage_config: StorageConfig) -> LogStoreResult { // turn location into scheme let scheme = Url::parse(&format!("{}://", location.scheme())) - .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; + .map_err(|_| LogStoreError::InvalidTableLocation(location.to_string()))?; if let Some(entry) = object_store_factories().get(&scheme) { debug!("Found a storage provider for {scheme} ({location})"); @@ -186,7 +198,7 @@ pub fn logstore_for(location: Url, storage_config: StorageConfig) -> DeltaResult return logstore_with(root_store, location, storage_config); } - Err(DeltaTableError::InvalidTableLocation(location.into())) + Err(LogStoreError::InvalidTableLocation(location.to_string())) } /// Return the [LogStoreRef] using the given [ObjectStoreRef] @@ -194,9 +206,9 @@ pub fn logstore_with( root_store: ObjectStoreRef, location: Url, storage_config: StorageConfig, -) -> DeltaResult { +) -> LogStoreResult { let scheme = Url::parse(&format!("{}://", location.scheme())) - .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; + .map_err(|_| LogStoreError::InvalidTableLocation(location.to_string()))?; if let Some(factory) = logstore_factories().get(&scheme) { debug!("Found a logstore provider for {scheme}"); @@ -206,9 +218,7 @@ pub fn logstore_with( } error!("Could not find a logstore for the scheme {scheme}"); - Err(DeltaTableError::InvalidTableLocation( - location.clone().into(), - )) + Err(LogStoreError::InvalidTableLocation(location.to_string())) } /// Holder whether it's tmp_commit path or commit bytes @@ -244,13 +254,13 @@ impl LogStoreConfig { &self, store: T, table_root: Option<&url::Url>, - ) -> DeltaResult> { + ) -> LogStoreResult> { let table_url = table_root.unwrap_or(&self.location); self.options.decorate_store(store, table_url) } pub fn object_store_factory(&self) -> ObjectStoreFactoryRegistry { - self::factories::object_store_factories() + crate::factories::object_store_factories() } } @@ -270,23 +280,23 @@ pub trait LogStore: Send + Sync + AsAny { fn name(&self) -> String; /// Trigger sync operation on log store to. - async fn refresh(&self) -> DeltaResult<()> { + async fn refresh(&self) -> LogStoreResult<()> { Ok(()) } /// Read data for commit entry with the given version. - async fn read_commit_entry(&self, version: i64) -> DeltaResult>; + async fn read_commit_entry(&self, version: i64) -> LogStoreResult>; /// Write list of actions as delta commit entry for given version. /// /// This operation can be retried with a higher version in case the write - /// fails with [`TransactionError::VersionAlreadyExists`]. + /// fails with [`LogStoreError::VersionAlreadyExists`]. async fn write_commit_entry( &self, version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), TransactionError>; + ) -> Result<(), LogStoreError>; /// Abort the commit entry for the given version. async fn abort_commit_entry( @@ -294,13 +304,13 @@ pub trait LogStore: Send + Sync + AsAny { version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), TransactionError>; + ) -> Result<(), LogStoreError>; /// Find latest version currently stored in the delta log. - async fn get_latest_version(&self, start_version: i64) -> DeltaResult; + async fn get_latest_version(&self, start_version: i64) -> LogStoreResult; - /// Get the list of actions for the next commit - async fn peek_next_commit(&self, current_version: i64) -> DeltaResult { + /// Get the actions for the next commit + async fn peek_next_commit(&self, current_version: i64) -> LogStoreResult { let next_version = current_version + 1; let commit_log_bytes = match self.read_commit_entry(next_version).await { Ok(Some(bytes)) => Ok(bytes), @@ -308,8 +318,8 @@ pub trait LogStore: Send + Sync + AsAny { Err(err) => Err(err), }?; - let actions = crate::logstore::get_actions(next_version, &commit_log_bytes); - Ok(PeekCommit::New(next_version, actions?)) + let actions = crate::get_actions(next_version, &commit_log_bytes)?; + Ok(PeekCommit::New(next_version, actions)) } /// Get object store, can pass operation_id for object stores linked to an operation @@ -342,12 +352,12 @@ pub trait LogStore: Send + Sync + AsAny { since = "0.1.0", note = "DO NOT USE: Just a stop gap to support lakefs during kernel migration" )] - fn transaction_url(&self, _operation_id: Uuid, base: &Url) -> DeltaResult { + fn transaction_url(&self, _operation_id: Uuid, base: &Url) -> LogStoreResult { Ok(base.clone()) } /// Check if the location is a delta table location - async fn is_delta_table_location(&self) -> DeltaResult { + async fn is_delta_table_location(&self) -> LogStoreResult { let object_store = self.object_store(None); let dummy_url = Url::parse("http://example.com").unwrap(); let log_path = Path::from("_delta_log"); @@ -381,20 +391,10 @@ pub trait LogStore: Send + Sync + AsAny { /// Get configuration representing configured log store. fn config(&self) -> &LogStoreConfig; - - #[cfg(feature = "datafusion")] - /// Generate a unique enough url to identify the store in datafusion. - /// The DF object store registry only cares about the scheme and the host of the url for - /// registering/fetching. In our case the scheme is hard-coded to "delta-rs", so to get a unique - /// host we convert the location from this `LogStore` to a valid name, combining the - /// original scheme, host and path with invalid characters replaced. - fn object_store_url(&self) -> ObjectStoreUrl { - crate::logstore::object_store_url(&self.config().location) - } } /// Extension trait for LogStore to handle some internal invariants. -pub(crate) trait LogStoreExt: LogStore { +pub trait LogStoreExt: LogStore { /// The fully qualified table URL /// /// The paths is guaranteed to end with a slash, @@ -424,11 +424,11 @@ impl LogStore for Arc { T::name(self) } - async fn refresh(&self) -> DeltaResult<()> { + async fn refresh(&self) -> LogStoreResult<()> { T::refresh(self).await } - async fn read_commit_entry(&self, version: i64) -> DeltaResult> { + async fn read_commit_entry(&self, version: i64) -> LogStoreResult> { T::read_commit_entry(self, version).await } @@ -437,7 +437,7 @@ impl LogStore for Arc { version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> Result<(), LogStoreError> { T::write_commit_entry(self, version, commit_or_bytes, operation_id).await } @@ -446,15 +446,15 @@ impl LogStore for Arc { version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> Result<(), LogStoreError> { T::abort_commit_entry(self, version, commit_or_bytes, operation_id).await } - async fn get_latest_version(&self, start_version: i64) -> DeltaResult { + async fn get_latest_version(&self, start_version: i64) -> LogStoreResult { T::get_latest_version(self, start_version).await } - async fn peek_next_commit(&self, current_version: i64) -> DeltaResult { + async fn peek_next_commit(&self, current_version: i64) -> LogStoreResult { T::peek_next_commit(self, current_version).await } @@ -482,18 +482,13 @@ impl LogStore for Arc { T::log_path(self) } - async fn is_delta_table_location(&self) -> DeltaResult { + async fn is_delta_table_location(&self) -> LogStoreResult { T::is_delta_table_location(self).await } fn config(&self) -> &LogStoreConfig { T::config(self) } - - #[cfg(feature = "datafusion")] - fn object_store_url(&self) -> ObjectStoreUrl { - T::object_store_url(self) - } } pub(crate) fn get_engine(store: Arc) -> Arc { @@ -511,24 +506,15 @@ pub(crate) fn get_engine(store: Arc) -> Arc { } } -#[cfg(feature = "datafusion")] -fn object_store_url(location: &Url) -> ObjectStoreUrl { - use object_store::path::DELIMITER; - ObjectStoreUrl::parse(format!( - "delta-rs://{}-{}{}", - location.scheme(), - location.host_str().unwrap_or("-"), - location.path().replace(DELIMITER, "-").replace(':', "-") - )) - .expect("Invalid object store url.") -} - /// Parse the path from a URL accounting for special case witjh S3 // TODO: find out why this is necessary -pub(crate) fn object_store_path(table_root: &Url) -> DeltaResult { +pub(crate) fn object_store_path(table_root: &Url) -> LogStoreResult { Ok(match ObjectStoreScheme::parse(table_root) { Ok((_, path)) => path, - _ => Path::parse(table_root.path())?, + _ => Path::parse(table_root.path()).map_err(|e| LogStoreError::InvalidPath { + path: table_root.path().to_string(), + source: Box::new(e), + })?, }) } @@ -566,14 +552,14 @@ pub fn to_uri(root: &Url, location: &Path) -> String { pub fn get_actions( version: i64, commit_log_bytes: &bytes::Bytes, -) -> Result, DeltaTableError> { +) -> Result, LogStoreError> { debug!("parsing commit with version {version}..."); Deserializer::from_slice(commit_log_bytes) - .into_iter::() + .into_iter::() .map(|result| { result.map_err(|e| { let line = format!("Error at line {}, column {}", e.line(), e.column()); - DeltaTableError::InvalidJsonLog { + LogStoreError::InvalidJsonLog { json_err: e, line, version, @@ -649,7 +635,7 @@ pub fn extract_version_from_filename(name: &str) -> Option { pub async fn get_latest_version( log_store: &dyn LogStore, current_version: i64, -) -> DeltaResult { +) -> LogStoreResult { let current_version = if current_version < 0 { 0 } else { @@ -663,14 +649,18 @@ pub async fn get_latest_version( LogSegment::for_table_changes(storage.as_ref(), log_root, current_version as u64, None) }) .await - .map_err(|e| DeltaTableError::Generic(e.to_string()))? + .map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })? .map_err(|e| { if e.to_string() .contains(&format!("to have version {current_version}")) { - DeltaTableError::InvalidVersion(current_version) + LogStoreError::InvalidVersion(current_version) } else { - DeltaTableError::Generic(e.to_string()) + LogStoreError::Generic { + source: Box::new(e), + } } })?; @@ -682,7 +672,7 @@ pub async fn get_latest_version( pub async fn read_commit_entry( storage: &dyn ObjectStore, version: i64, -) -> DeltaResult> { +) -> LogStoreResult> { let commit_uri = commit_uri_from_version(version); match storage.get(&commit_uri).await { Ok(res) => { @@ -707,21 +697,21 @@ pub async fn write_commit_entry( storage: &dyn ObjectStore, version: i64, tmp_commit: &Path, -) -> Result<(), TransactionError> { +) -> Result<(), LogStoreError> { // move temporary commit file to delta log directory // rely on storage to fail if the file already exists - storage .rename_if_not_exists(tmp_commit, &commit_uri_from_version(version)) .await - .map_err(|err| -> TransactionError { + .map_err(|err| -> LogStoreError { match err { ObjectStoreError::AlreadyExists { .. } => { warn!("commit entry already exists"); - TransactionError::VersionAlreadyExists(version) + LogStoreError::VersionAlreadyExists(version) } _ => { error!(error = %err, "failed to write commit entry"); - TransactionError::from(err) + LogStoreError::from(err) } } })?; @@ -735,7 +725,7 @@ pub async fn abort_commit_entry( storage: &dyn ObjectStore, _version: i64, tmp_commit: &Path, -) -> Result<(), TransactionError> { +) -> Result<(), LogStoreError> { storage.delete_with_retries(tmp_commit, 15).await?; debug!("commit entry aborted successfully"); Ok(()) @@ -937,8 +927,8 @@ pub(crate) mod tests { /// :w #[tokio::test] - async fn test_peek_with_invalid_json() -> DeltaResult<()> { - use crate::logstore::object_store::memory::InMemory; + async fn test_peek_with_invalid_json() -> LogStoreResult<()> { + use deltalake_logstore::object_store::memory::InMemory; let memory_store = Arc::new(InMemory::new()); let log_path = Path::from("delta-table/_delta_log/00000000000000000001.json"); @@ -971,35 +961,11 @@ pub(crate) mod tests { .try_collect::>() .await } -} - -#[cfg(all(test, feature = "datafusion"))] -mod datafusion_tests { - use super::*; - use url::Url; - - #[tokio::test] - async fn test_unique_object_store_url() { - for (location_1, location_2) in [ - // Same scheme, no host, different path - ("file:///path/to/table_1", "file:///path/to/table_2"), - // Different scheme/host, same path - ("s3://my_bucket/path/to/table_1", "file:///path/to/table_1"), - // Same scheme, different host, same path - ("s3://bucket_1/table_1", "s3://bucket_2/table_1"), - ] { - let url_1 = Url::parse(location_1).unwrap(); - let url_2 = Url::parse(location_2).unwrap(); - - assert_ne!( - object_store_url(&url_1).as_str(), - object_store_url(&url_2).as_str(), - ); - } - } #[tokio::test] async fn test_get_actions_malformed_json() { + use super::*; + let malformed_json = bytes::Bytes::from( r#"{"add": {"path": "test.parquet", "partitionValues": {}, "size": 100, "modificationTime": 1234567890, "dataChange": true}} {"invalid json without closing brace"#, @@ -1008,7 +974,7 @@ mod datafusion_tests { let result = get_actions(0, &malformed_json); match result { - Err(DeltaTableError::InvalidJsonLog { + Err(LogStoreError::InvalidJsonLog { line, version, json_err, diff --git a/crates/core/src/logstore/storage/mod.rs b/crates/logstore/src/storage/mod.rs similarity index 84% rename from crates/core/src/logstore/storage/mod.rs rename to crates/logstore/src/storage/mod.rs index 7f6af7386d..f4d12d0400 100644 --- a/crates/core/src/logstore/storage/mod.rs +++ b/crates/logstore/src/storage/mod.rs @@ -1,13 +1,12 @@ //! Object storage backend abstraction layer for Delta Table transaction logs and data -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; use dashmap::DashMap; use deltalake_derive::DeltaConfig; -use object_store::path::Path; use object_store::{DynObjectStore, ObjectStore}; use url::Url; -use crate::{DeltaResult, DeltaTableError}; +use crate::error::{LogStoreError, LogStoreResult}; pub use retry_ext::ObjectStoreRetryExt; pub use runtime::{DeltaIOStorageBackend, IORuntime}; @@ -16,8 +15,6 @@ pub(super) mod retry_ext; pub(super) mod runtime; pub(super) mod utils; -static DELTA_LOG_PATH: LazyLock = LazyLock::new(|| Path::from("_delta_log")); - /// Sharable reference to [`ObjectStore`] pub type ObjectStoreRef = Arc; @@ -33,7 +30,7 @@ pub trait ObjectStoreRegistry: Send + Sync + std::fmt::Debug + 'static { /// If no [`ObjectStore`] found for the `url`, ad-hoc discovery may be executed depending on /// the `url` and [`ObjectStoreRegistry`] implementation. An [`ObjectStore`] may be lazily /// created and registered. - fn get_store(&self, url: &Url) -> DeltaResult>; + fn get_store(&self, url: &Url) -> LogStoreResult>; } /// The default [`ObjectStoreRegistry`] @@ -80,12 +77,12 @@ impl ObjectStoreRegistry for DefaultObjectStoreRegistry { self.object_stores.insert(url.to_string(), store) } - fn get_store(&self, url: &Url) -> DeltaResult> { + fn get_store(&self, url: &Url) -> LogStoreResult> { self.object_stores .get(&url.to_string()) .map(|o| Arc::clone(o.value())) - .ok_or_else(|| { - DeltaTableError::generic(format!("No suitable object store found for '{url}'.")) + .ok_or_else(|| LogStoreError::ObjectStoreNotFound { + url: url.to_string(), }) } } @@ -105,8 +102,9 @@ mod tests { use rstest::*; use super::*; - use crate::logstore::config::TryUpdateKey; - use crate::test_utils::with_env; + use deltalake_logstore::config::TryUpdateKey; + // Note: test_utils::with_env removed - this requires core + // For testing, use environment variables directly or mock them #[test] fn test_limit_config() { @@ -122,13 +120,16 @@ mod tests { #[rstest] fn test_limit_config_env() { - let _env = with_env(vec![("OBJECT_STORE_CONCURRENCY_LIMIT", "100")]); + // Note: Using environment variables directly instead of test_utils::with_env + std::env::set_var("OBJECT_STORE_CONCURRENCY_LIMIT", "100"); let mut config = LimitConfig::default(); assert!(config.max_concurrency.is_none()); config.load_from_environment().unwrap(); assert_eq!(config.max_concurrency, Some(100)); + + std::env::remove_var("OBJECT_STORE_CONCURRENCY_LIMIT"); } #[rstest] diff --git a/crates/core/src/logstore/storage/retry_ext.rs b/crates/logstore/src/storage/retry_ext.rs similarity index 95% rename from crates/core/src/logstore/storage/retry_ext.rs rename to crates/logstore/src/storage/retry_ext.rs index c904366665..7c52892cdb 100644 --- a/crates/core/src/logstore/storage/retry_ext.rs +++ b/crates/logstore/src/storage/retry_ext.rs @@ -4,8 +4,7 @@ use ::object_store::path::Path; use ::object_store::{Error, ObjectStore, PutPayload, PutResult, Result}; use tracing::*; -#[cfg(feature = "cloud")] -use crate::logstore::config; +use crate::config; impl ObjectStoreRetryExt for T {} @@ -92,7 +91,7 @@ pub trait ObjectStoreRetryExt: ObjectStore { #[cfg(feature = "cloud")] impl config::TryUpdateKey for object_store::RetryConfig { - fn try_update_key(&mut self, key: &str, v: &str) -> crate::DeltaResult> { + fn try_update_key(&mut self, key: &str, v: &str) -> crate::LogStoreResult> { match key { "max_retries" => self.max_retries = config::parse_usize(v)?, "retry_timeout" => self.retry_timeout = config::parse_duration(v)?, @@ -110,7 +109,7 @@ impl config::TryUpdateKey for object_store::RetryConfig { Ok(Some(())) } - fn load_from_environment(&mut self) -> crate::DeltaResult<()> { + fn load_from_environment(&mut self) -> crate::LogStoreResult<()> { Ok(()) } } diff --git a/crates/core/src/logstore/storage/runtime.rs b/crates/logstore/src/storage/runtime.rs similarity index 100% rename from crates/core/src/logstore/storage/runtime.rs rename to crates/logstore/src/storage/runtime.rs diff --git a/crates/core/src/logstore/storage/utils.rs b/crates/logstore/src/storage/utils.rs similarity index 58% rename from crates/core/src/logstore/storage/utils.rs rename to crates/logstore/src/storage/utils.rs index 55a2f1c977..b26a58c5f4 100644 --- a/crates/core/src/logstore/storage/utils.rs +++ b/crates/logstore/src/storage/utils.rs @@ -1,11 +1,6 @@ //! Utility functions for working across Delta tables -use chrono::DateTime; use object_store::path::Path; -use object_store::ObjectMeta; - -use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::Add; /// Return the uri of commit version. /// @@ -17,38 +12,11 @@ use crate::kernel::Add; /// ``` pub fn commit_uri_from_version(version: i64) -> Path { let version = format!("{version:020}.json"); - super::DELTA_LOG_PATH.child(version.as_str()) -} - -impl TryFrom for ObjectMeta { - type Error = DeltaTableError; - - fn try_from(value: Add) -> DeltaResult { - (&value).try_into() - } + crate::DELTA_LOG_PATH.child(version.as_str()) } -impl TryFrom<&Add> for ObjectMeta { - type Error = DeltaTableError; - - fn try_from(value: &Add) -> DeltaResult { - let last_modified = DateTime::from_timestamp_millis(value.modification_time).ok_or( - DeltaTableError::MetadataError(format!( - "invalid modification_time: {:?}", - value.modification_time - )), - )?; - - Ok(Self { - // TODO this won't work for absolute paths, since Paths are always relative to store. - location: Path::parse(value.path.as_str())?, - last_modified, - size: value.size as u64, - e_tag: None, - version: None, - }) - } -} +// Note: Add type conversion removed - this requires core's Action type +// For logstore usage, work with ObjectMeta directly #[cfg(test)] mod tests { diff --git a/crates/types/Cargo.toml b/crates/types/Cargo.toml new file mode 100644 index 0000000000..c62ba94aee --- /dev/null +++ b/crates/types/Cargo.toml @@ -0,0 +1,27 @@ +[package] +name = "deltalake-types" +version = "0.29.2" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description = "Delta Lake action types" +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +delta_kernel.workspace = true +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true } +percent-encoding = "2" +strum = { workspace = true } +object_store = { workspace = true } + +[dev-dependencies] +serde_json = { workspace = true } + +[package.metadata.docs.rs] +all-features = true + diff --git a/crates/types/src/actions.rs b/crates/types/src/actions.rs new file mode 100644 index 0000000000..defaa300d9 --- /dev/null +++ b/crates/types/src/actions.rs @@ -0,0 +1,644 @@ +use std::borrow::Borrow; +use std::collections::HashMap; +use std::fmt::{self, Display}; +use std::hash::{Hash, Hasher}; +use std::str::FromStr; + +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; +use serde::{Deserialize, Serialize}; + +#[derive(Debug, PartialEq, Eq, Clone, Deserialize, Serialize)] +#[serde(rename_all = "camelCase")] +#[allow(missing_docs)] +pub enum Action { + #[serde(rename = "metaData")] + Metadata(delta_kernel::actions::Metadata), + Protocol(delta_kernel::actions::Protocol), + Add(Add), + Remove(Remove), + Cdc(AddCDCFile), + Txn(Transaction), + CommitInfo(CommitInfo), + DomainMetadata(DomainMetadata), +} + +impl Action { + /// Create a commit info from a map + pub fn commit_info(info: HashMap) -> Self { + Self::CommitInfo(CommitInfo { + info, + ..Default::default() + }) + } +} + +impl From for Action { + fn from(a: Add) -> Self { + Self::Add(a) + } +} + +impl From for Action { + fn from(a: Remove) -> Self { + Self::Remove(a) + } +} + +impl From for Action { + fn from(a: AddCDCFile) -> Self { + Self::Cdc(a) + } +} + +impl From for Action { + fn from(a: delta_kernel::actions::Metadata) -> Self { + Self::Metadata(a) + } +} + +impl From for Action { + fn from(a: delta_kernel::actions::Protocol) -> Self { + Self::Protocol(a) + } +} + +impl From for Action { + fn from(a: Transaction) -> Self { + Self::Txn(a) + } +} + +impl From for Action { + fn from(a: CommitInfo) -> Self { + Self::CommitInfo(a) + } +} + +impl From for Action { + fn from(a: DomainMetadata) -> Self { + Self::DomainMetadata(a) + } +} + +/// High level table features +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, Hash)] +#[serde(rename_all = "camelCase")] +pub enum TableFeatures { + /// Mapping of one column to another + ColumnMapping, + /// Deletion vectors for merge, update, delete + DeletionVectors, + /// timestamps without timezone support + #[serde(rename = "timestampNtz")] + TimestampWithoutTimezone, + /// version 2 of checkpointing + V2Checkpoint, + /// Append Only Tables + AppendOnly, + /// Table invariants + Invariants, + /// Check constraints on columns + CheckConstraints, + /// CDF on a table + ChangeDataFeed, + /// Columns with generated values + GeneratedColumns, + /// ID Columns + IdentityColumns, + /// Row tracking on tables + RowTracking, + /// domain specific metadata + DomainMetadata, + /// Iceberg compatibility support + IcebergCompatV1, +} + +impl FromStr for TableFeatures { + type Err = (); + + fn from_str(value: &str) -> Result { + match value { + "columnMapping" => Ok(TableFeatures::ColumnMapping), + "deletionVectors" => Ok(TableFeatures::DeletionVectors), + "timestampNtz" => Ok(TableFeatures::TimestampWithoutTimezone), + "v2Checkpoint" => Ok(TableFeatures::V2Checkpoint), + "appendOnly" => Ok(TableFeatures::AppendOnly), + "invariants" => Ok(TableFeatures::Invariants), + "checkConstraints" => Ok(TableFeatures::CheckConstraints), + "changeDataFeed" => Ok(TableFeatures::ChangeDataFeed), + "generatedColumns" => Ok(TableFeatures::GeneratedColumns), + "identityColumns" => Ok(TableFeatures::IdentityColumns), + "rowTracking" => Ok(TableFeatures::RowTracking), + "domainMetadata" => Ok(TableFeatures::DomainMetadata), + "icebergCompatV1" => Ok(TableFeatures::IcebergCompatV1), + _ => Err(()), + } + } +} + +impl AsRef for TableFeatures { + fn as_ref(&self) -> &str { + match self { + TableFeatures::ColumnMapping => "columnMapping", + TableFeatures::DeletionVectors => "deletionVectors", + TableFeatures::TimestampWithoutTimezone => "timestampNtz", + TableFeatures::V2Checkpoint => "v2Checkpoint", + TableFeatures::AppendOnly => "appendOnly", + TableFeatures::Invariants => "invariants", + TableFeatures::CheckConstraints => "checkConstraints", + TableFeatures::ChangeDataFeed => "changeDataFeed", + TableFeatures::GeneratedColumns => "generatedColumns", + TableFeatures::IdentityColumns => "identityColumns", + TableFeatures::RowTracking => "rowTracking", + TableFeatures::DomainMetadata => "domainMetadata", + TableFeatures::IcebergCompatV1 => "icebergCompatV1", + } + } +} + +impl fmt::Display for TableFeatures { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.as_ref()) + } +} + +impl TryFrom<&TableFeatures> for ReaderFeature { + type Error = strum::ParseError; + + fn try_from(value: &TableFeatures) -> Result { + ReaderFeature::try_from(value.as_ref()) + } +} + +impl TryFrom<&TableFeatures> for WriterFeature { + type Error = strum::ParseError; + + fn try_from(value: &TableFeatures) -> Result { + WriterFeature::try_from(value.as_ref()) + } +} + +impl TableFeatures { + /// Convert table feature to respective reader or/and write feature + pub fn to_reader_writer_features(&self) -> (Option, Option) { + let reader_feature = ReaderFeature::try_from(self) + .ok() + .and_then(|feature| match feature { + ReaderFeature::Unknown(_) => None, + _ => Some(feature), + }); + let writer_feature = WriterFeature::try_from(self) + .ok() + .and_then(|feature| match feature { + WriterFeature::Unknown(_) => None, + _ => Some(feature), + }); + (reader_feature, writer_feature) + } +} + +///Storage type of deletion vector +#[derive(Serialize, Deserialize, Copy, Clone, Debug, PartialEq, Eq)] +pub enum StorageType { + /// Stored at relative path derived from a UUID. + #[serde(rename = "u")] + UuidRelativePath, + /// Stored as inline string. + #[serde(rename = "i")] + Inline, + /// Stored at an absolute path. + #[serde(rename = "p")] + AbsolutePath, +} + +impl Default for StorageType { + fn default() -> Self { + Self::UuidRelativePath // seems to be used by Databricks and therefore most common + } +} + +impl FromStr for StorageType { + type Err = String; + + fn from_str(s: &str) -> Result { + match s { + "u" => Ok(Self::UuidRelativePath), + "i" => Ok(Self::Inline), + "p" => Ok(Self::AbsolutePath), + _ => Err(format!("Unknown storage format: '{s}'.")), + } + } +} + +impl AsRef for StorageType { + fn as_ref(&self) -> &str { + match self { + Self::UuidRelativePath => "u", + Self::Inline => "i", + Self::AbsolutePath => "p", + } + } +} + +impl Display for StorageType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.as_ref()) + } +} + +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +/// Defines a deletion vector +pub struct DeletionVectorDescriptor { + /// A single character to indicate how to access the DV. Legal options are: ['u', 'i', 'p']. + pub storage_type: StorageType, + + /// Three format options are currently proposed: + /// - If `storageType = 'u'` then ``: + /// The deletion vector is stored in a file with a path relative to the data + /// directory of this Delta table, and the file name can be reconstructed from + /// the UUID. See Derived Fields for how to reconstruct the file name. The random + /// prefix is recovered as the extra characters before the (20 characters fixed length) uuid. + /// - If `storageType = 'i'` then ``: The deletion vector + /// is stored inline in the log. The format used is the `RoaringBitmapArray` + /// format also used when the DV is stored on disk and described in [Deletion Vector Format]. + /// - If `storageType = 'p'` then ``: The DV is stored in a file with an + /// absolute path given by this path, which has the same format as the `path` field + /// in the `add`/`remove` actions. + /// + /// [Deletion Vector Format]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#Deletion-Vector-Format + pub path_or_inline_dv: String, + + /// Start of the data for this DV in number of bytes from the beginning of the file it is stored in. + /// Always None (absent in JSON) when `storageType = 'i'`. + #[serde(skip_serializing_if = "Option::is_none")] + pub offset: Option, + + /// Size of the serialized DV in bytes (raw data size, i.e. before base85 encoding, if inline). + pub size_in_bytes: i32, + + /// Number of rows the given DV logically removes from the file. + pub cardinality: i64, +} + +#[derive(Serialize, Deserialize, Debug, Clone, Default)] +#[serde(rename_all = "camelCase")] +/// Defines an add action +pub struct Add { + /// A relative path to a data file from the root of the table or an absolute path to a file + /// that should be added to the table. The path is a URI as specified by + /// [RFC 2396 URI Generic Syntax], which needs to be decoded to get the data file path. + /// + /// [RFC 2396 URI Generic Syntax]: https://www.ietf.org/rfc/rfc2396.txt + #[serde(with = "crate::serde_path")] + pub path: String, + + /// A map from partition column to value for this logical file. + pub partition_values: HashMap>, + + /// The size of this data file in bytes + pub size: i64, + + /// The time this logical file was created, as milliseconds since the epoch. + pub modification_time: i64, + + /// When `false` the logical file must already be present in the table or the records + /// in the added file must be contained in one or more remove actions in the same version. + pub data_change: bool, + + /// Contains [statistics] (e.g., count, min/max values for columns) about the data in this logical file. + /// + /// [statistics]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#Per-file-Statistics + pub stats: Option, + + /// Map containing metadata about this logical file. + pub tags: Option>>, + + #[serde(skip_serializing_if = "Option::is_none")] + /// Information about deletion vector (DV) associated with this add action + pub deletion_vector: Option, + + /// Default generated Row ID of the first row in the file. The default generated Row IDs + /// of the other rows in the file can be reconstructed by adding the physical index of the + /// row within the file to the base Row ID + pub base_row_id: Option, + + /// First commit version in which an add action with the same path was committed to the table. + pub default_row_commit_version: Option, + + /// The name of the clustering implementation + pub clustering_provider: Option, +} + +impl Hash for Add { + fn hash(&self, state: &mut H) { + self.path.hash(state); + } +} + +impl PartialEq for Add { + fn eq(&self, other: &Self) -> bool { + self.path == other.path + && self.size == other.size + && self.partition_values == other.partition_values + && self.modification_time == other.modification_time + && self.data_change == other.data_change + && self.stats == other.stats + && self.tags == other.tags + && self.deletion_vector == other.deletion_vector + } +} + +impl Eq for Add {} + +/// Represents a tombstone (deleted file) in the Delta log. +#[derive(Serialize, Deserialize, Debug, Clone, Default)] +#[serde(rename_all = "camelCase")] +pub struct Remove { + /// A relative path to a data file from the root of the table or an absolute path to a file + /// that should be added to the table. The path is a URI as specified by + /// [RFC 2396 URI Generic Syntax], which needs to be decoded to get the data file path. + /// + /// [RFC 2396 URI Generic Syntax]: https://www.ietf.org/rfc/rfc2396.txt + #[serde(with = "crate::serde_path")] + pub path: String, + + /// When `false` the logical file must already be present in the table or the records + /// in the added file must be contained in one or more remove actions in the same version. + pub data_change: bool, + + /// The time this logical file was created, as milliseconds since the epoch. + #[serde(skip_serializing_if = "Option::is_none")] + pub deletion_timestamp: Option, + + /// When true the fields `partition_values`, `size`, and `tags` are present + #[serde(skip_serializing_if = "Option::is_none")] + pub extended_file_metadata: Option, + + /// A map from partition column to value for this logical file. + #[serde(skip_serializing_if = "Option::is_none")] + pub partition_values: Option>>, + + /// The size of this data file in bytes + #[serde(skip_serializing_if = "Option::is_none")] + pub size: Option, + + /// Map containing metadata about this logical file. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, + + /// Information about deletion vector (DV) associated with this add action + #[serde(skip_serializing_if = "Option::is_none")] + pub deletion_vector: Option, + + /// Default generated Row ID of the first row in the file. The default generated Row IDs + /// of the other rows in the file can be reconstructed by adding the physical index of the + /// row within the file to the base Row ID + #[serde(skip_serializing_if = "Option::is_none")] + pub base_row_id: Option, + + /// First commit version in which an add action with the same path was committed to the table. + #[serde(skip_serializing_if = "Option::is_none")] + pub default_row_commit_version: Option, +} + +impl Hash for Remove { + fn hash(&self, state: &mut H) { + self.path.hash(state); + } +} + +/// Borrow `Remove` as str so we can look at tombstones hashset in `DeltaTableState` by using +/// a path from action `Add`. +impl Borrow for Remove { + fn borrow(&self) -> &str { + self.path.as_ref() + } +} + +impl PartialEq for Remove { + fn eq(&self, other: &Self) -> bool { + self.path == other.path + && self.deletion_timestamp == other.deletion_timestamp + && self.data_change == other.data_change + && self.extended_file_metadata == other.extended_file_metadata + && self.partition_values == other.partition_values + && self.size == other.size + && self.tags == other.tags + && self.deletion_vector == other.deletion_vector + } +} + +impl Eq for Remove {} +#[derive(Serialize, Deserialize, Clone, Debug, Default, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct AddCDCFile { + /// A relative path, from the root of the table, or an + /// absolute path to a CDC file + #[serde(with = "crate::serde_path")] + pub path: String, + + /// The size of this file in bytes + pub size: i64, + + /// A map from partition column to value for this file + pub partition_values: HashMap>, + + /// Should always be set to false because they do not change the underlying data of the table + pub data_change: bool, + + /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + +/// Action used by streaming systems to track progress using application-specific versions to +/// enable idempotency. +#[derive(Serialize, Deserialize, Debug, Default, Clone, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct Transaction { + /// A unique identifier for the application performing the transaction. + pub app_id: String, + + /// An application-specific numeric identifier for this transaction. + pub version: i64, + + /// The time when this transaction action was created in milliseconds since the Unix epoch. + #[serde(skip_serializing_if = "Option::is_none")] + pub last_updated: Option, +} + +impl Transaction { + /// Create a new application transactions. See [`Txn`] for details. + pub fn new(app_id: impl ToString, version: i64) -> Self { + Self::new_with_last_update(app_id, version, None) + } + + /// Create a new application transactions. See [`Txn`] for details. + pub fn new_with_last_update( + app_id: impl ToString, + version: i64, + last_updated: Option, + ) -> Self { + Transaction { + app_id: app_id.to_string(), + version, + last_updated, + } + } +} + +/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. +/// However the reference implementation as well as delta-rs store useful information that may for instance +/// allow us to be more permissive in commit conflict resolution. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct CommitInfo { + /// Timestamp in millis when the commit was created + #[serde(skip_serializing_if = "Option::is_none")] + pub timestamp: Option, + + /// Id of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_id: Option, + + /// Name of the user invoking the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub user_name: Option, + + /// The operation performed during the + #[serde(skip_serializing_if = "Option::is_none")] + pub operation: Option, + + /// Parameters used for table operation + #[serde(skip_serializing_if = "Option::is_none")] + pub operation_parameters: Option>, + + /// Version of the table when the operation was started + #[serde(skip_serializing_if = "Option::is_none")] + pub read_version: Option, + + /// The isolation level of the commit + #[serde(skip_serializing_if = "Option::is_none")] + pub isolation_level: Option, + + /// TODO + #[serde(skip_serializing_if = "Option::is_none")] + pub is_blind_append: Option, + + /// Delta engine which created the commit. + #[serde(skip_serializing_if = "Option::is_none")] + pub engine_info: Option, + + /// Additional provenance information for the commit + #[serde(flatten, default)] + pub info: HashMap, + + /// User defined metadata + #[serde(skip_serializing_if = "Option::is_none")] + pub user_metadata: Option, +} + +/// The domain metadata action contains a configuration (string) for a named metadata domain +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct DomainMetadata { + /// Identifier for this domain (system or user-provided) + pub domain: String, + + /// String containing configuration for the metadata domain + pub configuration: String, + + /// When `true` the action serves as a tombstone + pub removed: bool, +} + +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +/// This action is only allowed in checkpoints following V2 spec. It describes the details about the checkpoint. +pub struct CheckpointMetadata { + /// The flavor of the V2 checkpoint. Allowed values: "flat". + pub flavor: String, + + /// Map containing any additional metadata about the v2 spec checkpoint. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + +/// The sidecar action references a sidecar file which provides some of the checkpoint's file actions. +/// This action is only allowed in checkpoints following V2 spec. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct Sidecar { + /// The name of the sidecar file (not a path). + /// The file must reside in the _delta_log/_sidecars directory. + pub file_name: String, + + /// The size of the sidecar file in bytes + pub size_in_bytes: i64, + + /// The time this sidecar file was created, as milliseconds since the epoch. + pub modification_time: i64, + + /// Type of sidecar. Valid values are: "fileaction". + /// This could be extended in future to allow different kinds of sidecars. + #[serde(rename = "type")] + pub sidecar_type: String, + + /// Map containing any additional metadata about the checkpoint sidecar file. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + +#[derive(Serialize, Deserialize, Debug, Copy, Clone, PartialEq, Eq)] +/// The isolation level applied during transaction +pub enum IsolationLevel { + /// The strongest isolation level. It ensures that committed write operations + /// and all reads are Serializable. Operations are allowed as long as there + /// exists a serial sequence of executing them one-at-a-time that generates + /// the same outcome as that seen in the table. For the write operations, + /// the serial sequence is exactly the same as that seen in the table's history. + Serializable, + + /// A weaker isolation level than Serializable. It ensures only that the write + /// operations (that is, not reads) are serializable. However, this is still stronger + /// than Snapshot isolation. WriteSerializable is the default isolation level because + /// it provides great balance of data consistency and availability for most common operations. + WriteSerializable, + + /// SnapshotIsolation is a guarantee that all reads made in a transaction will see a consistent + /// snapshot of the database (in practice it reads the last committed values that existed at the + /// time it started), and the transaction itself will successfully commit only if no updates + /// it has made conflict with any concurrent updates made since that snapshot. + SnapshotIsolation, +} + +// Spark assumes Serializable as default isolation level +// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 +impl Default for IsolationLevel { + fn default() -> Self { + Self::Serializable + } +} + +impl AsRef for IsolationLevel { + fn as_ref(&self) -> &str { + match self { + Self::Serializable => "Serializable", + Self::WriteSerializable => "WriteSerializable", + Self::SnapshotIsolation => "SnapshotIsolation", + } + } +} + +impl FromStr for IsolationLevel { + type Err = String; + + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "serializable" => Ok(Self::Serializable), + "writeserializable" | "write_serializable" => Ok(Self::WriteSerializable), + "snapshotisolation" | "snapshot_isolation" => Ok(Self::SnapshotIsolation), + _ => Err("Invalid string for IsolationLevel".into()), + } + } +} diff --git a/crates/types/src/lib.rs b/crates/types/src/lib.rs new file mode 100644 index 0000000000..b0491d8a4b --- /dev/null +++ b/crates/types/src/lib.rs @@ -0,0 +1,13 @@ +//! Delta Lake action types +//! +//! This crate contains the fundamental action types used in Delta Lake. +//! Actions are the fundamental unit of work in Delta Lake. Each action performs a single atomic +//! operation on the state of a Delta table. Actions are stored in the `_delta_log` directory of a +//! Delta table in JSON format. The log is a time series of actions that represent all the changes +//! made to a table. + +mod actions; +mod serde_path; + +pub use actions::*; +pub use delta_kernel::actions::{Metadata, Protocol}; diff --git a/crates/types/src/serde_path.rs b/crates/types/src/serde_path.rs new file mode 100644 index 0000000000..023384465c --- /dev/null +++ b/crates/types/src/serde_path.rs @@ -0,0 +1,59 @@ +use std::str::Utf8Error; + +use percent_encoding::{percent_decode_str, percent_encode, AsciiSet, CONTROLS}; +use serde::{self, Deserialize, Deserializer, Serialize, Serializer}; + +pub fn deserialize<'de, D>(deserializer: D) -> Result +where + D: Deserializer<'de>, +{ + let s = String::deserialize(deserializer)?; + decode_path(&s).map_err(serde::de::Error::custom) +} + +pub fn serialize(value: &str, serializer: S) -> Result +where + S: Serializer, +{ + let encoded = encode_path(value); + String::serialize(&encoded, serializer) +} + +pub const _DELIMITER: &str = "/"; +/// The path delimiter as a single byte +pub const _DELIMITER_BYTE: u8 = _DELIMITER.as_bytes()[0]; + +/// Characters we want to encode. +const INVALID: &AsciiSet = &CONTROLS + // The delimiter we are reserving for internal hierarchy + // .add(DELIMITER_BYTE) + // Characters AWS recommends avoiding for object keys + // https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingMetadata.html + .add(b'\\') + .add(b'{') + .add(b'^') + .add(b'}') + .add(b'%') + .add(b'`') + .add(b']') + .add(b'"') + .add(b'>') + .add(b'[') + // .add(b'~') + .add(b'<') + .add(b'#') + .add(b'|') + // Characters Google Cloud Storage recommends avoiding for object names + // https://cloud.google.com/storage/docs/naming-objects + .add(b'\r') + .add(b'\n') + .add(b'*') + .add(b'?'); + +fn encode_path(path: &str) -> String { + percent_encode(path.as_bytes(), INVALID).to_string() +} + +pub fn decode_path(path: &str) -> Result { + Ok(percent_decode_str(path).decode_utf8()?.to_string()) +} From e8edb9d0907f5b7d55cb3d7118516a7bc4bcecbc Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 09:32:07 -0500 Subject: [PATCH 02/10] Get everything outside LakeFS compiling Signed-off-by: Abhi Agarwal --- Cargo.toml | 1 + crates/aws/src/logstore/dynamodb_logstore.rs | 4 +- crates/catalog-unity/Cargo.toml | 1 + crates/catalog-unity/src/lib.rs | 39 +- crates/core/Cargo.toml | 5 +- crates/core/src/kernel/models/actions.rs | 586 ------------------- crates/core/src/lib.rs | 3 - crates/core/src/operations/merge/mod.rs | 2 +- crates/core/src/operations/update.rs | 2 +- crates/core/src/operations/write/writer.rs | 2 +- crates/core/src/protocol/mod.rs | 29 + crates/core/src/table/builder.rs | 2 +- crates/core/src/table/mod.rs | 26 + crates/core/src/test_utils/mod.rs | 14 + crates/gcp/Cargo.toml | 1 + crates/gcp/src/config.rs | 8 +- crates/gcp/src/error.rs | 22 - crates/gcp/src/lib.rs | 10 +- crates/gcp/src/storage.rs | 4 +- crates/hdfs/src/lib.rs | 3 +- crates/lakefs/Cargo.toml | 1 + crates/lakefs/src/client.rs | 3 +- crates/lakefs/src/errors.rs | 9 + crates/lakefs/src/lib.rs | 5 +- crates/lakefs/src/logstore.rs | 67 ++- crates/lakefs/src/storage.rs | 18 +- crates/logstore/Cargo.toml | 2 +- crates/logstore/src/factories.rs | 2 +- crates/logstore/src/lib.rs | 6 +- crates/logstore/src/logstore_impl.rs | 37 -- crates/logstore/src/storage/mod.rs | 2 +- crates/logstore/src/storage/utils.rs | 37 -- crates/mount/Cargo.toml | 2 +- crates/mount/src/error.rs | 23 +- crates/mount/src/lib.rs | 64 +- crates/types/Cargo.toml | 1 - 36 files changed, 245 insertions(+), 798 deletions(-) delete mode 100644 crates/gcp/src/error.rs diff --git a/Cargo.toml b/Cargo.toml index 0c54f9cfaa..d8d63175fd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,6 +59,7 @@ bytes = { version = "1" } chrono = { version = "0.4.40", default-features = false, features = ["clock"] } tracing = { version = "0.1", features = ["log"] } regex = { version = "1" } +rstest = { version = "0.26.1" } thiserror = { version = "2" } url = { version = "2" } percent-encoding-rfc3986 = { version = "0.1.3" } diff --git a/crates/aws/src/logstore/dynamodb_logstore.rs b/crates/aws/src/logstore/dynamodb_logstore.rs index 8113fc8f66..311bc09ab0 100644 --- a/crates/aws/src/logstore/dynamodb_logstore.rs +++ b/crates/aws/src/logstore/dynamodb_logstore.rs @@ -107,7 +107,7 @@ impl S3DynamoDbLogStore { warn!("It looks like the {}.json has already been moved, we got 404 from ObjectStorage.", entry.version); return self.try_complete_entry(entry, false).await; } - Err(err) if retry == MAX_REPAIR_RETRIES => return Err(LogStoreError::from(err)), + Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), Err(err) => { debug!("retry #{retry} on log entry {entry:?} failed to move commit: '{err}'") } @@ -136,7 +136,7 @@ impl S3DynamoDbLogStore { source: Box::new(err), }) { Ok(x) => return Ok(Self::map_retry_result(x, copy_performed)), - Err(err) if retry == MAX_REPAIR_RETRIES => return Err(LogStoreError::from(err)), + Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), Err(err) => error!( "retry #{retry} on log entry {entry:?} failed to update lock db: '{err}'" ), diff --git a/crates/catalog-unity/Cargo.toml b/crates/catalog-unity/Cargo.toml index 326a456319..678e799539 100644 --- a/crates/catalog-unity/Cargo.toml +++ b/crates/catalog-unity/Cargo.toml @@ -21,6 +21,7 @@ futures.workspace = true chrono.workspace = true tracing.workspace = true deltalake-core = { version = "0.29.2", path = "../core" } +deltalake-logstore = { version = "0.29.0", path = "../logstore" } deltalake-aws = { version = "0.12.0", path = "../aws", optional = true } deltalake-azure = { version = "0.12.0", path = "../azure", optional = true } deltalake-gcp = { version = "0.13.0", path = "../gcp", optional = true } diff --git a/crates/catalog-unity/src/lib.rs b/crates/catalog-unity/src/lib.rs index 4c8c85ebc9..b650e5e4b3 100644 --- a/crates/catalog-unity/src/lib.rs +++ b/crates/catalog-unity/src/lib.rs @@ -7,10 +7,12 @@ compile_error!( for this crate to function properly." ); +use deltalake_core::logstore::LogStoreResult; use deltalake_core::logstore::{ default_logstore, logstore_factories, object_store::RetryConfig, LogStore, LogStoreFactory, StorageConfig, }; +use deltalake_logstore::LogStoreError; use reqwest::header::{HeaderValue, InvalidHeaderValue, AUTHORIZATION}; use reqwest::Url; use std::collections::HashMap; @@ -160,6 +162,14 @@ impl From for DeltaTableError { } } +impl From for LogStoreError { + fn from(value: UnityCatalogError) -> Self { + LogStoreError::Generic { + source: Box::new(value), + } + } +} + /// Configuration options for unity catalog client pub enum UnityCatalogConfigKey { /// Url of a Databricks workspace @@ -500,7 +510,7 @@ impl UnityCatalogBuilder { self } - fn execute_uc_future(future: F) -> DeltaResult + fn execute_uc_future(future: F) -> LogStoreResult where T: Send, F: Future + Send, @@ -517,7 +527,7 @@ impl UnityCatalogBuilder { cfg = Some(handle.block_on(future)); }); }); - cfg.ok_or(DeltaTableError::ObjectStore { + cfg.ok_or(LogStoreError::ObjectStore { source: ObjectStoreError::Generic { store: STORE_NAME, source: Box::new(UnityCatalogError::InitializationError), @@ -885,7 +895,7 @@ impl ObjectStoreFactory for UnityCatalogFactory { &self, table_uri: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let (table_path, temp_creds) = UnityCatalogBuilder::execute_uc_future( UnityCatalogBuilder::get_uc_location_and_token(table_uri.as_str(), Some(&config.raw)), )??; @@ -895,8 +905,13 @@ impl ObjectStoreFactory for UnityCatalogFactory { // TODO(roeap): we should not have to go through the table here. // ideally we just create the right storage ... - let table_url = ensure_table_uri(&table_path)?; - let mut builder = DeltaTableBuilder::from_uri(table_url)?; + let table_url = ensure_table_uri(&table_path).map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })?; + let mut builder = + DeltaTableBuilder::from_uri(table_url).map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })?; if let Some(runtime) = &config.runtime { builder = builder.with_io_runtime(runtime.clone()); @@ -905,8 +920,16 @@ impl ObjectStoreFactory for UnityCatalogFactory { if !storage_options.is_empty() { builder = builder.with_storage_options(storage_options.clone()); } - let prefix = Path::parse(table_uri.path())?; - let store = builder.build_storage()?.object_store(None); + let prefix = Path::parse(table_uri.path()).map_err(|e| LogStoreError::InvalidPath { + path: table_uri.path().to_string(), + source: Box::new(e), + })?; + let store = builder + .build_storage() + .map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })? + .object_store(None); Ok((store, prefix)) } @@ -919,7 +942,7 @@ impl LogStoreFactory for UnityCatalogFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { Ok(default_logstore( prefixed_store, root_store, diff --git a/crates/core/Cargo.toml b/crates/core/Cargo.toml index 83e3ecc097..c77c1e42a2 100644 --- a/crates/core/Cargo.toml +++ b/crates/core/Cargo.toml @@ -86,7 +86,6 @@ pin-project-lite = "^0.2.7" tracing = { workspace = true } rand = "0.8" sqlparser = { version = "0.59.0" } -humantime = { version = "2.1.0", optional = true } validator = { version = "0.19", features = ["derive"] } [dev-dependencies] @@ -98,7 +97,7 @@ dotenvy = "0" fs_extra = "1.2.0" pretty_assertions = "1.2.1" pretty_env_logger = "0.5.0" -rstest = { version = "0.26.1" } +rstest = { workspace = true } serial_test = "3" tempfile = { workspace = true } tokio = { version = "1", features = ["macros", "rt-multi-thread"] } @@ -112,7 +111,7 @@ json = ["parquet/json"] python = ["arrow/pyarrow"] native-tls = ["delta_kernel/default-engine-native-tls"] rustls = ["delta_kernel/default-engine-rustls"] -cloud = ["object_store/cloud", "dep:humantime"] +cloud = ["object_store/cloud", "deltalake-logstore/cloud"] # enable caching some file I/O operations when scanning delta logs delta-cache = ["foyer", "tempfile", "url/serde"] diff --git a/crates/core/src/kernel/models/actions.rs b/crates/core/src/kernel/models/actions.rs index b1f48df120..213ab3f6a8 100644 --- a/crates/core/src/kernel/models/actions.rs +++ b/crates/core/src/kernel/models/actions.rs @@ -574,595 +574,9 @@ impl ProtocolInner { } } -/// High level table features -#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, Hash)] -#[serde(rename_all = "camelCase")] -pub enum TableFeatures { - /// Mapping of one column to another - ColumnMapping, - /// Deletion vectors for merge, update, delete - DeletionVectors, - /// timestamps without timezone support - #[serde(rename = "timestampNtz")] - TimestampWithoutTimezone, - /// version 2 of checkpointing - V2Checkpoint, - /// Append Only Tables - AppendOnly, - /// Table invariants - Invariants, - /// Check constraints on columns - CheckConstraints, - /// CDF on a table - ChangeDataFeed, - /// Columns with generated values - GeneratedColumns, - /// ID Columns - IdentityColumns, - /// Row tracking on tables - RowTracking, - /// domain specific metadata - DomainMetadata, - /// Iceberg compatibility support - IcebergCompatV1, -} - -impl FromStr for TableFeatures { - type Err = (); - - fn from_str(value: &str) -> Result { - match value { - "columnMapping" => Ok(TableFeatures::ColumnMapping), - "deletionVectors" => Ok(TableFeatures::DeletionVectors), - "timestampNtz" => Ok(TableFeatures::TimestampWithoutTimezone), - "v2Checkpoint" => Ok(TableFeatures::V2Checkpoint), - "appendOnly" => Ok(TableFeatures::AppendOnly), - "invariants" => Ok(TableFeatures::Invariants), - "checkConstraints" => Ok(TableFeatures::CheckConstraints), - "changeDataFeed" => Ok(TableFeatures::ChangeDataFeed), - "generatedColumns" => Ok(TableFeatures::GeneratedColumns), - "identityColumns" => Ok(TableFeatures::IdentityColumns), - "rowTracking" => Ok(TableFeatures::RowTracking), - "domainMetadata" => Ok(TableFeatures::DomainMetadata), - "icebergCompatV1" => Ok(TableFeatures::IcebergCompatV1), - _ => Err(()), - } - } -} - -impl AsRef for TableFeatures { - fn as_ref(&self) -> &str { - match self { - TableFeatures::ColumnMapping => "columnMapping", - TableFeatures::DeletionVectors => "deletionVectors", - TableFeatures::TimestampWithoutTimezone => "timestampNtz", - TableFeatures::V2Checkpoint => "v2Checkpoint", - TableFeatures::AppendOnly => "appendOnly", - TableFeatures::Invariants => "invariants", - TableFeatures::CheckConstraints => "checkConstraints", - TableFeatures::ChangeDataFeed => "changeDataFeed", - TableFeatures::GeneratedColumns => "generatedColumns", - TableFeatures::IdentityColumns => "identityColumns", - TableFeatures::RowTracking => "rowTracking", - TableFeatures::DomainMetadata => "domainMetadata", - TableFeatures::IcebergCompatV1 => "icebergCompatV1", - } - } -} - -impl fmt::Display for TableFeatures { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.as_ref()) - } -} - -impl TryFrom<&TableFeatures> for ReaderFeature { - type Error = strum::ParseError; - - fn try_from(value: &TableFeatures) -> Result { - ReaderFeature::try_from(value.as_ref()) - } -} - -impl TryFrom<&TableFeatures> for WriterFeature { - type Error = strum::ParseError; - - fn try_from(value: &TableFeatures) -> Result { - WriterFeature::try_from(value.as_ref()) - } -} - -impl TableFeatures { - /// Convert table feature to respective reader or/and write feature - pub fn to_reader_writer_features(&self) -> (Option, Option) { - let reader_feature = ReaderFeature::try_from(self) - .ok() - .and_then(|feature| match feature { - ReaderFeature::Unknown(_) => None, - _ => Some(feature), - }); - let writer_feature = WriterFeature::try_from(self) - .ok() - .and_then(|feature| match feature { - WriterFeature::Unknown(_) => None, - _ => Some(feature), - }); - (reader_feature, writer_feature) - } -} - -///Storage type of deletion vector -#[derive(Serialize, Deserialize, Copy, Clone, Debug, PartialEq, Eq)] -pub enum StorageType { - /// Stored at relative path derived from a UUID. - #[serde(rename = "u")] - UuidRelativePath, - /// Stored as inline string. - #[serde(rename = "i")] - Inline, - /// Stored at an absolute path. - #[serde(rename = "p")] - AbsolutePath, -} - -impl Default for StorageType { - fn default() -> Self { - Self::UuidRelativePath // seems to be used by Databricks and therefore most common - } -} - -impl FromStr for StorageType { - type Err = Error; - - fn from_str(s: &str) -> Result { - match s { - "u" => Ok(Self::UuidRelativePath), - "i" => Ok(Self::Inline), - "p" => Ok(Self::AbsolutePath), - _ => Err(Error::DeletionVector(format!( - "Unknown storage format: '{s}'." - ))), - } - } -} - -impl AsRef for StorageType { - fn as_ref(&self) -> &str { - match self { - Self::UuidRelativePath => "u", - Self::Inline => "i", - Self::AbsolutePath => "p", - } - } -} - -impl Display for StorageType { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.as_ref()) - } -} - -#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -/// Defines a deletion vector -pub struct DeletionVectorDescriptor { - /// A single character to indicate how to access the DV. Legal options are: ['u', 'i', 'p']. - pub storage_type: StorageType, - - /// Three format options are currently proposed: - /// - If `storageType = 'u'` then ``: - /// The deletion vector is stored in a file with a path relative to the data - /// directory of this Delta table, and the file name can be reconstructed from - /// the UUID. See Derived Fields for how to reconstruct the file name. The random - /// prefix is recovered as the extra characters before the (20 characters fixed length) uuid. - /// - If `storageType = 'i'` then ``: The deletion vector - /// is stored inline in the log. The format used is the `RoaringBitmapArray` - /// format also used when the DV is stored on disk and described in [Deletion Vector Format]. - /// - If `storageType = 'p'` then ``: The DV is stored in a file with an - /// absolute path given by this path, which has the same format as the `path` field - /// in the `add`/`remove` actions. - /// - /// [Deletion Vector Format]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#Deletion-Vector-Format - pub path_or_inline_dv: String, - - /// Start of the data for this DV in number of bytes from the beginning of the file it is stored in. - /// Always None (absent in JSON) when `storageType = 'i'`. - #[serde(skip_serializing_if = "Option::is_none")] - pub offset: Option, - - /// Size of the serialized DV in bytes (raw data size, i.e. before base85 encoding, if inline). - pub size_in_bytes: i32, - - /// Number of rows the given DV logically removes from the file. - pub cardinality: i64, -} - -#[derive(Serialize, Deserialize, Debug, Clone, Default)] -#[serde(rename_all = "camelCase")] -/// Defines an add action -pub struct Add { - /// A relative path to a data file from the root of the table or an absolute path to a file - /// that should be added to the table. The path is a URI as specified by - /// [RFC 2396 URI Generic Syntax], which needs to be decoded to get the data file path. - /// - /// [RFC 2396 URI Generic Syntax]: https://www.ietf.org/rfc/rfc2396.txt - #[serde(with = "serde_path")] - pub path: String, - - /// A map from partition column to value for this logical file. - pub partition_values: HashMap>, - - /// The size of this data file in bytes - pub size: i64, - - /// The time this logical file was created, as milliseconds since the epoch. - pub modification_time: i64, - - /// When `false` the logical file must already be present in the table or the records - /// in the added file must be contained in one or more remove actions in the same version. - pub data_change: bool, - - /// Contains [statistics] (e.g., count, min/max values for columns) about the data in this logical file. - /// - /// [statistics]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#Per-file-Statistics - pub stats: Option, - - /// Map containing metadata about this logical file. - pub tags: Option>>, - - #[serde(skip_serializing_if = "Option::is_none")] - /// Information about deletion vector (DV) associated with this add action - pub deletion_vector: Option, - - /// Default generated Row ID of the first row in the file. The default generated Row IDs - /// of the other rows in the file can be reconstructed by adding the physical index of the - /// row within the file to the base Row ID - pub base_row_id: Option, - - /// First commit version in which an add action with the same path was committed to the table. - pub default_row_commit_version: Option, - - /// The name of the clustering implementation - pub clustering_provider: Option, -} - -/// Represents a tombstone (deleted file) in the Delta log. -#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, Default)] -#[serde(rename_all = "camelCase")] -pub struct Remove { - /// A relative path to a data file from the root of the table or an absolute path to a file - /// that should be added to the table. The path is a URI as specified by - /// [RFC 2396 URI Generic Syntax], which needs to be decoded to get the data file path. - /// - /// [RFC 2396 URI Generic Syntax]: https://www.ietf.org/rfc/rfc2396.txt - #[serde(with = "serde_path")] - pub path: String, - - /// When `false` the logical file must already be present in the table or the records - /// in the added file must be contained in one or more remove actions in the same version. - pub data_change: bool, - - /// The time this logical file was created, as milliseconds since the epoch. - #[serde(skip_serializing_if = "Option::is_none")] - pub deletion_timestamp: Option, - - /// When true the fields `partition_values`, `size`, and `tags` are present - #[serde(skip_serializing_if = "Option::is_none")] - pub extended_file_metadata: Option, - - /// A map from partition column to value for this logical file. - #[serde(skip_serializing_if = "Option::is_none")] - pub partition_values: Option>>, - - /// The size of this data file in bytes - #[serde(skip_serializing_if = "Option::is_none")] - pub size: Option, - - /// Map containing metadata about this logical file. - #[serde(skip_serializing_if = "Option::is_none")] - pub tags: Option>>, - - /// Information about deletion vector (DV) associated with this add action - #[serde(skip_serializing_if = "Option::is_none")] - pub deletion_vector: Option, - - /// Default generated Row ID of the first row in the file. The default generated Row IDs - /// of the other rows in the file can be reconstructed by adding the physical index of the - /// row within the file to the base Row ID - #[serde(skip_serializing_if = "Option::is_none")] - pub base_row_id: Option, - - /// First commit version in which an add action with the same path was committed to the table. - #[serde(skip_serializing_if = "Option::is_none")] - pub default_row_commit_version: Option, -} - -/// Delta AddCDCFile action that describes a parquet CDC data file. -#[derive(Serialize, Deserialize, Clone, Debug, Default, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -pub struct AddCDCFile { - /// A relative path, from the root of the table, or an - /// absolute path to a CDC file - #[serde(with = "serde_path")] - pub path: String, - - /// The size of this file in bytes - pub size: i64, - - /// A map from partition column to value for this file - pub partition_values: HashMap>, - - /// Should always be set to false because they do not change the underlying data of the table - pub data_change: bool, - - /// Map containing metadata about this file - #[serde(skip_serializing_if = "Option::is_none")] - pub tags: Option>>, -} - -/// Action used by streaming systems to track progress using application-specific versions to -/// enable idempotency. -#[derive(Serialize, Deserialize, Debug, Default, Clone, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -pub struct Transaction { - /// A unique identifier for the application performing the transaction. - pub app_id: String, - - /// An application-specific numeric identifier for this transaction. - pub version: i64, - - /// The time when this transaction action was created in milliseconds since the Unix epoch. - #[serde(skip_serializing_if = "Option::is_none")] - pub last_updated: Option, -} - -impl Transaction { - /// Create a new application transactions. See [`Txn`] for details. - pub fn new(app_id: impl ToString, version: i64) -> Self { - Self::new_with_last_update(app_id, version, None) - } - - /// Create a new application transactions. See [`Txn`] for details. - pub fn new_with_last_update( - app_id: impl ToString, - version: i64, - last_updated: Option, - ) -> Self { - Transaction { - app_id: app_id.to_string(), - version, - last_updated, - } - } -} - -/// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. -/// However the reference implementation as well as delta-rs store useful information that may for instance -/// allow us to be more permissive in commit conflict resolution. -#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -pub struct CommitInfo { - /// Timestamp in millis when the commit was created - #[serde(skip_serializing_if = "Option::is_none")] - pub timestamp: Option, - - /// Id of the user invoking the commit - #[serde(skip_serializing_if = "Option::is_none")] - pub user_id: Option, - - /// Name of the user invoking the commit - #[serde(skip_serializing_if = "Option::is_none")] - pub user_name: Option, - - /// The operation performed during the - #[serde(skip_serializing_if = "Option::is_none")] - pub operation: Option, - - /// Parameters used for table operation - #[serde(skip_serializing_if = "Option::is_none")] - pub operation_parameters: Option>, - - /// Version of the table when the operation was started - #[serde(skip_serializing_if = "Option::is_none")] - pub read_version: Option, - - /// The isolation level of the commit - #[serde(skip_serializing_if = "Option::is_none")] - pub isolation_level: Option, - - /// TODO - #[serde(skip_serializing_if = "Option::is_none")] - pub is_blind_append: Option, - - /// Delta engine which created the commit. - #[serde(skip_serializing_if = "Option::is_none")] - pub engine_info: Option, - - /// Additional provenance information for the commit - #[serde(flatten, default)] - pub info: HashMap, - - /// User defined metadata - #[serde(skip_serializing_if = "Option::is_none")] - pub user_metadata: Option, -} - -/// The domain metadata action contains a configuration (string) for a named metadata domain -#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -pub struct DomainMetadata { - /// Identifier for this domain (system or user-provided) - pub domain: String, - - /// String containing configuration for the metadata domain - pub configuration: String, - - /// When `true` the action serves as a tombstone - pub removed: bool, -} - -#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] -/// This action is only allowed in checkpoints following V2 spec. It describes the details about the checkpoint. -pub struct CheckpointMetadata { - /// The flavor of the V2 checkpoint. Allowed values: "flat". - pub flavor: String, - - /// Map containing any additional metadata about the v2 spec checkpoint. - #[serde(skip_serializing_if = "Option::is_none")] - pub tags: Option>>, -} - -/// The sidecar action references a sidecar file which provides some of the checkpoint's file actions. -/// This action is only allowed in checkpoints following V2 spec. -#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -pub struct Sidecar { - /// The name of the sidecar file (not a path). - /// The file must reside in the _delta_log/_sidecars directory. - pub file_name: String, - - /// The size of the sidecar file in bytes - pub size_in_bytes: i64, - - /// The time this sidecar file was created, as milliseconds since the epoch. - pub modification_time: i64, - - /// Type of sidecar. Valid values are: "fileaction". - /// This could be extended in future to allow different kinds of sidecars. - #[serde(rename = "type")] - pub sidecar_type: String, - - /// Map containing any additional metadata about the checkpoint sidecar file. - #[serde(skip_serializing_if = "Option::is_none")] - pub tags: Option>>, -} - -#[derive(Serialize, Deserialize, Debug, Copy, Clone, PartialEq, Eq)] -/// The isolation level applied during transaction -pub enum IsolationLevel { - /// The strongest isolation level. It ensures that committed write operations - /// and all reads are Serializable. Operations are allowed as long as there - /// exists a serial sequence of executing them one-at-a-time that generates - /// the same outcome as that seen in the table. For the write operations, - /// the serial sequence is exactly the same as that seen in the table’s history. - Serializable, - - /// A weaker isolation level than Serializable. It ensures only that the write - /// operations (that is, not reads) are serializable. However, this is still stronger - /// than Snapshot isolation. WriteSerializable is the default isolation level because - /// it provides great balance of data consistency and availability for most common operations. - WriteSerializable, - - /// SnapshotIsolation is a guarantee that all reads made in a transaction will see a consistent - /// snapshot of the database (in practice it reads the last committed values that existed at the - /// time it started), and the transaction itself will successfully commit only if no updates - /// it has made conflict with any concurrent updates made since that snapshot. - SnapshotIsolation, -} - -// Spark assumes Serializable as default isolation level -// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 -impl Default for IsolationLevel { - fn default() -> Self { - Self::Serializable - } -} - -impl AsRef for IsolationLevel { - fn as_ref(&self) -> &str { - match self { - Self::Serializable => "Serializable", - Self::WriteSerializable => "WriteSerializable", - Self::SnapshotIsolation => "SnapshotIsolation", - } - } -} - -impl FromStr for IsolationLevel { - type Err = Error; - - fn from_str(s: &str) -> Result { - match s.to_ascii_lowercase().as_str() { - "serializable" => Ok(Self::Serializable), - "writeserializable" | "write_serializable" => Ok(Self::WriteSerializable), - "snapshotisolation" | "snapshot_isolation" => Ok(Self::SnapshotIsolation), - _ => Err(Error::Generic("Invalid string for IsolationLevel".into())), - } - } -} - -pub(crate) mod serde_path { - use std::str::Utf8Error; - - use percent_encoding::{percent_decode_str, percent_encode, AsciiSet, CONTROLS}; - use serde::{self, Deserialize, Deserializer, Serialize, Serializer}; - - pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - decode_path(&s).map_err(serde::de::Error::custom) - } - - pub fn serialize(value: &str, serializer: S) -> Result - where - S: Serializer, - { - let encoded = encode_path(value); - String::serialize(&encoded, serializer) - } - - pub const _DELIMITER: &str = "/"; - /// The path delimiter as a single byte - pub const _DELIMITER_BYTE: u8 = _DELIMITER.as_bytes()[0]; - - /// Characters we want to encode. - const INVALID: &AsciiSet = &CONTROLS - // The delimiter we are reserving for internal hierarchy - // .add(DELIMITER_BYTE) - // Characters AWS recommends avoiding for object keys - // https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingMetadata.html - .add(b'\\') - .add(b'{') - .add(b'^') - .add(b'}') - .add(b'%') - .add(b'`') - .add(b']') - .add(b'"') - .add(b'>') - .add(b'[') - // .add(b'~') - .add(b'<') - .add(b'#') - .add(b'|') - // Characters Google Cloud Storage recommends avoiding for object names - // https://cloud.google.com/storage/docs/naming-objects - .add(b'\r') - .add(b'\n') - .add(b'*') - .add(b'?'); - - fn encode_path(path: &str) -> String { - percent_encode(path.as_bytes(), INVALID).to_string() - } - - pub fn decode_path(path: &str) -> Result { - Ok(percent_decode_str(path).decode_utf8()?.to_string()) - } -} - #[cfg(test)] mod tests { use super::*; - use crate::kernel::PrimitiveType; - - #[test] - fn test_primitive() { - let types: PrimitiveType = serde_json::from_str("\"string\"").unwrap(); - println!("{types:?}"); - } #[test] fn test_deserialize_protocol() { diff --git a/crates/core/src/lib.rs b/crates/core/src/lib.rs index 81452d1fcd..83c245b3ba 100644 --- a/crates/core/src/lib.rs +++ b/crates/core/src/lib.rs @@ -92,9 +92,6 @@ pub mod logstore { } } -// Compatibility shim for DeltaConfig derive macro which expects crate::DeltaResult -pub type DeltaResult = crate::errors::DeltaResult; - #[cfg(any(test, feature = "integration_test"))] pub mod test_utils; diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index 38dfbebffb..fe13be8909 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -4074,7 +4074,7 @@ mod tests { assert_merge(table.clone(), metrics).await; // Just checking that the data wasn't actually written instead! - if let Ok(files) = crate::logstore::tests::flatten_list_stream( + if let Ok(files) = crate::test_utils::flatten_list_stream( &table.object_store(), Some(&object_store::path::Path::from("_change_data")), ) diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index 8ab6683352..d25c7b1e86 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -1257,7 +1257,7 @@ mod tests { // Too close for missiles, switching to guns. Just checking that the data wasn't actually // written instead! - if let Ok(files) = crate::logstore::tests::flatten_list_stream( + if let Ok(files) = crate::test_utils::flatten_list_stream( &table.object_store(), Some(&object_store::path::Path::from("_change_data")), ) diff --git a/crates/core/src/operations/write/writer.rs b/crates/core/src/operations/write/writer.rs index eb9098e666..b5eb42dab2 100644 --- a/crates/core/src/operations/write/writer.rs +++ b/crates/core/src/operations/write/writer.rs @@ -537,8 +537,8 @@ impl PartitionWriter { #[cfg(test)] mod tests { use super::*; - use crate::logstore::tests::flatten_list_stream as list; use crate::table::config::DEFAULT_NUM_INDEX_COLS; + use crate::test_utils::flatten_list_stream as list; use crate::writer::test_utils::*; use crate::DeltaTableBuilder; use arrow::array::{Int32Array, StringArray}; diff --git a/crates/core/src/protocol/mod.rs b/crates/core/src/protocol/mod.rs index 94b3e8338e..6069b99f1a 100644 --- a/crates/core/src/protocol/mod.rs +++ b/crates/core/src/protocol/mod.rs @@ -683,6 +683,35 @@ mod tests { serde_json::from_str(buf).expect("Expected to be able to deserialize"); } + #[test] + fn test_object_meta_from_add_action() { + let add = Add { + path: "x=A%252FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" + .to_string(), + size: 123, + modification_time: 123456789, + data_change: true, + stats: None, + partition_values: Default::default(), + tags: Default::default(), + base_row_id: None, + default_row_commit_version: None, + deletion_vector: None, + clustering_provider: None, + }; + + let meta: ObjectMeta = (&add).try_into_object_meta().unwrap(); + assert_eq!( + meta.location, + Path::parse( + "x=A%252FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" + ) + .unwrap() + ); + assert_eq!(meta.size, 123); + assert_eq!(meta.last_modified.timestamp_millis(), 123456789); + } + mod arrow_tests { use arrow::array::{self, ArrayRef, StructArray}; use arrow::compute::kernels::cast_utils::Parser; diff --git a/crates/core/src/table/builder.rs b/crates/core/src/table/builder.rs index 2e2ffc3f05..97e4fb0177 100644 --- a/crates/core/src/table/builder.rs +++ b/crates/core/src/table/builder.rs @@ -446,7 +446,7 @@ fn ensure_file_location_exists(path: PathBuf) -> DeltaResult<()> { #[cfg(test)] mod tests { use super::*; - use crate::logstore::factories::DefaultObjectStoreFactory; + use deltalake_logstore::factories::DefaultObjectStoreFactory; #[test] fn test_ensure_table_uri() { diff --git a/crates/core/src/table/mod.rs b/crates/core/src/table/mod.rs index 3c170e84a7..689424f125 100644 --- a/crates/core/src/table/mod.rs +++ b/crates/core/src/table/mod.rs @@ -419,7 +419,9 @@ impl std::fmt::Debug for DeltaTable { #[cfg(test)] mod tests { + use object_store::memory::InMemory; use pretty_assertions::assert_eq; + use std::sync::Arc; use tempfile::TempDir; use super::*; @@ -460,4 +462,28 @@ mod tests { .unwrap(); (dt, tmp_dir) } + + /// :w + #[tokio::test] + async fn test_peek_with_invalid_json() -> DeltaResult<()> { + let memory_store = Arc::new(InMemory::new()); + let log_path = Path::from("delta-table/_delta_log/00000000000000000001.json"); + + let log_content = r#"{invalid_json"#; + + memory_store + .put(&log_path, log_content.into()) + .await + .expect("Failed to write log file"); + + let table_uri = url::Url::parse("memory:///delta-table").unwrap(); + let table = crate::DeltaTableBuilder::from_uri(table_uri.clone()) + .unwrap() + .with_storage_backend(memory_store, table_uri) + .build()?; + + let result = table.log_store().peek_next_commit(0).await; + assert!(result.is_err()); + Ok(()) + } } diff --git a/crates/core/src/test_utils/mod.rs b/crates/core/src/test_utils/mod.rs index c00be04f22..9d14b485c3 100644 --- a/crates/core/src/test_utils/mod.rs +++ b/crates/core/src/test_utils/mod.rs @@ -2,6 +2,8 @@ mod factories; use std::{collections::HashMap, path::PathBuf, process::Command}; +use deltalake_logstore::Path; +use futures::TryStreamExt; use url::Url; pub use self::factories::*; @@ -9,6 +11,18 @@ use crate::{DeltaResult, DeltaTableBuilder}; pub type TestResult = Result>; +/// Collect list stream +pub async fn flatten_list_stream( + storage: &object_store::DynObjectStore, + prefix: Option<&Path>, +) -> object_store::Result> { + storage + .list(prefix) + .map_ok(|meta| meta.location) + .try_collect::>() + .await +} + /// Reference tables from the test data folder pub enum TestTables { Simple, diff --git a/crates/gcp/Cargo.toml b/crates/gcp/Cargo.toml index 1f1da670e3..3e48f9601f 100644 --- a/crates/gcp/Cargo.toml +++ b/crates/gcp/Cargo.toml @@ -27,6 +27,7 @@ url = { workspace = true } [dev-dependencies] chrono = { workspace = true } serial_test = "3" +deltalake-core = { path = "../core" } deltalake-test = { path = "../test" } pretty_env_logger = "0.5.0" rand = "0.8" diff --git a/crates/gcp/src/config.rs b/crates/gcp/src/config.rs index d3f53e220d..80eeb2d7dd 100644 --- a/crates/gcp/src/config.rs +++ b/crates/gcp/src/config.rs @@ -11,7 +11,7 @@ use std::sync::LazyLock; use object_store::gcp::GoogleConfigKey; use object_store::Error as ObjectStoreError; -use crate::error::Result; +use deltalake_logstore::LogStoreResult; static CREDENTIAL_KEYS: LazyLock> = LazyLock::new(|| { Vec::from_iter([ @@ -54,7 +54,7 @@ impl GcpConfigHelper { /// Create a new [`ConfigHelper`] pub fn try_new( config: impl IntoIterator, impl Into)>, - ) -> Result { + ) -> LogStoreResult { let mut env_config = HashMap::new(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { @@ -70,7 +70,7 @@ impl GcpConfigHelper { config: config .into_iter() .map(|(key, value)| Ok((GoogleConfigKey::from_str(key.as_ref())?, value.into()))) - .collect::>()?, + .collect::>()?, env_config, priority: Vec::from_iter([ GcpCredential::ServiceAccountKey, @@ -97,7 +97,7 @@ impl GcpConfigHelper { } /// Generate a configuration augmented with options from the environment - pub fn build(mut self) -> Result> { + pub fn build(mut self) -> LogStoreResult> { let mut has_credential = false; // try using only passed config options diff --git a/crates/gcp/src/error.rs b/crates/gcp/src/error.rs deleted file mode 100644 index acc18f67f9..0000000000 --- a/crates/gcp/src/error.rs +++ /dev/null @@ -1,22 +0,0 @@ -use deltalake_core::errors::DeltaTableError; - -pub(crate) type Result = std::result::Result; - -#[derive(thiserror::Error, Debug)] -pub(crate) enum Error { - #[allow(dead_code)] - #[error("failed to parse config: {0}")] - Parse(String), - - #[error(transparent)] - ObjectStore(#[from] object_store::Error), -} - -impl From for DeltaTableError { - fn from(e: Error) -> Self { - match e { - Error::Parse(msg) => DeltaTableError::Generic(msg), - Error::ObjectStore(e) => DeltaTableError::ObjectStore { source: e }, - } - } -} diff --git a/crates/gcp/src/lib.rs b/crates/gcp/src/lib.rs index ac4052ba84..fe9a6c8e55 100644 --- a/crates/gcp/src/lib.rs +++ b/crates/gcp/src/lib.rs @@ -13,7 +13,6 @@ use object_store::client::SpawnedReqwestConnector; use url::Url; mod config; -pub mod error; mod storage; trait GcpOptions { @@ -51,11 +50,10 @@ impl ObjectStoreFactory for GcpFactory { } let config = config::GcpConfigHelper::try_new(config.raw.as_gcp_options())?.build()?; - let (_, path) = - ObjectStoreScheme::parse(url).map_err(|e| DeltaTableError::GenericError { - source: Box::new(e), - })?; - let prefix = Path::parse(path)?; + let (_, path) = ObjectStoreScheme::parse(url) + .map_err(|e| LogStoreError::Generic { source: e.into() })?; + let prefix = + Path::parse(path).map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; for (key, value) in config.iter() { builder = builder.with_config(*key, value.clone()); diff --git a/crates/gcp/src/storage.rs b/crates/gcp/src/storage.rs index cc845b65fb..600373e185 100644 --- a/crates/gcp/src/storage.rs +++ b/crates/gcp/src/storage.rs @@ -1,8 +1,8 @@ //! GCP GCS storage backend. use bytes::Bytes; -use deltalake_core::logstore::ObjectStoreRef; -use deltalake_core::Path; +use deltalake_logstore::ObjectStoreRef; +use deltalake_logstore::Path; use futures::stream::BoxStream; use object_store::{MultipartUpload, PutMultipartOptions, PutPayload}; use std::ops::Range; diff --git a/crates/hdfs/src/lib.rs b/crates/hdfs/src/lib.rs index 889ed89f11..10e76a67fa 100644 --- a/crates/hdfs/src/lib.rs +++ b/crates/hdfs/src/lib.rs @@ -27,7 +27,8 @@ impl ObjectStoreFactory for HdfsFactory { let store = Arc::new(builder.build()?); - let prefix = Path::parse(url.path())?; + let prefix = + Path::parse(url.path()).map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; Ok((store, prefix)) } } diff --git a/crates/lakefs/Cargo.toml b/crates/lakefs/Cargo.toml index 0267d13aff..3cf7a639c3 100644 --- a/crates/lakefs/Cargo.toml +++ b/crates/lakefs/Cargo.toml @@ -13,6 +13,7 @@ rust-version.workspace = true [dependencies] deltalake-core = { version = "0.29.0", path = "../core", features = ["cloud"]} +deltalake-logstore = { version = "0.29.0", path = "../logstore" } # workspace dependencies async-trait = { workspace = true } bytes = { workspace = true } diff --git a/crates/lakefs/src/client.rs b/crates/lakefs/src/client.rs index dfcf581633..8af07a2e1e 100644 --- a/crates/lakefs/src/client.rs +++ b/crates/lakefs/src/client.rs @@ -1,6 +1,7 @@ use dashmap::DashMap; use deltalake_core::kernel::transaction::TransactionError; use deltalake_core::DeltaResult; +use deltalake_logstore::LogStoreResult; use reqwest::Client; use reqwest::StatusCode; use serde::Deserialize; @@ -278,7 +279,7 @@ impl LakeFSClient { debug!("{}", format!("LakeFS Transaction `{id}` has been set.")); } - pub fn get_transaction(&self, id: Uuid) -> Result { + pub fn get_transaction(&self, id: Uuid) -> LogStoreResult { let transaction_branch = self .transactions .get(&id) diff --git a/crates/lakefs/src/errors.rs b/crates/lakefs/src/errors.rs index ff994f8c1a..031ddd3649 100644 --- a/crates/lakefs/src/errors.rs +++ b/crates/lakefs/src/errors.rs @@ -2,6 +2,7 @@ use deltalake_core::kernel::transaction::TransactionError; use deltalake_core::DeltaTableError; +use deltalake_logstore::LogStoreError; use reqwest::Error; #[derive(thiserror::Error, Debug)] @@ -77,3 +78,11 @@ impl From for DeltaTableError { } } } + +impl From for LogStoreError { + fn from(err: LakeFSConfigError) -> Self { + LogStoreError::Generic { + source: Box::new(err), + } + } +} diff --git a/crates/lakefs/src/lib.rs b/crates/lakefs/src/lib.rs index 0b75cb986d..11704ce728 100644 --- a/crates/lakefs/src/lib.rs +++ b/crates/lakefs/src/lib.rs @@ -8,9 +8,8 @@ pub mod errors; pub mod execute; pub mod logstore; pub mod storage; -use deltalake_core::logstore::{logstore_factories, LogStore, LogStoreFactory}; +use deltalake_core::logstore::{logstore_factories, LogStore, LogStoreFactory, LogStoreResult}; use deltalake_core::logstore::{object_store_factories, ObjectStoreRef, StorageConfig}; -use deltalake_core::DeltaResult; pub use execute::LakeFSCustomExecuteHandler; use logstore::lakefs_logstore; use std::sync::Arc; @@ -31,7 +30,7 @@ impl LogStoreFactory for LakeFSLogStoreFactory { root_store: ObjectStoreRef, location: &Url, config: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { let options = StorageConfig::parse_options(self.with_env_s3(&config.raw.clone()))?; debug!("LakeFSLogStoreFactory has been asked to create a LogStore"); lakefs_logstore(prefixed_store, root_store, location, &options) diff --git a/crates/lakefs/src/logstore.rs b/crates/lakefs/src/logstore.rs index 515d6d85ff..4235cab0ad 100644 --- a/crates/lakefs/src/logstore.rs +++ b/crates/lakefs/src/logstore.rs @@ -9,6 +9,7 @@ use deltalake_core::{ kernel::transaction::TransactionError, logstore::ObjectStoreRef, DeltaResult, }; use deltalake_core::{logstore::*, DeltaTableError}; +use deltalake_logstore::LogStoreError; use object_store::{Error as ObjectStoreError, ObjectStore, PutOptions}; use tracing::debug; use url::Url; @@ -24,7 +25,7 @@ pub fn lakefs_logstore( root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, -) -> DeltaResult> { +) -> LogStoreResult> { let host = options .raw .get("aws_endpoint") @@ -92,17 +93,17 @@ impl LakeFSLogStore { /// Build a new object store for an URL using the existing storage options. After /// branch creation a new object store needs to be created for the branch uri - fn build_new_store(&self, url: &Url) -> DeltaResult { + fn build_new_store(&self, url: &Url) -> LogStoreResult { // turn location into scheme let scheme = Url::parse(&format!("{}://", url.scheme())) - .map_err(|_| DeltaTableError::InvalidTableLocation(url.clone().into()))?; + .map_err(|_| LogStoreError::InvalidTableLocation(url.clone().into()))?; if let Some(entry) = self.config().object_store_factory().get(&scheme) { debug!("Creating new storage with storage provider for {scheme} ({url})"); let (store, _prefix) = entry.value().parse_url_opts(url, &self.config().options)?; return Ok(store); } - Err(DeltaTableError::InvalidTableLocation(url.to_string())) + Err(LogStoreError::InvalidTableLocation(url.to_string())) } fn register_object_store(&self, url: &Url, store: ObjectStoreRef) { @@ -113,19 +114,22 @@ impl LakeFSLogStore { self.root_registry.register_store(url, store); } - fn get_transaction_url(&self, operation_id: Uuid, base: String) -> DeltaResult { + fn get_transaction_url(&self, operation_id: Uuid, base: String) -> LogStoreResult { let (repo, _, table) = self.client.decompose_url(base); - let string_url = format!( - "lakefs://{repo}/{}/{table}", - self.client.get_transaction(operation_id)?, - ); + let transaction_id = + self.client + .get_transaction(operation_id) + .map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })?; + let string_url = format!("lakefs://{repo}/{}/{table}", transaction_id,); Ok(Url::parse(&string_url).unwrap()) } fn get_transaction_objectstore( &self, operation_id: Uuid, - ) -> DeltaResult<(String, ObjectStoreRef, ObjectStoreRef)> { + ) -> LogStoreResult<(String, ObjectStoreRef, ObjectStoreRef)> { let transaction_url = self.get_transaction_url(operation_id, self.config.location.to_string())?; Ok(( @@ -233,7 +237,7 @@ impl LogStore for LakeFSLogStore { "LakeFSLogStore".into() } - async fn read_commit_entry(&self, version: i64) -> DeltaResult> { + async fn read_commit_entry(&self, version: i64) -> LogStoreResult> { read_commit_entry( &self.prefixed_registry.get_store(&self.config.location)?, version, @@ -251,12 +255,12 @@ impl LogStore for LakeFSLogStore { version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { let (transaction_url, store, _root_store) = self .get_transaction_objectstore(operation_id) - .map_err(|e| TransactionError::LogStoreError { + .map_err(|e| LogStoreError::LogStoreError { msg: e.to_string(), - source: Box::new(e), + source: e.into(), })?; match commit_or_bytes { @@ -269,12 +273,12 @@ impl LogStore for LakeFSLogStore { put_options().clone(), ) .await - .map_err(|err| -> TransactionError { + .map_err(|err| -> LogStoreError { match err { ObjectStoreError::AlreadyExists { .. } => { - TransactionError::VersionAlreadyExists(version) + LogStoreError::VersionAlreadyExists(version) } - _ => TransactionError::from(err), + _ => LogStoreError::ObjectStore { source: err }, } })?; @@ -288,7 +292,7 @@ impl LogStore for LakeFSLogStore { false, ) .await - .map_err(|e| TransactionError::LogStoreError { + .map_err(|e| LogStoreError::LogStoreError { msg: e.to_string(), source: Box::new(e), })?; @@ -296,12 +300,16 @@ impl LogStore for LakeFSLogStore { // Try LakeFS Branch merge of transaction branch in source branch let (repo, target_branch, table) = self.client.decompose_url(self.config.location.to_string()); + let transaction_id = self + .client + .get_transaction(operation_id) + .map_err(LogStoreError::from)?; match self .client .merge( repo, target_branch, - self.client.get_transaction(operation_id)?, + transaction_id, version, format!("Finished deltalake transaction {{ table: {table}, version: {version} }}"), false, @@ -313,10 +321,10 @@ impl LogStore for LakeFSLogStore { store .delete(&commit_uri_from_version(version)) .await - .map_err(TransactionError::from)?; - return Err(TransactionError::VersionAlreadyExists(version)); + .map_err(|e| LogStoreError::ObjectStore { source: e })?; + return Err(LogStoreError::VersionAlreadyExists(version)); } - Err(err) => Err(err), + Err(err) => Err(LogStoreError::from(err)), }?; } _ => unreachable!(), // Default log store should never get a tmp_commit, since this is for conditional put stores @@ -329,13 +337,18 @@ impl LogStore for LakeFSLogStore { _version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), TransactionError> { + ) -> Result<(), LogStoreError> { match &commit_or_bytes { CommitOrBytes::LogBytes(_) => { let (repo, _, _) = self.client.decompose_url(self.config.location.to_string()); + let transaction_id = self + .client + .get_transaction(operation_id) + .map_err(LogStoreError::from)?; self.client - .delete_branch(repo, self.client.get_transaction(operation_id)?) - .await?; + .delete_branch(repo, transaction_id) + .await + .map_err(|e| LogStoreError::from(DeltaTableError::from(e)))?; self.client.clear_transaction(operation_id); Ok(()) } @@ -343,7 +356,7 @@ impl LogStore for LakeFSLogStore { } } - async fn get_latest_version(&self, current_version: i64) -> DeltaResult { + async fn get_latest_version(&self, current_version: i64) -> LogStoreResult { get_latest_version(self, current_version).await } @@ -374,7 +387,7 @@ impl LogStore for LakeFSLogStore { &self.config } - fn transaction_url(&self, operation_id: Uuid, base: &Url) -> DeltaResult { + fn transaction_url(&self, operation_id: Uuid, base: &Url) -> LogStoreResult { self.get_transaction_url(operation_id, base.to_string()) } } diff --git a/crates/lakefs/src/storage.rs b/crates/lakefs/src/storage.rs index e49cc87bcd..c001670bac 100644 --- a/crates/lakefs/src/storage.rs +++ b/crates/lakefs/src/storage.rs @@ -1,8 +1,10 @@ //! LakeFS storage backend (internally S3). use deltalake_core::logstore::object_store::aws::AmazonS3ConfigKey; -use deltalake_core::logstore::{ObjectStoreFactory, ObjectStoreRef, StorageConfig}; -use deltalake_core::{DeltaResult, DeltaTableError, Path}; +use deltalake_core::logstore::{ + LogStoreError, LogStoreResult, ObjectStoreFactory, ObjectStoreRef, StorageConfig, +}; +use deltalake_core::{DeltaTableError, Path}; use object_store::aws::AmazonS3Builder; use object_store::client::SpawnedReqwestConnector; use object_store::ObjectStoreScheme; @@ -62,11 +64,11 @@ impl ObjectStoreFactory for LakeFSObjectStoreFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { // Convert LakeFS URI to equivalent S3 URI. let s3_url = url.to_string().replace("lakefs://", "s3://"); let s3_url = Url::parse(&s3_url) - .map_err(|_| DeltaTableError::InvalidTableLocation(url.clone().into()))?; + .map_err(|_| LogStoreError::InvalidTableLocation(url.to_string()))?; let mut builder = AmazonS3Builder::new().with_url(s3_url.to_string()); // All S3-likes should start their builder the same way @@ -89,11 +91,9 @@ impl ObjectStoreFactory for LakeFSObjectStoreFactory { }) .collect::>(); - let (_, path) = - ObjectStoreScheme::parse(&s3_url).map_err(|e| DeltaTableError::GenericError { - source: Box::new(e), - })?; - let prefix = Path::parse(path)?; + let (_, path) = ObjectStoreScheme::parse(&s3_url) + .map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; + let prefix = Path::parse(path).map_err(|e| LogStoreError::Generic { source: e.into() })?; for (key, value) in config.iter() { builder = builder.with_config(*key, value.clone()); diff --git a/crates/logstore/Cargo.toml b/crates/logstore/Cargo.toml index c53bcd1b5e..f0075b886c 100644 --- a/crates/logstore/Cargo.toml +++ b/crates/logstore/Cargo.toml @@ -18,7 +18,6 @@ deltalake-types = { version = "0.29.2", path = "../types" } delta_kernel.workspace = true # arrow -arrow = { workspace = true } bytes = { workspace = true } chrono = { workspace = true, default-features = false, features = ["clock"] } @@ -50,6 +49,7 @@ humantime = { version = "2.1.0", optional = true } deltalake-test = { path = "../test" } tokio = { version = "1", features = ["macros", "rt-multi-thread"] } tracing-subscriber = { version = "0.3", features = ["env-filter"] } +rstest = { workspace = true } [features] default = ["rustls"] diff --git a/crates/logstore/src/factories.rs b/crates/logstore/src/factories.rs index 2eb33fd0f3..86b8f5ac67 100644 --- a/crates/logstore/src/factories.rs +++ b/crates/logstore/src/factories.rs @@ -31,7 +31,7 @@ pub trait ObjectStoreFactory: Send + Sync { } #[derive(Clone, Debug, Default)] -pub(crate) struct DefaultObjectStoreFactory {} +pub struct DefaultObjectStoreFactory {} impl ObjectStoreFactory for DefaultObjectStoreFactory { fn parse_url_opts( diff --git a/crates/logstore/src/lib.rs b/crates/logstore/src/lib.rs index 637707cf59..01d847739d 100644 --- a/crates/logstore/src/lib.rs +++ b/crates/logstore/src/lib.rs @@ -7,11 +7,11 @@ //! and access patterns to optimize our operations in terms of cost and performance. pub mod config; -pub(crate) mod default_logstore; +pub mod default_logstore; pub mod error; -pub(crate) mod factories; +pub mod factories; mod logstore_impl; -pub(crate) mod storage; +pub mod storage; // Compatibility shim for DeltaConfig derive macro which expects crate::logstore::config pub mod logstore { diff --git a/crates/logstore/src/logstore_impl.rs b/crates/logstore/src/logstore_impl.rs index 9d8920fb78..059a373a5b 100644 --- a/crates/logstore/src/logstore_impl.rs +++ b/crates/logstore/src/logstore_impl.rs @@ -925,43 +925,6 @@ pub(crate) mod tests { .expect("Failed to identify table")); } - /// :w - #[tokio::test] - async fn test_peek_with_invalid_json() -> LogStoreResult<()> { - use deltalake_logstore::object_store::memory::InMemory; - let memory_store = Arc::new(InMemory::new()); - let log_path = Path::from("delta-table/_delta_log/00000000000000000001.json"); - - let log_content = r#"{invalid_json"#; - - memory_store - .put(&log_path, log_content.into()) - .await - .expect("Failed to write log file"); - - let table_uri = url::Url::parse("memory:///delta-table").unwrap(); - let table = crate::DeltaTableBuilder::from_uri(table_uri.clone()) - .unwrap() - .with_storage_backend(memory_store, table_uri) - .build()?; - - let result = table.log_store().peek_next_commit(0).await; - assert!(result.is_err()); - Ok(()) - } - - /// Collect list stream - pub(crate) async fn flatten_list_stream( - storage: &object_store::DynObjectStore, - prefix: Option<&Path>, - ) -> object_store::Result> { - storage - .list(prefix) - .map_ok(|meta| meta.location) - .try_collect::>() - .await - } - #[tokio::test] async fn test_get_actions_malformed_json() { use super::*; diff --git a/crates/logstore/src/storage/mod.rs b/crates/logstore/src/storage/mod.rs index f4d12d0400..848cc1a8d5 100644 --- a/crates/logstore/src/storage/mod.rs +++ b/crates/logstore/src/storage/mod.rs @@ -102,7 +102,7 @@ mod tests { use rstest::*; use super::*; - use deltalake_logstore::config::TryUpdateKey; + use crate::config::TryUpdateKey; // Note: test_utils::with_env removed - this requires core // For testing, use environment variables directly or mock them diff --git a/crates/logstore/src/storage/utils.rs b/crates/logstore/src/storage/utils.rs index b26a58c5f4..aa50db5584 100644 --- a/crates/logstore/src/storage/utils.rs +++ b/crates/logstore/src/storage/utils.rs @@ -14,40 +14,3 @@ pub fn commit_uri_from_version(version: i64) -> Path { let version = format!("{version:020}.json"); crate::DELTA_LOG_PATH.child(version.as_str()) } - -// Note: Add type conversion removed - this requires core's Action type -// For logstore usage, work with ObjectMeta directly - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_object_meta_from_add_action() { - let add = Add { - path: "x=A%252FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" - .to_string(), - size: 123, - modification_time: 123456789, - data_change: true, - stats: None, - partition_values: Default::default(), - tags: Default::default(), - base_row_id: None, - default_row_commit_version: None, - deletion_vector: None, - clustering_provider: None, - }; - - let meta: ObjectMeta = (&add).try_into().unwrap(); - assert_eq!( - meta.location, - Path::parse( - "x=A%252FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" - ) - .unwrap() - ); - assert_eq!(meta.size, 123); - assert_eq!(meta.last_modified.timestamp_millis(), 123456789); - } -} diff --git a/crates/mount/Cargo.toml b/crates/mount/Cargo.toml index af2457bf13..91bae9f2a1 100644 --- a/crates/mount/Cargo.toml +++ b/crates/mount/Cargo.toml @@ -12,7 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.29.0", path = "../core" } +deltalake-logstore = { version = "0.29.0", path = "../logstore" } errno = "0.3" # workspace depenndecies diff --git a/crates/mount/src/error.rs b/crates/mount/src/error.rs index 3693b0be07..090ba304ff 100644 --- a/crates/mount/src/error.rs +++ b/crates/mount/src/error.rs @@ -1,4 +1,4 @@ -use deltalake_core::errors::DeltaTableError; +use deltalake_logstore::LogStoreError; pub(crate) type Result = std::result::Result; @@ -19,15 +19,22 @@ pub enum Error { ObjectStore(#[from] object_store::Error), } -impl From for DeltaTableError { +impl From for LogStoreError { fn from(e: Error) -> Self { match e { - Error::Parse(msg) => DeltaTableError::Generic(msg), - Error::UnknownConfigKey(msg) => DeltaTableError::Generic(msg), - Error::AllowUnsafeRenameNotSpecified => DeltaTableError::Generic( - "The `allow_unsafe_rename` parameter must be specified".to_string(), - ), - Error::ObjectStore(e) => DeltaTableError::ObjectStore { source: e }, + Error::Parse(msg) => LogStoreError::Generic { + source: Box::new(std::io::Error::new(std::io::ErrorKind::InvalidInput, msg)), + }, + Error::UnknownConfigKey(msg) => LogStoreError::Generic { + source: Box::new(std::io::Error::new(std::io::ErrorKind::InvalidInput, msg)), + }, + Error::AllowUnsafeRenameNotSpecified => LogStoreError::Generic { + source: Box::new(std::io::Error::new( + std::io::ErrorKind::InvalidInput, + "The `allow_unsafe_rename` parameter must be specified", + )), + }, + Error::ObjectStore(e) => LogStoreError::ObjectStore { source: e }, } } } diff --git a/crates/mount/src/lib.rs b/crates/mount/src/lib.rs index 04b316696a..a31ac27a1b 100644 --- a/crates/mount/src/lib.rs +++ b/crates/mount/src/lib.rs @@ -2,12 +2,12 @@ use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; -use deltalake_core::logstore::DeltaIOStorageBackend; -use deltalake_core::logstore::{ +use deltalake_logstore::Path; +use deltalake_logstore::{ config::str_is_truthy, default_logstore, logstore_factories, object_store_factories, LogStore, LogStoreFactory, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; -use deltalake_core::{DeltaResult, DeltaTableError, Path}; +use deltalake_logstore::{DeltaIOStorageBackend, LogStoreError, LogStoreResult}; use object_store::local::LocalFileSystem; use object_store::DynObjectStore; use url::Url; @@ -41,7 +41,7 @@ impl ObjectStoreFactory for MountFactory { &self, url: &Url, config: &StorageConfig, - ) -> DeltaResult<(ObjectStoreRef, Path)> { + ) -> LogStoreResult<(ObjectStoreRef, Path)> { let mount_config = config::MountConfigHelper::try_new(config.raw.as_mount_options())?.build()?; @@ -51,31 +51,41 @@ impl ObjectStoreFactory for MountFactory { .unwrap_or(&String::new()), ); - let (mut store, prefix) = match url.scheme() { - "dbfs" => { - if !allow_unsafe_rename { - // Just let the user know that they need to set the allow_unsafe_rename option - return Err(error::Error::AllowUnsafeRenameNotSpecified.into()); - } - // We need to convert the dbfs url to a file url - Url::parse(&format!("file:///dbfs{}", url.path())).unwrap(); - let store = Arc::new(file::MountFileStorageBackend::try_new()?) as ObjectStoreRef; - Ok((store, Path::from("/"))) - } - "file" => { - if allow_unsafe_rename { + let (mut store, prefix) = + match url.scheme() { + "dbfs" => { + if !allow_unsafe_rename { + // Just let the user know that they need to set the allow_unsafe_rename option + return Err(error::Error::AllowUnsafeRenameNotSpecified.into()); + } + // We need to convert the dbfs url to a file url + Url::parse(&format!("file:///dbfs{}", url.path())).unwrap(); let store = Arc::new(file::MountFileStorageBackend::try_new()?) as ObjectStoreRef; - let prefix = Path::from_filesystem_path(url.to_file_path().unwrap())?; - Ok((store, prefix)) - } else { - let store = Arc::new(LocalFileSystem::new()) as ObjectStoreRef; - let prefix = Path::from_filesystem_path(url.to_file_path().unwrap())?; - Ok((store, prefix)) + Ok((store, Path::from("/"))) + } + "file" => { + if allow_unsafe_rename { + let store = + Arc::new(file::MountFileStorageBackend::try_new()?) as ObjectStoreRef; + let prefix = Path::from_filesystem_path(url.to_file_path().unwrap()) + .map_err(|e| LogStoreError::InvalidPath { + path: url.to_string(), + source: Box::new(e), + })?; + Ok((store, prefix)) + } else { + let store = Arc::new(LocalFileSystem::new()) as ObjectStoreRef; + let prefix = Path::from_filesystem_path(url.to_file_path().unwrap()) + .map_err(|e| LogStoreError::InvalidPath { + path: url.to_string(), + source: Box::new(e), + })?; + Ok((store, prefix)) + } } - } - _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), - }?; + _ => Err(LogStoreError::InvalidTableLocation(url.to_string())), + }?; if let Some(runtime) = &config.runtime { store = @@ -92,7 +102,7 @@ impl LogStoreFactory for MountFactory { root_store: ObjectStoreRef, location: &Url, options: &StorageConfig, - ) -> DeltaResult> { + ) -> LogStoreResult> { Ok(default_logstore( prefixed_store, root_store, diff --git a/crates/types/Cargo.toml b/crates/types/Cargo.toml index c62ba94aee..75187794e7 100644 --- a/crates/types/Cargo.toml +++ b/crates/types/Cargo.toml @@ -17,7 +17,6 @@ serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true } percent-encoding = "2" strum = { workspace = true } -object_store = { workspace = true } [dev-dependencies] serde_json = { workspace = true } From 5ffbdb7aefe04f03da7f2f68c9406a747aadf04a Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 09:40:27 -0500 Subject: [PATCH 03/10] Minimize public API breakage Signed-off-by: Abhi Agarwal --- crates/core/src/kernel/schema/partitions.rs | 2 +- crates/core/src/kernel/schema/schema.rs | 2 +- .../core/src/kernel/transaction/conflict_checker.rs | 3 +-- crates/core/src/lib.rs | 7 +------ crates/core/src/operations/load_cdf.rs | 3 +-- crates/core/src/operations/write/mod.rs | 2 +- crates/core/src/table/builder.rs | 2 +- crates/core/src/table/mod.rs | 13 ++++++++++--- 8 files changed, 17 insertions(+), 17 deletions(-) diff --git a/crates/core/src/kernel/schema/partitions.rs b/crates/core/src/kernel/schema/partitions.rs index b0d11a28d8..7a392b4276 100644 --- a/crates/core/src/kernel/schema/partitions.rs +++ b/crates/core/src/kernel/schema/partitions.rs @@ -184,7 +184,7 @@ impl TryFrom<&str> for DeltaTablePartition { } #[allow(unused)] // TODO: remove once we use this in kernel log replay -pub fn to_kernel_predicate( +pub(crate) fn to_kernel_predicate( filters: &[PartitionFilter], table_schema: &StructType, ) -> DeltaResult { diff --git a/crates/core/src/kernel/schema/schema.rs b/crates/core/src/kernel/schema/schema.rs index cc586caf6a..0450124aed 100644 --- a/crates/core/src/kernel/schema/schema.rs +++ b/crates/core/src/kernel/schema/schema.rs @@ -10,7 +10,7 @@ pub use delta_kernel::schema::{ use serde_json::Value; use crate::kernel::error::Error; -use crate::kernel::schema::DataCheck; +use crate::schema::DataCheck; use crate::table::GeneratedColumn; /// Type alias for a top level schema diff --git a/crates/core/src/kernel/transaction/conflict_checker.rs b/crates/core/src/kernel/transaction/conflict_checker.rs index cbf4215394..4a4868e92e 100644 --- a/crates/core/src/kernel/transaction/conflict_checker.rs +++ b/crates/core/src/kernel/transaction/conflict_checker.rs @@ -7,9 +7,8 @@ use super::CommitInfo; #[cfg(feature = "datafusion")] use crate::delta_datafusion::DataFusionMixins; use crate::errors::DeltaResult; -use crate::kernel::get_actions; use crate::kernel::{Action, Add, LogDataHandler, Metadata, Protocol, Remove, Transaction}; -use crate::logstore::LogStore; +use crate::logstore::{get_actions, LogStore}; use crate::protocol::DeltaOperation; use crate::table::config::TablePropertiesExt as _; use crate::DeltaTableError; diff --git a/crates/core/src/lib.rs b/crates/core/src/lib.rs index 83c245b3ba..06ac4cc696 100644 --- a/crates/core/src/lib.rs +++ b/crates/core/src/lib.rs @@ -85,12 +85,7 @@ pub mod operations; pub mod protocol; pub use kernel::schema; pub mod table; -pub mod logstore { - pub use deltalake_logstore::*; - pub mod default_logstore { - pub use deltalake_logstore::DefaultLogStore; - } -} +pub use deltalake_logstore as logstore; #[cfg(any(test, feature = "integration_test"))] pub mod test_utils; diff --git a/crates/core/src/operations/load_cdf.rs b/crates/core/src/operations/load_cdf.rs index 3183a9fc6b..8495a0eb16 100644 --- a/crates/core/src/operations/load_cdf.rs +++ b/crates/core/src/operations/load_cdf.rs @@ -33,10 +33,9 @@ use tracing::log; use crate::delta_datafusion::{register_store, DataFusionMixins, DatafusionLogStore}; use crate::errors::DeltaResult; -use crate::kernel::get_actions; use crate::kernel::transaction::PROTOCOL; use crate::kernel::{resolve_snapshot, Action, Add, AddCDCFile, CommitInfo, EagerSnapshot}; -use crate::logstore::LogStoreRef; +use crate::logstore::{get_actions, LogStoreRef}; use crate::DeltaTableError; use crate::{delta_datafusion::cdf::*, kernel::Remove}; diff --git a/crates/core/src/operations/write/mod.rs b/crates/core/src/operations/write/mod.rs index c578d48b73..48145ff2ec 100644 --- a/crates/core/src/operations/write/mod.rs +++ b/crates/core/src/operations/write/mod.rs @@ -757,7 +757,7 @@ impl std::future::IntoFuture for WriteBuilder { mod tests { use super::*; use crate::ensure_table_uri; - use crate::kernel::get_actions; + use crate::logstore::get_actions; use crate::kernel::CommitInfo; use crate::operations::load_cdf::collect_batches; use crate::operations::{collect_sendable_stream, DeltaOps}; diff --git a/crates/core/src/table/builder.rs b/crates/core/src/table/builder.rs index 97e4fb0177..d977a04d09 100644 --- a/crates/core/src/table/builder.rs +++ b/crates/core/src/table/builder.rs @@ -11,7 +11,7 @@ use serde::{Deserialize, Serialize}; use tracing::debug; use url::Url; -use crate::logstore::IORuntime; +use crate::logstore::storage::IORuntime; use crate::logstore::{object_store_factories, LogStoreRef, StorageConfig}; use crate::{DeltaResult, DeltaTable, DeltaTableError}; diff --git a/crates/core/src/table/mod.rs b/crates/core/src/table/mod.rs index 689424f125..469a188316 100644 --- a/crates/core/src/table/mod.rs +++ b/crates/core/src/table/mod.rs @@ -5,6 +5,7 @@ use std::fmt; use std::fmt::Formatter; use chrono::{DateTime, Utc}; +use deltalake_logstore::LogStoreError; use futures::stream::BoxStream; use futures::{StreamExt, TryStreamExt}; use object_store::{path::Path, ObjectStore}; @@ -14,12 +15,12 @@ use serde::{Deserialize, Deserializer, Serialize, Serializer}; use self::builder::DeltaTableConfig; use self::state::DeltaTableState; -use crate::kernel::schema::partitions::PartitionFilter; use crate::kernel::{CommitInfo, DataCheck, LogicalFileView}; use crate::logstore::{ commit_uri_from_version, extract_version_from_filename, LogStoreConfig, LogStoreExt, LogStoreRef, ObjectStoreRef, }; +use crate::partitions::PartitionFilter; use crate::{DeltaResult, DeltaTableError}; // NOTE: this use can go away when peek_next_commit is removed off of [DeltaTable] @@ -369,9 +370,15 @@ impl DeltaTable { let mut max_version = match self .log_store .get_latest_version(self.version().unwrap_or(min_version)) - .await? + .await { - version => version, + Ok(version) => version, + Err(LogStoreError::InvalidVersion(_)) => { + return Err(DeltaTableError::NotATable( + log_store.table_root_url().to_string(), + )) + } + Err(e) => return Err(e.into()), }; let mut version = min_version; let lowest_table_version = min_version; From 53959e283c6b50bac6e0f8484eccd23edd8fa94f Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 09:44:40 -0500 Subject: [PATCH 04/10] Remove unused deps Signed-off-by: Abhi Agarwal --- crates/core/Cargo.toml | 1 - crates/logstore/Cargo.toml | 2 -- 2 files changed, 3 deletions(-) diff --git a/crates/core/Cargo.toml b/crates/core/Cargo.toml index c77c1e42a2..7292824426 100644 --- a/crates/core/Cargo.toml +++ b/crates/core/Cargo.toml @@ -43,7 +43,6 @@ datafusion-proto = { workspace = true, optional = true } # serde serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true } -strum = { workspace = true } # "stdlib" bytes = { workspace = true } diff --git a/crates/logstore/Cargo.toml b/crates/logstore/Cargo.toml index f0075b886c..c1311ab52c 100644 --- a/crates/logstore/Cargo.toml +++ b/crates/logstore/Cargo.toml @@ -17,9 +17,7 @@ deltalake-types = { version = "0.29.2", path = "../types" } delta_kernel.workspace = true -# arrow bytes = { workspace = true } -chrono = { workspace = true, default-features = false, features = ["clock"] } # serde serde = { workspace = true, features = ["derive"] } From 7449230c59c332f553aeb519ca76f75adad2efa2 Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 09:52:44 -0500 Subject: [PATCH 05/10] Fix LakeFS Signed-off-by: Abhi Agarwal --- crates/lakefs/src/client.rs | 15 +++++++-------- crates/lakefs/src/errors.rs | 8 ++++++++ crates/lakefs/src/execute.rs | 2 +- crates/lakefs/src/logstore.rs | 20 ++++++++------------ 4 files changed, 24 insertions(+), 21 deletions(-) diff --git a/crates/lakefs/src/client.rs b/crates/lakefs/src/client.rs index 8af07a2e1e..593409b1a5 100644 --- a/crates/lakefs/src/client.rs +++ b/crates/lakefs/src/client.rs @@ -1,6 +1,5 @@ use dashmap::DashMap; -use deltalake_core::kernel::transaction::TransactionError; -use deltalake_core::DeltaResult; +use deltalake_logstore::LogStoreError; use deltalake_logstore::LogStoreResult; use reqwest::Client; use reqwest::StatusCode; @@ -53,7 +52,7 @@ impl LakeFSClient { &self, source_url: &Url, operation_id: Uuid, - ) -> DeltaResult<(Url, String)> { + ) -> LogStoreResult<(Url, String)> { let (repo, source_branch, table) = self.decompose_url(source_url.to_string()); let request_url = format!("{}/api/v1/repositories/{repo}/branches", self.config.host); @@ -101,7 +100,7 @@ impl LakeFSClient { &self, repo: String, branch: String, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { let request_url = format!( "{}/api/v1/repositories/{repo}/branches/{branch}", self.config.host, @@ -138,7 +137,7 @@ impl LakeFSClient { branch: String, commit_message: String, allow_empty: bool, - ) -> DeltaResult<()> { + ) -> LogStoreResult<()> { let request_url = format!( "{}/api/v1/repositories/{repo}/branches/{branch}/commits", self.config.host, @@ -184,7 +183,7 @@ impl LakeFSClient { commit_version: i64, commit_message: String, allow_empty: bool, - ) -> Result<(), TransactionError> { + ) -> LogStoreResult<()> { let request_url = format!( "{}/api/v1/repositories/{repo}/refs/{transaction_branch}/merge/{target_branch}", self.config.host, @@ -209,7 +208,7 @@ impl LakeFSClient { // Handle the response; match response.status() { StatusCode::OK => Ok(()), - StatusCode::CONFLICT => Err(TransactionError::VersionAlreadyExists(commit_version)), + StatusCode::CONFLICT => Err(LogStoreError::VersionAlreadyExists(commit_version)), StatusCode::UNAUTHORIZED => Err(LakeFSOperationError::UnauthorizedAction.into()), _ => { let error: LakeFSErrorResponse = @@ -229,7 +228,7 @@ impl LakeFSClient { repo: &str, base_branch: &str, compare_branch: &str, - ) -> Result { + ) -> LogStoreResult { let request_url = format!( "{}/api/v1/repositories/{repo}/refs/{base_branch}/diff/{compare_branch}", self.config.host diff --git a/crates/lakefs/src/errors.rs b/crates/lakefs/src/errors.rs index 031ddd3649..c2669035ba 100644 --- a/crates/lakefs/src/errors.rs +++ b/crates/lakefs/src/errors.rs @@ -71,6 +71,14 @@ impl From for DeltaTableError { } } +impl From for LogStoreError { + fn from(err: LakeFSOperationError) -> Self { + LogStoreError::Generic { + source: Box::new(err), + } + } +} + impl From for DeltaTableError { fn from(err: LakeFSConfigError) -> Self { DeltaTableError::GenericError { diff --git a/crates/lakefs/src/execute.rs b/crates/lakefs/src/execute.rs index 24fae5cab5..7d7ca3df29 100644 --- a/crates/lakefs/src/execute.rs +++ b/crates/lakefs/src/execute.rs @@ -35,7 +35,7 @@ impl CustomExecuteHandler for LakeFSCustomExecuteHandler { .client .delete_branch(repo, lakefs_store.client.get_transaction(operation_id)?) .await - .map_err(|e| DeltaTableError::Transaction { source: e }); + .map_err(Into::into); lakefs_store.client.clear_transaction(operation_id); result } else { diff --git a/crates/lakefs/src/logstore.rs b/crates/lakefs/src/logstore.rs index 4235cab0ad..a137288a89 100644 --- a/crates/lakefs/src/logstore.rs +++ b/crates/lakefs/src/logstore.rs @@ -206,7 +206,7 @@ impl LakeFSLogStore { // Merge successful } // TODO: propagate better LakeFS errors. - Err(TransactionError::VersionAlreadyExists(_)) => { + Err(LogStoreError::VersionAlreadyExists(_)) => { return Err(DeltaTableError::Transaction { source: TransactionError::LogStoreError { msg: "Merge Failed".to_string(), @@ -214,7 +214,7 @@ impl LakeFSLogStore { }, }); } - Err(err) => return Err(DeltaTableError::Transaction { source: err }), + Err(err) => return Err(err.into()), }; } else { debug!("No changes detected, skipping merge"); @@ -302,9 +302,7 @@ impl LogStore for LakeFSLogStore { self.client.decompose_url(self.config.location.to_string()); let transaction_id = self .client - .get_transaction(operation_id) - .map_err(LogStoreError::from)?; - match self + .get_transaction(operation_id)?; match self .client .merge( repo, @@ -317,14 +315,14 @@ impl LogStore for LakeFSLogStore { .await { Ok(_) => Ok(()), - Err(TransactionError::VersionAlreadyExists(version)) => { + Err(LogStoreError::VersionAlreadyExists(version)) => { store .delete(&commit_uri_from_version(version)) .await .map_err(|e| LogStoreError::ObjectStore { source: e })?; return Err(LogStoreError::VersionAlreadyExists(version)); } - Err(err) => Err(LogStoreError::from(err)), + Err(err) => Err(err), }?; } _ => unreachable!(), // Default log store should never get a tmp_commit, since this is for conditional put stores @@ -337,18 +335,16 @@ impl LogStore for LakeFSLogStore { _version: i64, commit_or_bytes: CommitOrBytes, operation_id: Uuid, - ) -> Result<(), LogStoreError> { + ) -> LogStoreResult<()> { match &commit_or_bytes { CommitOrBytes::LogBytes(_) => { let (repo, _, _) = self.client.decompose_url(self.config.location.to_string()); let transaction_id = self .client - .get_transaction(operation_id) - .map_err(LogStoreError::from)?; + .get_transaction(operation_id)?; self.client .delete_branch(repo, transaction_id) - .await - .map_err(|e| LogStoreError::from(DeltaTableError::from(e)))?; + .await?; self.client.clear_transaction(operation_id); Ok(()) } From a8ea652111e023654ae540a482b58584fd422c28 Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 09:52:54 -0500 Subject: [PATCH 06/10] fmt Signed-off-by: Abhi Agarwal --- crates/core/src/operations/write/mod.rs | 2 +- crates/lakefs/src/client.rs | 6 +----- crates/lakefs/src/logstore.rs | 13 ++++--------- 3 files changed, 6 insertions(+), 15 deletions(-) diff --git a/crates/core/src/operations/write/mod.rs b/crates/core/src/operations/write/mod.rs index 73ad860564..91006a3a4c 100644 --- a/crates/core/src/operations/write/mod.rs +++ b/crates/core/src/operations/write/mod.rs @@ -755,8 +755,8 @@ impl std::future::IntoFuture for WriteBuilder { mod tests { use super::*; use crate::ensure_table_uri; - use crate::logstore::get_actions; use crate::kernel::CommitInfo; + use crate::logstore::get_actions; use crate::operations::load_cdf::collect_batches; use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::protocol::SaveMode; diff --git a/crates/lakefs/src/client.rs b/crates/lakefs/src/client.rs index 593409b1a5..fe37f9eeab 100644 --- a/crates/lakefs/src/client.rs +++ b/crates/lakefs/src/client.rs @@ -96,11 +96,7 @@ impl LakeFSClient { } } - pub async fn delete_branch( - &self, - repo: String, - branch: String, - ) -> LogStoreResult<()> { + pub async fn delete_branch(&self, repo: String, branch: String) -> LogStoreResult<()> { let request_url = format!( "{}/api/v1/repositories/{repo}/branches/{branch}", self.config.host, diff --git a/crates/lakefs/src/logstore.rs b/crates/lakefs/src/logstore.rs index a137288a89..be6af52362 100644 --- a/crates/lakefs/src/logstore.rs +++ b/crates/lakefs/src/logstore.rs @@ -300,9 +300,8 @@ impl LogStore for LakeFSLogStore { // Try LakeFS Branch merge of transaction branch in source branch let (repo, target_branch, table) = self.client.decompose_url(self.config.location.to_string()); - let transaction_id = self - .client - .get_transaction(operation_id)?; match self + let transaction_id = self.client.get_transaction(operation_id)?; + match self .client .merge( repo, @@ -339,12 +338,8 @@ impl LogStore for LakeFSLogStore { match &commit_or_bytes { CommitOrBytes::LogBytes(_) => { let (repo, _, _) = self.client.decompose_url(self.config.location.to_string()); - let transaction_id = self - .client - .get_transaction(operation_id)?; - self.client - .delete_branch(repo, transaction_id) - .await?; + let transaction_id = self.client.get_transaction(operation_id)?; + self.client.delete_branch(repo, transaction_id).await?; self.client.clear_transaction(operation_id); Ok(()) } From 70984066b39f4beeff348e1e15490716f95cdb69 Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 10:53:15 -0500 Subject: [PATCH 07/10] Get everything compiling and fix doctests Signed-off-by: Abhi Agarwal --- crates/core/src/errors.rs | 20 ++++++++++++++++++-- crates/logstore/src/config.rs | 17 +++++++++-------- crates/logstore/src/error.rs | 9 +++++++++ crates/logstore/src/logstore_impl.rs | 2 +- crates/logstore/src/storage/mod.rs | 9 ++++++--- crates/logstore/src/storage/utils.rs | 2 +- 6 files changed, 44 insertions(+), 15 deletions(-) diff --git a/crates/core/src/errors.rs b/crates/core/src/errors.rs index 669498ec97..7f340b2eca 100644 --- a/crates/core/src/errors.rs +++ b/crates/core/src/errors.rs @@ -1,5 +1,6 @@ //! Exceptions for the deltalake crate use chrono::{DateTime, Utc}; +use deltalake_logstore::LogStoreError; use object_store::Error as ObjectStoreError; use crate::kernel::transaction::{CommitBuilderError, TransactionError}; @@ -14,8 +15,10 @@ pub enum DeltaTableError { #[error("Kernel error: {0}")] KernelError(#[from] delta_kernel::error::Error), - #[error("LogStore error: {0}")] - LogStore(#[from] crate::logstore::error::LogStoreError), + #[error("LogStore error: {source}")] + LogStore { + source: crate::logstore::error::LogStoreError, + }, /// Error returned when reading the delta log object failed. #[error("Failed to read delta log object: {}", .source)] @@ -236,6 +239,19 @@ pub enum DeltaTableError { NoStartingVersionOrTimestamp, } +impl From for DeltaTableError { + fn from(err: LogStoreError) -> Self { + match err { + LogStoreError::VersionAlreadyExists(version) => { + DeltaTableError::VersionAlreadyExists(version) + } + LogStoreError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, + LogStoreError::InvalidVersion(version) => DeltaTableError::InvalidVersion(version), + _ => DeltaTableError::LogStore { source: err }, + } + } +} + impl From for DeltaTableError { fn from(err: object_store::path::Error) -> Self { Self::GenericError { diff --git a/crates/logstore/src/config.rs b/crates/logstore/src/config.rs index 46c375a909..bff2850d02 100644 --- a/crates/logstore/src/config.rs +++ b/crates/logstore/src/config.rs @@ -10,6 +10,7 @@ use ::object_store::RetryConfig; use object_store::{path::Path, prefix::PrefixStore, ObjectStore}; use std::collections::HashMap; +use std::sync::Arc; use super::storage::LimitConfig; use super::{storage::runtime::RuntimeConfig, IORuntime}; @@ -33,6 +34,8 @@ pub trait TryUpdateKey: Default { fn load_from_environment(&mut self) -> LogStoreResult<()>; } +type ParseErrors = Vec<(String, Arc)>; + #[derive(Debug)] /// Generic container for parsing configuration pub struct ParseResult { @@ -41,7 +44,7 @@ pub struct ParseResult { /// Unrecognized key value pairs. pub unparsed: HashMap, /// Errors encountered during parsing - pub errors: Vec<(String, String)>, + pub errors: ParseErrors, /// Whether the configuration is defaults only - i.e. no custom values were provided pub is_default: bool, } @@ -49,11 +52,9 @@ pub struct ParseResult { impl ParseResult { pub fn raise_errors(&self) -> LogStoreResult<()> { if !self.errors.is_empty() { - todo!() - // let error: Box = format!("Failed to parse config: {:?}", self.errors).into(); - // return Err(LogStoreError::Generic { - // source: Box::new(error), - // }); + return Err(LogStoreError::ParseErrors { + errors: self.errors.clone(), + }); } Ok(()) } @@ -76,7 +77,7 @@ where unparsed.insert(k.into(), v.into()); } Ok(Some(_)) => is_default = false, - Err(e) => errors.push((k.into(), e.to_string())), + Err(e) => errors.push((k.into(), Arc::new(e))), } } ParseResult { @@ -289,7 +290,7 @@ pub fn parse_string(value: &str) -> LogStoreResult { /// aka YAML booleans /// /// ```rust -/// # use deltalake_core::logstore::config::*; +/// # use deltalake_logstore::config::*; /// for value in ["1", "true", "on", "YES", "Y"] { /// assert!(str_is_truthy(value)); /// } diff --git a/crates/logstore/src/error.rs b/crates/logstore/src/error.rs index b1425b3355..76f49ae134 100644 --- a/crates/logstore/src/error.rs +++ b/crates/logstore/src/error.rs @@ -1,5 +1,7 @@ //! Error types for logstore operations +use std::sync::Arc; + use object_store::Error as ObjectStoreError; use thiserror::Error; @@ -66,6 +68,13 @@ pub enum LogStoreError { source: Box, }, + /// Error returned when multiple errors occurred during parsing. + #[error("Failed to parse configuration: {errors:?}")] + ParseErrors { + /// The errors that occurred during parsing. + errors: Vec<(String, Arc)>, + }, + /// Error returned when an object store is not found for a URL. #[error("No suitable object store found for '{url}'")] ObjectStoreNotFound { diff --git a/crates/logstore/src/logstore_impl.rs b/crates/logstore/src/logstore_impl.rs index 059a373a5b..73aebd166e 100644 --- a/crates/logstore/src/logstore_impl.rs +++ b/crates/logstore/src/logstore_impl.rs @@ -180,7 +180,7 @@ pub static DELTA_LOG_REGEX: LazyLock = /// This will use the built-in process global [crate::storage::ObjectStoreRegistry] by default /// /// ```rust -/// # use deltalake_core::logstore::*; +/// # use deltalake_logstore::*; /// # use std::collections::HashMap; /// # use url::Url; /// let location = Url::parse("memory:///").expect("Failed to make location"); diff --git a/crates/logstore/src/storage/mod.rs b/crates/logstore/src/storage/mod.rs index 848cc1a8d5..37e37c1fea 100644 --- a/crates/logstore/src/storage/mod.rs +++ b/crates/logstore/src/storage/mod.rs @@ -139,8 +139,11 @@ mod tests { // try get non-existent key let url = Url::parse("not-registered://host").unwrap(); let err = registry.get_store(&url).unwrap_err(); - assert!(err - .to_string() - .contains("No suitable object store found for 'not-registered://host'.")); + match err { + LogStoreError::ObjectStoreNotFound { url } => { + assert_eq!(url, "not-registered://host"); + } + _ => panic!("Expected LogStoreError::ObjectStoreNotFound"), + } } } diff --git a/crates/logstore/src/storage/utils.rs b/crates/logstore/src/storage/utils.rs index aa50db5584..ef57ab9b1c 100644 --- a/crates/logstore/src/storage/utils.rs +++ b/crates/logstore/src/storage/utils.rs @@ -5,7 +5,7 @@ use object_store::path::Path; /// Return the uri of commit version. /// /// ```rust -/// # use deltalake_core::logstore::*; +/// # use deltalake_logstore::*; /// use object_store::path::Path; /// let uri = commit_uri_from_version(1); /// assert_eq!(uri, Path::from("_delta_log/00000000000000000001.json")); From b91aa9f5feb4346e577427c895ce144d5253675a Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 12:22:30 -0500 Subject: [PATCH 08/10] Clean up error handling Signed-off-by: Abhi Agarwal --- crates/aws/src/logstore/dynamodb_logstore.rs | 21 +++++++------------ crates/azure/src/config.rs | 6 +++--- crates/azure/src/error.rs | 22 ++++++++++++++++++++ crates/azure/src/lib.rs | 6 ++++-- crates/gcp/src/config.rs | 9 ++++---- crates/gcp/src/error.rs | 22 ++++++++++++++++++++ crates/gcp/src/lib.rs | 1 + crates/lakefs/src/logstore.rs | 4 ++-- crates/logstore/src/error.rs | 4 ++-- crates/mount/src/error.rs | 15 +++---------- 10 files changed, 71 insertions(+), 39 deletions(-) create mode 100644 crates/azure/src/error.rs create mode 100644 crates/gcp/src/error.rs diff --git a/crates/aws/src/logstore/dynamodb_logstore.rs b/crates/aws/src/logstore/dynamodb_logstore.rs index a78c5113c9..7ff5a1b9b1 100644 --- a/crates/aws/src/logstore/dynamodb_logstore.rs +++ b/crates/aws/src/logstore/dynamodb_logstore.rs @@ -137,7 +137,7 @@ impl S3DynamoDbLogStore { .lock_client .update_commit_entry(entry.version, &self.table_path) .await - .map_err(|err| LogStoreError::LogStoreError { + .map_err(|err| LogStoreError::TransactionError { msg: format!( "unable to complete entry for '{}': failure to write to DynamoDb", entry.version @@ -232,7 +232,6 @@ impl LogStore for S3DynamoDbLogStore { .await .map_err(|err| match err { LockClientError::VersionAlreadyExists { version, .. } => { - warn!("LockClientError::VersionAlreadyExists({version})"); LogStoreError::VersionAlreadyExists(version) } LockClientError::ProvisionedThroughputExceeded => todo!( @@ -240,19 +239,15 @@ impl LogStore for S3DynamoDbLogStore { ), LockClientError::LockTableNotFound => { let table_name = self.lock_client.get_lock_table_name(); - error!("Lock table '{table_name}' not found"); - LogStoreError::LogStoreError { + LogStoreError::TransactionError { msg: format!("lock table '{table_name}' not found"), source: Box::new(err), } } - err => { - error!("dynamodb client failed to write log entry: {err:?}"); - LogStoreError::LogStoreError { - msg: "dynamodb client failed to write log entry".to_owned(), - source: Box::new(err), - } - } + err => LogStoreError::TransactionError { + msg: "dynamodb client failed to write log entry".to_owned(), + source: Box::new(err), + }, })?; // `repair_entry` performs the exact steps required to finalize the commit, but contains // retry logic and more robust error handling under the assumption that any other client @@ -283,12 +278,12 @@ impl LogStore for S3DynamoDbLogStore { ), LockClientError::VersionAlreadyCompleted { version, .. } => { error!("Trying to abort a completed commit"); - LogStoreError::LogStoreError { + LogStoreError::TransactionError { msg: format!("trying to abort a completed log entry: {version}"), source: Box::new(err), } } - err => LogStoreError::LogStoreError { + err => LogStoreError::TransactionError { msg: "dynamodb client failed to delete log entry".to_owned(), source: Box::new(err), }, diff --git a/crates/azure/src/config.rs b/crates/azure/src/config.rs index 3fb0bd24ee..17bfdfad11 100644 --- a/crates/azure/src/config.rs +++ b/crates/azure/src/config.rs @@ -11,7 +11,7 @@ use std::sync::LazyLock; use object_store::azure::AzureConfigKey; use object_store::Error as ObjectStoreError; -use deltalake_logstore::LogStoreResult; +use crate::error::Result; static CREDENTIAL_KEYS: LazyLock> = LazyLock::new(|| { Vec::from_iter([ @@ -79,7 +79,7 @@ impl AzureConfigHelper { /// Create a new [`ConfigHelper`] pub fn try_new( config: impl IntoIterator, impl Into)>, - ) -> LogStoreResult { + ) -> Result { let mut env_config = HashMap::new(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { @@ -125,7 +125,7 @@ impl AzureConfigHelper { } /// Generate a configuration augmented with options from the environment - pub fn build(mut self) -> LogStoreResult> { + pub fn build(mut self) -> Result> { let mut has_credential = false; if self.config.contains_key(&AzureConfigKey::UseAzureCli) { diff --git a/crates/azure/src/error.rs b/crates/azure/src/error.rs new file mode 100644 index 0000000000..8c0cb1d6a0 --- /dev/null +++ b/crates/azure/src/error.rs @@ -0,0 +1,22 @@ +use deltalake_logstore::LogStoreError; + +pub(crate) type Result = std::result::Result; + +#[derive(thiserror::Error, Debug)] +pub(crate) enum Error { + #[allow(dead_code)] + #[error("failed to parse config: {0}")] + Parse(String), + + #[error(transparent)] + ObjectStore(#[from] object_store::Error), +} + +impl From for LogStoreError { + fn from(e: Error) -> Self { + match e { + Error::Parse(msg) => LogStoreError::Generic { source: msg.into() }, + Error::ObjectStore(e) => LogStoreError::ObjectStore { source: e }, + } + } +} diff --git a/crates/azure/src/lib.rs b/crates/azure/src/lib.rs index af43eafb49..f6fde52173 100644 --- a/crates/azure/src/lib.rs +++ b/crates/azure/src/lib.rs @@ -13,6 +13,7 @@ use object_store::ObjectStoreScheme; use url::Url; mod config; +pub mod error; trait AzureOptions { fn as_azure_options(&self) -> HashMap; @@ -58,8 +59,9 @@ impl ObjectStoreFactory for AzureFactory { let (_, path) = ObjectStoreScheme::parse(url).map_err(|e| LogStoreError::Generic { source: Box::new(e), })?; - let prefix = - Path::parse(path).map_err(|e| LogStoreError::ObjectStore { source: e.into() })?; + let prefix = Path::parse(path).map_err(|e| LogStoreError::Generic { + source: Box::new(e), + })?; Ok((Arc::new(store), prefix)) } diff --git a/crates/gcp/src/config.rs b/crates/gcp/src/config.rs index 80eeb2d7dd..6c187fbf56 100644 --- a/crates/gcp/src/config.rs +++ b/crates/gcp/src/config.rs @@ -9,9 +9,8 @@ use std::str::FromStr; use std::sync::LazyLock; use object_store::gcp::GoogleConfigKey; -use object_store::Error as ObjectStoreError; -use deltalake_logstore::LogStoreResult; +use crate::error::Result; static CREDENTIAL_KEYS: LazyLock> = LazyLock::new(|| { Vec::from_iter([ @@ -54,7 +53,7 @@ impl GcpConfigHelper { /// Create a new [`ConfigHelper`] pub fn try_new( config: impl IntoIterator, impl Into)>, - ) -> LogStoreResult { + ) -> Result { let mut env_config = HashMap::new(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { @@ -70,7 +69,7 @@ impl GcpConfigHelper { config: config .into_iter() .map(|(key, value)| Ok((GoogleConfigKey::from_str(key.as_ref())?, value.into()))) - .collect::>()?, + .collect::>()?, env_config, priority: Vec::from_iter([ GcpCredential::ServiceAccountKey, @@ -97,7 +96,7 @@ impl GcpConfigHelper { } /// Generate a configuration augmented with options from the environment - pub fn build(mut self) -> LogStoreResult> { + pub fn build(mut self) -> Result> { let mut has_credential = false; // try using only passed config options diff --git a/crates/gcp/src/error.rs b/crates/gcp/src/error.rs new file mode 100644 index 0000000000..8c0cb1d6a0 --- /dev/null +++ b/crates/gcp/src/error.rs @@ -0,0 +1,22 @@ +use deltalake_logstore::LogStoreError; + +pub(crate) type Result = std::result::Result; + +#[derive(thiserror::Error, Debug)] +pub(crate) enum Error { + #[allow(dead_code)] + #[error("failed to parse config: {0}")] + Parse(String), + + #[error(transparent)] + ObjectStore(#[from] object_store::Error), +} + +impl From for LogStoreError { + fn from(e: Error) -> Self { + match e { + Error::Parse(msg) => LogStoreError::Generic { source: msg.into() }, + Error::ObjectStore(e) => LogStoreError::ObjectStore { source: e }, + } + } +} diff --git a/crates/gcp/src/lib.rs b/crates/gcp/src/lib.rs index fe9a6c8e55..070ff34c65 100644 --- a/crates/gcp/src/lib.rs +++ b/crates/gcp/src/lib.rs @@ -13,6 +13,7 @@ use object_store::client::SpawnedReqwestConnector; use url::Url; mod config; +pub mod error; mod storage; trait GcpOptions { diff --git a/crates/lakefs/src/logstore.rs b/crates/lakefs/src/logstore.rs index be6af52362..244f53bf41 100644 --- a/crates/lakefs/src/logstore.rs +++ b/crates/lakefs/src/logstore.rs @@ -258,7 +258,7 @@ impl LogStore for LakeFSLogStore { ) -> LogStoreResult<()> { let (transaction_url, store, _root_store) = self .get_transaction_objectstore(operation_id) - .map_err(|e| LogStoreError::LogStoreError { + .map_err(|e| LogStoreError::TransactionError { msg: e.to_string(), source: e.into(), })?; @@ -292,7 +292,7 @@ impl LogStore for LakeFSLogStore { false, ) .await - .map_err(|e| LogStoreError::LogStoreError { + .map_err(|e| LogStoreError::TransactionError { msg: e.to_string(), source: Box::new(e), })?; diff --git a/crates/logstore/src/error.rs b/crates/logstore/src/error.rs index 76f49ae134..87c8b78b53 100644 --- a/crates/logstore/src/error.rs +++ b/crates/logstore/src/error.rs @@ -21,8 +21,8 @@ pub enum LogStoreError { }, /// The transaction failed to commit due to an error in an implementation-specific layer. - #[error("Transaction failed: {msg}")] - LogStoreError { + #[error("Transaction failed: {msg}, error: {source}")] + TransactionError { /// Detailed message for the commit failure. msg: String, /// underlying error in the log store transactional layer. diff --git a/crates/mount/src/error.rs b/crates/mount/src/error.rs index 090ba304ff..cb3fb2258a 100644 --- a/crates/mount/src/error.rs +++ b/crates/mount/src/error.rs @@ -22,18 +22,9 @@ pub enum Error { impl From for LogStoreError { fn from(e: Error) -> Self { match e { - Error::Parse(msg) => LogStoreError::Generic { - source: Box::new(std::io::Error::new(std::io::ErrorKind::InvalidInput, msg)), - }, - Error::UnknownConfigKey(msg) => LogStoreError::Generic { - source: Box::new(std::io::Error::new(std::io::ErrorKind::InvalidInput, msg)), - }, - Error::AllowUnsafeRenameNotSpecified => LogStoreError::Generic { - source: Box::new(std::io::Error::new( - std::io::ErrorKind::InvalidInput, - "The `allow_unsafe_rename` parameter must be specified", - )), - }, + Error::Parse(msg) => LogStoreError::Generic { source: msg.into() }, + Error::UnknownConfigKey(_) => LogStoreError::Generic { source: e.into() }, + Error::AllowUnsafeRenameNotSpecified => LogStoreError::Generic { source: e.into() }, Error::ObjectStore(e) => LogStoreError::ObjectStore { source: e }, } } From 935a41ad1652731a7927c7ba886c21ee461fbe4a Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 12:25:50 -0500 Subject: [PATCH 09/10] Unused imports Signed-off-by: Abhi Agarwal --- crates/catalog-unity/src/lib.rs | 4 ++-- crates/core/src/kernel/models/actions.rs | 2 -- crates/core/src/operations/restore.rs | 2 +- crates/core/src/operations/write/mod.rs | 1 - crates/core/src/writer/stats.rs | 1 - crates/lakefs/src/storage.rs | 2 +- crates/logstore/src/logstore_impl.rs | 9 +++------ 7 files changed, 7 insertions(+), 14 deletions(-) diff --git a/crates/catalog-unity/src/lib.rs b/crates/catalog-unity/src/lib.rs index c03c0d2760..a6eac95284 100644 --- a/crates/catalog-unity/src/lib.rs +++ b/crates/catalog-unity/src/lib.rs @@ -32,8 +32,8 @@ use crate::models::{ use deltalake_core::data_catalog::DataCatalogResult; use deltalake_core::{ - ensure_table_uri, DataCatalog, DataCatalogError, DeltaResult, DeltaTableBuilder, - DeltaTableError, ObjectStoreError, Path, + ensure_table_uri, DataCatalog, DataCatalogError, DeltaTableBuilder, DeltaTableError, + ObjectStoreError, Path, }; use crate::client::retry::*; diff --git a/crates/core/src/kernel/models/actions.rs b/crates/core/src/kernel/models/actions.rs index 278e130bb5..2697e2e126 100644 --- a/crates/core/src/kernel/models/actions.rs +++ b/crates/core/src/kernel/models/actions.rs @@ -1,6 +1,4 @@ use std::collections::{HashMap, HashSet}; -use std::fmt::{self, Display}; -use std::str::FromStr; use delta_kernel::schema::{DataType, StructField}; use delta_kernel::table_features::{ReaderFeature, WriterFeature}; diff --git a/crates/core/src/operations/restore.rs b/crates/core/src/operations/restore.rs index a4e4473698..b7aa341e3d 100644 --- a/crates/core/src/operations/restore.rs +++ b/crates/core/src/operations/restore.rs @@ -36,7 +36,7 @@ use serde::Serialize; use uuid::Uuid; use super::{CustomExecuteHandler, Operation}; -use crate::kernel::transaction::{CommitBuilder, CommitProperties, TransactionError}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::{ resolve_snapshot, Action, Add, EagerSnapshot, ProtocolExt as _, ProtocolInner, Remove, }; diff --git a/crates/core/src/operations/write/mod.rs b/crates/core/src/operations/write/mod.rs index 91006a3a4c..ee5904bb02 100644 --- a/crates/core/src/operations/write/mod.rs +++ b/crates/core/src/operations/write/mod.rs @@ -56,7 +56,6 @@ use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::expr::parse_predicate_expression; use crate::delta_datafusion::logical::MetricObserver; use crate::delta_datafusion::physical::{find_metric_node, get_metric}; -use crate::delta_datafusion::planner::DeltaPlanner; use crate::delta_datafusion::{create_session, register_store}; use crate::delta_datafusion::{session_state_from_session, DataFusionMixins}; use crate::errors::{DeltaResult, DeltaTableError}; diff --git a/crates/core/src/writer/stats.rs b/crates/core/src/writer/stats.rs index 557dc184fb..34ffdcdc46 100644 --- a/crates/core/src/writer/stats.rs +++ b/crates/core/src/writer/stats.rs @@ -21,7 +21,6 @@ use tracing::warn; use super::*; use crate::kernel::{scalars::ScalarExt, Add}; -use crate::protocol::AddStatsExt; use crate::protocol::{ColumnValueStat, Stats}; /// Creates an [`Add`] log action struct. diff --git a/crates/lakefs/src/storage.rs b/crates/lakefs/src/storage.rs index c001670bac..9f2b2f1ba9 100644 --- a/crates/lakefs/src/storage.rs +++ b/crates/lakefs/src/storage.rs @@ -4,7 +4,7 @@ use deltalake_core::logstore::object_store::aws::AmazonS3ConfigKey; use deltalake_core::logstore::{ LogStoreError, LogStoreResult, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; -use deltalake_core::{DeltaTableError, Path}; +use deltalake_core::Path; use object_store::aws::AmazonS3Builder; use object_store::client::SpawnedReqwestConnector; use object_store::ObjectStoreScheme; diff --git a/crates/logstore/src/logstore_impl.rs b/crates/logstore/src/logstore_impl.rs index 73aebd166e..72a86b34cd 100644 --- a/crates/logstore/src/logstore_impl.rs +++ b/crates/logstore/src/logstore_impl.rs @@ -80,14 +80,11 @@ use crate::config::StorageConfig; use crate::default_logstore::DefaultLogStore; use crate::error::{LogStoreError, LogStoreResult}; use crate::factories::{ - logstore_factories, object_store_factories, store_for, LogStoreFactory, - LogStoreFactoryRegistry, ObjectStoreFactory, ObjectStoreFactoryRegistry, + logstore_factories, object_store_factories, LogStoreFactory, ObjectStoreFactory, + ObjectStoreFactoryRegistry, }; use crate::storage::utils::commit_uri_from_version; -use crate::storage::{ - DefaultObjectStoreRegistry, DeltaIOStorageBackend, IORuntime, ObjectStoreRef, - ObjectStoreRegistry, ObjectStoreRetryExt, -}; +use crate::storage::{ObjectStoreRef, ObjectStoreRegistry, ObjectStoreRetryExt}; /// Helper function to spawn blocking tasks with tracing span context pub(crate) fn spawn_blocking_with_span(f: F) -> tokio::task::JoinHandle From bbb1721a306ff4bad6fd68d7f02102937a7b832e Mon Sep 17 00:00:00 2001 From: Abhi Agarwal Date: Sun, 2 Nov 2025 12:28:51 -0500 Subject: [PATCH 10/10] Remove stray re-export Signed-off-by: Abhi Agarwal --- crates/core/src/kernel/mod.rs | 1 - crates/core/src/operations/merge/mod.rs | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/crates/core/src/kernel/mod.rs b/crates/core/src/kernel/mod.rs index 32594638b7..7fea67efec 100644 --- a/crates/core/src/kernel/mod.rs +++ b/crates/core/src/kernel/mod.rs @@ -18,7 +18,6 @@ pub mod transaction; pub use arrow::engine_ext::StructDataExt; pub use delta_kernel::engine; -pub use deltalake_logstore::get_actions; pub use error::*; pub use models::*; pub use schema::*; diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index fe13be8909..c994023bb6 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -2006,7 +2006,7 @@ mod tests { .await .unwrap() .expect("failed to get snapshot bytes"); - let actions = crate::kernel::get_actions(2, &snapshot_bytes).unwrap(); + let actions = crate::logstore::get_actions(2, &snapshot_bytes).unwrap(); let schema_actions = actions .iter() @@ -2080,7 +2080,7 @@ mod tests { .await .unwrap() .expect("failed to get snapshot bytes"); - let actions = crate::kernel::get_actions(2, &snapshot_bytes).unwrap(); + let actions = crate::logstore::get_actions(2, &snapshot_bytes).unwrap(); let schema_actions = actions .iter() @@ -2189,7 +2189,7 @@ mod tests { .await .unwrap() .expect("failed to get snapshot bytes"); - let actions = crate::kernel::get_actions(2, &snapshot_bytes).unwrap(); + let actions = crate::logstore::get_actions(2, &snapshot_bytes).unwrap(); let schema_actions = actions .iter()