From a811da1057cce832683ee8305ccba9a7c8863a83 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 12 Sep 2025 15:28:17 -0700 Subject: [PATCH 01/12] Make Storage a trait --- Cargo.lock | 51 +++ Cargo.toml | 1 + crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/arrow/reader.rs | 12 +- crates/iceberg/src/io/file_io.rs | 457 ++++++++++++++++++---- crates/iceberg/src/io/mod.rs | 8 +- crates/iceberg/src/io/storage.rs | 498 ++++++++++++++---------- crates/iceberg/src/io/storage_azdls.rs | 124 +++++- crates/iceberg/src/io/storage_fs.rs | 150 +++++++ crates/iceberg/src/io/storage_gcs.rs | 114 +++++- crates/iceberg/src/io/storage_memory.rs | 207 +++++++++- crates/iceberg/src/io/storage_oss.rs | 112 ++++++ crates/iceberg/src/io/storage_s3.rs | 191 ++++++++- crates/iceberg/src/puffin/metadata.rs | 11 +- crates/iceberg/src/puffin/reader.rs | 2 +- 15 files changed, 1627 insertions(+), 312 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2d464709fd..4d1fa9bd43 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2632,6 +2632,17 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" +[[package]] +name = "erased-serde" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89e8918065695684b2b0702da20382d5ae6065cf3327bc2d6436bd49a71ce9f3" +dependencies = [ + "serde", + "serde_core", + "typeid", +] + [[package]] name = "errno" version = "0.3.14" @@ -3398,6 +3409,7 @@ dependencies = [ "tempfile", "tokio", "typed-builder", + "typetag", "url", "uuid", "zstd", @@ -3786,6 +3798,15 @@ dependencies = [ "tokio", ] +[[package]] +name = "inventory" +version = "0.3.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc61209c082fbeb19919bee74b176221b27223e27b65d781eb91af24eb1fb46e" +dependencies = [ + "rustversion", +] + [[package]] name = "ipnet" version = "2.11.0" @@ -7110,12 +7131,42 @@ dependencies = [ "syn 2.0.111", ] +[[package]] +name = "typeid" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc7d623258602320d5c55d1bc22793b57daff0ec7efc270ea7d55ce1d5f5471c" + [[package]] name = "typenum" version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" +[[package]] +name = "typetag" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be2212c8a9b9bcfca32024de14998494cf9a5dfa59ea1b829de98bac374b86bf" +dependencies = [ + "erased-serde", + "inventory", + "once_cell", + "serde", + "typetag-impl", +] + +[[package]] +name = "typetag-impl" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "27a7a9b72ba121f6f1f6c3632b85604cac41aedb5ddc70accbebb6cac83de846" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.111", +] + [[package]] name = "typify" version = "0.5.0" diff --git a/Cargo.toml b/Cargo.toml index ded3aedecb..c56c808f1c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -127,6 +127,7 @@ toml = "0.8" tracing = "0.1.41" tracing-subscriber = "0.3.20" typed-builder = "0.20" +typetag = "0.2" url = "2.5.7" uuid = { version = "1.18", features = ["v7"] } volo = "0.10.6" diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 6f1332a444..9a5de7736d 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -86,6 +86,7 @@ serde_with = { workspace = true } strum = { workspace = true, features = ["derive"] } tokio = { workspace = true, optional = false, features = ["sync"] } typed-builder = { workspace = true } +typetag = { workspace = true } url = { workspace = true } uuid = { workspace = true } zstd = { workspace = true } diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 6209c1e261..1f0f70094d 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -447,7 +447,7 @@ impl ArrowReader { file_io: FileIO, should_load_page_index: bool, arrow_reader_options: Option, - ) -> Result>> { + ) -> Result> { // Get the metadata for the Parquet file we need to read and build // a reader for the data within let parquet_file = file_io.new_input(data_file_path)?; @@ -1646,18 +1646,18 @@ impl BoundPredicateVisitor for PredicateConverter<'_> { } /// ArrowFileReader is a wrapper around a FileRead that impls parquets AsyncFileReader. -pub struct ArrowFileReader { +pub struct ArrowFileReader { meta: FileMetadata, preload_column_index: bool, preload_offset_index: bool, preload_page_index: bool, metadata_size_hint: Option, - r: R, + r: Box, } -impl ArrowFileReader { +impl ArrowFileReader { /// Create a new ArrowFileReader - pub fn new(meta: FileMetadata, r: R) -> Self { + pub fn new(meta: FileMetadata, r: Box) -> Self { Self { meta, preload_column_index: false, @@ -1696,7 +1696,7 @@ impl ArrowFileReader { } } -impl AsyncFileReader for ArrowFileReader { +impl AsyncFileReader for ArrowFileReader { fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { Box::pin( self.r diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 24e91ca8a4..56f1d4bb45 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -17,14 +17,17 @@ use std::any::{Any, TypeId}; use std::collections::HashMap; +use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; +use async_trait::async_trait; use bytes::Bytes; -use opendal::Operator; use url::Url; -use super::storage::Storage; +// Re-export traits from storage module +pub use super::storage::{Storage, StorageFactory, StorageRegistry}; +use crate::io::STORAGE_LOCATION_SCHEME; use crate::{Error, ErrorKind, Result}; /// FileIO implementation, used to manipulate files in underlying storage. @@ -48,7 +51,7 @@ use crate::{Error, ErrorKind, Result}; pub struct FileIO { builder: FileIOBuilder, - inner: Arc, + inner: Arc, } impl FileIO { @@ -89,8 +92,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub async fn delete(&self, path: impl AsRef) -> Result<()> { - let (op, relative_path) = self.inner.create_operator(&path)?; - Ok(op.delete(relative_path).await?) + self.inner.delete(path.as_ref()).await } /// Remove the path and all nested dirs and files recursively. @@ -104,14 +106,8 @@ impl FileIO { /// - If the path is a file or not exist, this function will be no-op. /// - If the path is a empty directory, this function will remove the directory itself. /// - If the path is a non-empty directory, this function will remove the directory and all nested files and directories. - pub async fn remove_dir_all(&self, path: impl AsRef) -> Result<()> { - let (op, relative_path) = self.inner.create_operator(&path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) + pub async fn delete_prefix(&self, path: impl AsRef) -> Result<()> { + self.inner.delete_prefix(path.as_ref()).await } /// Check file exists. @@ -120,8 +116,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub async fn exists(&self, path: impl AsRef) -> Result { - let (op, relative_path) = self.inner.create_operator(&path)?; - Ok(op.exists(relative_path).await?) + self.inner.exists(path.as_ref()).await } /// Creates input file. @@ -130,14 +125,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub fn new_input(&self, path: impl AsRef) -> Result { - let (op, relative_path) = self.inner.create_operator(&path)?; - let path = path.as_ref().to_string(); - let relative_path_pos = path.len() - relative_path.len(); - Ok(InputFile { - op, - path, - relative_path_pos, - }) + self.inner.new_input(path.as_ref()) } /// Creates output file. @@ -146,14 +134,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub fn new_output(&self, path: impl AsRef) -> Result { - let (op, relative_path) = self.inner.create_operator(&path)?; - let path = path.as_ref().to_string(); - let relative_path_pos = path.len() - relative_path.len(); - Ok(OutputFile { - op, - path, - relative_path_pos, - }) + self.inner.new_output(path.as_ref()) } } @@ -183,6 +164,29 @@ impl Extensions { } /// Builder for [`FileIO`]. +/// +/// # Custom Storage Implementations +/// +/// You can use custom storage implementations by creating a custom +/// [`StorageRegistry`] and registering your storage factory: +/// +/// ```rust,ignore +/// use iceberg::io::{StorageRegistry, StorageFactory, FileIOBuilder}; +/// use std::sync::Arc; +/// +/// // Create your custom storage factory +/// let my_factory = Arc::new(MyCustomStorageFactory); +/// +/// // Register it with a custom scheme +/// let mut registry = StorageRegistry::new(); +/// registry.register("mycustom", my_factory); +/// +/// // Use it to build FileIO +/// let file_io = FileIOBuilder::new("mycustom") +/// .with_prop("key", "value") +/// .with_registry(registry) +/// .build()?; +/// ``` #[derive(Clone, Debug)] pub struct FileIOBuilder { /// This is used to infer scheme of operator. @@ -193,6 +197,8 @@ pub struct FileIOBuilder { props: HashMap, /// Optional extensions to configure the underlying FileIO behavior. extensions: Extensions, + /// Optional custom registry. If None, a default registry will be created. + registry: Option, } impl FileIOBuilder { @@ -203,6 +209,7 @@ impl FileIOBuilder { scheme_str: Some(scheme_str.to_string()), props: HashMap::default(), extensions: Extensions::default(), + registry: None, } } @@ -212,17 +219,26 @@ impl FileIOBuilder { scheme_str: None, props: HashMap::default(), extensions: Extensions::default(), + registry: None, } } /// Fetch the scheme string. /// /// The scheme_str will be empty if it's None. - pub fn into_parts(self) -> (String, HashMap, Extensions) { + pub fn into_parts( + self, + ) -> ( + String, + HashMap, + Extensions, + Option, + ) { ( self.scheme_str.unwrap_or_default(), self.props, self.extensions, + self.registry, ) } @@ -260,12 +276,49 @@ impl FileIOBuilder { self.extensions.get::() } + /// Sets a custom storage registry. + /// + /// This allows you to register custom storage implementations that can be used + /// when building the FileIO. If not set, a default registry with built-in + /// storage types will be used. + /// + /// # Example + /// + /// ```rust,ignore + /// use iceberg::io::{StorageRegistry, FileIOBuilder}; + /// use std::sync::Arc; + /// + /// let mut registry = StorageRegistry::new(); + /// registry.register("mycustom", Arc::new(MyCustomStorageFactory)); + /// + /// let file_io = FileIOBuilder::new("mycustom") + /// .with_registry(registry) + /// .build()?; + /// ``` + pub fn with_registry(mut self, registry: StorageRegistry) -> Self { + self.registry = Some(registry); + self + } + /// Builds [`FileIO`]. pub fn build(self) -> Result { - let storage = Storage::build(self.clone())?; + // Use the scheme to determine the storage type + let scheme = self.scheme_str.clone().unwrap_or_default(); + + // Use custom registry if provided, otherwise create default + let registry = self.registry.clone().unwrap_or_default(); + + let factory = registry.get_factory(scheme.as_str())?; + + let mut props_with_scheme = self.props.clone(); + props_with_scheme.insert(STORAGE_LOCATION_SCHEME.to_string(), scheme); + + // Build storage with props and extensions + let storage = factory.build(props_with_scheme, self.extensions.clone())?; + Ok(FileIO { builder: self, - inner: Arc::new(storage), + inner: storage, }) } } @@ -293,7 +346,7 @@ pub trait FileRead: Send + Sync + Unpin + 'static { #[async_trait::async_trait] impl FileRead for opendal::Reader { - async fn read(&self, range: Range) -> crate::Result { + async fn read(&self, range: Range) -> Result { Ok(opendal::Reader::read(self, range).await?.to_bytes()) } } @@ -301,49 +354,53 @@ impl FileRead for opendal::Reader { /// Input file is used for reading from files. #[derive(Debug)] pub struct InputFile { - op: Operator, - // Absolution path of file. + storage: Arc, path: String, - // Relative path of file to uri, starts at [`relative_path_pos`] - relative_path_pos: usize, } impl InputFile { + /// Creates a new input file. + /// + /// # Arguments + /// + /// * `storage` - The storage backend to use + /// * `path` - Absolute path to the file + pub fn new(storage: Arc, path: String) -> Self { + Self { storage, path } + } + + /// Returns the storage backend for this input file. + pub fn storage(&self) -> &Arc { + &self.storage + } + /// Absolute path to root uri. pub fn location(&self) -> &str { &self.path } /// Check if file exists. - pub async fn exists(&self) -> crate::Result { - Ok(self.op.exists(&self.path[self.relative_path_pos..]).await?) + pub async fn exists(&self) -> Result { + self.storage.exists(&self.path).await } /// Fetch and returns metadata of file. - pub async fn metadata(&self) -> crate::Result { - let meta = self.op.stat(&self.path[self.relative_path_pos..]).await?; - - Ok(FileMetadata { - size: meta.content_length(), - }) + pub async fn metadata(&self) -> Result { + self.storage.metadata(&self.path).await } /// Read and returns whole content of file. /// /// For continuous reading, use [`Self::reader`] instead. - pub async fn read(&self) -> crate::Result { - Ok(self - .op - .read(&self.path[self.relative_path_pos..]) - .await? - .to_bytes()) + pub async fn read(&self) -> Result { + self.storage.read(&self.path).await } /// Creates [`FileRead`] for continuous reading. /// /// For one-time reading, use [`Self::read`] instead. - pub async fn reader(&self) -> crate::Result> { - Ok(self.op.reader(&self.path[self.relative_path_pos..]).await?) + pub async fn reader(&self) -> Result> { + self.storage.reader(&self.path).await } } @@ -353,17 +410,17 @@ impl InputFile { /// /// It's possible for us to remove the async_trait, but we need to figure /// out how to handle the object safety. -#[async_trait::async_trait] -pub trait FileWrite: Send + Unpin + 'static { +#[async_trait] +pub trait FileWrite: Send + Sync + Unpin + 'static { /// Write bytes to file. /// /// TODO: we can support writing non-contiguous bytes in the future. - async fn write(&mut self, bs: Bytes) -> crate::Result<()>; + async fn write(&mut self, bs: Bytes) -> Result<()>; /// Close file. /// /// Calling close on closed file will generate an error. - async fn close(&mut self) -> crate::Result<()>; + async fn close(&mut self) -> Result<()>; } #[async_trait::async_trait] @@ -389,17 +446,29 @@ impl FileWrite for Box { } } -/// Output file is used for writing to files.. +/// Output file is used for writing to files. #[derive(Debug)] pub struct OutputFile { - op: Operator, - // Absolution path of file. + storage: Arc, path: String, - // Relative path of file to uri, starts at [`relative_path_pos`] - relative_path_pos: usize, } impl OutputFile { + /// Creates a new output file. + /// + /// # Arguments + /// + /// * `storage` - The storage backend to use + /// * `path` - Absolute path to the file + pub fn new(storage: Arc, path: String) -> Self { + Self { storage, path } + } + + /// Returns the storage backend for this output file. + pub fn storage(&self) -> &Arc { + &self.storage + } + /// Relative path to root uri. pub fn location(&self) -> &str { &self.path @@ -407,23 +476,19 @@ impl OutputFile { /// Checks if file exists. pub async fn exists(&self) -> Result { - Ok(self.op.exists(&self.path[self.relative_path_pos..]).await?) + self.storage.exists(&self.path).await } /// Deletes file. /// /// If the file does not exist, it will not return error. pub async fn delete(&self) -> Result<()> { - Ok(self.op.delete(&self.path[self.relative_path_pos..]).await?) + self.storage.delete(&self.path).await } /// Converts into [`InputFile`]. pub fn to_input_file(self) -> InputFile { - InputFile { - op: self.op, - path: self.path, - relative_path_pos: self.relative_path_pos, - } + InputFile::new(self.storage, self.path) } /// Create a new output file with given bytes. @@ -433,9 +498,7 @@ impl OutputFile { /// Calling `write` will overwrite the file if it exists. /// For continuous writing, use [`Self::writer`]. pub async fn write(&self, bs: Bytes) -> crate::Result<()> { - let mut writer = self.writer().await?; - writer.write(bs).await?; - writer.close().await + self.storage.write(self.path.as_str(), bs).await } /// Creates output file for continuous writing. @@ -444,24 +507,143 @@ impl OutputFile { /// /// For one-time writing, use [`Self::write`] instead. pub async fn writer(&self) -> crate::Result> { - Ok(Box::new( - self.op.writer(&self.path[self.relative_path_pos..]).await?, - )) + Ok(Box::new(self.storage.writer(&self.path).await?)) } } #[cfg(test)] mod tests { + use std::collections::HashMap; use std::fs::{File, create_dir_all}; use std::io::Write; use std::path::Path; + use std::sync::{Arc, Mutex, MutexGuard}; + use async_trait::async_trait; use bytes::Bytes; use futures::AsyncReadExt; use futures::io::AllowStdIo; + use serde::{Deserialize, Serialize}; use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; + use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, + STORAGE_LOCATION_SCHEME, Storage, StorageFactory, StorageRegistry, + }; + use crate::{Error, ErrorKind, Result}; + + // Test storage implementation that tracks write operations + #[derive(Debug, Clone, Serialize, Deserialize)] + struct TestStorage { + #[serde(skip, default = "default_written")] + written: Arc>>, + received_props: HashMap, + } + + fn default_written() -> Arc>> { + Arc::new(Mutex::new(Vec::new())) + } + + #[allow(dead_code)] + impl TestStorage { + pub fn written(&self) -> MutexGuard<'_, Vec> { + self.written.lock().unwrap() + } + + pub fn received_props(&self) -> &HashMap { + &self.received_props + } + } + + #[async_trait] + #[typetag::serde] + impl Storage for TestStorage { + async fn exists(&self, _path: &str) -> Result { + Ok(true) + } + + async fn metadata(&self, _path: &str) -> Result { + Ok(FileMetadata { size: 42 }) + } + + async fn read(&self, _path: &str) -> Result { + Ok(Bytes::from("test data")) + } + + async fn reader(&self, _path: &str) -> Result> { + Err(Error::new( + ErrorKind::FeatureUnsupported, + "TestStorage does not support reader", + )) + } + + async fn write(&self, path: &str, _bs: Bytes) -> Result<()> { + self.written.lock().unwrap().push(path.to_string()); + Ok(()) + } + + async fn writer(&self, _path: &str) -> Result> { + Err(Error::new( + ErrorKind::FeatureUnsupported, + "TestStorage does not support writer", + )) + } + + async fn delete(&self, _path: &str) -> Result<()> { + Ok(()) + } + + async fn delete_prefix(&self, _path: &str) -> Result<()> { + Ok(()) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } + } + + fn default_received_props() -> Arc>> { + Arc::new(Mutex::new(HashMap::new())) + } + + // Test storage factory + #[derive(Debug, Serialize, Deserialize)] + struct TestStorageFactory { + #[serde(skip, default = "default_written")] + written: Arc>>, + #[serde(skip, default = "default_received_props")] + received_props: Arc>>, + } + + impl TestStorageFactory { + pub fn written(&self) -> MutexGuard<'_, Vec> { + self.written.lock().unwrap() + } + + pub fn received_props(&self) -> MutexGuard<'_, HashMap> { + self.received_props.lock().unwrap() + } + } + + #[typetag::serde] + impl StorageFactory for TestStorageFactory { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + *self.received_props.lock().unwrap() = props.clone(); + Ok(Arc::new(TestStorage { + written: self.written.clone(), + received_props: props, + })) + } + } fn create_local_file_io() -> FileIO { FileIOBuilder::new_fs_io().build().unwrap() @@ -517,14 +699,14 @@ mod tests { assert!(file_io.exists(&a_path).await.unwrap()); // Remove a file should be no-op. - file_io.remove_dir_all(&a_path).await.unwrap(); + file_io.delete_prefix(&a_path).await.unwrap(); assert!(file_io.exists(&a_path).await.unwrap()); // Remove a not exist dir should be no-op. - file_io.remove_dir_all("not_exists/").await.unwrap(); + file_io.delete_prefix("not_exists/").await.unwrap(); // Remove a dir should remove all files in it. - file_io.remove_dir_all(&sub_dir_path).await.unwrap(); + file_io.delete_prefix(&sub_dir_path).await.unwrap(); assert!(!file_io.exists(&b_path).await.unwrap()); assert!(!file_io.exists(&c_path).await.unwrap()); assert!(file_io.exists(&a_path).await.unwrap()); @@ -543,7 +725,7 @@ mod tests { let file_io = create_local_file_io(); assert!(!file_io.exists(&full_path).await.unwrap()); assert!(file_io.delete(&full_path).await.is_ok()); - assert!(file_io.remove_dir_all(&full_path).await.is_ok()); + assert!(file_io.delete_prefix(&full_path).await.is_ok()); } #[tokio::test] @@ -605,4 +787,117 @@ mod tests { io.delete(&path).await.unwrap(); assert!(!io.exists(&path).await.unwrap()); } + + #[test] + fn test_custom_registry() { + // Create a custom registry and register test storage + let factory = Arc::new(TestStorageFactory { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageRegistry::new(); + registry.register("test", factory.clone()); + + // Build FileIO with custom storage + let file_io = FileIOBuilder::new("test") + .with_registry(registry) + .build() + .unwrap(); + + // Verify we can create files with the custom storage + assert!(file_io.new_output("test://test.txt").is_ok()); + assert!(file_io.new_input("test://test.txt").is_ok()); + } + + #[tokio::test] + async fn test_custom_registry_operations() { + // Create test storage with write tracking + let factory = Arc::new(TestStorageFactory { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageRegistry::new(); + registry.register("test", factory.clone()); + + // Build FileIO with test storage + let file_io = FileIOBuilder::new("test") + .with_registry(registry) + .build() + .unwrap(); + + // Perform operations + let output = file_io.new_output("test://bucket/file.txt").unwrap(); + output.write(Bytes::from("test")).await.unwrap(); + + let input = file_io.new_input("test://bucket/file.txt").unwrap(); + let data = input.read().await.unwrap(); + assert_eq!(data, Bytes::from("test data")); + + let metadata = input.metadata().await.unwrap(); + assert_eq!(metadata.size, 42); + + // Verify write was tracked + let tracked = factory.written(); + assert_eq!(tracked.len(), 1); + assert_eq!(tracked[0], "test://bucket/file.txt"); + } + + #[test] + fn test_scheme_and_props_propagation() { + // Create test storage that captures props + let factory = Arc::new(TestStorageFactory { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageRegistry::new(); + registry.register("myscheme", factory.clone()); + + // Build FileIO with custom scheme and additional props + let file_io = FileIOBuilder::new("myscheme") + .with_prop("custom.prop", "custom_value") + .with_registry(registry) + .build() + .unwrap(); + + // Verify the storage was created + assert!(file_io.new_output("myscheme://test.txt").is_ok()); + + // Verify the scheme was propagated to the factory + let props = factory.received_props(); + assert_eq!( + props.get(STORAGE_LOCATION_SCHEME), + Some(&"myscheme".to_string()) + ); + // Verify custom props were also passed + assert_eq!(props.get("custom.prop"), Some(&"custom_value".to_string())); + } + + #[test] + fn test_into_parts_includes_registry() { + let registry = StorageRegistry::new(); + + let builder = FileIOBuilder::new("memory") + .with_prop("key", "value") + .with_registry(registry.clone()); + + let (scheme, props, _extensions, returned_registry) = builder.into_parts(); + + assert_eq!(scheme, "memory"); + assert_eq!(props.get("key"), Some(&"value".to_string())); + assert!(returned_registry.is_some()); + } + + #[test] + fn test_into_parts_without_registry() { + let builder = FileIOBuilder::new("memory").with_prop("key", "value"); + + let (scheme, props, _extensions, returned_registry) = builder.into_parts(); + + assert_eq!(scheme, "memory"); + assert_eq!(props.get("key"), Some(&"value".to_string())); + assert!(returned_registry.is_none()); + } } diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 5eb5964345..b16db18563 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -70,8 +70,12 @@ mod file_io; mod storage; pub use file_io::*; +pub use storage::{Storage, StorageFactory, StorageRegistry}; pub(crate) mod object_cache; +/// Property key used to pass the scheme string from FileIOBuilder to StorageBuilder. +pub const STORAGE_LOCATION_SCHEME: &str = "iceberg.storage.location.scheme"; + #[cfg(feature = "storage-azdls")] mod storage_azdls; #[cfg(feature = "storage-fs")] @@ -87,12 +91,8 @@ mod storage_s3; #[cfg(feature = "storage-azdls")] pub use storage_azdls::*; -#[cfg(feature = "storage-fs")] -use storage_fs::*; #[cfg(feature = "storage-gcs")] pub use storage_gcs::*; -#[cfg(feature = "storage-memory")] -use storage_memory::*; #[cfg(feature = "storage-oss")] pub use storage_oss::*; #[cfg(feature = "storage-s3")] diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 03e43600dd..f2bdb8943d 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,228 +15,306 @@ // specific language governing permissions and limitations // under the License. -#[cfg(any( - feature = "storage-s3", - feature = "storage-gcs", - feature = "storage-oss", - feature = "storage-azdls", -))] +//! Storage traits and implementations for Iceberg. +//! +//! This module provides the core storage abstraction used throughout Iceberg Rust. +//! Storage implementations handle reading and writing files across different backends +//! (S3, GCS, Azure, local filesystem, etc.). + +use std::collections::HashMap; +use std::fmt::Debug; use std::sync::Arc; -use opendal::layers::RetryLayer; -#[cfg(feature = "storage-azdls")] -use opendal::services::AzdlsConfig; -#[cfg(feature = "storage-gcs")] -use opendal::services::GcsConfig; -#[cfg(feature = "storage-oss")] -use opendal::services::OssConfig; -#[cfg(feature = "storage-s3")] -use opendal::services::S3Config; -use opendal::{Operator, Scheme}; - -#[cfg(feature = "storage-azdls")] -use super::AzureStorageScheme; -use super::FileIOBuilder; -#[cfg(feature = "storage-s3")] -use crate::io::CustomAwsCredentialLoader; -use crate::{Error, ErrorKind}; - -/// The storage carries all supported storage services in iceberg -#[derive(Debug)] -pub(crate) enum Storage { - #[cfg(feature = "storage-memory")] - Memory(Operator), - #[cfg(feature = "storage-fs")] - LocalFs, - /// Expects paths of the form `s3[a]:///`. - #[cfg(feature = "storage-s3")] - S3 { - /// s3 storage could have `s3://` and `s3a://`. - /// Storing the scheme string here to return the correct path. - configured_scheme: String, - config: Arc, - customized_credential_load: Option, - }, - #[cfg(feature = "storage-gcs")] - Gcs { config: Arc }, - #[cfg(feature = "storage-oss")] - Oss { config: Arc }, - /// Expects paths of the form - /// `abfs[s]://@.dfs./` or - /// `wasb[s]://@.blob./`. - #[cfg(feature = "storage-azdls")] - Azdls { - /// Because Azdls accepts multiple possible schemes, we store the full - /// passed scheme here to later validate schemes passed via paths. - configured_scheme: AzureStorageScheme, - config: Arc, - }, -} +use async_trait::async_trait; +use bytes::Bytes; -impl Storage { - /// Convert iceberg config to opendal config. - pub(crate) fn build(file_io_builder: FileIOBuilder) -> crate::Result { - let (scheme_str, props, extensions) = file_io_builder.into_parts(); - let _ = (&props, &extensions); - let scheme = Self::parse_scheme(&scheme_str)?; - - match scheme { - #[cfg(feature = "storage-memory")] - Scheme::Memory => Ok(Self::Memory(super::memory_config_build()?)), - #[cfg(feature = "storage-fs")] - Scheme::Fs => Ok(Self::LocalFs), - #[cfg(feature = "storage-s3")] - Scheme::S3 => Ok(Self::S3 { - configured_scheme: scheme_str, - config: super::s3_config_parse(props)?.into(), - customized_credential_load: extensions - .get::() - .map(Arc::unwrap_or_clone), - }), - #[cfg(feature = "storage-gcs")] - Scheme::Gcs => Ok(Self::Gcs { - config: super::gcs_config_parse(props)?.into(), - }), - #[cfg(feature = "storage-oss")] - Scheme::Oss => Ok(Self::Oss { - config: super::oss_config_parse(props)?.into(), - }), - #[cfg(feature = "storage-azdls")] - Scheme::Azdls => { - let scheme = scheme_str.parse::()?; - Ok(Self::Azdls { - config: super::azdls_config_parse(props)?.into(), - configured_scheme: scheme, - }) - } - // Update doc on [`FileIO`] when adding new schemes. - _ => Err(Error::new( - ErrorKind::FeatureUnsupported, - format!("Constructing file io from scheme: {scheme} not supported now",), - )), - } - } +use super::{Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile}; +use crate::Result; + +/// Trait for storage operations in Iceberg. +/// +/// This trait defines the interface for all storage backends. Implementations +/// provide access to different storage systems like S3, GCS, Azure, local filesystem, etc. +/// +/// The trait supports serialization via `typetag`, allowing storage instances to be +/// serialized and deserialized across process boundaries. +/// +/// # Example +/// +/// ```rust,ignore +/// use iceberg::io::Storage; +/// +/// async fn example(storage: Arc) -> Result<()> { +/// // Check if file exists +/// if storage.exists("s3://bucket/path/file.parquet").await? { +/// // Read file +/// let data = storage.read("s3://bucket/path/file.parquet").await?; +/// } +/// Ok(()) +/// } +/// ``` +#[async_trait] +#[typetag::serde(tag = "type")] +pub trait Storage: Debug + Send + Sync { + /// Check if a file exists at the given path + async fn exists(&self, path: &str) -> Result; + + /// Get metadata from an input path + async fn metadata(&self, path: &str) -> Result; + + /// Read bytes from a path + async fn read(&self, path: &str) -> Result; + + /// Get FileRead from a path + async fn reader(&self, path: &str) -> Result>; + + /// Write bytes to an output path + async fn write(&self, path: &str, bs: Bytes) -> Result<()>; + + /// Get FileWrite from a path + async fn writer(&self, path: &str) -> Result>; + + /// Delete a file at the given path + async fn delete(&self, path: &str) -> Result<()>; + + /// Delete all files with the given prefix + async fn delete_prefix(&self, path: &str) -> Result<()>; + + /// Create a new input file for reading + fn new_input(&self, path: &str) -> Result; + + /// Create a new output file for writing + fn new_output(&self, path: &str) -> Result; +} - /// Creates operator from path. +/// Common interface for all storage factories. +/// +/// Storage factories are responsible for creating storage instances from configuration +/// properties and extensions. Each storage backend (S3, GCS, etc.) provides its own +/// factory implementation. +/// +/// The trait supports serialization via `typetag`, allowing factory instances to be +/// serialized and deserialized across process boundaries. +/// +/// # Example +/// +/// ```rust,ignore +/// use iceberg::io::{StorageFactory, Extensions}; +/// use std::collections::HashMap; +/// +/// #[derive(Debug, Serialize, Deserialize)] +/// struct MyStorageFactory; +/// +/// #[typetag::serde] +/// impl StorageFactory for MyStorageFactory { +/// fn build( +/// &self, +/// props: HashMap, +/// extensions: Extensions, +/// ) -> Result> { +/// // Parse configuration and create storage +/// Ok(Arc::new(MyStorage::new(props)?)) +/// } +/// } +/// ``` +#[typetag::serde(tag = "type")] +pub trait StorageFactory: Debug + Send + Sync { + /// Create a new storage instance with the given properties and extensions. /// /// # Arguments /// - /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + /// * `props` - Configuration properties for the storage backend + /// * `extensions` - Additional extensions (e.g., custom credential loaders) /// /// # Returns /// - /// The return value consists of two parts: - /// - /// * An [`opendal::Operator`] instance used to operate on file. - /// * Relative path to the root uri of [`opendal::Operator`]. - pub(crate) fn create_operator<'a>( + /// An `Arc` that can be used for file operations. + fn build( &self, - path: &'a impl AsRef, - ) -> crate::Result<(Operator, &'a str)> { - let path = path.as_ref(); - let _ = path; - let (operator, relative_path): (Operator, &str) = match self { - #[cfg(feature = "storage-memory")] - Storage::Memory(op) => { - if let Some(stripped) = path.strip_prefix("memory:/") { - Ok::<_, crate::Error>((op.clone(), stripped)) - } else { - Ok::<_, crate::Error>((op.clone(), &path[1..])) - } - } - #[cfg(feature = "storage-fs")] - Storage::LocalFs => { - let op = super::fs_config_build()?; - - if let Some(stripped) = path.strip_prefix("file:/") { - Ok::<_, crate::Error>((op, stripped)) - } else { - Ok::<_, crate::Error>((op, &path[1..])) - } - } - #[cfg(feature = "storage-s3")] - Storage::S3 { - configured_scheme, - config, - customized_credential_load, - } => { - let op = super::s3_config_build(config, customized_credential_load, path)?; - let op_info = op.info(); - - // Check prefix of s3 path. - let prefix = format!("{}://{}/", configured_scheme, op_info.name()); - if path.starts_with(&prefix) { - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid s3 url: {path}, should start with {prefix}"), - )) - } - } - #[cfg(feature = "storage-gcs")] - Storage::Gcs { config } => { - let operator = super::gcs_config_build(config, path)?; - let prefix = format!("gs://{}/", operator.info().name()); - if path.starts_with(&prefix) { - Ok((operator, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid gcs url: {path}, should start with {prefix}"), - )) - } - } - #[cfg(feature = "storage-oss")] - Storage::Oss { config } => { - let op = super::oss_config_build(config, path)?; - - // Check prefix of oss path. - let prefix = format!("oss://{}/", op.info().name()); - if path.starts_with(&prefix) { - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid oss url: {path}, should start with {prefix}"), - )) - } - } - #[cfg(feature = "storage-azdls")] - Storage::Azdls { - configured_scheme, - config, - } => super::azdls_create_operator(path, config, configured_scheme), - #[cfg(all( - not(feature = "storage-s3"), - not(feature = "storage-fs"), - not(feature = "storage-gcs"), - not(feature = "storage-oss"), - not(feature = "storage-azdls"), - ))] - _ => Err(Error::new( - ErrorKind::FeatureUnsupported, - "No storage service has been enabled", - )), - }?; - - // Transient errors are common for object stores; however there's no - // harm in retrying temporary failures for other storage backends as well. - let operator = operator.layer(RetryLayer::new()); - - Ok((operator, relative_path)) - } + props: HashMap, + extensions: Extensions, + ) -> Result>; +} + +/// A registry of storage factories. +/// +/// The registry allows you to register custom storage factories for different URI schemes. +/// By default, it includes factories for all enabled storage features. +/// +/// # Example +/// +/// ```rust +/// use iceberg::io::StorageRegistry; +/// +/// // Create a new registry with default factories +/// let registry = StorageRegistry::new(); +/// +/// // Get supported storage types +/// let types = registry.supported_types(); +/// println!("Supported storage types: {:?}", types); +/// +/// // Get a factory for a specific scheme +/// # #[cfg(feature = "storage-memory")] +/// # { +/// let factory = registry.get_factory("memory").unwrap(); +/// # } +/// ``` +/// +/// You can also register custom storage factories: +/// +/// ```rust,ignore +/// use std::sync::Arc; +/// use iceberg::io::{StorageRegistry, StorageFactory}; +/// +/// let mut registry = StorageRegistry::new(); +/// +/// // Register a custom storage factory +/// registry.register("custom", Arc::new(MyCustomStorageFactory)); +/// ``` +#[derive(Debug, Clone)] +pub struct StorageRegistry { + factories: HashMap>, +} + +impl StorageRegistry { + /// Create a new storage registry with default factories based on enabled features. + pub fn new() -> Self { + let mut factories: HashMap> = HashMap::new(); + + #[cfg(feature = "storage-memory")] + { + use crate::io::storage_memory::OpenDALMemoryStorageFactory; + let factory = Arc::new(OpenDALMemoryStorageFactory) as Arc; + factories.insert("memory".to_string(), factory); + } + + #[cfg(feature = "storage-fs")] + { + use crate::io::storage_fs::OpenDALFsStorageFactory; + let factory = Arc::new(OpenDALFsStorageFactory) as Arc; + factories.insert("file".to_string(), factory.clone()); + factories.insert("".to_string(), factory); + } + + #[cfg(feature = "storage-s3")] + { + use crate::io::storage_s3::OpenDALS3StorageFactory; + let factory = Arc::new(OpenDALS3StorageFactory) as Arc; + factories.insert("s3".to_string(), factory.clone()); + factories.insert("s3a".to_string(), factory); + } + + #[cfg(feature = "storage-gcs")] + { + use crate::io::storage_gcs::OpenDALGcsStorageFactory; + let factory = Arc::new(OpenDALGcsStorageFactory) as Arc; + factories.insert("gs".to_string(), factory.clone()); + factories.insert("gcs".to_string(), factory); + } - /// Parse scheme. - fn parse_scheme(scheme: &str) -> crate::Result { - match scheme { - "memory" => Ok(Scheme::Memory), - "file" | "" => Ok(Scheme::Fs), - "s3" | "s3a" => Ok(Scheme::S3), - "gs" | "gcs" => Ok(Scheme::Gcs), - "oss" => Ok(Scheme::Oss), - "abfss" | "abfs" | "wasbs" | "wasb" => Ok(Scheme::Azdls), - s => Ok(s.parse::()?), + #[cfg(feature = "storage-oss")] + { + use crate::io::storage_oss::OpenDALOssStorageFactory; + let factory = Arc::new(OpenDALOssStorageFactory) as Arc; + factories.insert("oss".to_string(), factory); } + + #[cfg(feature = "storage-azdls")] + { + use crate::io::storage_azdls::OpenDALAzdlsStorageFactory; + let factory = Arc::new(OpenDALAzdlsStorageFactory) as Arc; + factories.insert("abfs".to_string(), factory.clone()); + factories.insert("abfss".to_string(), factory.clone()); + factories.insert("wasb".to_string(), factory.clone()); + factories.insert("wasbs".to_string(), factory); + } + + Self { factories } + } + + /// Register a custom storage factory for a given scheme. + pub fn register(&mut self, scheme: impl Into, factory: Arc) { + self.factories.insert(scheme.into(), factory); + } + + /// Get a storage factory by scheme. + pub fn get_factory(&self, scheme: &str) -> Result> { + let key = scheme.trim(); + self.factories + .iter() + .find(|(k, _)| k.eq_ignore_ascii_case(key)) + .map(|(_, factory)| factory.clone()) + .ok_or_else(|| { + use crate::{Error, ErrorKind}; + Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Unsupported storage type: {}. Supported types: {}", + scheme, + self.supported_types().join(", ") + ), + ) + }) + } + + /// Return the list of supported storage types. + pub fn supported_types(&self) -> Vec { + self.factories.keys().cloned().collect() + } +} + +impl Default for StorageRegistry { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_storage_registry_new() { + let registry = StorageRegistry::new(); + let types = registry.supported_types(); + + // At least one storage type should be available + assert!(!types.is_empty()); + } + + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_registry_get_factory() { + let registry = StorageRegistry::new(); + + // Should be able to get memory storage factory + let factory = registry.get_factory("memory"); + assert!(factory.is_ok()); + + // Should be case-insensitive + let factory = registry.get_factory("MEMORY"); + assert!(factory.is_ok()); + } + + #[test] + fn test_storage_registry_unsupported_type() { + let registry = StorageRegistry::new(); + + // Should return error for unsupported type + let result = registry.get_factory("unsupported"); + assert!(result.is_err()); + } + + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_registry_clone() { + let registry = StorageRegistry::new(); + let cloned = registry.clone(); + + // Both should have the same factories + assert_eq!( + registry.supported_types().len(), + cloned.supported_types().len() + ); } } diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 5abb0cd6e0..e4f3567e54 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -18,11 +18,19 @@ use std::collections::HashMap; use std::fmt::Display; use std::str::FromStr; +use std::sync::Arc; -use opendal::Configurator; +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::AzdlsConfig; +use opendal::{Configurator, Operator}; +use serde::{Deserialize, Serialize}; use url::Url; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, + Storage, StorageFactory, +}; use crate::{Error, ErrorKind, Result, ensure_data_valid}; /// A connection string. @@ -125,7 +133,7 @@ pub(crate) fn azdls_create_operator<'a>( /// paths are expected to contain the `dfs` storage service. /// - `wasb[s]` is used to refer to files in Blob Storage directly; paths are /// expected to contain the `blob` storage service. -#[derive(Debug, PartialEq)] +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] pub(crate) enum AzureStorageScheme { Abfs, Abfss, @@ -597,3 +605,115 @@ mod tests { } } } + +/// Azure Data Lake Storage implementation using OpenDAL +/// +/// Stores configuration and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OpenDALAzdlsStorage { + /// Because Azdls accepts multiple possible schemes, we store the full + /// passed scheme here to later validate schemes passed via paths. + configured_scheme: AzureStorageScheme, + config: Arc, +} + +impl OpenDALAzdlsStorage { + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let (op, relative_path) = + azdls_create_operator(path, &self.config, &self.configured_scheme)?; + let op = op.layer(opendal::layers::RetryLayer::new()); + Ok((op, relative_path)) + } +} + +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALAzdlsStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn delete_prefix(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } +} + +/// Factory for Azure Data Lake Storage +#[derive(Debug, Serialize, Deserialize)] +pub struct OpenDALAzdlsStorageFactory; + +#[typetag::serde] +impl StorageFactory for OpenDALAzdlsStorageFactory { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + // Get the scheme string from the props or use default + let scheme_str = props + .get(STORAGE_LOCATION_SCHEME) + .cloned() + .unwrap_or_else(|| "abfs".to_string()); + + // Parse the scheme + let scheme = scheme_str.parse::()?; + + // Parse Azdls config from props + let config = azdls_config_parse(props)?; + + Ok(Arc::new(OpenDALAzdlsStorage { + configured_scheme: scheme, + config: Arc::new(config), + })) + } +} diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index d3e121a085..820cac393c 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -15,10 +15,19 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use bytes::Bytes; use opendal::Operator; use opendal::services::FsConfig; +use serde::{Deserialize, Serialize}; use crate::Result; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, +}; /// Build new opendal operator from give path. pub(crate) fn fs_config_build() -> Result { @@ -27,3 +36,144 @@ pub(crate) fn fs_config_build() -> Result { Ok(Operator::from_config(cfg)?.finish()) } + +/// Filesystem storage implementation using OpenDAL +/// +/// This storage is stateless and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OpenDALFsStorage; + +impl OpenDALFsStorage { + /// Extract relative path from file:// URLs + fn extract_relative_path<'a>(&self, path: &'a str) -> &'a str { + if let Some(stripped) = path.strip_prefix("file:/") { + stripped + } else { + &path[1..] + } + } +} + +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALFsStorage { + async fn exists(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(op.delete(relative_path).await?) + } + + async fn delete_prefix(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } +} + +/// Factory for OpenDAL Filesystem storage +#[derive(Debug, Serialize, Deserialize)] +pub struct OpenDALFsStorageFactory; + +#[typetag::serde] +impl StorageFactory for OpenDALFsStorageFactory { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> Result> { + Ok(Arc::new(OpenDALFsStorage)) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::io::Storage; + + #[test] + fn test_fs_storage_serialization() { + // Create a filesystem storage instance using the factory + let factory = OpenDALFsStorageFactory; + let storage = factory + .build(HashMap::new(), Extensions::default()) + .unwrap(); + + // Serialize the storage + let serialized = serde_json::to_string(&storage).unwrap(); + + // Deserialize the storage + let deserialized: Box = serde_json::from_str(&serialized).unwrap(); + + // Verify the type is correct + assert!(format!("{:?}", deserialized).contains("OpenDALFsStorage")); + } + + #[test] + fn test_fs_factory_serialization() { + use crate::io::StorageFactory; + + // Create a factory instance + let factory: Box = Box::new(OpenDALFsStorageFactory); + + // Serialize the factory + let serialized = serde_json::to_string(&factory).unwrap(); + + // Deserialize the factory + let deserialized: Box = serde_json::from_str(&serialized).unwrap(); + + // Verify the type is correct + assert!(format!("{:?}", deserialized).contains("OpenDALFsStorageFactory")); + } +} diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 7718df603f..b44d35a088 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -17,12 +17,19 @@ //! Google Cloud Storage properties use std::collections::HashMap; +use std::sync::Arc; +use async_trait::async_trait; +use bytes::Bytes; use opendal::Operator; use opendal::services::GcsConfig; +use serde::{Deserialize, Serialize}; use url::Url; -use crate::io::is_truthy; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, + is_truthy, +}; use crate::{Error, ErrorKind, Result}; // Reference: https://github.com/apache/iceberg/blob/main/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -104,3 +111,108 @@ pub(crate) fn gcs_config_build(cfg: &GcsConfig, path: &str) -> Result cfg.bucket = bucket.to_string(); Ok(Operator::from_config(cfg)?.finish()) } + +/// GCS storage implementation using OpenDAL +/// +/// Stores configuration and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OpenDALGcsStorage { + config: Arc, +} + +impl OpenDALGcsStorage { + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let operator = gcs_config_build(&self.config, path)?; + let prefix = format!("gs://{}/", operator.info().name()); + + if path.starts_with(&prefix) { + let op = operator.layer(opendal::layers::RetryLayer::new()); + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid gcs url: {}, should start with {}", path, prefix), + )) + } + } +} + +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALGcsStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn delete_prefix(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } +} + +/// Factory for OpenDAL GCS storage +#[derive(Debug, Serialize, Deserialize)] +pub struct OpenDALGcsStorageFactory; + +#[typetag::serde] +impl StorageFactory for OpenDALGcsStorageFactory { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + let cfg = gcs_config_parse(props)?; + Ok(Arc::new(OpenDALGcsStorage { + config: Arc::new(cfg), + })) + } +} diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index b8023717b6..b2e1ed5629 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -15,11 +15,214 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; + +use async_trait::async_trait; +use bytes::Bytes; use opendal::Operator; use opendal::services::MemoryConfig; +use serde::{Deserialize, Serialize}; use crate::Result; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, +}; + +/// Memory storage implementation using OpenDAL +/// +/// Uses lazy initialization - the operator is created on first use and then cached. +/// This allows the storage to be serialized/deserialized while maintaining state. +/// The operator field is skipped during serialization and recreated on first use. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OpenDALMemoryStorage { + #[serde(skip, default = "default_op")] + op: Arc>>, +} + +fn default_op() -> Arc>> { + Arc::new(Mutex::new(None)) +} + +impl Default for OpenDALMemoryStorage { + fn default() -> Self { + Self { + op: Arc::new(Mutex::new(None)), + } + } +} + +impl OpenDALMemoryStorage { + /// Get or create the memory operator (lazy initialization) + fn get_operator(&self) -> Result { + let mut guard = self.op.lock().unwrap(); + if guard.is_none() { + *guard = Some(Operator::from_config(MemoryConfig::default())?.finish()); + } + Ok(guard.as_ref().unwrap().clone()) + } + + /// Extract relative path from memory:// URLs + fn extract_relative_path<'a>(&self, path: &'a str) -> &'a str { + if let Some(stripped) = path.strip_prefix("memory:/") { + stripped + } else { + &path[1..] + } + } +} + +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALMemoryStorage { + async fn exists(&self, path: &str) -> Result { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + Ok(op.delete(relative_path).await?) + } + + async fn delete_prefix(&self, path: &str) -> Result<()> { + let op = self.get_operator()?; + let relative_path = self.extract_relative_path(path); + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } +} + +/// Factory for OpenDAL Memory storage +#[derive(Debug, Serialize, Deserialize)] +pub struct OpenDALMemoryStorageFactory; + +#[typetag::serde] +impl StorageFactory for OpenDALMemoryStorageFactory { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> Result> { + Ok(Arc::new(OpenDALMemoryStorage::default())) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::io::Storage; + + #[test] + fn test_memory_storage_serialization() { + // Create a memory storage instance using the factory + let factory = OpenDALMemoryStorageFactory; + let storage = factory + .build(HashMap::new(), Extensions::default()) + .unwrap(); + + // Serialize the storage + let serialized = serde_json::to_string(&storage).unwrap(); + + // Deserialize the storage + let deserialized: Box = serde_json::from_str(&serialized).unwrap(); + + // Verify the type is correct + assert!(format!("{:?}", deserialized).contains("OpenDALMemoryStorage")); + } + + #[test] + fn test_memory_factory_serialization() { + use crate::io::StorageFactory; + + // Create a factory instance + let factory: Box = Box::new(OpenDALMemoryStorageFactory); + + // Serialize the factory + let serialized = serde_json::to_string(&factory).unwrap(); + + // Deserialize the factory + let deserialized: Box = serde_json::from_str(&serialized).unwrap(); + + // Verify the type is correct + assert!(format!("{:?}", deserialized).contains("OpenDALMemoryStorageFactory")); + } + + #[test] + fn test_memory_factory_to_storage_serialization() { + use crate::io::{Extensions, StorageFactory}; + + // Create a factory and build storage + let factory = OpenDALMemoryStorageFactory; + let storage = factory + .build(HashMap::new(), Extensions::default()) + .unwrap(); + + // Serialize the storage + let storage_json = serde_json::to_string(&storage).unwrap(); + + // Deserialize the storage + let deserialized_storage: Box = serde_json::from_str(&storage_json).unwrap(); + + // Verify storage type + assert!(format!("{:?}", deserialized_storage).contains("OpenDALMemoryStorage")); + + // Serialize the factory + let factory_boxed: Box = Box::new(factory); + let factory_json = serde_json::to_string(&factory_boxed).unwrap(); + + // Deserialize the factory + let deserialized_factory: Box = + serde_json::from_str(&factory_json).unwrap(); -pub(crate) fn memory_config_build() -> Result { - Ok(Operator::from_config(MemoryConfig::default())?.finish()) + // Verify factory type + assert!(format!("{:?}", deserialized_factory).contains("OpenDALMemoryStorageFactory")); + } } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index e82dda23a5..5d701d84a6 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -16,11 +16,18 @@ // under the License. use std::collections::HashMap; +use std::sync::Arc; +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::OssConfig; use opendal::{Configurator, Operator}; +use serde::{Deserialize, Serialize}; use url::Url; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, +}; use crate::{Error, ErrorKind, Result}; /// Required configuration arguments for creating an Aliyun OSS Operator with OpenDAL: @@ -64,3 +71,108 @@ pub(crate) fn oss_config_build(cfg: &OssConfig, path: &str) -> Result Ok(Operator::new(builder)?.finish()) } + +/// OSS storage implementation using OpenDAL +/// +/// Stores configuration and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OpenDALOssStorage { + config: Arc, +} + +impl OpenDALOssStorage { + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let op = oss_config_build(&self.config, path)?; + let prefix = format!("oss://{}/", op.info().name()); + + if path.starts_with(&prefix) { + let op = op.layer(opendal::layers::RetryLayer::new()); + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid oss url: {}, should start with {}", path, prefix), + )) + } + } +} + +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALOssStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn delete_prefix(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } +} + +/// Factory for OpenDAL OSS storage +#[derive(Debug, Serialize, Deserialize)] +pub struct OpenDALOssStorageFactory; + +#[typetag::serde] +impl StorageFactory for OpenDALOssStorageFactory { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + let cfg = oss_config_parse(props)?; + Ok(Arc::new(OpenDALOssStorage { + config: Arc::new(cfg), + })) + } +} diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index f069e0e2f9..f72a410df7 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -19,13 +19,18 @@ use std::collections::HashMap; use std::sync::Arc; use async_trait::async_trait; +use bytes::Bytes; use opendal::services::S3Config; use opendal::{Configurator, Operator}; pub use reqsign::{AwsCredential, AwsCredentialLoad}; use reqwest::Client; +use serde::{Deserialize, Serialize}; use url::Url; -use crate::io::is_truthy; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, + Storage, StorageFactory, is_truthy, +}; use crate::{Error, ErrorKind, Result}; /// Following are arguments for [s3 file io](https://py.iceberg.apache.org/configuration/#s3). @@ -213,3 +218,187 @@ impl AwsCredentialLoad for CustomAwsCredentialLoader { self.0.load_credential(client).await } } + +/// S3 storage implementation using OpenDAL +/// +/// Stores configuration and creates operators on-demand. +/// The `customized_credential_load` field is not serialized. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OpenDALS3Storage { + /// s3 storage could have `s3://` and `s3a://`. + /// Storing the scheme string here to return the correct path. + configured_scheme: String, + config: Arc, + #[serde(skip)] + customized_credential_load: Option, +} + +impl OpenDALS3Storage { + /// Creates operator from path. + /// + /// # Arguments + /// + /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + /// + /// # Returns + /// + /// The return value consists of two parts: + /// + /// * An [`opendal::Operator`] instance used to operate on file. + /// * Relative path to the root uri of [`opendal::Operator`]. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let op = s3_config_build(&self.config, &self.customized_credential_load, path)?; + let op_info = op.info(); + + // Check prefix of s3 path. + let prefix = format!("{}://{}/", self.configured_scheme, op_info.name()); + if path.starts_with(&prefix) { + // Add retry layer for transient errors + let op = op.layer(opendal::layers::RetryLayer::new()); + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid s3 url: {}, should start with {}", path, prefix), + )) + } + } +} + +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALS3Storage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn delete_prefix(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } +} + +/// Factory for S3 storage +#[derive(Debug, Serialize, Deserialize)] +pub struct OpenDALS3StorageFactory; + +#[typetag::serde] +impl StorageFactory for OpenDALS3StorageFactory { + fn build( + &self, + props: HashMap, + extensions: Extensions, + ) -> Result> { + // Get the scheme string from the props or use "s3" as default + let scheme_str = props + .get(STORAGE_LOCATION_SCHEME) + .cloned() + .unwrap_or_else(|| "s3".to_string()); + + // Parse S3 config from props + let config = s3_config_parse(props)?; + + // Get customized credential loader from extensions if available + let customized_credential_load = extensions + .get::() + .map(Arc::unwrap_or_clone); + + Ok(Arc::new(OpenDALS3Storage { + configured_scheme: scheme_str, + config: Arc::new(config), + customized_credential_load, + })) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::io::{STORAGE_LOCATION_SCHEME, Storage}; + + #[test] + fn test_s3_storage_serialization() { + // Create an S3 storage instance using the factory + let factory = OpenDALS3StorageFactory; + let mut props = HashMap::new(); + props.insert(S3_REGION.to_string(), "us-east-1".to_string()); + props.insert(STORAGE_LOCATION_SCHEME.to_string(), "s3".to_string()); + + let storage = factory.build(props, Extensions::default()).unwrap(); + + // Serialize the storage + let serialized = serde_json::to_string(&storage).unwrap(); + + // Deserialize the storage + let deserialized: Box = serde_json::from_str(&serialized).unwrap(); + + // Verify the type is correct + assert!(format!("{:?}", deserialized).contains("OpenDALS3Storage")); + } + + #[test] + fn test_s3_factory_serialization() { + use crate::io::StorageFactory; + + // Create a factory instance + let factory: Box = Box::new(OpenDALS3StorageFactory); + + // Serialize the factory + let serialized = serde_json::to_string(&factory).unwrap(); + + // Deserialize the factory + let deserialized: Box = serde_json::from_str(&serialized).unwrap(); + + // Verify the type is correct + assert!(format!("{:?}", deserialized).contains("OpenDALS3StorageFactory")); + } +} diff --git a/crates/iceberg/src/puffin/metadata.rs b/crates/iceberg/src/puffin/metadata.rs index 15a8e9b33a..a4b7464704 100644 --- a/crates/iceberg/src/puffin/metadata.rs +++ b/crates/iceberg/src/puffin/metadata.rs @@ -285,10 +285,13 @@ impl FileMetadata { let input_file_length = input_file.metadata().await?.size; let footer_payload_length = - FileMetadata::read_footer_payload_length(&file_read, input_file_length).await?; - let footer_bytes = - FileMetadata::read_footer_bytes(&file_read, input_file_length, footer_payload_length) - .await?; + FileMetadata::read_footer_payload_length(file_read.as_ref(), input_file_length).await?; + let footer_bytes = FileMetadata::read_footer_bytes( + file_read.as_ref(), + input_file_length, + footer_payload_length, + ) + .await?; let magic_length = FileMetadata::MAGIC_LENGTH as usize; // check first four bytes of footer diff --git a/crates/iceberg/src/puffin/reader.rs b/crates/iceberg/src/puffin/reader.rs index dce53d93f0..8941d4a8fc 100644 --- a/crates/iceberg/src/puffin/reader.rs +++ b/crates/iceberg/src/puffin/reader.rs @@ -18,7 +18,7 @@ use tokio::sync::OnceCell; use crate::Result; -use crate::io::{FileRead, InputFile}; +use crate::io::InputFile; use crate::puffin::blob::Blob; use crate::puffin::metadata::{BlobMetadata, FileMetadata}; From 6488d67a086ff9d3b717d66f87c07ec8e7b2266e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 10 Dec 2025 21:29:25 -0800 Subject: [PATCH 02/12] fix clippy --- crates/iceberg/src/io/storage_gcs.rs | 2 +- crates/iceberg/src/io/storage_oss.rs | 2 +- crates/iceberg/src/io/storage_s3.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index b44d35a088..4cd13453a9 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -132,7 +132,7 @@ impl OpenDALGcsStorage { } else { Err(Error::new( ErrorKind::DataInvalid, - format!("Invalid gcs url: {}, should start with {}", path, prefix), + format!("Invalid gcs url: {path}, should start with {prefix}"), )) } } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 5d701d84a6..132b8ac349 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -92,7 +92,7 @@ impl OpenDALOssStorage { } else { Err(Error::new( ErrorKind::DataInvalid, - format!("Invalid oss url: {}, should start with {}", path, prefix), + format!("Invalid oss url: {path}, should start with {prefix}"), )) } } diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index f72a410df7..5384673c2f 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -259,7 +259,7 @@ impl OpenDALS3Storage { } else { Err(Error::new( ErrorKind::DataInvalid, - format!("Invalid s3 url: {}, should start with {}", path, prefix), + format!("Invalid s3 url: {path}, should start with {prefix}"), )) } } From 5561a200986247070d0fc880085b2d8dbd9897e7 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 10 Dec 2025 21:55:16 -0800 Subject: [PATCH 03/12] more clippy --- crates/iceberg/src/io/storage_fs.rs | 4 ++-- crates/iceberg/src/io/storage_memory.rs | 8 ++++---- crates/iceberg/src/io/storage_s3.rs | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index 820cac393c..b668016f76 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -157,7 +157,7 @@ mod tests { let deserialized: Box = serde_json::from_str(&serialized).unwrap(); // Verify the type is correct - assert!(format!("{:?}", deserialized).contains("OpenDALFsStorage")); + assert!(format!("{deserialized:?}").contains("OpenDALFsStorage")); } #[test] @@ -174,6 +174,6 @@ mod tests { let deserialized: Box = serde_json::from_str(&serialized).unwrap(); // Verify the type is correct - assert!(format!("{:?}", deserialized).contains("OpenDALFsStorageFactory")); + assert!(format!("{deserialized:?}").contains("OpenDALFsStorageFactory")); } } diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index b2e1ed5629..fc57fc7587 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -175,7 +175,7 @@ mod tests { let deserialized: Box = serde_json::from_str(&serialized).unwrap(); // Verify the type is correct - assert!(format!("{:?}", deserialized).contains("OpenDALMemoryStorage")); + assert!(format!("{deserialized:?}").contains("OpenDALMemoryStorage")); } #[test] @@ -192,7 +192,7 @@ mod tests { let deserialized: Box = serde_json::from_str(&serialized).unwrap(); // Verify the type is correct - assert!(format!("{:?}", deserialized).contains("OpenDALMemoryStorageFactory")); + assert!(format!("{deserialized:?}").contains("OpenDALMemoryStorageFactory")); } #[test] @@ -212,7 +212,7 @@ mod tests { let deserialized_storage: Box = serde_json::from_str(&storage_json).unwrap(); // Verify storage type - assert!(format!("{:?}", deserialized_storage).contains("OpenDALMemoryStorage")); + assert!(format!("{deserialized_storage:?}").contains("OpenDALMemoryStorage")); // Serialize the factory let factory_boxed: Box = Box::new(factory); @@ -223,6 +223,6 @@ mod tests { serde_json::from_str(&factory_json).unwrap(); // Verify factory type - assert!(format!("{:?}", deserialized_factory).contains("OpenDALMemoryStorageFactory")); + assert!(format!("{deserialized_factory:?}").contains("OpenDALMemoryStorageFactory")); } } diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 5384673c2f..3558b6a4b1 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -382,7 +382,7 @@ mod tests { let deserialized: Box = serde_json::from_str(&serialized).unwrap(); // Verify the type is correct - assert!(format!("{:?}", deserialized).contains("OpenDALS3Storage")); + assert!(format!("{deserialized:?}").contains("OpenDALS3Storage")); } #[test] @@ -399,6 +399,6 @@ mod tests { let deserialized: Box = serde_json::from_str(&serialized).unwrap(); // Verify the type is correct - assert!(format!("{:?}", deserialized).contains("OpenDALS3StorageFactory")); + assert!(format!("{deserialized:?}").contains("OpenDALS3StorageFactory")); } } From f5c88cbe93a6eac66ff31666768109803f7eda3b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 10:02:20 -0800 Subject: [PATCH 04/12] trying to keep it simple --- crates/iceberg/src/io/file_io.rs | 336 +--------------- crates/iceberg/src/io/mod.rs | 9 +- crates/iceberg/src/io/storage.rs | 495 ++++++++++++++---------- crates/iceberg/src/io/storage_azdls.rs | 134 +------ crates/iceberg/src/io/storage_fs.rs | 150 ------- crates/iceberg/src/io/storage_gcs.rs | 114 +----- crates/iceberg/src/io/storage_memory.rs | 207 +--------- crates/iceberg/src/io/storage_oss.rs | 112 ------ crates/iceberg/src/io/storage_s3.rs | 191 +-------- 9 files changed, 312 insertions(+), 1436 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 56f1d4bb45..6d43222503 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -25,9 +25,7 @@ use async_trait::async_trait; use bytes::Bytes; use url::Url; -// Re-export traits from storage module -pub use super::storage::{Storage, StorageFactory, StorageRegistry}; -use crate::io::STORAGE_LOCATION_SCHEME; +pub use super::storage::{OpenDALStorage, Storage}; use crate::{Error, ErrorKind, Result}; /// FileIO implementation, used to manipulate files in underlying storage. @@ -39,18 +37,17 @@ use crate::{Error, ErrorKind, Result}; /// /// Supported storages: /// -/// | Storage | Feature Flag | Expected Path Format | Schemes | -/// |--------------------|-------------------|----------------------------------| ------------------------------| -/// | Local file system | `storage-fs` | `file` | `file://path/to/file` | -/// | Memory | `storage-memory` | `memory` | `memory://path/to/file` | -/// | S3 | `storage-s3` | `s3`, `s3a` | `s3:///path/to/file` | -/// | GCS | `storage-gcs` | `gs`, `gcs` | `gs:///path/to/file` | -/// | OSS | `storage-oss` | `oss` | `oss:///path/to/file` | -/// | Azure Datalake | `storage-azdls` | `abfs`, `abfss`, `wasb`, `wasbs` | `abfs://@.dfs.core.windows.net/path/to/file` or `wasb://@.blob.core.windows.net/path/to/file` | +/// | Storage | Feature Flag | Schemes | +/// |--------------------|-------------------|----------------------------------| +/// | Local file system | `storage-fs` | `file://path/to/file` | +/// | Memory | `storage-memory` | `memory://path/to/file` | +/// | S3 | `storage-s3` | `s3:///path/to/file` | +/// | GCS | `storage-gcs` | `gs:///path/to/file` | +/// | OSS | `storage-oss` | `oss:///path/to/file` | +/// | Azure Datalake | `storage-azdls` | `abfs[s]://...` or `wasb[s]://...` | #[derive(Clone, Debug)] pub struct FileIO { builder: FileIOBuilder, - inner: Arc, } @@ -164,29 +161,6 @@ impl Extensions { } /// Builder for [`FileIO`]. -/// -/// # Custom Storage Implementations -/// -/// You can use custom storage implementations by creating a custom -/// [`StorageRegistry`] and registering your storage factory: -/// -/// ```rust,ignore -/// use iceberg::io::{StorageRegistry, StorageFactory, FileIOBuilder}; -/// use std::sync::Arc; -/// -/// // Create your custom storage factory -/// let my_factory = Arc::new(MyCustomStorageFactory); -/// -/// // Register it with a custom scheme -/// let mut registry = StorageRegistry::new(); -/// registry.register("mycustom", my_factory); -/// -/// // Use it to build FileIO -/// let file_io = FileIOBuilder::new("mycustom") -/// .with_prop("key", "value") -/// .with_registry(registry) -/// .build()?; -/// ``` #[derive(Clone, Debug)] pub struct FileIOBuilder { /// This is used to infer scheme of operator. @@ -197,8 +171,6 @@ pub struct FileIOBuilder { props: HashMap, /// Optional extensions to configure the underlying FileIO behavior. extensions: Extensions, - /// Optional custom registry. If None, a default registry will be created. - registry: Option, } impl FileIOBuilder { @@ -209,7 +181,6 @@ impl FileIOBuilder { scheme_str: Some(scheme_str.to_string()), props: HashMap::default(), extensions: Extensions::default(), - registry: None, } } @@ -219,26 +190,17 @@ impl FileIOBuilder { scheme_str: None, props: HashMap::default(), extensions: Extensions::default(), - registry: None, } } /// Fetch the scheme string. /// /// The scheme_str will be empty if it's None. - pub fn into_parts( - self, - ) -> ( - String, - HashMap, - Extensions, - Option, - ) { + pub fn into_parts(self) -> (String, HashMap, Extensions) { ( self.scheme_str.unwrap_or_default(), self.props, self.extensions, - self.registry, ) } @@ -276,49 +238,13 @@ impl FileIOBuilder { self.extensions.get::() } - /// Sets a custom storage registry. - /// - /// This allows you to register custom storage implementations that can be used - /// when building the FileIO. If not set, a default registry with built-in - /// storage types will be used. - /// - /// # Example - /// - /// ```rust,ignore - /// use iceberg::io::{StorageRegistry, FileIOBuilder}; - /// use std::sync::Arc; - /// - /// let mut registry = StorageRegistry::new(); - /// registry.register("mycustom", Arc::new(MyCustomStorageFactory)); - /// - /// let file_io = FileIOBuilder::new("mycustom") - /// .with_registry(registry) - /// .build()?; - /// ``` - pub fn with_registry(mut self, registry: StorageRegistry) -> Self { - self.registry = Some(registry); - self - } - /// Builds [`FileIO`]. pub fn build(self) -> Result { - // Use the scheme to determine the storage type - let scheme = self.scheme_str.clone().unwrap_or_default(); - - // Use custom registry if provided, otherwise create default - let registry = self.registry.clone().unwrap_or_default(); - - let factory = registry.get_factory(scheme.as_str())?; - - let mut props_with_scheme = self.props.clone(); - props_with_scheme.insert(STORAGE_LOCATION_SCHEME.to_string(), scheme); - - // Build storage with props and extensions - let storage = factory.build(props_with_scheme, self.extensions.clone())?; + let storage = OpenDALStorage::build(self.clone())?; Ok(FileIO { builder: self, - inner: storage, + inner: Arc::new(storage), }) } } @@ -513,137 +439,15 @@ impl OutputFile { #[cfg(test)] mod tests { - use std::collections::HashMap; use std::fs::{File, create_dir_all}; use std::io::Write; use std::path::Path; - use std::sync::{Arc, Mutex, MutexGuard}; - use async_trait::async_trait; - use bytes::Bytes; use futures::AsyncReadExt; use futures::io::AllowStdIo; - use serde::{Deserialize, Serialize}; use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; - use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, - STORAGE_LOCATION_SCHEME, Storage, StorageFactory, StorageRegistry, - }; - use crate::{Error, ErrorKind, Result}; - - // Test storage implementation that tracks write operations - #[derive(Debug, Clone, Serialize, Deserialize)] - struct TestStorage { - #[serde(skip, default = "default_written")] - written: Arc>>, - received_props: HashMap, - } - - fn default_written() -> Arc>> { - Arc::new(Mutex::new(Vec::new())) - } - - #[allow(dead_code)] - impl TestStorage { - pub fn written(&self) -> MutexGuard<'_, Vec> { - self.written.lock().unwrap() - } - - pub fn received_props(&self) -> &HashMap { - &self.received_props - } - } - - #[async_trait] - #[typetag::serde] - impl Storage for TestStorage { - async fn exists(&self, _path: &str) -> Result { - Ok(true) - } - - async fn metadata(&self, _path: &str) -> Result { - Ok(FileMetadata { size: 42 }) - } - - async fn read(&self, _path: &str) -> Result { - Ok(Bytes::from("test data")) - } - - async fn reader(&self, _path: &str) -> Result> { - Err(Error::new( - ErrorKind::FeatureUnsupported, - "TestStorage does not support reader", - )) - } - - async fn write(&self, path: &str, _bs: Bytes) -> Result<()> { - self.written.lock().unwrap().push(path.to_string()); - Ok(()) - } - - async fn writer(&self, _path: &str) -> Result> { - Err(Error::new( - ErrorKind::FeatureUnsupported, - "TestStorage does not support writer", - )) - } - - async fn delete(&self, _path: &str) -> Result<()> { - Ok(()) - } - - async fn delete_prefix(&self, _path: &str) -> Result<()> { - Ok(()) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } - } - - fn default_received_props() -> Arc>> { - Arc::new(Mutex::new(HashMap::new())) - } - - // Test storage factory - #[derive(Debug, Serialize, Deserialize)] - struct TestStorageFactory { - #[serde(skip, default = "default_written")] - written: Arc>>, - #[serde(skip, default = "default_received_props")] - received_props: Arc>>, - } - - impl TestStorageFactory { - pub fn written(&self) -> MutexGuard<'_, Vec> { - self.written.lock().unwrap() - } - - pub fn received_props(&self) -> MutexGuard<'_, HashMap> { - self.received_props.lock().unwrap() - } - } - - #[typetag::serde] - impl StorageFactory for TestStorageFactory { - fn build( - &self, - props: HashMap, - _extensions: Extensions, - ) -> Result> { - *self.received_props.lock().unwrap() = props.clone(); - Ok(Arc::new(TestStorage { - written: self.written.clone(), - received_props: props, - })) - } - } fn create_local_file_io() -> FileIO { FileIOBuilder::new_fs_io().build().unwrap() @@ -676,7 +480,6 @@ mod tests { let input_file = file_io.new_input(&full_path).unwrap(); assert!(input_file.exists().await.unwrap()); - // Remove heading slash assert_eq!(&full_path, input_file.location()); let read_content = read_from_file(full_path).await; @@ -782,122 +585,9 @@ mod tests { assert!(io.exists(&path.clone()).await.unwrap()); let input_file = io.new_input(&path).unwrap(); let content = input_file.read().await.unwrap(); - assert_eq!(content, Bytes::from("test")); + assert_eq!(content, bytes::Bytes::from("test")); io.delete(&path).await.unwrap(); assert!(!io.exists(&path).await.unwrap()); } - - #[test] - fn test_custom_registry() { - // Create a custom registry and register test storage - let factory = Arc::new(TestStorageFactory { - written: Arc::new(Mutex::new(Vec::new())), - received_props: Arc::new(Mutex::new(HashMap::new())), - }); - - let mut registry = StorageRegistry::new(); - registry.register("test", factory.clone()); - - // Build FileIO with custom storage - let file_io = FileIOBuilder::new("test") - .with_registry(registry) - .build() - .unwrap(); - - // Verify we can create files with the custom storage - assert!(file_io.new_output("test://test.txt").is_ok()); - assert!(file_io.new_input("test://test.txt").is_ok()); - } - - #[tokio::test] - async fn test_custom_registry_operations() { - // Create test storage with write tracking - let factory = Arc::new(TestStorageFactory { - written: Arc::new(Mutex::new(Vec::new())), - received_props: Arc::new(Mutex::new(HashMap::new())), - }); - - let mut registry = StorageRegistry::new(); - registry.register("test", factory.clone()); - - // Build FileIO with test storage - let file_io = FileIOBuilder::new("test") - .with_registry(registry) - .build() - .unwrap(); - - // Perform operations - let output = file_io.new_output("test://bucket/file.txt").unwrap(); - output.write(Bytes::from("test")).await.unwrap(); - - let input = file_io.new_input("test://bucket/file.txt").unwrap(); - let data = input.read().await.unwrap(); - assert_eq!(data, Bytes::from("test data")); - - let metadata = input.metadata().await.unwrap(); - assert_eq!(metadata.size, 42); - - // Verify write was tracked - let tracked = factory.written(); - assert_eq!(tracked.len(), 1); - assert_eq!(tracked[0], "test://bucket/file.txt"); - } - - #[test] - fn test_scheme_and_props_propagation() { - // Create test storage that captures props - let factory = Arc::new(TestStorageFactory { - written: Arc::new(Mutex::new(Vec::new())), - received_props: Arc::new(Mutex::new(HashMap::new())), - }); - - let mut registry = StorageRegistry::new(); - registry.register("myscheme", factory.clone()); - - // Build FileIO with custom scheme and additional props - let file_io = FileIOBuilder::new("myscheme") - .with_prop("custom.prop", "custom_value") - .with_registry(registry) - .build() - .unwrap(); - - // Verify the storage was created - assert!(file_io.new_output("myscheme://test.txt").is_ok()); - - // Verify the scheme was propagated to the factory - let props = factory.received_props(); - assert_eq!( - props.get(STORAGE_LOCATION_SCHEME), - Some(&"myscheme".to_string()) - ); - // Verify custom props were also passed - assert_eq!(props.get("custom.prop"), Some(&"custom_value".to_string())); - } - - #[test] - fn test_into_parts_includes_registry() { - let registry = StorageRegistry::new(); - - let builder = FileIOBuilder::new("memory") - .with_prop("key", "value") - .with_registry(registry.clone()); - - let (scheme, props, _extensions, returned_registry) = builder.into_parts(); - - assert_eq!(scheme, "memory"); - assert_eq!(props.get("key"), Some(&"value".to_string())); - assert!(returned_registry.is_some()); - } - - #[test] - fn test_into_parts_without_registry() { - let builder = FileIOBuilder::new("memory").with_prop("key", "value"); - - let (scheme, props, _extensions, returned_registry) = builder.into_parts(); - - assert_eq!(scheme, "memory"); - assert_eq!(props.get("key"), Some(&"value".to_string())); - assert!(returned_registry.is_none()); - } } diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index b16db18563..7e01d9b78b 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -70,12 +70,9 @@ mod file_io; mod storage; pub use file_io::*; -pub use storage::{Storage, StorageFactory, StorageRegistry}; +pub use storage::{OpenDALStorage, Storage}; pub(crate) mod object_cache; -/// Property key used to pass the scheme string from FileIOBuilder to StorageBuilder. -pub const STORAGE_LOCATION_SCHEME: &str = "iceberg.storage.location.scheme"; - #[cfg(feature = "storage-azdls")] mod storage_azdls; #[cfg(feature = "storage-fs")] @@ -91,8 +88,12 @@ mod storage_s3; #[cfg(feature = "storage-azdls")] pub use storage_azdls::*; +#[cfg(feature = "storage-fs")] +pub(crate) use storage_fs::*; #[cfg(feature = "storage-gcs")] pub use storage_gcs::*; +#[cfg(feature = "storage-memory")] +pub(crate) use storage_memory::*; #[cfg(feature = "storage-oss")] pub use storage_oss::*; #[cfg(feature = "storage-s3")] diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index f2bdb8943d..656d390f1f 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,44 +15,30 @@ // specific language governing permissions and limitations // under the License. -//! Storage traits and implementations for Iceberg. +//! Storage implementation for Iceberg using OpenDAL. //! -//! This module provides the core storage abstraction used throughout Iceberg Rust. -//! Storage implementations handle reading and writing files across different backends -//! (S3, GCS, Azure, local filesystem, etc.). +//! This module provides a unified storage abstraction that handles all supported +//! storage backends (S3, GCS, Azure, local filesystem, memory, etc.) through OpenDAL. -use std::collections::HashMap; use std::fmt::Debug; use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; +use opendal::layers::RetryLayer; +use opendal::Operator; +use serde::{Deserialize, Serialize}; -use super::{Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile}; -use crate::Result; +use super::{FileIOBuilder, FileMetadata, FileRead, FileWrite, InputFile, OutputFile}; +use crate::{Error, ErrorKind, Result}; /// Trait for storage operations in Iceberg. /// -/// This trait defines the interface for all storage backends. Implementations -/// provide access to different storage systems like S3, GCS, Azure, local filesystem, etc. +/// This trait defines the interface for all storage backends. The default implementation +/// uses OpenDAL to support various storage systems like S3, GCS, Azure, local filesystem, etc. /// /// The trait supports serialization via `typetag`, allowing storage instances to be /// serialized and deserialized across process boundaries. -/// -/// # Example -/// -/// ```rust,ignore -/// use iceberg::io::Storage; -/// -/// async fn example(storage: Arc) -> Result<()> { -/// // Check if file exists -/// if storage.exists("s3://bucket/path/file.parquet").await? { -/// // Read file -/// let data = storage.read("s3://bucket/path/file.parquet").await?; -/// } -/// Ok(()) -/// } -/// ``` #[async_trait] #[typetag::serde(tag = "type")] pub trait Storage: Debug + Send + Sync { @@ -87,234 +73,313 @@ pub trait Storage: Debug + Send + Sync { fn new_output(&self, path: &str) -> Result; } -/// Common interface for all storage factories. -/// -/// Storage factories are responsible for creating storage instances from configuration -/// properties and extensions. Each storage backend (S3, GCS, etc.) provides its own -/// factory implementation. -/// -/// The trait supports serialization via `typetag`, allowing factory instances to be -/// serialized and deserialized across process boundaries. +/// Unified OpenDAL-based storage implementation. /// -/// # Example -/// -/// ```rust,ignore -/// use iceberg::io::{StorageFactory, Extensions}; -/// use std::collections::HashMap; -/// -/// #[derive(Debug, Serialize, Deserialize)] -/// struct MyStorageFactory; -/// -/// #[typetag::serde] -/// impl StorageFactory for MyStorageFactory { -/// fn build( -/// &self, -/// props: HashMap, -/// extensions: Extensions, -/// ) -> Result> { -/// // Parse configuration and create storage -/// Ok(Arc::new(MyStorage::new(props)?)) -/// } -/// } -/// ``` -#[typetag::serde(tag = "type")] -pub trait StorageFactory: Debug + Send + Sync { - /// Create a new storage instance with the given properties and extensions. - /// - /// # Arguments - /// - /// * `props` - Configuration properties for the storage backend - /// * `extensions` - Additional extensions (e.g., custom credential loaders) - /// - /// # Returns - /// - /// An `Arc` that can be used for file operations. - fn build( - &self, - props: HashMap, - extensions: Extensions, - ) -> Result>; +/// This storage handles all supported schemes (S3, GCS, Azure, filesystem, memory) +/// through OpenDAL, creating operators on-demand based on the path scheme. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub enum OpenDALStorage { + /// In-memory storage, useful for testing + #[cfg(feature = "storage-memory")] + Memory { + /// Cached operator (lazily initialized) + #[serde(skip, default = "default_memory_op")] + op: Arc>>, + }, + /// Local filesystem storage + #[cfg(feature = "storage-fs")] + LocalFs, + /// Amazon S3 storage + #[cfg(feature = "storage-s3")] + S3 { + /// The configured scheme (s3 or s3a) + configured_scheme: String, + /// S3 configuration + config: opendal::services::S3Config, + /// Optional custom credential loader + #[serde(skip)] + customized_credential_load: Option, + }, + /// Google Cloud Storage + #[cfg(feature = "storage-gcs")] + Gcs { + /// GCS configuration + config: opendal::services::GcsConfig, + }, + /// Alibaba Cloud OSS + #[cfg(feature = "storage-oss")] + Oss { + /// OSS configuration + config: opendal::services::OssConfig, + }, + /// Azure Data Lake Storage + #[cfg(feature = "storage-azdls")] + Azdls { + /// The configured scheme (abfs, abfss, wasb, wasbs) + configured_scheme: super::AzureStorageScheme, + /// Azure DLS configuration + config: opendal::services::AzdlsConfig, + }, } -/// A registry of storage factories. -/// -/// The registry allows you to register custom storage factories for different URI schemes. -/// By default, it includes factories for all enabled storage features. -/// -/// # Example -/// -/// ```rust -/// use iceberg::io::StorageRegistry; -/// -/// // Create a new registry with default factories -/// let registry = StorageRegistry::new(); -/// -/// // Get supported storage types -/// let types = registry.supported_types(); -/// println!("Supported storage types: {:?}", types); -/// -/// // Get a factory for a specific scheme -/// # #[cfg(feature = "storage-memory")] -/// # { -/// let factory = registry.get_factory("memory").unwrap(); -/// # } -/// ``` -/// -/// You can also register custom storage factories: -/// -/// ```rust,ignore -/// use std::sync::Arc; -/// use iceberg::io::{StorageRegistry, StorageFactory}; -/// -/// let mut registry = StorageRegistry::new(); -/// -/// // Register a custom storage factory -/// registry.register("custom", Arc::new(MyCustomStorageFactory)); -/// ``` -#[derive(Debug, Clone)] -pub struct StorageRegistry { - factories: HashMap>, +#[cfg(feature = "storage-memory")] +fn default_memory_op() -> Arc>> { + Arc::new(std::sync::Mutex::new(None)) } -impl StorageRegistry { - /// Create a new storage registry with default factories based on enabled features. - pub fn new() -> Self { - let mut factories: HashMap> = HashMap::new(); - - #[cfg(feature = "storage-memory")] - { - use crate::io::storage_memory::OpenDALMemoryStorageFactory; - let factory = Arc::new(OpenDALMemoryStorageFactory) as Arc; - factories.insert("memory".to_string(), factory); +impl OpenDALStorage { + /// Build storage from FileIOBuilder + pub fn build(file_io_builder: FileIOBuilder) -> Result { + let (scheme_str, props, extensions) = file_io_builder.into_parts(); + let _ = (&props, &extensions); + + match scheme_str.to_lowercase().as_str() { + #[cfg(feature = "storage-memory")] + "memory" => Ok(OpenDALStorage::Memory { + op: Arc::new(std::sync::Mutex::new(None)), + }), + + #[cfg(feature = "storage-fs")] + "file" | "" => Ok(OpenDALStorage::LocalFs), + + #[cfg(feature = "storage-s3")] + "s3" | "s3a" => { + let config = super::s3_config_parse(props)?; + let customized_credential_load = extensions + .get::() + .map(Arc::unwrap_or_clone); + Ok(OpenDALStorage::S3 { + configured_scheme: scheme_str, + config, + customized_credential_load, + }) + } + + #[cfg(feature = "storage-gcs")] + "gs" | "gcs" => { + let config = super::gcs_config_parse(props)?; + Ok(OpenDALStorage::Gcs { config }) + } + + #[cfg(feature = "storage-oss")] + "oss" => { + let config = super::oss_config_parse(props)?; + Ok(OpenDALStorage::Oss { config }) + } + + #[cfg(feature = "storage-azdls")] + "abfs" | "abfss" | "wasb" | "wasbs" => { + let configured_scheme = scheme_str.parse::()?; + let config = super::azdls_config_parse(props)?; + Ok(OpenDALStorage::Azdls { + configured_scheme, + config, + }) + } + + _ => Err(Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Constructing file io from scheme: {} not supported now", + scheme_str + ), + )), } + } - #[cfg(feature = "storage-fs")] - { - use crate::io::storage_fs::OpenDALFsStorageFactory; - let factory = Arc::new(OpenDALFsStorageFactory) as Arc; - factories.insert("file".to_string(), factory.clone()); - factories.insert("".to_string(), factory); - } + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let (operator, relative_path): (Operator, &str) = match self { + #[cfg(feature = "storage-memory")] + OpenDALStorage::Memory { op } => { + let mut guard = op.lock().unwrap(); + if guard.is_none() { + *guard = Some(super::memory_config_build()?); + } + let op = guard.as_ref().unwrap().clone(); + + if let Some(stripped) = path.strip_prefix("memory:/") { + Ok((op, stripped)) + } else { + Ok((op, &path[1..])) + } + } + + #[cfg(feature = "storage-fs")] + OpenDALStorage::LocalFs => { + let op = super::fs_config_build()?; + + if let Some(stripped) = path.strip_prefix("file:/") { + Ok((op, stripped)) + } else { + Ok((op, &path[1..])) + } + } + + #[cfg(feature = "storage-s3")] + OpenDALStorage::S3 { + configured_scheme, + config, + customized_credential_load, + } => { + let op = super::s3_config_build(config, customized_credential_load, path)?; + let op_info = op.info(); + + let prefix = format!("{}://{}/", configured_scheme, op_info.name()); + if path.starts_with(&prefix) { + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid s3 url: {path}, should start with {prefix}"), + )) + } + } + + #[cfg(feature = "storage-gcs")] + OpenDALStorage::Gcs { config } => { + let operator = super::gcs_config_build(config, path)?; + let prefix = format!("gs://{}/", operator.info().name()); + if path.starts_with(&prefix) { + Ok((operator, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid gcs url: {path}, should start with {prefix}"), + )) + } + } + + #[cfg(feature = "storage-oss")] + OpenDALStorage::Oss { config } => { + let op = super::oss_config_build(config, path)?; + let prefix = format!("oss://{}/", op.info().name()); + if path.starts_with(&prefix) { + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid oss url: {path}, should start with {prefix}"), + )) + } + } + + #[cfg(feature = "storage-azdls")] + OpenDALStorage::Azdls { + configured_scheme, + config, + } => super::azdls_create_operator(path, config, configured_scheme), + }?; + + // Transient errors are common for object stores + let operator = operator.layer(RetryLayer::new()); + + Ok((operator, relative_path)) + } +} - #[cfg(feature = "storage-s3")] - { - use crate::io::storage_s3::OpenDALS3StorageFactory; - let factory = Arc::new(OpenDALS3StorageFactory) as Arc; - factories.insert("s3".to_string(), factory.clone()); - factories.insert("s3a".to_string(), factory); - } +#[async_trait] +#[typetag::serde] +impl Storage for OpenDALStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } - #[cfg(feature = "storage-gcs")] - { - use crate::io::storage_gcs::OpenDALGcsStorageFactory; - let factory = Arc::new(OpenDALGcsStorageFactory) as Arc; - factories.insert("gs".to_string(), factory.clone()); - factories.insert("gcs".to_string(), factory); - } + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } - #[cfg(feature = "storage-oss")] - { - use crate::io::storage_oss::OpenDALOssStorageFactory; - let factory = Arc::new(OpenDALOssStorageFactory) as Arc; - factories.insert("oss".to_string(), factory); - } + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } - #[cfg(feature = "storage-azdls")] - { - use crate::io::storage_azdls::OpenDALAzdlsStorageFactory; - let factory = Arc::new(OpenDALAzdlsStorageFactory) as Arc; - factories.insert("abfs".to_string(), factory.clone()); - factories.insert("abfss".to_string(), factory.clone()); - factories.insert("wasb".to_string(), factory.clone()); - factories.insert("wasbs".to_string(), factory); - } + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } - Self { factories } + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await } - /// Register a custom storage factory for a given scheme. - pub fn register(&mut self, scheme: impl Into, factory: Arc) { - self.factories.insert(scheme.into(), factory); + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) } - /// Get a storage factory by scheme. - pub fn get_factory(&self, scheme: &str) -> Result> { - let key = scheme.trim(); - self.factories - .iter() - .find(|(k, _)| k.eq_ignore_ascii_case(key)) - .map(|(_, factory)| factory.clone()) - .ok_or_else(|| { - use crate::{Error, ErrorKind}; - Error::new( - ErrorKind::FeatureUnsupported, - format!( - "Unsupported storage type: {}. Supported types: {}", - scheme, - self.supported_types().join(", ") - ), - ) - }) + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) } - /// Return the list of supported storage types. - pub fn supported_types(&self) -> Vec { - self.factories.keys().cloned().collect() + async fn delete_prefix(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) } -} -impl Default for StorageRegistry { - fn default() -> Self { - Self::new() + fn new_input(&self, path: &str) -> Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } #[cfg(test)] mod tests { use super::*; + use crate::io::FileIOBuilder; #[test] - fn test_storage_registry_new() { - let registry = StorageRegistry::new(); - let types = registry.supported_types(); - - // At least one storage type should be available - assert!(!types.is_empty()); + #[cfg(feature = "storage-memory")] + fn test_opendal_storage_memory() { + let builder = FileIOBuilder::new("memory"); + let storage = OpenDALStorage::build(builder).unwrap(); + assert!(matches!(storage, OpenDALStorage::Memory { .. })); } #[test] - #[cfg(feature = "storage-memory")] - fn test_storage_registry_get_factory() { - let registry = StorageRegistry::new(); - - // Should be able to get memory storage factory - let factory = registry.get_factory("memory"); - assert!(factory.is_ok()); - - // Should be case-insensitive - let factory = registry.get_factory("MEMORY"); - assert!(factory.is_ok()); + #[cfg(feature = "storage-fs")] + fn test_opendal_storage_fs() { + let builder = FileIOBuilder::new("file"); + let storage = OpenDALStorage::build(builder).unwrap(); + assert!(matches!(storage, OpenDALStorage::LocalFs)); } #[test] - fn test_storage_registry_unsupported_type() { - let registry = StorageRegistry::new(); - - // Should return error for unsupported type - let result = registry.get_factory("unsupported"); - assert!(result.is_err()); + #[cfg(feature = "storage-s3")] + fn test_opendal_storage_s3() { + let builder = FileIOBuilder::new("s3"); + let storage = OpenDALStorage::build(builder).unwrap(); + assert!(matches!(storage, OpenDALStorage::S3 { .. })); } #[test] #[cfg(feature = "storage-memory")] - fn test_storage_registry_clone() { - let registry = StorageRegistry::new(); - let cloned = registry.clone(); - - // Both should have the same factories - assert_eq!( - registry.supported_types().len(), - cloned.supported_types().len() - ); + fn test_storage_serialization() { + let builder = FileIOBuilder::new("memory"); + let storage = OpenDALStorage::build(builder).unwrap(); + + // Serialize + let serialized = serde_json::to_string(&storage).unwrap(); + + // Deserialize + let deserialized: OpenDALStorage = serde_json::from_str(&serialized).unwrap(); + + assert!(matches!(deserialized, OpenDALStorage::Memory { .. })); } } diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index e4f3567e54..0065bb91e7 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -18,19 +18,11 @@ use std::collections::HashMap; use std::fmt::Display; use std::str::FromStr; -use std::sync::Arc; -use async_trait::async_trait; -use bytes::Bytes; +use opendal::Configurator; use opendal::services::AzdlsConfig; -use opendal::{Configurator, Operator}; -use serde::{Deserialize, Serialize}; use url::Url; -use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, - Storage, StorageFactory, -}; use crate::{Error, ErrorKind, Result, ensure_data_valid}; /// A connection string. @@ -128,21 +120,27 @@ pub(crate) fn azdls_create_operator<'a>( Ok((op, relative_path)) } +/// Azure Storage scheme variants. +/// /// Note that `abf[s]` and `wasb[s]` variants have different implications: /// - `abfs[s]` is used to refer to files in ADLS Gen2, backed by blob storage; /// paths are expected to contain the `dfs` storage service. /// - `wasb[s]` is used to refer to files in Blob Storage directly; paths are /// expected to contain the `blob` storage service. -#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] -pub(crate) enum AzureStorageScheme { +#[derive(Debug, Clone, PartialEq, serde::Serialize, serde::Deserialize)] +pub enum AzureStorageScheme { + /// Azure Blob File System (unencrypted) Abfs, + /// Azure Blob File System (encrypted/secure) Abfss, + /// Windows Azure Storage Blob (unencrypted) Wasb, + /// Windows Azure Storage Blob (encrypted/secure) Wasbs, } impl AzureStorageScheme { - // Returns the respective encrypted or plain-text HTTP scheme. + /// Returns the respective encrypted or plain-text HTTP scheme. pub fn as_http_scheme(&self) -> &str { match self { AzureStorageScheme::Abfs | AzureStorageScheme::Wasb => "http", @@ -605,115 +603,3 @@ mod tests { } } } - -/// Azure Data Lake Storage implementation using OpenDAL -/// -/// Stores configuration and creates operators on-demand. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct OpenDALAzdlsStorage { - /// Because Azdls accepts multiple possible schemes, we store the full - /// passed scheme here to later validate schemes passed via paths. - configured_scheme: AzureStorageScheme, - config: Arc, -} - -impl OpenDALAzdlsStorage { - /// Creates operator from path. - fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { - let (op, relative_path) = - azdls_create_operator(path, &self.config, &self.configured_scheme)?; - let op = op.layer(opendal::layers::RetryLayer::new()); - Ok((op, relative_path)) - } -} - -#[async_trait] -#[typetag::serde] -impl Storage for OpenDALAzdlsStorage { - async fn exists(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.exists(relative_path).await?) - } - - async fn metadata(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - let meta = op.stat(relative_path).await?; - Ok(FileMetadata { - size: meta.content_length(), - }) - } - - async fn read(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.read(relative_path).await?.to_bytes()) - } - - async fn reader(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.reader(relative_path).await?)) - } - - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } - - async fn writer(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.writer(relative_path).await?)) - } - - async fn delete(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.delete(relative_path).await?) - } - - async fn delete_prefix(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } -} - -/// Factory for Azure Data Lake Storage -#[derive(Debug, Serialize, Deserialize)] -pub struct OpenDALAzdlsStorageFactory; - -#[typetag::serde] -impl StorageFactory for OpenDALAzdlsStorageFactory { - fn build( - &self, - props: HashMap, - _extensions: Extensions, - ) -> Result> { - // Get the scheme string from the props or use default - let scheme_str = props - .get(STORAGE_LOCATION_SCHEME) - .cloned() - .unwrap_or_else(|| "abfs".to_string()); - - // Parse the scheme - let scheme = scheme_str.parse::()?; - - // Parse Azdls config from props - let config = azdls_config_parse(props)?; - - Ok(Arc::new(OpenDALAzdlsStorage { - configured_scheme: scheme, - config: Arc::new(config), - })) - } -} diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index b668016f76..d3e121a085 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -15,19 +15,10 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashMap; -use std::sync::Arc; - -use async_trait::async_trait; -use bytes::Bytes; use opendal::Operator; use opendal::services::FsConfig; -use serde::{Deserialize, Serialize}; use crate::Result; -use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, -}; /// Build new opendal operator from give path. pub(crate) fn fs_config_build() -> Result { @@ -36,144 +27,3 @@ pub(crate) fn fs_config_build() -> Result { Ok(Operator::from_config(cfg)?.finish()) } - -/// Filesystem storage implementation using OpenDAL -/// -/// This storage is stateless and creates operators on-demand. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct OpenDALFsStorage; - -impl OpenDALFsStorage { - /// Extract relative path from file:// URLs - fn extract_relative_path<'a>(&self, path: &'a str) -> &'a str { - if let Some(stripped) = path.strip_prefix("file:/") { - stripped - } else { - &path[1..] - } - } -} - -#[async_trait] -#[typetag::serde] -impl Storage for OpenDALFsStorage { - async fn exists(&self, path: &str) -> Result { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - Ok(op.exists(relative_path).await?) - } - - async fn metadata(&self, path: &str) -> Result { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - let meta = op.stat(relative_path).await?; - Ok(FileMetadata { - size: meta.content_length(), - }) - } - - async fn read(&self, path: &str) -> Result { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - Ok(op.read(relative_path).await?.to_bytes()) - } - - async fn reader(&self, path: &str) -> Result> { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - Ok(Box::new(op.reader(relative_path).await?)) - } - - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } - - async fn writer(&self, path: &str) -> Result> { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - Ok(Box::new(op.writer(relative_path).await?)) - } - - async fn delete(&self, path: &str) -> Result<()> { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - Ok(op.delete(relative_path).await?) - } - - async fn delete_prefix(&self, path: &str) -> Result<()> { - let relative_path = self.extract_relative_path(path); - let op = fs_config_build()?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } -} - -/// Factory for OpenDAL Filesystem storage -#[derive(Debug, Serialize, Deserialize)] -pub struct OpenDALFsStorageFactory; - -#[typetag::serde] -impl StorageFactory for OpenDALFsStorageFactory { - fn build( - &self, - _props: HashMap, - _extensions: Extensions, - ) -> Result> { - Ok(Arc::new(OpenDALFsStorage)) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::io::Storage; - - #[test] - fn test_fs_storage_serialization() { - // Create a filesystem storage instance using the factory - let factory = OpenDALFsStorageFactory; - let storage = factory - .build(HashMap::new(), Extensions::default()) - .unwrap(); - - // Serialize the storage - let serialized = serde_json::to_string(&storage).unwrap(); - - // Deserialize the storage - let deserialized: Box = serde_json::from_str(&serialized).unwrap(); - - // Verify the type is correct - assert!(format!("{deserialized:?}").contains("OpenDALFsStorage")); - } - - #[test] - fn test_fs_factory_serialization() { - use crate::io::StorageFactory; - - // Create a factory instance - let factory: Box = Box::new(OpenDALFsStorageFactory); - - // Serialize the factory - let serialized = serde_json::to_string(&factory).unwrap(); - - // Deserialize the factory - let deserialized: Box = serde_json::from_str(&serialized).unwrap(); - - // Verify the type is correct - assert!(format!("{deserialized:?}").contains("OpenDALFsStorageFactory")); - } -} diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 4cd13453a9..7718df603f 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -17,19 +17,12 @@ //! Google Cloud Storage properties use std::collections::HashMap; -use std::sync::Arc; -use async_trait::async_trait; -use bytes::Bytes; use opendal::Operator; use opendal::services::GcsConfig; -use serde::{Deserialize, Serialize}; use url::Url; -use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, - is_truthy, -}; +use crate::io::is_truthy; use crate::{Error, ErrorKind, Result}; // Reference: https://github.com/apache/iceberg/blob/main/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -111,108 +104,3 @@ pub(crate) fn gcs_config_build(cfg: &GcsConfig, path: &str) -> Result cfg.bucket = bucket.to_string(); Ok(Operator::from_config(cfg)?.finish()) } - -/// GCS storage implementation using OpenDAL -/// -/// Stores configuration and creates operators on-demand. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct OpenDALGcsStorage { - config: Arc, -} - -impl OpenDALGcsStorage { - /// Creates operator from path. - fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { - let operator = gcs_config_build(&self.config, path)?; - let prefix = format!("gs://{}/", operator.info().name()); - - if path.starts_with(&prefix) { - let op = operator.layer(opendal::layers::RetryLayer::new()); - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid gcs url: {path}, should start with {prefix}"), - )) - } - } -} - -#[async_trait] -#[typetag::serde] -impl Storage for OpenDALGcsStorage { - async fn exists(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.exists(relative_path).await?) - } - - async fn metadata(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - let meta = op.stat(relative_path).await?; - Ok(FileMetadata { - size: meta.content_length(), - }) - } - - async fn read(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.read(relative_path).await?.to_bytes()) - } - - async fn reader(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.reader(relative_path).await?)) - } - - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } - - async fn writer(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.writer(relative_path).await?)) - } - - async fn delete(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.delete(relative_path).await?) - } - - async fn delete_prefix(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } -} - -/// Factory for OpenDAL GCS storage -#[derive(Debug, Serialize, Deserialize)] -pub struct OpenDALGcsStorageFactory; - -#[typetag::serde] -impl StorageFactory for OpenDALGcsStorageFactory { - fn build( - &self, - props: HashMap, - _extensions: Extensions, - ) -> Result> { - let cfg = gcs_config_parse(props)?; - Ok(Arc::new(OpenDALGcsStorage { - config: Arc::new(cfg), - })) - } -} diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index fc57fc7587..b8023717b6 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -15,214 +15,11 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashMap; -use std::sync::{Arc, Mutex}; - -use async_trait::async_trait; -use bytes::Bytes; use opendal::Operator; use opendal::services::MemoryConfig; -use serde::{Deserialize, Serialize}; use crate::Result; -use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, -}; - -/// Memory storage implementation using OpenDAL -/// -/// Uses lazy initialization - the operator is created on first use and then cached. -/// This allows the storage to be serialized/deserialized while maintaining state. -/// The operator field is skipped during serialization and recreated on first use. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct OpenDALMemoryStorage { - #[serde(skip, default = "default_op")] - op: Arc>>, -} - -fn default_op() -> Arc>> { - Arc::new(Mutex::new(None)) -} - -impl Default for OpenDALMemoryStorage { - fn default() -> Self { - Self { - op: Arc::new(Mutex::new(None)), - } - } -} - -impl OpenDALMemoryStorage { - /// Get or create the memory operator (lazy initialization) - fn get_operator(&self) -> Result { - let mut guard = self.op.lock().unwrap(); - if guard.is_none() { - *guard = Some(Operator::from_config(MemoryConfig::default())?.finish()); - } - Ok(guard.as_ref().unwrap().clone()) - } - - /// Extract relative path from memory:// URLs - fn extract_relative_path<'a>(&self, path: &'a str) -> &'a str { - if let Some(stripped) = path.strip_prefix("memory:/") { - stripped - } else { - &path[1..] - } - } -} - -#[async_trait] -#[typetag::serde] -impl Storage for OpenDALMemoryStorage { - async fn exists(&self, path: &str) -> Result { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - Ok(op.exists(relative_path).await?) - } - - async fn metadata(&self, path: &str) -> Result { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - let meta = op.stat(relative_path).await?; - Ok(FileMetadata { - size: meta.content_length(), - }) - } - - async fn read(&self, path: &str) -> Result { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - Ok(op.read(relative_path).await?.to_bytes()) - } - - async fn reader(&self, path: &str) -> Result> { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - Ok(Box::new(op.reader(relative_path).await?)) - } - - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } - - async fn writer(&self, path: &str) -> Result> { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - Ok(Box::new(op.writer(relative_path).await?)) - } - - async fn delete(&self, path: &str) -> Result<()> { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - Ok(op.delete(relative_path).await?) - } - - async fn delete_prefix(&self, path: &str) -> Result<()> { - let op = self.get_operator()?; - let relative_path = self.extract_relative_path(path); - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } -} - -/// Factory for OpenDAL Memory storage -#[derive(Debug, Serialize, Deserialize)] -pub struct OpenDALMemoryStorageFactory; - -#[typetag::serde] -impl StorageFactory for OpenDALMemoryStorageFactory { - fn build( - &self, - _props: HashMap, - _extensions: Extensions, - ) -> Result> { - Ok(Arc::new(OpenDALMemoryStorage::default())) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::io::Storage; - - #[test] - fn test_memory_storage_serialization() { - // Create a memory storage instance using the factory - let factory = OpenDALMemoryStorageFactory; - let storage = factory - .build(HashMap::new(), Extensions::default()) - .unwrap(); - - // Serialize the storage - let serialized = serde_json::to_string(&storage).unwrap(); - - // Deserialize the storage - let deserialized: Box = serde_json::from_str(&serialized).unwrap(); - - // Verify the type is correct - assert!(format!("{deserialized:?}").contains("OpenDALMemoryStorage")); - } - - #[test] - fn test_memory_factory_serialization() { - use crate::io::StorageFactory; - - // Create a factory instance - let factory: Box = Box::new(OpenDALMemoryStorageFactory); - - // Serialize the factory - let serialized = serde_json::to_string(&factory).unwrap(); - - // Deserialize the factory - let deserialized: Box = serde_json::from_str(&serialized).unwrap(); - - // Verify the type is correct - assert!(format!("{deserialized:?}").contains("OpenDALMemoryStorageFactory")); - } - - #[test] - fn test_memory_factory_to_storage_serialization() { - use crate::io::{Extensions, StorageFactory}; - - // Create a factory and build storage - let factory = OpenDALMemoryStorageFactory; - let storage = factory - .build(HashMap::new(), Extensions::default()) - .unwrap(); - - // Serialize the storage - let storage_json = serde_json::to_string(&storage).unwrap(); - - // Deserialize the storage - let deserialized_storage: Box = serde_json::from_str(&storage_json).unwrap(); - - // Verify storage type - assert!(format!("{deserialized_storage:?}").contains("OpenDALMemoryStorage")); - - // Serialize the factory - let factory_boxed: Box = Box::new(factory); - let factory_json = serde_json::to_string(&factory_boxed).unwrap(); - - // Deserialize the factory - let deserialized_factory: Box = - serde_json::from_str(&factory_json).unwrap(); - // Verify factory type - assert!(format!("{deserialized_factory:?}").contains("OpenDALMemoryStorageFactory")); - } +pub(crate) fn memory_config_build() -> Result { + Ok(Operator::from_config(MemoryConfig::default())?.finish()) } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 132b8ac349..e82dda23a5 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -16,18 +16,11 @@ // under the License. use std::collections::HashMap; -use std::sync::Arc; -use async_trait::async_trait; -use bytes::Bytes; use opendal::services::OssConfig; use opendal::{Configurator, Operator}; -use serde::{Deserialize, Serialize}; use url::Url; -use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, -}; use crate::{Error, ErrorKind, Result}; /// Required configuration arguments for creating an Aliyun OSS Operator with OpenDAL: @@ -71,108 +64,3 @@ pub(crate) fn oss_config_build(cfg: &OssConfig, path: &str) -> Result Ok(Operator::new(builder)?.finish()) } - -/// OSS storage implementation using OpenDAL -/// -/// Stores configuration and creates operators on-demand. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct OpenDALOssStorage { - config: Arc, -} - -impl OpenDALOssStorage { - /// Creates operator from path. - fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { - let op = oss_config_build(&self.config, path)?; - let prefix = format!("oss://{}/", op.info().name()); - - if path.starts_with(&prefix) { - let op = op.layer(opendal::layers::RetryLayer::new()); - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid oss url: {path}, should start with {prefix}"), - )) - } - } -} - -#[async_trait] -#[typetag::serde] -impl Storage for OpenDALOssStorage { - async fn exists(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.exists(relative_path).await?) - } - - async fn metadata(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - let meta = op.stat(relative_path).await?; - Ok(FileMetadata { - size: meta.content_length(), - }) - } - - async fn read(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.read(relative_path).await?.to_bytes()) - } - - async fn reader(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.reader(relative_path).await?)) - } - - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } - - async fn writer(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.writer(relative_path).await?)) - } - - async fn delete(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.delete(relative_path).await?) - } - - async fn delete_prefix(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } -} - -/// Factory for OpenDAL OSS storage -#[derive(Debug, Serialize, Deserialize)] -pub struct OpenDALOssStorageFactory; - -#[typetag::serde] -impl StorageFactory for OpenDALOssStorageFactory { - fn build( - &self, - props: HashMap, - _extensions: Extensions, - ) -> Result> { - let cfg = oss_config_parse(props)?; - Ok(Arc::new(OpenDALOssStorage { - config: Arc::new(cfg), - })) - } -} diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 3558b6a4b1..f069e0e2f9 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -19,18 +19,13 @@ use std::collections::HashMap; use std::sync::Arc; use async_trait::async_trait; -use bytes::Bytes; use opendal::services::S3Config; use opendal::{Configurator, Operator}; pub use reqsign::{AwsCredential, AwsCredentialLoad}; use reqwest::Client; -use serde::{Deserialize, Serialize}; use url::Url; -use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, - Storage, StorageFactory, is_truthy, -}; +use crate::io::is_truthy; use crate::{Error, ErrorKind, Result}; /// Following are arguments for [s3 file io](https://py.iceberg.apache.org/configuration/#s3). @@ -218,187 +213,3 @@ impl AwsCredentialLoad for CustomAwsCredentialLoader { self.0.load_credential(client).await } } - -/// S3 storage implementation using OpenDAL -/// -/// Stores configuration and creates operators on-demand. -/// The `customized_credential_load` field is not serialized. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct OpenDALS3Storage { - /// s3 storage could have `s3://` and `s3a://`. - /// Storing the scheme string here to return the correct path. - configured_scheme: String, - config: Arc, - #[serde(skip)] - customized_credential_load: Option, -} - -impl OpenDALS3Storage { - /// Creates operator from path. - /// - /// # Arguments - /// - /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. - /// - /// # Returns - /// - /// The return value consists of two parts: - /// - /// * An [`opendal::Operator`] instance used to operate on file. - /// * Relative path to the root uri of [`opendal::Operator`]. - fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { - let op = s3_config_build(&self.config, &self.customized_credential_load, path)?; - let op_info = op.info(); - - // Check prefix of s3 path. - let prefix = format!("{}://{}/", self.configured_scheme, op_info.name()); - if path.starts_with(&prefix) { - // Add retry layer for transient errors - let op = op.layer(opendal::layers::RetryLayer::new()); - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid s3 url: {path}, should start with {prefix}"), - )) - } - } -} - -#[async_trait] -#[typetag::serde] -impl Storage for OpenDALS3Storage { - async fn exists(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.exists(relative_path).await?) - } - - async fn metadata(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - let meta = op.stat(relative_path).await?; - - Ok(FileMetadata { - size: meta.content_length(), - }) - } - - async fn read(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.read(relative_path).await?.to_bytes()) - } - - async fn reader(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.reader(relative_path).await?)) - } - - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } - - async fn writer(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(path)?; - Ok(Box::new(op.writer(relative_path).await?)) - } - - async fn delete(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - Ok(op.delete(relative_path).await?) - } - - async fn delete_prefix(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) - } - - fn new_input(&self, path: &str) -> Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } -} - -/// Factory for S3 storage -#[derive(Debug, Serialize, Deserialize)] -pub struct OpenDALS3StorageFactory; - -#[typetag::serde] -impl StorageFactory for OpenDALS3StorageFactory { - fn build( - &self, - props: HashMap, - extensions: Extensions, - ) -> Result> { - // Get the scheme string from the props or use "s3" as default - let scheme_str = props - .get(STORAGE_LOCATION_SCHEME) - .cloned() - .unwrap_or_else(|| "s3".to_string()); - - // Parse S3 config from props - let config = s3_config_parse(props)?; - - // Get customized credential loader from extensions if available - let customized_credential_load = extensions - .get::() - .map(Arc::unwrap_or_clone); - - Ok(Arc::new(OpenDALS3Storage { - configured_scheme: scheme_str, - config: Arc::new(config), - customized_credential_load, - })) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::io::{STORAGE_LOCATION_SCHEME, Storage}; - - #[test] - fn test_s3_storage_serialization() { - // Create an S3 storage instance using the factory - let factory = OpenDALS3StorageFactory; - let mut props = HashMap::new(); - props.insert(S3_REGION.to_string(), "us-east-1".to_string()); - props.insert(STORAGE_LOCATION_SCHEME.to_string(), "s3".to_string()); - - let storage = factory.build(props, Extensions::default()).unwrap(); - - // Serialize the storage - let serialized = serde_json::to_string(&storage).unwrap(); - - // Deserialize the storage - let deserialized: Box = serde_json::from_str(&serialized).unwrap(); - - // Verify the type is correct - assert!(format!("{deserialized:?}").contains("OpenDALS3Storage")); - } - - #[test] - fn test_s3_factory_serialization() { - use crate::io::StorageFactory; - - // Create a factory instance - let factory: Box = Box::new(OpenDALS3StorageFactory); - - // Serialize the factory - let serialized = serde_json::to_string(&factory).unwrap(); - - // Deserialize the factory - let deserialized: Box = serde_json::from_str(&serialized).unwrap(); - - // Verify the type is correct - assert!(format!("{deserialized:?}").contains("OpenDALS3StorageFactory")); - } -} From 80cf08675f4e3eed10a169dd367210b51aed9c8c Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 10:38:41 -0800 Subject: [PATCH 05/12] clean up and keep it simple --- crates/iceberg/src/io/file_io.rs | 41 +++++++------------------- crates/iceberg/src/io/mod.rs | 1 - crates/iceberg/src/io/storage.rs | 4 ++- crates/iceberg/src/io/storage_azdls.rs | 10 ++----- 4 files changed, 17 insertions(+), 39 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 6d43222503..193d7f99f1 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -21,30 +21,21 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; -use async_trait::async_trait; use bytes::Bytes; use url::Url; -pub use super::storage::{OpenDALStorage, Storage}; +pub use super::storage::Storage; +use super::storage::OpenDALStorage; use crate::{Error, ErrorKind, Result}; -/// FileIO implementation, used to manipulate files in underlying storage. -/// -/// # Note -/// -/// All path passed to `FileIO` must be absolute path starting with scheme string used to construct `FileIO`. -/// For example, if you construct `FileIO` with `s3a` scheme, then all path passed to `FileIO` must start with `s3a://`. -/// -/// Supported storages: -/// -/// | Storage | Feature Flag | Schemes | -/// |--------------------|-------------------|----------------------------------| -/// | Local file system | `storage-fs` | `file://path/to/file` | -/// | Memory | `storage-memory` | `memory://path/to/file` | -/// | S3 | `storage-s3` | `s3:///path/to/file` | -/// | GCS | `storage-gcs` | `gs:///path/to/file` | -/// | OSS | `storage-oss` | `oss:///path/to/file` | -/// | Azure Datalake | `storage-azdls` | `abfs[s]://...` or `wasb[s]://...` | +/// | Storage | Feature Flag | Expected Path Format | Schemes | +/// |--------------------|-------------------|----------------------------------| ------------------------------| +/// | Local file system | `storage-fs` | `file` | `file://path/to/file` | +/// | Memory | `storage-memory` | `memory` | `memory://path/to/file` | +/// | S3 | `storage-s3` | `s3`, `s3a` | `s3:///path/to/file` | +/// | GCS | `storage-gcs` | `gs`, `gcs` | `gs:///path/to/file` | +/// | OSS | `storage-oss` | `oss` | `oss:///path/to/file` | +/// | Azure Datalake | `storage-azdls` | `abfs`, `abfss`, `wasb`, `wasbs` | `abfs://@.dfs.core.windows.net/path/to/file` or `wasb://@.blob.core.windows.net/path/to/file` | #[derive(Clone, Debug)] pub struct FileIO { builder: FileIOBuilder, @@ -295,11 +286,6 @@ impl InputFile { Self { storage, path } } - /// Returns the storage backend for this input file. - pub fn storage(&self) -> &Arc { - &self.storage - } - /// Absolute path to root uri. pub fn location(&self) -> &str { &self.path @@ -336,7 +322,7 @@ impl InputFile { /// /// It's possible for us to remove the async_trait, but we need to figure /// out how to handle the object safety. -#[async_trait] +#[async_trait::async_trait] pub trait FileWrite: Send + Sync + Unpin + 'static { /// Write bytes to file. /// @@ -390,11 +376,6 @@ impl OutputFile { Self { storage, path } } - /// Returns the storage backend for this output file. - pub fn storage(&self) -> &Arc { - &self.storage - } - /// Relative path to root uri. pub fn location(&self) -> &str { &self.path diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 7e01d9b78b..e582fe2e9b 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -70,7 +70,6 @@ mod file_io; mod storage; pub use file_io::*; -pub use storage::{OpenDALStorage, Storage}; pub(crate) mod object_cache; #[cfg(feature = "storage-azdls")] diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 656d390f1f..a8f0265737 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -39,6 +39,8 @@ use crate::{Error, ErrorKind, Result}; /// /// The trait supports serialization via `typetag`, allowing storage instances to be /// serialized and deserialized across process boundaries. +/// +/// Third-party implementations can implement this trait to provide custom storage backends. #[async_trait] #[typetag::serde(tag = "type")] pub trait Storage: Debug + Send + Sync { @@ -78,7 +80,7 @@ pub trait Storage: Debug + Send + Sync { /// This storage handles all supported schemes (S3, GCS, Azure, filesystem, memory) /// through OpenDAL, creating operators on-demand based on the path scheme. #[derive(Debug, Clone, Serialize, Deserialize)] -pub enum OpenDALStorage { +pub(crate) enum OpenDALStorage { /// In-memory storage, useful for testing #[cfg(feature = "storage-memory")] Memory { diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 0065bb91e7..0741496f84 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -128,20 +128,16 @@ pub(crate) fn azdls_create_operator<'a>( /// - `wasb[s]` is used to refer to files in Blob Storage directly; paths are /// expected to contain the `blob` storage service. #[derive(Debug, Clone, PartialEq, serde::Serialize, serde::Deserialize)] -pub enum AzureStorageScheme { - /// Azure Blob File System (unencrypted) +pub(crate) enum AzureStorageScheme { Abfs, - /// Azure Blob File System (encrypted/secure) Abfss, - /// Windows Azure Storage Blob (unencrypted) Wasb, - /// Windows Azure Storage Blob (encrypted/secure) Wasbs, } impl AzureStorageScheme { - /// Returns the respective encrypted or plain-text HTTP scheme. - pub fn as_http_scheme(&self) -> &str { + // Returns the respective encrypted or plain-text HTTP scheme. + pub(crate) fn as_http_scheme(&self) -> &str { match self { AzureStorageScheme::Abfs | AzureStorageScheme::Wasb => "http", AzureStorageScheme::Abfss | AzureStorageScheme::Wasbs => "https", From f2ae4921c4346e2e891169695adbc764792a8d9a Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 11:29:32 -0800 Subject: [PATCH 06/12] more clean up --- crates/iceberg/src/io/file_io.rs | 2 +- crates/iceberg/src/io/storage.rs | 70 +++++++++++++++++++++++++------- 2 files changed, 57 insertions(+), 15 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 193d7f99f1..dd95f218a3 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -24,8 +24,8 @@ use std::sync::Arc; use bytes::Bytes; use url::Url; -pub use super::storage::Storage; use super::storage::OpenDALStorage; +pub use super::storage::Storage; use crate::{Error, ErrorKind, Result}; /// | Storage | Feature Flag | Expected Path Format | Schemes | diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index a8f0265737..e6ce00969e 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -26,7 +26,15 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; use opendal::layers::RetryLayer; -use opendal::Operator; +#[cfg(feature = "storage-azdls")] +use opendal::services::AzdlsConfig; +#[cfg(feature = "storage-gcs")] +use opendal::services::GcsConfig; +#[cfg(feature = "storage-oss")] +use opendal::services::OssConfig; +#[cfg(feature = "storage-s3")] +use opendal::services::S3Config; +use opendal::{Operator, Scheme}; use serde::{Deserialize, Serialize}; use super::{FileIOBuilder, FileMetadata, FileRead, FileWrite, InputFile, OutputFile}; @@ -134,18 +142,19 @@ impl OpenDALStorage { pub fn build(file_io_builder: FileIOBuilder) -> Result { let (scheme_str, props, extensions) = file_io_builder.into_parts(); let _ = (&props, &extensions); + let scheme = Self::parse_scheme(&scheme_str)?; - match scheme_str.to_lowercase().as_str() { + match scheme { #[cfg(feature = "storage-memory")] - "memory" => Ok(OpenDALStorage::Memory { + Scheme::Memory => Ok(OpenDALStorage::Memory { op: Arc::new(std::sync::Mutex::new(None)), }), #[cfg(feature = "storage-fs")] - "file" | "" => Ok(OpenDALStorage::LocalFs), + Scheme::Fs => Ok(OpenDALStorage::LocalFs), #[cfg(feature = "storage-s3")] - "s3" | "s3a" => { + Scheme::S3 => { let config = super::s3_config_parse(props)?; let customized_credential_load = extensions .get::() @@ -158,19 +167,19 @@ impl OpenDALStorage { } #[cfg(feature = "storage-gcs")] - "gs" | "gcs" => { + Scheme::Gcs => { let config = super::gcs_config_parse(props)?; Ok(OpenDALStorage::Gcs { config }) } #[cfg(feature = "storage-oss")] - "oss" => { + Scheme::Oss => { let config = super::oss_config_parse(props)?; Ok(OpenDALStorage::Oss { config }) } #[cfg(feature = "storage-azdls")] - "abfs" | "abfss" | "wasb" | "wasbs" => { + Scheme::Azdls => { let configured_scheme = scheme_str.parse::()?; let config = super::azdls_config_parse(props)?; Ok(OpenDALStorage::Azdls { @@ -178,18 +187,26 @@ impl OpenDALStorage { config, }) } - + // Update doc on [`FileIO`] when adding new schemes. _ => Err(Error::new( ErrorKind::FeatureUnsupported, - format!( - "Constructing file io from scheme: {} not supported now", - scheme_str - ), + format!("Constructing file io from scheme: {scheme} not supported now",), )), } } /// Creates operator from path. + /// + /// # Arguments + /// + /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + /// + /// # Returns + /// + /// The return value consists of two parts: + /// + /// * An [`opendal::Operator`] instance used to operate on file. + /// * Relative path to the root uri of [`opendal::Operator`]. fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { let (operator, relative_path): (Operator, &str) = match self { #[cfg(feature = "storage-memory")] @@ -271,13 +288,38 @@ impl OpenDALStorage { configured_scheme, config, } => super::azdls_create_operator(path, config, configured_scheme), + #[cfg(all( + not(feature = "storage-s3"), + not(feature = "storage-fs"), + not(feature = "storage-gcs"), + not(feature = "storage-oss"), + not(feature = "storage-azdls"), + ))] + _ => Err(Error::new( + ErrorKind::FeatureUnsupported, + "No storage service has been enabled", + )), }?; - // Transient errors are common for object stores + // Transient errors are common for object stores; however there's no + // harm in retrying temporary failures for other storage backends as well. let operator = operator.layer(RetryLayer::new()); Ok((operator, relative_path)) } + + /// Parse scheme. + fn parse_scheme(scheme: &str) -> crate::Result { + match scheme { + "memory" => Ok(Scheme::Memory), + "file" | "" => Ok(Scheme::Fs), + "s3" | "s3a" => Ok(Scheme::S3), + "gs" | "gcs" => Ok(Scheme::Gcs), + "oss" => Ok(Scheme::Oss), + "abfss" | "abfs" | "wasbs" | "wasb" => Ok(Scheme::Azdls), + s => Ok(s.parse::()?), + } + } } #[async_trait] From fd7c158d2a80db1897c858fec6c599b77b26fcf9 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 12:14:57 -0800 Subject: [PATCH 07/12] cleaner --- crates/iceberg/src/io/file_io.rs | 13 ++++++++- crates/iceberg/src/io/storage.rs | 49 ++++++++++++++------------------ 2 files changed, 33 insertions(+), 29 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index dd95f218a3..0be94391a3 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -28,6 +28,15 @@ use super::storage::OpenDALStorage; pub use super::storage::Storage; use crate::{Error, ErrorKind, Result}; +/// FileIO implementation, used to manipulate files in underlying storage. +/// +/// # Note +/// +/// All path passed to `FileIO` must be absolute path starting with scheme string used to construct `FileIO`. +/// For example, if you construct `FileIO` with `s3a` scheme, then all path passed to `FileIO` must start with `s3a://`. +/// +/// Supported storages: +/// /// | Storage | Feature Flag | Expected Path Format | Schemes | /// |--------------------|-------------------|----------------------------------| ------------------------------| /// | Local file system | `storage-fs` | `file` | `file://path/to/file` | @@ -424,6 +433,7 @@ mod tests { use std::io::Write; use std::path::Path; + use bytes::Bytes; use futures::AsyncReadExt; use futures::io::AllowStdIo; use tempfile::TempDir; @@ -461,6 +471,7 @@ mod tests { let input_file = file_io.new_input(&full_path).unwrap(); assert!(input_file.exists().await.unwrap()); + // Remove heading slash assert_eq!(&full_path, input_file.location()); let read_content = read_from_file(full_path).await; @@ -566,7 +577,7 @@ mod tests { assert!(io.exists(&path.clone()).await.unwrap()); let input_file = io.new_input(&path).unwrap(); let content = input_file.read().await.unwrap(); - assert_eq!(content, bytes::Bytes::from("test")); + assert_eq!(content, Bytes::from("test")); io.delete(&path).await.unwrap(); assert!(!io.exists(&path).await.unwrap()); diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index e6ce00969e..93e8f924dd 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -91,11 +91,7 @@ pub trait Storage: Debug + Send + Sync { pub(crate) enum OpenDALStorage { /// In-memory storage, useful for testing #[cfg(feature = "storage-memory")] - Memory { - /// Cached operator (lazily initialized) - #[serde(skip, default = "default_memory_op")] - op: Arc>>, - }, + Memory(#[serde(skip, default = "default_memory_op")] Operator), /// Local filesystem storage #[cfg(feature = "storage-fs")] LocalFs, @@ -105,7 +101,7 @@ pub(crate) enum OpenDALStorage { /// The configured scheme (s3 or s3a) configured_scheme: String, /// S3 configuration - config: opendal::services::S3Config, + config: Arc, /// Optional custom credential loader #[serde(skip)] customized_credential_load: Option, @@ -114,27 +110,28 @@ pub(crate) enum OpenDALStorage { #[cfg(feature = "storage-gcs")] Gcs { /// GCS configuration - config: opendal::services::GcsConfig, + config: Arc, }, /// Alibaba Cloud OSS #[cfg(feature = "storage-oss")] Oss { /// OSS configuration - config: opendal::services::OssConfig, + config: Arc, }, /// Azure Data Lake Storage #[cfg(feature = "storage-azdls")] Azdls { - /// The configured scheme (abfs, abfss, wasb, wasbs) + /// Because Azdls accepts multiple possible schemes, we store the full + /// passed scheme here to later validate schemes passed via paths. configured_scheme: super::AzureStorageScheme, /// Azure DLS configuration - config: opendal::services::AzdlsConfig, + config: Arc, }, } #[cfg(feature = "storage-memory")] -fn default_memory_op() -> Arc>> { - Arc::new(std::sync::Mutex::new(None)) +fn default_memory_op() -> Operator { + super::memory_config_build().expect("Failed to build memory operator") } impl OpenDALStorage { @@ -146,9 +143,7 @@ impl OpenDALStorage { match scheme { #[cfg(feature = "storage-memory")] - Scheme::Memory => Ok(OpenDALStorage::Memory { - op: Arc::new(std::sync::Mutex::new(None)), - }), + Scheme::Memory => Ok(OpenDALStorage::Memory(super::memory_config_build()?)), #[cfg(feature = "storage-fs")] Scheme::Fs => Ok(OpenDALStorage::LocalFs), @@ -161,7 +156,7 @@ impl OpenDALStorage { .map(Arc::unwrap_or_clone); Ok(OpenDALStorage::S3 { configured_scheme: scheme_str, - config, + config: Arc::new(config), customized_credential_load, }) } @@ -169,13 +164,17 @@ impl OpenDALStorage { #[cfg(feature = "storage-gcs")] Scheme::Gcs => { let config = super::gcs_config_parse(props)?; - Ok(OpenDALStorage::Gcs { config }) + Ok(OpenDALStorage::Gcs { + config: Arc::new(config), + }) } #[cfg(feature = "storage-oss")] Scheme::Oss => { let config = super::oss_config_parse(props)?; - Ok(OpenDALStorage::Oss { config }) + Ok(OpenDALStorage::Oss { + config: Arc::new(config), + }) } #[cfg(feature = "storage-azdls")] @@ -184,7 +183,7 @@ impl OpenDALStorage { let config = super::azdls_config_parse(props)?; Ok(OpenDALStorage::Azdls { configured_scheme, - config, + config: Arc::new(config), }) } // Update doc on [`FileIO`] when adding new schemes. @@ -210,17 +209,11 @@ impl OpenDALStorage { fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { let (operator, relative_path): (Operator, &str) = match self { #[cfg(feature = "storage-memory")] - OpenDALStorage::Memory { op } => { - let mut guard = op.lock().unwrap(); - if guard.is_none() { - *guard = Some(super::memory_config_build()?); - } - let op = guard.as_ref().unwrap().clone(); - + OpenDALStorage::Memory(op) => { if let Some(stripped) = path.strip_prefix("memory:/") { - Ok((op, stripped)) + Ok((op.clone(), stripped)) } else { - Ok((op, &path[1..])) + Ok((op.clone(), &path[1..])) } } From b3e0ab45cc2128a990bc8252005e421854963b37 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 12:23:41 -0800 Subject: [PATCH 08/12] squeaky? --- crates/iceberg/src/io/storage.rs | 53 +++++++++++++++----------------- 1 file changed, 25 insertions(+), 28 deletions(-) diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 93e8f924dd..5e2e2f0dab 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -143,47 +143,42 @@ impl OpenDALStorage { match scheme { #[cfg(feature = "storage-memory")] - Scheme::Memory => Ok(OpenDALStorage::Memory(super::memory_config_build()?)), + Scheme::Memory => Ok(Self::Memory(super::memory_config_build()?)), #[cfg(feature = "storage-fs")] - Scheme::Fs => Ok(OpenDALStorage::LocalFs), + Scheme::Fs => Ok(Self::LocalFs), #[cfg(feature = "storage-s3")] Scheme::S3 => { - let config = super::s3_config_parse(props)?; - let customized_credential_load = extensions - .get::() - .map(Arc::unwrap_or_clone); - Ok(OpenDALStorage::S3 { + Ok(Self::S3 { configured_scheme: scheme_str, - config: Arc::new(config), - customized_credential_load, + config: super::s3_config_parse(props)?.into(), + customized_credential_load: extensions + .get::() + .map(Arc::unwrap_or_clone), }) } #[cfg(feature = "storage-gcs")] Scheme::Gcs => { - let config = super::gcs_config_parse(props)?; - Ok(OpenDALStorage::Gcs { - config: Arc::new(config), + Ok(Self::Gcs { + config: super::gcs_config_parse(props)?.into(), }) } #[cfg(feature = "storage-oss")] Scheme::Oss => { - let config = super::oss_config_parse(props)?; - Ok(OpenDALStorage::Oss { - config: Arc::new(config), + Ok(Self::Oss { + config: super::oss_config_parse(props)?.into(), }) } #[cfg(feature = "storage-azdls")] Scheme::Azdls => { let configured_scheme = scheme_str.parse::()?; - let config = super::azdls_config_parse(props)?; - Ok(OpenDALStorage::Azdls { + Ok(Self::Azdls { configured_scheme, - config: Arc::new(config), + config: super::azdls_config_parse(props)?.into(), }) } // Update doc on [`FileIO`] when adding new schemes. @@ -209,27 +204,27 @@ impl OpenDALStorage { fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { let (operator, relative_path): (Operator, &str) = match self { #[cfg(feature = "storage-memory")] - OpenDALStorage::Memory(op) => { + Self::Memory(op) => { if let Some(stripped) = path.strip_prefix("memory:/") { - Ok((op.clone(), stripped)) + Ok::<_, crate::Error>((op.clone(), stripped)) } else { - Ok((op.clone(), &path[1..])) + Ok::<_, crate::Error>((op.clone(), &path[1..])) } } #[cfg(feature = "storage-fs")] - OpenDALStorage::LocalFs => { + Self::LocalFs => { let op = super::fs_config_build()?; if let Some(stripped) = path.strip_prefix("file:/") { - Ok((op, stripped)) + Ok::<_, crate::Error>((op, stripped)) } else { - Ok((op, &path[1..])) + Ok::<_, crate::Error>((op, &path[1..])) } } #[cfg(feature = "storage-s3")] - OpenDALStorage::S3 { + Self::S3 { configured_scheme, config, customized_credential_load, @@ -237,6 +232,7 @@ impl OpenDALStorage { let op = super::s3_config_build(config, customized_credential_load, path)?; let op_info = op.info(); + // Check prefix of s3 path. let prefix = format!("{}://{}/", configured_scheme, op_info.name()); if path.starts_with(&prefix) { Ok((op, &path[prefix.len()..])) @@ -249,7 +245,7 @@ impl OpenDALStorage { } #[cfg(feature = "storage-gcs")] - OpenDALStorage::Gcs { config } => { + Self::Gcs { config } => { let operator = super::gcs_config_build(config, path)?; let prefix = format!("gs://{}/", operator.info().name()); if path.starts_with(&prefix) { @@ -263,8 +259,9 @@ impl OpenDALStorage { } #[cfg(feature = "storage-oss")] - OpenDALStorage::Oss { config } => { + Self::Oss { config } => { let op = super::oss_config_build(config, path)?; + // Check prefix of oss path. let prefix = format!("oss://{}/", op.info().name()); if path.starts_with(&prefix) { Ok((op, &path[prefix.len()..])) @@ -277,7 +274,7 @@ impl OpenDALStorage { } #[cfg(feature = "storage-azdls")] - OpenDALStorage::Azdls { + Self::Azdls { configured_scheme, config, } => super::azdls_create_operator(path, config, configured_scheme), From c128ee9aa46037dcc4d3d97d8c65946e8bfe59dd Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 12:31:06 -0800 Subject: [PATCH 09/12] dal clean yet? --- crates/iceberg/src/io/file_io.rs | 4 +-- crates/iceberg/src/io/storage.rs | 59 ++++++++++++++++---------------- 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 0be94391a3..e20cea2ed2 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use bytes::Bytes; use url::Url; -use super::storage::OpenDALStorage; +use super::storage::OpenDalStorage; pub use super::storage::Storage; use crate::{Error, ErrorKind, Result}; @@ -240,7 +240,7 @@ impl FileIOBuilder { /// Builds [`FileIO`]. pub fn build(self) -> Result { - let storage = OpenDALStorage::build(self.clone())?; + let storage = OpenDalStorage::build(self.clone())?; Ok(FileIO { builder: self, diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 5e2e2f0dab..d9442d5a45 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -88,7 +88,7 @@ pub trait Storage: Debug + Send + Sync { /// This storage handles all supported schemes (S3, GCS, Azure, filesystem, memory) /// through OpenDAL, creating operators on-demand based on the path scheme. #[derive(Debug, Clone, Serialize, Deserialize)] -pub(crate) enum OpenDALStorage { +pub(crate) enum OpenDalStorage { /// In-memory storage, useful for testing #[cfg(feature = "storage-memory")] Memory(#[serde(skip, default = "default_memory_op")] Operator), @@ -96,9 +96,11 @@ pub(crate) enum OpenDALStorage { #[cfg(feature = "storage-fs")] LocalFs, /// Amazon S3 storage + /// Expects paths of the form `s3[a]:///`. #[cfg(feature = "storage-s3")] S3 { - /// The configured scheme (s3 or s3a) + /// s3 storage could have `s3://` and `s3a://`. + /// Storing the scheme string here to return the correct path. configured_scheme: String, /// S3 configuration config: Arc, @@ -119,6 +121,9 @@ pub(crate) enum OpenDALStorage { config: Arc, }, /// Azure Data Lake Storage + /// Expects paths of the form + /// `abfs[s]://@.dfs./` or + /// `wasb[s]://@.blob./`. #[cfg(feature = "storage-azdls")] Azdls { /// Because Azdls accepts multiple possible schemes, we store the full @@ -134,7 +139,7 @@ fn default_memory_op() -> Operator { super::memory_config_build().expect("Failed to build memory operator") } -impl OpenDALStorage { +impl OpenDalStorage { /// Build storage from FileIOBuilder pub fn build(file_io_builder: FileIOBuilder) -> Result { let (scheme_str, props, extensions) = file_io_builder.into_parts(); @@ -149,29 +154,23 @@ impl OpenDALStorage { Scheme::Fs => Ok(Self::LocalFs), #[cfg(feature = "storage-s3")] - Scheme::S3 => { - Ok(Self::S3 { - configured_scheme: scheme_str, - config: super::s3_config_parse(props)?.into(), - customized_credential_load: extensions + Scheme::S3 => Ok(Self::S3 { + configured_scheme: scheme_str, + config: super::s3_config_parse(props)?.into(), + customized_credential_load: extensions .get::() .map(Arc::unwrap_or_clone), - }) - } + }), #[cfg(feature = "storage-gcs")] - Scheme::Gcs => { - Ok(Self::Gcs { - config: super::gcs_config_parse(props)?.into(), - }) - } + Scheme::Gcs => Ok(Self::Gcs { + config: super::gcs_config_parse(props)?.into(), + }), #[cfg(feature = "storage-oss")] - Scheme::Oss => { - Ok(Self::Oss { - config: super::oss_config_parse(props)?.into(), - }) - } + Scheme::Oss => Ok(Self::Oss { + config: super::oss_config_parse(props)?.into(), + }), #[cfg(feature = "storage-azdls")] Scheme::Azdls => { @@ -314,7 +313,7 @@ impl OpenDALStorage { #[async_trait] #[typetag::serde] -impl Storage for OpenDALStorage { +impl Storage for OpenDalStorage { async fn exists(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(path)?; Ok(op.exists(relative_path).await?) @@ -382,38 +381,38 @@ mod tests { #[cfg(feature = "storage-memory")] fn test_opendal_storage_memory() { let builder = FileIOBuilder::new("memory"); - let storage = OpenDALStorage::build(builder).unwrap(); - assert!(matches!(storage, OpenDALStorage::Memory { .. })); + let storage = OpenDalStorage::build(builder).unwrap(); + assert!(matches!(storage, OpenDalStorage::Memory { .. })); } #[test] #[cfg(feature = "storage-fs")] fn test_opendal_storage_fs() { let builder = FileIOBuilder::new("file"); - let storage = OpenDALStorage::build(builder).unwrap(); - assert!(matches!(storage, OpenDALStorage::LocalFs)); + let storage = OpenDalStorage::build(builder).unwrap(); + assert!(matches!(storage, OpenDalStorage::LocalFs)); } #[test] #[cfg(feature = "storage-s3")] fn test_opendal_storage_s3() { let builder = FileIOBuilder::new("s3"); - let storage = OpenDALStorage::build(builder).unwrap(); - assert!(matches!(storage, OpenDALStorage::S3 { .. })); + let storage = OpenDalStorage::build(builder).unwrap(); + assert!(matches!(storage, OpenDalStorage::S3 { .. })); } #[test] #[cfg(feature = "storage-memory")] fn test_storage_serialization() { let builder = FileIOBuilder::new("memory"); - let storage = OpenDALStorage::build(builder).unwrap(); + let storage = OpenDalStorage::build(builder).unwrap(); // Serialize let serialized = serde_json::to_string(&storage).unwrap(); // Deserialize - let deserialized: OpenDALStorage = serde_json::from_str(&serialized).unwrap(); + let deserialized: OpenDalStorage = serde_json::from_str(&serialized).unwrap(); - assert!(matches!(deserialized, OpenDALStorage::Memory { .. })); + assert!(matches!(deserialized, OpenDalStorage::Memory { .. })); } } From 459dc732cad207e8230e9a9aeef7b97a5e0d3058 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 12:50:15 -0800 Subject: [PATCH 10/12] sweekey cling? --- crates/iceberg/src/io/file_io.rs | 2 ++ crates/iceberg/src/io/storage.rs | 8 +------- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index e20cea2ed2..7955ab0afa 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -121,6 +121,7 @@ impl FileIO { /// # Arguments /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + #[must_use] pub fn new_input(&self, path: impl AsRef) -> Result { self.inner.new_input(path.as_ref()) } @@ -130,6 +131,7 @@ impl FileIO { /// # Arguments /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + #[must_use] pub fn new_output(&self, path: impl AsRef) -> Result { self.inner.new_output(path.as_ref()) } diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index d9442d5a45..e754c7e16a 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,10 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Storage implementation for Iceberg using OpenDAL. -//! -//! This module provides a unified storage abstraction that handles all supported -//! storage backends (S3, GCS, Azure, local filesystem, memory, etc.) through OpenDAL. +//! Storage interfaces of Iceberg use std::fmt::Debug; use std::sync::Arc; @@ -42,9 +39,6 @@ use crate::{Error, ErrorKind, Result}; /// Trait for storage operations in Iceberg. /// -/// This trait defines the interface for all storage backends. The default implementation -/// uses OpenDAL to support various storage systems like S3, GCS, Azure, local filesystem, etc. -/// /// The trait supports serialization via `typetag`, allowing storage instances to be /// serialized and deserialized across process boundaries. /// From 10917095dc71d756a278ff1b060a38fcbcb6e31d Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 12:56:22 -0800 Subject: [PATCH 11/12] the cleanest --- crates/iceberg/src/io/storage_azdls.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 0741496f84..ec11db6c07 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -120,8 +120,6 @@ pub(crate) fn azdls_create_operator<'a>( Ok((op, relative_path)) } -/// Azure Storage scheme variants. -/// /// Note that `abf[s]` and `wasb[s]` variants have different implications: /// - `abfs[s]` is used to refer to files in ADLS Gen2, backed by blob storage; /// paths are expected to contain the `dfs` storage service. From c8b56b2d229c8f95c0fd61e0c312941bef46e6c6 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 11 Dec 2025 13:36:26 -0800 Subject: [PATCH 12/12] clippy --- crates/iceberg/src/io/file_io.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 7955ab0afa..e20cea2ed2 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -121,7 +121,6 @@ impl FileIO { /// # Arguments /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. - #[must_use] pub fn new_input(&self, path: impl AsRef) -> Result { self.inner.new_input(path.as_ref()) } @@ -131,7 +130,6 @@ impl FileIO { /// # Arguments /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. - #[must_use] pub fn new_output(&self, path: impl AsRef) -> Result { self.inner.new_output(path.as_ref()) }