Skip to content

Commit

Permalink
Add verify_hostname tls flag
Browse files Browse the repository at this point in the history
  • Loading branch information
rukai committed Nov 2, 2022
1 parent 0b6f7c3 commit 07d9574
Show file tree
Hide file tree
Showing 10 changed files with 71 additions and 46 deletions.
4 changes: 4 additions & 0 deletions docs/src/transforms.md
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,8 @@ While `system.peers`/`system.peers_v2` will be rewritten to list the configured
# certificate_path: "tls/localhost.crt"
# # Path to the private key file, typically named with a .key extension.
# private_key_path: "tls/localhost.key"
# # Enable/disable verifying the hostname of the certificate provided by the destination. Enabled by default
# #verify_hostname: true

# Timeout in seconds after which to give up waiting for a response from the destination.
# This field is optional, if not provided, timeout will never occur.
Expand Down Expand Up @@ -156,6 +158,8 @@ No cluster discovery or routing occurs with this transform.
# certificate_path: "tls/localhost.crt"
# # Path to the private key file, typically named with a .key extension.
# private_key_path: "tls/localhost.key"
# # Enable/disable verifying the hostname of the certificate provided by the destination. Enabled by default
# #verify_hostname: true

# Timeout in seconds after which to give up waiting for a response from the destination.
# This field is optional, if not provided, timeout will never occur.
Expand Down
1 change: 1 addition & 0 deletions shotover-proxy/example-configs/cassandra-tls/topology.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -13,5 +13,6 @@ chain_config:
remote_address: "127.0.0.1:9042"
tls:
certificate_authority_path: "example-configs/docker-images/cassandra-tls-4.0.6/certs/localhost_CA.crt"
verify_hostname: false
source_to_chain_mapping:
cassandra_prod: main_chain
51 changes: 28 additions & 23 deletions shotover-proxy/src/tls.rs
Original file line number Diff line number Diff line change
Expand Up @@ -85,15 +85,29 @@ pub struct TlsConnectorConfig {
pub certificate_path: Option<String>,
/// Path to the private key in PEM format
pub private_key_path: Option<String>,
/// whether to enable verifying the hostname of the destination's certificate.
/// Some(true) - enable verify_hostname
/// Some(false) - disable verify_hostname
/// None - protocol specific default value
pub verify_hostname: Option<bool>,
}

#[derive(Clone, Debug)]
pub struct TlsConnector {
connector: Arc<SslConnector>,
verify_hostname: bool,
}

pub enum ApplicationProtocol {
Redis,
Cassandra,
}

impl TlsConnector {
pub fn new(tls_config: TlsConnectorConfig) -> Result<TlsConnector> {
pub fn new(
tls_config: TlsConnectorConfig,
protocol: ApplicationProtocol,
) -> Result<TlsConnector> {
check_file_field(
"certificate_authority_path",
&tls_config.certificate_authority_path,
Expand All @@ -118,38 +132,29 @@ impl TlsConnector {
.map_err(openssl_stack_error_to_anyhow)?;
}

let verify_hostname = match (protocol, tls_config.verify_hostname) {
(ApplicationProtocol::Redis, Some(true)) => {
return Err(anyhow!(
"verify_hostname is enabled in TLS config but redis does not support it."
))
}
(ApplicationProtocol::Redis, _) => false,
(_, None) => true,
(_, Some(true)) => true,
(_, Some(false)) => false,
};
Ok(TlsConnector {
connector: Arc::new(builder.build()),
verify_hostname,
})
}

pub async fn connect_unverified_hostname(
&self,
tcp_stream: TcpStream,
) -> Result<SslStream<TcpStream>> {
let ssl = self
.connector
.configure()
.map_err(openssl_stack_error_to_anyhow)?
.verify_hostname(false)
.into_ssl("localhost")
.map_err(openssl_stack_error_to_anyhow)?;

let mut ssl_stream =
SslStream::new(ssl, tcp_stream).map_err(openssl_stack_error_to_anyhow)?;
Pin::new(&mut ssl_stream).connect().await.map_err(|e| {
openssl_ssl_error_to_anyhow(e)
.context("Failed to establish TLS connection to destination")
})?;

Ok(ssl_stream)
}

pub async fn connect(&self, tcp_stream: TcpStream) -> Result<SslStream<TcpStream>> {
let ssl = self
.connector
.configure()
.map_err(openssl_stack_error_to_anyhow)?
.verify_hostname(self.verify_hostname)
.into_ssl("localhost")
.map_err(openssl_stack_error_to_anyhow)?;

Expand Down
8 changes: 6 additions & 2 deletions shotover-proxy/src/transforms/cassandra/sink_cluster/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ use crate::error::ChainResponse;
use crate::frame::cassandra::{parse_statement_single, CassandraMetadata, Tracing};
use crate::frame::{CassandraFrame, CassandraOperation, CassandraResult, Frame};
use crate::message::{IntSize, Message, MessageValue, Messages};
use crate::tls::{TlsConnector, TlsConnectorConfig};
use crate::tls::{ApplicationProtocol, TlsConnector, TlsConnectorConfig};
use crate::transforms::cassandra::connection::CassandraConnection;
use crate::transforms::util::Response;
use crate::transforms::{Transform, Transforms, Wrapper};
Expand Down Expand Up @@ -56,7 +56,11 @@ pub struct CassandraSinkClusterConfig {

impl CassandraSinkClusterConfig {
pub async fn get_transform(&self, chain_name: String) -> Result<Transforms> {
let tls = self.tls.clone().map(TlsConnector::new).transpose()?;
let tls = self
.tls
.clone()
.map(|c| TlsConnector::new(c, ApplicationProtocol::Cassandra))
.transpose()?;
let mut shotover_nodes = self.shotover_nodes.clone();
let index = self
.shotover_nodes
Expand Down
8 changes: 6 additions & 2 deletions shotover-proxy/src/transforms/cassandra/sink_single.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ use super::connection::CassandraConnection;
use crate::codec::cassandra::CassandraCodec;
use crate::error::ChainResponse;
use crate::message::Messages;
use crate::tls::{TlsConnector, TlsConnectorConfig};
use crate::tls::{ApplicationProtocol, TlsConnector, TlsConnectorConfig};
use crate::transforms::util::Response;
use crate::transforms::{Transform, Transforms, Wrapper};
use anyhow::Result;
Expand All @@ -24,7 +24,11 @@ pub struct CassandraSinkSingleConfig {

impl CassandraSinkSingleConfig {
pub async fn get_transform(&self, chain_name: String) -> Result<Transforms> {
let tls = self.tls.clone().map(TlsConnector::new).transpose()?;
let tls = self
.tls
.clone()
.map(|c| TlsConnector::new(c, ApplicationProtocol::Cassandra))
.transpose()?;
Ok(Transforms::CassandraSinkSingle(CassandraSinkSingle::new(
self.address.clone(),
chain_name,
Expand Down
9 changes: 7 additions & 2 deletions shotover-proxy/src/transforms/redis/sink_single.rs
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ use crate::frame::Frame;
use crate::frame::RedisFrame;
use crate::message::{Message, Messages};
use crate::server::CodecReadError;
use crate::tls::ApplicationProtocol;
use crate::tls::{AsyncStream, TlsConnector, TlsConnectorConfig};
use crate::transforms::{Transform, Transforms, Wrapper};
use anyhow::{anyhow, Context, Result};
Expand All @@ -30,7 +31,11 @@ pub struct RedisSinkSingleConfig {

impl RedisSinkSingleConfig {
pub async fn get_transform(&self, chain_name: String) -> Result<Transforms> {
let tls = self.tls.clone().map(TlsConnector::new).transpose()?;
let tls = self
.tls
.clone()
.map(|c| TlsConnector::new(c, ApplicationProtocol::Redis))
.transpose()?;
Ok(Transforms::RedisSinkSingle(RedisSinkSingle::new(
self.address.clone(),
tls,
Expand Down Expand Up @@ -106,7 +111,7 @@ impl Transform for RedisSinkSingle {
.map_err(|e| anyhow::Error::new(e).context("Failed to connect to upstream"))?;

let generic_stream = if let Some(tls) = self.tls.as_mut() {
let tls_stream = tls.connect_unverified_hostname(tcp_stream).await?;
let tls_stream = tls.connect(tcp_stream).await?;
Box::pin(tls_stream) as Pin<Box<dyn AsyncStream + Send + Sync>>
} else {
Box::pin(tcp_stream) as Pin<Box<dyn AsyncStream + Send + Sync>>
Expand Down
10 changes: 5 additions & 5 deletions shotover-proxy/src/transforms/util/cluster_connection_pool.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ use super::Response;
use crate::server::Codec;
use crate::server::CodecReadHalf;
use crate::server::CodecWriteHalf;
use crate::tls::ApplicationProtocol;
use crate::tls::{TlsConnector, TlsConnectorConfig};
use crate::transforms::util::{ConnectionError, Request};
use anyhow::{anyhow, Result};
Expand Down Expand Up @@ -78,7 +79,9 @@ impl<C: Codec + 'static, A: Authenticator<T>, T: Token> ConnectionPool<C, A, T>
) -> Result<Self> {
Ok(Self {
lanes: Arc::new(Mutex::new(HashMap::new())),
tls: tls.map(TlsConnector::new).transpose()?,
tls: tls
.map(|c| TlsConnector::new(c, ApplicationProtocol::Redis))
.transpose()?,
codec,
authenticator,
})
Expand Down Expand Up @@ -169,10 +172,7 @@ impl<C: Codec + 'static, A: Authenticator<T>, T: Token> ConnectionPool<C, A, T>
.map_err(ConnectionError::IO)?;

let mut connection = if let Some(tls) = &self.tls {
let tls_stream = tls
.connect_unverified_hostname(stream)
.await
.map_err(ConnectionError::TLS)?;
let tls_stream = tls.connect(stream).await.map_err(ConnectionError::TLS)?;
let (rx, tx) = tokio::io::split(tls_stream);
spawn_read_write_tasks(&self.codec, rx, tx)
} else {
Expand Down
16 changes: 10 additions & 6 deletions shotover-proxy/tests/cassandra_int_tests/cluster/mod.rs
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
use cassandra_protocol::frame::Version;
use shotover_proxy::frame::{cassandra::Tracing, CassandraFrame, CassandraOperation, Frame};
use shotover_proxy::message::Message;
use shotover_proxy::tls::{TlsConnector, TlsConnectorConfig};
use shotover_proxy::tls::{ApplicationProtocol, TlsConnector, TlsConnectorConfig};
use shotover_proxy::transforms::cassandra::sink_cluster::{
node::{CassandraNode, ConnectionFactory},
topology::{create_topology_task, TaskConnectionInfo},
Expand All @@ -17,11 +17,15 @@ pub async fn run_topology_task(ca_path: Option<&str>, port: Option<u32>) -> Vec<
let (nodes_tx, mut nodes_rx) = watch::channel(vec![]);
let (task_handshake_tx, task_handshake_rx) = mpsc::channel(1);
let tls = ca_path.map(|ca_path| {
TlsConnector::new(TlsConnectorConfig {
certificate_authority_path: ca_path.into(),
certificate_path: None,
private_key_path: None,
})
TlsConnector::new(
TlsConnectorConfig {
certificate_authority_path: ca_path.into(),
certificate_path: None,
private_key_path: None,
verify_hostname: None,
},
ApplicationProtocol::Cassandra,
)
.unwrap()
});

Expand Down
9 changes: 3 additions & 6 deletions shotover-proxy/tests/helpers/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ use anyhow::Result;
use redis::aio::AsyncStream;
use redis::Client;
use shotover_proxy::runner::{ConfigOpts, Runner};
use shotover_proxy::tls::{TlsConnector, TlsConnectorConfig};
use shotover_proxy::tls::{ApplicationProtocol, TlsConnector, TlsConnectorConfig};
use std::pin::Pin;
use std::sync::mpsc;
use std::time::Duration;
Expand Down Expand Up @@ -120,11 +120,8 @@ impl ShotoverManager {
let tcp_stream = tokio::net::TcpStream::connect((address, port))
.await
.unwrap();
let connector = TlsConnector::new(config).unwrap();
let tls_stream = connector
.connect_unverified_hostname(tcp_stream)
.await
.unwrap();
let connector = TlsConnector::new(config, ApplicationProtocol::Redis).unwrap();
let tls_stream = connector.connect(tcp_stream).await.unwrap();
ShotoverManager::redis_connection_async_inner(
Box::pin(tls_stream) as Pin<Box<dyn AsyncStream + Send + Sync>>
)
Expand Down
1 change: 1 addition & 0 deletions shotover-proxy/tests/redis_int_tests/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ async fn source_tls_and_single_tls() {
certificate_authority_path: "example-configs/redis-tls/certs/ca.crt".into(),
certificate_path: Some("example-configs/redis-tls/certs/redis.crt".into()),
private_key_path: Some("example-configs/redis-tls/certs/redis.key".into()),
verify_hostname: None,
};

let mut connection = shotover_manager
Expand Down

0 comments on commit 07d9574

Please sign in to comment.