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
//
// 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::segment::event::{Incoming, PendingEvent, RoutingInfo};
use crate::segment::reactor::Reactor;
use crate::util::get_random_u128;

use pravega_client_channel::{create_channel, ChannelSender};
use pravega_client_shared::{ScopedStream, WriterId};

use std::collections::VecDeque;
use tokio::sync::oneshot;
use tokio::sync::oneshot::error::TryRecvError;
use tracing::info_span;
use tracing_futures::Instrument;

/// Write events exactly once to a given stream.
///
/// EventWriter spawns a `Reactor` that runs in the background for processing incoming events.
/// The `write` method sends the event to the `Reactor` asynchronously and returns a `tokio::oneshot::Receiver`
/// which contains the result of the write to the caller. The maximum size of the serialized event
/// supported is 8MB, writing size larger than that will return an error.
///
/// ## Backpressure
/// Write has a backpressure mechanism. Internally, it uses [`Channel`] to send event to
/// Reactor for processing. [`Channel`] 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`].
///
/// [`channel`]: pravega_client_channel
/// [`capacity`]: EventWriter::CHANNEL_CAPACITY
///
/// ## Retry
///
/// The EventWriter 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.
///
/// [`retry`]: pravega_client_retry
///
/// # Examples
///
/// ```no_run
/// use pravega_client_config::ClientConfigBuilder;
/// use pravega_client::client_factory::ClientFactory;
/// use pravega_client_shared::ScopedStream;
///
/// #[tokio::main]
/// async fn main() {
///     // assuming Pravega controller is listening at endpoint `localhost:9090`
///     let config = ClientConfigBuilder::default()
///         .controller_uri("localhost:9090")
///         .build()
///         .expect("creating config");
///
///     let client_factory = ClientFactory::new(config);
///
///     // assuming scope:myscope and stream:mystream has been created before.
///     let stream = ScopedStream::from("myscope/mystream");
///
///     let mut event_writer = client_factory.create_event_writer(stream);
///
///     let payload = "hello world".to_string().into_bytes();
///     let result = event_writer.write_event(payload).await;
///
///     assert!(result.await.is_ok())
/// }
/// ```
pub struct EventWriter {
    writer_id: WriterId,
    sender: ChannelSender<Incoming>,
    event_handles: VecDeque<oneshot::Receiver<Result<(), Error>>>,
}

impl EventWriter {
    pub(crate) const MAX_EVENT_SIZE: usize = 8 * 1024 * 1024;
    // maximum 16 MB total size of events could be held in memory
    const CHANNEL_CAPACITY: usize = 16 * 1024 * 1024;

    pub(crate) fn new(stream: ScopedStream, factory: ClientFactoryAsync) -> Self {
        let (tx, rx) = create_channel(Self::CHANNEL_CAPACITY);
        let writer_id = WriterId::from(get_random_u128());
        let span = info_span!("Reactor", event_stream_writer = %writer_id);
        // spawn is tied to the factory runtime.
        factory
            .runtime_handle()
            .spawn(Reactor::run(stream, tx.clone(), rx, factory, None).instrument(span));
        EventWriter {
            writer_id,
            sender: tx,
            event_handles: VecDeque::new(),
        }
    }

    /// Write an event without routing key.
    ///
    /// A random routing key will be generated in this case.
    ///
    /// This method sends the payload to a background task called reactor to process,
    /// so the success of this method only means the payload has been sent to the reactor.
    /// Applications may want to call await on the returned tokio oneshot to check whether
    /// the payload is successfully persisted or not.
    /// If oneshot returns an error indicating something is going wrong on the server side, then
    /// subsequent calls are also likely to fail.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// let mut event_writer = client_factory.create_event_writer(stream);
    /// // result is a tokio oneshot
    /// let result = event_writer.write_event(payload).await;
    /// result.await.expect("flush to server");
    /// ```
    pub async fn write_event(&mut self, event: Vec<u8>) -> oneshot::Receiver<Result<(), Error>> {
        let size = event.len();
        let (tx, rx) = oneshot::channel();
        let (tx_flush, rx_flush) = oneshot::channel();
        let routing_info = RoutingInfo::RoutingKey(None);
        if event.len() > EventWriter::MAX_EVENT_SIZE {
            if let Err(err) = self.flush().await {
                let (tx_error, rx_error) = oneshot::channel();
                tx_error.send(Err(err)).expect("send error");
                return rx_error;
            }
        };
        if let Some(pending_event) =
            PendingEvent::with_header_flush(routing_info, event, None, tx, Some(tx_flush))
        {
            let append_event = Incoming::AppendEvent(pending_event);
            self.writer_event_internal(append_event, size, rx, rx_flush).await
        } else {
            rx
        }
    }

    /// Writes an event with a routing key.
    ///
    /// Same as the write_event.
    pub async fn write_event_by_routing_key(
        &mut self,
        routing_key: String,
        event: Vec<u8>,
    ) -> oneshot::Receiver<Result<(), Error>> {
        let size = event.len();
        let (tx, rx) = oneshot::channel();
        let (tx_flush, rx_flush) = oneshot::channel();
        let routing_info = RoutingInfo::RoutingKey(Some(routing_key));
        if event.len() > EventWriter::MAX_EVENT_SIZE {
            if let Err(err) = self.flush().await {
                let (tx_error, rx_error) = oneshot::channel();
                tx_error.send(Err(err)).expect("send error");
                return rx_error;
            }
        };
        if let Some(pending_event) =
            PendingEvent::with_header_flush(routing_info, event, None, tx, Some(tx_flush))
        {
            let append_event = Incoming::AppendEvent(pending_event);
            self.writer_event_internal(append_event, size, rx, rx_flush).await
        } else {
            rx
        }
    }

    async fn writer_event_internal(
        &mut self,
        append_event: Incoming,
        size: usize,
        rx: oneshot::Receiver<Result<(), Error>>,
        rx_flush: oneshot::Receiver<Result<(), Error>>,
    ) -> oneshot::Receiver<Result<(), Error>> {
        if let Err(err) = self.clear_initial_complete_events() {
            // fail fast upon checking previous write events
            let (tx_error, rx_error) = oneshot::channel();
            tx_error.send(Err(err)).expect("send error");
            rx_error
        } else if let Err(_e) = self.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");
            rx_error
        } else {
            self.event_handles.push_back(rx_flush);
            rx
        }
    }

    /// Flush data.
    ///
    /// It will wait until all pending appends have acknowledgment.
    pub async fn flush(&mut self) -> Result<(), Error> {
        while let Some(receiver) = self.event_handles.pop_front() {
            let recv = receiver.await.map_err(|e| Error::InternalFailure {
                msg: format!("oneshot error {:?}", e),
            })?;

            recv?;
        }
        Ok(())
    }

    /// Clear initial completed events from flush queue.
    fn clear_initial_complete_events(&mut self) -> Result<(), Error> {
        while let Some(mut receiver) = self.event_handles.pop_front() {
            let try_recv = receiver.try_recv();

            match try_recv {
                Err(TryRecvError::Empty) => {
                    self.event_handles.push_front(receiver);
                    break;
                }
                Err(TryRecvError::Closed) => {
                    let res = try_recv.map_err(|e| Error::InternalFailure {
                        msg: format!("Trying to flush a closed channel {:?}", e),
                    })?;

                    return res;
                }
                Ok(_) => {}
            }
        }

        Ok(())
    }
}

impl Drop for EventWriter {
    fn drop(&mut self) {
        let _res = self.sender.send_without_bp(Incoming::Close());
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::segment::event::{PendingEvent, RoutingInfo};

    #[test]
    fn test_pending_event() {
        // test with legal event size
        let (tx, _rx) = oneshot::channel();
        let data = vec![];
        let routing_info = RoutingInfo::RoutingKey(None);

        let event = PendingEvent::without_header(routing_info, data, None, tx).expect("create pending event");
        assert!(event.is_empty());

        // test with large event size
        let (tx, _rx) = oneshot::channel();
        let data = vec![0; (PendingEvent::MAX_WRITE_SIZE + 1) as usize];
        let routing_info = RoutingInfo::RoutingKey(None);

        let event = PendingEvent::with_header(routing_info, data, None, tx);
        assert!(event.is_some());
    }
}