pub struct ByteWriter { /* private fields */ }
Expand description

Allow for writing raw bytes directly to a segment.

ByteWriter does not frame, attach headers, or otherwise modify the bytes written to it in any way. So unlike EventWriter the data written cannot be split apart when read. As such, any bytes written by this API can ONLY be read using ByteReader.

§Atomicity

If buf length is less than or equal to 8 MiB, the entire buffer will be written atomically. If buf length is greater than 8 MiB, only the first 8 MiB will be written, and it will be written atomically. In either case, the actual number of bytes written will be returned and those bytes are written atomically.

§Parallelism

Multiple ByteWriters write to the same segment as this will result in interleaved data, which is not desirable in most cases. ByteWriter uses Conditional Append to make sure that writers are aware of the content in the segment. If another process writes data to the segment after this one began writing, all subsequent writes from this writer will not be written and flush will fail. This prevents data from being accidentally interleaved.

§Backpressure

Write has a backpressure mechanism. Internally, it uses Channel to send event to Reactor for processing. Channel can has a limited capacity, when its capacity is reached, any further write will not be accepted until enough space has been freed in the Channel.

§Retry

The ByteWriter implementation provides retry logic to handle connection failures and service host failures. Internal retries will not violate the exactly once semantic so it is better to rely on them than to wrap this with custom retry logic.

§Examples

use pravega_client_config::ClientConfigBuilder;
use pravega_client::client_factory::ClientFactoryAsync;
use pravega_client_shared::ScopedStream;
use pravega_client::byte::ByteWriter;
use tokio::runtime::Handle;

#[tokio::main]
async fn main() {
    // assuming Pravega controller is running at endpoint `localhost:9090`
    let config = ClientConfigBuilder::default()
        .controller_uri("localhost:9090")
        .build()
        .expect("creating config");

    let handle = Handle::current();
    let client_factory = ClientFactoryAsync::new(config, handle);

    // assuming scope:myscope, stream:mystream exist.
    // notice that this stream should be a fixed sized single segment stream
    let stream = ScopedStream::from("myscope/mystream");

    let mut byte_writer = client_factory.create_byte_writer(stream).await;

    let payload = "hello world".to_string().into_bytes();

    // It doesn't mean the data is persisted on the server side
    // when write method returns Ok, user should call flush to ensure
    // all data has been acknowledged by the server.
    byte_writer.write(&payload).await.expect("write");
    byte_writer.flush().await.expect("flush");
}

Implementations§

source§

impl ByteWriter

source

pub async fn write(&mut self, buf: &[u8]) -> Result<usize, Error>

Writes the given data to the server asynchronously. It doesn’t mean the data is persisted on the server side when this method returns Ok, user should call flush to ensure all data has been acknowledged by the server.

§Examples
let mut byte_writer = client_factory.create_byte_writer(segment).await;
let payload = vec![0; 8];
let size = byte_writer.write(&payload).await;
source

pub async fn flush(&mut self) -> Result<(), Error>

Flush data.

It will wait until all pending appends have acknowledgment.

§Examples
let mut byte_writer = client_factory.create_byte_writer(segment).await;
let payload = vec![0; 8];
let size = byte_writer.write(&payload).await;
byte_writer.flush().await;
source

pub async fn seal(&mut self) -> Result<(), Error>

Seal the segment and no further writes are allowed.

§Examples
let mut byte_writer = client_factory.create_byte_writer(segment).await;
byte_writer.seal().await.expect("seal segment");
source

pub async fn truncate_data_before(&self, offset: i64) -> Result<(), Error>

Truncate data before a given offset for the segment. No reads are allowed before truncation point after calling this method.

§Examples
let byte_writer = client_factory.create_byte_writer(segment).await;
byte_writer.truncate_data_before(1024).await.expect("truncate segment");
source

pub fn current_offset(&self) -> u64

Track the current write position for this writer.

§Examples
let byte_writer = client_factory.create_byte_writer(segment).await;
let offset = byte_writer.current_write_offset();
source

pub async fn seek_to_tail(&mut self)

Seek to the tail of the segment.

This method is useful for tail reads.

§Examples
let mut byte_writer = client_factory.create_byte_writer(segment).await;
byte_writer.seek_to_tail_async().await;
source

pub async fn reset(&mut self) -> Result<(), Error>

Reset the internal Reactor, making it ready for new appends.

Use this method if you want to continue to append after ConditionalCheckFailure error. It will clear all pending events and set the Reactor ready.

§Examples
if let Err(Error::ConditionalCheckFailure(_e)) = byte_writer.flush().await {
    byte_writer.reset().await.expect("reset");
    byte_writer.seek_to_tail().await;
}
byte_writer.write(&payload).await.expect("write");

Trait Implementations§

source§

impl Drop for ByteWriter

source§

fn drop(&mut self)

Executes the destructor for this type. Read more

Auto Trait Implementations§

Blanket Implementations§

source§

impl<T> Any for T
where T: 'static + ?Sized,

source§

fn type_id(&self) -> TypeId

Gets the TypeId of self. Read more
source§

impl<T> Borrow<T> for T
where T: ?Sized,

source§

fn borrow(&self) -> &T

Immutably borrows from an owned value. Read more
source§

impl<T> BorrowMut<T> for T
where T: ?Sized,

source§

fn borrow_mut(&mut self) -> &mut T

Mutably borrows from an owned value. Read more
source§

impl<T> From<T> for T

source§

fn from(t: T) -> T

Returns the argument unchanged.

source§

impl<T> Instrument for T

source§

fn instrument(self, span: Span) -> Instrumented<Self>

Instruments this type with the provided Span, returning an Instrumented wrapper. Read more
source§

fn in_current_span(self) -> Instrumented<Self>

Instruments this type with the current Span, returning an Instrumented wrapper. Read more
source§

impl<T> Instrument for T

source§

fn instrument(self, span: Span) -> Instrumented<Self>

Instruments this type with the provided Span, returning an Instrumented wrapper. Read more
source§

fn in_current_span(self) -> Instrumented<Self>

Instruments this type with the current Span, returning an Instrumented wrapper. Read more
source§

impl<T, U> Into<U> for T
where U: From<T>,

source§

fn into(self) -> U

Calls U::from(self).

That is, this conversion is whatever the implementation of From<T> for U chooses to do.

source§

impl<T> IntoRequest<T> for T

source§

fn into_request(self) -> Request<T>

Wrap the input message T in a tonic::Request
§

impl<T> Pointable for T

§

const ALIGN: usize = _

The alignment of pointer.
§

type Init = T

The type for initializers.
§

unsafe fn init(init: <T as Pointable>::Init) -> usize

Initializes a with the given initializer. Read more
§

unsafe fn deref<'a>(ptr: usize) -> &'a T

Dereferences the given pointer. Read more
§

unsafe fn deref_mut<'a>(ptr: usize) -> &'a mut T

Mutably dereferences the given pointer. Read more
§

unsafe fn drop(ptr: usize)

Drops the object pointed to by the given pointer. Read more
source§

impl<T> Same for T

§

type Output = T

Should always be Self
source§

impl<T, U> TryFrom<U> for T
where U: Into<T>,

§

type Error = Infallible

The type returned in the event of a conversion error.
source§

fn try_from(value: U) -> Result<T, <T as TryFrom<U>>::Error>

Performs the conversion.
source§

impl<T, U> TryInto<U> for T
where U: TryFrom<T>,

§

type Error = <U as TryFrom<T>>::Error

The type returned in the event of a conversion error.
source§

fn try_into(self) -> Result<U, <U as TryFrom<T>>::Error>

Performs the conversion.
§

impl<V, T> VZip<V> for T
where V: MultiLane<T>,

§

fn vzip(self) -> V

source§

impl<T> WithSubscriber for T

source§

fn with_subscriber<S>(self, subscriber: S) -> WithDispatch<Self>
where S: Into<Dispatch>,

Attaches the provided Subscriber to this type, returning a WithDispatch wrapper. Read more
source§

fn with_current_subscriber(self) -> WithDispatch<Self>

Attaches the current default Subscriber to this type, returning a WithDispatch wrapper. Read more
source§

impl<T> WithSubscriber for T

source§

fn with_subscriber<S>(self, subscriber: S) -> WithDispatch<Self>
where S: Into<Dispatch>,

Attaches the provided Subscriber to this type, returning a WithDispatch wrapper. Read more
source§

fn with_current_subscriber(self) -> WithDispatch<Self>

Attaches the current default Subscriber to this type, returning a WithDispatch wrapper. Read more