diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index f216830327..8ea03b8661 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -144,25 +144,3 @@ jobs: - name: Run tests with native-tls run: | cargo test --no-default-features --features integration_test,s3-native-tls,datafusion - - parquet2_test: - runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=line-tables-only" - CARGO_INCREMENTAL: 0 - - steps: - - uses: actions/checkout@v3 - - - name: Install minimal stable with clippy and rustfmt - uses: actions-rs/toolchain@v1 - with: - profile: default - toolchain: stable - override: true - - - uses: Swatinem/rust-cache@v2 - - - name: Run tests - working-directory: crates/deltalake-core - run: cargo test --no-default-features --features=parquet2 diff --git a/Cargo.toml b/Cargo.toml index 1e9f311693..2e58e375e2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,6 +46,7 @@ serde_json = "1" # "stdlib" bytes = { version = "1" } chrono = { version = "0.4.31", default-features = false, features = ["clock"] } +log = { version = "0.4" } regex = { version = "1" } thiserror = { version = "1" } url = { version = "2" } diff --git a/crates/deltalake-aws/Cargo.toml b/crates/deltalake-aws/Cargo.toml index 8b7e0f4655..b0f102ce7c 100644 --- a/crates/deltalake-aws/Cargo.toml +++ b/crates/deltalake-aws/Cargo.toml @@ -3,33 +3,43 @@ name = "deltalake-aws" version = "0.1.0" edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - [dependencies] +deltalake-core = { path = "../deltalake-core" } rusoto_core = { version = "0.47", default-features = false, optional = true } -rusoto_credential = { version = "0.47", optional = true } +rusoto_credential = { version = "0.47" } rusoto_sts = { version = "0.47", default-features = false, optional = true } rusoto_dynamodb = { version = "0.47", default-features = false, optional = true } -object_store = "0.7" +object_store = { version = "0.7.1", features = ["aws"]} lazy_static = "1" maplit = "1" +async-trait = { workspace = true } +bytes = { workspace = true } +futures = { workspace = true } +log = { workspace = true } thiserror = { workspace = true } +tokio = { workspace = true } regex = { workspace = true } +uuid = { workspace = true, features = ["serde", "v4"] } +url = { workspace = true } [dev-dependencies] +chrono = { workspace = true } +serial_test = "2" +deltalake-test = { path = "../deltalake-test" } +pretty_env_logger = "*" +rand = "0.8" +serde_json = { workspace = true } [features] +default = ["rustls"] +integration_test = [] native-tls = [ "rusoto_core/native-tls", - "rusoto_credential", "rusoto_sts/native-tls", "rusoto_dynamodb/native-tls", - "object_store/aws", ] rustls = [ "rusoto_core/rustls", - "rusoto_credential", "rusoto_sts/rustls", "rusoto_dynamodb/rustls", - "object_store/aws", ] diff --git a/crates/deltalake-core/tests/common/s3.rs b/crates/deltalake-aws/helpers.rs similarity index 100% rename from crates/deltalake-core/tests/common/s3.rs rename to crates/deltalake-aws/helpers.rs diff --git a/crates/deltalake-aws/src/lib.rs b/crates/deltalake-aws/src/lib.rs index 4a8d36657c..f6a2b2da31 100644 --- a/crates/deltalake-aws/src/lib.rs +++ b/crates/deltalake-aws/src/lib.rs @@ -1,16 +1,22 @@ //! Lock client implementation based on DynamoDb. pub mod errors; +pub mod logstore; +pub mod storage; use lazy_static::lazy_static; +use log::*; use regex::Regex; use std::{ collections::HashMap, str::FromStr, + sync::Arc, time::{Duration, SystemTime}, }; -use object_store::path::Path; +use deltalake_core::logstore::{logstores, LogStore, LogStoreFactory}; +use deltalake_core::storage::{factories, url_prefix_handler, ObjectStoreRef, StorageOptions}; +use deltalake_core::{DeltaResult, Path}; use rusoto_core::{HttpClient, Region, RusotoError}; use rusoto_credential::AutoRefreshingProvider; use rusoto_dynamodb::{ @@ -19,8 +25,48 @@ use rusoto_dynamodb::{ UpdateItemError, UpdateItemInput, }; use rusoto_sts::WebIdentityProvider; +use url::Url; use errors::{DynamoDbConfigError, LockClientError}; +use storage::{S3ObjectStoreFactory, S3StorageOptions}; + +#[derive(Clone, Debug, Default)] +struct S3LogStoreFactory {} + +impl LogStoreFactory for S3LogStoreFactory { + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + let store = url_prefix_handler(store, Path::parse(location.path())?)?; + let s3_options = S3StorageOptions::from_map(&options.0); + + if s3_options.locking_provider.as_deref() != Some("dynamodb") { + debug!("S3LogStoreFactory has been asked to create a LogStore without the dynamodb locking provider"); + return Ok(deltalake_core::logstore::default_logstore( + store, location, options, + )); + } + + Ok(Arc::new(logstore::S3DynamoDbLogStore::try_new( + location.clone(), + options.clone(), + &s3_options, + store, + )?)) + } +} + +/// Register an [ObjectStoreFactory] for common S3 [Url] schemes +pub fn register_handlers(_additional_prefixes: Option) { + for scheme in ["s3", "s3a"].iter() { + let url = Url::parse(&format!("{}://", scheme)).unwrap(); + factories().insert(url.clone(), Arc::new(S3ObjectStoreFactory::default())); + logstores().insert(url.clone(), Arc::new(S3LogStoreFactory::default())); + } +} /// Representation of a log entry stored in DynamoDb /// dynamo db item consists of: @@ -62,6 +108,12 @@ pub struct DynamoDbLockClient { config: DynamoDbConfig, } +impl std::fmt::Debug for DynamoDbLockClient { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(f, "DynamoDbLockClient(config: {:?})", self.config) + } +} + impl DynamoDbLockClient { /// Creates a new DynamoDbLockClient from the supplied storage options. pub fn try_new( @@ -514,8 +566,9 @@ fn extract_version_from_filename(name: &str) -> Option { #[cfg(test)] mod tests { - use super::*; + use object_store::memory::InMemory; + use serial_test::serial; fn commit_entry_roundtrip(c: &CommitEntry) -> Result<(), LockClientError> { let item_data: HashMap = create_value_map(c, "some_table"); @@ -547,4 +600,19 @@ mod tests { })?; Ok(()) } + + /// In cases where there is no dynamodb specified locking provider, this should get a default + /// logstore + #[test] + #[serial] + fn test_logstore_factory_default() { + let factory = S3LogStoreFactory::default(); + let store = InMemory::new(); + let url = Url::parse("s3://test-bucket").unwrap(); + std::env::remove_var(storage::s3_constants::AWS_S3_LOCKING_PROVIDER); + let logstore = factory + .with_options(Arc::new(store), &url, &StorageOptions::from(HashMap::new())) + .unwrap(); + assert_eq!(logstore.name(), "DefaultLogStore"); + } } diff --git a/crates/deltalake-core/src/logstore/s3/mod.rs b/crates/deltalake-aws/src/logstore.rs similarity index 76% rename from crates/deltalake-core/src/logstore/s3/mod.rs rename to crates/deltalake-aws/src/logstore.rs index 9e7883c7b2..295251c6ca 100644 --- a/crates/deltalake-core/src/logstore/s3/mod.rs +++ b/crates/deltalake-aws/src/logstore.rs @@ -3,22 +3,22 @@ //! when the underlying object storage does not support atomic `put_if_absent` //! or `rename_if_absent` operations, as is the case for S3. -use deltalake_aws::errors::LockClientError; -use deltalake_aws::{constants, CommitEntry, DynamoDbLockClient, UpdateLogEntryResult}; +use crate::errors::LockClientError; +use crate::storage::S3StorageOptions; +use crate::{constants, CommitEntry, DynamoDbLockClient, UpdateLogEntryResult}; use bytes::Bytes; -use object_store::path::Path; -use object_store::Error as ObjectStoreError; +use deltalake_core::{ObjectStoreError, Path}; +use log::*; use url::Url; -use crate::{ +use deltalake_core::logstore::*; +use deltalake_core::{ operations::transaction::TransactionError, - storage::{config::StorageOptions, s3::S3StorageOptions, ObjectStoreRef}, + storage::{ObjectStoreRef, StorageOptions}, DeltaResult, DeltaTableError, }; -use super::{LogStore, LogStoreConfig}; - const STORE_NAME: &str = "DeltaS3ObjectStore"; const MAX_REPAIR_RETRIES: i64 = 3; @@ -30,6 +30,12 @@ pub struct S3DynamoDbLogStore { table_path: String, } +impl std::fmt::Debug for S3DynamoDbLogStore { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(f, "S3DynamoDbLogStore({})", self.table_path) + } +} + impl S3DynamoDbLogStore { /// Create log store pub fn try_new( @@ -45,12 +51,13 @@ impl S3DynamoDbLogStore { s3_options.use_web_identity, ) .map_err(|err| DeltaTableError::ObjectStore { - source: object_store::Error::Generic { + source: ObjectStoreError::Generic { store: STORE_NAME, source: err.into(), }, })?; - let table_path = super::to_uri(&location, &Path::from("")); + debug!("S3DynamoDbLogStore configured with lock client: {lock_client:?}"); + let table_path = to_uri(&location, &Path::from("")); Ok(Self { storage: object_store, lock_client, @@ -73,22 +80,22 @@ impl S3DynamoDbLogStore { return Ok(RepairLogEntryResult::AlreadyCompleted); } for retry in 0..=MAX_REPAIR_RETRIES { - match super::write_commit_entry(self.storage.as_ref(), entry.version, &entry.temp_path) - .await - { + match write_commit_entry(&self.storage, entry.version, &entry.temp_path).await { Ok(()) => { + debug!("Successfully committed entry for version {}", entry.version); 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 { 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) => log::debug!( - "retry #{retry} on log entry {entry:?} failed to move commit: '{err}'" - ), + Err(err) => { + debug!("retry #{retry} on log entry {entry:?} failed to move commit: '{err}'") + } } } unreachable!("for loop yields Ok or Err in body when retry = MAX_REPAIR_RETRIES") @@ -100,6 +107,7 @@ impl S3DynamoDbLogStore { entry: &CommitEntry, copy_performed: bool, ) -> Result { + debug!("try_complete_entry for {:?}, {}", entry, copy_performed); for retry in 0..=MAX_REPAIR_RETRIES { match self .lock_client @@ -114,7 +122,7 @@ impl S3DynamoDbLogStore { }) { Ok(x) => return Ok(Self::map_retry_result(x, copy_performed)), Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), - Err(err) => log::debug!( + Err(err) => error!( "retry #{retry} on log entry {entry:?} failed to update lock db: '{err}'" ), } @@ -141,6 +149,10 @@ impl S3DynamoDbLogStore { #[async_trait::async_trait] impl LogStore for S3DynamoDbLogStore { + fn name(&self) -> String { + "S3DynamoDbLogStore".into() + } + fn root_uri(&self) -> String { self.table_path.clone() } @@ -153,7 +165,7 @@ impl LogStore for S3DynamoDbLogStore { if let Ok(Some(entry)) = entry { self.repair_entry(&entry).await?; } - super::read_commit_entry(self.storage.as_ref(), version).await + read_commit_entry(&self.storage, version).await } /// Tries to commit a prepared commit file. Returns [DeltaTableError::VersionAlreadyExists] @@ -167,26 +179,34 @@ impl LogStore for S3DynamoDbLogStore { tmp_commit: &Path, ) -> Result<(), TransactionError> { let entry = CommitEntry::new(version, tmp_commit.clone()); + debug!("Writing commit entry for {self:?}: {entry:?}"); // create log entry in dynamo db: complete = false, no expireTime self.lock_client .put_commit_entry(&self.table_path, &entry) .await .map_err(|err| match err { LockClientError::VersionAlreadyExists { version, .. } => { + warn!("LockClientError::VersionAlreadyExists({version})"); TransactionError::VersionAlreadyExists(version) } - LockClientError::ProvisionedThroughputExceeded => todo!(), - LockClientError::LockTableNotFound => TransactionError::LogStoreError { - msg: format!( - "lock table '{}' not found", - self.lock_client.get_lock_table_name() - ), - source: Box::new(err), - }, - err => TransactionError::LogStoreError { - msg: "dynamodb client failed to write log entry".to_owned(), - source: Box::new(err), - }, + LockClientError::ProvisionedThroughputExceeded => todo!( + "deltalake-aws does not yet handle DynamoDB provisioned throughput errors" + ), + LockClientError::LockTableNotFound => { + let table_name = self.lock_client.get_lock_table_name(); + error!("Lock table '{table_name}' not found"); + TransactionError::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 { + 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 @@ -198,6 +218,7 @@ impl LogStore for S3DynamoDbLogStore { } async fn get_latest_version(&self, current_version: i64) -> DeltaResult { + debug!("Retrieving latest version of {self:?} at v{current_version}"); let entry = self .lock_client .get_latest_entry(&self.table_path) @@ -210,7 +231,7 @@ impl LogStore for S3DynamoDbLogStore { self.repair_entry(&entry).await?; Ok(entry.version) } else { - super::get_latest_version(self, current_version).await + get_latest_version(self, current_version).await } } @@ -218,15 +239,6 @@ impl LogStore for S3DynamoDbLogStore { self.storage.clone() } - fn to_uri(&self, location: &Path) -> String { - super::to_uri(&self.config.location, location) - } - - #[cfg(feature = "datafusion")] - fn object_store_url(&self) -> datafusion::execution::object_store::ObjectStoreUrl { - super::object_store_url(&self.config.location) - } - fn config(&self) -> &LogStoreConfig { &self.config } diff --git a/crates/deltalake-aws/src/storage.rs b/crates/deltalake-aws/src/storage.rs new file mode 100644 index 0000000000..97786e9736 --- /dev/null +++ b/crates/deltalake-aws/src/storage.rs @@ -0,0 +1,597 @@ +//! AWS S3 storage backend. + +use bytes::Bytes; +use deltalake_core::storage::object_store::{ + aws::AmazonS3ConfigKey, parse_url_opts, GetOptions, GetResult, ListResult, MultipartId, + ObjectMeta, ObjectStore, Result as ObjectStoreResult, +}; +use deltalake_core::storage::{str_is_truthy, ObjectStoreFactory, ObjectStoreRef, StorageOptions}; +use deltalake_core::{DeltaResult, ObjectStoreError, Path}; +use futures::stream::BoxStream; +use rusoto_core::Region; +use std::collections::HashMap; +use std::fmt::Debug; +use std::ops::Range; +use std::str::FromStr; +use std::sync::Arc; +use std::time::Duration; +use tokio::io::AsyncWrite; +use url::Url; + +const STORE_NAME: &str = "DeltaS3ObjectStore"; + +#[derive(Clone, Default, Debug)] +pub struct S3ObjectStoreFactory {} + +impl S3ObjectStoreFactory { + fn with_env_s3(&self, options: &StorageOptions) -> StorageOptions { + let mut options = options.clone(); + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { + if !options.0.contains_key(config_key.as_ref()) { + options + .0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + options + } +} + +impl ObjectStoreFactory for S3ObjectStoreFactory { + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + let options = self.with_env_s3(options); + let (store, prefix) = parse_url_opts( + url, + options.0.iter().filter_map(|(key, value)| { + let s3_key = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; + Some((s3_key, value.clone())) + }), + )?; + + let options = S3StorageOptions::from_map(&options.0); + let store = S3StorageBackend::try_new( + store.into(), + Some("dynamodb") == options.locking_provider.as_deref() || options.allow_unsafe_rename, + )?; + + Ok((Arc::new(store), prefix)) + } +} + +/// Options used to configure the [S3StorageBackend]. +/// +/// Available options are described in [s3_constants]. +#[derive(Clone, Debug, PartialEq, Eq)] +#[allow(missing_docs)] +pub struct S3StorageOptions { + pub endpoint_url: Option, + pub region: Region, + pub profile: Option, + pub aws_access_key_id: Option, + pub aws_secret_access_key: Option, + pub aws_session_token: Option, + pub virtual_hosted_style_request: bool, + pub locking_provider: Option, + pub assume_role_arn: Option, + pub assume_role_session_name: Option, + pub use_web_identity: bool, + pub s3_pool_idle_timeout: Duration, + pub sts_pool_idle_timeout: Duration, + pub s3_get_internal_server_error_retries: usize, + pub allow_unsafe_rename: bool, + pub extra_opts: HashMap, +} + +impl S3StorageOptions { + /// Creates an instance of S3StorageOptions from the given HashMap. + pub fn from_map(options: &HashMap) -> S3StorageOptions { + let extra_opts = options + .iter() + .filter(|(k, _)| !s3_constants::S3_OPTS.contains(&k.as_str())) + .map(|(k, v)| (k.to_owned(), v.to_owned())) + .collect(); + + // Copy web identity values provided in options but not the environment into the environment + // to get picked up by the `from_k8s_env` call in `get_web_identity_provider`. + Self::ensure_env_var(options, s3_constants::AWS_REGION); + Self::ensure_env_var(options, s3_constants::AWS_PROFILE); + Self::ensure_env_var(options, s3_constants::AWS_ACCESS_KEY_ID); + Self::ensure_env_var(options, s3_constants::AWS_SECRET_ACCESS_KEY); + Self::ensure_env_var(options, s3_constants::AWS_SESSION_TOKEN); + Self::ensure_env_var(options, s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE); + Self::ensure_env_var(options, s3_constants::AWS_ROLE_ARN); + Self::ensure_env_var(options, s3_constants::AWS_ROLE_SESSION_NAME); + + let endpoint_url = str_option(options, s3_constants::AWS_ENDPOINT_URL); + let region = if let Some(endpoint_url) = endpoint_url.as_ref() { + Region::Custom { + name: Self::str_or_default(options, s3_constants::AWS_REGION, "custom".to_string()), + endpoint: endpoint_url.to_owned(), + } + } else { + Region::default() + }; + let profile = str_option(options, s3_constants::AWS_PROFILE); + + let s3_pool_idle_timeout = + Self::u64_or_default(options, s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, 15); + let sts_pool_idle_timeout = + Self::u64_or_default(options, s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, 10); + + let s3_get_internal_server_error_retries = Self::u64_or_default( + options, + s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + 10, + ) as usize; + + let virtual_hosted_style_request: bool = + str_option(options, s3_constants::AWS_S3_ADDRESSING_STYLE) + .map(|addressing_style| addressing_style == "virtual") + .unwrap_or(false); + + let allow_unsafe_rename = str_option(options, s3_constants::AWS_S3_ALLOW_UNSAFE_RENAME) + .map(|val| str_is_truthy(&val)) + .unwrap_or(false); + + Self { + endpoint_url, + region, + profile, + aws_access_key_id: str_option(options, s3_constants::AWS_ACCESS_KEY_ID), + aws_secret_access_key: str_option(options, s3_constants::AWS_SECRET_ACCESS_KEY), + aws_session_token: str_option(options, s3_constants::AWS_SESSION_TOKEN), + virtual_hosted_style_request, + locking_provider: str_option(options, s3_constants::AWS_S3_LOCKING_PROVIDER), + assume_role_arn: str_option(options, s3_constants::AWS_S3_ASSUME_ROLE_ARN), + assume_role_session_name: str_option(options, s3_constants::AWS_S3_ROLE_SESSION_NAME), + use_web_identity: std::env::var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE).is_ok(), + s3_pool_idle_timeout: Duration::from_secs(s3_pool_idle_timeout), + sts_pool_idle_timeout: Duration::from_secs(sts_pool_idle_timeout), + s3_get_internal_server_error_retries, + allow_unsafe_rename, + extra_opts, + } + } + + fn str_or_default(map: &HashMap, key: &str, default: String) -> String { + map.get(key) + .map(|v| v.to_owned()) + .unwrap_or_else(|| std::env::var(key).unwrap_or(default)) + } + + fn u64_or_default(map: &HashMap, key: &str, default: u64) -> u64 { + str_option(map, key) + .and_then(|v| v.parse().ok()) + .unwrap_or(default) + } + + fn ensure_env_var(map: &HashMap, key: &str) { + if let Some(val) = str_option(map, key) { + std::env::set_var(key, val); + } + } +} + +impl Default for S3StorageOptions { + /// Creates an instance of S3StorageOptions from environment variables. + fn default() -> S3StorageOptions { + Self::from_map(&HashMap::new()) + } +} + +/// An S3 implementation of the [ObjectStore] trait +pub struct S3StorageBackend { + inner: ObjectStoreRef, + /// Whether allowed to performance rename_if_not_exist as rename + allow_unsafe_rename: bool, +} + +impl std::fmt::Display for S3StorageBackend { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "S3StorageBackend") + } +} + +impl S3StorageBackend { + /// Creates a new S3StorageBackend. + /// + /// Options are described in [s3_constants]. + pub fn try_new(storage: ObjectStoreRef, allow_unsafe_rename: bool) -> ObjectStoreResult { + Ok(Self { + inner: storage, + allow_unsafe_rename, + }) + } +} + +impl std::fmt::Debug for S3StorageBackend { + fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(fmt, "S3StorageBackend") + } +} + +#[async_trait::async_trait] +impl ObjectStore for S3StorageBackend { + async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult<()> { + self.inner.put(location, bytes).await + } + + async fn get(&self, location: &Path) -> ObjectStoreResult { + self.inner.get(location).await + } + + async fn get_opts(&self, location: &Path, options: GetOptions) -> ObjectStoreResult { + self.inner.get_opts(location, options).await + } + + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + self.inner.get_range(location, range).await + } + + async fn head(&self, location: &Path) -> ObjectStoreResult { + self.inner.head(location).await + } + + async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { + self.inner.delete(location).await + } + + async fn list( + &self, + prefix: Option<&Path>, + ) -> ObjectStoreResult>> { + self.inner.list(prefix).await + } + + async fn list_with_offset( + &self, + prefix: Option<&Path>, + offset: &Path, + ) -> ObjectStoreResult>> { + self.inner.list_with_offset(prefix, offset).await + } + + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { + self.inner.list_with_delimiter(prefix).await + } + + async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + self.inner.copy(from, to).await + } + + async fn copy_if_not_exists(&self, _from: &Path, _to: &Path) -> ObjectStoreResult<()> { + todo!() + } + + async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + if self.allow_unsafe_rename { + self.inner.rename(from, to).await + } else { + Err(ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(crate::errors::LockClientError::LockClientRequired), + }) + } + } + + async fn put_multipart( + &self, + location: &Path, + ) -> ObjectStoreResult<(MultipartId, Box)> { + self.inner.put_multipart(location).await + } + + async fn abort_multipart( + &self, + location: &Path, + multipart_id: &MultipartId, + ) -> ObjectStoreResult<()> { + self.inner.abort_multipart(location, multipart_id).await + } +} + +/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. +/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. +/// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. +pub mod s3_constants { + /// Custom S3 endpoint. + pub const AWS_ENDPOINT_URL: &str = "AWS_ENDPOINT_URL"; + /// The AWS region. + pub const AWS_REGION: &str = "AWS_REGION"; + /// The AWS profile. + pub const AWS_PROFILE: &str = "AWS_PROFILE"; + /// The AWS_ACCESS_KEY_ID to use for S3. + pub const AWS_ACCESS_KEY_ID: &str = "AWS_ACCESS_KEY_ID"; + /// The AWS_SECRET_ACCESS_KEY to use for S3. + pub const AWS_SECRET_ACCESS_KEY: &str = "AWS_SECRET_ACCESS_KEY"; + /// The AWS_SESSION_TOKEN to use for S3. + pub const AWS_SESSION_TOKEN: &str = "AWS_SESSION_TOKEN"; + /// Uses either "path" (the default) or "virtual", which turns on + /// [virtual host addressing](http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). + pub const AWS_S3_ADDRESSING_STYLE: &str = "AWS_S3_ADDRESSING_STYLE"; + /// Locking provider to use for safe atomic rename. + /// `dynamodb` is currently the only supported locking provider. + /// If not set, safe atomic rename is not available. + pub const AWS_S3_LOCKING_PROVIDER: &str = "AWS_S3_LOCKING_PROVIDER"; + /// The role to assume for S3 writes. + pub const AWS_S3_ASSUME_ROLE_ARN: &str = "AWS_S3_ASSUME_ROLE_ARN"; + /// The role session name to use when a role is assumed. If not provided a random session name is generated. + pub const AWS_S3_ROLE_SESSION_NAME: &str = "AWS_S3_ROLE_SESSION_NAME"; + /// The `pool_idle_timeout` option of aws http client. Has to be lower than 20 seconds, which is + /// default S3 server timeout . + /// However, since rusoto uses hyper as a client, its default timeout is 90 seconds + /// . + /// Hence, the `connection closed before message completed` could occur. + /// To avoid that, the default value of this setting is 15 seconds if it's not set otherwise. + pub const AWS_S3_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_S3_POOL_IDLE_TIMEOUT_SECONDS"; + /// The `pool_idle_timeout` for the as3_constants sts client. See + /// the reasoning in `AWS_S3_POOL_IDLE_TIMEOUT_SECONDS`. + pub const AWS_STS_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_STS_POOL_IDLE_TIMEOUT_SECONDS"; + /// The number of retries for S3 GET requests failed with 500 Internal Server Error. + pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = + "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES"; + /// The web identity token file to use when using a web identity provider. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; + /// The role name to use for web identity. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; + /// The role session name to use for web identity. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; + /// Allow http connections - mainly useful for integration tests + pub const AWS_ALLOW_HTTP: &str = "AWS_ALLOW_HTTP"; + + /// If set to "true", allows creating commits without concurrent writer protection. + /// Only safe if there is one writer to a given table. + pub const AWS_S3_ALLOW_UNSAFE_RENAME: &str = "AWS_S3_ALLOW_UNSAFE_RENAME"; + + /// The list of option keys owned by the S3 module. + /// Option keys not contained in this list will be added to the `extra_opts` + /// field of [crate::storage::s3::S3StorageOptions]. + pub const S3_OPTS: &[&str] = &[ + AWS_ENDPOINT_URL, + AWS_REGION, + AWS_PROFILE, + AWS_ACCESS_KEY_ID, + AWS_SECRET_ACCESS_KEY, + AWS_SESSION_TOKEN, + AWS_S3_LOCKING_PROVIDER, + AWS_S3_ASSUME_ROLE_ARN, + AWS_S3_ROLE_SESSION_NAME, + AWS_WEB_IDENTITY_TOKEN_FILE, + AWS_ROLE_ARN, + AWS_ROLE_SESSION_NAME, + AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, + AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, + AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + ]; +} + +pub(crate) fn str_option(map: &HashMap, key: &str) -> Option { + map.get(key) + .map_or_else(|| std::env::var(key).ok(), |v| Some(v.to_owned())) +} + +#[cfg(test)] +mod tests { + use super::*; + + use maplit::hashmap; + use serial_test::serial; + + #[test] + #[serial] + fn storage_options_default_test() { + std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "http://localhost"); + std::env::set_var(s3_constants::AWS_REGION, "us-west-1"); + std::env::set_var(s3_constants::AWS_PROFILE, "default"); + std::env::set_var(s3_constants::AWS_ACCESS_KEY_ID, "default_key_id"); + std::env::set_var(s3_constants::AWS_SECRET_ACCESS_KEY, "default_secret_key"); + std::env::set_var(s3_constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + std::env::set_var( + s3_constants::AWS_S3_ASSUME_ROLE_ARN, + "arn:aws:iam::123456789012:role/some_role", + ); + std::env::set_var(s3_constants::AWS_S3_ROLE_SESSION_NAME, "session_name"); + std::env::set_var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE, "token_file"); + std::env::remove_var(s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS); + std::env::remove_var(s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS); + std::env::remove_var(s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES); + + let options = S3StorageOptions::default(); + + assert_eq!( + S3StorageOptions { + endpoint_url: Some("http://localhost".to_string()), + region: Region::Custom { + name: "us-west-1".to_string(), + endpoint: "http://localhost".to_string() + }, + profile: Some("default".to_string()), + aws_access_key_id: Some("default_key_id".to_string()), + aws_secret_access_key: Some("default_secret_key".to_string()), + aws_session_token: None, + virtual_hosted_style_request: false, + assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), + assume_role_session_name: Some("session_name".to_string()), + use_web_identity: true, + locking_provider: Some("dynamodb".to_string()), + s3_pool_idle_timeout: Duration::from_secs(15), + sts_pool_idle_timeout: Duration::from_secs(10), + s3_get_internal_server_error_retries: 10, + extra_opts: HashMap::new(), + allow_unsafe_rename: false, + }, + options + ); + } + + #[test] + #[serial] + fn storage_options_with_only_region_and_credentials() { + std::env::remove_var(s3_constants::AWS_ENDPOINT_URL); + let options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_REGION.to_string() => "eu-west-1".to_string(), + s3_constants::AWS_ACCESS_KEY_ID.to_string() => "test".to_string(), + s3_constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), + }); + + assert_eq!( + S3StorageOptions { + endpoint_url: None, + region: Region::default(), + aws_access_key_id: Some("test".to_string()), + aws_secret_access_key: Some("test_secret".to_string()), + ..Default::default() + }, + options + ); + } + + #[test] + #[serial] + fn storage_options_from_map_test() { + let options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_ENDPOINT_URL.to_string() => "http://localhost:1234".to_string(), + s3_constants::AWS_REGION.to_string() => "us-west-2".to_string(), + s3_constants::AWS_PROFILE.to_string() => "default".to_string(), + s3_constants::AWS_S3_ADDRESSING_STYLE.to_string() => "virtual".to_string(), + s3_constants::AWS_S3_LOCKING_PROVIDER.to_string() => "another_locking_provider".to_string(), + s3_constants::AWS_S3_ASSUME_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/another_role".to_string(), + s3_constants::AWS_S3_ROLE_SESSION_NAME.to_string() => "another_session_name".to_string(), + s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "another_token_file".to_string(), + s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "1".to_string(), + s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "2".to_string(), + s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES.to_string() => "3".to_string(), + s3_constants::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), + s3_constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), + }); + + assert_eq!( + S3StorageOptions { + endpoint_url: Some("http://localhost:1234".to_string()), + region: Region::Custom { + name: "us-west-2".to_string(), + endpoint: "http://localhost:1234".to_string() + }, + profile: Some("default".to_string()), + aws_access_key_id: Some("test_id".to_string()), + aws_secret_access_key: Some("test_secret".to_string()), + aws_session_token: None, + virtual_hosted_style_request: true, + assume_role_arn: Some("arn:aws:iam::123456789012:role/another_role".to_string()), + assume_role_session_name: Some("another_session_name".to_string()), + use_web_identity: true, + locking_provider: Some("another_locking_provider".to_string()), + s3_pool_idle_timeout: Duration::from_secs(1), + sts_pool_idle_timeout: Duration::from_secs(2), + s3_get_internal_server_error_retries: 3, + extra_opts: hashmap! { + s3_constants::AWS_S3_ADDRESSING_STYLE.to_string() => "virtual".to_string() + }, + allow_unsafe_rename: false, + }, + options + ); + } + + #[test] + #[serial] + fn storage_options_mixed_test() { + std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "http://localhost"); + std::env::set_var(s3_constants::AWS_REGION, "us-west-1"); + std::env::set_var(s3_constants::AWS_PROFILE, "default"); + std::env::set_var(s3_constants::AWS_ACCESS_KEY_ID, "wrong_key_id"); + std::env::set_var(s3_constants::AWS_SECRET_ACCESS_KEY, "wrong_secret_key"); + std::env::set_var(s3_constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + std::env::set_var( + s3_constants::AWS_S3_ASSUME_ROLE_ARN, + "arn:aws:iam::123456789012:role/some_role", + ); + std::env::set_var(s3_constants::AWS_S3_ROLE_SESSION_NAME, "session_name"); + std::env::set_var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE, "token_file"); + + std::env::set_var(s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, "1"); + std::env::set_var(s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, "2"); + std::env::set_var(s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, "3"); + let options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_ACCESS_KEY_ID.to_string() => "test_id_mixed".to_string(), + s3_constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret_mixed".to_string(), + s3_constants::AWS_REGION.to_string() => "us-west-2".to_string(), + "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), + "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES".to_string() => "3".to_string(), + }); + + assert_eq!( + S3StorageOptions { + endpoint_url: Some("http://localhost".to_string()), + region: Region::Custom { + name: "us-west-2".to_string(), + endpoint: "http://localhost".to_string() + }, + profile: Some("default".to_string()), + aws_access_key_id: Some("test_id_mixed".to_string()), + aws_secret_access_key: Some("test_secret_mixed".to_string()), + aws_session_token: None, + virtual_hosted_style_request: false, + assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), + assume_role_session_name: Some("session_name".to_string()), + use_web_identity: true, + locking_provider: Some("dynamodb".to_string()), + s3_pool_idle_timeout: Duration::from_secs(1), + sts_pool_idle_timeout: Duration::from_secs(2), + s3_get_internal_server_error_retries: 3, + extra_opts: hashmap! { + "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), + }, + allow_unsafe_rename: false, + }, + options + ); + } + #[test] + #[serial] + fn storage_options_web_identity_test() { + let _options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_REGION.to_string() => "eu-west-1".to_string(), + s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "web_identity_token_file".to_string(), + s3_constants::AWS_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/web_identity_role".to_string(), + s3_constants::AWS_ROLE_SESSION_NAME.to_string() => "web_identity_session_name".to_string(), + }); + + assert_eq!( + "eu-west-1", + std::env::var(s3_constants::AWS_REGION).unwrap() + ); + + assert_eq!( + "web_identity_token_file", + std::env::var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE).unwrap() + ); + + assert_eq!( + "arn:aws:iam::123456789012:role/web_identity_role", + std::env::var(s3_constants::AWS_ROLE_ARN).unwrap() + ); + + assert_eq!( + "web_identity_session_name", + std::env::var(s3_constants::AWS_ROLE_SESSION_NAME).unwrap() + ); + } +} diff --git a/crates/deltalake-aws/tests/common.rs b/crates/deltalake-aws/tests/common.rs new file mode 100644 index 0000000000..764c861c92 --- /dev/null +++ b/crates/deltalake-aws/tests/common.rs @@ -0,0 +1,173 @@ +use chrono::Utc; +use deltalake_aws::register_handlers; +use deltalake_aws::storage::*; +use deltalake_test::utils::*; +use rand::Rng; +use std::process::{Command, ExitStatus, Stdio}; + +#[derive(Clone, Debug)] +pub struct S3Integration { + bucket_name: String, +} + +impl Default for S3Integration { + fn default() -> Self { + register_handlers(None); + Self { + bucket_name: format!("test-delta-table-{}", Utc::now().timestamp()), + } + } +} + +impl StorageIntegration for S3Integration { + /// Create a new bucket + fn create_bucket(&self) -> std::io::Result { + set_env_if_not_set( + "DYNAMO_LOCK_PARTITION_KEY_VALUE", + format!("s3://{}", self.bucket_name()), + ); + Self::create_lock_table()?; + let mut child = Command::new("aws") + .args(["s3", "mb", &self.root_uri()]) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + fn bucket_name(&self) -> String { + self.bucket_name.clone() + } + + fn root_uri(&self) -> String { + format!("s3://{}", &self.bucket_name()) + } + + /// prepare_env + fn prepare_env(&self) { + std::env::set_var( + "DELTA_DYNAMO_TABLE_NAME", + format!("delta_log_it_{}", rand::thread_rng().gen::()), + ); + match std::env::var(s3_constants::AWS_ENDPOINT_URL).ok() { + Some(endpoint_url) if endpoint_url.to_lowercase() == "none" => { + std::env::remove_var(s3_constants::AWS_ENDPOINT_URL) + } + Some(_) => (), + None => std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "http://localhost:4566"), + } + set_env_if_not_set(s3_constants::AWS_ACCESS_KEY_ID, "deltalake"); + set_env_if_not_set(s3_constants::AWS_SECRET_ACCESS_KEY, "weloverust"); + set_env_if_not_set(s3_constants::AWS_REGION, "us-east-1"); + set_env_if_not_set(s3_constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + set_env_if_not_set("DYNAMO_LOCK_TABLE_NAME", "test_table"); + set_env_if_not_set("DYNAMO_LOCK_REFRESH_PERIOD_MILLIS", "100"); + set_env_if_not_set("DYNAMO_LOCK_ADDITIONAL_TIME_TO_WAIT_MILLIS", "100"); + } + + /// copy directory + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + let destination = format!("{}/{destination}", self.root_uri()); + let mut child = Command::new("aws") + .args(["s3", "cp", source, &destination, "--recursive"]) + .spawn() + .expect("aws command is installed"); + child.wait() + } +} + +impl S3Integration { + /// delete bucket + fn delete_bucket(bucket_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("aws") + .args(["s3", "rb", bucket_name.as_ref(), "--force"]) + .spawn() + .expect("aws command is installed"); + child.wait() + } + fn create_dynamodb_table( + table_name: &str, + attr_definitions: &[&str], + key_schema: &[&str], + ) -> std::io::Result { + let args = [ + "dynamodb", + "create-table", + "--table-name", + &table_name, + "--provisioned-throughput", + "ReadCapacityUnits=10,WriteCapacityUnits=10", + "--attribute-definitions", + ]; + let mut child = Command::new("aws") + .args(args) + .args(attr_definitions.iter()) + .arg("--key-schema") + .args(key_schema) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + let status = child.wait()?; + Self::wait_for_table(table_name)?; + Ok(status) + } + + fn find_subsequence(haystack: &[u8], needle: &[u8]) -> Option { + haystack + .windows(needle.len()) + .position(|window| window == needle) + } + + fn wait_for_table(table_name: &str) -> std::io::Result<()> { + let args = ["dynamodb", "describe-table", "--table-name", &table_name]; + loop { + let output = Command::new("aws") + .args(args) + .output() + .expect("aws command is installed"); + if Self::find_subsequence(&output.stdout, "CREATING".as_bytes()).is_some() { + std::thread::sleep(std::time::Duration::from_millis(200)); + continue; + } else { + return Ok(()); + } + } + } + + pub fn create_lock_table() -> std::io::Result { + let table_name = + std::env::var("DELTA_DYNAMO_TABLE_NAME").unwrap_or_else(|_| "delta_log".into()); + Self::create_dynamodb_table( + &table_name, + &[ + "AttributeName=tablePath,AttributeType=S", + "AttributeName=fileName,AttributeType=S", + ], + &[ + "AttributeName=tablePath,KeyType=HASH", + "AttributeName=fileName,KeyType=RANGE", + ], + ) + } + + fn delete_dynamodb_table(table_name: &str) -> std::io::Result { + let mut child = Command::new("aws") + .args(["dynamodb", "delete-table", "--table-name", &table_name]) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + pub fn delete_lock_table() -> std::io::Result { + let table_name = + std::env::var("DELTA_DYNAMO_TABLE_NAME").unwrap_or_else(|_| "delta_log".into()); + Self::delete_dynamodb_table(&table_name) + } +} + +impl Drop for S3Integration { + fn drop(&mut self) { + Self::delete_bucket(self.root_uri()).expect("Failed to drop bucket"); + Self::delete_lock_table().expect("Failed to delete lock table"); + } +} diff --git a/crates/deltalake-aws/tests/integration_read.rs b/crates/deltalake-aws/tests/integration_read.rs new file mode 100644 index 0000000000..5e9c6f1040 --- /dev/null +++ b/crates/deltalake-aws/tests/integration_read.rs @@ -0,0 +1,189 @@ +#![cfg(feature = "integration_test")] + +use deltalake_core::{DeltaTableBuilder, Path}; +use deltalake_test::utils::*; +use serial_test::serial; + +mod common; +use common::*; + +static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; + +/// TEST_PREFIXES as they should appear in object stores. +static TEST_PREFIXES_ENCODED: &[&str] = &["my%20table", "%E4%BD%A0%E5%A5%BD/%F0%9F%98%8A"]; + +#[tokio::test] +#[serial] +async fn test_read_tables_aws() -> TestResult { + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; + + read_tables(&context).await?; + + for (prefix, prefix_encoded) in TEST_PREFIXES.iter().zip(TEST_PREFIXES_ENCODED.iter()) { + read_table_paths(&context, prefix, prefix_encoded).await?; + } + + Ok(()) +} + +async fn read_tables(context: &IntegrationContext) -> TestResult { + context.load_table(TestTables::Simple).await?; + context.load_table(TestTables::Golden).await?; + context + .load_table(TestTables::Delta0_8_0SpecialPartitioned) + .await?; + + read_simple_table(&context).await?; + read_simple_table_with_version(&context).await?; + read_golden(&context).await?; + + Ok(()) +} + +async fn read_table_paths( + context: &IntegrationContext, + table_root: &str, + upload_path: &str, +) -> TestResult { + context + .load_table_with_name(TestTables::Delta0_8_0SpecialPartitioned, upload_path) + .await?; + + println!("table_root: {}", table_root); + verify_store(&context, table_root).await?; + + read_encoded_table(&context, table_root).await?; + + Ok(()) +} + +async fn verify_store(integration: &IntegrationContext, root_path: &str) -> TestResult { + let table_uri = format!("{}/{}", integration.root_uri(), root_path); + println!("working with table_uri: {}", table_uri); + let storage = DeltaTableBuilder::from_uri(table_uri.clone()) + .with_allow_http(true) + .build_storage()? + .object_store(); + + let files = storage.list_with_delimiter(None).await?; + println!("files: {files:?}"); + assert_eq!( + vec![ + Path::parse("_delta_log").unwrap(), + Path::parse("x=A%2FA").unwrap(), + Path::parse("x=B%20B").unwrap(), + ], + files.common_prefixes + ); + + Ok(()) +} + +async fn read_encoded_table(integration: &IntegrationContext, root_path: &str) -> TestResult { + let table_uri = format!("{}/{}", integration.root_uri(), root_path); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .load() + .await?; + + assert_eq!(table.version(), 0); + assert_eq!(table.get_files().len(), 2); + + Ok(()) +} + +async fn read_simple_table(integration: &IntegrationContext) -> TestResult { + let table_uri = integration.uri_for_table(TestTables::Simple); + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .load() + .await?; + + assert_eq!(table.version(), 4); + assert_eq!(table.protocol().min_writer_version, 2); + assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!( + table.get_files(), + vec![ + Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), + Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), + Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), + Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), + Path::from("part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"), + ] + ); + let tombstones = table.get_state().all_tombstones(); + assert_eq!(tombstones.len(), 31); + assert!(tombstones.contains(&deltalake_core::kernel::Remove { + path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), + deletion_timestamp: Some(1587968596250), + data_change: true, + extended_file_metadata: None, + deletion_vector: None, + base_row_id: None, + default_row_commit_version: None, + size: None, + partition_values: None, + tags: None, + })); + + Ok(()) +} + +async fn read_simple_table_with_version(integration: &IntegrationContext) -> TestResult { + let table_uri = integration.uri_for_table(TestTables::Simple); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .with_version(3) + .load() + .await?; + + assert_eq!(table.version(), 3); + assert_eq!(table.protocol().min_writer_version, 2); + assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!( + table.get_files(), + vec![ + Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), + Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), + Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), + Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), + Path::from("part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet"), + Path::from("part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet"), + ] + ); + let tombstones = table.get_state().all_tombstones(); + assert_eq!(tombstones.len(), 29); + assert!(tombstones.contains(&deltalake_core::kernel::Remove { + path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), + deletion_timestamp: Some(1587968596250), + data_change: true, + tags: None, + partition_values: None, + base_row_id: None, + default_row_commit_version: None, + size: None, + deletion_vector: None, + extended_file_metadata: None, + })); + + Ok(()) +} + +async fn read_golden(integration: &IntegrationContext) -> TestResult { + let table_uri = integration.uri_for_table(TestTables::Golden); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .load() + .await + .unwrap(); + + assert_eq!(table.version(), 0); + assert_eq!(table.protocol().min_writer_version, 2); + assert_eq!(table.protocol().min_reader_version, 1); + + Ok(()) +} diff --git a/crates/deltalake-core/tests/integration_s3_dynamodb.rs b/crates/deltalake-aws/tests/integration_s3_dynamodb.rs similarity index 91% rename from crates/deltalake-core/tests/integration_s3_dynamodb.rs rename to crates/deltalake-aws/tests/integration_s3_dynamodb.rs index 38bd8e3a16..7338ca1509 100644 --- a/crates/deltalake-core/tests/integration_s3_dynamodb.rs +++ b/crates/deltalake-aws/tests/integration_s3_dynamodb.rs @@ -1,32 +1,29 @@ //! Integration test to verify correct behavior of S3 DynamoDb locking. //! It inspects the state of the locking table after each operation. -#![cfg(all( - feature = "integration_test", - any(feature = "s3", feature = "s3-native-tls") -))] +#![cfg(feature = "integration_test")] use std::collections::HashMap; use std::time::{Duration, SystemTime, UNIX_EPOCH}; +use deltalake_aws::logstore::{RepairLogEntryResult, S3DynamoDbLogStore}; +use deltalake_aws::storage::S3StorageOptions; use deltalake_aws::{CommitEntry, DynamoDbLockClient}; use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; -use deltalake_core::logstore::s3::{RepairLogEntryResult, S3DynamoDbLogStore}; use deltalake_core::logstore::LogStore; use deltalake_core::operations::transaction::{commit, prepare_commit}; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use deltalake_core::storage::commit_uri_from_version; -use deltalake_core::storage::config::StorageOptions; -use deltalake_core::storage::s3::S3StorageOptions; +use deltalake_core::storage::StorageOptions; use deltalake_core::table::builder::ensure_table_uri; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestTables}; use deltalake_core::{DeltaOps, DeltaTable, DeltaTableBuilder}; +use deltalake_test::utils::*; use lazy_static::lazy_static; use object_store::path::Path; use serde_json::Value; use serial_test::serial; -#[allow(dead_code)] -mod fs_common; +mod common; +use common::*; pub type TestResult = Result>; @@ -49,7 +46,7 @@ fn make_client() -> TestResult { #[test] #[serial] fn client_config_picks_up_lock_table_name() -> TestResult<()> { - let _context = IntegrationContext::new(StorageIntegration::Amazon)?; + let _context = IntegrationContext::new(Box::new(S3Integration::default()))?; assert!(make_client()? .get_lock_table_name() .starts_with("delta_log_it_")); @@ -59,7 +56,7 @@ fn client_config_picks_up_lock_table_name() -> TestResult<()> { #[tokio::test] #[serial] async fn get_missing_item() -> TestResult<()> { - let _context = IntegrationContext::new(StorageIntegration::Amazon)?; + let _context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let version = i64::MAX; let result = client @@ -75,7 +72,7 @@ async fn get_missing_item() -> TestResult<()> { #[tokio::test] #[serial] async fn test_append() -> TestResult<()> { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let table = prepare_table(&context, "delta01").await?; validate_lock_table_state(&table, 0).await?; append_to_table("datav01.parquet", &table, None).await?; @@ -86,7 +83,7 @@ async fn test_append() -> TestResult<()> { #[tokio::test] #[serial] async fn test_repair_commit_entry() -> TestResult<()> { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let table = prepare_table(&context, "repair_needed").await?; let options: StorageOptions = OPTIONS.clone().into(); @@ -135,7 +132,7 @@ async fn test_repair_commit_entry() -> TestResult<()> { #[tokio::test] #[serial] async fn test_repair_on_update() -> TestResult<()> { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let mut table = prepare_table(&context, "repair_on_update").await?; let _entry = create_incomplete_commit_entry(&table, 1, "unfinished_commit").await?; table.update().await?; @@ -152,9 +149,12 @@ const COMMITS: i64 = 5; #[serial] async fn test_concurrent_writers() -> TestResult<()> { // Goal: a test with multiple writers, very similar to `integration_concurrent_writes` - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; + println!(">>> preparing table"); let table = prepare_table(&context, "concurrent_writes").await?; + println!(">>> table prepared"); let table_uri = table.table_uri(); + println!("Starting workers on {table_uri}"); let mut workers = Vec::new(); for w in 0..WORKERS { @@ -187,6 +187,7 @@ impl Worker { .load() .await .unwrap(); + println!("Loaded table in worker: {table:?}"); Self { table, name } } @@ -271,11 +272,13 @@ async fn prepare_table(context: &IntegrationContext, table_name: &str) -> TestRe .with_allow_http(true) .with_storage_options(OPTIONS.clone()) .build()?; + println!("table built: {table:?}"); // create delta table let table = DeltaOps(table) .create() .with_columns(schema.fields().clone()) .await?; + println!("table created: {table:?}"); Ok(table) } diff --git a/crates/deltalake-core/tests/repair_s3_rename_test.rs b/crates/deltalake-aws/tests/repair_s3_rename_test.rs similarity index 95% rename from crates/deltalake-core/tests/repair_s3_rename_test.rs rename to crates/deltalake-aws/tests/repair_s3_rename_test.rs index ecab792f39..a48af20d0f 100644 --- a/crates/deltalake-core/tests/repair_s3_rename_test.rs +++ b/crates/deltalake-aws/tests/repair_s3_rename_test.rs @@ -1,16 +1,14 @@ -#![cfg(all( - any(feature = "s3", feature = "s3-native-tls"), - feature = "integration_test" -))] +#![cfg(feature = "integration_test")] + use bytes::Bytes; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration}; -use deltalake_core::{storage::s3::S3StorageBackend, DeltaTableBuilder, ObjectStore}; -use futures::stream::BoxStream; -use object_store::path::Path; -use object_store::{ +use deltalake_aws::storage::S3StorageBackend; +use deltalake_core::storage::object_store::{ DynObjectStore, Error as ObjectStoreError, GetOptions, GetResult, ListResult, MultipartId, ObjectMeta, Result as ObjectStoreResult, }; +use deltalake_core::{DeltaTableBuilder, ObjectStore, Path}; +use deltalake_test::utils::{IntegrationContext, StorageIntegration}; +use futures::stream::BoxStream; use serial_test::serial; use std::ops::Range; use std::sync::{Arc, Mutex}; @@ -18,6 +16,9 @@ use tokio::io::AsyncWrite; use tokio::task::JoinHandle; use tokio::time::Duration; +mod common; +use common::*; + #[tokio::test(flavor = "multi_thread")] #[serial] #[ignore = "currently tests are hanging"] @@ -43,7 +44,7 @@ async fn repair_when_worker_pauses_after_rename_test() { async fn run_repair_test_case(path: &str, pause_copy: bool) -> Result<(), ObjectStoreError> { std::env::set_var("AWS_S3_LOCKING_PROVIDER", "dynamodb"); std::env::set_var("DYNAMO_LOCK_LEASE_DURATION", "2"); - let context = IntegrationContext::new(StorageIntegration::Amazon).unwrap(); + let context = IntegrationContext::new(Box::new(S3Integration::default())).unwrap(); let root_path = Path::from(path); let src1 = root_path.child("src1"); diff --git a/crates/deltalake-core/.gitignore b/crates/deltalake-core/.gitignore index a403c2926d..fd7fc6ad51 100644 --- a/crates/deltalake-core/.gitignore +++ b/crates/deltalake-core/.gitignore @@ -1,12 +1,4 @@ target/ /.idea/ *.bat -tests/data/checkpoints_tombstones/expired/ -tests/data/checkpoints_tombstones/metadata_broken/ -tests/data/checkpoints_tombstones/metadata_false/ -tests/data/checkpoints_tombstones/metadata_true/ -tests/data/checkpoints_with_expired_logs/ -tests/data/read_null_partitions_from_checkpoint/ -tests/data/action_reconciliation/ -tests/data/simple_table_with_no_checkpoint/ -tests/data/simple_table_with_no_checkpoint_2/ +tests/data diff --git a/crates/deltalake-core/Cargo.toml b/crates/deltalake-core/Cargo.toml index d29712baeb..8ace57f36c 100644 --- a/crates/deltalake-core/Cargo.toml +++ b/crates/deltalake-core/Cargo.toml @@ -81,7 +81,6 @@ num-traits = "0.2.15" object_store = "0.7" once_cell = "1.16.0" parking_lot = "0.12" -parquet2 = { version = "0.17", optional = true } percent-encoding = "2" roaring = "0.10.1" tracing = { version = "0.1", optional = true } @@ -89,19 +88,6 @@ rand = "0.8" z85 = "3.0.5" maplit = "1" -# hdfs -datafusion-objectstore-hdfs = { version = "0.1.3", default-features = false, features = [ - "hdfs3", - "try_spawn_blocking", -], optional = true } - -# S3 lock client -rusoto_core = { version = "0.47", default-features = false, optional = true } -rusoto_credential = { version = "0.47", optional = true } -rusoto_sts = { version = "0.47", default-features = false, optional = true } -deltalake-aws = { path = "../deltalake-aws", default-features = false, optional = true } - - # Unity reqwest = { version = "0.11.18", default-features = false, features = [ "rustls-tls", @@ -109,7 +95,7 @@ reqwest = { version = "0.11.18", default-features = false, features = [ ], optional = true } # Datafusion -dashmap = { version = "5", optional = true } +dashmap = "5" sqlparser = { version = "0.39", optional = true } @@ -118,18 +104,20 @@ fs_extra = { version = "1.3.0", optional = true } tempdir = { version = "0", optional = true } [dev-dependencies] +criterion = "0.5" ctor = "0" +deltalake-test = { path = "../deltalake-test", features = ["datafusion"] } dotenvy = "0" +hyper = { version = "0.14", features = ["server"] } maplit = "1" pretty_assertions = "1.2.1" +pretty_env_logger = "*" rand = "0.8" serial_test = "2" tempdir = "0" tempfile = "3" tokio = { version = "1", features = ["macros", "rt-multi-thread"] } utime = "0.3" -hyper = { version = "0.14", features = ["server"] } -criterion = "0.5" [features] azure = ["object_store/azure"] @@ -146,7 +134,6 @@ arrow = [ ] default = ["arrow", "parquet"] datafusion = [ - "dep:arrow", "dep:datafusion", "datafusion-expr", "datafusion-common", @@ -156,29 +143,13 @@ datafusion = [ "sqlparser", "arrow", "parquet", - "dashmap", ] datafusion-ext = ["datafusion"] gcs = ["object_store/gcp"] -hdfs = ["datafusion-objectstore-hdfs"] # used only for integration testing integration_test = ["fs_extra", "tempdir"] json = ["parquet/json"] python = ["arrow/pyarrow"] -s3-native-tls = [ - "rusoto_core/native-tls", - "rusoto_credential", - "rusoto_sts/native-tls", - "object_store/aws", - "deltalake-aws/native-tls", -] -s3 = [ - "rusoto_core/rustls", - "rusoto_credential", - "rusoto_sts/rustls", - "object_store/aws", - "deltalake-aws/rustls", -] unity-experimental = ["reqwest", "tracing", "hyper"] [[bench]] diff --git a/crates/deltalake-core/README.md b/crates/deltalake-core/README.md index 64d17dcae9..7cb674ea11 100644 --- a/crates/deltalake-core/README.md +++ b/crates/deltalake-core/README.md @@ -48,11 +48,7 @@ cargo run --example read_delta_table - `datafusion` - enable the `datafusion::datasource::TableProvider` trait implementation for Delta Tables, allowing them to be queried using [DataFusion](https://github.com/apache/arrow-datafusion). - `datafusion-ext` - DEPRECATED: alias for `datafusion` feature - `gcs` - enable the Google storage backend to work with Delta Tables in Google Cloud Storage. -- `hdfs` - enable the HDFS storage backend to work with Delta Tables in HDFS. - `json` - enable the JSON feature of the `parquet` crate for better JSON interoperability. -- `parquet2` - use parquet2 for checkpoint deserialization. Since `arrow` and `parquet` features are enabled by default for backwards compatibility, this feature needs to be used with `--no-default-features`. -- `s3` - enable the S3 storage backend to work with Delta Tables in AWS S3. Uses [rustls](https://github.com/ctz/rustls). -- `s3-native-tls` - enable the S3 storage backend but rely on OpenSSL. ## Development diff --git a/crates/deltalake-core/src/data_catalog/storage/mod.rs b/crates/deltalake-core/src/data_catalog/storage/mod.rs index 729b5de224..a332464952 100644 --- a/crates/deltalake-core/src/data_catalog/storage/mod.rs +++ b/crates/deltalake-core/src/data_catalog/storage/mod.rs @@ -14,7 +14,7 @@ use object_store::ObjectStore; use crate::errors::DeltaResult; use crate::open_table_with_storage_options; -use crate::storage::config::{configure_store, StorageOptions}; +use crate::storage::*; use crate::table::builder::ensure_table_uri; const DELTA_LOG_FOLDER: &str = "_delta_log"; @@ -47,9 +47,9 @@ impl ListingSchemaProvider { storage_options: Option>, ) -> DeltaResult { let uri = ensure_table_uri(root_uri)?; - let mut storage_options = storage_options.unwrap_or_default().into(); + let storage_options = storage_options.unwrap_or_default().into(); // We already parsed the url, so unwrapping is safe. - let store = configure_store(&uri, &mut storage_options)?; + let store = store_for(&uri)?; Ok(Self { authority: uri.to_string(), store, @@ -163,7 +163,7 @@ mod tests { #[tokio::test] async fn test_table_names() { - let fs = ListingSchemaProvider::try_new("./tests/data/", None).unwrap(); + let fs = ListingSchemaProvider::try_new("../deltalake-test/tests/data/", None).unwrap(); fs.refresh().await.unwrap(); let table_names = fs.table_names(); assert!(table_names.len() > 20); @@ -172,7 +172,9 @@ mod tests { #[tokio::test] async fn test_query_table() { - let schema = Arc::new(ListingSchemaProvider::try_new("./tests/data/", None).unwrap()); + let schema = Arc::new( + ListingSchemaProvider::try_new("../deltalake-test/tests/data/", None).unwrap(), + ); schema.refresh().await.unwrap(); let ctx = SessionContext::new(); diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 17d04c692a..038828d31b 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -1907,7 +1907,7 @@ mod tests { #[tokio::test] async fn delta_table_provider_with_config() { - let table = crate::open_table("tests/data/delta-2.2.0-partitioned-types") + let table = crate::open_table("../deltalake-test/tests/data/delta-2.2.0-partitioned-types") .await .unwrap(); let config = DeltaScanConfigBuilder::new() diff --git a/crates/deltalake-core/src/errors.rs b/crates/deltalake-core/src/errors.rs index aaa21a4801..67963042f8 100644 --- a/crates/deltalake-core/src/errors.rs +++ b/crates/deltalake-core/src/errors.rs @@ -23,17 +23,13 @@ pub enum DeltaTableError { }, /// Error returned when parsing checkpoint parquet. - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] #[error("Failed to parse parquet: {}", .source)] Parquet { /// Parquet error details returned when reading the checkpoint failed. #[cfg(feature = "parquet")] #[from] source: parquet::errors::ParquetError, - /// Parquet error details returned when reading the checkpoint failed. - #[cfg(feature = "parquet2")] - #[from] - source: parquet2::error::Error, }, /// Error returned when converting the schema in Arrow format failed. @@ -231,6 +227,7 @@ impl From for DeltaTableError { ProtocolError::Arrow { source } => DeltaTableError::Arrow { source }, ProtocolError::IO { source } => DeltaTableError::Io { source }, ProtocolError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, + #[cfg(feature = "parquet")] ProtocolError::ParquetParseError { source } => DeltaTableError::Parquet { source }, _ => DeltaTableError::Protocol { source: value }, } diff --git a/crates/deltalake-core/src/kernel/actions/types.rs b/crates/deltalake-core/src/kernel/actions/types.rs index f64a5caa08..67a94ec1c4 100644 --- a/crates/deltalake-core/src/kernel/actions/types.rs +++ b/crates/deltalake-core/src/kernel/actions/types.rs @@ -187,7 +187,7 @@ pub enum ReaderFeatures { Other(String), } -#[cfg(all(not(feature = "parquet2"), feature = "parquet"))] +#[cfg(feature = "parquet")] impl From<&parquet::record::Field> for ReaderFeatures { fn from(value: &parquet::record::Field) -> Self { match value { @@ -330,7 +330,7 @@ impl fmt::Display for WriterFeatures { } } -#[cfg(all(not(feature = "parquet2"), feature = "parquet"))] +#[cfg(feature = "parquet")] impl From<&parquet::record::Field> for WriterFeatures { fn from(value: &parquet::record::Field) -> Self { match value { @@ -599,10 +599,6 @@ pub struct Add { #[cfg(feature = "parquet")] #[serde(skip_serializing, skip_deserializing)] pub partition_values_parsed: Option, - /// Partition values parsed for parquet2 - #[cfg(feature = "parquet2")] - #[serde(skip_serializing, skip_deserializing)] - pub partition_values_parsed: Option, /// Contains statistics (e.g., count, min/max values for columns) about the data in this file in /// raw parquet format. This field needs to be written when statistics are available and the @@ -612,10 +608,6 @@ pub struct Add { #[cfg(feature = "parquet")] #[serde(skip_serializing, skip_deserializing)] pub stats_parsed: Option, - /// Stats parsed for parquet2 - #[cfg(feature = "parquet2")] - #[serde(skip_serializing, skip_deserializing)] - pub stats_parsed: Option, } impl Add { @@ -951,8 +943,10 @@ mod tests { let inline = dv_inline(); assert_eq!(None, inline.absolute_path(&parent).unwrap()); - let path = - std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + let path = std::fs::canonicalize(PathBuf::from( + "../deltalake-test/tests/data/table-with-dv-small/", + )) + .unwrap(); let parent = url::Url::from_directory_path(path).unwrap(); let dv_url = parent .join("deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin") @@ -971,7 +965,7 @@ mod tests { // fn test_deletion_vector_read() { // let store = Arc::new(LocalFileSystem::new()); // let path = - // std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + // std::fs::canonicalize(PathBuf::from("../deltalake-test/tests/data/table-with-dv-small/")).unwrap(); // let parent = url::Url::from_directory_path(path).unwrap(); // let root = object_store::path::Path::from(parent.path()); // let fs_client = Arc::new(ObjectStoreFileSystemClient::new( diff --git a/crates/deltalake-core/src/lib.rs b/crates/deltalake-core/src/lib.rs index d95f8d40e3..ba6f17d032 100644 --- a/crates/deltalake-core/src/lib.rs +++ b/crates/deltalake-core/src/lib.rs @@ -6,7 +6,7 @@ //! //! ```rust //! async { -//! let table = deltalake_core::open_table("./tests/data/simple_table").await.unwrap(); +//! let table = deltalake_core::open_table("../deltalake-test/tests/data/simple_table").await.unwrap(); //! let files = table.get_files(); //! }; //! ``` @@ -15,7 +15,7 @@ //! //! ```rust //! async { -//! let table = deltalake_core::open_table_with_version("./tests/data/simple_table", 0).await.unwrap(); +//! let table = deltalake_core::open_table_with_version("../deltalake-test/tests/data/simple_table", 0).await.unwrap(); //! let files = table.get_files_by_partitions(&[deltalake_core::PartitionFilter { //! key: "month".to_string(), //! value: deltalake_core::PartitionValue::Equal("12".to_string()), @@ -28,7 +28,7 @@ //! ```rust //! async { //! let table = deltalake_core::open_table_with_ds( -//! "./tests/data/simple_table", +//! "../deltalake-test/tests/data/simple_table", //! "2020-05-02T23:47:31-07:00", //! ).await.unwrap(); //! let files = table.get_files(); @@ -43,8 +43,6 @@ //! - `datafusion` - enable the `datafusion::datasource::TableProvider` trait implementation //! for Delta Tables, allowing them to be queried using [DataFusion](https://github.com/apache/arrow-datafusion). //! - `datafusion-ext` - DEPRECATED: alias for `datafusion` feature. -//! - `parquet2` - use parquet2 for checkpoint deserialization. Since `arrow` and `parquet` features -//! are enabled by default for backwards compatibility, this feature needs to be used with `--no-default-features`. //! //! # Querying Delta Tables with Datafusion //! @@ -55,7 +53,7 @@ //! //! async { //! let mut ctx = SessionContext::new(); -//! let table = deltalake_core::open_table("./tests/data/simple_table") +//! let table = deltalake_core::open_table("../deltalake-test/tests/data/simple_table") //! .await //! .unwrap(); //! ctx.register_table("demo", Arc::new(table)).unwrap(); @@ -71,16 +69,6 @@ #![allow(rustdoc::invalid_html_tags)] #![allow(clippy::nonminimal_bool)] -#[cfg(all(feature = "parquet", feature = "parquet2"))] -compile_error!( - "Features parquet and parquet2 are mutually exclusive and cannot be enabled together" -); - -#[cfg(all(feature = "s3", feature = "s3-native-tls"))] -compile_error!( - "Features s3 and s3-native-tls are mutually exclusive and cannot be enabled together" -); - #[cfg(all(feature = "glue", feature = "glue-native-tls"))] compile_error!( "Features glue and glue-native-tls are mutually exclusive and cannot be enabled together" @@ -122,16 +110,9 @@ pub use arrow; pub use datafusion; #[cfg(feature = "parquet")] pub use parquet; -#[cfg(feature = "parquet2")] -pub use parquet2; #[cfg(all(feature = "arrow", feature = "parquet"))] pub use protocol::checkpoints; -// needed only for integration tests -// TODO can / should we move this into the test crate? -#[cfg(feature = "integration_test")] -pub mod test_utils; - /// Creates and loads a DeltaTable from the given path with current metadata. /// Infers the storage backend to use from the scheme in the given table path. /// @@ -202,7 +183,9 @@ mod tests { #[tokio::test] async fn read_delta_2_0_table_without_version() { - let table = crate::open_table("./tests/data/delta-0.2.0").await.unwrap(); + let table = crate::open_table("../deltalake-test/tests/data/delta-0.2.0") + .await + .unwrap(); assert_eq!(table.version(), 3); assert_eq!(table.protocol().min_writer_version, 2); assert_eq!(table.protocol().min_reader_version, 1); @@ -232,7 +215,7 @@ mod tests { #[tokio::test] async fn read_delta_table_with_update() { - let path = "./tests/data/simple_table_with_checkpoint/"; + let path = "../deltalake-test/tests/data/simple_table_with_checkpoint/"; let table_newest_version = crate::open_table(path).await.unwrap(); let mut table_to_update = crate::open_table_with_version(path, 0).await.unwrap(); // calling update several times should not produce any duplicates @@ -247,9 +230,10 @@ mod tests { } #[tokio::test] async fn read_delta_2_0_table_with_version() { - let mut table = crate::open_table_with_version("./tests/data/delta-0.2.0", 0) - .await - .unwrap(); + let mut table = + crate::open_table_with_version("../deltalake-test/tests/data/delta-0.2.0", 0) + .await + .unwrap(); assert_eq!(table.version(), 0); assert_eq!(table.protocol().min_writer_version, 2); assert_eq!(table.protocol().min_reader_version, 1); @@ -261,7 +245,7 @@ mod tests { ], ); - table = crate::open_table_with_version("./tests/data/delta-0.2.0", 2) + table = crate::open_table_with_version("../deltalake-test/tests/data/delta-0.2.0", 2) .await .unwrap(); assert_eq!(table.version(), 2); @@ -275,7 +259,7 @@ mod tests { ] ); - table = crate::open_table_with_version("./tests/data/delta-0.2.0", 3) + table = crate::open_table_with_version("../deltalake-test/tests/data/delta-0.2.0", 3) .await .unwrap(); assert_eq!(table.version(), 3); @@ -293,7 +277,9 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_without_version() { - let table = crate::open_table("./tests/data/delta-0.8.0").await.unwrap(); + let table = crate::open_table("../deltalake-test/tests/data/delta-0.8.0") + .await + .unwrap(); assert_eq!(table.version(), 1); assert_eq!(table.protocol().min_writer_version, 2); assert_eq!(table.protocol().min_reader_version, 1); @@ -339,7 +325,9 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_with_load_version() { - let mut table = crate::open_table("./tests/data/delta-0.8.0").await.unwrap(); + let mut table = crate::open_table("../deltalake-test/tests/data/delta-0.8.0") + .await + .unwrap(); assert_eq!(table.version(), 1); assert_eq!(table.protocol().min_writer_version, 2); assert_eq!(table.protocol().min_reader_version, 1); @@ -365,8 +353,7 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_with_partitions() { - let current_dir = Path::from_filesystem_path(std::env::current_dir().unwrap()).unwrap(); - let table = crate::open_table("./tests/data/delta-0.8.0-partitioned") + let table = crate::open_table("../deltalake-test/tests/data/delta-0.8.0-partitioned") .await .unwrap(); @@ -388,23 +375,13 @@ mod tests { Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet") ] ); - - #[cfg(unix)] assert_eq!( table.get_file_uris_by_partitions(&filters).unwrap(), vec![ - format!("/{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet", current_dir.as_ref()), - format!("/{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet", current_dir.as_ref()) + std::fs::canonicalize("../deltalake-test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet").unwrap().as_path().to_string_lossy(), + std::fs::canonicalize("../deltalake-test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet").unwrap().as_path().to_string_lossy(), ] - ); - #[cfg(windows)] - assert_eq!( - table.get_file_uris_by_partitions(&filters).unwrap(), - vec![ - format!("{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet", current_dir.as_ref()), - format!("{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet", current_dir.as_ref()) - ] - ); + ); let filters = vec![crate::PartitionFilter { key: "month".to_string(), @@ -449,7 +426,7 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_with_null_partition() { - let table = crate::open_table("./tests/data/delta-0.8.0-null-partition") + let table = crate::open_table("../deltalake-test/tests/data/delta-0.8.0-null-partition") .await .unwrap(); @@ -478,7 +455,7 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_with_special_partition() { - let table = crate::open_table("./tests/data/delta-0.8.0-special-partition") + let table = crate::open_table("../deltalake-test/tests/data/delta-0.8.0-special-partition") .await .unwrap(); @@ -511,7 +488,7 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_partition_with_compare_op() { - let table = crate::open_table("./tests/data/delta-0.8.0-numeric-partition") + let table = crate::open_table("../deltalake-test/tests/data/delta-0.8.0-numeric-partition") .await .unwrap(); @@ -538,11 +515,10 @@ mod tests { ); } - // TODO: enable this for parquet2 #[cfg(feature = "parquet")] #[tokio::test] async fn read_delta_1_2_1_struct_stats_table() { - let table_uri = "./tests/data/delta-1.2.1-only-struct-stats"; + let table_uri = "../deltalake-test/tests/data/delta-1.2.1-only-struct-stats"; let table_from_struct_stats = crate::open_table(table_uri).await.unwrap(); let table_from_json_stats = crate::open_table_with_version(table_uri, 1).await.unwrap(); @@ -574,7 +550,7 @@ mod tests { #[tokio::test] async fn test_table_history() { - let path = "./tests/data/simple_table_with_checkpoint"; + let path = "../deltalake-test/tests/data/simple_table_with_checkpoint"; let mut latest_table = crate::open_table(path).await.unwrap(); let mut table = crate::open_table_with_version(path, 1).await.unwrap(); @@ -596,7 +572,7 @@ mod tests { #[tokio::test] async fn test_poll_table_commits() { - let path = "./tests/data/simple_table_with_checkpoint"; + let path = "../deltalake-test/tests/data/simple_table_with_checkpoint"; let mut table = crate::open_table_with_version(path, 9).await.unwrap(); let peek = table.peek_next_commit(table.version()).await.unwrap(); assert!(matches!(peek, PeekCommit::New(..))); @@ -626,14 +602,14 @@ mod tests { #[tokio::test] async fn test_read_vacuumed_log() { - let path = "./tests/data/checkpoints_vacuumed"; + let path = "../deltalake-test/tests/data/checkpoints_vacuumed"; let table = crate::open_table(path).await.unwrap(); assert_eq!(table.version(), 12); } #[tokio::test] async fn test_read_vacuumed_log_history() { - let path = "./tests/data/checkpoints_vacuumed"; + let path = "../deltalake-test/tests/data/checkpoints_vacuumed"; let mut table = crate::open_table(path).await.unwrap(); // load history for table version with available log file @@ -678,7 +654,7 @@ mod tests { #[tokio::test] async fn read_delta_table_with_cdc() { - let table = crate::open_table("./tests/data/simple_table_with_cdc") + let table = crate::open_table("../deltalake-test/tests/data/simple_table_with_cdc") .await .unwrap(); assert_eq!(table.version(), 2); @@ -692,7 +668,7 @@ mod tests { #[tokio::test()] async fn test_version_zero_table_load() { - let path = "./tests/data/COVID-19_NYT"; + let path = "../deltalake-test/tests/data/COVID-19_NYT"; let mut latest_table: DeltaTable = crate::open_table(path).await.unwrap(); let mut version_0_table = crate::open_table_with_version(path, 0).await.unwrap(); @@ -713,7 +689,7 @@ mod tests { async fn test_fail_fast_on_not_existing_path() { use std::path::Path as FolderPath; - let non_existing_path_str = "./tests/data/folder_doesnt_exist"; + let non_existing_path_str = "../deltalake-test/tests/data/folder_doesnt_exist"; // Check that there is no such path at the beginning let path_doesnt_exist = !FolderPath::new(non_existing_path_str).exists(); diff --git a/crates/deltalake-core/src/logstore/default_logstore.rs b/crates/deltalake-core/src/logstore/default_logstore.rs index 275732fb1a..ed463e9947 100644 --- a/crates/deltalake-core/src/logstore/default_logstore.rs +++ b/crates/deltalake-core/src/logstore/default_logstore.rs @@ -29,6 +29,10 @@ impl DefaultLogStore { #[async_trait::async_trait] impl LogStore for DefaultLogStore { + fn name(&self) -> String { + "DefaultLogStore".into() + } + async fn read_commit_entry(&self, version: i64) -> DeltaResult> { super::read_commit_entry(self.storage.as_ref(), version).await } @@ -54,15 +58,6 @@ impl LogStore for DefaultLogStore { self.storage.clone() } - fn to_uri(&self, location: &Path) -> String { - super::to_uri(&self.config.location, location) - } - - #[cfg(feature = "datafusion")] - fn object_store_url(&self) -> datafusion::execution::object_store::ObjectStoreUrl { - super::object_store_url(&self.config.location) - } - fn config(&self) -> &LogStoreConfig { &self.config } diff --git a/crates/deltalake-core/src/logstore/mod.rs b/crates/deltalake-core/src/logstore/mod.rs index dd13b6bdc5..8bb6b3cd75 100644 --- a/crates/deltalake-core/src/logstore/mod.rs +++ b/crates/deltalake-core/src/logstore/mod.rs @@ -1,4 +1,5 @@ //! Delta log store. +use dashmap::DashMap; use futures::StreamExt; use lazy_static::lazy_static; use regex::Regex; @@ -8,6 +9,7 @@ use serde::{ Deserialize, Serialize, }; use std::io::{BufRead, BufReader, Cursor}; +use std::sync::OnceLock; use std::{cmp::max, collections::HashMap, sync::Arc}; use url::Url; @@ -16,19 +18,71 @@ use crate::{ kernel::Action, operations::transaction::TransactionError, protocol::{get_last_checkpoint, ProtocolError}, - storage::{commit_uri_from_version, config::StorageOptions}, + storage::{commit_uri_from_version, ObjectStoreRef, StorageOptions}, DeltaTableError, }; use bytes::Bytes; -use log::debug; +use log::*; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; #[cfg(feature = "datafusion")] use datafusion::datasource::object_store::ObjectStoreUrl; pub mod default_logstore; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -pub mod s3; + +/// Trait for generating [LogStore] implementations +pub trait LogStoreFactory: Send + Sync { + /// Create a new [LogStore] + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + Ok(default_logstore(store, location, options)) + } +} + +/// Return the [DefaultLogStore] implementation with the provided configuration options +pub fn default_logstore( + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, +) -> Arc { + Arc::new(default_logstore::DefaultLogStore::new( + store, + LogStoreConfig { + location: location.clone(), + options: options.clone(), + }, + )) +} + +#[derive(Clone, Debug, Default)] +struct DefaultLogStoreFactory {} +impl LogStoreFactory for DefaultLogStoreFactory {} + +/// Registry of [LogStoreFactory] instances +pub type FactoryRegistry = Arc>>; + +/// TODO +pub fn logstores() -> FactoryRegistry { + static REGISTRY: OnceLock = OnceLock::new(); + REGISTRY + .get_or_init(|| { + let registry = FactoryRegistry::default(); + registry.insert( + Url::parse("memory://").unwrap(), + Arc::new(DefaultLogStoreFactory::default()), + ); + registry.insert( + Url::parse("file://").unwrap(), + Arc::new(DefaultLogStoreFactory::default()), + ); + registry + }) + .clone() +} /// Sharable reference to [`LogStore`] pub type LogStoreRef = Arc; @@ -37,6 +91,56 @@ lazy_static! { static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); } +/// Return the [LogStoreRef] for the provided [Url] location +/// +/// This will use the built-in process global [crate::storage::ObjectStoreRegistry] by default +/// +/// ```rust +/// # use deltalake_core::logstore::*; +/// # use std::collections::HashMap; +/// # use url::Url; +/// let location = Url::parse("file:///tmp").expect("Failed to make location"); +/// let logstore = logstore_for(location, HashMap::new()).expect("Failed to get a logstore"); +/// ``` +pub fn logstore_for( + location: Url, + options: impl Into + Clone, +) -> DeltaResult { + // turn location into scheme + let scheme = Url::parse(&format!("{}://", location.scheme())) + .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; + + if let Some(entry) = crate::storage::factories().get(&scheme) { + debug!("Found a storage provider for {scheme} ({location})"); + let (store, _prefix) = entry + .value() + .parse_url_opts(&location, &options.clone().into())?; + return logstore_with(store, location, options); + } + Err(DeltaTableError::InvalidTableLocation(location.into())) +} + +/// Return the [LogStoreRef] using the given [ObjectStoreRef] +pub fn logstore_with( + store: ObjectStoreRef, + location: Url, + options: impl Into + Clone, +) -> DeltaResult { + let scheme = Url::parse(&format!("{}://", location.scheme())) + .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; + + if let Some(factory) = logstores().get(&scheme) { + debug!("Found a logstore provider for {scheme}"); + return factory.with_options(store, &location, &options.into()); + } else { + println!("Could not find a logstore for the scheme {scheme}"); + warn!("Could not find a logstore for the scheme {scheme}"); + } + Err(DeltaTableError::InvalidTableLocation( + location.clone().into(), + )) +} + /// Configuration parameters for a log store #[derive(Debug, Clone)] pub struct LogStoreConfig { @@ -58,6 +162,9 @@ pub struct LogStoreConfig { /// become visible immediately. #[async_trait::async_trait] pub trait LogStore: Sync + Send { + /// Return the name of this LogStore implementation + fn name(&self) -> String; + /// Read data for commit entry with the given version. async fn read_commit_entry(&self, version: i64) -> DeltaResult>; @@ -78,7 +185,10 @@ pub trait LogStore: Sync + Send { fn object_store(&self) -> Arc; /// [Path] to Delta log - fn to_uri(&self, location: &Path) -> String; + fn to_uri(&self, location: &Path) -> String { + let root = &self.config().location; + to_uri(root, location) + } /// Get fully qualified uri for table root fn root_uri(&self) -> String { @@ -112,12 +222,56 @@ pub trait LogStore: Sync + Send { /// 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; + fn object_store_url(&self) -> ObjectStoreUrl { + crate::logstore::object_store_url(&self.config().location) + } /// Get configuration representing configured log store. fn config(&self) -> &LogStoreConfig; } +#[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.") +} + +/// TODO +pub fn to_uri(root: &Url, location: &Path) -> String { + match root.scheme() { + "file" => { + #[cfg(windows)] + let uri = format!( + "{}/{}", + root.as_ref().trim_end_matches('/'), + location.as_ref() + ) + .replace("file:///", ""); + #[cfg(unix)] + let uri = format!( + "{}/{}", + root.as_ref().trim_end_matches('/'), + location.as_ref() + ) + .replace("file://", ""); + uri + } + _ => { + if location.as_ref().is_empty() || location.as_ref() == "/" { + root.as_ref().to_string() + } else { + format!("{}/{}", root.as_ref(), location.as_ref()) + } + } + } +} + /// Reads a commit and gets list of actions pub async fn get_actions( version: i64, @@ -199,50 +353,6 @@ lazy_static! { static ref DELTA_LOG_REGEX: Regex = Regex::new(r"(\d{20})\.(json|checkpoint).*$").unwrap(); } -fn to_uri(root: &Url, location: &Path) -> String { - match root.scheme() { - "file" => { - #[cfg(windows)] - let uri = format!( - "{}/{}", - root.as_ref().trim_end_matches('/'), - location.as_ref() - ) - .replace("file:///", ""); - #[cfg(unix)] - let uri = format!( - "{}/{}", - root.as_ref().trim_end_matches('/'), - location.as_ref() - ) - .replace("file://", ""); - uri - } - _ => { - if location.as_ref().is_empty() || location.as_ref() == "/" { - root.as_ref().to_string() - } else { - format!("{}/{}", root.as_ref(), location.as_ref()) - } - } - } -} - -#[cfg(feature = "datafusion")] -fn object_store_url(location: &Url) -> ObjectStoreUrl { - // we are certain, that the URL can be parsed, since - // we make sure when we are parsing the table uri - - 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.") -} - /// Extract version from a file name in the delta log pub fn extract_version_from_filename(name: &str) -> Option { DELTA_LOG_REGEX @@ -250,7 +360,11 @@ pub fn extract_version_from_filename(name: &str) -> Option { .map(|captures| captures.get(1).unwrap().as_str().parse().unwrap()) } -async fn get_latest_version(log_store: &dyn LogStore, current_version: i64) -> DeltaResult { +/// Default implementation for retrieving the latest version +pub async fn get_latest_version( + log_store: &dyn LogStore, + current_version: i64, +) -> DeltaResult { let version_start = match get_last_checkpoint(log_store).await { Ok(last_check_point) => last_check_point.version, Err(ProtocolError::CheckpointNotFound) => { @@ -296,7 +410,10 @@ async fn get_latest_version(log_store: &dyn LogStore, current_version: i64) -> D } /// Read delta log for a specific version -async fn read_commit_entry(storage: &dyn ObjectStore, version: i64) -> DeltaResult> { +pub async fn read_commit_entry( + storage: &dyn ObjectStore, + version: i64, +) -> DeltaResult> { let commit_uri = commit_uri_from_version(version); match storage.get(&commit_uri).await { Ok(res) => Ok(Some(res.bytes().await?)), @@ -305,7 +422,8 @@ async fn read_commit_entry(storage: &dyn ObjectStore, version: i64) -> DeltaResu } } -async fn write_commit_entry( +/// Default implementation for writing a commit entry +pub async fn write_commit_entry( storage: &dyn ObjectStore, version: i64, tmp_commit: &Path, @@ -326,9 +444,29 @@ async fn write_commit_entry( Ok(()) } -#[cfg(feature = "datafusion")] #[cfg(test)] mod tests { + use super::*; + + #[test] + fn logstore_with_invalid_url() { + let location = Url::parse("nonexistent://table").unwrap(); + let store = logstore_for(location, HashMap::default()); + assert!(store.is_err()); + } + + #[test] + fn logstore_with_memory() { + let location = Url::parse("memory://table").unwrap(); + let store = logstore_for(location, HashMap::default()); + assert!(store.is_ok()); + } +} + +#[cfg(feature = "datafusion")] +#[cfg(test)] +mod datafusion_tests { + use super::*; use url::Url; #[tokio::test] @@ -345,8 +483,8 @@ mod tests { let url_2 = Url::parse(location_2).unwrap(); assert_ne!( - super::object_store_url(&url_1).as_str(), - super::object_store_url(&url_2).as_str(), + object_store_url(&url_1).as_str(), + object_store_url(&url_2).as_str(), ); } } diff --git a/crates/deltalake-core/src/operations/convert_to_delta.rs b/crates/deltalake-core/src/operations/convert_to_delta.rs index 97cb08f560..48dc90b2dc 100644 --- a/crates/deltalake-core/src/operations/convert_to_delta.rs +++ b/crates/deltalake-core/src/operations/convert_to_delta.rs @@ -6,7 +6,7 @@ use crate::{ logstore::{LogStore, LogStoreRef}, operations::create::CreateBuilder, protocol::SaveMode, - storage::config::configure_log_store, + table::builder::ensure_table_uri, table::config::DeltaConfigKey, DeltaResult, DeltaTable, DeltaTableError, DeltaTablePartition, ObjectStoreError, NULL_PARTITION_VALUE_DATA_PATH, @@ -55,6 +55,8 @@ enum Error { DeltaTableAlready, #[error("Location must be provided to convert a Parquet table to a Delta table")] MissingLocation, + #[error("The location provided must be a valid URL")] + InvalidLocation(#[from] url::ParseError), } impl From for DeltaTableError { @@ -231,7 +233,10 @@ impl ConvertToDeltaBuilder { let log_store = if let Some(log_store) = self.log_store { log_store } else if let Some(location) = self.location { - configure_log_store(&location, self.storage_options.unwrap_or_default(), None)? + crate::logstore::logstore_for( + ensure_table_uri(location)?, + self.storage_options.unwrap_or_default(), + )? } else { return Err(Error::MissingLocation); }; @@ -389,11 +394,11 @@ impl std::future::IntoFuture for ConvertToDeltaBuilder { #[cfg(test)] mod tests { - use super::{configure_log_store, ConvertToDeltaBuilder, DeltaTable, LogStoreRef, StructField}; + use super::*; use crate::{ kernel::schema::{DataType, PrimitiveType}, open_table, - storage::config::StorageOptions, + storage::StorageOptions, Path, }; use itertools::Itertools; @@ -424,7 +429,9 @@ mod tests { } fn log_store(path: impl Into) -> LogStoreRef { - configure_log_store(&path.into(), StorageOptions::default(), None) + let path: String = path.into(); + let location = ensure_table_uri(path).expect("Failed to get the URI from the path"); + crate::logstore::logstore_for(location, StorageOptions::default()) .expect("Failed to create an object store") } @@ -442,7 +449,9 @@ mod tests { // Copy all files to a temp directory to perform testing. Skip Delta log copy_files(format!("{}/{}", env!("CARGO_MANIFEST_DIR"), path), temp_dir); let builder = if from_path { - ConvertToDeltaBuilder::new().with_location(temp_dir) + ConvertToDeltaBuilder::new().with_location( + ensure_table_uri(temp_dir).expect("Failed to turn temp dir into a URL"), + ) } else { ConvertToDeltaBuilder::new().with_log_store(log_store(temp_dir)) }; @@ -519,7 +528,7 @@ mod tests { // Test Parquet files in object store location #[tokio::test] async fn test_convert_to_delta() { - let path = "tests/data/delta-0.8.0-date"; + let path = "../deltalake-test/tests/data/delta-0.8.0-date"; let table = create_delta_table(path, Vec::new(), false).await; let action = table .get_active_add_actions_by_partitions(&[]) @@ -545,7 +554,7 @@ mod tests { &[], ); - let path = "tests/data/delta-0.8.0-null-partition"; + let path = "../deltalake-test/tests/data/delta-0.8.0-null-partition"; let table = create_delta_table( path, vec![schema_field("k", PrimitiveType::String, true)], @@ -570,7 +579,7 @@ mod tests { ], ); - let path = "tests/data/delta-0.8.0-special-partition"; + let path = "../deltalake-test/tests/data/delta-0.8.0-special-partition"; let table = create_delta_table( path, vec![schema_field("x", PrimitiveType::String, true)], @@ -601,7 +610,7 @@ mod tests { ], ); - let path = "tests/data/delta-0.8.0-partitioned"; + let path = "../deltalake-test/tests/data/delta-0.8.0-partitioned"; let table = create_delta_table( path, vec![ @@ -668,7 +677,7 @@ mod tests { // Test opening the newly created Delta table #[tokio::test] async fn test_open_created_delta_table() { - let path = "tests/data/delta-0.2.0"; + let path = "../deltalake-test/tests/data/delta-0.2.0"; let table = open_created_delta_table(path, Vec::new()).await; assert_delta_table( table, @@ -687,7 +696,7 @@ mod tests { &[], ); - let path = "tests/data/delta-0.8-empty"; + let path = "../deltalake-test/tests/data/delta-0.8-empty"; let table = open_created_delta_table(path, Vec::new()).await; assert_delta_table( table, @@ -701,7 +710,7 @@ mod tests { &[], ); - let path = "tests/data/delta-0.8.0"; + let path = "../deltalake-test/tests/data/delta-0.8.0"; let table = open_created_delta_table(path, Vec::new()).await; assert_delta_table( table, @@ -720,7 +729,7 @@ mod tests { // Test Parquet files in path #[tokio::test] async fn test_convert_to_delta_from_path() { - let path = "tests/data/delta-2.2.0-partitioned-types"; + let path = "../deltalake-test/tests/data/delta-2.2.0-partitioned-types"; let table = create_delta_table( path, vec![ @@ -760,7 +769,7 @@ mod tests { ], ); - let path = "tests/data/delta-0.8.0-numeric-partition"; + let path = "../deltalake-test/tests/data/delta-0.8.0-numeric-partition"; let table = create_delta_table( path, vec![ @@ -819,7 +828,7 @@ mod tests { #[tokio::test] async fn test_partition_column_not_exist() { let _table = ConvertToDeltaBuilder::new() - .with_location("tests/data/delta-0.8.0-null-partition") + .with_location("../deltalake-test/tests/data/delta-0.8.0-null-partition") .with_partition_schema(vec![schema_field("foo", PrimitiveType::String, true)]) .await .expect_err( @@ -830,7 +839,7 @@ mod tests { #[tokio::test] async fn test_missing_partition_schema() { let _table = ConvertToDeltaBuilder::new() - .with_location("tests/data/delta-0.8.0-numeric-partition") + .with_location("../deltalake-test/tests/data/delta-0.8.0-numeric-partition") .await .expect_err("The schema of a partition column is not provided by user. Should error"); } @@ -838,7 +847,7 @@ mod tests { #[tokio::test] async fn test_delta_table_already() { let _table = ConvertToDeltaBuilder::new() - .with_location("tests/data/delta-0.2.0") + .with_location("../deltalake-test/tests/data/delta-0.2.0") .await .expect_err("The given location is already a delta table location. Should error"); } diff --git a/crates/deltalake-core/src/operations/load.rs b/crates/deltalake-core/src/operations/load.rs index 610f86dee6..0189381922 100644 --- a/crates/deltalake-core/src/operations/load.rs +++ b/crates/deltalake-core/src/operations/load.rs @@ -88,7 +88,7 @@ mod tests { #[tokio::test] async fn test_load_local() -> TestResult { - let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + let table = DeltaTableBuilder::from_uri("../deltalake-test/tests/data/delta-0.8.0") .load() .await .unwrap(); diff --git a/crates/deltalake-core/src/operations/vacuum.rs b/crates/deltalake-core/src/operations/vacuum.rs index 7b321400e6..03d9cffed1 100644 --- a/crates/deltalake-core/src/operations/vacuum.rs +++ b/crates/deltalake-core/src/operations/vacuum.rs @@ -404,7 +404,9 @@ mod tests { #[tokio::test] async fn vacuum_delta_8_0_table() { - let table = open_table("./tests/data/delta-0.8.0").await.unwrap(); + let table = open_table("../deltalake-test/tests/data/delta-0.8.0") + .await + .unwrap(); let result = VacuumBuilder::new(table.log_store, table.state.clone()) .with_retention_period(Duration::hours(1)) @@ -413,7 +415,9 @@ mod tests { assert!(result.is_err()); - let table = open_table("./tests/data/delta-0.8.0").await.unwrap(); + let table = open_table("../deltalake-test/tests/data/delta-0.8.0") + .await + .unwrap(); let (table, result) = VacuumBuilder::new(table.log_store, table.state) .with_retention_period(Duration::hours(0)) .with_dry_run(true) diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index 311f6dac7e..53b2f471d1 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -4,8 +4,6 @@ #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod checkpoints; -#[cfg(feature = "parquet2")] -pub mod parquet2_read; #[cfg(feature = "parquet")] mod parquet_read; mod time_utils; @@ -60,14 +58,10 @@ pub enum ProtocolError { #[error("Generic action error: {0}")] Generic(String), - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] /// Error returned when parsing checkpoint parquet using the parquet crate. #[error("Failed to parse parquet checkpoint: {source}")] ParquetParseError { - /// Parquet error details returned when parsing the checkpoint parquet - #[cfg(feature = "parquet2")] - #[from] - source: parquet2::error::Error, /// Parquet error details returned when parsing the checkpoint parquet #[cfg(feature = "parquet")] #[from] @@ -235,16 +229,10 @@ pub struct StatsParsed { /// Contains a value smaller than all values present in the file for all columns. #[cfg(feature = "parquet")] pub min_values: HashMap, - /// Contains a value smaller than all values present in the file for all columns. - #[cfg(feature = "parquet2")] - pub min_values: HashMap, /// Contains a value larger than all values present in the file for all columns. #[cfg(feature = "parquet")] /// Contains a value larger than all values present in the file for all columns. pub max_values: HashMap, - #[cfg(feature = "parquet2")] - /// Contains a value larger than all values present in the file for all columns. - pub max_values: HashMap, /// The number of null values for all columns. pub null_count: HashMap, } @@ -272,7 +260,7 @@ impl Eq for Add {} impl Add { /// Get whatever stats are available. Uses (parquet struct) parsed_stats if present falling back to json stats. - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] pub fn get_stats(&self) -> Result, serde_json::error::Error> { match self.get_stats_parsed() { Ok(Some(stats)) => Ok(Some(stats)), @@ -288,7 +276,7 @@ impl Add { } /// Get whatever stats are available. - #[cfg(not(any(feature = "parquet", feature = "parquet2")))] + #[cfg(not(any(feature = "parquet")))] pub fn get_stats(&self) -> Result, serde_json::error::Error> { self.get_json_stats() } @@ -912,7 +900,7 @@ mod tests { #[tokio::test] async fn test_with_partitions() { // test table with partitions - let path = "./tests/data/delta-0.8.0-null-partition"; + let path = "../deltalake-test/tests/data/delta-0.8.0-null-partition"; let table = crate::open_table(path).await.unwrap(); let actions = table.get_state().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); @@ -951,7 +939,7 @@ mod tests { #[tokio::test] async fn test_with_deletion_vector() { // test table with partitions - let path = "./tests/data/table_with_deletion_logs"; + let path = "../deltalake-test/tests/data/table_with_deletion_logs"; let table = crate::open_table(path).await.unwrap(); let actions = table.get_state().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); @@ -1057,7 +1045,7 @@ mod tests { #[tokio::test] async fn test_without_partitions() { // test table without partitions - let path = "./tests/data/simple_table"; + let path = "../deltalake-test/tests/data/simple_table"; let table = crate::open_table(path).await.unwrap(); let actions = table.get_state().add_actions_table(true).unwrap(); @@ -1115,7 +1103,7 @@ mod tests { #[tokio::test] async fn test_with_column_mapping() { // test table with column mapping and partitions - let path = "./tests/data/table_with_column_mapping"; + let path = "../deltalake-test/tests/data/table_with_column_mapping"; let table = crate::open_table(path).await.unwrap(); let actions = table.get_state().add_actions_table(true).unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -1189,7 +1177,7 @@ mod tests { #[tokio::test] async fn test_with_stats() { // test table with stats - let path = "./tests/data/delta-0.8.0"; + let path = "../deltalake-test/tests/data/delta-0.8.0"; let table = crate::open_table(path).await.unwrap(); let actions = table.get_state().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); @@ -1233,7 +1221,7 @@ mod tests { #[tokio::test] async fn test_only_struct_stats() { // test table with no json stats - let path = "./tests/data/delta-1.2.1-only-struct-stats"; + let path = "../deltalake-test/tests/data/delta-1.2.1-only-struct-stats"; let mut table = crate::open_table(path).await.unwrap(); table.load_version(1).await.unwrap(); diff --git a/crates/deltalake-core/src/protocol/parquet2_read/boolean.rs b/crates/deltalake-core/src/protocol/parquet2_read/boolean.rs deleted file mode 100644 index e68971be42..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/boolean.rs +++ /dev/null @@ -1,76 +0,0 @@ -use parquet2::encoding::hybrid_rle::BitmapIter; -use parquet2::metadata::ColumnDescriptor; -use parquet2::page::DataPage; - -use super::validity::ValidityRowIndexIter; -use super::{split_page, ActionVariant, ParseError}; -use crate::kernel::Action; - -/// Parquet dictionary primitive value reader -pub struct SomeBooleanValueIter<'a> { - valid_row_idx_iter: ValidityRowIndexIter<'a>, - value_iter: BitmapIter<'a>, -} - -impl<'a> SomeBooleanValueIter<'a> { - /// Create parquet primitive value reader - pub fn try_new( - page: &'a DataPage, - descriptor: &'a ColumnDescriptor, - ) -> Result { - let (max_def_level, validity_iter, values_buffer) = split_page(page, descriptor)?; - - let valid_row_idx_iter = ValidityRowIndexIter::new(max_def_level, validity_iter); - let value_len_upper_bound = values_buffer.len() * 8; - let value_iter = BitmapIter::new(values_buffer, 0, value_len_upper_bound); - Ok(Self { - valid_row_idx_iter, - value_iter, - }) - } -} - -impl<'a> Iterator for SomeBooleanValueIter<'a> { - type Item = Result<(usize, bool), ParseError>; - - fn next(&mut self) -> Option { - self.valid_row_idx_iter.next().map(|idx_result| { - idx_result.map(|idx| { - let value = self.value_iter.next().unwrap(); - (idx, value) - }) - }) - } -} - -#[inline] -pub fn for_each_boolean_field_value( - actions: &mut Vec>, - page: &DataPage, - descriptor: &ColumnDescriptor, - set_fn: SetFn, -) -> Result<(), ParseError> -where - ActType: ActionVariant, - SetFn: Fn(&mut ActType, bool), -{ - #[cfg(debug_assertions)] - { - use parquet2::schema::types::PhysicalType; - if page.descriptor.primitive_type.physical_type != PhysicalType::Boolean { - return Err(ParseError::InvalidAction(format!( - "expect physical parquet type boolean, got {:?}", - page.descriptor.primitive_type, - ))); - } - } - - let some_value_iter = SomeBooleanValueIter::try_new(page, descriptor)?; - for entry in some_value_iter { - let (idx, value) = entry?; - let a = actions[idx].get_or_insert_with(ActType::default_action); - set_fn(ActType::try_mut_from_action(a)?, value); - } - - Ok(()) -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/dictionary/binary.rs b/crates/deltalake-core/src/protocol/parquet2_read/dictionary/binary.rs deleted file mode 100644 index ff7db3c524..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/dictionary/binary.rs +++ /dev/null @@ -1,48 +0,0 @@ -use parquet2::encoding::get_length; -use parquet2::error::Error; - -#[derive(Debug)] -pub struct BinaryPageDict<'a> { - values: Vec<&'a [u8]>, -} - -impl<'a> BinaryPageDict<'a> { - pub fn new(values: Vec<&'a [u8]>) -> Self { - Self { values } - } - - #[inline] - pub fn value(&self, index: usize) -> Result<&[u8], Error> { - self.values - .get(index) - .map(|v| *v) - .ok_or_else(|| Error::OutOfSpec("invalid index".to_string())) - } -} - -fn read_plain<'a>(bytes: &'a [u8], length: usize) -> Result, Error> { - let mut bytes = bytes; - let mut values = Vec::new(); - - for _ in 0..length { - let slot_length = get_length(bytes).unwrap(); - bytes = &bytes[4..]; - - if slot_length > bytes.len() { - return Err(Error::OutOfSpec( - "The string on a dictionary page has a length that is out of bounds".to_string(), - )); - } - let (result, remaining) = bytes.split_at(slot_length); - - values.push(result); - bytes = remaining; - } - - Ok(values) -} - -pub fn read<'a>(buf: &'a [u8], num_values: usize) -> Result, Error> { - let values = read_plain(buf, num_values)?; - Ok(BinaryPageDict::new(values)) -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/dictionary/mod.rs b/crates/deltalake-core/src/protocol/parquet2_read/dictionary/mod.rs deleted file mode 100644 index 16e1f3f7b1..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/dictionary/mod.rs +++ /dev/null @@ -1,2 +0,0 @@ -pub(crate) mod binary; -pub(crate) mod primitive; diff --git a/crates/deltalake-core/src/protocol/parquet2_read/dictionary/primitive.rs b/crates/deltalake-core/src/protocol/parquet2_read/dictionary/primitive.rs deleted file mode 100644 index 89b557c439..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/dictionary/primitive.rs +++ /dev/null @@ -1,19 +0,0 @@ -use parquet2::error::{Error, Result}; -use parquet2::types::{decode, NativeType}; - -pub fn read(buf: &[u8], num_values: usize) -> Result> { - let size_of = std::mem::size_of::(); - - let typed_size = num_values.wrapping_mul(size_of); - - let values = buf.get(..typed_size).ok_or_else(|| { - Error::OutOfSpec( - "The number of values declared in the dict page does not match the length of the page" - .to_string(), - ) - })?; - - let values = values.chunks_exact(size_of).map(decode::).collect(); - - Ok(values) -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/map.rs b/crates/deltalake-core/src/protocol/parquet2_read/map.rs deleted file mode 100644 index df4dc94ab7..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/map.rs +++ /dev/null @@ -1,111 +0,0 @@ -use parquet2::metadata::ColumnDescriptor; -use parquet2::page::{DataPage, DictPage}; - -use super::string::for_each_repeated_string_field_value_with_idx; -use super::{ActionVariant, ParseError}; -use crate::kernel::Action; - -#[derive(Default)] -pub struct MapState { - keys: Option)>>, - values: Option)>>, -} - -pub fn for_each_map_field_value( - field: &[String], - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - state: &mut MapState, - set_map_fn: SetMapFn, -) -> Result<(), ParseError> -where - ActType: ActionVariant, - SetMapFn: Fn(&mut ActType, (Vec, Vec>)), -{ - debug_assert!(field[0] == "key_value"); - #[cfg(debug_assertions)] - { - use parquet2::schema::types::PhysicalType; - if page.descriptor.primitive_type.physical_type != PhysicalType::ByteArray { - return Err(ParseError::InvalidAction(format!( - "expect parquet utf8 type for map key/value, got primitive type: {:?}", - page.descriptor.primitive_type, - ))); - } - } - - match field[1].as_str() { - "key" => { - let mut keys = vec![]; - for_each_repeated_string_field_value_with_idx( - page, - dict, - descriptor, - |result: Result<(usize, Vec), ParseError>| -> Result<(), ParseError> { - let (row_idx, strings) = result?; - keys.push((row_idx, strings)); - Ok(()) - }, - )?; - state.keys = Some(keys); - } - "value" => { - let mut values = vec![]; - for_each_repeated_string_field_value_with_idx( - page, - dict, - descriptor, - |result: Result<(usize, Vec), ParseError>| -> Result<(), ParseError> { - let (row_idx, strings) = result?; - values.push((row_idx, strings)); - Ok(()) - }, - )?; - state.values = Some(values); - } - _ => { - return Err(ParseError::InvalidAction(format!( - "Unexpected map key: {:?}", - field, - ))); - } - } - - if state.keys.is_some() && state.values.is_some() { - let keys = state.keys.take().unwrap(); - let values = state.values.take().unwrap(); - - let mut values_iter = values.into_iter().peekable(); - - keys.into_iter() - .try_for_each(|(key_row_idx, keys)| -> Result<(), ParseError> { - let (row_idx, (keys, vals)) = match values_iter.peek() { - Some((val_row_idx, _)) if *val_row_idx == key_row_idx => { - let (_, vals) = values_iter.next().unwrap(); - ( - key_row_idx, - ( - keys, - vals.into_iter() - .map(|val| if val == "" { None } else { Some(val) }) - .collect(), - ), - ) - } - _ => { - let vals = std::iter::repeat(None).take(keys.len()).collect(); - (key_row_idx, (keys, vals)) - } - }; - - let a = actions[row_idx].get_or_insert_with(ActType::default_action); - set_map_fn(ActType::try_mut_from_action(a)?, (keys, vals)); - - Ok(()) - })?; - } - - Ok(()) -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/mod.rs b/crates/deltalake-core/src/protocol/parquet2_read/mod.rs deleted file mode 100644 index d07ae88ae0..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/mod.rs +++ /dev/null @@ -1,898 +0,0 @@ -//! Parquet deserialization for Action enum - -use std::collections::HashMap; - -use log::warn; -use parquet2::encoding::hybrid_rle; -use parquet2::metadata::ColumnDescriptor; -use parquet2::page::{DataPage, DictPage, Page}; -use parquet2::read::decompress; -use parquet2::read::get_page_iterator; -use parquet2::read::levels::get_bit_width; - -use super::ProtocolError; -use crate::kernel::{ - Action, Add, CommitInfo, Metadata, Protocol, ReaderFeatures, Remove, Txn, WriterFeatures, -}; -use boolean::for_each_boolean_field_value; -use map::for_each_map_field_value; -use primitive::for_each_primitive_field_value; -use string::{for_each_repeated_string_field_value, for_each_string_field_value}; - -mod boolean; -mod dictionary; -mod map; -mod primitive; -mod stats; -mod string; -mod validity; - -/// Parquet deserialization error -#[derive(thiserror::Error, Debug)] -pub enum ParseError { - /// Generic parsing error - #[error("{0}")] - Generic(String), - /// Invalid action found during parsing - #[error("Invalid action: {0}")] - InvalidAction(String), - /// Error returned when parsing checkpoint parquet using parquet2 crate. - #[error("Failed to parse parquet: {}", .source)] - Parquet { - /// Parquet error details returned when parsing the checkpoint parquet - #[from] - source: parquet2::error::Error, - }, -} - -impl From for ProtocolError { - fn from(value: ParseError) -> Self { - match value { - ParseError::Generic(msg) => Self::Generic(msg), - ParseError::InvalidAction(msg) => Self::InvalidRow(msg), - ParseError::Parquet { source } => Self::ParquetParseError { source }, - } - } -} - -#[derive(Default)] -struct DeserState { - add_partition_values: map::MapState, - add_tags: map::MapState, - remove_partition_values: map::MapState, - remove_tags: map::MapState, - metadata_fromat_options: map::MapState, - metadata_configuration: map::MapState, -} - -fn hashmap_from_kvpairs( - keys: impl IntoIterator, - values: impl IntoIterator, -) -> HashMap -where - Key: std::hash::Hash + std::cmp::Eq, -{ - keys.into_iter().zip(values.into_iter()).collect() -} - -fn split_page<'a>( - page: &'a DataPage, - descriptor: &'a ColumnDescriptor, -) -> Result<(i16, hybrid_rle::HybridRleDecoder<'a>, &'a [u8]), ParseError> { - let (_rep_levels, def_levels_buf, values_buf) = parquet2::page::split_buffer(page)?; - - let max_def_level = descriptor.descriptor.max_def_level; - let def_bit_width = get_bit_width(max_def_level); - let validity_iter = - hybrid_rle::HybridRleDecoder::try_new(def_levels_buf, def_bit_width, page.num_values())?; - - Ok((max_def_level, validity_iter, values_buf)) -} - -fn split_page_nested<'a>( - page: &'a DataPage, - descriptor: &'a ColumnDescriptor, -) -> Result< - ( - i16, - hybrid_rle::HybridRleDecoder<'a>, - i16, - hybrid_rle::HybridRleDecoder<'a>, - &'a [u8], - ), - ParseError, -> { - let (rep_levels, def_levels_buf, values_buf) = parquet2::page::split_buffer(page)?; - - let max_rep_level = descriptor.descriptor.max_rep_level; - let rep_bit_width = get_bit_width(max_rep_level); - let rep_iter = - hybrid_rle::HybridRleDecoder::try_new(rep_levels, rep_bit_width, page.num_values())?; - - let max_def_level = descriptor.descriptor.max_def_level; - let def_bit_width = get_bit_width(max_def_level); - let validity_iter = - hybrid_rle::HybridRleDecoder::try_new(def_levels_buf, def_bit_width, page.num_values())?; - - Ok(( - max_rep_level, - rep_iter, - max_def_level, - validity_iter, - values_buf, - )) -} - -/// Trait for conversion between concrete action struct and Action enum variant -pub trait ActionVariant { - /// Conrete action struct type - type Variant; - - /// Return action struct wrapped in corresponding Action enum variant - fn default_action() -> Action; - - /// Extract action struct from Action enum - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError>; -} - -impl ActionVariant for Add { - type Variant = Add; - - fn default_action() -> Action { - Action::Add(Self::default()) - } - - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { - match a { - Action::Add(v) => Ok(v), - _ => Err(ParseError::Generic(format!( - "expect Add action, got: {:?}", - a - ))), - } - } -} - -impl ActionVariant for Remove { - type Variant = Remove; - - fn default_action() -> Action { - Action::Remove(Self { - data_change: true, - extended_file_metadata: Some(false), - ..Default::default() - }) - } - - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { - match a { - Action::Remove(v) => Ok(v), - _ => Err(ParseError::Generic(format!( - "expect remove action, got: {:?}", - a - ))), - } - } -} - -impl ActionVariant for Metadata { - type Variant = Metadata; - - fn default_action() -> Action { - Action::Metadata(Self::default()) - } - - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { - match a { - Action::Metadata(v) => Ok(v), - _ => Err(ParseError::Generic(format!( - "expect metadata action, got: {:?}", - a - ))), - } - } -} - -impl ActionVariant for Txn { - type Variant = Txn; - - fn default_action() -> Action { - Action::Txn(Self::default()) - } - - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { - match a { - Action::Txn(v) => Ok(v), - _ => Err(ParseError::Generic(format!( - "expect txn action, got: {:?}", - a - ))), - } - } -} - -impl ActionVariant for Protocol { - type Variant = Protocol; - - fn default_action() -> Action { - Action::Protocol(Self::default()) - } - - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { - match a { - Action::Protocol(v) => Ok(v), - _ => Err(ParseError::Generic(format!( - "expect protocol action, got: {:?}", - a - ))), - } - } -} - -impl ActionVariant for CommitInfo { - type Variant = CommitInfo; - - fn default_action() -> Action { - Action::CommitInfo(CommitInfo::default()) - } - - fn try_mut_from_action(a: &mut Action) -> Result<&mut Self, ParseError> { - match a { - Action::CommitInfo(v) => Ok(v), - _ => Err(ParseError::Generic(format!( - "expect commitInfo action, got: {:?}", - a - ))), - } - } -} - -fn deserialize_txn_column_page( - field: &[String], - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - _state: &mut DeserState, -) -> Result<(), ParseError> { - let f = field[0].as_ref(); - match f { - "version" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Txn, v: i64| action.version = v, - )?; - } - "appId" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Txn, v: String| action.app_id = v, - )?; - } - "lastUpdated" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Txn, v: i64| action.last_updated = Some(v), - )?; - } - _ => { - return Err(ParseError::InvalidAction(format!( - "Unexpected field `{}` in txn", - f - ))) - } - } - Ok(()) -} - -fn deserialize_add_column_page( - field: &[String], - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - state: &mut DeserState, -) -> Result<(), ParseError> { - let f = field[0].as_ref(); - match f { - "path" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Add, v: String| action.path = v, - )?; - } - "size" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Add, v: i64| action.size = v, - )?; - } - "partitionValues" => { - for_each_map_field_value( - &field[1..], - actions, - page, - dict, - descriptor, - &mut state.add_partition_values, - |action: &mut Add, v: (Vec, Vec>)| { - action.partition_values = hashmap_from_kvpairs(v.0, v.1); - }, - )?; - } - // FIXME support partitionValueParsed - "dataChange" => { - for_each_boolean_field_value( - actions, - page, - descriptor, - |action: &mut Add, v: bool| action.data_change = v, - )?; - } - "tags" => { - for_each_map_field_value( - &field[1..], - actions, - page, - dict, - descriptor, - &mut state.add_tags, - |action: &mut Add, v: (Vec, Vec>)| { - action.tags = Some(hashmap_from_kvpairs(v.0, v.1)); - }, - )?; - } - // FIXME: support statsParsed - "stats" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Add, v: String| action.stats = Some(v), - )?; - } - "modificationTime" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Add, v: i64| action.modification_time = v, - )?; - } - _ => { - warn!("Unexpected field `{}` in add", f); - } - } - Ok(()) -} - -fn deserialize_remove_column_page( - field: &[String], - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - state: &mut DeserState, -) -> Result<(), ParseError> { - let f = field[0].as_ref(); - match f { - "path" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Remove, v: String| action.path = v, - )?; - } - "deletionTimestamp" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Remove, v: i64| action.deletion_timestamp = Some(v), - )?; - } - "size" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Remove, v: i64| action.size = Some(v), - )?; - } - // FIXME support partitionValueParsed - "partitionValues" => { - for_each_map_field_value( - &field[1..], - actions, - page, - dict, - descriptor, - &mut state.remove_partition_values, - |action: &mut Remove, v: (Vec, Vec>)| { - action.partition_values = Some(hashmap_from_kvpairs(v.0, v.1)); - }, - )?; - } - "dataChange" => { - for_each_boolean_field_value( - actions, - page, - descriptor, - |action: &mut Remove, v: bool| action.data_change = v, - )?; - } - "extendedFileMetadata" => { - for_each_boolean_field_value( - actions, - page, - descriptor, - |action: &mut Remove, v: bool| action.extended_file_metadata = Some(v), - )?; - } - "tags" => { - for_each_map_field_value( - &field[1..], - actions, - page, - dict, - descriptor, - &mut state.remove_tags, - |action: &mut Remove, v: (Vec, Vec>)| { - action.tags = Some(hashmap_from_kvpairs(v.0, v.1)); - }, - )?; - } - _ => { - warn!("Unexpected field `{}` in remove", f); - } - } - Ok(()) -} - -fn deserialize_metadata_column_page( - field: &[String], - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - state: &mut DeserState, -) -> Result<(), ParseError> { - let f = field[0].as_ref(); - match f { - "id" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: String| action.id = v, - )?; - } - "name" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: String| action.name = Some(v), - )?; - } - "description" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: String| action.description = Some(v), - )?; - } - "format" => { - let sub_f = field[1].as_ref(); - match sub_f { - "provider" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: String| action.format.provider = v, - )?; - } - "options" => { - for_each_map_field_value( - &field[2..], - actions, - page, - dict, - descriptor, - &mut state.metadata_fromat_options, - |action: &mut Metadata, v: (Vec, Vec>)| { - action.format.options = hashmap_from_kvpairs(v.0, v.1); - }, - )?; - } - _ => { - return Err(ParseError::InvalidAction(format!( - "Unexpected field `{}` in metaData.format", - sub_f, - ))) - } - } - } - "schemaString" => { - for_each_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: String| action.schema_string = v, - )?; - } - "partitionColumns" => { - for_each_repeated_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: Vec| action.partition_columns = v, - )?; - } - "createdTime" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Metadata, v: i64| action.created_time = Some(v), - )?; - } - "configuration" => { - for_each_map_field_value( - &field[1..], - actions, - page, - dict, - descriptor, - &mut state.metadata_configuration, - |action: &mut Metadata, v: (Vec, Vec>)| { - action.configuration = hashmap_from_kvpairs(v.0, v.1); - }, - )?; - } - _ => { - warn!("Unexpected field `{}` in metaData", f); - } - } - Ok(()) -} - -fn deserialize_protocol_column_page( - field: &[String], - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - _state: &mut DeserState, -) -> Result<(), ParseError> { - let f = field[0].as_ref(); - match f { - "minReaderVersion" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Protocol, v: i32| action.min_reader_version = v, - )?; - } - "minWriterVersion" => { - for_each_primitive_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Protocol, v: i32| action.min_writer_version = v, - )?; - } - "readerFeatures" => { - for_each_repeated_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Protocol, v: Vec| { - action.reader_features = - Some(v.into_iter().map(ReaderFeatures::from).collect()); - }, - )?; - } - "writerFeatures" => { - for_each_repeated_string_field_value( - actions, - page, - dict, - descriptor, - |action: &mut Protocol, v: Vec| { - action.writer_features = - Some(v.into_iter().map(WriterFeatures::from).collect()); - }, - )?; - } - _ => { - warn!("Unexpected field `{}` in protocol", f); - } - } - Ok(()) -} - -fn deserialize_commit_info_column_page( - _obj_keys: &[String], - _actions: &mut Vec>, - _page: &DataPage, - _dict: &Option, - _descriptor: &ColumnDescriptor, - _state: &mut DeserState, -) -> Result<(), ParseError> { - // parquet snapshots shouldn't contain commit info - Ok(()) -} - -fn deserialize_cdc_column_page( - _field: &[String], - _actions: &mut Vec>, - _page: &DataPage, - _dict: &Option, - _descriptor: &ColumnDescriptor, - _state: &mut DeserState, -) -> Result<(), ParseError> { - // FIXME: support cdc action - Ok(()) -} - -// TODO: find a proper max size to avoid OOM -// see: https://github.com/jorgecarleitao/parquet2/pull/172 -const MAX_PARQUET_HEADER_SIZE: usize = usize::MAX; - -/// Return a vector of action from a given parquet row group -pub fn actions_from_row_group( - row_group: parquet2::metadata::RowGroupMetaData, - reader: &mut R, -) -> Result, ProtocolError> { - let row_count = row_group.num_rows(); - // TODO: reuse actions buffer - let mut actions: Vec> = vec![None; row_count as usize]; - let mut state = DeserState::default(); - - for column_metadata in row_group.columns() { - let column_desc = column_metadata.descriptor(); - let schema_path = &column_desc.path_in_schema; - - let deserialize_column_page = match schema_path[0].as_ref() { - "txn" => deserialize_txn_column_page, - "add" => deserialize_add_column_page, - "remove" => deserialize_remove_column_page, - "metaData" => deserialize_metadata_column_page, - "protocol" => deserialize_protocol_column_page, - "commitInfo" => deserialize_commit_info_column_page, - "cdc" => deserialize_cdc_column_page, - _ => { - return Err(ParseError::InvalidAction(format!( - "unexpected action: {}", - &schema_path[0] - )) - .into()); - } - }; - let field = &schema_path[1..]; - - let buffer = Vec::new(); - let pages = get_page_iterator( - column_metadata, - &mut *reader, - None, - buffer, - MAX_PARQUET_HEADER_SIZE, - )?; - - let mut decompress_buffer = vec![]; - let mut dict = None; - for maybe_page in pages { - // TODO: leverage null count and skip page if possible - let page = maybe_page?; - let page = decompress(page, &mut decompress_buffer)?; - - match page { - Page::Dict(page) => { - // the first page may be a dictionary page, which needs to be deserialized - // depending on your target in-memory format - dict = Some(page); - } - Page::Data(page) => { - deserialize_column_page( - field, - &mut actions, - // TODO: pass by value? - &page, - &dict, - column_desc, - &mut state, - )?; - } - } - } - } - - Ok(actions.into_iter().map(|a| a.unwrap()).collect()) -} - -#[cfg(test)] -mod tests { - use super::*; - use std::collections::HashMap; - use std::fs::File; - - #[test] - fn test_add_action_without_partition_values_and_stats() { - use parquet2::read::read_metadata; - - let path = "./tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet"; - let mut reader = File::open(path).unwrap(); - let meta_data = read_metadata(&mut reader).unwrap(); - - for row_group in meta_data.row_groups { - let actions = actions_from_row_group(row_group, &mut reader).unwrap(); - match &actions[0] { - Action::Protocol(protocol) => { - assert_eq!(protocol.min_reader_version, 1,); - assert_eq!(protocol.min_writer_version, 2,); - } - _ => panic!("expect protocol action"), - } - match &actions[1] { - Action::Metadata(meta_data) => { - assert_eq!(meta_data.id, "22ef18ba-191c-4c36-a606-3dad5cdf3830"); - assert_eq!(meta_data.name, None); - assert_eq!(meta_data.description, None); - assert_eq!( - meta_data.format, - crate::kernel::Format::new("parquet".to_string(), None), - ); - assert_eq!(meta_data.schema_string, "{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"); - assert_eq!(meta_data.partition_columns.len(), 0); - assert_eq!(meta_data.created_time, Some(1564524294376)); - assert_eq!(meta_data.configuration, HashMap::new()); - } - _ => panic!("expect txn action, got: {:?}", &actions[1]), - } - - match &actions[2] { - Action::Txn(txn) => { - assert_eq!(txn.app_id, "e4a20b59-dd0e-4c50-b074-e8ae4786df30"); - assert_eq!(txn.version, 0); - assert_eq!(txn.last_updated, Some(1564524299648)); - } - _ => panic!("expect txn action, got: {:?}", &actions[1]), - } - match &actions[3] { - Action::Remove(remove) => { - assert_eq!( - remove.path, - "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet" - ); - assert_eq!(remove.deletion_timestamp, Some(1564524298213)); - assert_eq!(remove.data_change, false); - assert_eq!(remove.extended_file_metadata, Some(false)); - assert_eq!(remove.partition_values, None); - assert_eq!(remove.size, None); - assert_eq!(remove.tags, None); - } - _ => panic!("expect remove action, got: {:?}", &actions[2]), - } - match &actions[9] { - Action::Add(add_action) => { - assert_eq!( - add_action.path, - "part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet" - ); - assert_eq!(add_action.size, 400); - assert_eq!(add_action.modification_time, 1564524297000); - assert_eq!(add_action.partition_values.len(), 0); - assert_eq!(add_action.data_change, false); - assert_eq!(add_action.stats, None); - assert_eq!(add_action.tags, None); - } - _ => panic!("expect add action, got: {:?}", &actions[9]), - } - } - } - - #[test] - fn test_add_action_with_partition_values() { - use parquet2::read::read_metadata; - - let path = "./tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet"; - let mut reader = File::open(path).unwrap(); - let metadata = read_metadata(&mut reader).unwrap(); - - for row_group in metadata.row_groups { - let actions = actions_from_row_group(row_group, &mut reader).unwrap(); - match &actions[0] { - Action::Protocol(protocol) => { - assert_eq!(protocol.min_reader_version, 1,); - assert_eq!(protocol.min_writer_version, 2,); - } - _ => panic!("expect protocol action"), - } - match &actions[1] { - Action::Metadata(meta_data) => { - assert_eq!(meta_data.id, "94ba8468-c676-4468-b326-adde3ab9dcd2"); - assert_eq!(meta_data.name, None); - assert_eq!(meta_data.description, None); - assert_eq!( - meta_data.format, - crate::kernel::Format::new("parquet".to_string(), None), - ); - assert_eq!( - meta_data.schema_string, - r#"{"type":"struct","fields":[{"name":"id","type":"integer","nullable":true,"metadata":{}},{"name":"color","type":"string","nullable":true,"metadata":{}}]}"# - ); - assert_eq!(meta_data.partition_columns, vec!["color"]); - assert_eq!(meta_data.created_time, Some(1661662807027)); - assert_eq!(meta_data.configuration, HashMap::new()); - } - _ => panic!("expect txn action, got: {:?}", &actions[1]), - } - - match &actions[2] { - Action::Add(add_action) => { - assert_eq!(add_action.path, "f62d8868-d952-4f9d-8bb2-fd4e011ebf36"); - assert_eq!(add_action.size, 100); - assert_eq!(add_action.modification_time, 1661662807080); - assert_eq!(add_action.partition_values.len(), 1); - assert_eq!( - add_action.partition_values.get("color").unwrap(), - &Some("red".to_string()) - ); - assert_eq!(add_action.data_change, false); - assert_eq!(add_action.stats, None); - assert_eq!(add_action.tags, None); - } - _ => panic!("expect add action, got: {:?}", &actions[9]), - } - match &actions[3] { - Action::Add(add_action) => { - assert_eq!(add_action.path, "8ac7d8e1-daab-48ef-9d05-ec22fb4b0d2f"); - assert_eq!(add_action.size, 100); - assert_eq!(add_action.modification_time, 1661662807097); - assert_eq!(add_action.partition_values.len(), 1); - assert_eq!(add_action.partition_values.get("color").unwrap(), &None); - assert_eq!(add_action.data_change, false); - assert_eq!(add_action.stats, None); - assert_eq!(add_action.tags, None); - } - _ => panic!("expect add action, got: {:?}", &actions[9]), - } - } - } -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/primitive.rs b/crates/deltalake-core/src/protocol/parquet2_read/primitive.rs deleted file mode 100644 index 16cb850f05..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/primitive.rs +++ /dev/null @@ -1,185 +0,0 @@ -//! Parquet primitive type deserialization for Action enum - -use std::convert::TryInto; - -use parquet2::encoding::hybrid_rle; -use parquet2::encoding::Encoding; -use parquet2::metadata::ColumnDescriptor; -use parquet2::page::DataPage; -use parquet2::page::DictPage; -use parquet2::types::NativeType; - -use super::dictionary; -use super::validity::ValidityRowIndexIter; -use super::{split_page, ActionVariant, ParseError}; -use crate::kernel::Action; - -struct ExactChunksIter<'a, T: NativeType> { - chunks: std::slice::ChunksExact<'a, u8>, - phantom: std::marker::PhantomData, -} - -impl<'a, T: NativeType> ExactChunksIter<'a, T> { - #[inline] - pub fn new(slice: &'a [u8]) -> Self { - assert_eq!(slice.len() % std::mem::size_of::(), 0); - let chunks = slice.chunks_exact(std::mem::size_of::()); - Self { - chunks, - phantom: std::marker::PhantomData, - } - } -} - -impl<'a, T: NativeType> Iterator for ExactChunksIter<'a, T> { - type Item = T; - - #[inline] - fn next(&mut self) -> Option { - self.chunks.next().map(|chunk| { - let chunk: ::Bytes = match chunk.try_into() { - Ok(v) => v, - Err(_) => unreachable!(), - }; - T::from_le_bytes(chunk) - }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - self.chunks.size_hint() - } -} - -/// Parquet primitive value reader -pub struct SomePrimitiveValueIter<'a, T: NativeType> { - valid_row_idx_iter: ValidityRowIndexIter<'a>, - value_iter: ExactChunksIter<'a, T>, -} - -impl<'a, T: NativeType> SomePrimitiveValueIter<'a, T> { - /// Create parquet primitive value reader - pub fn try_new( - page: &'a DataPage, - descriptor: &'a ColumnDescriptor, - ) -> Result { - let (max_def_level, validity_iter, values_buffer) = split_page(page, descriptor)?; - let value_iter = ExactChunksIter::::new(values_buffer); - let valid_row_idx_iter = ValidityRowIndexIter::new(max_def_level, validity_iter); - Ok(Self { - value_iter, - valid_row_idx_iter, - }) - } -} - -impl<'a, T: NativeType> Iterator for SomePrimitiveValueIter<'a, T> { - type Item = Result<(usize, T), ParseError>; - - fn next(&mut self) -> Option { - self.valid_row_idx_iter - .next() - .map(|idx_result| idx_result.map(|idx| (idx, self.value_iter.next().unwrap()))) - } -} - -/// Parquet dictionary primitive value reader -pub struct SomeDictionaryPrimitiveValueIter<'a, T: NativeType> { - valid_row_idx_iter: ValidityRowIndexIter<'a>, - index_iter: hybrid_rle::HybridRleDecoder<'a>, - dict_values: Vec, -} - -impl<'a, T: NativeType> SomeDictionaryPrimitiveValueIter<'a, T> { - /// Create parquet primitive value reader - pub fn try_new( - page: &'a DataPage, - dict: &DictPage, - descriptor: &'a ColumnDescriptor, - ) -> Result { - let (max_def_level, validity_iter, values_buffer) = split_page(page, descriptor)?; - - let valid_row_idx_iter = ValidityRowIndexIter::new(max_def_level, validity_iter); - - let dict_values = dictionary::primitive::read::(&dict.buffer, dict.num_values)?; - - let indices_buffer = values_buffer; - let bit_width = indices_buffer[0]; - let indices_buffer = &indices_buffer[1..]; - - let additional = page.num_values(); - let index_iter = - hybrid_rle::HybridRleDecoder::try_new(indices_buffer, bit_width as u32, additional)?; - Ok(Self { - index_iter, - dict_values, - valid_row_idx_iter, - }) - } -} - -impl<'a, T: NativeType> Iterator for SomeDictionaryPrimitiveValueIter<'a, T> { - type Item = Result<(usize, T), ParseError>; - - fn next(&mut self) -> Option { - self.valid_row_idx_iter.next().map(|idx_result| { - let idx = idx_result?; - let dict_idx = self.index_iter.next().ok_or_else(|| { - ParseError::Generic(format!("No dict index matches row index: {}", idx)) - })??; - let value = self.dict_values[dict_idx as usize]; - Ok((idx, value)) - }) - } -} - -#[inline] -pub fn for_each_primitive_field_value( - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - set_fn: SetFn, -) -> Result<(), ParseError> -where - T: NativeType, - ActType: ActionVariant, - SetFn: Fn(&mut ActType, T), -{ - #[cfg(debug_assertions)] - if page.descriptor.primitive_type.physical_type != T::TYPE { - return Err(ParseError::InvalidAction(format!( - "expect physical parquet type {:?}, got {:?}", - T::TYPE, - page.descriptor.primitive_type, - ))); - } - - match (&page.encoding(), dict) { - (Encoding::Plain, None) => { - let some_value_iter = SomePrimitiveValueIter::::try_new(page, descriptor)?; - for entry in some_value_iter { - let (idx, value) = entry?; - let a = actions[idx].get_or_insert_with(ActType::default_action); - set_fn(ActType::try_mut_from_action(a)?, value); - } - } - (Encoding::PlainDictionary | Encoding::RleDictionary, Some(dict)) => { - let some_value_iter = - SomeDictionaryPrimitiveValueIter::try_new(page, &dict, descriptor)?; - for entry in some_value_iter { - let (idx, value) = entry?; - let a = actions[idx].get_or_insert_with(ActType::default_action); - set_fn(ActType::try_mut_from_action(a)?, value); - } - } - _ => { - return Err(ParseError::InvalidAction(format!( - "unsupported page encoding type for primitive column: {:?}", - page.encoding() - ))); - } - } - - Ok(()) -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/stats.rs b/crates/deltalake-core/src/protocol/parquet2_read/stats.rs deleted file mode 100644 index 689dfea6c0..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/stats.rs +++ /dev/null @@ -1,9 +0,0 @@ -use crate::protocol::{Add, ProtocolError, Stats}; - -impl Add { - /// Returns the composite HashMap representation of stats contained in the action if present. - /// Since stats are defined as optional in the protocol, this may be None. - pub fn get_stats_parsed(&self) -> Result, ProtocolError> { - Ok(None) - } -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/string.rs b/crates/deltalake-core/src/protocol/parquet2_read/string.rs deleted file mode 100644 index 391a9b9390..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/string.rs +++ /dev/null @@ -1,312 +0,0 @@ -//! Parquet string deserialization for Action enum - -use parquet2::encoding::hybrid_rle::HybridRleDecoder; -use parquet2::encoding::Encoding; -use parquet2::metadata::ColumnDescriptor; -use parquet2::page::{DataPage, DictPage}; - -use super::dictionary; -use super::dictionary::binary::BinaryPageDict; -use super::validity::{ValidityRepeatedRowIndexIter, ValidityRowIndexIter}; -use super::{split_page, split_page_nested, ActionVariant, ParseError}; -use crate::kernel::Action; - -pub trait StringValueIter<'a>: Iterator> { - fn try_from_encoded_values( - buffer: &'a [u8], - num_values: usize, - _dict: &'a Option, - ) -> Result - where - Self: Sized; -} - -pub struct PlainStringValueIter<'a> { - values_buffer: &'a [u8], -} - -impl<'a> StringValueIter<'a> for PlainStringValueIter<'a> { - fn try_from_encoded_values( - values_buffer: &'a [u8], - _num_values: usize, - _dict: &Option, - ) -> Result { - Ok(Self { values_buffer }) - } -} - -impl<'a> Iterator for PlainStringValueIter<'a> { - type Item = Result; - - fn next(&mut self) -> Option { - let bytes_len = parquet2::encoding::get_length(self.values_buffer).unwrap() as usize; - let bytes_end = bytes_len + 4; - // skip first 4 bytes (length) - let bytes = &self.values_buffer[4..bytes_end]; - self.values_buffer = &self.values_buffer[bytes_end..]; - - Some(Ok(std::str::from_utf8(bytes).unwrap().to_string())) - } -} - -pub struct DictionaryStringValueIter<'a> { - dict_idx_iter: HybridRleDecoder<'a>, - dict: BinaryPageDict<'a>, -} - -impl<'a> StringValueIter<'a> for DictionaryStringValueIter<'a> { - fn try_from_encoded_values( - values_buf: &'a [u8], - num_values: usize, - dict: &'a Option, - ) -> Result { - let bit_width = values_buf[0]; - let indices_buf = &values_buf[1..]; - let dict = dict.as_ref().unwrap(); - let binary_dict = dictionary::binary::read(&dict.buffer, dict.num_values)?; - - Ok(Self { - dict_idx_iter: HybridRleDecoder::try_new(indices_buf, bit_width.into(), num_values)?, - dict: binary_dict, - }) - } -} - -impl<'a> Iterator for DictionaryStringValueIter<'a> { - type Item = Result; - - fn next(&mut self) -> Option { - self.dict_idx_iter.next().map(|result| { - result - .map(|dict_idx| { - let dict_idx = dict_idx as usize; - std::str::from_utf8( - &self.dict.value(dict_idx).expect("Invalid dictionary index"), - ) - .unwrap() - .to_string() - }) - .map_err(|e| e.into()) - }) - } -} - -/// Parquet string value reader -pub struct SomeStringValueIter<'a, ValIter> -where - ValIter: StringValueIter<'a>, -{ - valid_row_idx_iter: ValidityRowIndexIter<'a>, - values_iter: ValIter, -} - -impl<'a, ValIter> SomeStringValueIter<'a, ValIter> -where - ValIter: StringValueIter<'a>, -{ - /// Create parquet string value reader - pub fn try_new( - page: &'a DataPage, - dict: &'a Option, - descriptor: &'a ColumnDescriptor, - ) -> Result { - let (max_def_level, validity_iter, values_buffer) = split_page(page, descriptor)?; - let valid_row_idx_iter = ValidityRowIndexIter::new(max_def_level, validity_iter); - Ok(Self { - valid_row_idx_iter, - // TODO: page.num_values is more than what's being packed in rle - values_iter: ValIter::try_from_encoded_values(values_buffer, page.num_values(), dict)?, - }) - } -} - -impl<'a, ValIter> Iterator for SomeStringValueIter<'a, ValIter> -where - ValIter: StringValueIter<'a>, -{ - type Item = Result<(usize, String), ParseError>; - - fn next(&mut self) -> Option { - self.valid_row_idx_iter.next().map(|result| { - let idx = result?; - let value = self.values_iter.next().ok_or_else(|| { - ParseError::Generic(format!("No string value matches row index: {}", idx)) - })??; - Ok((idx, value)) - }) - } -} - -/// Parquet repeated string value reader -pub struct SomeRepeatedStringValueIter<'a, ValIter> -where - ValIter: StringValueIter<'a>, -{ - repeated_row_idx_iter: ValidityRepeatedRowIndexIter<'a>, - values_iter: ValIter, -} - -impl<'a, ValIter> SomeRepeatedStringValueIter<'a, ValIter> -where - ValIter: StringValueIter<'a>, -{ - /// Create parquet string value reader - pub fn try_new( - page: &'a DataPage, - dict: &'a Option, - descriptor: &'a ColumnDescriptor, - ) -> Result { - let (max_rep_level, rep_iter, max_def_level, validity_iter, values_buffer) = - split_page_nested(page, descriptor)?; - let repeated_row_idx_iter = ValidityRepeatedRowIndexIter::new( - max_rep_level, - rep_iter, - max_def_level, - validity_iter, - ); - - Ok(Self { - values_iter: ValIter::try_from_encoded_values(values_buffer, page.num_values(), dict)?, - repeated_row_idx_iter, - }) - } -} - -impl<'a, ValIter> Iterator for SomeRepeatedStringValueIter<'a, ValIter> -where - ValIter: StringValueIter<'a>, -{ - type Item = Result<(usize, Vec), ParseError>; - - fn next(&mut self) -> Option { - self.repeated_row_idx_iter.next().map(|result| { - let (idx, item_count) = result?; - - let strings = (0..item_count) - .map(|i| { - self.values_iter.next().ok_or_else(|| { - ParseError::Generic(format!("No string value found list index: {}", i)) - })? - }) - .collect::, _>>()?; - - Ok((idx, strings)) - }) - } -} - -pub fn for_each_repeated_string_field_value_with_idx( - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - map_fn: MapFn, -) -> Result<(), ParseError> -where - MapFn: FnMut(Result<(usize, Vec), ParseError>) -> Result<(), ParseError>, -{ - #[cfg(debug_assertions)] - { - use parquet2::schema::types::PhysicalType; - if page.descriptor.primitive_type.physical_type != PhysicalType::ByteArray { - return Err(ParseError::InvalidAction(format!( - "expect parquet utf8 type, got primitive type: {:?}", - page.descriptor.primitive_type, - ))); - } - } - - match page.encoding() { - Encoding::Plain => { - SomeRepeatedStringValueIter::::try_new(page, dict, descriptor)? - .try_for_each(map_fn)?; - } - Encoding::RleDictionary | Encoding::PlainDictionary => { - SomeRepeatedStringValueIter::::try_new( - page, dict, descriptor, - )? - .try_for_each(map_fn)?; - } - _ => { - return Err(ParseError::InvalidAction(format!( - "unsupported page encoding type for string list column: {:?}", - page.encoding() - ))); - } - } - - Ok(()) -} - -pub fn for_each_repeated_string_field_value( - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - set_fn: SetFn, -) -> Result<(), ParseError> -where - ActType: ActionVariant, - SetFn: Fn(&mut ActType, Vec), -{ - for_each_repeated_string_field_value_with_idx( - page, - dict, - descriptor, - |entry: Result<(usize, Vec), ParseError>| -> Result<(), ParseError> { - let (idx, strings) = entry?; - let a = actions[idx].get_or_insert_with(ActType::default_action); - set_fn(ActType::try_mut_from_action(a)?, strings); - Ok(()) - }, - ) -} - -pub fn for_each_string_field_value( - actions: &mut Vec>, - page: &DataPage, - dict: &Option, - descriptor: &ColumnDescriptor, - set_fn: SetFn, -) -> Result<(), ParseError> -where - ActType: ActionVariant, - SetFn: Fn(&mut ActType, String), -{ - #[cfg(debug_assertions)] - { - use parquet2::schema::types::PhysicalType; - if page.descriptor.primitive_type.physical_type != PhysicalType::ByteArray { - return Err(ParseError::InvalidAction(format!( - "expect parquet utf8 type, got primitive type: {:?}", - page.descriptor.primitive_type, - ))); - } - } - - let map_fn = |entry: Result<(usize, String), ParseError>| -> Result<(), ParseError> { - let (idx, value) = entry?; - let a = actions[idx].get_or_insert_with(ActType::default_action); - set_fn(ActType::try_mut_from_action(a)?, value); - - Ok(()) - }; - - match page.encoding() { - Encoding::Plain => { - SomeStringValueIter::::try_new(page, dict, descriptor)? - .try_for_each(map_fn)?; - } - Encoding::RleDictionary | Encoding::PlainDictionary => { - SomeStringValueIter::::try_new(page, dict, descriptor)? - .try_for_each(map_fn)?; - } - _ => { - return Err(ParseError::InvalidAction(format!( - "unsupported page encoding type for string column: {:?}", - page.encoding() - ))); - } - } - - Ok(()) -} diff --git a/crates/deltalake-core/src/protocol/parquet2_read/validity.rs b/crates/deltalake-core/src/protocol/parquet2_read/validity.rs deleted file mode 100644 index 42f9a4d5b3..0000000000 --- a/crates/deltalake-core/src/protocol/parquet2_read/validity.rs +++ /dev/null @@ -1,137 +0,0 @@ -//! Parquet deserialization for row validity - -use super::ParseError; -use parquet2::encoding::hybrid_rle::HybridRleDecoder; - -/// Iterator that returns row index for rows that are not null -pub struct ValidityRowIndexIter<'a> { - row_idx: usize, - max_def_level: u32, - validity_iter: HybridRleDecoder<'a>, -} - -impl<'a> ValidityRowIndexIter<'a> { - /// Create parquet primitive value reader - pub fn new(max_def_level: i16, validity_iter: HybridRleDecoder<'a>) -> Self { - Self { - max_def_level: max_def_level as u32, - validity_iter, - row_idx: 0, - } - } -} - -impl<'a> Iterator for ValidityRowIndexIter<'a> { - type Item = Result; - - fn next(&mut self) -> Option { - for def_lvl in self.validity_iter.by_ref() { - match def_lvl { - Ok(def_lvl) => { - if def_lvl == self.max_def_level { - let row_idx = self.row_idx; - self.row_idx += 1; - return Some(Ok(row_idx)); - } else { - self.row_idx += 1; - continue; - } - } - Err(e) => return Some(Err(e.into())), - } - } - None - } -} - -/// Iterator that returns row index for leaf repeated rows that are not null -/// -/// For example, used in List type where each index contains multiple values. -#[allow(dead_code)] -pub struct ValidityRepeatedRowIndexIter<'a> { - row_idx: usize, - max_def_level: u32, - max_rep_level: u32, - repeat_count: usize, - lvl_iter: std::iter::Zip, HybridRleDecoder<'a>>, -} - -impl<'a> ValidityRepeatedRowIndexIter<'a> { - /// Create parquet primitive value reader - pub fn new( - max_rep_level: i16, - rep_iter: HybridRleDecoder<'a>, - max_def_level: i16, - validity_iter: HybridRleDecoder<'a>, - ) -> Self { - Self { - lvl_iter: rep_iter.zip(validity_iter), - max_rep_level: max_rep_level as u32, - max_def_level: max_def_level as u32, - row_idx: 0, - repeat_count: 0, - } - } -} - -impl<'a> Iterator for ValidityRepeatedRowIndexIter<'a> { - // (index, item_count) - type Item = Result<(usize, usize), ParseError>; - - fn next(&mut self) -> Option { - for (rep_lvl, def_lvl) in self.lvl_iter.by_ref() { - match (rep_lvl, def_lvl) { - (Ok(rep_lvl), Ok(def_lvl)) => { - if def_lvl == self.max_def_level { - if rep_lvl == 0 { - match self.repeat_count { - 0 => self.repeat_count = 1, - item_count => { - // reached start of next batch - // return current batch - let row_idx = self.row_idx; - self.row_idx += 1; - self.repeat_count = 1; - return Some(Ok((row_idx, item_count))); - } - } - } else { - // accumulate count for current batch - self.repeat_count += 1; - } - } else { - if self.repeat_count >= 1 { - // current row is None, emit previous row - let row_idx = self.row_idx; - let item_count = self.repeat_count; - self.row_idx += 1; - // set to 0 becauze def_lvl not at max def level - self.repeat_count = 0; - return Some(Ok((row_idx, item_count))); - } else { - // both previous and current row are None, proceed to the next row - self.row_idx += 1; - continue; - } - } - } - (_, Err(e)) => { - return Some(Err(e.into())); - } - (Err(e), _) => { - return Some(Err(e.into())); - } - } - } - - // end of iteration, emit the last row - if self.repeat_count >= 1 { - let item_count = self.repeat_count; - // set repeat count to 0 so we can end the iteration - self.repeat_count = 0; - Some(Ok((self.row_idx, item_count))) - } else { - None - } - } -} diff --git a/crates/deltalake-core/src/protocol/parquet_read/mod.rs b/crates/deltalake-core/src/protocol/parquet_read/mod.rs index a546e4b0b0..ecad3b7865 100644 --- a/crates/deltalake-core/src/protocol/parquet_read/mod.rs +++ b/crates/deltalake-core/src/protocol/parquet_read/mod.rs @@ -736,7 +736,7 @@ mod tests { use parquet::file::reader::{FileReader, SerializedFileReader}; use std::fs::File; - let path = "./tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet"; + let path = "../deltalake-test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet"; let preader = SerializedFileReader::new(File::open(path).unwrap()).unwrap(); let mut iter = preader.get_row_iter(None).unwrap(); diff --git a/crates/deltalake-core/src/protocol/time_utils.rs b/crates/deltalake-core/src/protocol/time_utils.rs index 407185b927..cf77edb862 100644 --- a/crates/deltalake-core/src/protocol/time_utils.rs +++ b/crates/deltalake-core/src/protocol/time_utils.rs @@ -5,78 +5,6 @@ use arrow::temporal_conversions; #[cfg(feature = "parquet")] use parquet::basic::TimeUnit; -#[cfg(feature = "parquet2")] -use parquet2::schema::types::TimeUnit; - -// vendored from arrow-rs and arrow2 so we don't need to depend on arrow2 when the parquet2 feature -// is enabled. -#[cfg(not(feature = "arrow"))] -mod temporal_conversions { - use chrono::NaiveDateTime; - - /// Number of milliseconds in a second - pub const MILLISECONDS: i64 = 1_000; - /// Number of microseconds in a second - pub const MICROSECONDS: i64 = 1_000_000; - /// Number of nanoseconds in a second - pub const NANOSECONDS: i64 = 1_000_000_000; - - /// converts a `i64` representing a `timestamp(ms)` to [`NaiveDateTime`] - #[inline] - pub fn timestamp_ms_to_datetime(v: i64) -> Option { - let (sec, milli_sec) = split_second(v, MILLISECONDS); - - NaiveDateTime::from_timestamp_opt( - // extract seconds from milliseconds - sec, - // discard extracted seconds and convert milliseconds to nanoseconds - milli_sec * MICROSECONDS as u32, - ) - } - - /// converts a `i64` representing a `timestamp(us)` to [`NaiveDateTime`] - #[inline] - pub fn timestamp_us_to_datetime(v: i64) -> Option { - let (sec, micro_sec) = split_second(v, MICROSECONDS); - - NaiveDateTime::from_timestamp_opt( - // extract seconds from microseconds - sec, - // discard extracted seconds and convert microseconds to nanoseconds - micro_sec * MILLISECONDS as u32, - ) - } - - /// converts a `i64` representing a `timestamp(ns)` to [`NaiveDateTime`] - #[inline] - pub fn timestamp_ns_to_datetime(v: i64) -> Option { - let (sec, nano_sec) = split_second(v, NANOSECONDS); - - NaiveDateTime::from_timestamp_opt( - // extract seconds from nanoseconds - sec, // discard extracted seconds - nano_sec, - ) - } - - /// - #[inline] - pub(crate) fn split_second(v: i64, base: i64) -> (i64, u32) { - if v < 0 { - let v = -v; - let mut seconds = v / base; - let mut part = v % base; - - if part > 0 { - seconds += 1; - part = base - part; - } - (-seconds, part as u32) - } else { - (v / base, (v % base) as u32) - } - } -} /// Convert an ISO-8601/RFC3339 timestamp string to a numeric microsecond epoch representation. /// Stats strings are written with millisecond precision as described by the delta protocol. @@ -85,7 +13,7 @@ pub fn timestamp_micros_from_stats_string(s: &str) -> Result Option { let dt = match time_unit { TimeUnit::MILLIS(_) => temporal_conversions::timestamp_ms_to_datetime(n), @@ -96,25 +24,11 @@ pub fn timestamp_to_delta_stats_string(n: i64, time_unit: &TimeUnit) -> Option Option { - let dt = match time_unit { - TimeUnit::Milliseconds => temporal_conversions::timestamp_ms_to_datetime(n), - TimeUnit::Microseconds => temporal_conversions::timestamp_us_to_datetime(n), - TimeUnit::Nanoseconds => temporal_conversions::timestamp_ns_to_datetime(n), - }?; - - Some(format!("{}", dt.format("%Y-%m-%dT%H:%M:%S%.3fZ"))) -} - -#[cfg(test)] +#[cfg(all(test, feature = "parquet"))] mod tests { use super::*; - #[cfg(not(feature = "parquet2"))] use parquet::format::{MicroSeconds, MilliSeconds, NanoSeconds, TimeUnit}; - #[cfg(not(feature = "parquet2"))] #[test] fn test_timestamp_to_delta_stats_string() { let s = @@ -135,18 +49,6 @@ mod tests { assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); } - #[cfg(feature = "parquet2")] - #[test] - fn test_timestamp_to_delta_stats_string() { - let s = timestamp_to_delta_stats_string(1628685199541, &TimeUnit::Milliseconds).unwrap(); - assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); - let s = timestamp_to_delta_stats_string(1628685199541000, &TimeUnit::Microseconds).unwrap(); - assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); - let s = - timestamp_to_delta_stats_string(1628685199541000000, &TimeUnit::Nanoseconds).unwrap(); - assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); - } - #[test] fn test_timestamp_micros_from_stats_string() { let us = timestamp_micros_from_stats_string("2021-08-11T12:33:19.541Z").unwrap(); diff --git a/crates/deltalake-core/src/storage/config.rs b/crates/deltalake-core/src/storage/config.rs deleted file mode 100644 index a9ef5fc272..0000000000 --- a/crates/deltalake-core/src/storage/config.rs +++ /dev/null @@ -1,394 +0,0 @@ -//! Configurltion handling for defining Storage backends for DeltaTables. -use std::collections::HashMap; -use std::sync::Arc; - -use object_store::memory::InMemory; -use object_store::path::Path; -use object_store::prefix::PrefixStore; -use object_store::{parse_url_opts, DynObjectStore, Error as ObjectStoreError, ObjectStore}; -use serde::{Deserialize, Serialize}; -use url::Url; - -use super::file::FileStorageBackend; -use super::utils::str_is_truthy; -use super::ObjectStoreRef; -use crate::errors::{DeltaResult, DeltaTableError}; -use crate::logstore::default_logstore::DefaultLogStore; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -use crate::logstore::s3::S3DynamoDbLogStore; -use crate::logstore::{LogStoreConfig, LogStoreRef}; -use crate::table::builder::ensure_table_uri; - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -use super::s3::{S3StorageBackend, S3StorageOptions}; -#[cfg(feature = "hdfs")] -use datafusion_objectstore_hdfs::object_store::hdfs::HadoopFileSystem; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -use object_store::aws::AmazonS3ConfigKey; -#[cfg(feature = "azure")] -use object_store::azure::AzureConfigKey; -#[cfg(feature = "gcs")] -use object_store::gcp::GoogleConfigKey; -#[cfg(any( - feature = "s3", - feature = "s3-native-tls", - feature = "gcs", - feature = "azure" -))] -use std::str::FromStr; - -#[cfg(feature = "azure")] -mod azure; - -/// Recognises various URL formats, identifying the relevant [`ObjectStore`](crate::ObjectStore) -#[derive(Debug, Eq, PartialEq)] -enum ObjectStoreScheme { - /// Url corresponding to LocalFileSystem - Local, - /// Url corresponding to InMemory - Memory, - /// Url corresponding to S3 - AmazonS3, - /// Url corresponding to GoogleCloudStorage - GoogleCloudStorage, - /// Url corresponding to MicrosoftAzure - MicrosoftAzure, - /// Url corresponding to HttpStore - Http, - /// Url corresponding to Hdfs - Hdfs, -} - -impl ObjectStoreScheme { - /// Create an [`ObjectStoreScheme`] from the provided [`Url`] - /// - /// Returns the [`ObjectStoreScheme`] and the remaining [`Path`] - fn parse( - url: &Url, - #[allow(unused)] options: &mut StorageOptions, - ) -> Result<(Self, Path), ObjectStoreError> { - let strip_bucket = || Some(url.path().strip_prefix('/')?.split_once('/')?.1); - - let (scheme, path) = match (url.scheme(), url.host_str()) { - ("file", None) => (Self::Local, url.path()), - ("memory", None) => (Self::Memory, url.path()), - ("s3" | "s3a", Some(_)) => (Self::AmazonS3, url.path()), - ("gs", Some(_)) => (Self::GoogleCloudStorage, url.path()), - ("az" | "adl" | "azure" | "abfs" | "abfss", Some(_)) => { - (Self::MicrosoftAzure, url.path()) - } - ("http", Some(_)) => (Self::Http, url.path()), - ("hdfs", Some(_)) => (Self::Hdfs, url.path()), - ("https", Some(host)) => { - if host.ends_with("dfs.core.windows.net") || host.ends_with("blob.core.windows.net") - { - (Self::MicrosoftAzure, url.path()) - } else if host.contains("dfs.fabric.microsoft.com") - || host.contains("blob.fabric.microsoft.com") - { - #[cfg(feature = "azure")] - if !options - .as_azure_options() - .contains_key(&AzureConfigKey::UseFabricEndpoint) - { - options.0.insert( - AzureConfigKey::UseFabricEndpoint.as_ref().to_string(), - "true".to_string(), - ); - } - (Self::MicrosoftAzure, url.path()) - } else if host.ends_with("amazonaws.com") { - match host.starts_with("s3") { - true => (Self::AmazonS3, strip_bucket().unwrap_or_default()), - false => (Self::AmazonS3, url.path()), - } - } else if host.ends_with("r2.cloudflarestorage.com") { - (Self::AmazonS3, strip_bucket().unwrap_or_default()) - } else { - (Self::Http, url.path()) - } - } - _ => return Err(ObjectStoreError::NotImplemented), - }; - - let path = Path::parse(path)?; - Ok((scheme, path)) - } -} - -/// Options used for configuring backend storage -#[derive(Clone, Debug, Serialize, Deserialize, Default)] -pub struct StorageOptions(pub HashMap); - -impl StorageOptions { - /// Create a new instance of [`StorageOptions`] - pub fn new(options: HashMap) -> Self { - let mut options = options; - if let Ok(value) = std::env::var("AZURE_STORAGE_ALLOW_HTTP") { - options.insert("allow_http".into(), value); - } - if let Ok(value) = std::env::var("AZURE_STORAGE_USE_HTTP") { - options.insert("allow_http".into(), value); - } - if let Ok(value) = std::env::var("AWS_ALLOW_HTTP") { - options.insert("allow_http".into(), value); - } - Self(options) - } - - /// Add values from the environment to storage options - #[cfg(feature = "azure")] - pub fn with_env_azure(&mut self) { - for (os_key, os_value) in std::env::vars_os() { - if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { - if let Ok(config_key) = AzureConfigKey::from_str(&key.to_ascii_lowercase()) { - if !self.0.contains_key(config_key.as_ref()) { - self.0 - .insert(config_key.as_ref().to_string(), value.to_string()); - } - } - } - } - } - - /// Add values from the environment to storage options - #[cfg(feature = "gcs")] - pub fn with_env_gcs(&mut self) { - for (os_key, os_value) in std::env::vars_os() { - if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { - if let Ok(config_key) = GoogleConfigKey::from_str(&key.to_ascii_lowercase()) { - if !self.0.contains_key(config_key.as_ref()) { - self.0 - .insert(config_key.as_ref().to_string(), value.to_string()); - } - } - } - } - } - - /// Add values from the environment to storage options - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - pub fn with_env_s3(&mut self) { - for (os_key, os_value) in std::env::vars_os() { - if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { - if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { - if !self.0.contains_key(config_key.as_ref()) { - self.0 - .insert(config_key.as_ref().to_string(), value.to_string()); - } - } - } - } - } - - /// Denotes if unsecure connections via http are allowed - pub fn allow_http(&self) -> bool { - self.0.iter().any(|(key, value)| { - key.to_ascii_lowercase().contains("allow_http") & str_is_truthy(value) - }) - } - - /// Subset of options relevant for azure storage - #[cfg(feature = "azure")] - pub fn as_azure_options(&self) -> HashMap { - self.0 - .iter() - .filter_map(|(key, value)| { - let az_key = AzureConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; - Some((az_key, value.clone())) - }) - .collect() - } - - /// Subset of options relevant for s3 storage - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - pub fn as_s3_options(&self) -> HashMap { - self.0 - .iter() - .filter_map(|(key, value)| { - let s3_key = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; - Some((s3_key, value.clone())) - }) - .collect() - } - - /// Subset of options relevant for gcs storage - #[cfg(feature = "gcs")] - pub fn as_gcs_options(&self) -> HashMap { - self.0 - .iter() - .filter_map(|(key, value)| { - let gcs_key = GoogleConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; - Some((gcs_key, value.clone())) - }) - .collect() - } -} - -impl From> for StorageOptions { - fn from(value: HashMap) -> Self { - Self::new(value) - } -} - -/// Configure a [`LogStoreRef`] for the given url and configuration -pub fn configure_log_store( - location: &str, - options: impl Into + Clone, - storage_backend: Option<(ObjectStoreRef, Url)>, -) -> DeltaResult { - let mut options = options.into(); - let (object_store, location) = match storage_backend { - Some((object_store, url)) => (object_store, url), - None => { - let url = ensure_table_uri(location)?; - let object_store = crate::storage::config::configure_store(&url, &mut options)?; - (object_store, url) - } - }; - - let (scheme, _prefix) = ObjectStoreScheme::parse(&location, &mut options)?; - match scheme { - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - ObjectStoreScheme::AmazonS3 => { - let s3_options = S3StorageOptions::from_map(&options.0); - if Some("dynamodb".to_owned()) - == s3_options - .locking_provider - .as_ref() - .map(|v| v.to_lowercase()) - { - Ok(Arc::new(S3DynamoDbLogStore::try_new( - location, - options, - &s3_options, - object_store, - )?)) - } else { - Ok(Arc::new(DefaultLogStore::new( - object_store, - LogStoreConfig { location, options }, - ))) - } - } - _ => Ok(Arc::new(DefaultLogStore::new( - object_store, - LogStoreConfig { location, options }, - ))), - } -} - -/// Configure an instance of an [`ObjectStore`] for the given url and configuration -pub fn configure_store( - url: &Url, - options: &mut StorageOptions, -) -> DeltaResult> { - let (scheme, _prefix) = ObjectStoreScheme::parse(url, options)?; - match scheme { - ObjectStoreScheme::Local => { - let path = url - .to_file_path() - .map_err(|_| DeltaTableError::InvalidTableLocation(url.to_string()))?; - Ok(Arc::new(FileStorageBackend::try_new(path)?)) - } - ObjectStoreScheme::Memory => url_prefix_handler(InMemory::new(), Path::parse(url.path())?), - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - ObjectStoreScheme::AmazonS3 => { - options.with_env_s3(); - let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; - let s3_options = S3StorageOptions::from_map(&options.0); - if options - .as_s3_options() - .contains_key(&AmazonS3ConfigKey::CopyIfNotExists) - { - url_prefix_handler(store, prefix) - } else if Some("dynamodb".to_owned()) - == s3_options - .locking_provider - .as_ref() - .map(|v| v.to_lowercase()) - { - // if a lock client is requested, unsafe rename is always safe - let store = S3StorageBackend::try_new(Arc::new(store), true)?; - url_prefix_handler(store, prefix) - } else { - let store = - S3StorageBackend::try_new(Arc::new(store), s3_options.allow_unsafe_rename)?; - url_prefix_handler(store, prefix) - } - } - #[cfg(feature = "azure")] - ObjectStoreScheme::MicrosoftAzure => { - let config = azure::AzureConfigHelper::try_new(options.as_azure_options())?.build()?; - let (store, prefix) = parse_url_opts(url, config)?; - url_prefix_handler(store, prefix) - } - #[cfg(feature = "gcs")] - ObjectStoreScheme::GoogleCloudStorage => { - options.with_env_gcs(); - let (store, prefix) = parse_url_opts(url, options.as_gcs_options())?; - url_prefix_handler(store, prefix) - } - #[cfg(feature = "hdfs")] - ObjectStoreScheme::Hdfs => { - let store = HadoopFileSystem::new(url.as_ref()).ok_or_else(|| { - DeltaTableError::Generic(format!( - "failed to create HadoopFileSystem for {}", - url.as_ref() - )) - })?; - url_prefix_handler(store, _prefix) - } - #[cfg(not(feature = "hdfs"))] - ObjectStoreScheme::Hdfs => Err(DeltaTableError::MissingFeature { - feature: "hdfs", - url: url.as_ref().into(), - }), - _ => { - let (store, prefix) = parse_url_opts(url, options.0.clone())?; - url_prefix_handler(store, prefix) - } - } -} - -fn url_prefix_handler(store: T, prefix: Path) -> DeltaResult> { - if prefix != Path::from("/") { - Ok(Arc::new(PrefixStore::new(store, prefix))) - } else { - Ok(Arc::new(store)) - } -} - -#[cfg(test)] -mod test { - use crate::table::builder::ensure_table_uri; - - use super::*; - - #[tokio::test] - async fn test_configure_store_local() -> Result<(), Box> { - let temp_dir = tempfile::tempdir().unwrap(); - let temp_dir_path = temp_dir.path(); - let path = temp_dir_path.join("test space 😁"); - - let table_uri = ensure_table_uri(path.as_os_str().to_str().unwrap()).unwrap(); - - let store = configure_store(&table_uri, &mut StorageOptions::default()).unwrap(); - - let contents = b"test"; - let key = "test.txt"; - let file_path = path.join(key); - std::fs::write(&file_path, contents).unwrap(); - - let res = store - .get(&object_store::path::Path::from(key)) - .await - .unwrap() - .bytes() - .await - .unwrap(); - assert_eq!(res.as_ref(), contents); - - Ok(()) - } -} diff --git a/crates/deltalake-core/src/storage/mod.rs b/crates/deltalake-core/src/storage/mod.rs index 37bdfcb2e0..2398276011 100644 --- a/crates/deltalake-core/src/storage/mod.rs +++ b/crates/deltalake-core/src/storage/mod.rs @@ -1,17 +1,23 @@ //! Object storage backend abstraction layer for Delta Table transaction logs and data -use std::sync::Arc; +use dashmap::DashMap; +use std::collections::HashMap; +use std::sync::{Arc, OnceLock}; use lazy_static::lazy_static; +use serde::{Deserialize, Serialize}; +use url::Url; -pub mod config; pub mod file; pub mod utils; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -pub mod s3; +use crate::{DeltaResult, DeltaTableError}; +pub use object_store; +use object_store::local::LocalFileSystem; +use object_store::memory::InMemory; pub use object_store::path::{Path, DELIMITER}; +use object_store::prefix::PrefixStore; pub use object_store::{ DynObjectStore, Error as ObjectStoreError, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, Result as ObjectStoreResult, @@ -22,11 +28,147 @@ lazy_static! { static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); } +/// Sharable reference to [`ObjectStore`] +pub type ObjectStoreRef = Arc; + +/// Factory trait for creating [ObjectStoreRef] instances at runtime +pub trait ObjectStoreFactory: Send + Sync { + #[allow(missing_docs)] + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)>; +} + +#[derive(Clone, Debug, Default)] +struct DefaultObjectStoreFactory {} + +impl ObjectStoreFactory for DefaultObjectStoreFactory { + fn parse_url_opts( + &self, + url: &Url, + _options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + match url.scheme() { + "memory" => { + let path = Path::from_url_path(url.path())?; + let store = Arc::new(InMemory::new()) as ObjectStoreRef; + Ok((url_prefix_handler(store, path.clone())?, path)) + } + "file" => { + let store = Arc::new(LocalFileSystem::new_with_prefix( + url.to_file_path().unwrap(), + )?) as ObjectStoreRef; + Ok((store, Path::from("/"))) + } + _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), + } + } +} + +/// TODO +pub type FactoryRegistry = Arc>>; + +/// TODO +pub fn factories() -> FactoryRegistry { + static REGISTRY: OnceLock = OnceLock::new(); + REGISTRY + .get_or_init(|| { + let registry = FactoryRegistry::default(); + registry.insert( + Url::parse("memory://").unwrap(), + Arc::new(DefaultObjectStoreFactory::default()), + ); + registry.insert( + Url::parse("file://").unwrap(), + Arc::new(DefaultObjectStoreFactory::default()), + ); + registry + }) + .clone() +} + +/// Simpler access pattern for the [FactoryRegistry] to get a single store +pub fn store_for(url: &Url) -> DeltaResult { + let scheme = Url::parse(&format!("{}://", url.scheme())).unwrap(); + if let Some(factory) = factories().get(&scheme) { + let (store, _prefix) = factory.parse_url_opts(url, &StorageOptions::default())?; + Ok(store) + } else { + Err(DeltaTableError::InvalidTableLocation(url.clone().into())) + } +} + +/// Options used for configuring backend storage +#[derive(Clone, Debug, Serialize, Deserialize, Default)] +pub struct StorageOptions(pub HashMap); + +impl From> for StorageOptions { + fn from(value: HashMap) -> Self { + Self(value) + } +} + /// Return the uri of commit version. +/// +/// ```rust +/// # use deltalake_core::storage::*; +/// use object_store::path::Path; +/// let uri = commit_uri_from_version(1); +/// assert_eq!(uri, Path::from("_delta_log/00000000000000000001.json")); +/// ``` pub fn commit_uri_from_version(version: i64) -> Path { let version = format!("{version:020}.json"); DELTA_LOG_PATH.child(version.as_str()) } -/// Sharable reference to [`ObjectStore`] -pub type ObjectStoreRef = Arc; +#[allow(unused)] +/// Return true for all the stringly values typically associated with true +/// +/// aka YAML booleans +pub fn str_is_truthy(val: &str) -> bool { + val.eq_ignore_ascii_case("1") + | val.eq_ignore_ascii_case("true") + | val.eq_ignore_ascii_case("on") + | val.eq_ignore_ascii_case("yes") + | val.eq_ignore_ascii_case("y") +} + +/// Simple function to wrap the given [ObjectStore] in a [PrefixStore] if necessary +/// +/// This simplifies the use of t he storage since it ensures that list/get/etc operations +/// start from the prefix in the object storage rather than from the root configured URI of the +/// [ObjectStore] +pub fn url_prefix_handler(store: T, prefix: Path) -> DeltaResult { + if prefix != Path::from("/") { + Ok(Arc::new(PrefixStore::new(store, prefix))) + } else { + Ok(Arc::new(store)) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_url_prefix_handler() { + let store = InMemory::new(); + let path = Path::parse("/databases/foo/bar").expect("Failed to parse path"); + + let prefixed = url_prefix_handler(store, path); + assert!(prefixed.is_ok()); + } + + #[test] + fn test_str_is_truthy() { + for value in ["1", "true", "on", "YES", "Y"].iter() { + assert!(str_is_truthy(value)); + } + + for value in ["0", "FALSE", "off", "NO", "n", "bork"].iter() { + assert!(!str_is_truthy(value)); + } + } +} diff --git a/crates/deltalake-core/src/storage/s3.rs b/crates/deltalake-core/src/storage/s3.rs deleted file mode 100644 index b7bf446317..0000000000 --- a/crates/deltalake-core/src/storage/s3.rs +++ /dev/null @@ -1,510 +0,0 @@ -//! AWS S3 storage backend. - -use super::utils::str_is_truthy; -use crate::table::builder::{s3_storage_options, str_option}; -use bytes::Bytes; -use futures::stream::BoxStream; -use object_store::{path::Path, Error as ObjectStoreError}; -use object_store::{ - DynObjectStore, GetOptions, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, - Result as ObjectStoreResult, -}; -use rusoto_core::Region; -use std::collections::HashMap; -use std::fmt::Debug; -use std::ops::Range; -use std::sync::Arc; -use std::time::Duration; -use tokio::io::AsyncWrite; - -const STORE_NAME: &str = "DeltaS3ObjectStore"; - -/// Options used to configure the S3StorageBackend. -/// -/// Available options are described in [s3_storage_options]. -#[derive(Clone, Debug, PartialEq, Eq)] -#[allow(missing_docs)] -pub struct S3StorageOptions { - pub endpoint_url: Option, - pub region: Region, - pub profile: Option, - pub aws_access_key_id: Option, - pub aws_secret_access_key: Option, - pub aws_session_token: Option, - pub virtual_hosted_style_request: bool, - pub locking_provider: Option, - pub assume_role_arn: Option, - pub assume_role_session_name: Option, - pub use_web_identity: bool, - pub s3_pool_idle_timeout: Duration, - pub sts_pool_idle_timeout: Duration, - pub s3_get_internal_server_error_retries: usize, - pub allow_unsafe_rename: bool, - pub extra_opts: HashMap, -} - -impl S3StorageOptions { - /// Creates an instance of S3StorageOptions from the given HashMap. - pub fn from_map(options: &HashMap) -> S3StorageOptions { - let extra_opts = options - .iter() - .filter(|(k, _)| !s3_storage_options::S3_OPTS.contains(&k.as_str())) - .map(|(k, v)| (k.to_owned(), v.to_owned())) - .collect(); - - // Copy web identity values provided in options but not the environment into the environment - // to get picked up by the `from_k8s_env` call in `get_web_identity_provider`. - Self::ensure_env_var(options, s3_storage_options::AWS_REGION); - Self::ensure_env_var(options, s3_storage_options::AWS_PROFILE); - Self::ensure_env_var(options, s3_storage_options::AWS_ACCESS_KEY_ID); - Self::ensure_env_var(options, s3_storage_options::AWS_SECRET_ACCESS_KEY); - Self::ensure_env_var(options, s3_storage_options::AWS_SESSION_TOKEN); - Self::ensure_env_var(options, s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE); - Self::ensure_env_var(options, s3_storage_options::AWS_ROLE_ARN); - Self::ensure_env_var(options, s3_storage_options::AWS_ROLE_SESSION_NAME); - - let endpoint_url = str_option(options, s3_storage_options::AWS_ENDPOINT_URL); - let region = if let Some(endpoint_url) = endpoint_url.as_ref() { - Region::Custom { - name: Self::str_or_default( - options, - s3_storage_options::AWS_REGION, - "custom".to_string(), - ), - endpoint: endpoint_url.to_owned(), - } - } else { - Region::default() - }; - let profile = str_option(options, s3_storage_options::AWS_PROFILE); - - let s3_pool_idle_timeout = Self::u64_or_default( - options, - s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, - 15, - ); - let sts_pool_idle_timeout = Self::u64_or_default( - options, - s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, - 10, - ); - - let s3_get_internal_server_error_retries = Self::u64_or_default( - options, - s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - 10, - ) as usize; - - let virtual_hosted_style_request: bool = - str_option(options, s3_storage_options::AWS_S3_ADDRESSING_STYLE) - .map(|addressing_style| addressing_style == "virtual") - .unwrap_or(false); - - let allow_unsafe_rename = - str_option(options, s3_storage_options::AWS_S3_ALLOW_UNSAFE_RENAME) - .map(|val| str_is_truthy(&val)) - .unwrap_or(false); - - Self { - endpoint_url, - region, - profile, - aws_access_key_id: str_option(options, s3_storage_options::AWS_ACCESS_KEY_ID), - aws_secret_access_key: str_option(options, s3_storage_options::AWS_SECRET_ACCESS_KEY), - aws_session_token: str_option(options, s3_storage_options::AWS_SESSION_TOKEN), - virtual_hosted_style_request, - locking_provider: str_option(options, s3_storage_options::AWS_S3_LOCKING_PROVIDER), - assume_role_arn: str_option(options, s3_storage_options::AWS_S3_ASSUME_ROLE_ARN), - assume_role_session_name: str_option( - options, - s3_storage_options::AWS_S3_ROLE_SESSION_NAME, - ), - use_web_identity: std::env::var(s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE) - .is_ok(), - s3_pool_idle_timeout: Duration::from_secs(s3_pool_idle_timeout), - sts_pool_idle_timeout: Duration::from_secs(sts_pool_idle_timeout), - s3_get_internal_server_error_retries, - allow_unsafe_rename, - extra_opts, - } - } - - fn str_or_default(map: &HashMap, key: &str, default: String) -> String { - map.get(key) - .map(|v| v.to_owned()) - .unwrap_or_else(|| std::env::var(key).unwrap_or(default)) - } - - fn u64_or_default(map: &HashMap, key: &str, default: u64) -> u64 { - str_option(map, key) - .and_then(|v| v.parse().ok()) - .unwrap_or(default) - } - - fn ensure_env_var(map: &HashMap, key: &str) { - if let Some(val) = str_option(map, key) { - std::env::set_var(key, val); - } - } -} - -impl Default for S3StorageOptions { - /// Creates an instance of S3StorageOptions from environment variables. - fn default() -> S3StorageOptions { - Self::from_map(&HashMap::new()) - } -} - -/// An S3 implementation of the [ObjectStore] trait -pub struct S3StorageBackend { - inner: Arc, - /// Whether allowed to performance rename_if_not_exist as rename - allow_unsafe_rename: bool, -} - -impl std::fmt::Display for S3StorageBackend { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "S3StorageBackend") - } -} - -impl S3StorageBackend { - /// Creates a new S3StorageBackend. - /// - /// Options are described in [s3_storage_options]. - /// - /// ```rust - /// use object_store::aws::AmazonS3Builder; - /// use deltalake_core::storage::s3::{S3StorageBackend, S3StorageOptions}; - /// use std::sync::Arc; - /// - /// let inner = AmazonS3Builder::new() - /// .with_region("us-east-1") - /// .with_bucket_name("my-bucket") - /// .with_access_key_id("") - /// .with_secret_access_key("") - /// .build() - /// .unwrap(); - /// let allow_unsafe_rename = true; - /// let store = S3StorageBackend::try_new(Arc::new(inner), allow_unsafe_rename).unwrap(); - /// ``` - pub fn try_new( - storage: Arc, - allow_unsafe_rename: bool, - ) -> ObjectStoreResult { - Ok(Self { - inner: storage, - allow_unsafe_rename, - }) - } -} - -impl std::fmt::Debug for S3StorageBackend { - fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - write!(fmt, "S3StorageBackend") - } -} - -#[async_trait::async_trait] -impl ObjectStore for S3StorageBackend { - async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult<()> { - self.inner.put(location, bytes).await - } - - async fn get(&self, location: &Path) -> ObjectStoreResult { - self.inner.get(location).await - } - - async fn get_opts(&self, location: &Path, options: GetOptions) -> ObjectStoreResult { - self.inner.get_opts(location, options).await - } - - async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { - self.inner.get_range(location, range).await - } - - async fn head(&self, location: &Path) -> ObjectStoreResult { - self.inner.head(location).await - } - - async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { - self.inner.delete(location).await - } - - async fn list( - &self, - prefix: Option<&Path>, - ) -> ObjectStoreResult>> { - self.inner.list(prefix).await - } - - async fn list_with_offset( - &self, - prefix: Option<&Path>, - offset: &Path, - ) -> ObjectStoreResult>> { - self.inner.list_with_offset(prefix, offset).await - } - - async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { - self.inner.list_with_delimiter(prefix).await - } - - async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { - self.inner.copy(from, to).await - } - - async fn copy_if_not_exists(&self, _from: &Path, _to: &Path) -> ObjectStoreResult<()> { - todo!() - } - - async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { - if self.allow_unsafe_rename { - self.inner.rename(from, to).await - } else { - Err(ObjectStoreError::Generic { - store: STORE_NAME, - source: Box::new(deltalake_aws::errors::LockClientError::LockClientRequired), - }) - } - } - - async fn put_multipart( - &self, - location: &Path, - ) -> ObjectStoreResult<(MultipartId, Box)> { - self.inner.put_multipart(location).await - } - - async fn abort_multipart( - &self, - location: &Path, - multipart_id: &MultipartId, - ) -> ObjectStoreResult<()> { - self.inner.abort_multipart(location, multipart_id).await - } -} - -#[cfg(test)] -mod tests { - use super::*; - - use maplit::hashmap; - use serial_test::serial; - - #[test] - #[serial] - fn storage_options_default_test() { - std::env::set_var(s3_storage_options::AWS_ENDPOINT_URL, "http://localhost"); - std::env::set_var(s3_storage_options::AWS_REGION, "us-west-1"); - std::env::set_var(s3_storage_options::AWS_PROFILE, "default"); - std::env::set_var(s3_storage_options::AWS_ACCESS_KEY_ID, "default_key_id"); - std::env::set_var( - s3_storage_options::AWS_SECRET_ACCESS_KEY, - "default_secret_key", - ); - std::env::set_var(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); - std::env::set_var( - s3_storage_options::AWS_S3_ASSUME_ROLE_ARN, - "arn:aws:iam::123456789012:role/some_role", - ); - std::env::set_var(s3_storage_options::AWS_S3_ROLE_SESSION_NAME, "session_name"); - std::env::set_var( - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE, - "token_file", - ); - std::env::remove_var(s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS); - std::env::remove_var(s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS); - std::env::remove_var(s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES); - - let options = S3StorageOptions::default(); - - assert_eq!( - S3StorageOptions { - endpoint_url: Some("http://localhost".to_string()), - region: Region::Custom { - name: "us-west-1".to_string(), - endpoint: "http://localhost".to_string() - }, - profile: Some("default".to_string()), - aws_access_key_id: Some("default_key_id".to_string()), - aws_secret_access_key: Some("default_secret_key".to_string()), - aws_session_token: None, - virtual_hosted_style_request: false, - assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), - assume_role_session_name: Some("session_name".to_string()), - use_web_identity: true, - locking_provider: Some("dynamodb".to_string()), - s3_pool_idle_timeout: Duration::from_secs(15), - sts_pool_idle_timeout: Duration::from_secs(10), - s3_get_internal_server_error_retries: 10, - extra_opts: HashMap::new(), - allow_unsafe_rename: false, - }, - options - ); - } - - #[test] - #[serial] - fn storage_options_with_only_region_and_credentials() { - std::env::remove_var(s3_storage_options::AWS_ENDPOINT_URL); - let options = S3StorageOptions::from_map(&hashmap! { - s3_storage_options::AWS_REGION.to_string() => "eu-west-1".to_string(), - s3_storage_options::AWS_ACCESS_KEY_ID.to_string() => "test".to_string(), - s3_storage_options::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), - }); - - assert_eq!( - S3StorageOptions { - endpoint_url: None, - region: Region::default(), - aws_access_key_id: Some("test".to_string()), - aws_secret_access_key: Some("test_secret".to_string()), - ..Default::default() - }, - options - ); - } - - #[test] - #[serial] - fn storage_options_from_map_test() { - let options = S3StorageOptions::from_map(&hashmap! { - s3_storage_options::AWS_ENDPOINT_URL.to_string() => "http://localhost:1234".to_string(), - s3_storage_options::AWS_REGION.to_string() => "us-west-2".to_string(), - s3_storage_options::AWS_PROFILE.to_string() => "default".to_string(), - s3_storage_options::AWS_S3_ADDRESSING_STYLE.to_string() => "virtual".to_string(), - s3_storage_options::AWS_S3_LOCKING_PROVIDER.to_string() => "another_locking_provider".to_string(), - s3_storage_options::AWS_S3_ASSUME_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/another_role".to_string(), - s3_storage_options::AWS_S3_ROLE_SESSION_NAME.to_string() => "another_session_name".to_string(), - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "another_token_file".to_string(), - s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "1".to_string(), - s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "2".to_string(), - s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES.to_string() => "3".to_string(), - s3_storage_options::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), - s3_storage_options::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), - }); - - assert_eq!( - S3StorageOptions { - endpoint_url: Some("http://localhost:1234".to_string()), - region: Region::Custom { - name: "us-west-2".to_string(), - endpoint: "http://localhost:1234".to_string() - }, - profile: Some("default".to_string()), - aws_access_key_id: Some("test_id".to_string()), - aws_secret_access_key: Some("test_secret".to_string()), - aws_session_token: None, - virtual_hosted_style_request: true, - assume_role_arn: Some("arn:aws:iam::123456789012:role/another_role".to_string()), - assume_role_session_name: Some("another_session_name".to_string()), - use_web_identity: true, - locking_provider: Some("another_locking_provider".to_string()), - s3_pool_idle_timeout: Duration::from_secs(1), - sts_pool_idle_timeout: Duration::from_secs(2), - s3_get_internal_server_error_retries: 3, - extra_opts: hashmap! { - s3_storage_options::AWS_S3_ADDRESSING_STYLE.to_string() => "virtual".to_string() - }, - allow_unsafe_rename: false, - }, - options - ); - } - - #[test] - #[serial] - fn storage_options_mixed_test() { - std::env::set_var(s3_storage_options::AWS_ENDPOINT_URL, "http://localhost"); - std::env::set_var(s3_storage_options::AWS_REGION, "us-west-1"); - std::env::set_var(s3_storage_options::AWS_PROFILE, "default"); - std::env::set_var(s3_storage_options::AWS_ACCESS_KEY_ID, "wrong_key_id"); - std::env::set_var( - s3_storage_options::AWS_SECRET_ACCESS_KEY, - "wrong_secret_key", - ); - std::env::set_var(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); - std::env::set_var( - s3_storage_options::AWS_S3_ASSUME_ROLE_ARN, - "arn:aws:iam::123456789012:role/some_role", - ); - std::env::set_var(s3_storage_options::AWS_S3_ROLE_SESSION_NAME, "session_name"); - std::env::set_var( - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE, - "token_file", - ); - - std::env::set_var(s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, "1"); - std::env::set_var(s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, "2"); - std::env::set_var( - s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - "3", - ); - let options = S3StorageOptions::from_map(&hashmap! { - s3_storage_options::AWS_ACCESS_KEY_ID.to_string() => "test_id_mixed".to_string(), - s3_storage_options::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret_mixed".to_string(), - s3_storage_options::AWS_REGION.to_string() => "us-west-2".to_string(), - "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), - "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES".to_string() => "3".to_string(), - }); - - assert_eq!( - S3StorageOptions { - endpoint_url: Some("http://localhost".to_string()), - region: Region::Custom { - name: "us-west-2".to_string(), - endpoint: "http://localhost".to_string() - }, - profile: Some("default".to_string()), - aws_access_key_id: Some("test_id_mixed".to_string()), - aws_secret_access_key: Some("test_secret_mixed".to_string()), - aws_session_token: None, - virtual_hosted_style_request: false, - assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), - assume_role_session_name: Some("session_name".to_string()), - use_web_identity: true, - locking_provider: Some("dynamodb".to_string()), - s3_pool_idle_timeout: Duration::from_secs(1), - sts_pool_idle_timeout: Duration::from_secs(2), - s3_get_internal_server_error_retries: 3, - extra_opts: hashmap! { - "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), - }, - allow_unsafe_rename: false, - }, - options - ); - } - #[test] - #[serial] - fn storage_options_web_identity_test() { - let _options = S3StorageOptions::from_map(&hashmap! { - s3_storage_options::AWS_REGION.to_string() => "eu-west-1".to_string(), - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "web_identity_token_file".to_string(), - s3_storage_options::AWS_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/web_identity_role".to_string(), - s3_storage_options::AWS_ROLE_SESSION_NAME.to_string() => "web_identity_session_name".to_string(), - }); - - assert_eq!( - "eu-west-1", - std::env::var(s3_storage_options::AWS_REGION).unwrap() - ); - - assert_eq!( - "web_identity_token_file", - std::env::var(s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE).unwrap() - ); - - assert_eq!( - "arn:aws:iam::123456789012:role/web_identity_role", - std::env::var(s3_storage_options::AWS_ROLE_ARN).unwrap() - ); - - assert_eq!( - "web_identity_session_name", - std::env::var(s3_storage_options::AWS_ROLE_SESSION_NAME).unwrap() - ); - } -} diff --git a/crates/deltalake-core/src/storage/utils.rs b/crates/deltalake-core/src/storage/utils.rs index 7b8e76c47d..39f052a89e 100644 --- a/crates/deltalake-core/src/storage/utils.rs +++ b/crates/deltalake-core/src/storage/utils.rs @@ -60,14 +60,6 @@ pub async fn flatten_list_stream( .await } -pub(crate) fn str_is_truthy(val: &str) -> bool { - val.eq_ignore_ascii_case("1") - | val.eq_ignore_ascii_case("true") - | val.eq_ignore_ascii_case("on") - | val.eq_ignore_ascii_case("yes") - | val.eq_ignore_ascii_case("y") -} - impl TryFrom for ObjectMeta { type Error = DeltaTableError; diff --git a/crates/deltalake-core/src/table/builder.rs b/crates/deltalake-core/src/table/builder.rs index b47411383f..19221d630a 100644 --- a/crates/deltalake-core/src/table/builder.rs +++ b/crates/deltalake-core/src/table/builder.rs @@ -5,6 +5,7 @@ use std::path::PathBuf; use std::sync::Arc; use chrono::{DateTime, FixedOffset, Utc}; +use log::*; use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; use url::Url; @@ -12,7 +13,7 @@ use url::Url; use super::DeltaTable; use crate::errors::{DeltaResult, DeltaTableError}; use crate::logstore::LogStoreRef; -use crate::storage::config::{self, StorageOptions}; +use crate::storage::StorageOptions; #[allow(dead_code)] #[derive(Debug, thiserror::Error)] @@ -134,11 +135,6 @@ impl DeltaTableLoadOptions { } } -enum UriType { - LocalPath(PathBuf), - Url(Url), -} - /// builder for configuring a delta table load. #[derive(Debug)] pub struct DeltaTableBuilder { @@ -150,31 +146,48 @@ pub struct DeltaTableBuilder { impl DeltaTableBuilder { /// Creates `DeltaTableBuilder` from table uri + /// + /// Can panic on an invalid URI + /// + /// ```rust + /// # use deltalake_core::table::builder::*; + /// let builder = DeltaTableBuilder::from_uri("../deltalake-test/tests/data/delta-0.8.0"); + /// assert!(true); + /// ``` pub fn from_uri(table_uri: impl AsRef) -> Self { - Self { - options: DeltaTableLoadOptions::new(table_uri.as_ref()), - storage_options: None, - allow_http: None, - } + let url = ensure_table_uri(&table_uri).expect("The specified table_uri is not valid"); + DeltaTableBuilder::from_valid_uri(url).expect("Failed to create valid builder") } /// Creates `DeltaTableBuilder` from verified table uri. - /// Will fail fast if specified `table_uri` is a local path but doesn't exist. + /// + /// ```rust + /// # use deltalake_core::table::builder::*; + /// let builder = DeltaTableBuilder::from_valid_uri("/tmp"); + /// assert!(builder.is_ok(), "Builder failed with {builder:?}"); + /// ``` pub fn from_valid_uri(table_uri: impl AsRef) -> DeltaResult { - let table_uri = table_uri.as_ref(); - - if let UriType::LocalPath(path) = resolve_uri_type(table_uri)? { - if !path.exists() { - let msg = format!( - "Local path \"{}\" does not exist or you don't have access!", - table_uri - ); - return Err(DeltaTableError::InvalidTableLocation(msg)); + if let Ok(url) = Url::parse(table_uri.as_ref()) { + if url.scheme() == "file" { + let path = url.to_file_path().map_err(|_| { + DeltaTableError::InvalidTableLocation(table_uri.as_ref().to_string()) + })?; + ensure_file_location_exists(path)?; } + } else { + ensure_file_location_exists(PathBuf::from(table_uri.as_ref()))?; } - Ok(DeltaTableBuilder::from_uri(table_uri)) + let url = ensure_table_uri(&table_uri).expect("The specified table_uri is not valid"); + debug!("creating table builder with {url}"); + + Ok(Self { + options: DeltaTableLoadOptions::new(url), + storage_options: None, + allow_http: None, + }) } + /// Sets `require_tombstones=false` to the builder pub fn without_tombstones(mut self) -> Self { self.options.require_tombstones = false; @@ -266,11 +279,22 @@ impl DeltaTableBuilder { /// Build a delta storage backend for the given config pub fn build_storage(self) -> DeltaResult { - config::configure_log_store( - &self.options.table_uri, - self.storage_options(), - self.options.storage_backend, - ) + debug!("build_storage() with {}", &self.options.table_uri); + let location = Url::parse(&self.options.table_uri).map_err(|_| { + DeltaTableError::NotATable(format!( + "Could not turn {} into a URL", + self.options.table_uri + )) + })?; + + if let Some((store, _url)) = self.options.storage_backend.as_ref() { + debug!("Loading a logstore with a custom store: {store:?}"); + crate::logstore::logstore_with(store.clone(), location, self.storage_options()) + } 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, self.storage_options()) + } } /// Build the [`DeltaTable`] from specified options. @@ -299,126 +323,17 @@ impl DeltaTableBuilder { } } -/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. -/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. -/// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. -pub mod s3_storage_options { - /// Custom S3 endpoint. - pub const AWS_ENDPOINT_URL: &str = "AWS_ENDPOINT_URL"; - /// The AWS region. - pub const AWS_REGION: &str = "AWS_REGION"; - /// The AWS profile. - pub const AWS_PROFILE: &str = "AWS_PROFILE"; - /// The AWS_ACCESS_KEY_ID to use for S3. - pub const AWS_ACCESS_KEY_ID: &str = "AWS_ACCESS_KEY_ID"; - /// The AWS_SECRET_ACCESS_KEY to use for S3. - pub const AWS_SECRET_ACCESS_KEY: &str = "AWS_SECRET_ACCESS_KEY"; - /// The AWS_SESSION_TOKEN to use for S3. - pub const AWS_SESSION_TOKEN: &str = "AWS_SESSION_TOKEN"; - /// Uses either "path" (the default) or "virtual", which turns on - /// [virtual host addressing](http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). - pub const AWS_S3_ADDRESSING_STYLE: &str = "AWS_S3_ADDRESSING_STYLE"; - /// Locking provider to use for safe atomic rename. - /// `dynamodb` is currently the only supported locking provider. - /// If not set, safe atomic rename is not available. - pub const AWS_S3_LOCKING_PROVIDER: &str = "AWS_S3_LOCKING_PROVIDER"; - /// The role to assume for S3 writes. - pub const AWS_S3_ASSUME_ROLE_ARN: &str = "AWS_S3_ASSUME_ROLE_ARN"; - /// The role session name to use when a role is assumed. If not provided a random session name is generated. - pub const AWS_S3_ROLE_SESSION_NAME: &str = "AWS_S3_ROLE_SESSION_NAME"; - /// The `pool_idle_timeout` option of aws http client. Has to be lower than 20 seconds, which is - /// default S3 server timeout . - /// However, since rusoto uses hyper as a client, its default timeout is 90 seconds - /// . - /// Hence, the `connection closed before message completed` could occur. - /// To avoid that, the default value of this setting is 15 seconds if it's not set otherwise. - pub const AWS_S3_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_S3_POOL_IDLE_TIMEOUT_SECONDS"; - /// The `pool_idle_timeout` for the as3_storage_options sts client. See - /// the reasoning in `AWS_S3_POOL_IDLE_TIMEOUT_SECONDS`. - pub const AWS_STS_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_STS_POOL_IDLE_TIMEOUT_SECONDS"; - /// The number of retries for S3 GET requests failed with 500 Internal Server Error. - pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = - "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES"; - /// The web identity token file to use when using a web identity provider. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; - /// The role name to use for web identity. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; - /// The role session name to use for web identity. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; - /// Allow http connections - mainly useful for integration tests - pub const AWS_ALLOW_HTTP: &str = "AWS_ALLOW_HTTP"; - - /// If set to "true", allows creating commits without concurrent writer protection. - /// Only safe if there is one writer to a given table. - pub const AWS_S3_ALLOW_UNSAFE_RENAME: &str = "AWS_S3_ALLOW_UNSAFE_RENAME"; - - /// The list of option keys owned by the S3 module. - /// Option keys not contained in this list will be added to the `extra_opts` - /// field of [crate::storage::s3::S3StorageOptions]. - pub const S3_OPTS: &[&str] = &[ - AWS_ENDPOINT_URL, - AWS_REGION, - AWS_PROFILE, - AWS_ACCESS_KEY_ID, - AWS_SECRET_ACCESS_KEY, - AWS_SESSION_TOKEN, - AWS_S3_LOCKING_PROVIDER, - AWS_S3_ASSUME_ROLE_ARN, - AWS_S3_ROLE_SESSION_NAME, - AWS_WEB_IDENTITY_TOKEN_FILE, - AWS_ROLE_ARN, - AWS_ROLE_SESSION_NAME, - AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, - AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, - AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - ]; -} - -#[allow(dead_code)] -pub(crate) fn str_option(map: &HashMap, key: &str) -> Option { - map.get(key) - .map_or_else(|| std::env::var(key).ok(), |v| Some(v.to_owned())) -} - -lazy_static::lazy_static! { - static ref KNOWN_SCHEMES: Vec<&'static str> = - Vec::from([ - "file", "memory", "az", "abfs", "abfss", "azure", "wasb", "wasbs", "adl", "s3", "s3a", - "gs", "hdfs", "https", "http", - ]); -} - -/// Utility function to figure out whether string representation of the path -/// is either local path or some kind or URL. -/// -/// Will return an error if the path is not valid. -fn resolve_uri_type(table_uri: impl AsRef) -> DeltaResult { - let table_uri = table_uri.as_ref(); - - if let Ok(url) = Url::parse(table_uri) { - if url.scheme() == "file" { - Ok(UriType::LocalPath(url.to_file_path().map_err(|err| { - let msg = format!("Invalid table location: {}\nError: {:?}", table_uri, err); - DeltaTableError::InvalidTableLocation(msg) - })?)) - // NOTE this check is required to support absolute windows paths which may properly parse as url - } else if KNOWN_SCHEMES.contains(&url.scheme()) { - Ok(UriType::Url(url)) - } else { - Ok(UriType::LocalPath(PathBuf::from(table_uri))) - } - } else { - Ok(UriType::LocalPath(PathBuf::from(table_uri))) +fn create_filetree_from_path(path: &PathBuf) -> DeltaResult<()> { + if !path.exists() { + std::fs::create_dir_all(path).map_err(|err| { + let msg = format!( + "Could not create local directory: {:?}\nError: {:?}", + path, err + ); + DeltaTableError::InvalidTableLocation(msg) + })?; } + Ok(()) } /// Attempt to create a Url from given table location. @@ -435,41 +350,54 @@ fn resolve_uri_type(table_uri: impl AsRef) -> DeltaResult { pub fn ensure_table_uri(table_uri: impl AsRef) -> DeltaResult { let table_uri = table_uri.as_ref(); - let uri_type: UriType = resolve_uri_type(table_uri)?; - - // If it is a local path, we need to create it if it does not exist. - let mut url = match uri_type { - UriType::LocalPath(path) => { - if !path.exists() { - std::fs::create_dir_all(&path).map_err(|err| { - let msg = format!( - "Could not create local directory: {}\nError: {:?}", - table_uri, err - ); - DeltaTableError::InvalidTableLocation(msg) - })?; + debug!("ensure_table_uri {table_uri}"); + let mut url = match Url::parse(table_uri) { + Ok(url) => { + if url.scheme() == "file" { + create_filetree_from_path( + &url.to_file_path() + .expect("Failed to convert a file:// URL to a file path"), + )?; } - let path = std::fs::canonicalize(path).map_err(|err| { - let msg = format!("Invalid table location: {}\nError: {:?}", table_uri, err); + Ok(url) + } + Err(_) => { + let path = PathBuf::from(table_uri); + create_filetree_from_path(&path)?; + let path = std::fs::canonicalize(path.clone()).map_err(|err| { + let msg = format!("Invalid table location: {:?}\nError: {:?}", path, err); DeltaTableError::InvalidTableLocation(msg) })?; - Url::from_directory_path(path).map_err(|_| { + + Url::from_directory_path(path.clone()).map_err(|_| { let msg = format!( - "Could not construct a URL from canonicalized path: {}.\n\ + "Could not construct a URL from canonicalized path: {:?}.\n\ Something must be very wrong with the table path.", - table_uri + path, ); DeltaTableError::InvalidTableLocation(msg) - })? + }) } - UriType::Url(url) => url, - }; + }?; let trimmed_path = url.path().trim_end_matches('/').to_owned(); url.set_path(&trimmed_path); Ok(url) } +/// Validate that the given [PathBuf] does exist, otherwise return a +/// [DeltaTableError::InvalidTableLocation] +fn ensure_file_location_exists(path: PathBuf) -> DeltaResult<()> { + if !path.exists() { + let msg = format!( + "Local path \"{}\" does not exist or you don't have access!", + path.as_path().display(), + ); + return Err(DeltaTableError::InvalidTableLocation(msg)); + } + Ok(()) +} + #[cfg(test)] mod tests { use super::*; @@ -483,6 +411,8 @@ mod tests { assert!(uri.is_ok()); let _uri = ensure_table_uri("./nonexistent"); assert!(uri.is_ok()); + let uri = ensure_table_uri("file:///tmp/nonexistent/some/path"); + assert!(uri.is_ok()); let uri = ensure_table_uri("s3://container/path"); assert!(uri.is_ok()); @@ -567,7 +497,7 @@ mod tests { #[test] fn test_ensure_table_uri_url() { // Urls should round trips as-is - let expected = Url::parse("s3://tests/data/delta-0.8.0").unwrap(); + let expected = Url::parse("s3://deltalake-test/tests/data/delta-0.8.0").unwrap(); let url = ensure_table_uri(&expected).unwrap(); assert_eq!(expected, url); @@ -581,7 +511,7 @@ mod tests { #[tokio::test] async fn read_delta_table_ignoring_tombstones() { - let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + let table = DeltaTableBuilder::from_uri("../deltalake-test/tests/data/delta-0.8.0") .without_tombstones() .load() .await @@ -602,7 +532,7 @@ mod tests { #[tokio::test] async fn read_delta_table_ignoring_files() { - let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + let table = DeltaTableBuilder::from_uri("../deltalake-test/tests/data/delta-0.8.0") .without_files() .load() .await @@ -617,7 +547,7 @@ mod tests { #[tokio::test] async fn read_delta_table_with_ignoring_files_on_apply_log() { - let mut table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + let mut table = DeltaTableBuilder::from_uri("../deltalake-test/tests/data/delta-0.8.0") .with_version(0) .without_files() .load() diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index 94fef6ae1b..018e79ebe9 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -30,7 +30,6 @@ use crate::partitions::PartitionFilter; use crate::protocol::{ find_latest_check_point_for_version, get_last_checkpoint, ProtocolError, Stats, }; -use crate::storage::config::configure_log_store; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; pub mod builder; @@ -347,12 +346,9 @@ impl<'de> Deserialize<'de> for DeltaTable { let storage_config: LogStoreConfig = seq .next_element()? .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let log_store = configure_log_store( - storage_config.location.as_ref(), - storage_config.options, - None, - ) - .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; + let log_store = + crate::logstore::logstore_for(storage_config.location, storage_config.options) + .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; let last_check_point = seq .next_element()? .ok_or_else(|| A::Error::invalid_length(0, &self))?; @@ -474,7 +470,7 @@ impl DeltaTable { Ok(current_delta_log_ver) } - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] async fn restore_checkpoint(&mut self, check_point: CheckPoint) -> Result<(), DeltaTableError> { self.state = DeltaTableState::from_checkpoint(self, &check_point).await?; @@ -516,7 +512,7 @@ impl DeltaTable { /// Updates the DeltaTable to the most recent state committed to the transaction log by /// loading the last checkpoint and incrementally applying each version since. - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] pub async fn update(&mut self) -> Result<(), DeltaTableError> { match get_last_checkpoint(self.log_store.as_ref()).await { Ok(last_check_point) => { @@ -538,7 +534,7 @@ impl DeltaTable { } /// Updates the DeltaTable to the most recent state committed to the transaction log. - #[cfg(not(any(feature = "parquet", feature = "parquet2")))] + #[cfg(not(feature = "parquet"))] pub async fn update(&mut self) -> Result<(), DeltaTableError> { self.update_incremental(None).await } @@ -615,7 +611,7 @@ impl DeltaTable { } // 1. find latest checkpoint below version - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] match find_latest_check_point_for_version(self.log_store.as_ref(), version).await? { Some(check_point) => { self.restore_checkpoint(check_point).await?; @@ -940,8 +936,6 @@ mod tests { use super::*; use crate::kernel::{DataType, PrimitiveType, StructField}; use crate::operations::create::CreateBuilder; - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - use crate::table::builder::DeltaTableBuilder; #[tokio::test] async fn table_round_trip() { @@ -981,6 +975,7 @@ mod tests { drop(tmp_dir); } + /* TODO move into deltalake-aws crate #[cfg(any(feature = "s3", feature = "s3-native-tls"))] #[test] fn normalize_table_uri_s3() { @@ -992,10 +987,11 @@ mod tests { ] .iter() { - let table = DeltaTableBuilder::from_uri(table_uri).build().unwrap(); + let table = crate::DeltaTableBuilder::from_uri(table_uri).build().unwrap(); assert_eq!(table.table_uri(), "s3://tests/data/delta-0.8.0"); } } + */ #[test] fn get_table_constraints() { diff --git a/crates/deltalake-core/src/table/state.rs b/crates/deltalake-core/src/table/state.rs index fa9078997c..1aaf095d4f 100644 --- a/crates/deltalake-core/src/table/state.rs +++ b/crates/deltalake-core/src/table/state.rs @@ -20,7 +20,7 @@ use crate::storage::commit_uri_from_version; use crate::table::DeltaTableMetaData; use crate::DeltaTable; -#[cfg(any(feature = "parquet", feature = "parquet2"))] +#[cfg(feature = "parquet")] use super::{CheckPoint, DeltaTableConfig}; /// State snapshot currently held by the Delta Table instance. @@ -92,58 +92,34 @@ impl DeltaTableState { } /// Update DeltaTableState with checkpoint data. - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] pub fn process_checkpoint_bytes( &mut self, data: bytes::Bytes, table_config: &DeltaTableConfig, ) -> Result<(), DeltaTableError> { - #[cfg(feature = "parquet")] - { - use parquet::file::reader::{FileReader, SerializedFileReader}; - - let preader = SerializedFileReader::new(data)?; - let schema = preader.metadata().file_metadata().schema(); - if !schema.is_group() { - return Err(DeltaTableError::from(ProtocolError::Generic( - "Action record in checkpoint should be a struct".to_string(), - ))); - } - for record in preader.get_row_iter(None)? { - self.process_action( - Action::from_parquet_record(schema, &record.unwrap())?, - table_config.require_tombstones, - table_config.require_files, - )?; - } + use parquet::file::reader::{FileReader, SerializedFileReader}; + + let preader = SerializedFileReader::new(data)?; + let schema = preader.metadata().file_metadata().schema(); + if !schema.is_group() { + return Err(DeltaTableError::from(ProtocolError::Generic( + "Action record in checkpoint should be a struct".to_string(), + ))); } - - #[cfg(feature = "parquet2")] - { - use crate::protocol::parquet2_read::actions_from_row_group; - use parquet2::read::read_metadata; - - let mut reader = std::io::Cursor::new(data); - let metadata = read_metadata(&mut reader)?; - - for row_group in metadata.row_groups { - for action in - actions_from_row_group(row_group, &mut reader).map_err(ProtocolError::from)? - { - self.process_action( - action, - table_config.require_tombstones, - table_config.require_files, - )?; - } - } + for record in preader.get_row_iter(None)? { + self.process_action( + Action::from_parquet_record(schema, &record.unwrap())?, + table_config.require_tombstones, + table_config.require_files, + )?; } Ok(()) } /// Construct a delta table state object from checkpoint. - #[cfg(any(feature = "parquet", feature = "parquet2"))] + #[cfg(feature = "parquet")] pub async fn from_checkpoint( table: &DeltaTable, check_point: &CheckPoint, diff --git a/crates/deltalake-core/tests/checkpoint_writer.rs b/crates/deltalake-core/tests/checkpoint_writer.rs index ca6fbccf97..72b39b0878 100644 --- a/crates/deltalake-core/tests/checkpoint_writer.rs +++ b/crates/deltalake-core/tests/checkpoint_writer.rs @@ -14,7 +14,7 @@ mod simple_checkpoint { #[tokio::test] async fn simple_checkpoint_test() { - let table_location = "./tests/data/checkpoints"; + let table_location = "../deltalake-test/tests/data/checkpoints"; let table_path = PathBuf::from(table_location); let log_path = table_path.join("_delta_log"); @@ -97,7 +97,7 @@ mod delete_expired_delta_log_in_checkpoint { #[tokio::test] async fn test_delete_expired_logs() { let mut table = fs_common::create_table( - "./tests/data/checkpoints_with_expired_logs/expired", + "../deltalake-test/tests/data/checkpoints_with_expired_logs/expired", Some(hashmap! { DeltaConfigKey::LogRetentionDuration.as_ref().into() => Some("interval 10 minute".to_string()), DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into() => Some("true".to_string()) @@ -161,7 +161,7 @@ mod delete_expired_delta_log_in_checkpoint { #[tokio::test] async fn test_not_delete_expired_logs() { let mut table = fs_common::create_table( - "./tests/data/checkpoints_with_expired_logs/not_delete_expired", + "../deltalake-test/tests/data/checkpoints_with_expired_logs/not_delete_expired", Some(hashmap! { DeltaConfigKey::LogRetentionDuration.as_ref().into() => Some("interval 1 second".to_string()), DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into() => Some("false".to_string()) @@ -237,7 +237,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_expired_tombstones() { - let mut table = fs_common::create_table("./tests/data/checkpoints_tombstones/expired", Some(hashmap! { + let mut table = fs_common::create_table("../deltalake-test/tests/data/checkpoints_tombstones/expired", Some(hashmap! { DeltaConfigKey::DeletedFileRetentionDuration.as_ref().into() => Some("interval 1 minute".to_string()) })).await; @@ -274,7 +274,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_checkpoint_with_extended_file_metadata_true() { - let path = "./tests/data/checkpoints_tombstones/metadata_true"; + let path = "../deltalake-test/tests/data/checkpoints_tombstones/metadata_true"; let mut table = fs_common::create_table(path, None).await; let r1 = remove_metadata_true(); let r2 = remove_metadata_true(); @@ -290,7 +290,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_checkpoint_with_extended_file_metadata_false() { - let path = "./tests/data/checkpoints_tombstones/metadata_false"; + let path = "../deltalake-test/tests/data/checkpoints_tombstones/metadata_false"; let mut table = fs_common::create_table(path, None).await; let r1 = remove_metadata_true(); let r2 = remove_metadata_false(); @@ -313,7 +313,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_checkpoint_with_extended_file_metadata_broken() { - let path = "./tests/data/checkpoints_tombstones/metadata_broken"; + let path = "../deltalake-test/tests/data/checkpoints_tombstones/metadata_broken"; let mut table = fs_common::create_table(path, None).await; let r1 = remove_metadata_broken(); let r2 = remove_metadata_false(); diff --git a/crates/deltalake-core/tests/command_filesystem_check.rs b/crates/deltalake-core/tests/command_filesystem_check.rs index ac11c5d376..8d0eee6ac6 100644 --- a/crates/deltalake-core/tests/command_filesystem_check.rs +++ b/crates/deltalake-core/tests/command_filesystem_check.rs @@ -1,52 +1,19 @@ #![cfg(feature = "integration_test")] -use deltalake_core::test_utils::{ - set_env_if_not_set, IntegrationContext, StorageIntegration, TestResult, TestTables, -}; use deltalake_core::Path; use deltalake_core::{errors::DeltaTableError, DeltaOps}; +use deltalake_test::utils::*; use serial_test::serial; -mod common; - #[tokio::test] #[serial] async fn test_filesystem_check_local() -> TestResult { - test_filesystem_check(StorageIntegration::Local).await -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[tokio::test] -#[serial] -async fn test_filesystem_check_aws() -> TestResult { - set_env_if_not_set("AWS_S3_ALLOW_UNSAFE_RENAME", "true"); - set_env_if_not_set("AWS_S3_LOCKING_PROVIDER", "none"); - test_filesystem_check(StorageIntegration::Amazon).await -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_filesystem_check_azure() -> TestResult { - test_filesystem_check(StorageIntegration::Microsoft).await -} - -#[cfg(feature = "gcs")] -#[tokio::test] -#[serial] -async fn test_filesystem_check_gcp() -> TestResult { - test_filesystem_check(StorageIntegration::Google).await -} - -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn test_filesystem_check_hdfs() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Hdfs).await?) + let storage = Box::new(LocalStorageIntegration::default()); + let context = IntegrationContext::new(storage)?; + test_filesystem_check(&context).await } -async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(storage)?; +async fn test_filesystem_check(context: &IntegrationContext) -> TestResult { context.load_table(TestTables::Simple).await?; let file = "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"; let path = Path::from_iter([&TestTables::Simple.as_name(), file]); @@ -88,7 +55,7 @@ async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { #[tokio::test] #[serial] async fn test_filesystem_check_partitioned() -> TestResult { - let storage = StorageIntegration::Local; + let storage = Box::new(LocalStorageIntegration::default()); let context = IntegrationContext::new(storage)?; context .load_table(TestTables::Delta0_8_0Partitioned) @@ -122,7 +89,8 @@ async fn test_filesystem_check_partitioned() -> TestResult { #[serial] async fn test_filesystem_check_fails_for_concurrent_delete() -> TestResult { // Validate failure when a non dry only executes on the latest version - let context = IntegrationContext::new(StorageIntegration::Local)?; + let storage = Box::new(LocalStorageIntegration::default()); + let context = IntegrationContext::new(storage)?; context.load_table(TestTables::Simple).await?; let file = "part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet"; let path = Path::from_iter([&TestTables::Simple.as_name(), file]); @@ -144,34 +112,3 @@ async fn test_filesystem_check_fails_for_concurrent_delete() -> TestResult { Ok(()) } - -#[tokio::test] -#[serial] -#[ignore = "should this actually fail? with conflict resolution, we are re-trying again."] -async fn test_filesystem_check_outdated() -> TestResult { - // Validate failure when a non dry only executes on the latest version - let context = IntegrationContext::new(StorageIntegration::Local)?; - context.load_table(TestTables::Simple).await?; - let file = "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"; - let path = Path::from_iter([&TestTables::Simple.as_name(), file]); - - // Delete an active file from underlying storage without an update to the log to simulate an external fault - context.object_store().delete(&path).await?; - - let table = context - .table_builder(TestTables::Simple) - .with_version(2) - .load() - .await?; - - let op = DeltaOps::from(table); - let res = op.filesystem_check().with_dry_run(false).await; - println!("{:?}", res); - if let Err(DeltaTableError::VersionAlreadyExists(version)) = res { - assert!(version == 3); - } else { - panic!(); - } - - Ok(()) -} diff --git a/crates/deltalake-core/tests/command_vacuum.rs b/crates/deltalake-core/tests/command_vacuum.rs index 51ff3217b3..54ec03cdb2 100644 --- a/crates/deltalake-core/tests/command_vacuum.rs +++ b/crates/deltalake-core/tests/command_vacuum.rs @@ -1,15 +1,13 @@ use chrono::Duration; -use common::clock::TestClock; -use common::TestContext; use deltalake_core::kernel::StructType; use deltalake_core::operations::vacuum::Clock; use deltalake_core::operations::DeltaOps; +use deltalake_test::clock::TestClock; +use deltalake_test::*; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use serde_json::json; use std::sync::Arc; -mod common; - /// Basic schema pub fn get_xy_date_schema() -> StructType { serde_json::from_value(json!({ @@ -51,7 +49,7 @@ async fn test_non_partitioned_table() { ]; for path in paths { - common::add_file( + add_file( &mut table, &path, "random junk".as_bytes().into(), @@ -64,7 +62,7 @@ async fn test_non_partitioned_table() { clock.tick(Duration::seconds(10)); - common::remove_file( + remove_file( &mut table, "delete_me.parquet", &[], @@ -103,7 +101,7 @@ async fn test_partitioned_table() { let partition_values = [("date", Some("2022-07-03")), ("x", Some("2"))]; for path in paths { - common::add_file( + add_file( &mut table, &path, "random junk".as_bytes().into(), @@ -116,7 +114,7 @@ async fn test_partitioned_table() { clock.tick(Duration::seconds(10)); - common::remove_file( + remove_file( &mut table, "date=2022-07-03/x=2/delete_me.parquet", &partition_values, @@ -168,7 +166,7 @@ async fn test_partitions_included() { let partition_values = &[("_date", Some("2022-07-03"))]; for path in paths { - common::add_file( + add_file( &mut table, &path, "random junk".as_bytes().into(), @@ -181,7 +179,7 @@ async fn test_partitions_included() { clock.tick(Duration::seconds(10)); - common::remove_file( + remove_file( &mut table, "_date=2022-07-03/delete_me.parquet", partition_values, @@ -247,7 +245,7 @@ async fn test_non_managed_files() { ]; for path in paths_delete.iter().chain(paths_ignore.iter()) { - common::add_file( + add_file( &mut table, path, "random junk".as_bytes().into(), diff --git a/crates/deltalake-core/tests/common/adls.rs b/crates/deltalake-core/tests/common/adls.rs deleted file mode 100644 index 4c441e0325..0000000000 --- a/crates/deltalake-core/tests/common/adls.rs +++ /dev/null @@ -1,93 +0,0 @@ -use super::TestContext; -use chrono::Utc; -use rand::Rng; -use std::collections::HashMap; - -pub struct AzureGen2 { - #[allow(dead_code)] - account_name: String, - #[allow(dead_code)] - account_key: String, - file_system_name: String, -} - -impl Drop for AzureGen2 { - fn drop(&mut self) { - let file_system_name = self.file_system_name.clone(); - az_cli::delete_container(file_system_name).unwrap(); - } -} - -pub async fn setup_azure_gen2_context() -> TestContext { - let mut config = HashMap::new(); - - let storage_account_name = std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let storage_account_key = std::env::var("AZURE_STORAGE_ACCOUNT_KEY").unwrap(); - let storage_container_name = - std::env::var("AZURE_STORAGE_CONTAINER_NAME").unwrap_or_else(|_| "deltars".to_string()); - - let rand: u16 = rand::thread_rng().gen(); - let file_system_name = format!("delta-rs-test-{}-{}", Utc::now().timestamp(), rand); - - az_cli::create_container(&file_system_name).unwrap(); - - let table_uri = format!("azure://{file_system_name}/"); - - config.insert("URI".to_string(), table_uri); - config.insert( - "AZURE_STORAGE_ACCOUNT_NAME".to_string(), - storage_account_name.clone(), - ); - config.insert( - "AZURE_STORAGE_ACCOUNT_KEY".to_string(), - storage_account_key.clone(), - ); - config.insert( - "AZURE_STORAGE_CONTAINER_NAME".to_string(), - storage_container_name, - ); - - TestContext { - storage_context: Some(Box::new(AzureGen2 { - account_name: storage_account_name, - account_key: storage_account_key, - file_system_name, - })), - config, - ..TestContext::default() - } -} - -pub mod az_cli { - use std::process::{Command, ExitStatus}; - - /// Create a new bucket - pub fn create_container(container_name: impl AsRef) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "container", - "create", - "-n", - container_name.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } - - /// delete bucket - pub fn delete_container(container_name: impl AsRef) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "container", - "delete", - "-n", - container_name.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } -} diff --git a/crates/deltalake-core/tests/common/hdfs.rs b/crates/deltalake-core/tests/common/hdfs.rs deleted file mode 100644 index 8da5ef83b6..0000000000 --- a/crates/deltalake-core/tests/common/hdfs.rs +++ /dev/null @@ -1,20 +0,0 @@ -use super::TestContext; -use std::collections::HashMap; - -pub struct Hdfs { - name_node: String, -} - -pub fn setup_hdfs_context() -> TestContext { - let mut config = HashMap::new(); - - let name_node = "hdfs://localhost:9000".to_owned(); - - config.insert("URI".to_owned(), name_node.clone()); - - TestContext { - storage_context: Some(Box::new(Hdfs { name_node })), - config, - ..TestContext::default() - } -} diff --git a/crates/deltalake-core/tests/fs_common/mod.rs b/crates/deltalake-core/tests/fs_common/mod.rs index c79fc833da..ebd4d50b88 100644 --- a/crates/deltalake-core/tests/fs_common/mod.rs +++ b/crates/deltalake-core/tests/fs_common/mod.rs @@ -5,7 +5,6 @@ use deltalake_core::kernel::{ use deltalake_core::operations::create::CreateBuilder; use deltalake_core::operations::transaction::commit; use deltalake_core::protocol::{DeltaOperation, SaveMode}; -use deltalake_core::storage::config::configure_store; use deltalake_core::storage::{GetResult, ObjectStoreResult}; use deltalake_core::DeltaTable; use object_store::path::Path as StorePath; @@ -37,7 +36,7 @@ pub async fn create_table_from_json( partition_columns: Vec<&str>, config: Value, ) -> DeltaTable { - assert!(path.starts_with("./tests/data")); + assert!(path.starts_with("../deltalake-test/tests/data")); std::fs::create_dir_all(path).unwrap(); std::fs::remove_dir_all(path).unwrap(); std::fs::create_dir_all(path).unwrap(); @@ -144,15 +143,13 @@ impl std::fmt::Display for SlowStore { } } -#[allow(dead_code)] impl SlowStore { pub fn new( location: Url, - options: impl Into + Clone, + _options: impl Into + Clone, ) -> deltalake_core::DeltaResult { - let mut options = options.into(); Ok(Self { - inner: configure_store(&location, &mut options).unwrap(), + inner: deltalake_core::storage::store_for(&location)?, }) } } diff --git a/crates/deltalake-core/tests/integration_checkpoint.rs b/crates/deltalake-core/tests/integration_checkpoint.rs index 768b1172db..ce4525ba83 100644 --- a/crates/deltalake-core/tests/integration_checkpoint.rs +++ b/crates/deltalake-core/tests/integration_checkpoint.rs @@ -3,9 +3,9 @@ use chrono::Utc; use deltalake_core::checkpoints::{cleanup_expired_logs_for, create_checkpoint}; use deltalake_core::kernel::{DataType, PrimitiveType}; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult}; use deltalake_core::writer::{DeltaWriter, JsonWriter}; use deltalake_core::{errors::DeltaResult, DeltaOps, DeltaTableBuilder, ObjectStore}; +use deltalake_test::utils::*; use object_store::path::Path; use serde_json::json; use serial_test::serial; @@ -15,43 +15,8 @@ use tokio::time::sleep; #[tokio::test] #[serial] async fn cleanup_metadata_fs_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Local)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[tokio::test] -#[serial] -async fn cleanup_metadata_aws_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn cleanup_metadata_azure_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Microsoft)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(feature = "gcs")] -#[tokio::test] -#[serial] -async fn cleanup_metadata_gcp_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Google)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn cleanup_metadata_hdfs_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Hdfs)?; + let storage = Box::new(LocalStorageIntegration::default()); + let context = IntegrationContext::new(storage)?; cleanup_metadata_test(&context).await?; Ok(()) } diff --git a/crates/deltalake-core/tests/integration_concurrent_writes.rs b/crates/deltalake-core/tests/integration_concurrent_writes.rs index f57167f2c1..4e66a9f93f 100644 --- a/crates/deltalake-core/tests/integration_concurrent_writes.rs +++ b/crates/deltalake-core/tests/integration_concurrent_writes.rs @@ -1,11 +1,13 @@ #![cfg(feature = "integration_test")] +use log::*; + use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; use deltalake_core::operations::transaction::commit; use deltalake_core::operations::DeltaOps; use deltalake_core::protocol::{DeltaOperation, SaveMode}; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; use deltalake_core::{DeltaTable, DeltaTableBuilder}; +use deltalake_test::utils::*; use serial_test::serial; use std::collections::HashMap; use std::future::Future; @@ -15,38 +17,13 @@ use std::time::Duration; #[tokio::test] #[serial] async fn test_concurrent_writes_local() -> TestResult { - test_concurrent_writes(StorageIntegration::Local).await?; - Ok(()) -} - -#[cfg(feature = "s3")] -#[tokio::test] -#[serial] -async fn concurrent_writes_s3() -> TestResult { - test_concurrent_writes(StorageIntegration::Amazon).await?; - Ok(()) -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_concurrent_writes_azure() -> TestResult { - test_concurrent_writes(StorageIntegration::Microsoft).await?; - Ok(()) -} - -// tracked via https://github.com/datafusion-contrib/datafusion-objectstore-hdfs/issues/13 -#[ignore] -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn test_concurrent_writes_hdfs() -> TestResult { - test_concurrent_writes(StorageIntegration::Hdfs).await?; + let storage = Box::new(LocalStorageIntegration::default()); + let context = IntegrationContext::new(storage)?; + test_concurrent_writes(&context).await?; Ok(()) } -async fn test_concurrent_writes(integration: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(integration)?; +async fn test_concurrent_writes(context: &IntegrationContext) -> TestResult { let (_table, table_uri) = prepare_table(&context).await?; run_test(|name| Worker::new(&table_uri, name)).await; Ok(()) diff --git a/crates/deltalake-core/tests/integration_datafusion.rs b/crates/deltalake-core/tests/integration_datafusion.rs index 51be591071..25a3fddbce 100644 --- a/crates/deltalake-core/tests/integration_datafusion.rs +++ b/crates/deltalake-core/tests/integration_datafusion.rs @@ -1,8 +1,8 @@ #![cfg(all(feature = "integration_test", feature = "datafusion"))] use arrow::array::Int64Array; -use common::datafusion::context_with_delta_table_factory; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; +use deltalake_test::datafusion::*; +use deltalake_test::utils::*; use serial_test::serial; use std::collections::{HashMap, HashSet}; @@ -32,19 +32,17 @@ use url::Url; use deltalake_core::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; use deltalake_core::kernel::{DataType, MapType, PrimitiveType, StructField, StructType}; +use deltalake_core::logstore::logstore_for; use deltalake_core::operations::create::CreateBuilder; use deltalake_core::protocol::SaveMode; use deltalake_core::writer::{DeltaWriter, RecordBatchWriter}; use deltalake_core::{ open_table, operations::{write::WriteBuilder, DeltaOps}, - storage::config::configure_log_store, DeltaTable, DeltaTableError, }; use std::error::Error; -mod common; - mod local { use datafusion::common::stats::Precision; use deltalake_core::writer::JsonWriter; @@ -53,7 +51,9 @@ mod local { #[tokio::test] #[serial] async fn test_datafusion_local() -> TestResult { - test_datafusion(StorageIntegration::Local).await + let storage = Box::new(LocalStorageIntegration::default()); + let context = IntegrationContext::new(storage)?; + test_datafusion(&context).await } fn get_scanned_files(node: &dyn ExecutionPlan) -> HashSet