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_connection::{read_wirecommand, write_wirecommand};
use crate::commands::{HelloCommand, TableKey, TableValue, OLDEST_COMPATIBLE_VERSION, WIRE_VERSION};
use crate::connection::{Connection, TlsConnection, TokioConnection};
use crate::error::*;
use crate::mock_connection::MockConnection;
use crate::wire_commands::{Replies, Requests};
use async_trait::async_trait;
use pravega_client_config::connection_type::MockType;
use pravega_client_config::{connection_type::ConnectionType, ClientConfig};
use pravega_client_shared::{PravegaNodeUri, SegmentInfo};
use pravega_connection_pool::connection_pool::{ConnectionPoolError, Manager};
use snafu::ResultExt;
use std::collections::HashMap;
use std::fmt;
use std::fs::File;
use std::io::BufReader;
use std::sync::Arc;
use tokio::net::TcpStream;
use tokio::sync::Mutex;
use tokio_rustls::{rustls, webpki::DNSNameRef, TlsConnector};
use tracing::info;
use uuid::Uuid;

/// ConnectionFactory trait is the factory used to establish the TCP connection with remote servers.
#[async_trait]
pub trait ConnectionFactory: Send + Sync {
    /// establish_connection will return a Connection future that used to send and read data.
    ///
    /// # Example
    ///
    /// ```no_run
    /// use pravega_wire_protocol::connection_factory::{ConnectionFactory, ConnectionFactoryConfig};
    /// use pravega_client_shared::PravegaNodeUri;
    /// use pravega_client_config::connection_type::ConnectionType;
    /// use tokio::runtime::Runtime;
    ///
    /// fn main() {
    ///   let mut rt = Runtime::new().unwrap();
    ///   let endpoint = PravegaNodeUri::from("localhost:9090".to_string());
    ///   let config = ConnectionFactoryConfig::new(ConnectionType::Tokio);
    ///   let cf = ConnectionFactory::create(config);
    ///   let connection_future = cf.establish_connection(endpoint);
    ///   let mut connection = rt.block_on(connection_future).unwrap();
    /// }
    /// ```
    async fn establish_connection(
        &self,
        endpoint: PravegaNodeUri,
    ) -> Result<Box<dyn Connection>, ConnectionFactoryError>;
}

impl dyn ConnectionFactory {
    pub fn create(config: ConnectionFactoryConfig) -> Box<dyn ConnectionFactory> {
        match config.connection_type {
            ConnectionType::Tokio => Box::new(TokioConnectionFactory::new(
                config.is_tls_enabled,
                &config.cert_path,
            )),
            ConnectionType::Mock(mock_type) => Box::new(MockConnectionFactory::new(mock_type)),
        }
    }
}

struct TokioConnectionFactory {
    tls_enabled: bool,
    path: String,
}

impl TokioConnectionFactory {
    fn new(tls_enabled: bool, path: &str) -> Self {
        TokioConnectionFactory {
            tls_enabled,
            path: path.to_owned(),
        }
    }
}

#[async_trait]
impl ConnectionFactory for TokioConnectionFactory {
    async fn establish_connection(
        &self,
        endpoint: PravegaNodeUri,
    ) -> Result<Box<dyn Connection>, ConnectionFactoryError> {
        let connection_type = ConnectionType::Tokio;
        let uuid = Uuid::new_v4();
        let mut tokio_connection = if self.tls_enabled {
            info!(
                "establish connection to segmentstore {:?} using TLS channel",
                endpoint
            );
            let mut config = rustls::ClientConfig::new();
            let mut pem = BufReader::new(File::open(&self.path).expect("open pem file"));
            config.root_store.add_pem_file(&mut pem).expect("add pem file");
            let connector = TlsConnector::from(Arc::new(config));
            let stream = TcpStream::connect(endpoint.to_socket_addr())
                .await
                .context(Connect {
                    connection_type,
                    endpoint: endpoint.clone(),
                })?;
            // Endpoint returned by controller by default is an IP address, it is necessary to configure
            // Pravega to return a hostname. Check pravegaservice.service.published.host.nameOrIp property.
            let domain_name = endpoint.domain_name();
            let domain = DNSNameRef::try_from_ascii_str(&domain_name).expect("get domain name");
            let stream = connector
                .connect(domain, stream)
                .await
                .expect("connect to tls stream");
            Box::new(TlsConnection {
                uuid,
                endpoint: endpoint.clone(),
                stream: Some(stream),
            }) as Box<dyn Connection>
        } else {
            let stream = TcpStream::connect(endpoint.to_socket_addr())
                .await
                .context(Connect {
                    connection_type,
                    endpoint: endpoint.clone(),
                })?;
            Box::new(TokioConnection {
                uuid,
                endpoint: endpoint.clone(),
                stream: Some(stream),
            }) as Box<dyn Connection>
        };
        verify_connection(&mut *tokio_connection)
            .await
            .context(Verify {})?;
        Ok(tokio_connection)
    }
}

type TableSegmentIndex = HashMap<String, HashMap<TableKey, TableValue>>;
type TableSegment = HashMap<String, Vec<(TableKey, TableValue)>>;

struct MockConnectionFactory {
    segments: Arc<Mutex<HashMap<String, SegmentInfo>>>,
    writers: Arc<Mutex<HashMap<u128, String>>>,
    table_segment_index: Arc<Mutex<TableSegmentIndex>>,
    table_segment: Arc<Mutex<TableSegment>>,
    mock_type: MockType,
}

impl MockConnectionFactory {
    pub fn new(mock_type: MockType) -> Self {
        MockConnectionFactory {
            segments: Arc::new(Mutex::new(HashMap::new())),
            writers: Arc::new(Mutex::new(HashMap::new())),
            table_segment_index: Arc::new(Mutex::new(HashMap::new())),
            table_segment: Arc::new(Mutex::new(HashMap::new())),
            mock_type,
        }
    }
}

#[async_trait]
impl ConnectionFactory for MockConnectionFactory {
    async fn establish_connection(
        &self,
        endpoint: PravegaNodeUri,
    ) -> Result<Box<dyn Connection>, ConnectionFactoryError> {
        let mock = MockConnection::new(
            endpoint,
            self.segments.clone(),
            self.writers.clone(),
            self.table_segment_index.clone(),
            self.table_segment.clone(),
            self.mock_type,
        );
        Ok(Box::new(mock) as Box<dyn Connection>)
    }
}

async fn verify_connection(conn: &mut dyn Connection) -> Result<(), ClientConnectionError> {
    let request = Requests::Hello(HelloCommand {
        high_version: WIRE_VERSION,
        low_version: OLDEST_COMPATIBLE_VERSION,
    });
    write_wirecommand(conn, &request).await?;
    let reply = read_wirecommand(conn).await?;

    match reply {
        Replies::Hello(cmd) => {
            if cmd.low_version <= WIRE_VERSION && cmd.high_version >= WIRE_VERSION {
                Ok(())
            } else {
                Err(ClientConnectionError::WrongHelloVersion {
                    wire_version: WIRE_VERSION,
                    oldest_compatible: OLDEST_COMPATIBLE_VERSION,
                    wire_version_received: cmd.high_version,
                    oldest_compatible_received: cmd.low_version,
                })
            }
        }
        _ => Err(ClientConnectionError::WrongReply { reply }),
    }
}

/// An implementation of the Manager trait to integrate with ConnectionPool.
/// This is for creating connections between Rust client and Segmentstore server.
pub struct SegmentConnectionManager {
    /// connection_factory is used to establish connection to the remote server
    /// when there is no connection available in the internal pool.
    connection_factory: Box<dyn ConnectionFactory>,

    /// The client configuration.
    max_connections_in_pool: u32,
}

impl SegmentConnectionManager {
    pub fn new(connection_factory: Box<dyn ConnectionFactory>, max_connections_in_pool: u32) -> Self {
        SegmentConnectionManager {
            connection_factory,
            max_connections_in_pool,
        }
    }
}

#[async_trait]
impl Manager for SegmentConnectionManager {
    type Conn = Box<dyn Connection>;

    async fn establish_connection(
        &self,
        endpoint: PravegaNodeUri,
    ) -> Result<Self::Conn, ConnectionPoolError> {
        let result = self
            .connection_factory
            .establish_connection(endpoint.clone())
            .await;

        match result {
            Ok(conn) => Ok(conn),
            Err(e) => Err(ConnectionPoolError::EstablishConnection {
                endpoint: endpoint.to_string(),
                error_msg: format!("Could not establish connection due to {:?}", e),
            }),
        }
    }

    fn is_valid(&self, conn: &Self::Conn) -> bool {
        conn.is_valid()
    }

    fn get_max_connections(&self) -> u32 {
        self.max_connections_in_pool
    }

    fn name(&self) -> String {
        "SegmentConnectionManager".to_owned()
    }
}

impl fmt::Debug for SegmentConnectionManager {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.debug_struct("SegmentConnectionManager")
            .field("max connections in pool", &self.max_connections_in_pool)
            .finish()
    }
}

/// The configuration for ConnectionFactory.
#[derive(new)]
pub struct ConnectionFactoryConfig {
    connection_type: ConnectionType,
    #[new(value = "false")]
    is_tls_enabled: bool,
    #[new(default)]
    cert_path: String,
}

/// ConnectionFactoryConfig can be built from ClientConfig.
impl From<&ClientConfig> for ConnectionFactoryConfig {
    fn from(client_config: &ClientConfig) -> Self {
        ConnectionFactoryConfig {
            connection_type: client_config.connection_type,
            is_tls_enabled: client_config.is_tls_enabled,
            cert_path: client_config.trustcert.clone(),
        }
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::wire_commands::{Decode, Encode};
    use log::info;
    use pravega_client_config::connection_type::{ConnectionType, MockType};
    use tokio::runtime::Runtime;

    #[test]
    fn test_mock_connection() {
        info!("test mock connection factory");
        let rt = Runtime::new().unwrap();
        let config = ConnectionFactoryConfig::new(ConnectionType::Mock(MockType::Happy));
        let connection_factory = ConnectionFactory::create(config);
        let connection_future =
            connection_factory.establish_connection(PravegaNodeUri::from("127.1.1.1:9090"));
        let mut mock_connection = rt.block_on(connection_future).unwrap();

        let request = Requests::Hello(HelloCommand {
            high_version: 9,
            low_version: 5,
        })
        .write_fields()
        .unwrap();
        let len = request.len();
        rt.block_on(mock_connection.send_async(&request))
            .expect("write to mock connection");
        let mut buf = vec![0; len];
        rt.block_on(mock_connection.read_async(&mut buf))
            .expect("read from mock connection");
        let reply = Replies::read_from(&buf).unwrap();
        let expected = Replies::Hello(HelloCommand {
            high_version: 9,
            low_version: 5,
        });
        assert_eq!(reply, expected);
        info!("mock connection factory test passed");
    }

    #[test]
    #[should_panic]
    fn test_tokio_connection() {
        info!("test tokio connection factory");
        let rt = Runtime::new().unwrap();
        let config = ConnectionFactoryConfig::new(ConnectionType::Tokio);
        let connection_factory = ConnectionFactory::create(config);
        let connection_future =
            connection_factory.establish_connection(PravegaNodeUri::from("127.1.1.1:9090".to_string()));
        let mut _connection = rt.block_on(connection_future).expect("create tokio connection");

        info!("tokio connection factory test passed");
    }
}