1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346
//
// Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved.
//
// Licensed 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
//
use crate::client_factory::ClientFactoryAsync;
use crate::error::Error;
use crate::event::writer::EventWriter;
use crate::segment::event::{Incoming, PendingEvent, RoutingInfo};
use crate::segment::metadata::SegmentMetadataClient;
use crate::segment::reactor::Reactor;
use crate::util::get_random_u128;
use pravega_client_channel::{create_channel, ChannelSender};
use pravega_client_shared::{ScopedSegment, ScopedStream, WriterId};
use std::collections::VecDeque;
use tokio::sync::oneshot;
use tracing::info_span;
use tracing_futures::Instrument;
type EventHandle = oneshot::Receiver<Result<(), Error>>;
/// 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.
///
/// [`channel`]: pravega_client_channel
/// [`capacity`]: ByteWriter::CHANNEL_CAPACITY
/// [`EventWriter`]: crate::event::writer::EventWriter
/// [`ByteReader`]: crate::byte::reader::ByteReader
/// [`flush`]: ByteWriter::flush
/// [`retry`]: pravega_client_retry
///
/// # Examples
/// ```no_run
/// 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");
/// }
/// ```
pub struct ByteWriter {
writer_id: WriterId,
scoped_segment: ScopedSegment,
sender: ChannelSender<Incoming>,
metadata_client: SegmentMetadataClient,
factory: ClientFactoryAsync,
event_handles: VecDeque<EventHandle>,
write_offset: i64,
}
impl ByteWriter {
// maximum 16 MB total size of events could be held in memory
const CHANNEL_CAPACITY: usize = 16 * 1024 * 1024;
pub(crate) async fn new(stream: ScopedStream, factory: ClientFactoryAsync) -> Self {
let (sender, receiver) = create_channel(Self::CHANNEL_CAPACITY);
let writer_id = WriterId(get_random_u128());
let segments = factory
.controller_client()
.get_head_segments(&stream)
.await
.expect("get head segments");
assert_eq!(
segments.len(),
1,
"Byte stream is configured with more than one segment"
);
let segment = segments.iter().next().unwrap().0.clone();
let scoped_segment = ScopedSegment {
scope: stream.scope.clone(),
stream: stream.stream.clone(),
segment,
};
let metadata_client = factory
.create_segment_metadata_client(scoped_segment.clone())
.await;
let span = info_span!("Reactor", byte_stream_writer = %writer_id);
// spawn is tied to the factory runtime.
let _h = factory.runtime_handle().enter();
tokio::spawn(Reactor::run(stream, sender.clone(), receiver, factory.clone(), None).instrument(span));
ByteWriter {
writer_id,
scoped_segment,
sender,
metadata_client,
factory,
event_handles: VecDeque::new(),
write_offset: 0,
}
}
/// 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
/// ```ignore
/// let mut byte_writer = client_factory.create_byte_writer(segment).await;
/// let payload = vec![0; 8];
/// let size = byte_writer.write(&payload).await;
/// ```
pub async fn write(&mut self, buf: &[u8]) -> Result<usize, Error> {
let bytes_to_write = std::cmp::min(buf.len(), EventWriter::MAX_EVENT_SIZE);
let payload = buf[0..bytes_to_write].to_vec();
let oneshot_receiver = self.write_internal(self.sender.clone(), payload).await;
self.write_offset += bytes_to_write as i64;
self.event_handles.push_back(oneshot_receiver);
while let Some(handle) = self.event_handles.front_mut() {
if let Ok(res) = handle.try_recv() {
res?;
self.event_handles.pop_front().expect("remove successful handle");
} else {
break;
}
}
Ok(bytes_to_write)
}
/// Flush data.
///
/// It will wait until all pending appends have acknowledgment.
///
/// # Examples
/// ```ignore
/// 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;
/// ```
pub async fn flush(&mut self) -> Result<(), Error> {
while self.event_handles.front().is_some() {
let handle = self.event_handles.pop_front().expect("get first handle");
let event_result = handle.await.map_err(|e| Error::InternalFailure {
msg: format!("oneshot error {:?}", e),
})?;
event_result?;
}
Ok(())
}
/// Seal the segment and no further writes are allowed.
///
/// # Examples
/// ```ignore
/// let mut byte_writer = client_factory.create_byte_writer(segment).await;
/// byte_writer.seal().await.expect("seal segment");
/// ```
pub async fn seal(&mut self) -> Result<(), Error> {
self.flush().await?;
self.metadata_client
.seal_segment()
.await
.map_err(|e| Error::InternalFailure {
msg: format!("segment seal error: {:?}", e),
})
}
/// Truncate data before a given offset for the segment. No reads are allowed before
/// truncation point after calling this method.
///
/// # Examples
/// ```ignore
/// let byte_writer = client_factory.create_byte_writer(segment).await;
/// byte_writer.truncate_data_before(1024).await.expect("truncate segment");
/// ```
pub async fn truncate_data_before(&self, offset: i64) -> Result<(), Error> {
self.metadata_client
.truncate_segment(offset)
.await
.map_err(|e| Error::InternalFailure {
msg: format!("segment truncation error: {:?}", e),
})
}
/// Track the current write position for this writer.
///
/// # Examples
/// ```ignore
/// let byte_writer = client_factory.create_byte_writer(segment).await;
/// let offset = byte_writer.current_write_offset();
/// ```
pub fn current_offset(&self) -> u64 {
self.write_offset as u64
}
/// Seek to the tail of the segment.
///
/// This method is useful for tail reads.
/// # Examples
/// ```ignore
/// let mut byte_writer = client_factory.create_byte_writer(segment).await;
/// byte_writer.seek_to_tail_async().await;
/// ```
pub async fn seek_to_tail(&mut self) {
let segment_info = self
.metadata_client
.get_segment_info()
.await
.expect("failed to get segment info");
self.write_offset = segment_info.write_offset;
}
/// 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
/// ```ignore
/// 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");
/// ```
pub async fn reset(&mut self) -> Result<(), Error> {
self.event_handles.clear();
// send reset signal to reactor
self.sender
.send((Incoming::Reset(self.scoped_segment.clone()), 0))
.await
.map_err(|e| Error::InternalFailure {
msg: format!("failed to send request to reactor: {:?}", e),
})?;
Ok(())
}
async fn write_internal(
&self,
sender: ChannelSender<Incoming>,
event: Vec<u8>,
) -> oneshot::Receiver<Result<(), Error>> {
let size = event.len();
let (tx, rx) = oneshot::channel();
let routing_info = RoutingInfo::Segment(self.scoped_segment.clone());
if let Some(pending_event) =
PendingEvent::without_header(routing_info, event, Some(self.write_offset), tx)
{
let append_event = Incoming::AppendEvent(pending_event);
if let Err(_e) = sender.send((append_event, size)).await {
let (tx_error, rx_error) = oneshot::channel();
tx_error
.send(Err(Error::InternalFailure {
msg: "failed to send request to reactor".to_string(),
}))
.expect("send error");
return rx_error;
}
}
rx
}
}
impl Drop for ByteWriter {
fn drop(&mut self) {
let _res = self.sender.send_without_bp(Incoming::Close());
}
}
#[cfg(test)]
mod test {
use super::*;
use crate::client_factory::ClientFactory;
use crate::util::create_stream;
use pravega_client_config::connection_type::{ConnectionType, MockType};
use pravega_client_config::ClientConfigBuilder;
use pravega_client_shared::PravegaNodeUri;
#[test]
#[should_panic(expected = "Byte stream is configured with more than one segment")]
fn test_invalid_stream_config() {
let config = ClientConfigBuilder::default()
.connection_type(ConnectionType::Mock(MockType::Happy))
.mock(true)
.controller_uri(PravegaNodeUri::from("127.0.0.2:9091".to_string()))
.build()
.unwrap();
let factory = ClientFactory::new(config);
factory.runtime().block_on(create_stream(
&factory,
"testScopeInvalid",
"testStreamInvalid",
2,
));
let stream = ScopedStream::from("testScopeInvalid/testStreamInvalid");
factory.runtime().block_on(factory.create_byte_writer(stream));
}
}