tustvold commented on code in PR #5500:
URL: https://github.com/apache/arrow-rs/pull/5500#discussion_r1527934511


##########
object_store/src/upload.rs:
##########
@@ -0,0 +1,172 @@
+// 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<()>>;
+
+/// A trait allowing writing an object in fixed size chunks
+///
+/// Consecutive chunks of data can be written by calling 
[`MultipartUpload::put_part`] and polling
+/// the returned futures to completion. Multiple futures returned by 
[`MultipartUpload::put_part`]
+/// may be polled in parallel, allowing for concurrent uploads.
+///
+/// Once all part uploads have been polled to completion, the upload can be 
completed by
+/// calling [`MultipartUpload::complete`]. This will make the entire uploaded 
object visible
+/// as an atomic operation.It is implementation behind behaviour if 
[`MultipartUpload::complete`]
+/// is called before all [`UploadPart`] have been polled to completion.
+#[async_trait]
+pub trait MultipartUpload: Send + std::fmt::Debug {
+    /// Upload the next part
+    ///
+    /// 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::MultipartUpload;
+    /// #
+    /// # async fn test() {
+    /// #
+    /// let mut upload: Box<&dyn MultipartUpload> = todo!();
+    /// let p1 = upload.put_part(vec![0; 10 * 1024 * 1024].into());
+    /// let p2 = upload.put_part(vec![1; 10 * 1024 * 1024].into());
+    /// futures::future::try_join(p1, p2).await.unwrap();
+    /// 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
+    ///
+    /// It is implementation defined behaviour if this method is called before 
polling
+    /// all [`UploadPart`] returned by [`MultipartUpload::put_part`] to 
completion. Additionally,
+    /// it is implementation defined behaviour to call 
[`MultipartUpload::complete`]
+    /// on an already completed or aborted [`MultipartUpload`].
+    async fn complete(&mut self) -> Result<PutResult>;
+
+    /// Abort the multipart upload
+    ///
+    /// If a [`MultipartUpload`] is dropped without calling 
[`MultipartUpload::complete`],
+    /// some implementations will automatically reap any uploaded parts. 
However,
+    /// this is not always possible, e.g. for S3 and GCS. 
[`MultipartUpload::abort`] can
+    /// therefore be invoked to perform this cleanup.
+    ///
+    /// It is recommended that where possible users configure lifecycle rules
+    /// to automatically reap unused parts older than some threshold, as this
+    /// will more reliably handle different failure modes. See [crate::aws] and
+    /// [crate::gcp] for more information.
+    ///
+    /// It is implementation defined behaviour to call 
[`MultipartUpload::abort`]
+    /// on an already completed or aborted [`MultipartUpload`]
+    async fn abort(&mut self) -> Result<()>;

Review Comment:
   I am very tempted to just not include this, in favour of just encouraging 
people to configure appropriate lifecycle rules, but am curious what other 
people think



-- 
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]

Reply via email to