BlakeOrth commented on issue #172:
URL: https://github.com/apache/iceberg-rust/issues/172#issuecomment-2486497892

   @liurenjie1024 I have taken some time to explore an implementation based on 
your suggestion above, just as I did for the user extensible `Storage` proposed 
earlier. Unfortunately, I don't currently see a way to implement what you've 
suggested without either some API breaking change or making every component in 
the FileIO chain (`Storage`, `FileIO`, `InputFile` and `OutputFile`) aware of 
their concrete implementation, effectively rendering the proposed `Storage` 
trait somewhat moot. I'll summarize the issues here, and would be happy to push 
a draft PR if anyone would like to look at the full implementation details.
   
   There's a chance I didn't fully grasp your proposed solution, so if this 
solution isn't what you had in mind, please let me know. Based on your 
proposal, I changed `Storage` from an enum to a trait that fulfills the basic 
functions of the existing `FileIO` API.
   
   ```rust
   #[async_trait::async_trait]
   pub(crate) trait Storage: Debug {
       async fn create_reader(&self, path: &str) -> Result<Arc<dyn FileRead>>;
       async fn create_writer(&self, path: &str) -> Result<Arc<dyn FileWrite>>;
       async fn metadata(&self, path: &str) -> Result<FileMetadata>;
       async fn exists(&self, path: &str) -> Result<bool>;
       async fn delete(&self, path: &str) -> Result<()>;
   }
   ```
   
   Ideally I believe `FileIO`, `InputFile` and `OutputFile` would end up 
looking as follows:
   
   ```rust
   #[derive(Clone, Debug)]
   pub struct FileIO {
       inner: Arc<dyn Storage>,
   }
   
   pub struct InputFile {
       storage: Arc<dyn Storage>,
       path: String,
   }
   
   pub struct OutputFile {
       storage: Arc<dyn Storage>,
       path: String,
   }
   ```
   
   It seems like the `Storage` trait as defined above should allow each of them 
to operate as expected, but the API for both `InputFile` and `OutputFile` have 
methods that are incompatible.
   
   ```rust
   impl InputFile {
       ...
       pub async fn reader(&self) -> crate::Result<impl FileRead> {
           ...
       }
   }
   
   impl OutputFile {
       ...
       pub async fn writer(&self) -> crate::Result<Box<dyn FileWrite>> {
          ...
       }
   }
   ```
   
   Since both of these methods return an owned trait implementation the 
problems with them are similar. For `reader` it seemed simple enough to write 
`impl FileRead for Arc<dyn FileRead>`, but this unfortunately requires 
`FileRead` be become `Send` which breaks the existing API. For `writer` the 
only way to consistently go from an `Arc<dyn FileWrite>` to a `Box<dyn 
FileWrite>` would be for `FileWrite` to be `Clone`, which breaks the existing 
API.
   
   To get around the above issues I experimented with embedding an 
implementation specific `Config` inside `InputFile`, `OutputFile` that allows 
the full creation of a new, owned `FileRead` and `FileWrite`. While this seems 
possible to do, it does make every component implementation specific which I 
feel goes against the purpose of the proposed `Storage` trait.
   
   As I said earlier, I'm happy to post a draft PR with the full change set 
(there are multiple other items to work out that I'm ignoring here) if anyone 
would like to see it. If there are other ideas on how to make the `Storage` 
trait work with the above methods I'd be happy to explore additional changes 
there as well.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to