Trait object_store::MultipartUpload

source ·
pub trait MultipartUpload: Send + Debug {
    // Required methods
    fn put_part(&mut self, data: PutPayload) -> UploadPart;
    fn complete<'life0, 'async_trait>(
        &'life0 mut self,
    ) -> Pin<Box<dyn Future<Output = Result<PutResult>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait;
    fn abort<'life0, 'async_trait>(
        &'life0 mut self,
    ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait;
}
Expand description

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.

Required Methods§

source

fn put_part(&mut self, data: PutPayload) -> UploadPart

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

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();
source

fn complete<'life0, 'async_trait>( &'life0 mut self, ) -> Pin<Box<dyn Future<Output = Result<PutResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait,

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.

source

fn abort<'life0, 'async_trait>( &'life0 mut self, ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait,

Abort the multipart upload

If a MultipartUpload is dropped without calling MultipartUpload::complete, some object stores will automatically clean up any previously uploaded parts. However, some stores, such as S3 and GCS, cannot perform cleanup on drop. As such MultipartUpload::abort can be invoked to perform this cleanup.

It will not be possible to call abort in all failure scenarios, for example non-graceful shutdown of the calling application. It is therefore recommended object stores are configured with lifecycle rules to automatically cleanup unused parts older than some threshold. 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

Implementations on Foreign Types§

source§

impl<W: MultipartUpload + ?Sized> MultipartUpload for Box<W>

source§

fn put_part(&mut self, data: PutPayload) -> UploadPart

source§

fn complete<'life0, 'async_trait>( &'life0 mut self, ) -> Pin<Box<dyn Future<Output = Result<PutResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait,

source§

fn abort<'life0, 'async_trait>( &'life0 mut self, ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait,

Implementors§