Skip to content

Commit

Permalink
Merge pull request #1233 from muzarski/ccm-tls-support
Browse files Browse the repository at this point in the history
ccm: Add simple support for `updateconf` and setting up TLS
  • Loading branch information
Lorak-mmk authored Feb 12, 2025
2 parents c839847 + 3bc9362 commit a253772
Show file tree
Hide file tree
Showing 9 changed files with 307 additions and 701 deletions.
2 changes: 2 additions & 0 deletions .github/workflows/rust.yml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,8 @@ jobs:
run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --features "full-serialization"
- name: Cargo check with all features
run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --all-features
- name: Cargo check with ssl feature
run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --features "ssl"
- name: Cargo check with secrecy-08 feature
run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --features "secrecy-08"
- name: Cargo check with chrono-04 feature
Expand Down
44 changes: 0 additions & 44 deletions .github/workflows/tls.yml

This file was deleted.

124 changes: 124 additions & 0 deletions scylla/tests/ccm_integration/ccm/cluster.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ use crate::ccm::{IP_ALLOCATOR, ROOT_CCM_DIR};

use super::ip_allocator::NetPrefix;
use super::logged_cmd::{LoggedCmd, RunOptions};
use super::{DB_TLS_CERT_PATH, DB_TLS_KEY_PATH};
use anyhow::{Context, Error};
use scylla::client::session_builder::SessionBuilder;
use std::collections::HashMap;
Expand Down Expand Up @@ -264,6 +265,68 @@ impl Node {
env
}

/// Executes `ccm updateconf` and applies it for this node.
/// It accepts the key-value pairs to update the configuration.
///
/// ### Example
/// ```
/// # use crate::ccm::cluster::Node;
/// # async fn check_only_compiles(node: &Node) -> Result<(), Box<dyn Error>> {
/// let args = [
/// ("client_encryption_options.enabled", "true"),
/// ("client_encryption_options.certificate", "db.cert"),
/// ("client_encryption_options.keyfile", "db.key"),
/// ];
///
/// node.updateconf(args).await?
/// # Ok(())
/// # }
/// ```
///
/// The code above is equivalent to the following scylla.yaml:
/// ```yaml
/// client_encryption_options:
/// enabled: true
/// certificate: db.cert
/// keyfile: db.key
/// ```
pub(crate) async fn updateconf<K, V>(
&self,
key_values: impl IntoIterator<Item = (K, V)>,
) -> Result<(), Error>
where
K: AsRef<str>,
V: AsRef<str>,
{
let config_dir = &self.config_dir;
let mut args: Vec<String> = vec![
self.opts.name(),
"updateconf".to_string(),
"--config-dir".to_string(),
config_dir.to_string_lossy().into_owned(),
];
for (k, v) in key_values.into_iter() {
args.push(format!("{}:{}", k.as_ref(), v.as_ref()));
}

self.logged_cmd
.run_command("ccm", &args, RunOptions::new())
.await?;
Ok(())
}

/// Configures TLS based on the paths provided in the environment variables `DB_TLS_CERT_PATH` and `DB_TLS_KEY_PATH`.
/// If the paths are not provided, the default certificate and key are taken from `./test/tls/db.crt` and `./test/tls/db.key`.
pub(crate) async fn configure_tls(&self) -> Result<(), Error> {
let args = [
("client_encryption_options.enabled", "true"),
("client_encryption_options.certificate", &DB_TLS_CERT_PATH),
("client_encryption_options.keyfile", &DB_TLS_KEY_PATH),
];

self.updateconf(args).await
}

/// This method starts the node. User can provide optional [`NodeStartOptions`] to control the behavior of the node start.
/// If `None` is provided, the default options are used (see the implementation of Default for [`NodeStartOptions`]).
pub(crate) async fn start(&mut self, opts: Option<NodeStartOptions>) -> Result<(), Error> {
Expand Down Expand Up @@ -581,6 +644,67 @@ impl Cluster {
Ok(())
}

/// Executes `ccm updateconf` and applies it for all nodes in the cluster.
/// It accepts the key-value pairs to update the configuration.
///
/// ### Example
/// ```
/// # use crate::ccm::cluster::Cluster;
/// # async fn check_only_compiles(cluster: &Cluster) -> Result<(), Box<dyn Error>> {
/// let args = [
/// ("client_encryption_options.enabled", "true"),
/// ("client_encryption_options.certificate", "db.cert"),
/// ("client_encryption_options.keyfile", "db.key"),
/// ];
///
/// cluster.updateconf(args).await?
/// # Ok(())
/// # }
/// ```
///
/// The code above is equivalent to the following scylla.yaml:
/// ```yaml
/// client_encryption_options:
/// enabled: true
/// certificate: db.cert
/// keyfile: db.key
/// ```
pub(crate) async fn updateconf<K, V>(
&self,
key_values: impl IntoIterator<Item = (K, V)>,
) -> Result<(), Error>
where
K: AsRef<str>,
V: AsRef<str>,
{
let config_dir = self.config_dir();
let mut args: Vec<String> = vec![
"updateconf".to_string(),
"--config-dir".to_string(),
config_dir.to_string_lossy().into_owned(),
];
for (k, v) in key_values.into_iter() {
args.push(format!("{}:{}", k.as_ref(), v.as_ref()));
}

self.logged_cmd
.run_command("ccm", &args, RunOptions::new())
.await?;
Ok(())
}

/// Configures TLS based on the paths provided in the environment variables `DB_TLS_CERT_PATH` and `DB_TLS_KEY_PATH`.
/// If the paths are not provided, the default certificate and key are taken from `./test/tls/db.crt` and `./test/tls/db.key`.
pub(crate) async fn configure_tls(&self) -> Result<(), Error> {
let args = [
("client_encryption_options.enabled", "true"),
("client_encryption_options.certificate", &DB_TLS_CERT_PATH),
("client_encryption_options.keyfile", &DB_TLS_KEY_PATH),
];

self.updateconf(args).await
}

fn get_ccm_env(&self) -> HashMap<String, String> {
let mut env: HashMap<String, String> = HashMap::new();
env.insert(
Expand Down
109 changes: 109 additions & 0 deletions scylla/tests/ccm_integration/ccm/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -55,17 +55,126 @@ static ROOT_CCM_DIR: LazyLock<String> = LazyLock::new(|| {
ccm_root_dir
});

pub(crate) static DB_TLS_CERT_PATH: LazyLock<String> = LazyLock::new(|| {
let cargo_manifest_dir = env!("CARGO_MANIFEST_DIR");
let db_cert_path_env = std::env::var("DB_TLS_CERT_PATH");
let db_cert_path = match db_cert_path_env {
Ok(x) => x,
Err(e) => {
info!(
"DB_TLS_CERT_PATH env malformed or not present: {}. Using {}/../test/tls/db.crt for db cert.",
e, cargo_manifest_dir
);
cargo_manifest_dir.to_string() + "/../test/tls/db.crt"
}
};

let path = PathBuf::from(&db_cert_path);
if !path.try_exists().unwrap() {
panic!("DB cert file {:?} not found", path);
}

db_cert_path
});

pub(crate) static DB_TLS_KEY_PATH: LazyLock<String> = LazyLock::new(|| {
let cargo_manifest_dir = env!("CARGO_MANIFEST_DIR");
let db_key_path_env = std::env::var("DB_TLS_KEY_PATH");
let db_key_path = match db_key_path_env {
Ok(x) => x,
Err(e) => {
info!(
"DB_TLS_KEY_PATH env malformed or not present: {}. Using {}/../test/tls/db.key for db key.",
e, cargo_manifest_dir
);
cargo_manifest_dir.to_string() + "/../test/tls/db.key"
}
};

let path = PathBuf::from(&db_key_path);
if !path.try_exists().unwrap() {
panic!("DB key file {:?} not found", path);
}

db_key_path
});

#[cfg(feature = "ssl")]
pub(crate) static CA_TLS_CERT_PATH: LazyLock<String> = LazyLock::new(|| {
let cargo_manifest_dir = env!("CARGO_MANIFEST_DIR");
let ca_cert_path_env = std::env::var("CA_TLS_CERT_PATH");
let ca_cert_path = match ca_cert_path_env {
Ok(x) => x,
Err(e) => {
info!(
"CA_TLS_CERT_PATH env malformed or not present: {}. Using {}/../test/tls/ca.crt for ca cert.",
e, cargo_manifest_dir
);
cargo_manifest_dir.to_string() + "/../test/tls/ca.crt"
}
};

let path = PathBuf::from(&ca_cert_path);
if !path.try_exists().unwrap() {
panic!("DB cert file {:?} not found", path);
}

ca_cert_path
});

pub(crate) async fn run_ccm_test<C, T, Fut>(make_cluster_options: C, test_body: T)
where
C: FnOnce() -> ClusterOptions,
T: FnOnce(Arc<tokio::sync::Mutex<Cluster>>) -> Fut,
Fut: Future<Output = ()>,
{
run_ccm_test_with_configuration(
make_cluster_options,
|cluster| async move { cluster },
test_body,
)
.await
}

/// Run a CCM test with a custom configuration logic before the cluster starts.
///
/// ### Example
/// ```
/// # use crate::ccm::cluster::Cluster;
/// # use crate::ccm::run_ccm_test_with_configuration;
/// # use std::sync::{Arc, Mutex};
/// async fn configure_cluster(cluster: Cluster) -> Cluster {
/// // Do some configuration here
/// cluster.updateconf([("foo", "bar")]).await.expect("failed to update conf");
/// cluster
/// }
///
/// async fn test(cluster: Arc<Mutex<Cluster>>) {
/// let cluster = cluster.lock().await;
/// let session = cluster.make_session_builder().await.build().await.unwrap();
///
/// println!("Succesfully connected to the cluster!");
/// }
///
/// run_ccm_test_with_configuration(ClusterOptions::default, configure_cluster, test).await;
/// ```
pub(crate) async fn run_ccm_test_with_configuration<C, Conf, ConfFut, T, TFut>(
make_cluster_options: C,
configure: Conf,
test_body: T,
) where
C: FnOnce() -> ClusterOptions,
Conf: FnOnce(Cluster) -> ConfFut,
ConfFut: Future<Output = Cluster>,
T: FnOnce(Arc<tokio::sync::Mutex<Cluster>>) -> TFut,
TFut: Future<Output = ()>,
{
let cluster_options = make_cluster_options();
let mut cluster = Cluster::new(cluster_options)
.await
.expect("Failed to create cluster");
cluster.init().await.expect("failed to initialize cluster");
cluster = configure(cluster).await;
cluster.start(None).await.expect("failed to start cluster");

struct ClusterWrapper(Arc<tokio::sync::Mutex<Cluster>>);
Expand Down
2 changes: 2 additions & 0 deletions scylla/tests/ccm_integration/main.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,3 +3,5 @@ mod common;

pub(crate) mod ccm;
mod test_example;
#[cfg(feature = "ssl")]
mod tls;
Loading

0 comments on commit a253772

Please sign in to comment.