Xuanwo opened a new issue, #356: URL: https://github.com/apache/iceberg-rust/issues/356
Hi, iceberger! OpenDAL's coming v0.46 release will have API changes that could affect our project. ## Changes I list the major changes that we need to take care: ### OpenDAL Reader doens't impl `AsyncRead + AsyncSeek` anymore OpenDAL's Reader now transformed into range based read. ```rust let r = op.reader("test.txt").await?; let buf = r.read(1024..2048).await?; ``` Users can transform into `AsyncRead + AsyncSeek` by using `into_futures_async_read`: ```rust let r = op.reader("test.txt").await?; let reader: FuturesAsyncReader = r.into_futures_async_read(0..4096); ``` But please note: - `opendal::Reader` adopts zero-cost abstraction, no extra bytes copy and allocation happened. - `opendal::FuturesAsyncReader` is the same as our old reader, it might have extra bytes copy. ### OpenDAL Writer doens't impl `AsyncWrite` anymore Just like Reader, `opendal::Writer` doesn't impl `AsyncWrite` anymore. Users could use opendal's native `Buffer` for both contiguous and non-contiguous buffers support. ```rust let w = op.writer("test.txt").await?; // Buffer can be created from continues bytes. w.write("hello, world").await?; // Buffer can also be created from non-continues bytes. w.write(vec![Bytes::from("hello,"), Bytes::from("world!")]).await?; // Make sure file has been written completely. w.close().await?; ``` Users can transform into `AsyncWrite` by using `into_futures_async_write`: ```rust let w = op.writer("test.txt").await?; let writer: FuturesAsyncWriter = r.into_futures_async_write(); ``` ### OpenDAL Reader now has concurrent support OpenDAL Reader now has concurrent support that can read multiple chunks concurrently. ```rust let r = op.reader_with("test.txt").concurrent(4).chunk(4 * 1024 * 1024).await?; let buf = r.read(0..16 * 1024 * 1024).await?; ``` The `buf` here will be fetched in 4 concurrent requests. To read non-contiguous buffers, please use our `fetch` API: ```rust let r = op.reader_with("test.txt").concurrent(4).chunk(4 * 1024 * 1024).await?; let bufs = r.fetch(vec![0..1024 * 1024, 1024..3 * 1024 * 1024]).await?; ``` OpenDAL will merge close ranges and read them concurrently. The detailed upgrade guide could be found [here](https://opendal.apache.org/docs/rust/opendal/docs/upgrade/index.html#upgrade-to-v046). OpenDAL v0.46 is not related yet so those changes are still possible to be altered. I will try my best to keep this issue update. ## Tasks Although it's possible to simply convert opendal's `Reader` and `Writer` into `AsyncXxx`-based structures, I aim to prepare Iceberg for the most efficient IO methods. In the near future, we will support compilation-based IO and vectorization. The `AsyncXxx`-based traits do not integrate well with these methods. - [ ] Polish the iceberg `Reader` related API - [ ] Polish the iceberg `Writer` realted API I believe only read side needs to do some changes. write side should be simple to update. -- 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.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