pub trait ObjectStore: Display + Send + Sync + Debug + 'static {
Show 18 methods // Required methods fn put<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, bytes: Bytes ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn put_multipart<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<(MultipartId, Box<dyn AsyncWrite + Unpin + Send>)>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, multipart_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; fn get_opts<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, options: GetOptions ) -> Pin<Box<dyn Future<Output = Result<GetResult>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn head<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<ObjectMeta>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn delete<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn list<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn list_with_delimiter<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<ListResult>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait; fn copy<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait; fn copy_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait; // Provided methods fn append<'life0, 'life1, 'async_trait>( &'life0 self, _location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<Box<dyn AsyncWrite + Unpin + Send>>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait { ... } fn get<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<GetResult>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait { ... } fn get_range<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, range: Range<usize> ) -> Pin<Box<dyn Future<Output = Result<Bytes>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait { ... } fn get_ranges<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, ranges: &'life2 [Range<usize>] ) -> Pin<Box<dyn Future<Output = Result<Vec<Bytes>>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait { ... } fn delete_stream<'a>( &'a self, locations: BoxStream<'a, Result<Path>> ) -> BoxStream<'a, Result<Path>> { ... } fn list_with_offset<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path>, offset: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait { ... } fn rename<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait { ... } fn rename_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait { ... }
}
Expand description

Universal API to multiple object store services.

Required Methods§

source

fn put<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, bytes: Bytes ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Save the provided bytes to the specified location

The operation is guaranteed to be atomic, it will either successfully write the entirety of bytes to location, or fail. No clients should be able to observe a partially written object

source

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

Get a multi-part upload that allows writing data in chunks

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.

source

fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, multipart_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,

Cleanup an aborted upload.

See documentation for individual stores for exact behavior, as capabilities vary by object store.

source

fn get_opts<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, options: GetOptions ) -> Pin<Box<dyn Future<Output = Result<GetResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Perform a get request with options

source

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

Return the metadata for the specified location

source

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

Delete the object at the specified location.

source

fn list<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

List all the objects with the given prefix.

Prefixes are evaluated on a path segment basis, i.e. foo/bar/ is a prefix of foo/bar/x but not of foo/bar_baz/x.

Note: the order of returned ObjectMeta is not guaranteed

source

fn list_with_delimiter<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<ListResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

List objects with the given prefix and an implementation specific delimiter. Returns common prefixes (directories) in addition to object metadata.

Prefixes are evaluated on a path segment basis, i.e. foo/bar/ is a prefix of foo/bar/x but not of foo/bar_baz/x.

source

fn copy<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Copy an object from one path to another in the same object store.

If there exists an object at the destination, it will be overwritten.

source

fn copy_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Copy an object from one path to another, only if destination is empty.

Will return an error if the destination already has an object.

Performs an atomic operation if the underlying object storage supports it. If atomic operations are not supported by the underlying object storage (like S3) it will return an error.

Provided Methods§

source

fn append<'life0, 'life1, 'async_trait>( &'life0 self, _location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<Box<dyn AsyncWrite + Unpin + Send>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Returns an AsyncWrite that can be used to append to the object at location

A new object will be created if it doesn’t already exist, otherwise it will be opened, with subsequent writes appended to the end.

This operation cannot be supported by all stores, most use-cases should prefer ObjectStore::put and ObjectStore::put_multipart for better portability and stronger guarantees

This API is not guaranteed to be atomic, in particular

  • On error, location may contain partial data
  • Concurrent calls to ObjectStore::list may return partially written objects
  • Concurrent calls to ObjectStore::get may return partially written data
  • Concurrent calls to ObjectStore::put may result in data loss / corruption
  • Concurrent calls to ObjectStore::append may result in data loss / corruption

Additionally some stores, such as Azure, may only support appending to objects created with ObjectStore::append, and not with ObjectStore::put, ObjectStore::copy, or ObjectStore::put_multipart

source

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

Return the bytes that are stored at the specified location.

source

fn get_range<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, range: Range<usize> ) -> Pin<Box<dyn Future<Output = Result<Bytes>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Return the bytes that are stored at the specified location in the given byte range

source

fn get_ranges<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, ranges: &'life2 [Range<usize>] ) -> Pin<Box<dyn Future<Output = Result<Vec<Bytes>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Return the bytes that are stored at the specified location in the given byte ranges

source

fn delete_stream<'a>( &'a self, locations: BoxStream<'a, Result<Path>> ) -> BoxStream<'a, Result<Path>>

Delete all the objects at the specified locations

When supported, this method will use bulk operations that delete more than one object per a request. The default implementation will call the single object delete method for each location, but with up to 10 concurrent requests.

The returned stream yields the results of the delete operations in the same order as the input locations. However, some errors will be from an overall call to a bulk delete operation, and not from a specific location.

If the object did not exist, the result may be an error or a success, depending on the behavior of the underlying store. For example, local filesystems, GCP, and Azure return an error, while S3 and in-memory will return Ok. If it is an error, it will be Error::NotFound.

use object_store::{ObjectStore, ObjectMeta};
use object_store::path::Path;
use futures::{StreamExt, TryStreamExt};
use bytes::Bytes;

// Create two objects
store.put(&Path::from("foo"), Bytes::from("foo")).await?;
store.put(&Path::from("bar"), Bytes::from("bar")).await?;

// List object
let locations = store.list(None).await?
  .map(|meta: Result<ObjectMeta, _>| meta.map(|m| m.location))
  .boxed();

// Delete them
store.delete_stream(locations).try_collect::<Vec<Path>>().await?;
source

fn list_with_offset<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path>, offset: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

List all the objects with the given prefix and a location greater than offset

Some stores, such as S3 and GCS, may be able to push offset down to reduce the number of network requests required

Note: the order of returned ObjectMeta is not guaranteed

source

fn rename<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Move an object from one path to another in the same object store.

By default, this is implemented as a copy and then delete source. It may not check when deleting source that it was the same object that was originally copied.

If there exists an object at the destination, it will be overwritten.

source

fn rename_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Move an object from one path to another in the same object store.

Will return an error if the destination already has an object.

Trait Implementations§

source§

impl ObjectStore for Box<dyn ObjectStore>

source§

fn put<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, bytes: Bytes ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Save the provided bytes to the specified location Read more
source§

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

Get a multi-part upload that allows writing data in chunks Read more
source§

fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, multipart_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,

Cleanup an aborted upload. Read more
source§

fn append<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<Box<dyn AsyncWrite + Unpin + Send>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Returns an AsyncWrite that can be used to append to the object at location Read more
source§

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

Return the bytes that are stored at the specified location.
source§

fn get_opts<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, options: GetOptions ) -> Pin<Box<dyn Future<Output = Result<GetResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Perform a get request with options
source§

fn get_range<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, range: Range<usize> ) -> Pin<Box<dyn Future<Output = Result<Bytes>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Return the bytes that are stored at the specified location in the given byte range
source§

fn get_ranges<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, ranges: &'life2 [Range<usize>] ) -> Pin<Box<dyn Future<Output = Result<Vec<Bytes>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Return the bytes that are stored at the specified location in the given byte ranges
source§

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

Return the metadata for the specified location
source§

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

Delete the object at the specified location.
source§

fn delete_stream<'a>( &'a self, locations: BoxStream<'a, Result<Path>> ) -> BoxStream<'a, Result<Path>>

Delete all the objects at the specified locations Read more
source§

fn list<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

List all the objects with the given prefix. Read more
source§

fn list_with_offset<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path>, offset: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

List all the objects with the given prefix and a location greater than offset Read more
source§

fn list_with_delimiter<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<ListResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

List objects with the given prefix and an implementation specific delimiter. Returns common prefixes (directories) in addition to object metadata. Read more
source§

fn copy<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Copy an object from one path to another in the same object store. Read more
source§

fn rename<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Move an object from one path to another in the same object store. Read more
source§

fn copy_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Copy an object from one path to another, only if destination is empty. Read more
source§

fn rename_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Move an object from one path to another in the same object store. Read more

Implementations on Foreign Types§

source§

impl ObjectStore for Box<dyn ObjectStore>

source§

fn put<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, bytes: Bytes ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

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

source§

fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, multipart_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,

source§

fn append<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<Box<dyn AsyncWrite + Unpin + Send>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

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

source§

fn get_opts<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, options: GetOptions ) -> Pin<Box<dyn Future<Output = Result<GetResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn get_range<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, range: Range<usize> ) -> Pin<Box<dyn Future<Output = Result<Bytes>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn get_ranges<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, ranges: &'life2 [Range<usize>] ) -> Pin<Box<dyn Future<Output = Result<Vec<Bytes>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

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

source§

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

source§

fn delete_stream<'a>( &'a self, locations: BoxStream<'a, Result<Path>> ) -> BoxStream<'a, Result<Path>>

source§

fn list<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn list_with_offset<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path>, offset: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn list_with_delimiter<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<ListResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn copy<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn rename<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn copy_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn rename_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

impl ObjectStore for Arc<dyn ObjectStore>

source§

fn put<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, bytes: Bytes ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

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

source§

fn abort_multipart<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, multipart_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,

source§

fn append<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path ) -> Pin<Box<dyn Future<Output = Result<Box<dyn AsyncWrite + Unpin + Send>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

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

source§

fn get_opts<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, options: GetOptions ) -> Pin<Box<dyn Future<Output = Result<GetResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn get_range<'life0, 'life1, 'async_trait>( &'life0 self, location: &'life1 Path, range: Range<usize> ) -> Pin<Box<dyn Future<Output = Result<Bytes>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn get_ranges<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, location: &'life1 Path, ranges: &'life2 [Range<usize>] ) -> Pin<Box<dyn Future<Output = Result<Vec<Bytes>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

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

source§

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

source§

fn delete_stream<'a>( &'a self, locations: BoxStream<'a, Result<Path>> ) -> BoxStream<'a, Result<Path>>

source§

fn list<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn list_with_offset<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path>, offset: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<BoxStream<'_, Result<ObjectMeta>>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn list_with_delimiter<'life0, 'life1, 'async_trait>( &'life0 self, prefix: Option<&'life1 Path> ) -> Pin<Box<dyn Future<Output = Result<ListResult>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

source§

fn copy<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn rename<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn copy_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

source§

fn rename_if_not_exists<'life0, 'life1, 'life2, 'async_trait>( &'life0 self, from: &'life1 Path, to: &'life2 Path ) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait,

Implementors§