From a9d8fdf4d1eaa51cbfc7f6d041aa7ee59927a928 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 12 Sep 2025 15:28:17 -0700 Subject: [PATCH 1/6] Make Storage a trait --- crates/iceberg/src/arrow/reader.rs | 12 +- crates/iceberg/src/io/file_io.rs | 433 +++++++++++++++++---- crates/iceberg/src/io/mod.rs | 8 +- crates/iceberg/src/io/storage.rs | 480 ++++++++++++++---------- crates/iceberg/src/io/storage_azdls.rs | 119 +++++- crates/iceberg/src/io/storage_fs.rs | 104 +++++ crates/iceberg/src/io/storage_gcs.rs | 109 +++++- crates/iceberg/src/io/storage_memory.rs | 100 ++++- crates/iceberg/src/io/storage_oss.rs | 107 ++++++ crates/iceberg/src/io/storage_s3.rs | 141 ++++++- crates/iceberg/src/puffin/metadata.rs | 11 +- crates/iceberg/src/puffin/reader.rs | 2 +- 12 files changed, 1327 insertions(+), 299 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index ab5a96f751..217d466e5a 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -428,7 +428,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)?; @@ -1628,18 +1628,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, @@ -1678,7 +1678,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..9a4bd8115d 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, StorageBuilder, StorageBuilderRegistry}; +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. @@ -105,13 +107,7 @@ impl FileIO { /// - 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?) + self.inner.remove_dir_all(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 +/// [`StorageBuilderRegistry`] and registering your storage builder: +/// +/// ```rust,ignore +/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder, FileIOBuilder}; +/// use std::sync::Arc; +/// +/// // Create your custom storage builder +/// let my_builder = Arc::new(MyCustomStorageBuilder); +/// +/// // Register it with a custom scheme +/// let mut registry = StorageBuilderRegistry::new(); +/// registry.register("mycustom", my_builder); +/// +/// // 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 builder 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::{StorageBuilderRegistry, FileIOBuilder}; + /// use std::sync::Arc; + /// + /// let mut registry = StorageBuilderRegistry::new(); + /// registry.register("mycustom", Arc::new(MyCustomStorageBuilder)); + /// + /// let file_io = FileIOBuilder::new("mycustom") + /// .with_registry(registry) + /// .build()?; + /// ``` + pub fn with_registry(mut self, registry: StorageBuilderRegistry) -> 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 builder = registry.get_builder(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 = builder.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,129 @@ 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 tempfile::TempDir; use super::{FileIO, FileIOBuilder}; + use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, + STORAGE_LOCATION_SCHEME, Storage, StorageBuilder, StorageBuilderRegistry, + }; + use crate::{Error, ErrorKind, Result}; + + // Test storage implementation that tracks write operations + #[derive(Debug, Clone)] + struct TestStorage { + written: Arc>>, + received_props: HashMap, + } + + #[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] + 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 remove_dir_all(&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())) + } + } + + // Test storage builder + #[derive(Debug)] + struct TestStorageBuilder { + written: Arc>>, + received_props: Arc>>, + } + + impl TestStorageBuilder { + pub fn written(&self) -> MutexGuard<'_, Vec> { + self.written.lock().unwrap() + } + + pub fn received_props(&self) -> MutexGuard<'_, HashMap> { + self.received_props.lock().unwrap() + } + } + + impl StorageBuilder for TestStorageBuilder { + 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() @@ -605,4 +773,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 builder = Arc::new(TestStorageBuilder { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageBuilderRegistry::new(); + registry.register("test", builder.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 builder = Arc::new(TestStorageBuilder { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageBuilderRegistry::new(); + registry.register("test", builder.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 = builder.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 builder = Arc::new(TestStorageBuilder { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageBuilderRegistry::new(); + registry.register("myscheme", builder.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 builder + let props = builder.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 = StorageBuilderRegistry::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..209be5321d 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, StorageBuilder, StorageBuilderRegistry}; 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 d5f2ad8fab..03f5b77155 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,220 +15,296 @@ // specific language governing permissions and limitations // under the License. +//! 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 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. +/// +/// # 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] +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<()>; + + /// Remove a directory and all its contents recursively + async fn remove_dir_all(&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 builders. +/// +/// Storage builders are responsible for creating storage instances from configuration +/// properties and extensions. Each storage backend (S3, GCS, etc.) provides its own +/// builder implementation. +/// +/// # Example +/// +/// ```rust,ignore +/// use iceberg::io::{StorageBuilder, Extensions}; +/// use std::collections::HashMap; +/// +/// struct MyStorageBuilder; +/// +/// impl StorageBuilder for MyStorageBuilder { +/// fn build( +/// &self, +/// props: HashMap, +/// extensions: Extensions, +/// ) -> Result> { +/// // Parse configuration and create storage +/// Ok(Arc::new(MyStorage::new(props)?)) +/// } +/// } +/// ``` +pub trait StorageBuilder: 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 (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: {}, should start with {}", path, 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: {}, should start with {}", path, 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 builders. +/// +/// The registry allows you to register custom storage builders for different URI schemes. +/// By default, it includes builders for all enabled storage features. +/// +/// # Example +/// +/// ```rust +/// use iceberg::io::StorageBuilderRegistry; +/// +/// // Create a new registry with default builders +/// let registry = StorageBuilderRegistry::new(); +/// +/// // Get supported storage types +/// let types = registry.supported_types(); +/// println!("Supported storage types: {:?}", types); +/// +/// // Get a builder for a specific scheme +/// # #[cfg(feature = "storage-memory")] +/// # { +/// let builder = registry.get_builder("memory").unwrap(); +/// # } +/// ``` +/// +/// You can also register custom storage builders: +/// +/// ```rust,ignore +/// use std::sync::Arc; +/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder}; +/// +/// let mut registry = StorageBuilderRegistry::new(); +/// +/// // Register a custom storage builder +/// registry.register("custom", Arc::new(MyCustomStorageBuilder)); +/// ``` +#[derive(Debug, Clone)] +pub struct StorageBuilderRegistry { + builders: HashMap>, +} + +impl StorageBuilderRegistry { + /// Create a new storage registry with default builders based on enabled features. + pub fn new() -> Self { + let mut builders: HashMap> = HashMap::new(); + + #[cfg(feature = "storage-memory")] + { + use crate::io::storage_memory::OpenDALMemoryStorageBuilder; + let builder = Arc::new(OpenDALMemoryStorageBuilder) as Arc; + builders.insert("memory".to_string(), builder); + } + + #[cfg(feature = "storage-fs")] + { + use crate::io::storage_fs::OpenDALFsStorageBuilder; + let builder = Arc::new(OpenDALFsStorageBuilder) as Arc; + builders.insert("file".to_string(), builder.clone()); + builders.insert("".to_string(), builder); + } + + #[cfg(feature = "storage-s3")] + { + use crate::io::storage_s3::OpenDALS3StorageBuilder; + let builder = Arc::new(OpenDALS3StorageBuilder) as Arc; + builders.insert("s3".to_string(), builder.clone()); + builders.insert("s3a".to_string(), builder); + } + + #[cfg(feature = "storage-gcs")] + { + use crate::io::storage_gcs::OpenDALGcsStorageBuilder; + let builder = Arc::new(OpenDALGcsStorageBuilder) as Arc; + builders.insert("gs".to_string(), builder.clone()); + builders.insert("gcs".to_string(), builder); + } - /// 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::OpenDALOssStorageBuilder; + let builder = Arc::new(OpenDALOssStorageBuilder) as Arc; + builders.insert("oss".to_string(), builder); } + + #[cfg(feature = "storage-azdls")] + { + use crate::io::storage_azdls::OpenDALAzdlsStorageBuilder; + let builder = Arc::new(OpenDALAzdlsStorageBuilder) as Arc; + builders.insert("abfs".to_string(), builder.clone()); + builders.insert("abfss".to_string(), builder.clone()); + builders.insert("wasb".to_string(), builder.clone()); + builders.insert("wasbs".to_string(), builder); + } + + Self { builders } + } + + /// Register a custom storage builder for a given scheme. + pub fn register(&mut self, scheme: impl Into, builder: Arc) { + self.builders.insert(scheme.into(), builder); + } + + /// Get a storage builder by scheme. + pub fn get_builder(&self, scheme: &str) -> Result> { + let key = scheme.trim(); + self.builders + .iter() + .find(|(k, _)| k.eq_ignore_ascii_case(key)) + .map(|(_, builder)| builder.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.builders.keys().cloned().collect() + } +} + +impl Default for StorageBuilderRegistry { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_storage_builder_registry_new() { + let registry = StorageBuilderRegistry::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_builder_registry_get_builder() { + let registry = StorageBuilderRegistry::new(); + + // Should be able to get memory storage builder + let builder = registry.get_builder("memory"); + assert!(builder.is_ok()); + + // Should be case-insensitive + let builder = registry.get_builder("MEMORY"); + assert!(builder.is_ok()); + } + + #[test] + fn test_storage_builder_registry_unsupported_type() { + let registry = StorageBuilderRegistry::new(); + + // Should return error for unsupported type + let result = registry.get_builder("unsupported"); + assert!(result.is_err()); + } + + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_builder_registry_clone() { + let registry = StorageBuilderRegistry::new(); + let cloned = registry.clone(); + + // Both should have the same builders + 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 fe12167f6f..f0a85e44b1 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -18,11 +18,18 @@ 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 url::Url; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, + Storage, StorageBuilder, +}; use crate::{Error, ErrorKind, Result, ensure_data_valid}; /// A connection string. @@ -125,7 +132,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)] pub(crate) enum AzureStorageScheme { Abfs, Abfss, @@ -597,3 +604,111 @@ mod tests { } } } + +/// Azure Data Lake Storage implementation using OpenDAL +#[derive(Debug, Clone)] +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] +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 remove_dir_all(&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())) + } +} + +/// Builder for Azure Data Lake Storage +#[derive(Debug)] +pub struct OpenDALAzdlsStorageBuilder; + +impl StorageBuilder for OpenDALAzdlsStorageBuilder { + 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..0b612351f7 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -15,10 +15,18 @@ // 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 crate::Result; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, +}; /// Build new opendal operator from give path. pub(crate) fn fs_config_build() -> Result { @@ -27,3 +35,99 @@ pub(crate) fn fs_config_build() -> Result { Ok(Operator::from_config(cfg)?.finish()) } + +/// Filesystem storage implementation using OpenDAL +#[derive(Debug, Clone)] +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] +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 remove_dir_all(&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())) + } +} + +/// Builder for OpenDAL Filesystem storage +#[derive(Debug)] +pub struct OpenDALFsStorageBuilder; + +impl StorageBuilder for OpenDALFsStorageBuilder { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> Result> { + Ok(Arc::new(OpenDALFsStorage)) + } +} diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 8c3d914c86..f3dc0212a9 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -17,12 +17,18 @@ //! 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 url::Url; -use crate::io::is_truthy; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + 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 +110,104 @@ 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 +#[derive(Debug, Clone)] +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] +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 remove_dir_all(&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())) + } +} + +/// Builder for OpenDAL GCS storage +#[derive(Debug)] +pub struct OpenDALGcsStorageBuilder; + +impl StorageBuilder for OpenDALGcsStorageBuilder { + 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..b72a0b7abc 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -15,11 +15,107 @@ // 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::MemoryConfig; use crate::Result; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, +}; + +/// Memory storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALMemoryStorage { + op: Operator, +} + +impl OpenDALMemoryStorage { + /// 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] +impl Storage for OpenDALMemoryStorage { + async fn exists(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + Ok(self.op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let meta = self.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); + Ok(self.op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + Ok(Box::new(self.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); + Ok(Box::new(self.op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + Ok(self.op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(self.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())) + } +} + +/// Builder for OpenDAL Memory storage +#[derive(Debug)] +pub struct OpenDALMemoryStorageBuilder; -pub(crate) fn memory_config_build() -> Result { - Ok(Operator::from_config(MemoryConfig::default())?.finish()) +impl StorageBuilder for OpenDALMemoryStorageBuilder { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> Result> { + let op = Operator::from_config(MemoryConfig::default())?.finish(); + Ok(Arc::new(OpenDALMemoryStorage { op })) + } } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 8bfffc6ca8..c070edef01 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -16,11 +16,17 @@ // 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 url::Url; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, +}; use crate::{Error, ErrorKind, Result}; /// Required configuration arguments for creating an Aliyun OSS Operator with OpenDAL: @@ -64,3 +70,104 @@ pub(crate) fn oss_config_build(cfg: &OssConfig, path: &str) -> Result Ok(Operator::new(builder)?.finish()) } + +/// OSS storage implementation using OpenDAL +#[derive(Debug, Clone)] +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] +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 remove_dir_all(&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())) + } +} + +/// Builder for OpenDAL OSS storage +#[derive(Debug)] +pub struct OpenDALOssStorageBuilder; + +impl StorageBuilder for OpenDALOssStorageBuilder { + 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 fcf9afed1f..2c167782cf 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -19,13 +19,17 @@ 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 url::Url; -use crate::io::is_truthy; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, + Storage, StorageBuilder, is_truthy, +}; use crate::{Error, ErrorKind, Result}; /// Following are arguments for [s3 file io](https://py.iceberg.apache.org/configuration/#s3). @@ -213,3 +217,138 @@ impl AwsCredentialLoad for CustomAwsCredentialLoader { self.0.load_credential(client).await } } + +/// S3 storage implementation using OpenDAL +#[derive(Debug, Clone)] +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, + 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] +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 remove_dir_all(&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())) + } +} + +/// Builder for S3 storage +#[derive(Debug)] +pub struct OpenDALS3StorageBuilder; + +impl StorageBuilder for OpenDALS3StorageBuilder { + 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, + })) + } +} 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 4ecae742765caee5109c8578a159b5b9110cad3b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 1 Dec 2025 16:19:51 -0800 Subject: [PATCH 2/6] remane to factory --- crates/iceberg/src/io/file_io.rs | 68 +++++------ crates/iceberg/src/io/mod.rs | 2 +- crates/iceberg/src/io/storage.rs | 148 ++++++++++++------------ crates/iceberg/src/io/storage_azdls.rs | 8 +- crates/iceberg/src/io/storage_fs.rs | 8 +- crates/iceberg/src/io/storage_gcs.rs | 8 +- crates/iceberg/src/io/storage_memory.rs | 8 +- crates/iceberg/src/io/storage_oss.rs | 8 +- crates/iceberg/src/io/storage_s3.rs | 8 +- 9 files changed, 133 insertions(+), 133 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 9a4bd8115d..b7bab85980 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -26,7 +26,7 @@ use bytes::Bytes; use url::Url; // Re-export traits from storage module -pub use super::storage::{Storage, StorageBuilder, StorageBuilderRegistry}; +pub use super::storage::{Storage, StorageFactory, StorageRegistry}; use crate::io::STORAGE_LOCATION_SCHEME; use crate::{Error, ErrorKind, Result}; @@ -168,18 +168,18 @@ impl Extensions { /// # Custom Storage Implementations /// /// You can use custom storage implementations by creating a custom -/// [`StorageBuilderRegistry`] and registering your storage builder: +/// [`StorageRegistry`] and registering your storage factory: /// /// ```rust,ignore -/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder, FileIOBuilder}; +/// use iceberg::io::{StorageRegistry, StorageFactory, FileIOBuilder}; /// use std::sync::Arc; /// -/// // Create your custom storage builder -/// let my_builder = Arc::new(MyCustomStorageBuilder); +/// // Create your custom storage factory +/// let my_factory = Arc::new(MyCustomStorageFactory); /// /// // Register it with a custom scheme -/// let mut registry = StorageBuilderRegistry::new(); -/// registry.register("mycustom", my_builder); +/// let mut registry = StorageRegistry::new(); +/// registry.register("mycustom", my_factory); /// /// // Use it to build FileIO /// let file_io = FileIOBuilder::new("mycustom") @@ -198,7 +198,7 @@ pub struct FileIOBuilder { /// Optional extensions to configure the underlying FileIO behavior. extensions: Extensions, /// Optional custom registry. If None, a default registry will be created. - registry: Option, + registry: Option, } impl FileIOBuilder { @@ -232,7 +232,7 @@ impl FileIOBuilder { String, HashMap, Extensions, - Option, + Option, ) { ( self.scheme_str.unwrap_or_default(), @@ -276,7 +276,7 @@ impl FileIOBuilder { self.extensions.get::() } - /// Sets a custom storage builder registry. + /// 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 @@ -285,17 +285,17 @@ impl FileIOBuilder { /// # Example /// /// ```rust,ignore - /// use iceberg::io::{StorageBuilderRegistry, FileIOBuilder}; + /// use iceberg::io::{StorageRegistry, FileIOBuilder}; /// use std::sync::Arc; /// - /// let mut registry = StorageBuilderRegistry::new(); - /// registry.register("mycustom", Arc::new(MyCustomStorageBuilder)); + /// 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: StorageBuilderRegistry) -> Self { + pub fn with_registry(mut self, registry: StorageRegistry) -> Self { self.registry = Some(registry); self } @@ -308,13 +308,13 @@ impl FileIOBuilder { // Use custom registry if provided, otherwise create default let registry = self.registry.clone().unwrap_or_default(); - let builder = registry.get_builder(scheme.as_str())?; + 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 = builder.build(props_with_scheme, self.extensions.clone())?; + let storage = factory.build(props_with_scheme, self.extensions.clone())?; Ok(FileIO { builder: self, @@ -528,7 +528,7 @@ mod tests { use super::{FileIO, FileIOBuilder}; use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, - STORAGE_LOCATION_SCHEME, Storage, StorageBuilder, StorageBuilderRegistry, + STORAGE_LOCATION_SCHEME, Storage, StorageFactory, StorageRegistry, }; use crate::{Error, ErrorKind, Result}; @@ -600,14 +600,14 @@ mod tests { } } - // Test storage builder + // Test storage factory #[derive(Debug)] - struct TestStorageBuilder { + struct TestStorageFactory { written: Arc>>, received_props: Arc>>, } - impl TestStorageBuilder { + impl TestStorageFactory { pub fn written(&self) -> MutexGuard<'_, Vec> { self.written.lock().unwrap() } @@ -617,7 +617,7 @@ mod tests { } } - impl StorageBuilder for TestStorageBuilder { + impl StorageFactory for TestStorageFactory { fn build( &self, props: HashMap, @@ -777,13 +777,13 @@ mod tests { #[test] fn test_custom_registry() { // Create a custom registry and register test storage - let builder = Arc::new(TestStorageBuilder { + let factory = Arc::new(TestStorageFactory { written: Arc::new(Mutex::new(Vec::new())), received_props: Arc::new(Mutex::new(HashMap::new())), }); - let mut registry = StorageBuilderRegistry::new(); - registry.register("test", builder.clone()); + let mut registry = StorageRegistry::new(); + registry.register("test", factory.clone()); // Build FileIO with custom storage let file_io = FileIOBuilder::new("test") @@ -799,13 +799,13 @@ mod tests { #[tokio::test] async fn test_custom_registry_operations() { // Create test storage with write tracking - let builder = Arc::new(TestStorageBuilder { + let factory = Arc::new(TestStorageFactory { written: Arc::new(Mutex::new(Vec::new())), received_props: Arc::new(Mutex::new(HashMap::new())), }); - let mut registry = StorageBuilderRegistry::new(); - registry.register("test", builder.clone()); + let mut registry = StorageRegistry::new(); + registry.register("test", factory.clone()); // Build FileIO with test storage let file_io = FileIOBuilder::new("test") @@ -825,7 +825,7 @@ mod tests { assert_eq!(metadata.size, 42); // Verify write was tracked - let tracked = builder.written(); + let tracked = factory.written(); assert_eq!(tracked.len(), 1); assert_eq!(tracked[0], "test://bucket/file.txt"); } @@ -833,13 +833,13 @@ mod tests { #[test] fn test_scheme_and_props_propagation() { // Create test storage that captures props - let builder = Arc::new(TestStorageBuilder { + let factory = Arc::new(TestStorageFactory { written: Arc::new(Mutex::new(Vec::new())), received_props: Arc::new(Mutex::new(HashMap::new())), }); - let mut registry = StorageBuilderRegistry::new(); - registry.register("myscheme", builder.clone()); + let mut registry = StorageRegistry::new(); + registry.register("myscheme", factory.clone()); // Build FileIO with custom scheme and additional props let file_io = FileIOBuilder::new("myscheme") @@ -851,8 +851,8 @@ mod tests { // Verify the storage was created assert!(file_io.new_output("myscheme://test.txt").is_ok()); - // Verify the scheme was propagated to the builder - let props = builder.received_props(); + // Verify the scheme was propagated to the factory + let props = factory.received_props(); assert_eq!( props.get(STORAGE_LOCATION_SCHEME), Some(&"myscheme".to_string()) @@ -863,7 +863,7 @@ mod tests { #[test] fn test_into_parts_includes_registry() { - let registry = StorageBuilderRegistry::new(); + let registry = StorageRegistry::new(); let builder = FileIOBuilder::new("memory") .with_prop("key", "value") diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 209be5321d..b16db18563 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -70,7 +70,7 @@ mod file_io; mod storage; pub use file_io::*; -pub use storage::{Storage, StorageBuilder, StorageBuilderRegistry}; +pub use storage::{Storage, StorageFactory, StorageRegistry}; pub(crate) mod object_cache; /// Property key used to pass the scheme string from FileIOBuilder to StorageBuilder. diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 03f5b77155..92ac0f936f 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -83,21 +83,21 @@ pub trait Storage: Debug + Send + Sync { fn new_output(&self, path: &str) -> Result; } -/// Common interface for all storage builders. +/// Common interface for all storage factories. /// -/// Storage builders are responsible for creating storage instances from configuration +/// Storage factories are responsible for creating storage instances from configuration /// properties and extensions. Each storage backend (S3, GCS, etc.) provides its own -/// builder implementation. +/// factory implementation. /// /// # Example /// /// ```rust,ignore -/// use iceberg::io::{StorageBuilder, Extensions}; +/// use iceberg::io::{StorageFactory, Extensions}; /// use std::collections::HashMap; /// -/// struct MyStorageBuilder; +/// struct MyStorageFactory; /// -/// impl StorageBuilder for MyStorageBuilder { +/// impl StorageFactory for MyStorageFactory { /// fn build( /// &self, /// props: HashMap, @@ -108,7 +108,7 @@ pub trait Storage: Debug + Send + Sync { /// } /// } /// ``` -pub trait StorageBuilder: Debug + Send + Sync { +pub trait StorageFactory: Debug + Send + Sync { /// Create a new storage instance with the given properties and extensions. /// /// # Arguments @@ -126,114 +126,114 @@ pub trait StorageBuilder: Debug + Send + Sync { ) -> Result>; } -/// A registry of storage builders. +/// A registry of storage factories. /// -/// The registry allows you to register custom storage builders for different URI schemes. -/// By default, it includes builders for all enabled storage features. +/// 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::StorageBuilderRegistry; +/// use iceberg::io::StorageRegistry; /// -/// // Create a new registry with default builders -/// let registry = StorageBuilderRegistry::new(); +/// // 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 builder for a specific scheme +/// // Get a factory for a specific scheme /// # #[cfg(feature = "storage-memory")] /// # { -/// let builder = registry.get_builder("memory").unwrap(); +/// let factory = registry.get_factory("memory").unwrap(); /// # } /// ``` /// -/// You can also register custom storage builders: +/// You can also register custom storage factories: /// /// ```rust,ignore /// use std::sync::Arc; -/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder}; +/// use iceberg::io::{StorageRegistry, StorageFactory}; /// -/// let mut registry = StorageBuilderRegistry::new(); +/// let mut registry = StorageRegistry::new(); /// -/// // Register a custom storage builder -/// registry.register("custom", Arc::new(MyCustomStorageBuilder)); +/// // Register a custom storage factory +/// registry.register("custom", Arc::new(MyCustomStorageFactory)); /// ``` #[derive(Debug, Clone)] -pub struct StorageBuilderRegistry { - builders: HashMap>, +pub struct StorageRegistry { + factories: HashMap>, } -impl StorageBuilderRegistry { - /// Create a new storage registry with default builders based on enabled features. +impl StorageRegistry { + /// Create a new storage registry with default factories based on enabled features. pub fn new() -> Self { - let mut builders: HashMap> = HashMap::new(); + let mut factories: HashMap> = HashMap::new(); #[cfg(feature = "storage-memory")] { - use crate::io::storage_memory::OpenDALMemoryStorageBuilder; - let builder = Arc::new(OpenDALMemoryStorageBuilder) as Arc; - builders.insert("memory".to_string(), builder); + 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::OpenDALFsStorageBuilder; - let builder = Arc::new(OpenDALFsStorageBuilder) as Arc; - builders.insert("file".to_string(), builder.clone()); - builders.insert("".to_string(), builder); + 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::OpenDALS3StorageBuilder; - let builder = Arc::new(OpenDALS3StorageBuilder) as Arc; - builders.insert("s3".to_string(), builder.clone()); - builders.insert("s3a".to_string(), builder); + 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::OpenDALGcsStorageBuilder; - let builder = Arc::new(OpenDALGcsStorageBuilder) as Arc; - builders.insert("gs".to_string(), builder.clone()); - builders.insert("gcs".to_string(), builder); + 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); } #[cfg(feature = "storage-oss")] { - use crate::io::storage_oss::OpenDALOssStorageBuilder; - let builder = Arc::new(OpenDALOssStorageBuilder) as Arc; - builders.insert("oss".to_string(), builder); + 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::OpenDALAzdlsStorageBuilder; - let builder = Arc::new(OpenDALAzdlsStorageBuilder) as Arc; - builders.insert("abfs".to_string(), builder.clone()); - builders.insert("abfss".to_string(), builder.clone()); - builders.insert("wasb".to_string(), builder.clone()); - builders.insert("wasbs".to_string(), builder); + 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 { builders } + Self { factories } } - /// Register a custom storage builder for a given scheme. - pub fn register(&mut self, scheme: impl Into, builder: Arc) { - self.builders.insert(scheme.into(), builder); + /// 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 builder by scheme. - pub fn get_builder(&self, scheme: &str) -> Result> { + /// Get a storage factory by scheme. + pub fn get_factory(&self, scheme: &str) -> Result> { let key = scheme.trim(); - self.builders + self.factories .iter() .find(|(k, _)| k.eq_ignore_ascii_case(key)) - .map(|(_, builder)| builder.clone()) + .map(|(_, factory)| factory.clone()) .ok_or_else(|| { use crate::{Error, ErrorKind}; Error::new( @@ -249,11 +249,11 @@ impl StorageBuilderRegistry { /// Return the list of supported storage types. pub fn supported_types(&self) -> Vec { - self.builders.keys().cloned().collect() + self.factories.keys().cloned().collect() } } -impl Default for StorageBuilderRegistry { +impl Default for StorageRegistry { fn default() -> Self { Self::new() } @@ -264,8 +264,8 @@ mod tests { use super::*; #[test] - fn test_storage_builder_registry_new() { - let registry = StorageBuilderRegistry::new(); + fn test_storage_registry_new() { + let registry = StorageRegistry::new(); let types = registry.supported_types(); // At least one storage type should be available @@ -274,34 +274,34 @@ mod tests { #[test] #[cfg(feature = "storage-memory")] - fn test_storage_builder_registry_get_builder() { - let registry = StorageBuilderRegistry::new(); + fn test_storage_registry_get_factory() { + let registry = StorageRegistry::new(); - // Should be able to get memory storage builder - let builder = registry.get_builder("memory"); - assert!(builder.is_ok()); + // Should be able to get memory storage factory + let factory = registry.get_factory("memory"); + assert!(factory.is_ok()); // Should be case-insensitive - let builder = registry.get_builder("MEMORY"); - assert!(builder.is_ok()); + let factory = registry.get_factory("MEMORY"); + assert!(factory.is_ok()); } #[test] - fn test_storage_builder_registry_unsupported_type() { - let registry = StorageBuilderRegistry::new(); + fn test_storage_registry_unsupported_type() { + let registry = StorageRegistry::new(); // Should return error for unsupported type - let result = registry.get_builder("unsupported"); + let result = registry.get_factory("unsupported"); assert!(result.is_err()); } #[test] #[cfg(feature = "storage-memory")] - fn test_storage_builder_registry_clone() { - let registry = StorageBuilderRegistry::new(); + fn test_storage_registry_clone() { + let registry = StorageRegistry::new(); let cloned = registry.clone(); - // Both should have the same builders + // 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 f0a85e44b1..7938160866 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -28,7 +28,7 @@ use url::Url; use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, - Storage, StorageBuilder, + Storage, StorageFactory, }; use crate::{Error, ErrorKind, Result, ensure_data_valid}; @@ -684,11 +684,11 @@ impl Storage for OpenDALAzdlsStorage { } } -/// Builder for Azure Data Lake Storage +/// Factory for Azure Data Lake Storage #[derive(Debug)] -pub struct OpenDALAzdlsStorageBuilder; +pub struct OpenDALAzdlsStorageFactory; -impl StorageBuilder for OpenDALAzdlsStorageBuilder { +impl StorageFactory for OpenDALAzdlsStorageFactory { fn build( &self, props: HashMap, diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index 0b612351f7..3845a2f688 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -25,7 +25,7 @@ use opendal::services::FsConfig; use crate::Result; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, }; /// Build new opendal operator from give path. @@ -118,11 +118,11 @@ impl Storage for OpenDALFsStorage { } } -/// Builder for OpenDAL Filesystem storage +/// Factory for OpenDAL Filesystem storage #[derive(Debug)] -pub struct OpenDALFsStorageBuilder; +pub struct OpenDALFsStorageFactory; -impl StorageBuilder for OpenDALFsStorageBuilder { +impl StorageFactory for OpenDALFsStorageFactory { fn build( &self, _props: HashMap, diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index f3dc0212a9..8f59b2c09c 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -26,7 +26,7 @@ use opendal::services::GcsConfig; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, is_truthy, }; use crate::{Error, ErrorKind, Result}; @@ -195,11 +195,11 @@ impl Storage for OpenDALGcsStorage { } } -/// Builder for OpenDAL GCS storage +/// Factory for OpenDAL GCS storage #[derive(Debug)] -pub struct OpenDALGcsStorageBuilder; +pub struct OpenDALGcsStorageFactory; -impl StorageBuilder for OpenDALGcsStorageBuilder { +impl StorageFactory for OpenDALGcsStorageFactory { fn build( &self, props: HashMap, diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index b72a0b7abc..f4eb789e32 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -25,7 +25,7 @@ use opendal::services::MemoryConfig; use crate::Result; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, }; /// Memory storage implementation using OpenDAL @@ -105,11 +105,11 @@ impl Storage for OpenDALMemoryStorage { } } -/// Builder for OpenDAL Memory storage +/// Factory for OpenDAL Memory storage #[derive(Debug)] -pub struct OpenDALMemoryStorageBuilder; +pub struct OpenDALMemoryStorageFactory; -impl StorageBuilder for OpenDALMemoryStorageBuilder { +impl StorageFactory for OpenDALMemoryStorageFactory { fn build( &self, _props: HashMap, diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index c070edef01..b8deb2ed32 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -25,7 +25,7 @@ use opendal::{Configurator, Operator}; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageFactory, }; use crate::{Error, ErrorKind, Result}; @@ -155,11 +155,11 @@ impl Storage for OpenDALOssStorage { } } -/// Builder for OpenDAL OSS storage +/// Factory for OpenDAL OSS storage #[derive(Debug)] -pub struct OpenDALOssStorageBuilder; +pub struct OpenDALOssStorageFactory; -impl StorageBuilder for OpenDALOssStorageBuilder { +impl StorageFactory for OpenDALOssStorageFactory { fn build( &self, props: HashMap, diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 2c167782cf..1a2117ce3d 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -28,7 +28,7 @@ use url::Url; use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, - Storage, StorageBuilder, is_truthy, + Storage, StorageFactory, is_truthy, }; use crate::{Error, ErrorKind, Result}; @@ -321,11 +321,11 @@ impl Storage for OpenDALS3Storage { } } -/// Builder for S3 storage +/// Factory for S3 storage #[derive(Debug)] -pub struct OpenDALS3StorageBuilder; +pub struct OpenDALS3StorageFactory; -impl StorageBuilder for OpenDALS3StorageBuilder { +impl StorageFactory for OpenDALS3StorageFactory { fn build( &self, props: HashMap, From b5ae637f62f13298c3e2d1b873a056200d31338d Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 1 Dec 2025 16:42:54 -0800 Subject: [PATCH 3/6] typetag serde for Storage --- Cargo.lock | 51 +++++++++++++++ Cargo.toml | 1 + crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/io/file_io.rs | 9 ++- crates/iceberg/src/io/storage.rs | 4 ++ crates/iceberg/src/io/storage_azdls.rs | 7 ++- crates/iceberg/src/io/storage_fs.rs | 30 ++++++++- crates/iceberg/src/io/storage_gcs.rs | 6 +- crates/iceberg/src/io/storage_memory.rs | 82 +++++++++++++++++++++---- crates/iceberg/src/io/storage_oss.rs | 6 +- crates/iceberg/src/io/storage_s3.rs | 34 +++++++++- 11 files changed, 212 insertions(+), 19 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 736f2b1461..271a2a0795 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2802,6 +2802,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" @@ -3573,6 +3584,7 @@ dependencies = [ "thrift", "tokio", "typed-builder", + "typetag", "url", "uuid", "zstd", @@ -3961,6 +3973,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" @@ -7416,12 +7437,42 @@ dependencies = [ "syn 2.0.108", ] +[[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.108", +] + [[package]] name = "typify" version = "0.4.3" diff --git a/Cargo.toml b/Cargo.toml index 0512155534..7b33fa3ef9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -128,6 +128,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 895a5cf5e4..a2f01c3ae9 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -90,6 +90,7 @@ strum = { workspace = true, features = ["derive"] } thrift = { workspace = true } 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/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index b7bab85980..868cda3b4c 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -523,6 +523,7 @@ mod tests { use bytes::Bytes; use futures::AsyncReadExt; use futures::io::AllowStdIo; + use serde::{Deserialize, Serialize}; use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; @@ -533,12 +534,17 @@ mod tests { use crate::{Error, ErrorKind, Result}; // Test storage implementation that tracks write operations - #[derive(Debug, Clone)] + #[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> { @@ -551,6 +557,7 @@ mod tests { } #[async_trait] + #[typetag::serde] impl Storage for TestStorage { async fn exists(&self, _path: &str) -> Result { Ok(true) diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 92ac0f936f..3569171c47 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -36,6 +36,9 @@ use crate::Result; /// 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 @@ -51,6 +54,7 @@ use crate::Result; /// } /// ``` #[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; diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 7938160866..5037b3d51a 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -132,7 +132,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, Clone, PartialEq)] +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] pub(crate) enum AzureStorageScheme { Abfs, Abfss, @@ -606,7 +606,9 @@ mod tests { } /// Azure Data Lake Storage implementation using OpenDAL -#[derive(Debug, Clone)] +/// +/// 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. @@ -625,6 +627,7 @@ impl OpenDALAzdlsStorage { } #[async_trait] +#[typetag::serde] impl Storage for OpenDALAzdlsStorage { async fn exists(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(path)?; diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index 3845a2f688..d40957a177 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -22,6 +22,7 @@ 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::{ @@ -37,7 +38,9 @@ pub(crate) fn fs_config_build() -> Result { } /// Filesystem storage implementation using OpenDAL -#[derive(Debug, Clone)] +/// +/// This storage is stateless and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] pub struct OpenDALFsStorage; impl OpenDALFsStorage { @@ -52,6 +55,7 @@ impl OpenDALFsStorage { } #[async_trait] +#[typetag::serde] impl Storage for OpenDALFsStorage { async fn exists(&self, path: &str) -> Result { let relative_path = self.extract_relative_path(path); @@ -131,3 +135,27 @@ impl StorageFactory for OpenDALFsStorageFactory { 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")); + } +} diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 8f59b2c09c..069c872005 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -23,6 +23,7 @@ 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::{ @@ -112,7 +113,9 @@ pub(crate) fn gcs_config_build(cfg: &GcsConfig, path: &str) -> Result } /// GCS storage implementation using OpenDAL -#[derive(Debug, Clone)] +/// +/// Stores configuration and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] pub struct OpenDALGcsStorage { config: Arc, } @@ -136,6 +139,7 @@ impl OpenDALGcsStorage { } #[async_trait] +#[typetag::serde] impl Storage for OpenDALGcsStorage { async fn exists(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(path)?; diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index f4eb789e32..0478b7ea0e 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -16,12 +16,13 @@ // under the License. use std::collections::HashMap; -use std::sync::Arc; +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::{ @@ -29,12 +30,38 @@ use crate::io::{ }; /// Memory storage implementation using OpenDAL -#[derive(Debug, Clone)] +/// +/// 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 { - op: Operator, + #[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:/") { @@ -46,28 +73,33 @@ impl OpenDALMemoryStorage { } #[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(self.op.exists(relative_path).await?) + 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 = self.op.stat(relative_path).await?; + 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(self.op.read(relative_path).await?.to_bytes()) + 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(self.op.reader(relative_path).await?)) + Ok(Box::new(op.reader(relative_path).await?)) } async fn write(&self, path: &str, bs: Bytes) -> Result<()> { @@ -77,23 +109,26 @@ impl Storage for OpenDALMemoryStorage { } async fn writer(&self, path: &str) -> Result> { + let op = self.get_operator()?; let relative_path = self.extract_relative_path(path); - Ok(Box::new(self.op.writer(relative_path).await?)) + 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(self.op.delete(relative_path).await?) + Ok(op.delete(relative_path).await?) } async fn remove_dir_all(&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(self.op.remove_all(&path).await?) + Ok(op.remove_all(&path).await?) } fn new_input(&self, path: &str) -> Result { @@ -115,7 +150,30 @@ impl StorageFactory for OpenDALMemoryStorageFactory { _props: HashMap, _extensions: Extensions, ) -> Result> { - let op = Operator::from_config(MemoryConfig::default())?.finish(); - Ok(Arc::new(OpenDALMemoryStorage { op })) + 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")); } } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index b8deb2ed32..817f7ff2f6 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -22,6 +22,7 @@ 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::{ @@ -72,7 +73,9 @@ pub(crate) fn oss_config_build(cfg: &OssConfig, path: &str) -> Result } /// OSS storage implementation using OpenDAL -#[derive(Debug, Clone)] +/// +/// Stores configuration and creates operators on-demand. +#[derive(Debug, Clone, Serialize, Deserialize)] pub struct OpenDALOssStorage { config: Arc, } @@ -96,6 +99,7 @@ impl OpenDALOssStorage { } #[async_trait] +#[typetag::serde] impl Storage for OpenDALOssStorage { async fn exists(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(path)?; diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 1a2117ce3d..7a6da2a4ad 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -24,6 +24,7 @@ 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::{ @@ -219,12 +220,16 @@ impl AwsCredentialLoad for CustomAwsCredentialLoader { } /// S3 storage implementation using OpenDAL -#[derive(Debug, Clone)] +/// +/// 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, } @@ -261,6 +266,7 @@ impl OpenDALS3Storage { } #[async_trait] +#[typetag::serde] impl Storage for OpenDALS3Storage { async fn exists(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(path)?; @@ -352,3 +358,29 @@ impl StorageFactory for OpenDALS3StorageFactory { })) } } + +#[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")); + } +} From 3cedfe10c3bd9b3a946548b7fe48a0e794872d9f Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 1 Dec 2025 16:46:27 -0800 Subject: [PATCH 4/6] rename to delete_prefix --- crates/iceberg/src/io/file_io.rs | 14 +++++++------- crates/iceberg/src/io/storage.rs | 4 ++-- crates/iceberg/src/io/storage_azdls.rs | 2 +- crates/iceberg/src/io/storage_fs.rs | 2 +- crates/iceberg/src/io/storage_gcs.rs | 2 +- crates/iceberg/src/io/storage_memory.rs | 2 +- crates/iceberg/src/io/storage_oss.rs | 2 +- crates/iceberg/src/io/storage_s3.rs | 2 +- 8 files changed, 15 insertions(+), 15 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 868cda3b4c..b6d304ae00 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -106,8 +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<()> { - self.inner.remove_dir_all(path.as_ref()).await + pub async fn delete_prefix(&self, path: impl AsRef) -> Result<()> { + self.inner.delete_prefix(path.as_ref()).await } /// Check file exists. @@ -594,7 +594,7 @@ mod tests { Ok(()) } - async fn remove_dir_all(&self, _path: &str) -> Result<()> { + async fn delete_prefix(&self, _path: &str) -> Result<()> { Ok(()) } @@ -692,14 +692,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()); @@ -718,7 +718,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] diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 3569171c47..8df3e14f6c 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -77,8 +77,8 @@ pub trait Storage: Debug + Send + Sync { /// Delete a file at the given path async fn delete(&self, path: &str) -> Result<()>; - /// Remove a directory and all its contents recursively - async fn remove_dir_all(&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; diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 5037b3d51a..4e5e2c8eed 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -668,7 +668,7 @@ impl Storage for OpenDALAzdlsStorage { Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> Result<()> { + 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() diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index d40957a177..f87c2e4823 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -102,7 +102,7 @@ impl Storage for OpenDALFsStorage { Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> Result<()> { + 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('/') { diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 069c872005..27c9c79420 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -180,7 +180,7 @@ impl Storage for OpenDALGcsStorage { Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> Result<()> { + 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() diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index 0478b7ea0e..a88475338e 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -120,7 +120,7 @@ impl Storage for OpenDALMemoryStorage { Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> Result<()> { + 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('/') { diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 817f7ff2f6..929eb016ff 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -140,7 +140,7 @@ impl Storage for OpenDALOssStorage { Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> Result<()> { + 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() diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 7a6da2a4ad..e99a0e4ab8 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -308,7 +308,7 @@ impl Storage for OpenDALS3Storage { Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> Result<()> { + 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() From 8922ccda317eae19734aa7133d555aef9d15d542 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 1 Dec 2025 16:56:44 -0800 Subject: [PATCH 5/6] typetag serde for StorageFactory --- crates/iceberg/src/io/file_io.rs | 9 ++++- crates/iceberg/src/io/storage.rs | 6 +++ crates/iceberg/src/io/storage_azdls.rs | 3 +- crates/iceberg/src/io/storage_fs.rs | 20 +++++++++- crates/iceberg/src/io/storage_gcs.rs | 3 +- crates/iceberg/src/io/storage_memory.rs | 51 ++++++++++++++++++++++++- crates/iceberg/src/io/storage_oss.rs | 3 +- crates/iceberg/src/io/storage_s3.rs | 20 +++++++++- 8 files changed, 108 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index b6d304ae00..56f1d4bb45 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -607,10 +607,16 @@ mod tests { } } + fn default_received_props() -> Arc>> { + Arc::new(Mutex::new(HashMap::new())) + } + // Test storage factory - #[derive(Debug)] + #[derive(Debug, Serialize, Deserialize)] struct TestStorageFactory { + #[serde(skip, default = "default_written")] written: Arc>>, + #[serde(skip, default = "default_received_props")] received_props: Arc>>, } @@ -624,6 +630,7 @@ mod tests { } } + #[typetag::serde] impl StorageFactory for TestStorageFactory { fn build( &self, diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 8df3e14f6c..f2bdb8943d 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -93,14 +93,19 @@ pub trait Storage: Debug + Send + Sync { /// 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, @@ -112,6 +117,7 @@ pub trait Storage: Debug + Send + Sync { /// } /// } /// ``` +#[typetag::serde(tag = "type")] pub trait StorageFactory: Debug + Send + Sync { /// Create a new storage instance with the given properties and extensions. /// diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 4e5e2c8eed..0d05bc0701 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -688,9 +688,10 @@ impl Storage for OpenDALAzdlsStorage { } /// Factory for Azure Data Lake Storage -#[derive(Debug)] +#[derive(Debug, Serialize, Deserialize)] pub struct OpenDALAzdlsStorageFactory; +#[typetag::serde] impl StorageFactory for OpenDALAzdlsStorageFactory { fn build( &self, diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index f87c2e4823..820cac393c 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -123,9 +123,10 @@ impl Storage for OpenDALFsStorage { } /// Factory for OpenDAL Filesystem storage -#[derive(Debug)] +#[derive(Debug, Serialize, Deserialize)] pub struct OpenDALFsStorageFactory; +#[typetag::serde] impl StorageFactory for OpenDALFsStorageFactory { fn build( &self, @@ -158,4 +159,21 @@ mod tests { // 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 27c9c79420..60425c73d4 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -200,9 +200,10 @@ impl Storage for OpenDALGcsStorage { } /// Factory for OpenDAL GCS storage -#[derive(Debug)] +#[derive(Debug, Serialize, Deserialize)] pub struct OpenDALGcsStorageFactory; +#[typetag::serde] impl StorageFactory for OpenDALGcsStorageFactory { fn build( &self, diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index a88475338e..b2e1ed5629 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -141,9 +141,10 @@ impl Storage for OpenDALMemoryStorage { } /// Factory for OpenDAL Memory storage -#[derive(Debug)] +#[derive(Debug, Serialize, Deserialize)] pub struct OpenDALMemoryStorageFactory; +#[typetag::serde] impl StorageFactory for OpenDALMemoryStorageFactory { fn build( &self, @@ -176,4 +177,52 @@ mod tests { // 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")); + } } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 929eb016ff..d2fff01f2a 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -160,9 +160,10 @@ impl Storage for OpenDALOssStorage { } /// Factory for OpenDAL OSS storage -#[derive(Debug)] +#[derive(Debug, Serialize, Deserialize)] pub struct OpenDALOssStorageFactory; +#[typetag::serde] impl StorageFactory for OpenDALOssStorageFactory { fn build( &self, diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index e99a0e4ab8..013a82fce3 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -328,9 +328,10 @@ impl Storage for OpenDALS3Storage { } /// Factory for S3 storage -#[derive(Debug)] +#[derive(Debug, Serialize, Deserialize)] pub struct OpenDALS3StorageFactory; +#[typetag::serde] impl StorageFactory for OpenDALS3StorageFactory { fn build( &self, @@ -383,4 +384,21 @@ mod tests { // 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 77f8b1d2635a48032b0d907f06f41c4738351c63 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 1 Dec 2025 17:04:58 -0800 Subject: [PATCH 6/6] minor --- crates/iceberg/src/io/storage_azdls.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 0d05bc0701..00bfd81a03 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -24,6 +24,7 @@ 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::{