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
//
// 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::{PendingEvent, RoutingInfo};
use crate::segment::raw_client::{RawClient, RawClientError, RawClientImpl};
use crate::segment::selector::SegmentSelector;
use crate::util::{get_random_u128, get_request_id};

use pravega_client_auth::DelegationTokenProvider;
use pravega_client_retry::retry_result::RetryError;
use pravega_client_shared::*;
use pravega_controller_client::ControllerError;
use pravega_wire_protocol::client_connection::ClientConnection;
use pravega_wire_protocol::commands::{
    ConditionalBlockEndCommand, CreateTransientSegmentCommand, MergeSegmentsCommand, SetupAppendCommand,
    NULL_ATTRIBUTE_VALUE,
};
use pravega_wire_protocol::error::ClientConnectionError;
use pravega_wire_protocol::wire_commands::{Replies, Requests};

use snafu::{ResultExt, Snafu};
use std::sync::Arc;
use tracing::{debug, info, trace};

pub(crate) struct LargeEventWriter {
    /// Unique id for each SegmentWriter.
    pub(crate) id: WriterId,
    // Delegation token provider used to authenticate client when communicating with segmentstore.
    delegation_token_provider: Arc<DelegationTokenProvider>,
}

impl LargeEventWriter {
    pub(crate) fn new(delegation_token_provider: Arc<DelegationTokenProvider>) -> Self {
        LargeEventWriter {
            id: WriterId::from(get_random_u128()),
            delegation_token_provider,
        }
    }

    pub(crate) async fn write(
        &mut self,
        factory: &ClientFactoryAsync,
        selector: &mut SegmentSelector,
        event: PendingEvent,
    ) -> Result<(), LargeEventWriterError> {
        while let Err(err) = self.write_internal(factory, selector, &event).await {
            if let LargeEventWriterError::SegmentSealed { segment } = err {
                let segment = ScopedSegment::from(&*segment);
                if selector
                    .refresh_segment_event_writers_upon_sealed(&segment)
                    .await
                    .is_some()
                {
                    selector.remove_segment_writer(&segment);
                } else {
                    let sealed_err = LargeEventWriterError::StreamSealed {
                        stream: segment.stream.name,
                    };
                    if event
                        .oneshot_sender
                        .send(Err(Error::InternalFailure {
                            msg: sealed_err.to_string(),
                        }))
                        .is_err()
                    {
                        trace!("failed to send ack back to caller using oneshot due to Receiver dropped");
                    }
                    if let Some(flush_sender) = event.flush_oneshot_sender {
                        if flush_sender.send(Result::Ok(())).is_err() {
                            info!("failed to send ack back to caller using oneshot due to Receiver dropped: event id");
                        }
                    }
                    return Err(sealed_err);
                }
            }
        }
        if event.oneshot_sender.send(Result::Ok(())).is_err() {
            trace!("failed to send ack back to caller using oneshot due to Receiver dropped");
        }
        if let Some(flush_sender) = event.flush_oneshot_sender {
            if flush_sender.send(Result::Ok(())).is_err() {
                info!("failed to send ack back to caller using oneshot due to Receiver dropped: event id");
            }
        }
        Ok(())
    }

    async fn write_internal(
        &mut self,
        factory: &ClientFactoryAsync,
        selector: &mut SegmentSelector,
        event: &PendingEvent,
    ) -> Result<(), LargeEventWriterError> {
        let segment = match &event.routing_info {
            RoutingInfo::RoutingKey(key) => selector.get_segment(key),
            RoutingInfo::Segment(segment) => segment,
        };

        let raw_client = factory.create_raw_client(segment).await;

        let (created_segment, mut connection) = self
            .create_transient_segment(factory, &raw_client, segment)
            .await?;

        self.setup_append(factory, &raw_client, &mut *connection, created_segment.clone())
            .await?;

        let mut expected_offset: i64 = 0;
        let chunks = event.data.chunks(EventWriter::MAX_EVENT_SIZE);
        for (event_number, chunk) in (0_i64..).zip(chunks) {
            let data = chunk.to_vec();
            self.append_data_chunck(&mut *connection, event_number, expected_offset, data)
                .await?;
            expected_offset += EventWriter::MAX_EVENT_SIZE as i64;
        }

        self.merge_segments(factory, &raw_client, &mut *connection, segment, created_segment)
            .await?;
        Ok(())
    }

    async fn create_transient_segment<'a>(
        &mut self,
        factory: &ClientFactoryAsync,
        raw_client: &RawClientImpl<'a>,
        segment: &ScopedSegment,
    ) -> Result<(String, Box<dyn ClientConnection + 'a>), LargeEventWriterError> {
        let request = Requests::CreateTransientSegment(CreateTransientSegmentCommand {
            request_id: get_request_id(),
            writer_id: self.id.0,
            segment: segment.to_string(),
            delegation_token: self
                .delegation_token_provider
                .retrieve_token(factory.controller_client())
                .await,
        });
        debug!(
            "creating transient segment for writer:{:?}/segment:{:?}",
            self.id, segment
        );
        let (reply, connection) = raw_client
            .send_setup_request(&request)
            .await
            .map_err(|e| LargeEventWriterError::RetryRawClient { err: e })?;
        let created_segment = match reply {
            Replies::SegmentCreated(cmd) => {
                debug!(
                    "transient segment {} created for writer:{:?}/segment:{:?}",
                    cmd.segment, self.id, segment
                );
                cmd.segment
            }
            _ => {
                info!("creating transient segment failed due to {:?}", reply);
                return Err(LargeEventWriterError::WrongReply {
                    expected: String::from("SegmentCreated"),
                    actual: reply,
                });
            }
        };
        Ok((created_segment, connection))
    }

    async fn setup_append(
        &mut self,
        factory: &ClientFactoryAsync,
        raw_client: &RawClientImpl<'_>,
        connection: &mut ClientConnection,
        segment: String,
    ) -> Result<(), LargeEventWriterError> {
        let request = Requests::SetupAppend(SetupAppendCommand {
            request_id: get_request_id(),
            writer_id: self.id.0,
            segment: segment.clone(),
            delegation_token: self
                .delegation_token_provider
                .retrieve_token(factory.controller_client())
                .await,
        });
        debug!("setting up append for writer:{:?}/segment:{:?}", self.id, segment);
        let reply = raw_client
            .send_request_with_connection(&request, connection)
            .await
            .map_err(|e| LargeEventWriterError::RetryRawClient { err: e })?;
        match reply {
            Replies::AppendSetup(cmd) => {
                debug!(
                    "append setup completed for writer:{:?}/segment:{:?} with latest event number {}",
                    self.id, segment, cmd.last_event_number
                );
                if cmd.last_event_number != NULL_ATTRIBUTE_VALUE {
                    return Err(LargeEventWriterError::IllegalState { segment });
                }
            }
            _ => {
                info!("append setup failed due to {:?}", reply);
                return Err(LargeEventWriterError::WrongReply {
                    expected: String::from("AppendSetup"),
                    actual: reply,
                });
            }
        };
        Ok(())
    }

    async fn append_data_chunck(
        &mut self,
        connection: &mut ClientConnection,
        event_number: i64,
        expected_offset: i64,
        data: Vec<u8>,
    ) -> Result<(), LargeEventWriterError> {
        let request = Requests::ConditionalBlockEnd(ConditionalBlockEndCommand {
            writer_id: self.id.0,
            event_number,
            expected_offset,
            data,
            request_id: get_request_id(),
        });
        connection.write(&request).await.context(SegmentWriting {})?;

        let reply = connection.read().await.context(SegmentWriting {})?;
        match reply {
            Replies::DataAppended(cmd) => {
                debug!(
                    "data appended for writer {:?}, latest event id is: {:?}",
                    self.id, cmd.event_number
                );
            }
            Replies::SegmentIsSealed(cmd) => {
                debug!(
                    "segment {:?} sealed: stack trace {}",
                    cmd.segment, cmd.server_stack_trace
                );
                return Err(LargeEventWriterError::SegmentSealed { segment: cmd.segment });
            }
            Replies::NoSuchSegment(cmd) => {
                debug!(
                    "no such segment {:?} due to segment truncation: stack trace {}",
                    cmd.segment, cmd.server_stack_trace
                );
                return Err(LargeEventWriterError::SegmentSealed { segment: cmd.segment });
            }
            _ => {
                info!("append data failed due to {:?}", reply);
                return Err(LargeEventWriterError::WrongReply {
                    expected: String::from("DataAppended"),
                    actual: reply,
                });
            }
        };
        Ok(())
    }

    async fn merge_segments(
        &mut self,
        factory: &ClientFactoryAsync,
        raw_client: &RawClientImpl<'_>,
        connection: &mut ClientConnection,
        segment: &ScopedSegment,
        source_segment: String,
    ) -> Result<(), LargeEventWriterError> {
        let request = Requests::MergeSegments(MergeSegmentsCommand {
            request_id: get_request_id(),
            target: segment.to_string(),
            source: source_segment.clone(),
            delegation_token: self
                .delegation_token_provider
                .retrieve_token(factory.controller_client())
                .await,
        });
        debug!(
            "merge segments {} for writer:{:?}/segment:{:?}",
            source_segment, self.id, segment
        );
        let reply = raw_client
            .send_request_with_connection(&request, &mut *connection)
            .await
            .map_err(|e| LargeEventWriterError::RetryRawClient { err: e })?;
        match reply {
            Replies::SegmentsMerged(_) => {
                debug!(
                    "merge segments completed for writer:{:?}/segment:{:?}",
                    self.id, segment
                );
            }
            _ => {
                info!("merge segments failed due to {:?}", reply);
                return Err(LargeEventWriterError::WrongReply {
                    expected: String::from("SegmentsMerged"),
                    actual: reply,
                });
            }
        };
        Ok(())
    }
}

#[derive(Debug, Snafu)]
pub enum LargeEventWriterError {
    #[snafu(display("Failed to send request to segmentstore due to: {:?}", source))]
    SegmentWriting { source: ClientConnectionError },

    #[snafu(display("Retry failed due to error: {:?}", err))]
    RetryControllerWriting { err: RetryError<ControllerError> },

    #[snafu(display("Raw client failed due to error {:?}", err))]
    RetryRawClient { err: RawClientError },

    #[snafu(display("Wrong reply, expected {:?} but get {:?}", expected, actual))]
    WrongReply { expected: String, actual: Replies },

    #[snafu(display("Segment {} is either sealed or truncated", segment))]
    SegmentSealed { segment: String },

    #[snafu(display("Stream {} is sealed", stream))]
    StreamSealed { stream: String },

    #[snafu(display("Server indicates that transient segment was already written to: {}", segment))]
    IllegalState { segment: String },
}