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
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
//
// 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 pravega_client_shared::PravegaNodeUri;
use pravega_connection_pool::connection_pool::{ConnectionPool, ConnectionPoolError};
use pravega_wire_protocol::client_connection::{ClientConnection, ClientConnectionImpl};
use pravega_wire_protocol::commands::{Reply, Request};
use pravega_wire_protocol::connection_factory::SegmentConnectionManager;
use pravega_wire_protocol::error::ClientConnectionError;
use pravega_wire_protocol::wire_commands::{Replies, Requests};

use async_trait::async_trait;
use snafu::ResultExt;
use snafu::Snafu;
use std::fmt;
use std::fmt::Debug;
use tokio::time::error::Elapsed;
use tokio::time::{timeout, Duration};

#[derive(Debug, Snafu)]
pub enum RawClientError {
    #[snafu(display("Auth token has expired, refresh to try again: {}", reply))]
    AuthTokenExpired { reply: Replies },

    #[snafu(display("Failed to get connection from connection pool: {}", source))]
    GetConnectionFromPool { source: ConnectionPoolError },

    #[snafu(display("Failed to write request: {}", source))]
    WriteRequest { source: ClientConnectionError },

    #[snafu(display("Failed to read reply: {}", source))]
    ReadReply { source: ClientConnectionError },

    #[snafu(display("Reply incompatible wirecommand version: low {}, high {}", low, high))]
    IncompatibleVersion { low: i32, high: i32 },

    #[snafu(display("Request has timed out: {:?}", source))]
    RequestTimeout { source: Elapsed },

    #[snafu(display("Wrong reply id {:?} for request {:?}", reply_id, request_id))]
    WrongReplyId { reply_id: i64, request_id: i64 },
}

impl RawClientError {
    pub fn is_token_expired(&self) -> bool {
        matches!(self, RawClientError::AuthTokenExpired { .. })
    }
}

// RawClient is on top of the ClientConnection. It provides methods that take
// Request enums and return Reply enums asynchronously. It has logic to process some of the replies from
// server and return the processed result to caller.
#[async_trait]
pub(crate) trait RawClient<'a>: Send + Sync {
    // Asynchronously send a request to the server and receive a response.
    async fn send_request_with_connection(
        &self,
        request: &Requests,
        client_connection: &mut ClientConnection,
    ) -> Result<Replies, RawClientError>
    where
        'a: 'async_trait;

    // Asynchronously send a request to the server and receive a response.
    async fn send_request(&self, request: &Requests) -> Result<Replies, RawClientError>
    where
        'a: 'async_trait;

    // Asynchronously send a request to the server and receive a response and return the connection to the caller.
    async fn send_setup_request(
        &self,
        request: &Requests,
    ) -> Result<(Replies, Box<dyn ClientConnection + 'a>), RawClientError>
    where
        'a: 'async_trait;
}

pub(crate) struct RawClientImpl<'a> {
    pool: &'a ConnectionPool<SegmentConnectionManager>,
    endpoint: PravegaNodeUri,
    timeout: Duration,
}

impl<'a> fmt::Debug for RawClientImpl<'a> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        write!(f, "RawClient endpoint: {:?}", self.endpoint)
    }
}

impl<'a> RawClientImpl<'a> {
    pub(crate) fn new(
        pool: &'a ConnectionPool<SegmentConnectionManager>,
        endpoint: PravegaNodeUri,
        timeout: Duration,
    ) -> RawClientImpl<'a> {
        RawClientImpl {
            pool,
            endpoint,
            timeout,
        }
    }
}

#[allow(clippy::needless_lifetimes)]
#[async_trait]
impl<'a> RawClient<'a> for RawClientImpl<'a> {
    async fn send_request_with_connection(
        &self,
        request: &Requests,
        client_connection: &mut ClientConnection,
    ) -> Result<Replies, RawClientError> {
        client_connection.write(request).await.context(WriteRequest {})?;
        let read_future = client_connection.read();
        let result = timeout(self.timeout, read_future)
            .await
            .context(RequestTimeout {})?;
        let reply = result.context(ReadReply {})?;
        if reply.get_request_id() != request.get_request_id() {
            client_connection.set_failure();
            return Err(RawClientError::WrongReplyId {
                reply_id: reply.get_request_id(),
                request_id: request.get_request_id(),
            });
        }
        check_auth_token_expired(&reply)?;
        Ok(reply)
    }

    async fn send_request(&self, request: &Requests) -> Result<Replies, RawClientError> {
        let connection = self
            .pool
            .get_connection(self.endpoint.clone())
            .await
            .context(GetConnectionFromPool {})?;
        let mut client_connection = ClientConnectionImpl::new(connection);
        client_connection.write(request).await.context(WriteRequest {})?;
        let read_future = client_connection.read();
        let result = timeout(self.timeout, read_future)
            .await
            .context(RequestTimeout {})?;
        let reply = result.context(ReadReply {})?;
        if reply.get_request_id() != request.get_request_id() {
            client_connection.set_failure();
            return Err(RawClientError::WrongReplyId {
                reply_id: reply.get_request_id(),
                request_id: request.get_request_id(),
            });
        }
        check_auth_token_expired(&reply)?;
        Ok(reply)
    }

    async fn send_setup_request(
        &self,
        request: &Requests,
    ) -> Result<(Replies, Box<dyn ClientConnection + 'a>), RawClientError> {
        let connection = self
            .pool
            .get_connection(self.endpoint.clone())
            .await
            .context(GetConnectionFromPool {})?;
        let mut client_connection = ClientConnectionImpl::new(connection);
        client_connection.write(request).await.context(WriteRequest {})?;
        let read_future = client_connection.read();
        let result = timeout(self.timeout, read_future)
            .await
            .context(RequestTimeout {})?;
        let reply = result.context(ReadReply {})?;
        if reply.get_request_id() != request.get_request_id() {
            client_connection.set_failure();
            return Err(RawClientError::WrongReplyId {
                reply_id: reply.get_request_id(),
                request_id: request.get_request_id(),
            });
        }
        check_auth_token_expired(&reply)?;
        Ok((reply, Box::new(client_connection) as Box<dyn ClientConnection>))
    }
}

fn check_auth_token_expired(reply: &Replies) -> Result<(), RawClientError> {
    if let Replies::AuthTokenCheckFailed(ref cmd) = reply {
        if cmd.is_token_expired() {
            return Err(RawClientError::AuthTokenExpired { reply: reply.clone() });
        }
    }
    Ok(())
}

#[cfg(test)]
mod tests {
    use super::*;
    use pravega_client_config::connection_type::ConnectionType;
    use pravega_wire_protocol::commands::{HelloCommand, ReadSegmentCommand, SegmentReadCommand};
    use pravega_wire_protocol::connection_factory::{ConnectionFactory, ConnectionFactoryConfig};
    use pravega_wire_protocol::wire_commands::Encode;
    use std::io::{Read, Write};
    use std::net::{SocketAddr, TcpListener};
    use std::thread;
    use tokio::runtime::Runtime;

    struct Common {
        rt: Runtime,
        pool: ConnectionPool<SegmentConnectionManager>,
    }

    impl Common {
        fn new() -> Self {
            let rt = Runtime::new().expect("create tokio Runtime");
            let config = ConnectionFactoryConfig::new(ConnectionType::Tokio);
            let connection_factory = ConnectionFactory::create(config);
            let manager = SegmentConnectionManager::new(connection_factory, 1);
            let pool = ConnectionPool::new(manager);
            Common { rt, pool }
        }
    }

    struct Server {
        address: SocketAddr,
        listener: TcpListener,
    }

    impl Server {
        pub fn new() -> Server {
            let listener = TcpListener::bind("127.0.0.1:0").expect("local server");
            let address = listener.local_addr().unwrap();
            Server { address, listener }
        }

        pub fn send_hello(&mut self) {
            let reply = Replies::Hello(HelloCommand {
                high_version: 9,
                low_version: 5,
            })
            .write_fields()
            .expect("serialize hello wirecommand");

            for stream in self.listener.incoming() {
                let mut stream = stream.expect("get tcp stream");
                stream.write_all(&reply).expect("reply with hello wirecommand");
                break;
            }
        }

        pub fn send_hello_wrong_version(&mut self) {
            let reply = Replies::Hello(HelloCommand {
                high_version: 10,
                low_version: 10,
            })
            .write_fields()
            .expect("serialize hello wirecommand");

            for stream in self.listener.incoming() {
                let mut stream = stream.expect("get tcp stream");
                stream.write_all(&reply).expect("reply with hello wirecommand");
                break;
            }
        }
    }

    #[test]
    #[should_panic] // since connection verify will panic
    fn test_hello() {
        let common = Common::new();
        let mut server = Server::new();

        let raw_client = RawClientImpl::new(
            &common.pool,
            PravegaNodeUri::from(server.address),
            Duration::from_secs(3600),
        );
        let h = thread::spawn(move || {
            server.send_hello();
        });
        let request = Requests::Hello(HelloCommand {
            low_version: 5,
            high_version: 9,
        });

        let reply = common
            .rt
            .block_on(raw_client.send_request(&request))
            .expect("get reply");

        assert_eq!(
            reply,
            Replies::Hello(HelloCommand {
                high_version: 9,
                low_version: 5,
            })
        );
        h.join().expect("thread finished");
    }

    #[test]
    fn test_invalid_connection() {
        let common = Common::new();
        let server = Server::new();

        let raw_client = RawClientImpl::new(
            &common.pool,
            PravegaNodeUri::from(server.address),
            Duration::from_secs(30),
        );

        let h = thread::spawn(move || {
            let mut conn = 0;
            for stream in server.listener.incoming() {
                conn += 1;
                if let Ok(mut stream) = stream {
                    if conn == 1 {
                        let mut cnt = 0;
                        let mut buf = vec![0; 8];
                        while let Ok(_size) = stream.read_exact(&mut buf) {
                            cnt += 1;
                            if cnt == 1 {
                                let reply = Replies::Hello(HelloCommand {
                                    high_version: 15,
                                    low_version: 5,
                                });
                                let bytes = reply.write_fields().expect("serialize reply");
                                stream.write_all(&bytes).expect("send hello");
                            } else if cnt == 2 {
                                let buf =
                                    vec![0, 0, 0, 0, 255, 255, 255, 255, 0, 0, 0, 0, 255, 255, 255, 255];
                                stream.write_all(&buf).expect("send invalid payload");
                                break;
                            }
                        }
                    } else {
                        let mut cnt = 0;
                        let mut buf = vec![0; 8];
                        while let Ok(_size) = stream.read_exact(&mut buf) {
                            cnt += 1;
                            if cnt == 1 {
                                let reply = Replies::Hello(HelloCommand {
                                    high_version: 15,
                                    low_version: 5,
                                });
                                let bytes = reply.write_fields().expect("serialize reply");
                                stream.write_all(&bytes).expect("send valid payload");
                            } else if cnt == 2 {
                                let reply = Replies::SegmentRead(SegmentReadCommand {
                                    segment: "foo".to_string(),
                                    offset: 0,
                                    at_tail: false,
                                    end_of_segment: false,
                                    data: vec![0, 0, 0, 0],
                                    request_id: 0,
                                });
                                let bytes = reply.write_fields().expect("serialize reply");
                                stream.write_all(&bytes).expect("send valid payload");
                                break;
                            }
                        }
                    }
                }
                if conn == 2 {
                    break;
                }
            }
        });
        let request = Requests::ReadSegment(ReadSegmentCommand {
            segment: "foo".to_string(),
            offset: 0,
            suggested_length: 0,
            delegation_token: "".to_string(),
            request_id: 0,
        });

        let res = common.rt.block_on(raw_client.send_request(&request));
        // payload length too long
        assert!(res.is_err());

        // retry
        let reply = common
            .rt
            .block_on(raw_client.send_request(&request))
            .expect("get reply");

        assert_eq!(
            reply,
            Replies::SegmentRead(SegmentReadCommand {
                segment: "foo".to_string(),
                offset: 0,
                at_tail: false,
                end_of_segment: false,
                data: vec![0, 0, 0, 0],
                request_id: 0
            })
        );
        h.join().expect("thread finished");
    }
}