tustvold commented on code in PR #5500: URL: https://github.com/apache/arrow-rs/pull/5500#discussion_r1522796980
########## object_store/src/upload.rs: ########## @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{PutResult, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use futures::future::BoxFuture; +use tokio::task::JoinSet; + +/// An upload part request +pub type UploadPart = BoxFuture<'static, Result<()>>; + +#[async_trait] +pub trait Upload: Send + std::fmt::Debug { + /// Upload the next part + /// + /// Returns a stream Review Comment: ```suggestion ``` ########## object_store/src/upload.rs: ########## @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{PutResult, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use futures::future::BoxFuture; +use tokio::task::JoinSet; + +/// An upload part request +pub type UploadPart = BoxFuture<'static, Result<()>>; + +#[async_trait] +pub trait Upload: Send + std::fmt::Debug { + /// Upload the next part + /// + /// Returns a stream + /// + /// Most stores require that all parts excluding the last are at least 5 MiB, and some + /// further require that all parts excluding the last be the same size, e.g. [R2]. + /// Clients wanting to maximise compatibility should therefore perform writes in + /// fixed size blocks larger than 5 MiB. + /// + /// Implementations may invoke this method multiple times and then await on the + /// returned futures in parallel + /// + /// ```no_run + /// # use futures::StreamExt; + /// # use object_store::Upload; + /// # + /// # async fn test() { + /// # + /// let mut upload: Box<&dyn Upload> = todo!(); + /// let mut p1 = upload.put_part(vec![0; 10 * 1024 * 1024].into()); + /// let mut p2 = upload.put_part(vec![1; 10 * 1024 * 1024].into()); + /// + /// let (u1, u2) = futures::future::join(p1.next(), p2.next()).await; + /// u1.unwrap().unwrap(); + /// u2.unwrap().unwrap(); + /// let result = upload.complete().await.unwrap(); + /// # } + /// ``` + /// + /// [R2]: https://developers.cloudflare.com/r2/objects/multipart-objects/#limitations + fn put_part(&mut self, data: Bytes) -> UploadPart; + + /// Complete the multipart upload + async fn complete(&mut self) -> Result<PutResult>; + + /// Abort the multipart upload + /// + /// It is implementation defined behaviour if called concurrently with [`UploadPart::execute`] + async fn abort(&mut self) -> Result<()>; +} + +/// A synchronous write API for uploading data in parallel +/// +/// Makes use of [`JoinSet`] under the hood to multiplex upload tasks, +/// avoiding issues caused by sharing a single tokio's cooperative task +/// budget across multiple IO operations. +/// +/// The design also takes inspiration from [`Sink`] with [`ChunkedUpload::wait_for_capacity`] +/// allowing back pressure on producers, prior to buffering the next part. However, unlike +/// [`Sink`] this back pressure is optional, allowing integration with synchronous producers +/// +/// [`Sink`]: futures::sink::Sink +#[derive(Debug)] +pub struct ChunkedUpload { + upload: Box<dyn Upload>, + + buffer: Vec<u8>, + + tasks: JoinSet<Result<()>>, +} + +impl ChunkedUpload { + /// Create a new [`ChunkedUpload`] + pub fn new(upload: Box<dyn Upload>) -> Self { + Self::new_with_capacity(upload, 5 * 1024 * 1024) + } + + /// Create a new [`ChunkedUpload`] that will upload in fixed `capacity` sized chunks + pub fn new_with_capacity(upload: Box<dyn Upload>, capacity: usize) -> Self { + Self { + upload, + buffer: Vec::with_capacity(capacity), + tasks: Default::default(), + } + } + + /// Wait until there are `max_concurrency` or fewer requests in-flight + pub async fn wait_for_capacity(&mut self, max_concurrency: usize) -> Result<()> { + while self.tasks.len() > max_concurrency { + self.tasks.join_next().await.unwrap()??; + } + Ok(()) + } + + /// Write data to this [`ChunkedUpload`] + /// + /// Back pressure can optionally be applied to producers by calling + /// [`Self::wait_for_capacity`] prior to calling this method + pub fn write(&mut self, mut buf: &[u8]) { Review Comment: It is worth highlighting that this is a synchronous method, and so users could wrap this in a type and feed it directly into a synchronous writer such as parquet's ArrowWriter. If back pressure is required they could potentially call `wait_for_capacity` between row groups. ########## object_store/src/upload.rs: ########## @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{PutResult, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use futures::future::BoxFuture; +use tokio::task::JoinSet; + +/// An upload part request +pub type UploadPart = BoxFuture<'static, Result<()>>; + +#[async_trait] +pub trait Upload: Send + std::fmt::Debug { + /// Upload the next part + /// + /// Returns a stream + /// + /// Most stores require that all parts excluding the last are at least 5 MiB, and some + /// further require that all parts excluding the last be the same size, e.g. [R2]. + /// Clients wanting to maximise compatibility should therefore perform writes in + /// fixed size blocks larger than 5 MiB. + /// + /// Implementations may invoke this method multiple times and then await on the + /// returned futures in parallel + /// + /// ```no_run + /// # use futures::StreamExt; + /// # use object_store::Upload; + /// # + /// # async fn test() { + /// # + /// let mut upload: Box<&dyn Upload> = todo!(); + /// let mut p1 = upload.put_part(vec![0; 10 * 1024 * 1024].into()); + /// let mut p2 = upload.put_part(vec![1; 10 * 1024 * 1024].into()); + /// + /// let (u1, u2) = futures::future::join(p1.next(), p2.next()).await; + /// u1.unwrap().unwrap(); + /// u2.unwrap().unwrap(); + /// let result = upload.complete().await.unwrap(); + /// # } + /// ``` + /// + /// [R2]: https://developers.cloudflare.com/r2/objects/multipart-objects/#limitations + fn put_part(&mut self, data: Bytes) -> UploadPart; Review Comment: The design of this is somewhat subtle. A part index would run into issues for `LocalFilesystem` as we would need to know the offset to write the chunk to. If we permit out of order writes, e.g. writing the final partial chunk first, this chunk size would need to be provided at creation time of `Upload`. Aside from being an unfortunate API, this which would create non-trivial behaviour differences between stores that respect this config and those that ignore it. Instead by taking a mutable borrow and providing the `data` to be written, we prohibit out of order writes. The final piece is we return a `BoxFuture<'static, Result<()>>` instead of this being an `async fn`, i.e. returning `BoxFuture<'_, Result<()>>`. This allows multiple `UploadPart` to be created and polled in parallel, without the borrow checker complaining about concurrent mutable borrows. This is strictly a more flexible interface than the current AsyncWrite API, and whilst it still doesn't permit out of order writes, use-cases that require this can use MultiPartStore. ########## object_store/src/lib.rs: ########## @@ -538,48 +539,11 @@ pub trait ObjectStore: std::fmt::Display + Send + Sync + Debug + 'static { /// Save the provided bytes to the specified location with the given options async fn put_opts(&self, location: &Path, bytes: Bytes, opts: PutOptions) -> Result<PutResult>; - /// Get a multi-part upload that allows writing data in chunks. + /// Perform a multipart upload /// - /// Most cloud-based uploads will buffer and upload parts in parallel. - /// - /// To complete the upload, [AsyncWrite::poll_shutdown] must be called - /// to completion. This operation is guaranteed to be atomic, it will either - /// make all the written data available at `location`, or fail. No clients - /// should be able to observe a partially written object. - /// - /// For some object stores (S3, GCS, and local in particular), if the - /// writer fails or panics, you must call [ObjectStore::abort_multipart] - /// to clean up partially written data. - /// - /// <div class="warning"> - /// It is recommended applications wait for any in-flight requests to complete by calling `flush`, if - /// there may be a significant gap in time (> ~30s) before the next write. - /// These gaps can include times where the function returns control to the - /// caller while keeping the writer open. If `flush` is not called, futures - /// for in-flight requests may be left unpolled long enough for the requests - /// to time out, causing the write to fail. - /// </div> - /// - /// For applications requiring fine-grained control of multipart uploads - /// see [`MultiPartStore`], although note that this interface cannot be - /// supported by all [`ObjectStore`] backends. - /// - /// For applications looking to implement this interface for a custom - /// multipart API, see [`WriteMultiPart`] which handles the complexities - /// of performing parallel uploads of fixed size parts. - /// - /// [`WriteMultiPart`]: multipart::WriteMultiPart - /// [`MultiPartStore`]: multipart::MultiPartStore - async fn put_multipart( - &self, - location: &Path, - ) -> Result<(MultipartId, Box<dyn AsyncWrite + Unpin + Send>)>; - - /// Cleanup an aborted upload. - /// - /// See documentation for individual stores for exact behavior, as capabilities - /// vary by object store. - async fn abort_multipart(&self, location: &Path, multipart_id: &MultipartId) -> Result<()>; + /// Client should prefer [`ObjectStore::put`] for small payloads, as streaming uploads + /// typically require multiple separate requests. See [`Upload`] for more information + async fn upload(&self, location: &Path) -> Result<Box<dyn Upload>>; Review Comment: Unlike before we don't expose a notion of `MultipartId`. In hindsight this makes for a confusing API. We instead encourage users to configure automatic cleanup of incomplete uploads, which is the more reliable mechanism anyway ########## object_store/src/lib.rs: ########## @@ -538,48 +539,11 @@ pub trait ObjectStore: std::fmt::Display + Send + Sync + Debug + 'static { /// Save the provided bytes to the specified location with the given options async fn put_opts(&self, location: &Path, bytes: Bytes, opts: PutOptions) -> Result<PutResult>; - /// Get a multi-part upload that allows writing data in chunks. + /// Perform a multipart upload /// - /// Most cloud-based uploads will buffer and upload parts in parallel. - /// - /// To complete the upload, [AsyncWrite::poll_shutdown] must be called - /// to completion. This operation is guaranteed to be atomic, it will either - /// make all the written data available at `location`, or fail. No clients - /// should be able to observe a partially written object. - /// - /// For some object stores (S3, GCS, and local in particular), if the - /// writer fails or panics, you must call [ObjectStore::abort_multipart] - /// to clean up partially written data. - /// - /// <div class="warning"> - /// It is recommended applications wait for any in-flight requests to complete by calling `flush`, if - /// there may be a significant gap in time (> ~30s) before the next write. - /// These gaps can include times where the function returns control to the - /// caller while keeping the writer open. If `flush` is not called, futures - /// for in-flight requests may be left unpolled long enough for the requests - /// to time out, causing the write to fail. - /// </div> Review Comment: This warning is no longer necessary :tada: ########## object_store/src/limit.rs: ########## @@ -81,18 +81,12 @@ impl<T: ObjectStore> ObjectStore for LimitStore<T> { let _permit = self.semaphore.acquire().await.unwrap(); self.inner.put_opts(location, bytes, opts).await } - async fn put_multipart( - &self, - location: &Path, - ) -> Result<(MultipartId, Box<dyn AsyncWrite + Unpin + Send>)> { - let permit = Arc::clone(&self.semaphore).acquire_owned().await.unwrap(); - let (id, write) = self.inner.put_multipart(location).await?; - Ok((id, Box::new(PermitWrapper::new(write, permit)))) - } - - async fn abort_multipart(&self, location: &Path, multipart_id: &MultipartId) -> Result<()> { - let _permit = self.semaphore.acquire().await.unwrap(); - self.inner.abort_multipart(location, multipart_id).await + async fn upload(&self, location: &Path) -> Result<Box<dyn Upload>> { Review Comment: We can actually properly limit/throttle uploads now, as the public APIs now mirror the underlying requests :tada: ########## object_store/src/upload.rs: ########## @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{PutResult, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use futures::future::BoxFuture; +use tokio::task::JoinSet; + +/// An upload part request +pub type UploadPart = BoxFuture<'static, Result<()>>; Review Comment: A previous version of this returned `BoxStream<'static, Result<()>>` to allow for retries, but this was a tad confusing and tbh I suspect it would get little use in practice. If people want fine-grained retries they can use MultiPartStore directly ########## object_store/src/upload.rs: ########## @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{PutResult, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use futures::future::BoxFuture; +use tokio::task::JoinSet; + +/// An upload part request +pub type UploadPart = BoxFuture<'static, Result<()>>; + +#[async_trait] +pub trait Upload: Send + std::fmt::Debug { + /// Upload the next part + /// + /// Returns a stream + /// + /// Most stores require that all parts excluding the last are at least 5 MiB, and some + /// further require that all parts excluding the last be the same size, e.g. [R2]. + /// Clients wanting to maximise compatibility should therefore perform writes in + /// fixed size blocks larger than 5 MiB. + /// + /// Implementations may invoke this method multiple times and then await on the + /// returned futures in parallel + /// + /// ```no_run + /// # use futures::StreamExt; + /// # use object_store::Upload; + /// # + /// # async fn test() { + /// # + /// let mut upload: Box<&dyn Upload> = todo!(); + /// let mut p1 = upload.put_part(vec![0; 10 * 1024 * 1024].into()); + /// let mut p2 = upload.put_part(vec![1; 10 * 1024 * 1024].into()); + /// + /// let (u1, u2) = futures::future::join(p1.next(), p2.next()).await; + /// u1.unwrap().unwrap(); + /// u2.unwrap().unwrap(); + /// let result = upload.complete().await.unwrap(); + /// # } + /// ``` + /// + /// [R2]: https://developers.cloudflare.com/r2/objects/multipart-objects/#limitations + fn put_part(&mut self, data: Bytes) -> UploadPart; + + /// Complete the multipart upload + async fn complete(&mut self) -> Result<PutResult>; + + /// Abort the multipart upload + /// + /// It is implementation defined behaviour if called concurrently with [`UploadPart::execute`] + async fn abort(&mut self) -> Result<()>; +} + +/// A synchronous write API for uploading data in parallel +/// +/// Makes use of [`JoinSet`] under the hood to multiplex upload tasks, +/// avoiding issues caused by sharing a single tokio's cooperative task +/// budget across multiple IO operations. +/// +/// The design also takes inspiration from [`Sink`] with [`ChunkedUpload::wait_for_capacity`] +/// allowing back pressure on producers, prior to buffering the next part. However, unlike +/// [`Sink`] this back pressure is optional, allowing integration with synchronous producers +/// +/// [`Sink`]: futures::sink::Sink +#[derive(Debug)] +pub struct ChunkedUpload { Review Comment: This is not only significantly simpler than the WriteUpload it replaces, but avoids a number of the issues. I think it is also a nice example of the flexibility of the `Upload` API, if a downstream wants to handle chunking/concurrency differently they're entirely able to do so. ########## object_store/src/upload.rs: ########## @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{PutResult, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use futures::future::BoxFuture; +use tokio::task::JoinSet; + +/// An upload part request +pub type UploadPart = BoxFuture<'static, Result<()>>; + +#[async_trait] +pub trait Upload: Send + std::fmt::Debug { + /// Upload the next part + /// + /// Returns a stream + /// + /// Most stores require that all parts excluding the last are at least 5 MiB, and some + /// further require that all parts excluding the last be the same size, e.g. [R2]. + /// Clients wanting to maximise compatibility should therefore perform writes in + /// fixed size blocks larger than 5 MiB. + /// + /// Implementations may invoke this method multiple times and then await on the + /// returned futures in parallel + /// + /// ```no_run + /// # use futures::StreamExt; + /// # use object_store::Upload; + /// # + /// # async fn test() { + /// # + /// let mut upload: Box<&dyn Upload> = todo!(); + /// let mut p1 = upload.put_part(vec![0; 10 * 1024 * 1024].into()); + /// let mut p2 = upload.put_part(vec![1; 10 * 1024 * 1024].into()); + /// + /// let (u1, u2) = futures::future::join(p1.next(), p2.next()).await; + /// u1.unwrap().unwrap(); + /// u2.unwrap().unwrap(); + /// let result = upload.complete().await.unwrap(); + /// # } + /// ``` + /// + /// [R2]: https://developers.cloudflare.com/r2/objects/multipart-objects/#limitations + fn put_part(&mut self, data: Bytes) -> UploadPart; + + /// Complete the multipart upload + async fn complete(&mut self) -> Result<PutResult>; Review Comment: Unlike `MultiPartStore` we expect `Upload` to handle the `PartId` internally, I think this makes for a more intuitive interface and avoids issues relating to the behaviour if the `PartId` are provided out of order. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
