Trait object_store::multipart::MultipartStore

source ·
pub trait MultipartStore: Send + Sync + 'static {
    // Required methods
    fn create_multipart<'life0, 'life1, 'async_trait>(
        &'life0 self,
        path: &'life1 Path,
    ) -> Pin<Box<dyn Future<Output = Result<MultipartId>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait,
             'life1: 'async_trait;
    fn put_part<'life0, 'life1, 'life2, 'async_trait>(
        &'life0 self,
        path: &'life1 Path,
        id: &'life2 MultipartId,
        part_idx: usize,
        data: PutPayload,
    ) -> Pin<Box<dyn Future<Output = Result<PartId>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait,
             'life1: 'async_trait,
             'life2: 'async_trait;
    fn complete_multipart<'life0, 'life1, 'life2, 'async_trait>(
        &'life0 self,
        path: &'life1 Path,
        id: &'life2 MultipartId,
        parts: Vec<PartId>,
    ) -> Pin<Box<dyn Future<Output = Result<PutResult>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait,
             'life1: 'async_trait,
             'life2: 'async_trait;
    fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>(
        &'life0 self,
        path: &'life1 Path,
        id: &'life2 MultipartId,
    ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait,
             'life1: 'async_trait,
             'life2: 'async_trait;
}
Expand description

A low-level interface for interacting with multipart upload APIs

Most use-cases should prefer ObjectStore::put_multipart as this is supported by more backends, including LocalFileSystem, and automatically handles uploading fixed size parts of sufficient size in parallel

Required Methods§

source

fn create_multipart<'life0, 'life1, 'async_trait>( &'life0 self, path: &'life1 Path, ) -> Pin<Box<dyn Future<Output = Result<MultipartId>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Creates a new multipart upload, returning the MultipartId

source

fn put_part<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, path: &'life1 Path, id: &'life2 MultipartId, part_idx: usize, data: PutPayload, ) -> Pin<Box<dyn Future<Output = Result<PartId>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Uploads a new part with index part_idx

part_idx should be an integer in the range 0..N where N is the number of parts in the upload. Parts may be uploaded concurrently and in any order.

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. WriteMultipart performs writes in fixed size blocks of 5 MiB, and clients wanting to maximise compatibility should look to do likewise.

source

fn complete_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, path: &'life1 Path, id: &'life2 MultipartId, parts: Vec<PartId>, ) -> Pin<Box<dyn Future<Output = Result<PutResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Completes a multipart upload

The i’th value of parts must be a PartId returned by a call to Self::put_part with a part_idx of i, and the same path and id as provided to this method. Calling this method with out of sequence or repeated PartId, or PartId returned for other values of path or id, will result in implementation-defined behaviour

source

fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, path: &'life1 Path, id: &'life2 MultipartId, ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Aborts a multipart upload

Implementors§