From edafe273b9743d5fd477d3037c84d911b395102e Mon Sep 17 00:00:00 2001 From: JianMinTang Date: Mon, 18 Nov 2024 16:21:51 +0800 Subject: [PATCH 1/9] refactor: split config to multiple modules Signed-off-by: JianMinTang chore: obey clippy rules Signed-off-by: JianMinTang chore: fix needless borrow Signed-off-by: JianMinTang refactor: fix wrong import Signed-off-by: JianMinTang chore: fix fmt format Signed-off-by: JianMinTang chore: change ServerTimeout => XlineServerTimeout Signed-off-by: JianMinTang --- crates/benchmark/src/runner.rs | 2 +- crates/curp-test-utils/src/test_cmd.rs | 2 +- crates/curp/src/client/mod.rs | 2 +- crates/curp/src/server/curp_node.rs | 2 +- crates/curp/src/server/mod.rs | 2 +- crates/curp/src/server/raw_curp/log.rs | 2 +- crates/curp/src/server/raw_curp/mod.rs | 2 +- crates/curp/src/server/raw_curp/tests.rs | 2 +- crates/curp/src/server/storage/db.rs | 2 +- crates/curp/tests/it/common/curp_group.rs | 2 +- crates/curp/tests/it/server.rs | 2 +- crates/simulation/src/curp_group.rs | 2 +- crates/simulation/src/xline_group.rs | 6 +- crates/utils/src/config.rs | 1532 ------------------ crates/utils/src/config/auth.rs | 27 + crates/utils/src/config/client.rs | 105 ++ crates/utils/src/config/cluster.rs | 136 ++ crates/utils/src/config/compact.rs | 82 + crates/utils/src/config/curp.rs | 262 +++ crates/utils/src/config/engine.rs | 25 + crates/utils/src/config/log.rs | 151 ++ crates/utils/src/config/metrics.rs | 151 ++ crates/utils/src/config/mod.rs | 400 +++++ crates/utils/src/config/prelude.rs | 31 + crates/utils/src/config/server.rs | 126 ++ crates/utils/src/config/storage.rs | 46 + crates/utils/src/config/tls.rs | 59 + crates/utils/src/config/trace.rs | 56 + crates/utils/src/parser.rs | 18 +- crates/xline-client/src/lib.rs | 2 +- crates/xline-test-utils/src/lib.rs | 4 +- crates/xline/src/server/maintenance.rs | 2 +- crates/xline/src/server/watch_server.rs | 2 +- crates/xline/src/server/xline_server.rs | 10 +- crates/xline/src/storage/auth_store/store.rs | 2 +- crates/xline/src/storage/compact/mod.rs | 2 +- crates/xline/src/storage/db.rs | 2 +- crates/xline/src/storage/kv_store.rs | 2 +- crates/xline/src/storage/kvwatcher.rs | 2 +- crates/xline/src/storage/lease_store/mod.rs | 2 +- crates/xline/src/utils/args.rs | 12 +- crates/xline/src/utils/metrics.rs | 6 +- crates/xline/src/utils/trace.rs | 2 +- crates/xline/tests/it/auth_test.rs | 2 +- crates/xline/tests/it/tls_test.rs | 2 +- crates/xlinectl/src/main.rs | 2 +- 46 files changed, 1710 insertions(+), 1585 deletions(-) delete mode 100644 crates/utils/src/config.rs create mode 100644 crates/utils/src/config/auth.rs create mode 100644 crates/utils/src/config/client.rs create mode 100644 crates/utils/src/config/cluster.rs create mode 100644 crates/utils/src/config/compact.rs create mode 100644 crates/utils/src/config/curp.rs create mode 100644 crates/utils/src/config/engine.rs create mode 100644 crates/utils/src/config/log.rs create mode 100644 crates/utils/src/config/metrics.rs create mode 100644 crates/utils/src/config/mod.rs create mode 100644 crates/utils/src/config/prelude.rs create mode 100644 crates/utils/src/config/server.rs create mode 100644 crates/utils/src/config/storage.rs create mode 100644 crates/utils/src/config/tls.rs create mode 100644 crates/utils/src/config/trace.rs diff --git a/crates/benchmark/src/runner.rs b/crates/benchmark/src/runner.rs index fb167716f..d60997aa6 100644 --- a/crates/benchmark/src/runner.rs +++ b/crates/benchmark/src/runner.rs @@ -20,7 +20,7 @@ use tokio::{ time::{Duration, Instant}, }; use tracing::debug; -use utils::config::ClientConfig; +use utils::config::prelude::ClientConfig; use xline_client::ClientOptions; use crate::{args::Commands, bench_client::BenchClient, Benchmark}; diff --git a/crates/curp-test-utils/src/test_cmd.rs b/crates/curp-test-utils/src/test_cmd.rs index c3fa23895..48c7807ff 100644 --- a/crates/curp-test-utils/src/test_cmd.rs +++ b/crates/curp-test-utils/src/test_cmd.rs @@ -20,7 +20,7 @@ use serde::{Deserialize, Serialize}; use thiserror::Error; use tokio::sync::mpsc; use tracing::debug; -use utils::config::EngineConfig; +use utils::config::prelude::EngineConfig; use crate::{META_TABLE, REVISION_TABLE, TEST_TABLE}; diff --git a/crates/curp/src/client/mod.rs b/crates/curp/src/client/mod.rs index 214de7ec1..4c93decaf 100644 --- a/crates/curp/src/client/mod.rs +++ b/crates/curp/src/client/mod.rs @@ -35,7 +35,7 @@ use tonic::transport::ClientTlsConfig; use tracing::{debug, warn}; #[cfg(madsim)] use utils::ClientTlsConfig; -use utils::{build_endpoint, config::ClientConfig}; +use utils::{build_endpoint, config::prelude::ClientConfig}; use self::{ retry::{Retry, RetryConfig}, diff --git a/crates/curp/src/server/curp_node.rs b/crates/curp/src/server/curp_node.rs index 6e89cff8f..f2ce80c16 100644 --- a/crates/curp/src/server/curp_node.rs +++ b/crates/curp/src/server/curp_node.rs @@ -23,7 +23,7 @@ use tracing::{debug, error, info, trace, warn}; use utils::ClientTlsConfig; use utils::{ barrier::IdBarrier, - config::CurpConfig, + config::prelude::CurpConfig, task_manager::{tasks::TaskName, Listener, State, TaskManager}, }; diff --git a/crates/curp/src/server/mod.rs b/crates/curp/src/server/mod.rs index a1a734fc1..2a10c8925 100644 --- a/crates/curp/src/server/mod.rs +++ b/crates/curp/src/server/mod.rs @@ -8,7 +8,7 @@ use tonic::transport::ClientTlsConfig; use tracing::instrument; #[cfg(madsim)] use utils::ClientTlsConfig; -use utils::{config::CurpConfig, task_manager::TaskManager, tracing::Extract}; +use utils::{config::prelude::CurpConfig, task_manager::TaskManager, tracing::Extract}; use self::curp_node::CurpNode; pub use self::{ diff --git a/crates/curp/src/server/raw_curp/log.rs b/crates/curp/src/server/raw_curp/log.rs index 472b1d90e..9336264e9 100644 --- a/crates/curp/src/server/raw_curp/log.rs +++ b/crates/curp/src/server/raw_curp/log.rs @@ -596,7 +596,7 @@ mod tests { use std::{iter::repeat, ops::Index, sync::Arc}; use curp_test_utils::test_cmd::TestCommand; - use utils::config::{default_batch_max_size, default_log_entries_cap}; + use utils::config::prelude::{default_batch_max_size, default_log_entries_cap}; use super::*; diff --git a/crates/curp/src/server/raw_curp/mod.rs b/crates/curp/src/server/raw_curp/mod.rs index b6f529c12..399c0c8aa 100644 --- a/crates/curp/src/server/raw_curp/mod.rs +++ b/crates/curp/src/server/raw_curp/mod.rs @@ -39,7 +39,7 @@ use tracing::{ use utils::ClientTlsConfig; use utils::{ barrier::IdBarrier, - config::CurpConfig, + config::prelude::CurpConfig, parking_lot_lock::{MutexMap, RwLockMap}, task_manager::TaskManager, }; diff --git a/crates/curp/src/server/raw_curp/tests.rs b/crates/curp/src/server/raw_curp/tests.rs index d2eda551a..f13c150d7 100644 --- a/crates/curp/src/server/raw_curp/tests.rs +++ b/crates/curp/src/server/raw_curp/tests.rs @@ -2,7 +2,7 @@ use curp_test_utils::{mock_role_change, test_cmd::TestCommand, TestRoleChange, T use test_macros::abort_on_panic; use tokio::time::{sleep, Instant}; use tracing_test::traced_test; -use utils::config::{ +use utils::config::prelude::{ default_candidate_timeout_ticks, default_follower_timeout_ticks, default_heartbeat_interval, CurpConfigBuilder, }; diff --git a/crates/curp/src/server/storage/db.rs b/crates/curp/src/server/storage/db.rs index 6d8963508..4a9a203b3 100644 --- a/crates/curp/src/server/storage/db.rs +++ b/crates/curp/src/server/storage/db.rs @@ -3,7 +3,7 @@ use std::ops::Deref; use engine::{Engine, EngineType, StorageEngine, StorageOps, WriteOperation}; use parking_lot::Mutex; use prost::Message; -use utils::config::EngineConfig; +use utils::config::prelude::EngineConfig; use super::{ wal::{codec::DataFrame, config::WALConfig, WALStorage, WALStorageOps}, diff --git a/crates/curp/tests/it/common/curp_group.rs b/crates/curp/tests/it/common/curp_group.rs index 8fe32ae18..e7ee921ec 100644 --- a/crates/curp/tests/it/common/curp_group.rs +++ b/crates/curp/tests/it/common/curp_group.rs @@ -39,7 +39,7 @@ use tonic::transport::{Certificate, Channel, ClientTlsConfig, Endpoint, ServerTl use tracing::debug; use utils::{ build_endpoint, - config::{ + config::prelude::{ default_quota, ClientConfig, CurpConfig, CurpConfigBuilder, EngineConfig, StorageConfig, }, task_manager::{tasks::TaskName, Listener, TaskManager}, diff --git a/crates/curp/tests/it/server.rs b/crates/curp/tests/it/server.rs index 512b603ad..4d00cc1ce 100644 --- a/crates/curp/tests/it/server.rs +++ b/crates/curp/tests/it/server.rs @@ -17,7 +17,7 @@ use madsim::rand::{thread_rng, Rng}; use test_macros::abort_on_panic; use tokio::net::TcpListener; use tokio_stream::StreamExt; -use utils::{config::ClientConfig, timestamp}; +use utils::{config::prelude::ClientConfig, timestamp}; use crate::common::curp_group::{CurpGroup, FetchClusterRequest, DEFAULT_SHUTDOWN_TIMEOUT}; diff --git a/crates/simulation/src/curp_group.rs b/crates/simulation/src/curp_group.rs index e9d3aebe0..1d87524f3 100644 --- a/crates/simulation/src/curp_group.rs +++ b/crates/simulation/src/curp_group.rs @@ -36,7 +36,7 @@ use parking_lot::Mutex; use tokio::sync::mpsc; use tracing::debug; use utils::{ - config::{ClientConfig, CurpConfigBuilder, EngineConfig}, + config::prelude::{ClientConfig, CurpConfigBuilder, EngineConfig}, task_manager::TaskManager, }; diff --git a/crates/simulation/src/xline_group.rs b/crates/simulation/src/xline_group.rs index d3a0c41ae..8585845d7 100644 --- a/crates/simulation/src/xline_group.rs +++ b/crates/simulation/src/xline_group.rs @@ -4,9 +4,9 @@ use itertools::Itertools; use madsim::runtime::NodeHandle; use tonic::transport::Channel; use tracing::debug; -use utils::config::{ +use utils::config::prelude::{ AuthConfig, ClientConfig, ClusterConfig, CompactConfig, CurpConfig, InitialClusterState, - ServerTimeout, StorageConfig, TlsConfig, + StorageConfig, TlsConfig, XlineServerTimeout, }; use xline::server::XlineServer; use xline_client::{ @@ -57,7 +57,7 @@ impl XlineGroup { i == 0, CurpConfig::default(), ClientConfig::default(), - ServerTimeout::default(), + XlineServerTimeout::default(), InitialClusterState::New, ); diff --git a/crates/utils/src/config.rs b/crates/utils/src/config.rs deleted file mode 100644 index 0f59dc853..000000000 --- a/crates/utils/src/config.rs +++ /dev/null @@ -1,1532 +0,0 @@ -use std::{collections::HashMap, path::PathBuf, time::Duration}; - -use derive_builder::Builder; -use getset::Getters; -use serde::Deserialize; -use tracing_appender::rolling::RollingFileAppender; - -/// Xline server configuration object -#[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Default)] -pub struct XlineServerConfig { - /// cluster configuration object - #[getset(get = "pub")] - cluster: ClusterConfig, - /// xline storage configuration object - #[getset(get = "pub")] - storage: StorageConfig, - /// log configuration object - #[getset(get = "pub")] - log: LogConfig, - /// trace configuration object - #[getset(get = "pub")] - trace: TraceConfig, - /// auth configuration object - #[getset(get = "pub")] - auth: AuthConfig, - /// compactor configuration object - #[getset(get = "pub")] - compact: CompactConfig, - /// tls configuration object - #[getset(get = "pub")] - tls: TlsConfig, - /// Metrics config - #[getset(get = "pub")] - #[serde(default = "MetricsConfig::default")] - metrics: MetricsConfig, -} - -/// Cluster Range type alias -pub type ClusterRange = std::ops::Range; -/// Log verbosity level alias -#[allow(clippy::module_name_repetitions)] -pub type LevelConfig = tracing::metadata::LevelFilter; - -/// `Duration` deserialization formatter -pub mod duration_format { - use std::time::Duration; - - use serde::{Deserialize, Deserializer}; - - use crate::parse_duration; - - /// deserializes a cluster duration - #[allow(single_use_lifetimes)] // the false positive case blocks us - pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - parse_duration(&s).map_err(serde::de::Error::custom) - } -} - -/// batch size deserialization formatter -pub mod bytes_format { - use serde::{Deserialize, Deserializer}; - - use crate::parse_batch_bytes; - - /// deserializes a cluster duration - #[allow(single_use_lifetimes)] // the false positive case blocks us - pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - parse_batch_bytes(&s).map_err(serde::de::Error::custom) - } -} - -/// Cluster configuration object, including cluster relevant configuration fields -#[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] -pub struct ClusterConfig { - /// Get xline server name - #[getset(get = "pub")] - name: String, - /// Xline server peer listen urls - #[getset(get = "pub")] - peer_listen_urls: Vec, - /// Xline server peer advertise urls - #[getset(get = "pub")] - peer_advertise_urls: Vec, - /// Xline server client listen urls - #[getset(get = "pub")] - client_listen_urls: Vec, - /// Xline server client advertise urls - #[getset(get = "pub")] - client_advertise_urls: Vec, - /// All the nodes in the xline cluster - #[getset(get = "pub")] - peers: HashMap>, - /// Leader node. - #[getset(get = "pub")] - is_leader: bool, - /// Curp server timeout settings - #[getset(get = "pub")] - #[serde(default = "CurpConfig::default")] - curp_config: CurpConfig, - /// Curp client config settings - #[getset(get = "pub")] - #[serde(default = "ClientConfig::default")] - client_config: ClientConfig, - /// Xline server timeout settings - #[getset(get = "pub")] - #[serde(default = "ServerTimeout::default")] - server_timeout: ServerTimeout, - /// Xline server initial state - #[getset(get = "pub")] - #[serde(with = "state_format", default = "InitialClusterState::default")] - initial_cluster_state: InitialClusterState, -} - -impl Default for ClusterConfig { - #[inline] - fn default() -> Self { - Self { - name: "default".to_owned(), - peer_listen_urls: vec!["http://127.0.0.1:2380".to_owned()], - peer_advertise_urls: vec!["http://127.0.0.1:2380".to_owned()], - client_listen_urls: vec!["http://127.0.0.1:2379".to_owned()], - client_advertise_urls: vec!["http://127.0.0.1:2379".to_owned()], - peers: HashMap::from([( - "default".to_owned(), - vec!["http://127.0.0.1:2379".to_owned()], - )]), - is_leader: false, - curp_config: CurpConfig::default(), - client_config: ClientConfig::default(), - server_timeout: ServerTimeout::default(), - initial_cluster_state: InitialClusterState::default(), - } - } -} - -/// Initial cluster state of xline server -#[derive(Debug, Clone, Copy, Default, PartialEq, Eq, Deserialize)] -#[non_exhaustive] -pub enum InitialClusterState { - /// Create a new cluster - #[default] - New, - /// Join an existing cluster - Existing, -} - -/// `InitialClusterState` deserialization formatter -pub mod state_format { - use serde::{Deserialize, Deserializer}; - - use super::InitialClusterState; - use crate::parse_state; - - /// deserializes a cluster log rotation strategy - #[allow(single_use_lifetimes)] - pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - parse_state(&s).map_err(serde::de::Error::custom) - } -} - -impl ClusterConfig { - /// Generate a new `ClusterConfig` object - #[must_use] - #[inline] - #[allow(clippy::too_many_arguments)] - pub fn new( - name: String, - peer_listen_urls: Vec, - peer_advertise_urls: Vec, - client_listen_urls: Vec, - client_advertise_urls: Vec, - peers: HashMap>, - is_leader: bool, - curp: CurpConfig, - client_config: ClientConfig, - server_timeout: ServerTimeout, - initial_cluster_state: InitialClusterState, - ) -> Self { - Self { - name, - peer_listen_urls, - peer_advertise_urls, - client_listen_urls, - client_advertise_urls, - peers, - is_leader, - curp_config: curp, - client_config, - server_timeout, - initial_cluster_state, - } - } -} - -/// Compaction configuration -#[derive(Clone, Copy, Debug, Deserialize, PartialEq, Eq, Getters)] -#[allow(clippy::module_name_repetitions)] -pub struct CompactConfig { - /// The max number of historical versions processed in a single compact operation - #[getset(get = "pub")] - #[serde(default = "default_compact_batch_size")] - compact_batch_size: usize, - /// The interval between two compaction batches - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_compact_sleep_interval")] - compact_sleep_interval: Duration, - /// The auto compactor config - #[getset(get = "pub")] - auto_compact_config: Option, -} - -impl Default for CompactConfig { - #[inline] - fn default() -> Self { - Self { - compact_batch_size: default_compact_batch_size(), - compact_sleep_interval: default_compact_sleep_interval(), - auto_compact_config: None, - } - } -} - -impl CompactConfig { - /// Create a new compact config - #[must_use] - #[inline] - pub fn new( - compact_batch_size: usize, - compact_sleep_interval: Duration, - auto_compact_config: Option, - ) -> Self { - Self { - compact_batch_size, - compact_sleep_interval, - auto_compact_config, - } - } -} - -/// default compact batch size -#[must_use] -#[inline] -pub const fn default_compact_batch_size() -> usize { - 1000 -} - -/// default compact interval -#[must_use] -#[inline] -pub const fn default_compact_sleep_interval() -> Duration { - Duration::from_millis(10) -} - -/// Curp server timeout settings -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Builder)] -#[allow(clippy::module_name_repetitions, clippy::exhaustive_structs)] -pub struct CurpConfig { - /// Heartbeat Interval - #[builder(default = "default_heartbeat_interval()")] - #[serde(with = "duration_format", default = "default_heartbeat_interval")] - pub heartbeat_interval: Duration, - - /// Curp wait sync timeout - #[builder(default = "default_server_wait_synced_timeout()")] - #[serde( - with = "duration_format", - default = "default_server_wait_synced_timeout" - )] - pub wait_synced_timeout: Duration, - - /// Curp propose retry count - #[builder(default = "default_retry_count()")] - #[serde(default = "default_retry_count")] - pub retry_count: usize, - - /// Curp rpc timeout - #[builder(default = "default_rpc_timeout()")] - #[serde(with = "duration_format", default = "default_rpc_timeout")] - pub rpc_timeout: Duration, - - /// Curp append entries batch timeout - /// - /// If the `batch_timeout` has expired, then it will be dispatched - /// whether its size reaches the `BATCHING_MSG_MAX_SIZE` or not. - #[builder(default = "default_batch_timeout()")] - #[serde(with = "duration_format", default = "default_batch_timeout")] - pub batch_timeout: Duration, - - /// The maximum number of bytes per batch. - #[builder(default = "default_batch_max_size()")] - #[serde(with = "bytes_format", default = "default_batch_max_size")] - pub batch_max_size: u64, - - /// How many ticks a follower is allowed to miss before it starts a new round of election - /// - /// The actual timeout will be randomized and in between heartbeat_interval * [follower_timeout_ticks, 2 * follower_timeout_ticks) - #[builder(default = "default_follower_timeout_ticks()")] - #[serde(default = "default_follower_timeout_ticks")] - pub follower_timeout_ticks: u8, - - /// How many ticks a candidate needs to wait before it starts a new round of election - /// - /// It should be smaller than `follower_timeout_ticks` - /// - /// The actual timeout will be randomized and in between heartbeat_interval * [candidate_timeout_ticks, 2 * candidate_timeout_ticks) - #[builder(default = "default_candidate_timeout_ticks()")] - #[serde(default = "default_candidate_timeout_ticks")] - pub candidate_timeout_ticks: u8, - - /// Curp storage path - #[builder(default = "EngineConfig::default()")] - #[serde(default = "EngineConfig::default")] - pub engine_cfg: EngineConfig, - - /// Number of command execute workers - #[builder(default = "default_cmd_workers()")] - #[serde(default = "default_cmd_workers")] - pub cmd_workers: u8, - - /// How often should the gc task run - #[builder(default = "default_gc_interval()")] - #[serde(with = "duration_format", default = "default_gc_interval")] - pub gc_interval: Duration, - - /// Number of log entries to keep in memory - #[builder(default = "default_log_entries_cap()")] - #[serde(default = "default_log_entries_cap")] - pub log_entries_cap: usize, -} - -/// default heartbeat interval -#[must_use] -#[inline] -pub const fn default_heartbeat_interval() -> Duration { - Duration::from_millis(300) -} - -/// default batch timeout -#[must_use] -#[inline] -pub const fn default_batch_timeout() -> Duration { - Duration::from_millis(15) -} - -/// default batch timeout -#[must_use] -#[inline] -#[allow(clippy::arithmetic_side_effects)] -pub const fn default_batch_max_size() -> u64 { - 2 * 1024 * 1024 -} - -/// default wait synced timeout -#[must_use] -#[inline] -pub const fn default_server_wait_synced_timeout() -> Duration { - Duration::from_secs(5) -} - -/// default initial retry timeout -/// FIXME: etcd client has it's own retry mechanism, which may lead to nested retry timeouts. -/// Consider bypassing for proxied etcd client requests. -#[must_use] -#[inline] -pub const fn default_initial_retry_timeout() -> Duration { - Duration::from_millis(1500) -} - -/// default max retry timeout -#[must_use] -#[inline] -pub const fn default_max_retry_timeout() -> Duration { - Duration::from_millis(10_000) -} - -/// default retry count -#[cfg(not(madsim))] -#[must_use] -#[inline] -pub const fn default_retry_count() -> usize { - 3 -} -/// default retry count -#[cfg(madsim)] -#[must_use] -#[inline] -pub const fn default_retry_count() -> usize { - 10 -} - -/// default use backoff -#[must_use] -#[inline] -pub const fn default_fixed_backoff() -> bool { - false -} - -/// default rpc timeout -#[must_use] -#[inline] -pub const fn default_rpc_timeout() -> Duration { - Duration::from_millis(150) -} - -/// default candidate timeout ticks -#[must_use] -#[inline] -pub const fn default_candidate_timeout_ticks() -> u8 { - 2 -} - -/// default client wait synced timeout -#[must_use] -#[inline] -pub const fn default_client_wait_synced_timeout() -> Duration { - Duration::from_secs(2) -} - -/// default client propose timeout -#[must_use] -#[inline] -pub const fn default_propose_timeout() -> Duration { - Duration::from_secs(1) -} - -/// default client id keep alive interval -#[must_use] -#[inline] -pub const fn default_client_id_keep_alive_interval() -> Duration { - Duration::from_secs(1) -} - -/// default follower timeout -#[must_use] -#[inline] -pub const fn default_follower_timeout_ticks() -> u8 { - 5 -} - -/// default number of execute workers -#[must_use] -#[inline] -pub const fn default_cmd_workers() -> u8 { - 8 -} - -/// default range retry timeout -#[must_use] -#[inline] -pub const fn default_range_retry_timeout() -> Duration { - Duration::from_secs(2) -} - -/// default compact timeout -#[must_use] -#[inline] -pub const fn default_compact_timeout() -> Duration { - Duration::from_secs(5) -} - -/// default sync victims interval -#[must_use] -#[inline] -pub const fn default_sync_victims_interval() -> Duration { - Duration::from_millis(10) -} - -/// default gc interval -#[must_use] -#[inline] -pub const fn default_gc_interval() -> Duration { - Duration::from_secs(20) -} - -/// default number of log entries to keep in memory -#[must_use] -#[inline] -pub const fn default_log_entries_cap() -> usize { - 5000 -} - -/// default watch progress notify interval -#[must_use] -#[inline] -pub const fn default_watch_progress_notify_interval() -> Duration { - Duration::from_secs(600) -} - -impl Default for CurpConfig { - #[inline] - fn default() -> Self { - Self { - heartbeat_interval: default_heartbeat_interval(), - wait_synced_timeout: default_server_wait_synced_timeout(), - retry_count: default_retry_count(), - rpc_timeout: default_rpc_timeout(), - batch_timeout: default_batch_timeout(), - batch_max_size: default_batch_max_size(), - follower_timeout_ticks: default_follower_timeout_ticks(), - candidate_timeout_ticks: default_candidate_timeout_ticks(), - engine_cfg: EngineConfig::default(), - cmd_workers: default_cmd_workers(), - gc_interval: default_gc_interval(), - log_entries_cap: default_log_entries_cap(), - } - } -} - -/// Curp client settings -#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq, Getters)] -#[allow(clippy::module_name_repetitions)] -pub struct ClientConfig { - /// Curp client wait sync timeout - #[getset(get = "pub")] - #[serde( - with = "duration_format", - default = "default_client_wait_synced_timeout" - )] - wait_synced_timeout: Duration, - - /// Curp client propose request timeout - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_propose_timeout")] - propose_timeout: Duration, - - /// Curp client initial retry interval - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_initial_retry_timeout")] - initial_retry_timeout: Duration, - - /// Curp client max retry interval - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_max_retry_timeout")] - max_retry_timeout: Duration, - - /// Curp client retry interval - #[getset(get = "pub")] - #[serde(default = "default_retry_count")] - retry_count: usize, - - /// Whether to use exponential backoff in retries - #[getset(get = "pub")] - #[serde(default = "default_fixed_backoff")] - fixed_backoff: bool, - - /// Curp client keep client id alive interval - #[getset(get = "pub")] - #[serde( - with = "duration_format", - default = "default_client_id_keep_alive_interval" - )] - keep_alive_interval: Duration, -} - -impl ClientConfig { - /// Create a new client timeout - /// - /// # Panics - /// - /// Panics if `initial_retry_timeout` is larger than `max_retry_timeout` - #[must_use] - #[inline] - pub fn new( - wait_synced_timeout: Duration, - propose_timeout: Duration, - initial_retry_timeout: Duration, - max_retry_timeout: Duration, - retry_count: usize, - fixed_backoff: bool, - keep_alive_interval: Duration, - ) -> Self { - assert!( - initial_retry_timeout <= max_retry_timeout, - "`initial_retry_timeout` should less or equal to `max_retry_timeout`" - ); - Self { - wait_synced_timeout, - propose_timeout, - initial_retry_timeout, - max_retry_timeout, - retry_count, - fixed_backoff, - keep_alive_interval, - } - } -} - -impl Default for ClientConfig { - #[inline] - fn default() -> Self { - Self { - wait_synced_timeout: default_client_wait_synced_timeout(), - propose_timeout: default_propose_timeout(), - initial_retry_timeout: default_initial_retry_timeout(), - max_retry_timeout: default_max_retry_timeout(), - retry_count: default_retry_count(), - fixed_backoff: default_fixed_backoff(), - keep_alive_interval: default_client_id_keep_alive_interval(), - } - } -} - -/// Xline server settings -#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq, Getters)] -pub struct ServerTimeout { - /// Range request retry timeout settings - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_range_retry_timeout")] - range_retry_timeout: Duration, - /// Range request retry timeout settings - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_compact_timeout")] - compact_timeout: Duration, - /// Sync victims interval - #[getset(get = "pub")] - #[serde(with = "duration_format", default = "default_sync_victims_interval")] - sync_victims_interval: Duration, - /// Watch progress notify interval settings - #[getset(get = "pub")] - #[serde( - with = "duration_format", - default = "default_watch_progress_notify_interval" - )] - watch_progress_notify_interval: Duration, -} - -impl ServerTimeout { - /// Create a new server timeout - #[must_use] - #[inline] - pub fn new( - range_retry_timeout: Duration, - compact_timeout: Duration, - sync_victims_interval: Duration, - watch_progress_notify_interval: Duration, - ) -> Self { - Self { - range_retry_timeout, - compact_timeout, - sync_victims_interval, - watch_progress_notify_interval, - } - } -} - -impl Default for ServerTimeout { - #[inline] - fn default() -> Self { - Self { - range_retry_timeout: default_range_retry_timeout(), - compact_timeout: default_compact_timeout(), - sync_victims_interval: default_sync_victims_interval(), - watch_progress_notify_interval: default_watch_progress_notify_interval(), - } - } -} - -/// Auto Compactor Configuration -#[allow(clippy::module_name_repetitions)] -#[non_exhaustive] -#[derive(Debug, Clone, Copy, Deserialize, PartialEq, Eq)] -#[serde( - tag = "mode", - content = "retention", - rename_all(deserialize = "lowercase") -)] -pub enum AutoCompactConfig { - /// auto periodic compactor - #[serde(with = "duration_format")] - Periodic(Duration), - /// auto revision compactor - Revision(i64), -} - -/// Engine Configuration -#[allow(clippy::module_name_repetitions)] -#[non_exhaustive] -#[derive(Debug, Clone, Deserialize, PartialEq, Eq)] -#[serde( - tag = "type", - content = "data_dir", - rename_all(deserialize = "lowercase") -)] -pub enum EngineConfig { - /// Memory Storage Engine - Memory, - /// RocksDB Storage Engine - RocksDB(PathBuf), -} - -impl Default for EngineConfig { - #[inline] - fn default() -> Self { - Self::Memory - } -} - -/// /// Storage Configuration -#[derive(Debug, Clone, Deserialize, PartialEq, Eq)] -#[allow(clippy::module_name_repetitions)] -#[non_exhaustive] -pub struct StorageConfig { - /// Engine Configuration - #[serde(default = "EngineConfig::default")] - pub engine: EngineConfig, - /// Quota - #[serde(default = "default_quota")] - pub quota: u64, -} - -impl StorageConfig { - /// Create a new storage config - #[inline] - #[must_use] - pub fn new(engine: EngineConfig, quota: u64) -> Self { - Self { engine, quota } - } -} - -impl Default for StorageConfig { - #[inline] - fn default() -> Self { - Self { - engine: EngineConfig::default(), - quota: default_quota(), - } - } -} - -/// Default quota: 8GB -#[inline] -#[must_use] -pub fn default_quota() -> u64 { - // 8 * 1024 * 1024 * 1024 - 0x0002_0000_0000 -} - -/// Log configuration object -#[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] -pub struct LogConfig { - /// Log file path - #[getset(get = "pub")] - #[serde(default)] - path: Option, - /// Log rotation strategy - #[getset(get = "pub")] - #[serde(with = "rotation_format", default = "default_rotation")] - rotation: RotationConfig, - /// Log verbosity level - #[getset(get = "pub")] - #[serde(with = "level_format", default = "default_log_level")] - level: LevelConfig, -} - -impl Default for LogConfig { - #[inline] - fn default() -> Self { - Self { - path: None, - rotation: default_rotation(), - level: default_log_level(), - } - } -} - -/// `LevelConfig` deserialization formatter -pub mod level_format { - use serde::{Deserialize, Deserializer}; - - use super::LevelConfig; - use crate::parse_log_level; - - /// deserializes a cluster duration - #[allow(single_use_lifetimes)] - pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - parse_log_level(&s).map_err(serde::de::Error::custom) - } -} - -/// default log level -#[must_use] -#[inline] -pub const fn default_log_level() -> LevelConfig { - LevelConfig::INFO -} - -impl LogConfig { - /// Generate a new `LogConfig` object - #[must_use] - #[inline] - pub fn new(path: Option, rotation: RotationConfig, level: LevelConfig) -> Self { - Self { - path, - rotation, - level, - } - } -} - -/// Xline log rotation strategy -#[non_exhaustive] -#[allow(clippy::module_name_repetitions)] -#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq)] -#[serde(rename_all(deserialize = "lowercase"))] -pub enum RotationConfig { - /// Rotate log file in every hour - Hourly, - /// Rotate log file every day - Daily, - /// Never rotate log file - Never, -} - -impl std::fmt::Display for RotationConfig { - #[inline] - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match *self { - RotationConfig::Hourly => write!(f, "hourly"), - RotationConfig::Daily => write!(f, "daily"), - RotationConfig::Never => write!(f, "never"), - } - } -} - -/// `RotationConfig` deserialization formatter -pub mod rotation_format { - use serde::{Deserialize, Deserializer}; - - use super::RotationConfig; - use crate::parse_rotation; - - /// deserializes a cluster log rotation strategy - #[allow(single_use_lifetimes)] - pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - parse_rotation(&s).map_err(serde::de::Error::custom) - } -} - -/// default log rotation strategy -#[must_use] -#[inline] -pub const fn default_rotation() -> RotationConfig { - RotationConfig::Never -} - -/// Generates a `RollingFileAppender` from the given `RotationConfig` and `name` -#[must_use] -#[inline] -pub fn file_appender( - rotation: RotationConfig, - file_path: &PathBuf, - name: &str, -) -> RollingFileAppender { - match rotation { - RotationConfig::Hourly => { - tracing_appender::rolling::hourly(file_path, format!("xline_{name}.log")) - } - RotationConfig::Daily => { - tracing_appender::rolling::daily(file_path, format!("xline_{name}.log")) - } - RotationConfig::Never => { - tracing_appender::rolling::never(file_path, format!("xline_{name}.log")) - } - #[allow(unreachable_patterns)] - // It's ok because `parse_rotation` have check the validity before. - _ => unreachable!("should not call file_appender when parse_rotation failed"), - } -} - -/// Xline tracing configuration object -#[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] -pub struct TraceConfig { - /// Open jaeger online, sending data to jaeger agent directly - #[getset(get = "pub")] - jaeger_online: bool, - /// Open jaeger offline, saving data to the `jaeger_output_dir` - #[getset(get = "pub")] - jaeger_offline: bool, - /// The dir path to save the data when `jaeger_offline` is on - #[getset(get = "pub")] - jaeger_output_dir: PathBuf, - /// The verbosity level of tracing - #[getset(get = "pub")] - #[serde(with = "level_format", default = "default_log_level")] - jaeger_level: LevelConfig, -} - -impl Default for TraceConfig { - #[inline] - fn default() -> Self { - Self { - jaeger_online: false, - jaeger_offline: false, - jaeger_output_dir: "".into(), - jaeger_level: default_log_level(), - } - } -} - -impl TraceConfig { - /// Generate a new `TraceConfig` object - #[must_use] - #[inline] - pub fn new( - jaeger_online: bool, - jaeger_offline: bool, - jaeger_output_dir: PathBuf, - jaeger_level: LevelConfig, - ) -> Self { - Self { - jaeger_online, - jaeger_offline, - jaeger_output_dir, - jaeger_level, - } - } -} - -/// Xline tracing configuration object -#[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Default)] -pub struct AuthConfig { - /// The public key file - #[getset(get = "pub")] - auth_public_key: Option, - /// The private key file - #[getset(get = "pub")] - auth_private_key: Option, -} - -impl AuthConfig { - /// Generate a new `AuthConfig` object - #[must_use] - #[inline] - pub fn new(auth_public_key: Option, auth_private_key: Option) -> Self { - Self { - auth_public_key, - auth_private_key, - } - } -} - -/// Xline tls configuration object -#[allow(clippy::module_name_repetitions)] -#[non_exhaustive] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Default)] -pub struct TlsConfig { - /// The CA certificate file used by peer to verify client certificates - #[getset(get = "pub")] - pub peer_ca_cert_path: Option, - /// The public key file used by peer - #[getset(get = "pub")] - pub peer_cert_path: Option, - /// The private key file used by peer - #[getset(get = "pub")] - pub peer_key_path: Option, - /// The CA certificate file used by client to verify peer certificates - #[getset(get = "pub")] - pub client_ca_cert_path: Option, - /// The public key file used by client - #[getset(get = "pub")] - pub client_cert_path: Option, - /// The private key file used by client - #[getset(get = "pub")] - pub client_key_path: Option, -} - -impl TlsConfig { - /// Create a new `TlsConfig` object - #[must_use] - #[inline] - pub fn new( - peer_ca_cert_path: Option, - peer_cert_path: Option, - peer_key_path: Option, - client_ca_cert_path: Option, - client_cert_path: Option, - client_key_path: Option, - ) -> Self { - Self { - peer_ca_cert_path, - peer_cert_path, - peer_key_path, - client_ca_cert_path, - client_cert_path, - client_key_path, - } - } - - /// Whether the server tls is enabled - #[must_use] - #[inline] - pub fn server_tls_enabled(&self) -> bool { - self.peer_cert_path.is_some() && self.peer_key_path.is_some() - } -} - -/// Xline metrics push protocol -#[non_exhaustive] -#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq, Default)] -#[serde(rename_all(deserialize = "lowercase"))] -pub enum MetricsPushProtocol { - /// HTTP protocol - HTTP, - /// GRPC protocol - #[default] - GRPC, -} - -impl std::fmt::Display for MetricsPushProtocol { - #[inline] - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match *self { - MetricsPushProtocol::HTTP => write!(f, "http"), - MetricsPushProtocol::GRPC => write!(f, "grpc"), - } - } -} - -/// Metrics push protocol format -pub mod protocol_format { - use serde::{Deserialize, Deserializer}; - - use super::MetricsPushProtocol; - use crate::parse_metrics_push_protocol; - - /// deserializes a cluster duration - #[allow(single_use_lifetimes)] - pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let s = String::deserialize(deserializer)?; - parse_metrics_push_protocol(&s).map_err(serde::de::Error::custom) - } -} - -/// Xline metrics configuration object -#[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] -pub struct MetricsConfig { - /// Enable or not - #[getset(get = "pub")] - #[serde(default = "default_metrics_enable")] - enable: bool, - /// The http port to expose - #[getset(get = "pub")] - #[serde(default = "default_metrics_port")] - port: u16, - /// The http path to expose - #[getset(get = "pub")] - #[serde(default = "default_metrics_path")] - path: String, - /// Enable push or not - #[getset(get = "pub")] - #[serde(default = "default_metrics_push")] - push: bool, - /// Push endpoint - #[getset(get = "pub")] - #[serde(default = "default_metrics_push_endpoint")] - push_endpoint: String, - /// Push protocol - #[getset(get = "pub")] - #[serde(with = "protocol_format", default = "default_metrics_push_protocol")] - push_protocol: MetricsPushProtocol, -} - -impl MetricsConfig { - /// Create a new `MetricsConfig` - #[must_use] - #[inline] - pub fn new( - enable: bool, - port: u16, - path: String, - push: bool, - push_endpoint: String, - push_protocol: MetricsPushProtocol, - ) -> Self { - Self { - enable, - port, - path, - push, - push_endpoint, - push_protocol, - } - } -} - -impl Default for MetricsConfig { - #[inline] - fn default() -> Self { - Self { - enable: default_metrics_enable(), - port: default_metrics_port(), - path: default_metrics_path(), - push: default_metrics_push(), - push_endpoint: default_metrics_push_endpoint(), - push_protocol: default_metrics_push_protocol(), - } - } -} - -/// Default metrics enable -#[must_use] -#[inline] -pub const fn default_metrics_enable() -> bool { - true -} - -/// Default metrics port -#[must_use] -#[inline] -pub const fn default_metrics_port() -> u16 { - 9100 -} - -/// Default metrics path -#[must_use] -#[inline] -pub fn default_metrics_path() -> String { - "/metrics".to_owned() -} - -/// Default metrics push option -#[must_use] -#[inline] -pub fn default_metrics_push() -> bool { - false -} - -/// Default metrics push protocol -#[must_use] -#[inline] -pub fn default_metrics_push_protocol() -> MetricsPushProtocol { - MetricsPushProtocol::GRPC -} - -/// Default metrics push endpoint -#[must_use] -#[inline] -pub fn default_metrics_push_endpoint() -> String { - "http://127.0.0.1:4318".to_owned() -} - -impl XlineServerConfig { - /// Generates a new `XlineServerConfig` object - #[must_use] - #[inline] - #[allow(clippy::too_many_arguments)] - pub fn new( - cluster: ClusterConfig, - storage: StorageConfig, - log: LogConfig, - trace: TraceConfig, - auth: AuthConfig, - compact: CompactConfig, - tls: TlsConfig, - metrics: MetricsConfig, - ) -> Self { - Self { - cluster, - storage, - log, - trace, - auth, - compact, - tls, - metrics, - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[allow(clippy::too_many_lines)] // just a testcase, not too bad - #[test] - fn test_xline_server_config_should_be_loaded() { - let config: XlineServerConfig = toml::from_str( - r#"[cluster] - name = 'node1' - is_leader = true - initial_cluster_state = 'new' - peer_listen_urls = ['127.0.0.1:2380'] - peer_advertise_urls = ['127.0.0.1:2380'] - client_listen_urls = ['127.0.0.1:2379'] - client_advertise_urls = ['127.0.0.1:2379'] - - [cluster.server_timeout] - range_retry_timeout = '3s' - compact_timeout = '5s' - sync_victims_interval = '20ms' - watch_progress_notify_interval = '1s' - - [cluster.peers] - node1 = ['127.0.0.1:2378', '127.0.0.1:2379'] - node2 = ['127.0.0.1:2380'] - node3 = ['127.0.0.1:2381'] - - [cluster.curp_config] - heartbeat_interval = '200ms' - wait_synced_timeout = '100ms' - rpc_timeout = '100ms' - retry_timeout = '100ms' - - [cluster.client_config] - initial_retry_timeout = '5s' - max_retry_timeout = '50s' - - [storage] - engine = { type = 'memory'} - - [compact] - compact_batch_size = 123 - compact_sleep_interval = '5ms' - - [compact.auto_compact_config] - mode = 'periodic' - retention = '10h' - - [log] - path = '/var/log/xline' - rotation = 'daily' - level = 'info' - - [trace] - jaeger_online = false - jaeger_offline = false - jaeger_output_dir = './jaeger_jsons' - jaeger_level = 'info' - - [auth] - auth_public_key = './public_key.pem' - auth_private_key = './private_key.pem' - - [tls] - peer_cert_path = './cert.pem' - peer_key_path = './key.pem' - client_ca_cert_path = './ca.pem' - - [metrics] - enable = true - port = 9100 - path = "/metrics" - push = true - push_endpoint = 'http://some-endpoint.com:4396' - push_protocol = 'http' - "#, - ) - .unwrap(); - - let curp_config = CurpConfigBuilder::default() - .heartbeat_interval(Duration::from_millis(200)) - .wait_synced_timeout(Duration::from_millis(100)) - .rpc_timeout(Duration::from_millis(100)) - .build() - .unwrap(); - - let client_config = ClientConfig::new( - default_client_wait_synced_timeout(), - default_propose_timeout(), - Duration::from_secs(5), - Duration::from_secs(50), - default_retry_count(), - default_fixed_backoff(), - default_client_id_keep_alive_interval(), - ); - - let server_timeout = ServerTimeout::new( - Duration::from_secs(3), - Duration::from_secs(5), - Duration::from_millis(20), - Duration::from_secs(1), - ); - - assert_eq!( - config.cluster, - ClusterConfig::new( - "node1".to_owned(), - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - HashMap::from_iter([ - ( - "node1".to_owned(), - vec!["127.0.0.1:2378".to_owned(), "127.0.0.1:2379".to_owned()] - ), - ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), - ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), - ]), - true, - curp_config, - client_config, - server_timeout, - InitialClusterState::New - ) - ); - - assert_eq!( - config.storage, - StorageConfig::new(EngineConfig::Memory, default_quota()) - ); - - assert_eq!( - config.log, - LogConfig::new( - Some(PathBuf::from("/var/log/xline")), - RotationConfig::Daily, - LevelConfig::INFO - ) - ); - assert_eq!( - config.trace, - TraceConfig::new( - false, - false, - PathBuf::from("./jaeger_jsons"), - LevelConfig::INFO - ) - ); - - assert_eq!( - config.compact, - CompactConfig { - compact_batch_size: 123, - compact_sleep_interval: Duration::from_millis(5), - auto_compact_config: Some(AutoCompactConfig::Periodic(Duration::from_secs( - 10 * 60 * 60 - ))) - } - ); - - assert_eq!( - config.auth, - AuthConfig { - auth_private_key: Some(PathBuf::from("./private_key.pem")), - auth_public_key: Some(PathBuf::from("./public_key.pem")), - } - ); - - assert_eq!( - config.tls, - TlsConfig { - peer_cert_path: Some(PathBuf::from("./cert.pem")), - peer_key_path: Some(PathBuf::from("./key.pem")), - client_ca_cert_path: Some(PathBuf::from("./ca.pem")), - ..Default::default() - } - ); - - assert_eq!( - config.metrics, - MetricsConfig { - enable: true, - port: 9100, - path: "/metrics".to_owned(), - push: true, - push_endpoint: "http://some-endpoint.com:4396".to_owned(), - push_protocol: MetricsPushProtocol::HTTP, - }, - ); - } - - #[test] - fn test_xline_server_default_config_should_be_loaded() { - let config: XlineServerConfig = toml::from_str( - "[cluster] - name = 'node1' - is_leader = true - peer_listen_urls = ['127.0.0.1:2380'] - peer_advertise_urls = ['127.0.0.1:2380'] - client_listen_urls = ['127.0.0.1:2379'] - client_advertise_urls = ['127.0.0.1:2379'] - - [cluster.peers] - node1 = ['127.0.0.1:2379'] - node2 = ['127.0.0.1:2380'] - node3 = ['127.0.0.1:2381'] - - [cluster.storage] - - [log] - path = '/var/log/xline' - - [storage] - engine = { type = 'rocksdb', data_dir = '/usr/local/xline/data-dir' } - - [compact] - - [trace] - jaeger_online = false - jaeger_offline = false - jaeger_output_dir = './jaeger_jsons' - jaeger_level = 'info' - - [auth] - - [tls] - ", - ) - .unwrap(); - - assert_eq!( - config.cluster, - ClusterConfig::new( - "node1".to_owned(), - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - HashMap::from([ - ("node1".to_owned(), vec!["127.0.0.1:2379".to_owned()]), - ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), - ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), - ]), - true, - CurpConfigBuilder::default().build().unwrap(), - ClientConfig::default(), - ServerTimeout::default(), - InitialClusterState::default() - ) - ); - - if let EngineConfig::RocksDB(path) = config.storage.engine { - assert_eq!(path, PathBuf::from("/usr/local/xline/data-dir")); - } else { - unreachable!(); - } - - assert_eq!( - config.log, - LogConfig::new( - Some(PathBuf::from("/var/log/xline")), - RotationConfig::Never, - LevelConfig::INFO - ) - ); - assert_eq!( - config.trace, - TraceConfig::new( - false, - false, - PathBuf::from("./jaeger_jsons"), - LevelConfig::INFO - ) - ); - assert_eq!(config.compact, CompactConfig::default()); - assert_eq!(config.auth, AuthConfig::default()); - assert_eq!(config.tls, TlsConfig::default()); - assert_eq!(config.metrics, MetricsConfig::default()); - } - - #[test] - fn test_auto_revision_compactor_config_should_be_loaded() { - let config: XlineServerConfig = toml::from_str( - "[cluster] - name = 'node1' - is_leader = true - peer_listen_urls = ['127.0.0.1:2380'] - peer_advertise_urls = ['127.0.0.1:2380'] - client_listen_urls = ['127.0.0.1:2379'] - client_advertise_urls = ['127.0.0.1:2379'] - - [cluster.peers] - node1 = ['127.0.0.1:2379'] - node2 = ['127.0.0.1:2380'] - node3 = ['127.0.0.1:2381'] - - [cluster.storage] - - [log] - path = '/var/log/xline' - - [storage] - engine = { type = 'memory' } - - [compact] - - [compact.auto_compact_config] - mode = 'revision' - retention = 10000 - - [trace] - jaeger_online = false - jaeger_offline = false - jaeger_output_dir = './jaeger_jsons' - jaeger_level = 'info' - - [auth] - - [tls] - ", - ) - .unwrap(); - - assert_eq!( - config.compact, - CompactConfig { - auto_compact_config: Some(AutoCompactConfig::Revision(10000)), - ..Default::default() - } - ); - } -} diff --git a/crates/utils/src/config/auth.rs b/crates/utils/src/config/auth.rs new file mode 100644 index 000000000..1c3c8e587 --- /dev/null +++ b/crates/utils/src/config/auth.rs @@ -0,0 +1,27 @@ +use getset::Getters; +use serde::Deserialize; +use std::path::PathBuf; + +/// Xline tracing configuration object +#[allow(clippy::module_name_repetitions)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Default)] +pub struct AuthConfig { + /// The public key file + #[getset(get = "pub")] + auth_public_key: Option, + /// The private key file + #[getset(get = "pub")] + auth_private_key: Option, +} + +impl AuthConfig { + /// Generate a new `AuthConfig` object + #[must_use] + #[inline] + pub fn new(auth_public_key: Option, auth_private_key: Option) -> Self { + Self { + auth_public_key, + auth_private_key, + } + } +} diff --git a/crates/utils/src/config/client.rs b/crates/utils/src/config/client.rs new file mode 100644 index 000000000..111966298 --- /dev/null +++ b/crates/utils/src/config/client.rs @@ -0,0 +1,105 @@ +use std::time::Duration; + +use getset::Getters; +use serde::Deserialize; + +use super::duration_format; +use super::prelude::{ + default_client_id_keep_alive_interval, default_client_wait_synced_timeout, + default_fixed_backoff, default_initial_retry_timeout, default_max_retry_timeout, + default_propose_timeout, default_retry_count, +}; + +/// Curp client settings +#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +#[allow(clippy::module_name_repetitions)] +pub struct ClientConfig { + /// Curp client wait sync timeout + #[getset(get = "pub")] + #[serde( + with = "duration_format", + default = "default_client_wait_synced_timeout" + )] + wait_synced_timeout: Duration, + + /// Curp client propose request timeout + #[getset(get = "pub")] + #[serde(with = "duration_format", default = "default_propose_timeout")] + propose_timeout: Duration, + + /// Curp client initial retry interval + #[getset(get = "pub")] + #[serde(with = "duration_format", default = "default_initial_retry_timeout")] + initial_retry_timeout: Duration, + + /// Curp client max retry interval + #[getset(get = "pub")] + #[serde(with = "duration_format", default = "default_max_retry_timeout")] + max_retry_timeout: Duration, + + /// Curp client retry interval + #[getset(get = "pub")] + #[serde(default = "default_retry_count")] + retry_count: usize, + + /// Whether to use exponential backoff in retries + #[getset(get = "pub")] + #[serde(default = "default_fixed_backoff")] + fixed_backoff: bool, + + /// Curp client keep client id alive interval + #[getset(get = "pub")] + #[serde( + with = "duration_format", + default = "default_client_id_keep_alive_interval" + )] + keep_alive_interval: Duration, +} + +impl ClientConfig { + /// Create a new client timeout + /// + /// # Panics + /// + /// Panics if `initial_retry_timeout` is larger than `max_retry_timeout` + #[must_use] + #[inline] + pub fn new( + wait_synced_timeout: Duration, + propose_timeout: Duration, + initial_retry_timeout: Duration, + max_retry_timeout: Duration, + retry_count: usize, + fixed_backoff: bool, + keep_alive_interval: Duration, + ) -> Self { + assert!( + initial_retry_timeout <= max_retry_timeout, + "`initial_retry_timeout` should less or equal to `max_retry_timeout`" + ); + Self { + wait_synced_timeout, + propose_timeout, + initial_retry_timeout, + max_retry_timeout, + retry_count, + fixed_backoff, + keep_alive_interval, + } + } +} + +impl Default for ClientConfig { + #[inline] + fn default() -> Self { + Self { + wait_synced_timeout: default_client_wait_synced_timeout(), + propose_timeout: default_propose_timeout(), + initial_retry_timeout: default_initial_retry_timeout(), + max_retry_timeout: default_max_retry_timeout(), + retry_count: default_retry_count(), + fixed_backoff: default_fixed_backoff(), + keep_alive_interval: default_client_id_keep_alive_interval(), + } + } +} diff --git a/crates/utils/src/config/cluster.rs b/crates/utils/src/config/cluster.rs new file mode 100644 index 000000000..4b1a60c25 --- /dev/null +++ b/crates/utils/src/config/cluster.rs @@ -0,0 +1,136 @@ +use getset::Getters; +use serde::Deserialize; +use std::collections::HashMap; + +use super::prelude::{ClientConfig, CurpConfig, XlineServerTimeout}; + +/// Cluster configuration object, including cluster relevant configuration fields +#[allow(clippy::module_name_repetitions)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +pub struct ClusterConfig { + /// Get xline server name + #[getset(get = "pub")] + name: String, + /// Xline server peer listen urls + #[getset(get = "pub")] + peer_listen_urls: Vec, + /// Xline server peer advertise urls + #[getset(get = "pub")] + peer_advertise_urls: Vec, + /// Xline server client listen urls + #[getset(get = "pub")] + client_listen_urls: Vec, + /// Xline server client advertise urls + #[getset(get = "pub")] + client_advertise_urls: Vec, + /// All the nodes in the xline cluster + #[getset(get = "pub")] + peers: HashMap>, + /// Leader node. + #[getset(get = "pub")] + is_leader: bool, + /// Curp server timeout settings + #[getset(get = "pub")] + #[serde(default = "CurpConfig::default")] + curp_config: CurpConfig, + /// Curp client config settings + #[getset(get = "pub")] + #[serde(default = "ClientConfig::default")] + client_config: ClientConfig, + /// Xline server timeout settings + #[getset(get = "pub")] + #[serde(default = "XlineServerTimeout::default")] + server_timeout: XlineServerTimeout, + /// Xline server initial state + #[getset(get = "pub")] + #[serde(with = "state_format", default = "InitialClusterState::default")] + initial_cluster_state: InitialClusterState, +} + +impl Default for ClusterConfig { + #[inline] + fn default() -> Self { + Self { + name: "default".to_owned(), + peer_listen_urls: vec!["http://127.0.0.1:2380".to_owned()], + peer_advertise_urls: vec!["http://127.0.0.1:2380".to_owned()], + client_listen_urls: vec!["http://127.0.0.1:2379".to_owned()], + client_advertise_urls: vec!["http://127.0.0.1:2379".to_owned()], + peers: HashMap::from([( + "default".to_owned(), + vec!["http://127.0.0.1:2379".to_owned()], + )]), + is_leader: false, + curp_config: CurpConfig::default(), + client_config: ClientConfig::default(), + server_timeout: XlineServerTimeout::default(), + initial_cluster_state: InitialClusterState::default(), + } + } +} + +/// Initial cluster state of xline server +#[derive(Debug, Clone, Copy, Default, PartialEq, Eq, Deserialize)] +#[non_exhaustive] +pub enum InitialClusterState { + /// Create a new cluster + #[default] + New, + /// Join an existing cluster + Existing, +} + +/// `InitialClusterState` deserialization formatter +pub mod state_format { + use serde::{Deserialize, Deserializer}; + + use super::InitialClusterState; + use crate::parse_state; + + /// deserializes a cluster log rotation strategy + #[allow(single_use_lifetimes)] + pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + parse_state(&s).map_err(serde::de::Error::custom) + } +} + +impl ClusterConfig { + /// Generate a new `ClusterConfig` object + #[must_use] + #[inline] + #[allow(clippy::too_many_arguments)] + pub fn new( + name: String, + peer_listen_urls: Vec, + peer_advertise_urls: Vec, + client_listen_urls: Vec, + client_advertise_urls: Vec, + peers: HashMap>, + is_leader: bool, + curp: CurpConfig, + client_config: ClientConfig, + server_timeout: XlineServerTimeout, + initial_cluster_state: InitialClusterState, + ) -> Self { + Self { + name, + peer_listen_urls, + peer_advertise_urls, + client_listen_urls, + client_advertise_urls, + peers, + is_leader, + curp_config: curp, + client_config, + server_timeout, + initial_cluster_state, + } + } +} + +/// Cluster Range type alias +pub type RangeBound = std::ops::Range; diff --git a/crates/utils/src/config/compact.rs b/crates/utils/src/config/compact.rs new file mode 100644 index 000000000..9def42083 --- /dev/null +++ b/crates/utils/src/config/compact.rs @@ -0,0 +1,82 @@ +use getset::Getters; +use serde::Deserialize; +use std::time::Duration; + +use super::duration_format; + +/// Compaction configuration +#[derive(Clone, Copy, Debug, Deserialize, PartialEq, Eq, Getters)] +#[allow(clippy::module_name_repetitions)] +pub struct CompactConfig { + /// The max number of historical versions processed in a single compact operation + #[serde(default = "default_compact_batch_size")] + #[getset(get = "pub")] + compact_batch_size: usize, + /// The interval between two compaction batches + #[serde(with = "duration_format", default = "default_compact_sleep_interval")] + #[getset(get = "pub")] + compact_sleep_interval: Duration, + /// The auto compactor config + #[getset(get = "pub")] + auto_compact_config: Option, +} + +impl Default for CompactConfig { + #[inline] + fn default() -> Self { + Self { + compact_batch_size: default_compact_batch_size(), + compact_sleep_interval: default_compact_sleep_interval(), + auto_compact_config: None, + } + } +} + +impl CompactConfig { + /// Create a new compact config + #[must_use] + #[inline] + pub fn new( + compact_batch_size: usize, + compact_sleep_interval: Duration, + auto_compact_config: Option, + ) -> Self { + Self { + compact_batch_size, + compact_sleep_interval, + auto_compact_config, + } + } +} + +/// default compact batch size +#[must_use] +#[inline] +pub const fn default_compact_batch_size() -> usize { + 1000 +} + +/// default compact interval +#[must_use] +#[inline] +pub const fn default_compact_sleep_interval() -> Duration { + Duration::from_millis(10) +} + +/// Auto Compactor Configuration +#[allow(clippy::module_name_repetitions)] +#[non_exhaustive] +#[derive(Debug, Clone, Copy, Deserialize, PartialEq, Eq)] +#[serde( + tag = "mode", + content = "retention", + rename_all(deserialize = "lowercase") +)] + +pub enum AutoCompactConfig { + /// auto periodic compactor + #[serde(with = "duration_format")] + Periodic(Duration), + /// auto revision compactor + Revision(i64), +} diff --git a/crates/utils/src/config/curp.rs b/crates/utils/src/config/curp.rs new file mode 100644 index 000000000..16baeca4e --- /dev/null +++ b/crates/utils/src/config/curp.rs @@ -0,0 +1,262 @@ +use derive_builder::Builder; +use getset::Getters; +use serde::Deserialize; +use std::time::Duration; + +use super::prelude::EngineConfig; +use super::{bytes_format, duration_format}; + +/// Curp server timeout settings +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Builder)] +#[allow(clippy::module_name_repetitions, clippy::exhaustive_structs)] +pub struct CurpConfig { + /// Heartbeat Interval + #[builder(default = "default_heartbeat_interval()")] + #[serde(with = "duration_format", default = "default_heartbeat_interval")] + pub heartbeat_interval: Duration, + + /// Curp wait sync timeout + #[builder(default = "default_server_wait_synced_timeout()")] + #[serde( + with = "duration_format", + default = "default_server_wait_synced_timeout" + )] + pub wait_synced_timeout: Duration, + + /// Curp propose retry count + #[builder(default = "default_retry_count()")] + #[serde(default = "default_retry_count")] + pub retry_count: usize, + + /// Curp rpc timeout + #[builder(default = "default_rpc_timeout()")] + #[serde(with = "duration_format", default = "default_rpc_timeout")] + pub rpc_timeout: Duration, + + /// Curp append entries batch timeout + /// + /// If the `batch_timeout` has expired, then it will be dispatched + /// whether its size reaches the `BATCHING_MSG_MAX_SIZE` or not. + #[builder(default = "default_batch_timeout()")] + #[serde(with = "duration_format", default = "default_batch_timeout")] + pub batch_timeout: Duration, + + /// The maximum number of bytes per batch. + #[builder(default = "default_batch_max_size()")] + #[serde(with = "bytes_format", default = "default_batch_max_size")] + pub batch_max_size: u64, + + /// How many ticks a follower is allowed to miss before it starts a new round of election + /// + /// The actual timeout will be randomized and in between heartbeat_interval * [follower_timeout_ticks, 2 * follower_timeout_ticks) + #[builder(default = "default_follower_timeout_ticks()")] + #[serde(default = "default_follower_timeout_ticks")] + pub follower_timeout_ticks: u8, + + /// How many ticks a candidate needs to wait before it starts a new round of election + /// + /// It should be smaller than `follower_timeout_ticks` + /// + /// The actual timeout will be randomized and in between heartbeat_interval * [candidate_timeout_ticks, 2 * candidate_timeout_ticks) + #[builder(default = "default_candidate_timeout_ticks()")] + #[serde(default = "default_candidate_timeout_ticks")] + pub candidate_timeout_ticks: u8, + + /// Curp storage path + #[builder(default = "EngineConfig::default()")] + #[serde(default = "EngineConfig::default")] + pub engine_cfg: EngineConfig, + + /// Number of command execute workers + #[builder(default = "default_cmd_workers()")] + #[serde(default = "default_cmd_workers")] + pub cmd_workers: u8, + + /// How often should the gc task run + #[builder(default = "default_gc_interval()")] + #[serde(with = "duration_format", default = "default_gc_interval")] + pub gc_interval: Duration, + + /// Number of log entries to keep in memory + #[builder(default = "default_log_entries_cap()")] + #[serde(default = "default_log_entries_cap")] + pub log_entries_cap: usize, +} + +/// default heartbeat interval +#[must_use] +#[inline] +pub const fn default_heartbeat_interval() -> Duration { + Duration::from_millis(300) +} + +/// default batch timeout +#[must_use] +#[inline] +pub const fn default_batch_timeout() -> Duration { + Duration::from_millis(15) +} + +/// default batch timeout +#[must_use] +#[inline] +#[allow(clippy::arithmetic_side_effects)] +pub const fn default_batch_max_size() -> u64 { + 2 * 1024 * 1024 +} + +/// default wait synced timeout +#[must_use] +#[inline] +pub const fn default_server_wait_synced_timeout() -> Duration { + Duration::from_secs(5) +} + +/// default initial retry timeout +/// FIXME: etcd client has it's own retry mechanism, which may lead to nested retry timeouts. +/// Consider bypassing for proxied etcd client requests. +#[must_use] +#[inline] +pub const fn default_initial_retry_timeout() -> Duration { + Duration::from_millis(1500) +} + +/// default max retry timeout +#[must_use] +#[inline] +pub const fn default_max_retry_timeout() -> Duration { + Duration::from_millis(10_000) +} + +/// default retry count +#[cfg(not(madsim))] +#[must_use] +#[inline] +pub const fn default_retry_count() -> usize { + 3 +} +/// default retry count +#[cfg(madsim)] +#[must_use] +#[inline] +pub const fn default_retry_count() -> usize { + 10 +} + +/// default use backoff +#[must_use] +#[inline] +pub const fn default_fixed_backoff() -> bool { + false +} + +/// default rpc timeout +#[must_use] +#[inline] +pub const fn default_rpc_timeout() -> Duration { + Duration::from_millis(150) +} + +/// default candidate timeout ticks +#[must_use] +#[inline] +pub const fn default_candidate_timeout_ticks() -> u8 { + 2 +} + +/// default client wait synced timeout +#[must_use] +#[inline] +pub const fn default_client_wait_synced_timeout() -> Duration { + Duration::from_secs(2) +} + +/// default client propose timeout +#[must_use] +#[inline] +pub const fn default_propose_timeout() -> Duration { + Duration::from_secs(1) +} + +/// default client id keep alive interval +#[must_use] +#[inline] +pub const fn default_client_id_keep_alive_interval() -> Duration { + Duration::from_secs(1) +} + +/// default follower timeout +#[must_use] +#[inline] +pub const fn default_follower_timeout_ticks() -> u8 { + 5 +} + +/// default number of execute workers +#[must_use] +#[inline] +pub const fn default_cmd_workers() -> u8 { + 8 +} + +/// default range retry timeout +#[must_use] +#[inline] +pub const fn default_range_retry_timeout() -> Duration { + Duration::from_secs(2) +} + +/// default compact timeout +#[must_use] +#[inline] +pub const fn default_compact_timeout() -> Duration { + Duration::from_secs(5) +} + +/// default sync victims interval +#[must_use] +#[inline] +pub const fn default_sync_victims_interval() -> Duration { + Duration::from_millis(10) +} + +/// default gc interval +#[must_use] +#[inline] +pub const fn default_gc_interval() -> Duration { + Duration::from_secs(20) +} + +/// default number of log entries to keep in memory +#[must_use] +#[inline] +pub const fn default_log_entries_cap() -> usize { + 5000 +} + +/// default watch progress notify interval +#[must_use] +#[inline] +pub const fn default_watch_progress_notify_interval() -> Duration { + Duration::from_secs(600) +} + +impl Default for CurpConfig { + #[inline] + fn default() -> Self { + Self { + heartbeat_interval: default_heartbeat_interval(), + wait_synced_timeout: default_server_wait_synced_timeout(), + retry_count: default_retry_count(), + rpc_timeout: default_rpc_timeout(), + batch_timeout: default_batch_timeout(), + batch_max_size: default_batch_max_size(), + follower_timeout_ticks: default_follower_timeout_ticks(), + candidate_timeout_ticks: default_candidate_timeout_ticks(), + engine_cfg: EngineConfig::default(), + cmd_workers: default_cmd_workers(), + gc_interval: default_gc_interval(), + log_entries_cap: default_log_entries_cap(), + } + } +} diff --git a/crates/utils/src/config/engine.rs b/crates/utils/src/config/engine.rs new file mode 100644 index 000000000..1bbedd3e9 --- /dev/null +++ b/crates/utils/src/config/engine.rs @@ -0,0 +1,25 @@ +use serde::Deserialize; +use std::path::PathBuf; + +/// Engine Configuration +#[allow(clippy::module_name_repetitions)] +#[non_exhaustive] +#[derive(Debug, Clone, Deserialize, PartialEq, Eq)] +#[serde( + tag = "type", + content = "data_dir", + rename_all(deserialize = "lowercase") +)] +pub enum EngineConfig { + /// Memory Storage Engine + Memory, + /// RocksDB Storage Engine + RocksDB(PathBuf), +} + +impl Default for EngineConfig { + #[inline] + fn default() -> Self { + Self::Memory + } +} diff --git a/crates/utils/src/config/log.rs b/crates/utils/src/config/log.rs new file mode 100644 index 000000000..14426fc50 --- /dev/null +++ b/crates/utils/src/config/log.rs @@ -0,0 +1,151 @@ +use std::path::PathBuf; + +use getset::Getters; +use serde::Deserialize; + +use tracing_appender::rolling::RollingFileAppender; + +/// Log verbosity level alias +#[allow(clippy::module_name_repetitions)] +pub type LevelConfig = tracing::metadata::LevelFilter; + +/// Log configuration object +#[allow(clippy::module_name_repetitions)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +pub struct LogConfig { + /// Log file path + #[getset(get = "pub")] + #[serde(default)] + path: Option, + /// Log rotation strategy + #[getset(get = "pub")] + #[serde(with = "rotation_format", default = "default_rotation")] + rotation: RotationConfig, + /// Log verbosity level + #[getset(get = "pub")] + #[serde(with = "level_format", default = "default_log_level")] + level: LevelConfig, +} + +impl Default for LogConfig { + #[inline] + fn default() -> Self { + Self { + path: None, + rotation: default_rotation(), + level: default_log_level(), + } + } +} + +/// `LevelConfig` deserialization formatter +pub mod level_format { + use serde::{Deserialize, Deserializer}; + + use super::LevelConfig; + use crate::parse_log_level; + + /// deserializes a cluster duration + #[allow(single_use_lifetimes)] + pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + parse_log_level(&s).map_err(serde::de::Error::custom) + } +} + +/// default log level +#[must_use] +#[inline] +pub const fn default_log_level() -> LevelConfig { + LevelConfig::INFO +} + +impl LogConfig { + /// Generate a new `LogConfig` object + #[must_use] + #[inline] + pub fn new(path: Option, rotation: RotationConfig, level: LevelConfig) -> Self { + Self { + path, + rotation, + level, + } + } +} + +/// Xline log rotation strategy +#[non_exhaustive] +#[allow(clippy::module_name_repetitions)] +#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq)] +#[serde(rename_all(deserialize = "lowercase"))] +pub enum RotationConfig { + /// Rotate log file in every hour + Hourly, + /// Rotate log file every day + Daily, + /// Never rotate log file + Never, +} + +impl std::fmt::Display for RotationConfig { + #[inline] + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match *self { + RotationConfig::Hourly => write!(f, "hourly"), + RotationConfig::Daily => write!(f, "daily"), + RotationConfig::Never => write!(f, "never"), + } + } +} + +/// `RotationConfig` deserialization formatter +pub mod rotation_format { + use serde::{Deserialize, Deserializer}; + + use super::RotationConfig; + use crate::parse_rotation; + + /// deserializes a cluster log rotation strategy + #[allow(single_use_lifetimes)] + pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + parse_rotation(&s).map_err(serde::de::Error::custom) + } +} + +/// default log rotation strategy +#[must_use] +#[inline] +pub const fn default_rotation() -> RotationConfig { + RotationConfig::Never +} + +/// Generates a `RollingFileAppender` from the given `RotationConfig` and `name` +#[must_use] +#[inline] +pub fn file_appender( + rotation: RotationConfig, + file_path: &PathBuf, + name: &str, +) -> RollingFileAppender { + match rotation { + RotationConfig::Hourly => { + tracing_appender::rolling::hourly(file_path, format!("xline_{name}.log")) + } + RotationConfig::Daily => { + tracing_appender::rolling::daily(file_path, format!("xline_{name}.log")) + } + RotationConfig::Never => { + tracing_appender::rolling::never(file_path, format!("xline_{name}.log")) + } + #[allow(unreachable_patterns)] + // It's ok because `parse_rotation` have check the validity before. + _ => unreachable!("should not call file_appender when parse_rotation failed"), + } +} diff --git a/crates/utils/src/config/metrics.rs b/crates/utils/src/config/metrics.rs new file mode 100644 index 000000000..e2a3f6435 --- /dev/null +++ b/crates/utils/src/config/metrics.rs @@ -0,0 +1,151 @@ +use getset::Getters; +use serde::Deserialize; + +/// Metrics push protocol format +pub mod protocol_format { + use serde::{Deserialize, Deserializer}; + + use super::PushProtocol; + use crate::parse_metrics_push_protocol; + + /// deserializes a cluster duration + #[allow(single_use_lifetimes)] + pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + parse_metrics_push_protocol(&s).map_err(serde::de::Error::custom) + } +} + +/// Xline metrics configuration object +#[allow(clippy::module_name_repetitions)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +pub struct MetricsConfig { + /// Enable or not + #[serde(default = "default_metrics_enable")] + #[getset(get = "pub")] + enable: bool, + /// The http port to expose + #[serde(default = "default_metrics_port")] + #[getset(get = "pub")] + port: u16, + /// The http path to expose + #[serde(default = "default_metrics_path")] + #[getset(get = "pub")] + path: String, + /// Enable push or not + #[serde(default = "default_metrics_push")] + #[getset(get = "pub")] + push: bool, + /// Push endpoint + #[serde(default = "default_metrics_push_endpoint")] + #[getset(get = "pub")] + push_endpoint: String, + /// Push protocol + #[serde(with = "protocol_format", default = "default_metrics_push_protocol")] + #[getset(get = "pub")] + push_protocol: PushProtocol, +} + +impl MetricsConfig { + /// Create a new `MetricsConfig` + #[must_use] + #[inline] + pub fn new( + enable: bool, + port: u16, + path: String, + push: bool, + push_endpoint: String, + push_protocol: PushProtocol, + ) -> Self { + Self { + enable, + port, + path, + push, + push_endpoint, + push_protocol, + } + } +} + +impl Default for MetricsConfig { + #[inline] + fn default() -> Self { + Self { + enable: default_metrics_enable(), + port: default_metrics_port(), + path: default_metrics_path(), + push: default_metrics_push(), + push_endpoint: default_metrics_push_endpoint(), + push_protocol: default_metrics_push_protocol(), + } + } +} + +/// Default metrics enable +#[must_use] +#[inline] +pub const fn default_metrics_enable() -> bool { + true +} + +/// Default metrics port +#[must_use] +#[inline] +pub const fn default_metrics_port() -> u16 { + 9100 +} + +/// Default metrics path +#[must_use] +#[inline] +pub fn default_metrics_path() -> String { + "/metrics".to_owned() +} + +/// Default metrics push option +#[must_use] +#[inline] +pub fn default_metrics_push() -> bool { + false +} + +/// Default metrics push protocol +#[must_use] +#[inline] +pub fn default_metrics_push_protocol() -> PushProtocol { + PushProtocol::GRPC +} + +/// Default metrics push endpoint +#[must_use] +#[inline] +pub fn default_metrics_push_endpoint() -> String { + "http://127.0.0.1:4318".to_owned() +} + +/// Xline metrics push protocol +#[non_exhaustive] +#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq, Default)] +#[serde(rename_all(deserialize = "lowercase"))] +pub enum PushProtocol { + /// HTTP protocol + HTTP, + /// GRPC protocol + #[default] + GRPC, +} + +impl std::fmt::Display for PushProtocol { + #[inline] + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match *self { + PushProtocol::HTTP => write!(f, "http"), + PushProtocol::GRPC => write!(f, "grpc"), + } + } +} diff --git a/crates/utils/src/config/mod.rs b/crates/utils/src/config/mod.rs new file mode 100644 index 000000000..14aba4c5d --- /dev/null +++ b/crates/utils/src/config/mod.rs @@ -0,0 +1,400 @@ +/// Xline auth configuration module +pub mod auth; +/// Curp client module +pub mod client; +/// Cluster configuration module +pub mod cluster; +/// Compaction configuration module +pub mod compact; +/// Curp server module +pub mod curp; +/// Engine Configuration module +pub mod engine; +/// Log configuration module +pub mod log; +/// Xline metrics configuration module +pub mod metrics; +/// Prelude module +pub mod prelude; +/// Xline server configuration +pub mod server; +/// Storage Configuration module +pub mod storage; +/// Xline tls configuration module +pub mod tls; +/// Xline tracing configuration module +pub mod trace; + +/// `Duration` deserialization formatter +pub mod duration_format { + use std::time::Duration; + + use serde::{Deserialize, Deserializer}; + + use crate::parse_duration; + + /// deserializes a cluster duration + #[allow(single_use_lifetimes)] // the false positive case blocks us + pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + parse_duration(&s).map_err(serde::de::Error::custom) + } +} + +/// batch size deserialization formatter +pub mod bytes_format { + use serde::{Deserialize, Deserializer}; + + use crate::parse_batch_bytes; + + /// deserializes a cluster duration + #[allow(single_use_lifetimes)] // the false positive case blocks us + pub(crate) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + parse_batch_bytes(&s).map_err(serde::de::Error::custom) + } +} + +#[cfg(test)] +mod tests { + use std::{collections::HashMap, path::PathBuf, time::Duration}; + + use super::prelude::*; + + #[allow(clippy::too_many_lines)] // just a testcase, not too bad + #[test] + fn test_xline_server_config_should_be_loaded() { + let config: XlineServerConfig = toml::from_str( + r#"[cluster] + name = 'node1' + is_leader = true + initial_cluster_state = 'new' + peer_listen_urls = ['127.0.0.1:2380'] + peer_advertise_urls = ['127.0.0.1:2380'] + client_listen_urls = ['127.0.0.1:2379'] + client_advertise_urls = ['127.0.0.1:2379'] + + [cluster.server_timeout] + range_retry_timeout = '3s' + compact_timeout = '5s' + sync_victims_interval = '20ms' + watch_progress_notify_interval = '1s' + + [cluster.peers] + node1 = ['127.0.0.1:2378', '127.0.0.1:2379'] + node2 = ['127.0.0.1:2380'] + node3 = ['127.0.0.1:2381'] + + [cluster.curp_config] + heartbeat_interval = '200ms' + wait_synced_timeout = '100ms' + rpc_timeout = '100ms' + retry_timeout = '100ms' + + [cluster.client_config] + initial_retry_timeout = '5s' + max_retry_timeout = '50s' + + [storage] + engine = { type = 'memory'} + + [compact] + compact_batch_size = 123 + compact_sleep_interval = '5ms' + + [compact.auto_compact_config] + mode = 'periodic' + retention = '10h' + + [log] + path = '/var/log/xline' + rotation = 'daily' + level = 'info' + + [trace] + jaeger_online = false + jaeger_offline = false + jaeger_output_dir = './jaeger_jsons' + jaeger_level = 'info' + + [auth] + auth_public_key = './public_key.pem' + auth_private_key = './private_key.pem' + + [tls] + peer_cert_path = './cert.pem' + peer_key_path = './key.pem' + client_ca_cert_path = './ca.pem' + + [metrics] + enable = true + port = 9100 + path = "/metrics" + push = true + push_endpoint = 'http://some-endpoint.com:4396' + push_protocol = 'http' + "#, + ) + .unwrap(); + + let curp_config = CurpConfigBuilder::default() + .heartbeat_interval(Duration::from_millis(200)) + .wait_synced_timeout(Duration::from_millis(100)) + .rpc_timeout(Duration::from_millis(100)) + .build() + .unwrap(); + + let client_config = ClientConfig::new( + default_client_wait_synced_timeout(), + default_propose_timeout(), + Duration::from_secs(5), + Duration::from_secs(50), + default_retry_count(), + default_fixed_backoff(), + default_client_id_keep_alive_interval(), + ); + + let server_timeout = XlineServerTimeout::new( + Duration::from_secs(3), + Duration::from_secs(5), + Duration::from_millis(20), + Duration::from_secs(1), + ); + + assert_eq!( + *config.cluster(), + ClusterConfig::new( + "node1".to_owned(), + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + HashMap::from_iter([ + ( + "node1".to_owned(), + vec!["127.0.0.1:2378".to_owned(), "127.0.0.1:2379".to_owned()] + ), + ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), + ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), + ]), + true, + curp_config, + client_config, + server_timeout, + InitialClusterState::New + ) + ); + + assert_eq!( + *config.storage(), + StorageConfig::new(EngineConfig::Memory, default_quota()) + ); + + assert_eq!( + *config.log(), + LogConfig::new( + Some(PathBuf::from("/var/log/xline")), + RotationConfig::Daily, + LevelConfig::INFO + ) + ); + assert_eq!( + *config.trace(), + TraceConfig::new( + false, + false, + PathBuf::from("./jaeger_jsons"), + LevelConfig::INFO + ) + ); + + assert_eq!( + *config.compact(), + CompactConfig::new( + 123, + Duration::from_millis(5), + Some(AutoCompactConfig::Periodic(Duration::from_secs( + 10 * 60 * 60 + ))) + ) + ); + + assert_eq!( + *config.auth(), + AuthConfig::new( + Some(PathBuf::from("./public_key.pem")), + Some(PathBuf::from("./private_key.pem")) + ) + ); + + assert_eq!( + *config.tls(), + TlsConfig { + peer_cert_path: Some(PathBuf::from("./cert.pem")), + peer_key_path: Some(PathBuf::from("./key.pem")), + client_ca_cert_path: Some(PathBuf::from("./ca.pem")), + ..Default::default() + } + ); + + assert_eq!( + *config.metrics(), + MetricsConfig::new( + true, + 9100, + "/metrics".to_owned(), + true, + "http://some-endpoint.com:4396".to_owned(), + PushProtocol::HTTP + ), + ); + } + + #[test] + fn test_xline_server_default_config_should_be_loaded() { + let config: XlineServerConfig = toml::from_str( + "[cluster] + name = 'node1' + is_leader = true + peer_listen_urls = ['127.0.0.1:2380'] + peer_advertise_urls = ['127.0.0.1:2380'] + client_listen_urls = ['127.0.0.1:2379'] + client_advertise_urls = ['127.0.0.1:2379'] + + [cluster.peers] + node1 = ['127.0.0.1:2379'] + node2 = ['127.0.0.1:2380'] + node3 = ['127.0.0.1:2381'] + + [cluster.storage] + + [log] + path = '/var/log/xline' + + [storage] + engine = { type = 'rocksdb', data_dir = '/usr/local/xline/data-dir' } + + [compact] + + [trace] + jaeger_online = false + jaeger_offline = false + jaeger_output_dir = './jaeger_jsons' + jaeger_level = 'info' + + [auth] + + [tls] + ", + ) + .unwrap(); + + assert_eq!( + *config.cluster(), + ClusterConfig::new( + "node1".to_owned(), + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + HashMap::from([ + ("node1".to_owned(), vec!["127.0.0.1:2379".to_owned()]), + ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), + ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), + ]), + true, + CurpConfigBuilder::default().build().unwrap(), + ClientConfig::default(), + XlineServerTimeout::default(), + InitialClusterState::default() + ) + ); + + if let EngineConfig::RocksDB(path) = config.storage().engine().clone() { + assert_eq!(path, PathBuf::from("/usr/local/xline/data-dir")); + } else { + unreachable!(); + } + + assert_eq!( + *config.log(), + LogConfig::new( + Some(PathBuf::from("/var/log/xline")), + RotationConfig::Never, + LevelConfig::INFO + ) + ); + assert_eq!( + *config.trace(), + TraceConfig::new( + false, + false, + PathBuf::from("./jaeger_jsons"), + LevelConfig::INFO + ) + ); + assert_eq!(*config.compact(), CompactConfig::default()); + assert_eq!(*config.auth(), AuthConfig::default()); + assert_eq!(*config.tls(), TlsConfig::default()); + assert_eq!(*config.metrics(), MetricsConfig::default()); + } + + #[test] + fn test_auto_revision_compactor_config_should_be_loaded() { + let config: XlineServerConfig = toml::from_str( + "[cluster] + name = 'node1' + is_leader = true + peer_listen_urls = ['127.0.0.1:2380'] + peer_advertise_urls = ['127.0.0.1:2380'] + client_listen_urls = ['127.0.0.1:2379'] + client_advertise_urls = ['127.0.0.1:2379'] + + [cluster.peers] + node1 = ['127.0.0.1:2379'] + node2 = ['127.0.0.1:2380'] + node3 = ['127.0.0.1:2381'] + + [cluster.storage] + + [log] + path = '/var/log/xline' + + [storage] + engine = { type = 'memory' } + + [compact] + + [compact.auto_compact_config] + mode = 'revision' + retention = 10000 + + [trace] + jaeger_online = false + jaeger_offline = false + jaeger_output_dir = './jaeger_jsons' + jaeger_level = 'info' + + [auth] + + [tls] + ", + ) + .unwrap(); + + assert_eq!( + *config.compact(), + CompactConfig::new( + default_compact_batch_size(), + default_compact_sleep_interval(), + Some(AutoCompactConfig::Revision(10000)) + ) + ); + } +} diff --git a/crates/utils/src/config/prelude.rs b/crates/utils/src/config/prelude.rs new file mode 100644 index 000000000..085d773b3 --- /dev/null +++ b/crates/utils/src/config/prelude.rs @@ -0,0 +1,31 @@ +pub use super::cluster::{state_format, ClusterConfig, InitialClusterState, RangeBound}; + +pub use super::compact::{ + default_compact_batch_size, default_compact_sleep_interval, AutoCompactConfig, CompactConfig, +}; + +pub use super::auth::AuthConfig; +pub use super::client::ClientConfig; +pub use super::curp::{ + default_batch_max_size, default_batch_timeout, default_candidate_timeout_ticks, + default_client_id_keep_alive_interval, default_client_wait_synced_timeout, default_cmd_workers, + default_compact_timeout, default_fixed_backoff, default_follower_timeout_ticks, + default_gc_interval, default_heartbeat_interval, default_initial_retry_timeout, + default_log_entries_cap, default_max_retry_timeout, default_propose_timeout, + default_range_retry_timeout, default_retry_count, default_rpc_timeout, + default_server_wait_synced_timeout, default_sync_victims_interval, + default_watch_progress_notify_interval, CurpConfig, CurpConfigBuilder, CurpConfigBuilderError, +}; +pub use super::engine::EngineConfig; +pub use super::log::{ + default_log_level, default_rotation, file_appender, level_format, rotation_format, LevelConfig, + LogConfig, RotationConfig, +}; +pub use super::metrics::{ + default_metrics_enable, default_metrics_path, default_metrics_port, default_metrics_push, + default_metrics_push_endpoint, default_metrics_push_protocol, MetricsConfig, PushProtocol, +}; +pub use super::server::{XlineServerConfig, XlineServerTimeout}; +pub use super::storage::{default_quota, StorageConfig}; +pub use super::tls::TlsConfig; +pub use super::trace::TraceConfig; diff --git a/crates/utils/src/config/server.rs b/crates/utils/src/config/server.rs new file mode 100644 index 000000000..ee9b6ae28 --- /dev/null +++ b/crates/utils/src/config/server.rs @@ -0,0 +1,126 @@ +use std::time::Duration; + +use getset::Getters; +use serde::Deserialize; + +use super::prelude::{ + default_compact_timeout, default_range_retry_timeout, default_sync_victims_interval, + default_watch_progress_notify_interval, AuthConfig, ClusterConfig, CompactConfig, LogConfig, + MetricsConfig, StorageConfig, TlsConfig, TraceConfig, +}; + +use super::duration_format; + +/// Xline server configuration object +#[allow(clippy::module_name_repetitions)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Default, Getters)] +pub struct XlineServerConfig { + /// cluster configuration object + #[getset(get = "pub")] + cluster: ClusterConfig, + /// xline storage configuration object + #[getset(get = "pub")] + storage: StorageConfig, + /// log configuration object + #[getset(get = "pub")] + log: LogConfig, + /// trace configuration object + #[getset(get = "pub")] + trace: TraceConfig, + /// auth configuration object + #[getset(get = "pub")] + auth: AuthConfig, + /// compactor configuration object + #[getset(get = "pub")] + compact: CompactConfig, + /// tls configuration object + #[getset(get = "pub")] + tls: TlsConfig, + /// Metrics config + #[getset(get = "pub")] + #[serde(default = "MetricsConfig::default")] + metrics: MetricsConfig, +} + +impl XlineServerConfig { + /// Generates a new `XlineServerConfig` object + #[must_use] + #[inline] + #[allow(clippy::too_many_arguments)] + pub fn new( + cluster: ClusterConfig, + storage: StorageConfig, + log: LogConfig, + trace: TraceConfig, + auth: AuthConfig, + compact: CompactConfig, + tls: TlsConfig, + metrics: MetricsConfig, + ) -> Self { + Self { + cluster, + storage, + log, + trace, + auth, + compact, + tls, + metrics, + } + } +} + +/// Xline server settings +#[derive(Copy, Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +pub struct XlineServerTimeout { + /// Range request retry timeout settings + #[getset(get = "pub")] + #[serde(with = "duration_format", default = "default_range_retry_timeout")] + range_retry_timeout: Duration, + /// Range request retry timeout settings + #[getset(get = "pub")] + #[serde(with = "duration_format", default = "default_compact_timeout")] + compact_timeout: Duration, + /// Sync victims interval + #[getset(get = "pub")] + #[serde(with = "duration_format", default = "default_sync_victims_interval")] + sync_victims_interval: Duration, + /// Watch progress notify interval settings + #[getset(get = "pub")] + #[serde( + with = "duration_format", + default = "default_watch_progress_notify_interval" + )] + watch_progress_notify_interval: Duration, +} + +impl XlineServerTimeout { + /// Create a new server timeout + #[must_use] + #[inline] + pub fn new( + range_retry_timeout: Duration, + compact_timeout: Duration, + sync_victims_interval: Duration, + watch_progress_notify_interval: Duration, + ) -> Self { + Self { + range_retry_timeout, + compact_timeout, + sync_victims_interval, + watch_progress_notify_interval, + } + } +} + +impl Default for XlineServerTimeout { + #[inline] + fn default() -> Self { + Self { + range_retry_timeout: default_range_retry_timeout(), + compact_timeout: default_compact_timeout(), + sync_victims_interval: default_sync_victims_interval(), + watch_progress_notify_interval: default_watch_progress_notify_interval(), + } + } +} diff --git a/crates/utils/src/config/storage.rs b/crates/utils/src/config/storage.rs new file mode 100644 index 000000000..d2721e7fb --- /dev/null +++ b/crates/utils/src/config/storage.rs @@ -0,0 +1,46 @@ +use getset::Getters; +use serde::Deserialize; + +use super::prelude::EngineConfig; + +/// /// Storage Configuration +#[derive(Debug, Clone, Deserialize, PartialEq, Eq, Getters)] +#[allow(clippy::module_name_repetitions)] +#[non_exhaustive] +pub struct StorageConfig { + /// Engine Configuration + #[serde(default = "EngineConfig::default")] + #[getset(get = "pub")] + engine: EngineConfig, + /// Quota + #[serde(default = "default_quota")] + #[getset(get = "pub")] + quota: u64, +} + +impl StorageConfig { + /// Create a new storage config + #[inline] + #[must_use] + pub fn new(engine: EngineConfig, quota: u64) -> Self { + Self { engine, quota } + } +} + +impl Default for StorageConfig { + #[inline] + fn default() -> Self { + Self { + engine: EngineConfig::default(), + quota: default_quota(), + } + } +} + +/// Default quota: 8GB +#[inline] +#[must_use] +pub fn default_quota() -> u64 { + // 8 * 1024 * 1024 * 1024 + 0x0002_0000_0000 +} diff --git a/crates/utils/src/config/tls.rs b/crates/utils/src/config/tls.rs new file mode 100644 index 000000000..de56d1bc3 --- /dev/null +++ b/crates/utils/src/config/tls.rs @@ -0,0 +1,59 @@ +use std::path::PathBuf; + +use getset::Getters; +use serde::Deserialize; + +/// Xline tls configuration object +#[allow(clippy::module_name_repetitions)] +#[non_exhaustive] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Default)] +pub struct TlsConfig { + /// The CA certificate file used by peer to verify client certificates + #[getset(get = "pub")] + pub peer_ca_cert_path: Option, + /// The public key file used by peer + #[getset(get = "pub")] + pub peer_cert_path: Option, + /// The private key file used by peer + #[getset(get = "pub")] + pub peer_key_path: Option, + /// The CA certificate file used by client to verify peer certificates + #[getset(get = "pub")] + pub client_ca_cert_path: Option, + /// The public key file used by client + #[getset(get = "pub")] + pub client_cert_path: Option, + /// The private key file used by client + #[getset(get = "pub")] + pub client_key_path: Option, +} + +impl TlsConfig { + /// Create a new `TlsConfig` object + #[must_use] + #[inline] + pub fn new( + peer_ca_cert_path: Option, + peer_cert_path: Option, + peer_key_path: Option, + client_ca_cert_path: Option, + client_cert_path: Option, + client_key_path: Option, + ) -> Self { + Self { + peer_ca_cert_path, + peer_cert_path, + peer_key_path, + client_ca_cert_path, + client_cert_path, + client_key_path, + } + } + + /// Whether the server tls is enabled + #[must_use] + #[inline] + pub fn server_tls_enabled(&self) -> bool { + self.peer_cert_path.is_some() && self.peer_key_path.is_some() + } +} diff --git a/crates/utils/src/config/trace.rs b/crates/utils/src/config/trace.rs new file mode 100644 index 000000000..935f366ce --- /dev/null +++ b/crates/utils/src/config/trace.rs @@ -0,0 +1,56 @@ +use std::path::PathBuf; + +use getset::Getters; +use serde::Deserialize; + +use super::prelude::{default_log_level, level_format, LevelConfig}; + +/// Xline tracing configuration object +#[allow(clippy::module_name_repetitions)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +pub struct TraceConfig { + /// Open jaeger online, sending data to jaeger agent directly + #[getset(get = "pub")] + jaeger_online: bool, + /// Open jaeger offline, saving data to the `jaeger_output_dir` + #[getset(get = "pub")] + jaeger_offline: bool, + /// The dir path to save the data when `jaeger_offline` is on + #[getset(get = "pub")] + jaeger_output_dir: PathBuf, + /// The verbosity level of tracing + #[getset(get = "pub")] + #[serde(with = "level_format", default = "default_log_level")] + jaeger_level: LevelConfig, +} + +impl Default for TraceConfig { + #[inline] + fn default() -> Self { + Self { + jaeger_online: false, + jaeger_offline: false, + jaeger_output_dir: "".into(), + jaeger_level: default_log_level(), + } + } +} + +impl TraceConfig { + /// Generate a new `TraceConfig` object + #[must_use] + #[inline] + pub fn new( + jaeger_online: bool, + jaeger_offline: bool, + jaeger_output_dir: PathBuf, + jaeger_level: LevelConfig, + ) -> Self { + Self { + jaeger_online, + jaeger_offline, + jaeger_output_dir, + jaeger_level, + } + } +} diff --git a/crates/utils/src/parser.rs b/crates/utils/src/parser.rs index 75289a5f7..488883598 100644 --- a/crates/utils/src/parser.rs +++ b/crates/utils/src/parser.rs @@ -4,8 +4,8 @@ use clippy_utilities::OverflowArithmetic; use regex::Regex; use thiserror::Error; -use crate::config::{ - ClusterRange, InitialClusterState, LevelConfig, MetricsPushProtocol, RotationConfig, +use crate::config::prelude::{ + InitialClusterState, LevelConfig, PushProtocol, RangeBound, RotationConfig, }; /// seconds per minute @@ -76,9 +76,9 @@ pub fn parse_members(s: &str) -> Result>, ConfigPars /// /// Return error when parsing the given string to `ClusterRange` failed #[inline] -pub fn parse_range(s: &str) -> Result { +pub fn parse_range(s: &str) -> Result { if let Some((start, end)) = s.split_once("..") { - Ok(ClusterRange { + Ok(RangeBound { start: start.parse::()?, end: end.parse::()?, }) @@ -286,10 +286,10 @@ pub fn parse_batch_bytes(s: &str) -> Result { /// /// Return error when parsing the given string to `MetricsPushProtocol` failed #[inline] -pub fn parse_metrics_push_protocol(s: &str) -> Result { +pub fn parse_metrics_push_protocol(s: &str) -> Result { match s { - "http" => Ok(MetricsPushProtocol::HTTP), - "grpc" => Ok(MetricsPushProtocol::GRPC), + "http" => Ok(PushProtocol::HTTP), + "grpc" => Ok(PushProtocol::GRPC), _ => Err(ConfigParseError::InvalidValue(format!( "the metrics push protocol should be one of 'http' or 'grpc' ({s})" ))), @@ -409,11 +409,11 @@ mod test { fn test_parse_metrics_push_protocol() { assert_eq!( parse_metrics_push_protocol("http").unwrap(), - MetricsPushProtocol::HTTP + PushProtocol::HTTP ); assert_eq!( parse_metrics_push_protocol("grpc").unwrap(), - MetricsPushProtocol::GRPC + PushProtocol::GRPC ); assert!(parse_metrics_push_protocol("thrift").is_err()); } diff --git a/crates/xline-client/src/lib.rs b/crates/xline-client/src/lib.rs index 3bc638ba2..77e42cb78 100644 --- a/crates/xline-client/src/lib.rs +++ b/crates/xline-client/src/lib.rs @@ -174,7 +174,7 @@ use tonic::transport::ClientTlsConfig; use tower::Service; #[cfg(madsim)] use utils::ClientTlsConfig; -use utils::{build_endpoint, config::ClientConfig}; +use utils::{build_endpoint, config::prelude::ClientConfig}; use xlineapi::command::{Command, CurpClient}; use crate::{ diff --git a/crates/xline-test-utils/src/lib.rs b/crates/xline-test-utils/src/lib.rs index b3135bf24..d86a3a3fb 100644 --- a/crates/xline-test-utils/src/lib.rs +++ b/crates/xline-test-utils/src/lib.rs @@ -9,7 +9,7 @@ use tokio::{ time::{self, Duration}, }; use tonic::transport::ClientTlsConfig; -use utils::config::{ +use utils::config::prelude::{ default_quota, AuthConfig, ClusterConfig, CompactConfig, EngineConfig, InitialClusterState, LogConfig, MetricsConfig, StorageConfig, TlsConfig, TraceConfig, XlineServerConfig, }; @@ -319,7 +319,7 @@ impl Drop for Cluster { { let _ignore = tokio::fs::remove_dir_all(path).await; } - if let EngineConfig::RocksDB(ref path) = cfg.storage().engine { + if let EngineConfig::RocksDB(ref path) = cfg.storage().engine() { let _ignore = tokio::fs::remove_dir_all(path).await; } } diff --git a/crates/xline/src/server/maintenance.rs b/crates/xline/src/server/maintenance.rs index 528625aa1..30844f459 100644 --- a/crates/xline/src/server/maintenance.rs +++ b/crates/xline/src/server/maintenance.rs @@ -278,7 +278,7 @@ mod test { use tempfile::TempDir; use test_macros::abort_on_panic; use tokio_stream::StreamExt; - use utils::config::EngineConfig; + use utils::config::prelude::EngineConfig; use super::*; use crate::storage::db::DB; diff --git a/crates/xline/src/server/watch_server.rs b/crates/xline/src/server/watch_server.rs index 29f67cf74..09c1434df 100644 --- a/crates/xline/src/server/watch_server.rs +++ b/crates/xline/src/server/watch_server.rs @@ -424,7 +424,7 @@ mod test { sync::mpsc, time::{sleep, timeout}, }; - use utils::config::{default_watch_progress_notify_interval, EngineConfig}; + use utils::config::prelude::{default_watch_progress_notify_interval, EngineConfig}; use xlineapi::RequestWrapper; use super::*; diff --git a/crates/xline/src/server/xline_server.rs b/crates/xline/src/server/xline_server.rs index 73a8a4ac6..b940c8489 100644 --- a/crates/xline/src/server/xline_server.rs +++ b/crates/xline/src/server/xline_server.rs @@ -24,7 +24,7 @@ use tonic::transport::{server::Router, Server}; use tracing::{info, warn}; use utils::{ barrier::IdBarrier, - config::{ + config::prelude::{ AuthConfig, ClusterConfig, CompactConfig, EngineConfig, InitialClusterState, StorageConfig, TlsConfig, }, @@ -350,7 +350,7 @@ impl XlineServer { .get_shutdown_listener(TaskName::TonicServer) .unwrap_or_else(|| unreachable!("cluster should never shutdown before start")); let n2 = n1.clone(); - let db = DB::open(&self.storage_config.engine)?; + let db = DB::open(self.storage_config.engine())?; let key_pair = Self::read_key_pair(&self.auth_config).await?; let (xline_router, curp_router, curp_client) = self.init_router(db, key_pair).await?; let handle = tokio::spawn(async move { @@ -376,7 +376,7 @@ impl XlineServer { IO::ConnectInfo: Clone + Send + Sync + 'static, IE: Into> + Send, { - let db = DB::open(&self.storage_config.engine)?; + let db = DB::open(self.storage_config.engine())?; let key_pair = Self::read_key_pair(&self.auth_config).await?; let (xline_router, curp_router, curp_client) = self.init_router(db, key_pair).await?; self.task_manager @@ -476,9 +476,9 @@ impl XlineServer { Arc::clone(&db), Arc::clone(&id_barrier), Arc::clone(&compact_events), - self.storage_config.quota, + *self.storage_config.quota(), )); - let snapshot_allocator: Box = match self.storage_config.engine { + let snapshot_allocator: Box = match *self.storage_config.engine() { EngineConfig::Memory => Box::::default(), EngineConfig::RocksDB(_) => Box::::default(), #[allow(clippy::unimplemented)] diff --git a/crates/xline/src/storage/auth_store/store.rs b/crates/xline/src/storage/auth_store/store.rs index 66fd776ce..41b9d396d 100644 --- a/crates/xline/src/storage/auth_store/store.rs +++ b/crates/xline/src/storage/auth_store/store.rs @@ -1176,7 +1176,7 @@ mod test { use std::collections::HashMap; use merged_range::MergedRange; - use utils::config::EngineConfig; + use utils::config::prelude::EngineConfig; use super::*; use crate::{ diff --git a/crates/xline/src/storage/compact/mod.rs b/crates/xline/src/storage/compact/mod.rs index 7768667e5..66b68ec68 100644 --- a/crates/xline/src/storage/compact/mod.rs +++ b/crates/xline/src/storage/compact/mod.rs @@ -7,7 +7,7 @@ use periodic_compactor::PeriodicCompactor; use revision_compactor::RevisionCompactor; use tokio::time::sleep; use utils::{ - config::AutoCompactConfig, + config::prelude::AutoCompactConfig, task_manager::{tasks::TaskName, Listener, TaskManager}, }; use xlineapi::{command::Command, execute_error::ExecuteError, RequestWrapper}; diff --git a/crates/xline/src/storage/db.rs b/crates/xline/src/storage/db.rs index 4870b9211..37a6f626b 100644 --- a/crates/xline/src/storage/db.rs +++ b/crates/xline/src/storage/db.rs @@ -7,7 +7,7 @@ use engine::{ }; use prost::Message; use utils::{ - config::EngineConfig, + config::prelude::EngineConfig, table_names::{ ALARM_TABLE, AUTH_TABLE, KV_TABLE, LEASE_TABLE, META_TABLE, ROLE_TABLE, USER_TABLE, XLINE_TABLES, diff --git a/crates/xline/src/storage/kv_store.rs b/crates/xline/src/storage/kv_store.rs index 19b8fb20a..05d900c30 100644 --- a/crates/xline/src/storage/kv_store.rs +++ b/crates/xline/src/storage/kv_store.rs @@ -1253,7 +1253,7 @@ mod test { use test_macros::abort_on_panic; use tokio::{runtime::Handle, task::block_in_place}; use utils::{ - config::EngineConfig, + config::prelude::EngineConfig, task_manager::{tasks::TaskName, TaskManager}, }; diff --git a/crates/xline/src/storage/kvwatcher.rs b/crates/xline/src/storage/kvwatcher.rs index 6b8524b56..49d969e9b 100644 --- a/crates/xline/src/storage/kvwatcher.rs +++ b/crates/xline/src/storage/kvwatcher.rs @@ -595,7 +595,7 @@ mod test { use engine::TransactionApi; use test_macros::abort_on_panic; use tokio::time::{sleep, timeout}; - use utils::config::EngineConfig; + use utils::config::prelude::EngineConfig; use xlineapi::RequestWrapper; use super::*; diff --git a/crates/xline/src/storage/lease_store/mod.rs b/crates/xline/src/storage/lease_store/mod.rs index b9fd0f52e..262809fd8 100644 --- a/crates/xline/src/storage/lease_store/mod.rs +++ b/crates/xline/src/storage/lease_store/mod.rs @@ -385,7 +385,7 @@ mod test { use std::{error::Error, time::Duration}; use test_macros::abort_on_panic; - use utils::config::EngineConfig; + use utils::config::prelude::EngineConfig; use super::*; use crate::{ diff --git a/crates/xline/src/utils/args.rs b/crates/xline/src/utils/args.rs index f8b6d44c8..5a9da207f 100644 --- a/crates/xline/src/utils/args.rs +++ b/crates/xline/src/utils/args.rs @@ -4,7 +4,7 @@ use anyhow::Result; use clap::Parser; use tokio::fs; use utils::{ - config::{ + config::prelude::{ default_batch_max_size, default_batch_timeout, default_candidate_timeout_ticks, default_client_id_keep_alive_interval, default_client_wait_synced_timeout, default_cmd_workers, default_compact_batch_size, default_compact_sleep_interval, @@ -16,9 +16,9 @@ use utils::{ default_rotation, default_rpc_timeout, default_server_wait_synced_timeout, default_sync_victims_interval, default_watch_progress_notify_interval, AuthConfig, AutoCompactConfig, ClientConfig, ClusterConfig, CompactConfig, CurpConfigBuilder, - EngineConfig, InitialClusterState, LevelConfig, LogConfig, MetricsConfig, - MetricsPushProtocol, RotationConfig, ServerTimeout, StorageConfig, TlsConfig, TraceConfig, - XlineServerConfig, + EngineConfig, InitialClusterState, LevelConfig, LogConfig, MetricsConfig, PushProtocol, + RotationConfig, StorageConfig, TlsConfig, TraceConfig, XlineServerConfig, + XlineServerTimeout, }, parse_batch_bytes, parse_duration, parse_log_file, parse_log_level, parse_members, parse_metrics_push_protocol, parse_rotation, parse_state, ConfigFileError, @@ -90,7 +90,7 @@ pub struct ServerArgs { metrics_push_endpoint: String, /// Collector protocol to collect metrics #[clap(long, value_parser = parse_metrics_push_protocol, default_value_t = default_metrics_push_protocol())] - metrics_push_protocol: MetricsPushProtocol, + metrics_push_protocol: PushProtocol, /// Log file path #[clap(long, value_parser = parse_log_file, default_value = None)] log_file: Option, @@ -269,7 +269,7 @@ impl From for XlineServerConfig { args.client_keep_alive_interval .unwrap_or_else(default_client_id_keep_alive_interval), ); - let server_timeout = ServerTimeout::new( + let server_timeout = XlineServerTimeout::new( args.range_retry_timeout .unwrap_or_else(default_range_retry_timeout), args.compact_timeout.unwrap_or_else(default_compact_timeout), diff --git a/crates/xline/src/utils/metrics.rs b/crates/xline/src/utils/metrics.rs index 3621936b6..47b10af46 100644 --- a/crates/xline/src/utils/metrics.rs +++ b/crates/xline/src/utils/metrics.rs @@ -4,7 +4,7 @@ use opentelemetry::global; use opentelemetry_otlp::WithExportConfig; use opentelemetry_sdk::{metrics::SdkMeterProvider, runtime::Tokio}; use tracing::info; -use utils::config::{MetricsConfig, MetricsPushProtocol}; +use utils::config::prelude::{MetricsConfig, PushProtocol}; /// Start metrics server /// @@ -24,7 +24,7 @@ pub fn init_metrics(config: &MetricsConfig) -> anyhow::Result<()> { // push mode let _ig = match *config.push_protocol() { - MetricsPushProtocol::HTTP => opentelemetry_otlp::new_pipeline() + PushProtocol::HTTP => opentelemetry_otlp::new_pipeline() .metrics(Tokio) .with_exporter( opentelemetry_otlp::new_exporter() @@ -32,7 +32,7 @@ pub fn init_metrics(config: &MetricsConfig) -> anyhow::Result<()> { .with_endpoint(config.push_endpoint()), ) .build(), - MetricsPushProtocol::GRPC => opentelemetry_otlp::new_pipeline() + PushProtocol::GRPC => opentelemetry_otlp::new_pipeline() .metrics(Tokio) .with_exporter( opentelemetry_otlp::new_exporter() diff --git a/crates/xline/src/utils/trace.rs b/crates/xline/src/utils/trace.rs index 9fad02fa9..7bb1e0e63 100644 --- a/crates/xline/src/utils/trace.rs +++ b/crates/xline/src/utils/trace.rs @@ -9,7 +9,7 @@ use tracing_subscriber::layer::SubscriberExt; use tracing_subscriber::util::SubscriberInitExt; use tracing_subscriber::EnvFilter; use tracing_subscriber::{fmt::format, Layer}; -use utils::config::{file_appender, LogConfig, RotationConfig, TraceConfig}; +use utils::config::prelude::{file_appender, LogConfig, RotationConfig, TraceConfig}; /// Return a Box trait from the config fn generate_writer(name: &str, log_config: &LogConfig) -> Box { diff --git a/crates/xline/tests/it/auth_test.rs b/crates/xline/tests/it/auth_test.rs index 935fcc0c5..ddba12298 100644 --- a/crates/xline/tests/it/auth_test.rs +++ b/crates/xline/tests/it/auth_test.rs @@ -1,7 +1,7 @@ use std::{error::Error, iter, path::PathBuf}; use test_macros::abort_on_panic; -use utils::config::{ +use utils::config::prelude::{ AuthConfig, ClusterConfig, CompactConfig, LogConfig, MetricsConfig, StorageConfig, TlsConfig, TraceConfig, XlineServerConfig, }; diff --git a/crates/xline/tests/it/tls_test.rs b/crates/xline/tests/it/tls_test.rs index 00b42c84b..d1024ce65 100644 --- a/crates/xline/tests/it/tls_test.rs +++ b/crates/xline/tests/it/tls_test.rs @@ -3,7 +3,7 @@ use std::{fs, iter, path::PathBuf}; use etcd_client::ConnectOptions; use test_macros::abort_on_panic; use tonic::transport::{Certificate, ClientTlsConfig, Identity}; -use utils::config::{ +use utils::config::prelude::{ AuthConfig, ClusterConfig, CompactConfig, LogConfig, MetricsConfig, StorageConfig, TlsConfig, TraceConfig, XlineServerConfig, }; diff --git a/crates/xlinectl/src/main.rs b/crates/xlinectl/src/main.rs index 42fbd82b3..4eaea1928 100644 --- a/crates/xlinectl/src/main.rs +++ b/crates/xlinectl/src/main.rs @@ -162,7 +162,7 @@ use std::{path::PathBuf, time::Duration}; use anyhow::Result; use clap::{arg, value_parser, Command}; use command::compaction; -use ext_utils::config::ClientConfig; +use ext_utils::config::prelude::ClientConfig; use tokio::fs; use tonic::transport::{Certificate, ClientTlsConfig}; use xline_client::{Client, ClientOptions}; From e15f39364ec89cb1c4abc26e5fc9380c3eca4642 Mon Sep 17 00:00:00 2001 From: iGxnon Date: Wed, 30 Oct 2024 17:24:23 +0800 Subject: [PATCH 2/9] chore: remove redundant stuff Signed-off-by: iGxnon --- .github/ISSUE_TEMPLATE/bug_report.yaml | 2 +- .github/ISSUE_TEMPLATE/feature_request.yaml | 2 +- .github/workflows/benchmark.yml | 11 --- .github/workflows/validation.yml | 11 --- CHANGELOG.md | 6 +- USAGE.md | 84 --------------------- 6 files changed, 3 insertions(+), 113 deletions(-) delete mode 100644 USAGE.md diff --git a/.github/ISSUE_TEMPLATE/bug_report.yaml b/.github/ISSUE_TEMPLATE/bug_report.yaml index 2b4c7532b..b44cea89b 100644 --- a/.github/ISSUE_TEMPLATE/bug_report.yaml +++ b/.github/ISSUE_TEMPLATE/bug_report.yaml @@ -46,7 +46,7 @@ body: id: terms attributes: label: Code of Conduct - description: By submitting this issue, you agree to follow our [Code of Conduct](https://github.com/datenlord/Xline/blob/master/CODE_OF_CONDUCT.md) + description: By submitting this issue, you agree to follow our [Code of Conduct](https://github.com/xline-kv/Xline/blob/master/CODE_OF_CONDUCT.md) options: - label: I agree to follow this project's Code of Conduct required: true diff --git a/.github/ISSUE_TEMPLATE/feature_request.yaml b/.github/ISSUE_TEMPLATE/feature_request.yaml index 46594997e..400bb7d74 100644 --- a/.github/ISSUE_TEMPLATE/feature_request.yaml +++ b/.github/ISSUE_TEMPLATE/feature_request.yaml @@ -23,7 +23,7 @@ body: id: terms attributes: label: Code of Conduct - description: By submitting this issue, you agree to follow our [Code of Conduct](https://github.com/datenlord/Xline/blob/master/CODE_OF_CONDUCT.md) + description: By submitting this issue, you agree to follow our [Code of Conduct](https://github.com/xline-kv/Xline/blob/master/CODE_OF_CONDUCT.md) options: - label: I agree to follow this project's Code of Conduct required: true diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml index 8d38ad63e..8c489334f 100644 --- a/.github/workflows/benchmark.yml +++ b/.github/workflows/benchmark.yml @@ -10,17 +10,6 @@ env: jobs: benchmark: - name: benchmark - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - with: - submodules: recursive - - - name: Setup sccache-cache - uses: mozilla-actions/sccache-action@v0.0.5 - - call_build_xline: name: Build and Upload Artifacts uses: ./.github/workflows/build_xline.yml with: diff --git a/.github/workflows/validation.yml b/.github/workflows/validation.yml index cef9c3851..053808f10 100644 --- a/.github/workflows/validation.yml +++ b/.github/workflows/validation.yml @@ -9,17 +9,6 @@ on: jobs: validation: - name: Validation - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - with: - submodules: recursive - - - name: Setup sccache-cache - uses: mozilla-actions/sccache-action@v0.0.5 - - call_build_xline: name: Build and Upload Artifacts uses: ./.github/workflows/build_xline.yml with: diff --git a/CHANGELOG.md b/CHANGELOG.md index 4fc36d72b..c8db39b79 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -252,16 +252,12 @@ Existing distributed KV stores mostly adopt the `Raft` consensus protocol, which - basic `Xline` client (use `Curp` directly) - benchmark tool -### Usage -[Usage doc](./USAGE.md) - ### Note In this release, we only provide binary files for X86_64 linux. Other platforms need to be compiled by yourself. we will add more support in the future. - ### Links -- GitHub: https://github.com/datenlord/Xline +- GitHub: https://github.com/xline-kv/Xline - Crate: https://crates.io/crates/xline - Docs: https://docs.rs/xline - Paper of Curp: https://www.usenix.org/system/files/nsdi19-park.pdf diff --git a/USAGE.md b/USAGE.md deleted file mode 100644 index 44c208c32..000000000 --- a/USAGE.md +++ /dev/null @@ -1,84 +0,0 @@ -# Usage - -## Configurations - -The Xline configuration file is written in toml format and the default path is `/etc/xline_server`.conf. If you need to change the path of the configuration file, you can set it via the environment variable `XLINE_SERVER_CONFIG`. - -The configuration file has four sections, as follows: - -1. cluster section: contains information about curp cluster, including basic information, cluster member configuration, curp server timeout settings (optional), curp client timeout settings (optional). -2. log section: contains the Xline log-related configuration, where path is required, rotation (optional, default value is `daily`), level (optional, default value is `info`) -3. trace section: contains the jaeger's trace mode (online or offline), trace level and the log directory in offline mode -4. auth section: contains the address of the key pair required for authentication - -A minimum config file looks like: - -```toml -[cluster] -name = 'node1' # server identity -is_leader = true # leader flag which indicates that the current server - # is a leader or not when booting up a cluster - -[cluster.members] -node1 = '127.0.0.1:2379' # the server name and its address -node2 = '127.0.0.1:2380' -node3 = '127.0.0.1:2381' - -[log] -path = '/var/log/xline' - -[trace] -jaeger_online = false # jaeger tracing online pattern -jaeger_offline = false # jaeger tracing offline pattern -jaeger_output_dir = 'var/log/xline/jaeger_jsons' -jaeger_level = 'info' # tracing log level - -[auth] -auth_public_key = '/etc/xline/public_key.pem' -auth_private_key = '/etc/xline/private_key.pem' -``` - -For tuning and development purpose, the cluster section provides two subsections, curp_cfg, and client_config, with the following definitions and default values. - -```toml -[cluster.curp_config] -heartbeat_interval = '300ms' # the curp heartbeat(tick) interval of which default value is 300ms -wait_synced_timeout = '5s' # server wait synced timeout -rpc_timeout = '50ms' # the rpc timeout of which default value is 50ms -retry_timeout = '50ms' # the rpc retry interval of which default value is 50ms -follower_timeout_ticks = 5 # if a follower cannot receive heartbeats from a leader during - # after `follower_timeout_ticks` ticks, then it will issue an - # election. Its default value is 5. -candidate_timeout_ticks = 2 # if a candidate cannot win an election, it will retry election - # after `candidate_timeout_ticks` ticks. Its default value is 2 - - -[cluster.client_config] -propose_timeout = '1s' # client propose timeout -wait_synced_timeout = '2s' # client wait synced timeout -retry_timeout = '50ms' # the rpc retry interval, of which the default is 50ms -``` - -## Boot up an Xline cluster - -1. Download binary from [release](https://github.com/xline-kv/Xline/releases) page. -2. Use the following command to start cluster: - - ```bash - # Run in 3 terminals. If you want more logs, add `RUST_LOG=curp=debug,xline=debug` before the command. - - ./xline --name node1 --members node1=127.0.0.1:2379,node2=127.0.0.1:2380,node3=127.0.0.1:2381 --is-leader - - ./xline --name node2 --members node1=127.0.0.1:2379,node2=127.0.0.1:2380,node3=127.0.0.1:2381 - - ./xline --name node3 --members node1=127.0.0.1:2379,node2=127.0.0.1:2380,node3=127.0.0.1:2381 - ``` - -3. Download or build `etcdctl` from [etcd](https://github.com/etcd-io/etcd) project. -4. Use `etcdctl` to operate the cluster: - - ```bash - etcdctl --endpoints=http://127.0.0.1:2379 put foo bar - - etcdctl --endpoints=http://127.0.0.1:2379 get foo - ``` From d357f9776d03298e1429719d084591dc78565277 Mon Sep 17 00:00:00 2001 From: JianMinTang Date: Mon, 25 Nov 2024 17:06:50 +0800 Subject: [PATCH 3/9] chore: place test on server configuration file Signed-off-by: JianMinTang --- crates/utils/src/config/mod.rs | 338 ----------------------------- crates/utils/src/config/server.rs | 338 +++++++++++++++++++++++++++++ crates/utils/src/config/storage.rs | 2 +- 3 files changed, 339 insertions(+), 339 deletions(-) diff --git a/crates/utils/src/config/mod.rs b/crates/utils/src/config/mod.rs index 14aba4c5d..947ad7fb5 100644 --- a/crates/utils/src/config/mod.rs +++ b/crates/utils/src/config/mod.rs @@ -60,341 +60,3 @@ pub mod bytes_format { parse_batch_bytes(&s).map_err(serde::de::Error::custom) } } - -#[cfg(test)] -mod tests { - use std::{collections::HashMap, path::PathBuf, time::Duration}; - - use super::prelude::*; - - #[allow(clippy::too_many_lines)] // just a testcase, not too bad - #[test] - fn test_xline_server_config_should_be_loaded() { - let config: XlineServerConfig = toml::from_str( - r#"[cluster] - name = 'node1' - is_leader = true - initial_cluster_state = 'new' - peer_listen_urls = ['127.0.0.1:2380'] - peer_advertise_urls = ['127.0.0.1:2380'] - client_listen_urls = ['127.0.0.1:2379'] - client_advertise_urls = ['127.0.0.1:2379'] - - [cluster.server_timeout] - range_retry_timeout = '3s' - compact_timeout = '5s' - sync_victims_interval = '20ms' - watch_progress_notify_interval = '1s' - - [cluster.peers] - node1 = ['127.0.0.1:2378', '127.0.0.1:2379'] - node2 = ['127.0.0.1:2380'] - node3 = ['127.0.0.1:2381'] - - [cluster.curp_config] - heartbeat_interval = '200ms' - wait_synced_timeout = '100ms' - rpc_timeout = '100ms' - retry_timeout = '100ms' - - [cluster.client_config] - initial_retry_timeout = '5s' - max_retry_timeout = '50s' - - [storage] - engine = { type = 'memory'} - - [compact] - compact_batch_size = 123 - compact_sleep_interval = '5ms' - - [compact.auto_compact_config] - mode = 'periodic' - retention = '10h' - - [log] - path = '/var/log/xline' - rotation = 'daily' - level = 'info' - - [trace] - jaeger_online = false - jaeger_offline = false - jaeger_output_dir = './jaeger_jsons' - jaeger_level = 'info' - - [auth] - auth_public_key = './public_key.pem' - auth_private_key = './private_key.pem' - - [tls] - peer_cert_path = './cert.pem' - peer_key_path = './key.pem' - client_ca_cert_path = './ca.pem' - - [metrics] - enable = true - port = 9100 - path = "/metrics" - push = true - push_endpoint = 'http://some-endpoint.com:4396' - push_protocol = 'http' - "#, - ) - .unwrap(); - - let curp_config = CurpConfigBuilder::default() - .heartbeat_interval(Duration::from_millis(200)) - .wait_synced_timeout(Duration::from_millis(100)) - .rpc_timeout(Duration::from_millis(100)) - .build() - .unwrap(); - - let client_config = ClientConfig::new( - default_client_wait_synced_timeout(), - default_propose_timeout(), - Duration::from_secs(5), - Duration::from_secs(50), - default_retry_count(), - default_fixed_backoff(), - default_client_id_keep_alive_interval(), - ); - - let server_timeout = XlineServerTimeout::new( - Duration::from_secs(3), - Duration::from_secs(5), - Duration::from_millis(20), - Duration::from_secs(1), - ); - - assert_eq!( - *config.cluster(), - ClusterConfig::new( - "node1".to_owned(), - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - HashMap::from_iter([ - ( - "node1".to_owned(), - vec!["127.0.0.1:2378".to_owned(), "127.0.0.1:2379".to_owned()] - ), - ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), - ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), - ]), - true, - curp_config, - client_config, - server_timeout, - InitialClusterState::New - ) - ); - - assert_eq!( - *config.storage(), - StorageConfig::new(EngineConfig::Memory, default_quota()) - ); - - assert_eq!( - *config.log(), - LogConfig::new( - Some(PathBuf::from("/var/log/xline")), - RotationConfig::Daily, - LevelConfig::INFO - ) - ); - assert_eq!( - *config.trace(), - TraceConfig::new( - false, - false, - PathBuf::from("./jaeger_jsons"), - LevelConfig::INFO - ) - ); - - assert_eq!( - *config.compact(), - CompactConfig::new( - 123, - Duration::from_millis(5), - Some(AutoCompactConfig::Periodic(Duration::from_secs( - 10 * 60 * 60 - ))) - ) - ); - - assert_eq!( - *config.auth(), - AuthConfig::new( - Some(PathBuf::from("./public_key.pem")), - Some(PathBuf::from("./private_key.pem")) - ) - ); - - assert_eq!( - *config.tls(), - TlsConfig { - peer_cert_path: Some(PathBuf::from("./cert.pem")), - peer_key_path: Some(PathBuf::from("./key.pem")), - client_ca_cert_path: Some(PathBuf::from("./ca.pem")), - ..Default::default() - } - ); - - assert_eq!( - *config.metrics(), - MetricsConfig::new( - true, - 9100, - "/metrics".to_owned(), - true, - "http://some-endpoint.com:4396".to_owned(), - PushProtocol::HTTP - ), - ); - } - - #[test] - fn test_xline_server_default_config_should_be_loaded() { - let config: XlineServerConfig = toml::from_str( - "[cluster] - name = 'node1' - is_leader = true - peer_listen_urls = ['127.0.0.1:2380'] - peer_advertise_urls = ['127.0.0.1:2380'] - client_listen_urls = ['127.0.0.1:2379'] - client_advertise_urls = ['127.0.0.1:2379'] - - [cluster.peers] - node1 = ['127.0.0.1:2379'] - node2 = ['127.0.0.1:2380'] - node3 = ['127.0.0.1:2381'] - - [cluster.storage] - - [log] - path = '/var/log/xline' - - [storage] - engine = { type = 'rocksdb', data_dir = '/usr/local/xline/data-dir' } - - [compact] - - [trace] - jaeger_online = false - jaeger_offline = false - jaeger_output_dir = './jaeger_jsons' - jaeger_level = 'info' - - [auth] - - [tls] - ", - ) - .unwrap(); - - assert_eq!( - *config.cluster(), - ClusterConfig::new( - "node1".to_owned(), - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2380".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - vec!["127.0.0.1:2379".to_owned()], - HashMap::from([ - ("node1".to_owned(), vec!["127.0.0.1:2379".to_owned()]), - ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), - ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), - ]), - true, - CurpConfigBuilder::default().build().unwrap(), - ClientConfig::default(), - XlineServerTimeout::default(), - InitialClusterState::default() - ) - ); - - if let EngineConfig::RocksDB(path) = config.storage().engine().clone() { - assert_eq!(path, PathBuf::from("/usr/local/xline/data-dir")); - } else { - unreachable!(); - } - - assert_eq!( - *config.log(), - LogConfig::new( - Some(PathBuf::from("/var/log/xline")), - RotationConfig::Never, - LevelConfig::INFO - ) - ); - assert_eq!( - *config.trace(), - TraceConfig::new( - false, - false, - PathBuf::from("./jaeger_jsons"), - LevelConfig::INFO - ) - ); - assert_eq!(*config.compact(), CompactConfig::default()); - assert_eq!(*config.auth(), AuthConfig::default()); - assert_eq!(*config.tls(), TlsConfig::default()); - assert_eq!(*config.metrics(), MetricsConfig::default()); - } - - #[test] - fn test_auto_revision_compactor_config_should_be_loaded() { - let config: XlineServerConfig = toml::from_str( - "[cluster] - name = 'node1' - is_leader = true - peer_listen_urls = ['127.0.0.1:2380'] - peer_advertise_urls = ['127.0.0.1:2380'] - client_listen_urls = ['127.0.0.1:2379'] - client_advertise_urls = ['127.0.0.1:2379'] - - [cluster.peers] - node1 = ['127.0.0.1:2379'] - node2 = ['127.0.0.1:2380'] - node3 = ['127.0.0.1:2381'] - - [cluster.storage] - - [log] - path = '/var/log/xline' - - [storage] - engine = { type = 'memory' } - - [compact] - - [compact.auto_compact_config] - mode = 'revision' - retention = 10000 - - [trace] - jaeger_online = false - jaeger_offline = false - jaeger_output_dir = './jaeger_jsons' - jaeger_level = 'info' - - [auth] - - [tls] - ", - ) - .unwrap(); - - assert_eq!( - *config.compact(), - CompactConfig::new( - default_compact_batch_size(), - default_compact_sleep_interval(), - Some(AutoCompactConfig::Revision(10000)) - ) - ); - } -} diff --git a/crates/utils/src/config/server.rs b/crates/utils/src/config/server.rs index ee9b6ae28..54fc6f69e 100644 --- a/crates/utils/src/config/server.rs +++ b/crates/utils/src/config/server.rs @@ -124,3 +124,341 @@ impl Default for XlineServerTimeout { } } } + +#[cfg(test)] +mod tests { + use std::{collections::HashMap, path::PathBuf, time::Duration}; + + use crate::config::prelude::*; + + #[allow(clippy::too_many_lines)] // just a testcase, not too bad + #[test] + fn test_xline_server_config_should_be_loaded() { + let config: XlineServerConfig = toml::from_str( + r#"[cluster] + name = 'node1' + is_leader = true + initial_cluster_state = 'new' + peer_listen_urls = ['127.0.0.1:2380'] + peer_advertise_urls = ['127.0.0.1:2380'] + client_listen_urls = ['127.0.0.1:2379'] + client_advertise_urls = ['127.0.0.1:2379'] + + [cluster.server_timeout] + range_retry_timeout = '3s' + compact_timeout = '5s' + sync_victims_interval = '20ms' + watch_progress_notify_interval = '1s' + + [cluster.peers] + node1 = ['127.0.0.1:2378', '127.0.0.1:2379'] + node2 = ['127.0.0.1:2380'] + node3 = ['127.0.0.1:2381'] + + [cluster.curp_config] + heartbeat_interval = '200ms' + wait_synced_timeout = '100ms' + rpc_timeout = '100ms' + retry_timeout = '100ms' + + [cluster.client_config] + initial_retry_timeout = '5s' + max_retry_timeout = '50s' + + [storage] + engine = { type = 'memory'} + + [compact] + compact_batch_size = 123 + compact_sleep_interval = '5ms' + + [compact.auto_compact_config] + mode = 'periodic' + retention = '10h' + + [log] + path = '/var/log/xline' + rotation = 'daily' + level = 'info' + + [trace] + jaeger_online = false + jaeger_offline = false + jaeger_output_dir = './jaeger_jsons' + jaeger_level = 'info' + + [auth] + auth_public_key = './public_key.pem' + auth_private_key = './private_key.pem' + + [tls] + peer_cert_path = './cert.pem' + peer_key_path = './key.pem' + client_ca_cert_path = './ca.pem' + + [metrics] + enable = true + port = 9100 + path = "/metrics" + push = true + push_endpoint = 'http://some-endpoint.com:4396' + push_protocol = 'http' + "#, + ) + .unwrap(); + + let curp_config = CurpConfigBuilder::default() + .heartbeat_interval(Duration::from_millis(200)) + .wait_synced_timeout(Duration::from_millis(100)) + .rpc_timeout(Duration::from_millis(100)) + .build() + .unwrap(); + + let client_config = ClientConfig::new( + default_client_wait_synced_timeout(), + default_propose_timeout(), + Duration::from_secs(5), + Duration::from_secs(50), + default_retry_count(), + default_fixed_backoff(), + default_client_id_keep_alive_interval(), + ); + + let server_timeout = XlineServerTimeout::new( + Duration::from_secs(3), + Duration::from_secs(5), + Duration::from_millis(20), + Duration::from_secs(1), + ); + + assert_eq!( + *config.cluster(), + ClusterConfig::new( + "node1".to_owned(), + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + HashMap::from_iter([ + ( + "node1".to_owned(), + vec!["127.0.0.1:2378".to_owned(), "127.0.0.1:2379".to_owned()] + ), + ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), + ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), + ]), + true, + curp_config, + client_config, + server_timeout, + InitialClusterState::New + ) + ); + + assert_eq!( + *config.storage(), + StorageConfig::new(EngineConfig::Memory, default_quota()) + ); + + assert_eq!( + *config.log(), + LogConfig::new( + Some(PathBuf::from("/var/log/xline")), + RotationConfig::Daily, + LevelConfig::INFO + ) + ); + assert_eq!( + *config.trace(), + TraceConfig::new( + false, + false, + PathBuf::from("./jaeger_jsons"), + LevelConfig::INFO + ) + ); + + assert_eq!( + *config.compact(), + CompactConfig::new( + 123, + Duration::from_millis(5), + Some(AutoCompactConfig::Periodic(Duration::from_secs( + 10 * 60 * 60 + ))) + ) + ); + + assert_eq!( + *config.auth(), + AuthConfig::new( + Some(PathBuf::from("./public_key.pem")), + Some(PathBuf::from("./private_key.pem")) + ) + ); + + assert_eq!( + *config.tls(), + TlsConfig { + peer_cert_path: Some(PathBuf::from("./cert.pem")), + peer_key_path: Some(PathBuf::from("./key.pem")), + client_ca_cert_path: Some(PathBuf::from("./ca.pem")), + ..Default::default() + } + ); + + assert_eq!( + *config.metrics(), + MetricsConfig::new( + true, + 9100, + "/metrics".to_owned(), + true, + "http://some-endpoint.com:4396".to_owned(), + PushProtocol::HTTP + ), + ); + } + + #[test] + fn test_xline_server_default_config_should_be_loaded() { + let config: XlineServerConfig = toml::from_str( + "[cluster] + name = 'node1' + is_leader = true + peer_listen_urls = ['127.0.0.1:2380'] + peer_advertise_urls = ['127.0.0.1:2380'] + client_listen_urls = ['127.0.0.1:2379'] + client_advertise_urls = ['127.0.0.1:2379'] + + [cluster.peers] + node1 = ['127.0.0.1:2379'] + node2 = ['127.0.0.1:2380'] + node3 = ['127.0.0.1:2381'] + + [cluster.storage] + + [log] + path = '/var/log/xline' + + [storage] + engine = { type = 'rocksdb', data_dir = '/usr/local/xline/data-dir' } + + [compact] + + [trace] + jaeger_online = false + jaeger_offline = false + jaeger_output_dir = './jaeger_jsons' + jaeger_level = 'info' + + [auth] + + [tls] + ", + ) + .unwrap(); + + assert_eq!( + *config.cluster(), + ClusterConfig::new( + "node1".to_owned(), + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2380".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + vec!["127.0.0.1:2379".to_owned()], + HashMap::from([ + ("node1".to_owned(), vec!["127.0.0.1:2379".to_owned()]), + ("node2".to_owned(), vec!["127.0.0.1:2380".to_owned()]), + ("node3".to_owned(), vec!["127.0.0.1:2381".to_owned()]), + ]), + true, + CurpConfigBuilder::default().build().unwrap(), + ClientConfig::default(), + XlineServerTimeout::default(), + InitialClusterState::default() + ) + ); + + if let EngineConfig::RocksDB(path) = config.storage().engine().clone() { + assert_eq!(path, PathBuf::from("/usr/local/xline/data-dir")); + } else { + unreachable!(); + } + + assert_eq!( + *config.log(), + LogConfig::new( + Some(PathBuf::from("/var/log/xline")), + RotationConfig::Never, + LevelConfig::INFO + ) + ); + assert_eq!( + *config.trace(), + TraceConfig::new( + false, + false, + PathBuf::from("./jaeger_jsons"), + LevelConfig::INFO + ) + ); + assert_eq!(*config.compact(), CompactConfig::default()); + assert_eq!(*config.auth(), AuthConfig::default()); + assert_eq!(*config.tls(), TlsConfig::default()); + assert_eq!(*config.metrics(), MetricsConfig::default()); + } + + #[test] + fn test_auto_revision_compactor_config_should_be_loaded() { + let config: XlineServerConfig = toml::from_str( + "[cluster] + name = 'node1' + is_leader = true + peer_listen_urls = ['127.0.0.1:2380'] + peer_advertise_urls = ['127.0.0.1:2380'] + client_listen_urls = ['127.0.0.1:2379'] + client_advertise_urls = ['127.0.0.1:2379'] + + [cluster.peers] + node1 = ['127.0.0.1:2379'] + node2 = ['127.0.0.1:2380'] + node3 = ['127.0.0.1:2381'] + + [cluster.storage] + + [log] + path = '/var/log/xline' + + [storage] + engine = { type = 'memory' } + + [compact] + + [compact.auto_compact_config] + mode = 'revision' + retention = 10000 + + [trace] + jaeger_online = false + jaeger_offline = false + jaeger_output_dir = './jaeger_jsons' + jaeger_level = 'info' + + [auth] + + [tls] + ", + ) + .unwrap(); + + assert_eq!( + *config.compact(), + CompactConfig::new( + default_compact_batch_size(), + default_compact_sleep_interval(), + Some(AutoCompactConfig::Revision(10000)) + ) + ); + } +} diff --git a/crates/utils/src/config/storage.rs b/crates/utils/src/config/storage.rs index d2721e7fb..d0ebf45b7 100644 --- a/crates/utils/src/config/storage.rs +++ b/crates/utils/src/config/storage.rs @@ -3,7 +3,7 @@ use serde::Deserialize; use super::prelude::EngineConfig; -/// /// Storage Configuration +/// Storage Configuration #[derive(Debug, Clone, Deserialize, PartialEq, Eq, Getters)] #[allow(clippy::module_name_repetitions)] #[non_exhaustive] From 614d6bcdd5125b7ba9bef03bfc742d23952ccc35 Mon Sep 17 00:00:00 2001 From: bsbds <69835502+bsbds@users.noreply.github.com> Date: Wed, 28 Aug 2024 20:03:01 +0800 Subject: [PATCH 4/9] feat: add auto reconnect implementation for curp client This PR add the auto reconnect implementation for curp client, as a workaround for https://github.com/hyperium/tonic/issues/1254. Signed-off-by: bsbds <69835502+bsbds@users.noreply.github.com> --- crates/curp/src/client/mod.rs | 68 ++---- crates/curp/src/client/retry.rs | 8 +- crates/curp/src/client/state.rs | 55 +++-- crates/curp/src/client/tests.rs | 4 +- crates/curp/src/client/unary/mod.rs | 15 +- crates/curp/src/members.rs | 2 - crates/curp/src/rpc/connect.rs | 101 +++++---- crates/curp/src/rpc/mod.rs | 3 + crates/curp/src/rpc/reconnect.rs | 198 ++++++++++++++++++ crates/curp/src/server/curp_node.rs | 21 +- crates/curp/src/server/mod.rs | 9 +- crates/curp/tests/it/common/curp_group.rs | 59 +++--- crates/curp/tests/it/server.rs | 2 - crates/simulation/src/curp_group.rs | 2 - .../tests/it/curp/server_recovery.rs | 13 +- crates/xline-client/src/lib.rs | 3 +- crates/xline/src/server/xline_server.rs | 6 +- crates/xline/tests/it/lock_test.rs | 9 +- 18 files changed, 371 insertions(+), 207 deletions(-) create mode 100644 crates/curp/src/rpc/reconnect.rs diff --git a/crates/curp/src/client/mod.rs b/crates/curp/src/client/mod.rs index 4c93decaf..7cb754135 100644 --- a/crates/curp/src/client/mod.rs +++ b/crates/curp/src/client/mod.rs @@ -163,7 +163,7 @@ impl Drop for ProposeIdGuard<'_> { #[async_trait] trait RepeatableClientApi: ClientApi { /// Generate a unique propose id during the retry process. - fn gen_propose_id(&self) -> Result, Self::Error>; + async fn gen_propose_id(&self) -> Result, Self::Error>; /// Send propose to the whole cluster, `use_fast_path` set to `false` to fallback into ordered /// requests (event the requests are commutative). @@ -422,51 +422,23 @@ impl ClientBuilder { }) } - /// Wait for client id - async fn wait_for_client_id(&self, state: Arc) -> Result<(), tonic::Status> { - /// Max retry count for waiting for a client ID - /// - /// TODO: This retry count is set relatively high to avoid test cluster startup timeouts. - /// We should consider setting this to a more reasonable value. - const RETRY_COUNT: usize = 30; - /// The interval for each retry - const RETRY_INTERVAL: Duration = Duration::from_secs(1); - - for _ in 0..RETRY_COUNT { - if state.client_id() != 0 { - return Ok(()); - } - debug!("waiting for client_id"); - tokio::time::sleep(RETRY_INTERVAL).await; - } - - Err(tonic::Status::deadline_exceeded( - "timeout waiting for client id", - )) - } - /// Build the client /// /// # Errors /// /// Return `tonic::transport::Error` for connection failure. #[inline] - pub async fn build( + pub fn build( &self, ) -> Result + Send + Sync + 'static, tonic::Status> { - let state = Arc::new( - self.init_state_builder() - .build() - .await - .map_err(|e| tonic::Status::internal(e.to_string()))?, - ); + let state = Arc::new(self.init_state_builder().build()); let client = Retry::new( Unary::new(Arc::clone(&state), self.init_unary_config()), self.init_retry_config(), Some(self.spawn_bg_tasks(Arc::clone(&state))), ); - self.wait_for_client_id(state).await?; + Ok(client) } @@ -477,21 +449,14 @@ impl ClientBuilder { /// /// Return `tonic::transport::Error` for connection failure. #[inline] - pub async fn build_with_client_id( + #[must_use] + pub fn build_with_client_id( &self, - ) -> Result< - ( - impl ClientApi + Send + Sync + 'static, - Arc, - ), - tonic::Status, - > { - let state = Arc::new( - self.init_state_builder() - .build() - .await - .map_err(|e| tonic::Status::internal(e.to_string()))?, - ); + ) -> ( + impl ClientApi + Send + Sync + 'static, + Arc, + ) { + let state = Arc::new(self.init_state_builder().build()); let client = Retry::new( Unary::new(Arc::clone(&state), self.init_unary_config()), @@ -499,9 +464,8 @@ impl ClientBuilder { Some(self.spawn_bg_tasks(Arc::clone(&state))), ); let client_id = state.clone_client_id(); - self.wait_for_client_id(state).await?; - Ok((client, client_id)) + (client, client_id) } } @@ -512,22 +476,20 @@ impl ClientBuilderWithBypass

{ /// /// Return `tonic::transport::Error` for connection failure. #[inline] - pub async fn build( + pub fn build( self, ) -> Result, tonic::Status> { let state = self .inner .init_state_builder() - .build_bypassed::

(self.local_server_id, self.local_server) - .await - .map_err(|e| tonic::Status::internal(e.to_string()))?; + .build_bypassed::

(self.local_server_id, self.local_server); let state = Arc::new(state); let client = Retry::new( Unary::new(Arc::clone(&state), self.inner.init_unary_config()), self.inner.init_retry_config(), Some(self.inner.spawn_bg_tasks(Arc::clone(&state))), ); - self.inner.wait_for_client_id(state).await?; + Ok(client) } } diff --git a/crates/curp/src/client/retry.rs b/crates/curp/src/client/retry.rs index 06e670a89..c67db6019 100644 --- a/crates/curp/src/client/retry.rs +++ b/crates/curp/src/client/retry.rs @@ -231,7 +231,7 @@ where use_fast_path: bool, ) -> Result, tonic::Status> { self.retry::<_, _>(|client| async move { - let propose_id = self.inner.gen_propose_id()?; + let propose_id = self.inner.gen_propose_id().await?; RepeatableClientApi::propose(client, *propose_id, cmd, token, use_fast_path).await }) .await @@ -245,7 +245,7 @@ where self.retry::<_, _>(|client| { let changes_c = changes.clone(); async move { - let propose_id = self.inner.gen_propose_id()?; + let propose_id = self.inner.gen_propose_id().await?; RepeatableClientApi::propose_conf_change(client, *propose_id, changes_c).await } }) @@ -255,7 +255,7 @@ where /// Send propose to shutdown cluster async fn propose_shutdown(&self) -> Result<(), tonic::Status> { self.retry::<_, _>(|client| async move { - let propose_id = self.inner.gen_propose_id()?; + let propose_id = self.inner.gen_propose_id().await?; RepeatableClientApi::propose_shutdown(client, *propose_id).await }) .await @@ -272,7 +272,7 @@ where let name_c = node_name.clone(); let node_client_urls_c = node_client_urls.clone(); async move { - let propose_id = self.inner.gen_propose_id()?; + let propose_id = self.inner.gen_propose_id().await?; RepeatableClientApi::propose_publish( client, *propose_id, diff --git a/crates/curp/src/client/state.rs b/crates/curp/src/client/state.rs index 8476e46b8..8a9b53081 100644 --- a/crates/curp/src/client/state.rs +++ b/crates/curp/src/client/state.rs @@ -95,7 +95,8 @@ impl State { tls_config, is_raw_curp: true, }, - client_id: Arc::new(AtomicU64::new(0)), + // Sets the client id to non-zero to avoid waiting for client id in tests + client_id: Arc::new(AtomicU64::new(1)), }) } @@ -146,8 +147,8 @@ impl State { }; let resp = rand_conn .fetch_cluster(FetchClusterRequest::default(), REFRESH_TIMEOUT) - .await?; - self.check_and_update(&resp.into_inner()).await?; + .await; + self.check_and_update(&resp?.into_inner()).await?; Ok(()) } @@ -327,7 +328,7 @@ impl State { .remove(&diff) .unwrap_or_else(|| unreachable!("{diff} must in new member addrs")); debug!("client connects to a new server({diff}), address({addrs:?})"); - let new_conn = rpc::connect(diff, addrs, self.immutable.tls_config.clone()).await?; + let new_conn = rpc::connect(diff, addrs, self.immutable.tls_config.clone()); let _ig = e.insert(new_conn); } else { debug!("client removes old server({diff})"); @@ -347,6 +348,30 @@ impl State { Ok(()) } + + /// Wait for client id + pub(super) async fn wait_for_client_id(&self) -> Result { + /// Max retry count for waiting for a client ID + /// + /// TODO: This retry count is set relatively high to avoid test cluster startup timeouts. + /// We should consider setting this to a more reasonable value. + const RETRY_COUNT: usize = 30; + /// The interval for each retry + const RETRY_INTERVAL: Duration = Duration::from_secs(1); + + for _ in 0..RETRY_COUNT { + let client_id = self.client_id(); + if client_id != 0 { + return Ok(client_id); + } + debug!("waiting for client_id"); + tokio::time::sleep(RETRY_INTERVAL).await; + } + + Err(tonic::Status::deadline_exceeded( + "timeout waiting for client id", + )) + } } /// Builder for state @@ -395,24 +420,22 @@ impl StateBuilder { } /// Build the state with local server - pub(super) async fn build_bypassed( + pub(super) fn build_bypassed( mut self, local_server_id: ServerId, local_server: P, - ) -> Result { + ) -> State { debug!("client bypassed server({local_server_id})"); let _ig = self.all_members.remove(&local_server_id); let mut connects: HashMap<_, _> = - rpc::connects(self.all_members.clone(), self.tls_config.as_ref()) - .await? - .collect(); + rpc::connects(self.all_members.clone(), self.tls_config.as_ref()).collect(); let __ig = connects.insert( local_server_id, Arc::new(BypassedConnect::new(local_server_id, local_server)), ); - Ok(State { + State { mutable: RwLock::new(StateMut { leader: self.leader_state.map(|state| state.0), term: self.leader_state.map_or(0, |state| state.1), @@ -426,16 +449,14 @@ impl StateBuilder { is_raw_curp: self.is_raw_curp, }, client_id: Arc::new(AtomicU64::new(0)), - }) + } } /// Build the state - pub(super) async fn build(self) -> Result { + pub(super) fn build(self) -> State { let connects: HashMap<_, _> = - rpc::connects(self.all_members.clone(), self.tls_config.as_ref()) - .await? - .collect(); - Ok(State { + rpc::connects(self.all_members.clone(), self.tls_config.as_ref()).collect(); + State { mutable: RwLock::new(StateMut { leader: self.leader_state.map(|state| state.0), term: self.leader_state.map_or(0, |state| state.1), @@ -449,6 +470,6 @@ impl StateBuilder { is_raw_curp: self.is_raw_curp, }, client_id: Arc::new(AtomicU64::new(0)), - }) + } } } diff --git a/crates/curp/src/client/tests.rs b/crates/curp/src/client/tests.rs index e97fce4ce..39c8b88bc 100644 --- a/crates/curp/src/client/tests.rs +++ b/crates/curp/src/client/tests.rs @@ -751,7 +751,7 @@ async fn test_stream_client_keep_alive_works() { Box::pin(async move { client_id .compare_exchange( - 0, + 1, 10, std::sync::atomic::Ordering::Relaxed, std::sync::atomic::Ordering::Relaxed, @@ -775,7 +775,7 @@ async fn test_stream_client_keep_alive_on_redirect() { Box::pin(async move { client_id .compare_exchange( - 0, + 1, 10, std::sync::atomic::Ordering::Relaxed, std::sync::atomic::Ordering::Relaxed, diff --git a/crates/curp/src/client/unary/mod.rs b/crates/curp/src/client/unary/mod.rs index 8219ec04b..90986bdb7 100644 --- a/crates/curp/src/client/unary/mod.rs +++ b/crates/curp/src/client/unary/mod.rs @@ -134,7 +134,7 @@ impl ClientApi for Unary { token: Option<&String>, use_fast_path: bool, ) -> Result, CurpError> { - let propose_id = self.gen_propose_id()?; + let propose_id = self.gen_propose_id().await?; RepeatableClientApi::propose(self, *propose_id, cmd, token, use_fast_path).await } @@ -143,13 +143,13 @@ impl ClientApi for Unary { &self, changes: Vec, ) -> Result, CurpError> { - let propose_id = self.gen_propose_id()?; + let propose_id = self.gen_propose_id().await?; RepeatableClientApi::propose_conf_change(self, *propose_id, changes).await } /// Send propose to shutdown cluster async fn propose_shutdown(&self) -> Result<(), CurpError> { - let propose_id = self.gen_propose_id()?; + let propose_id = self.gen_propose_id().await?; RepeatableClientApi::propose_shutdown(self, *propose_id).await } @@ -160,7 +160,7 @@ impl ClientApi for Unary { node_name: String, node_client_urls: Vec, ) -> Result<(), Self::Error> { - let propose_id = self.gen_propose_id()?; + let propose_id = self.gen_propose_id().await?; RepeatableClientApi::propose_publish( self, *propose_id, @@ -306,8 +306,11 @@ impl ClientApi for Unary { #[async_trait] impl RepeatableClientApi for Unary { /// Generate a unique propose id during the retry process. - fn gen_propose_id(&self) -> Result, Self::Error> { - let client_id = self.state.client_id(); + async fn gen_propose_id(&self) -> Result, Self::Error> { + let mut client_id = self.state.client_id(); + if client_id == 0 { + client_id = self.state.wait_for_client_id().await?; + }; let seq_num = self.new_seq_num(); Ok(ProposeIdGuard::new( &self.tracker, diff --git a/crates/curp/src/members.rs b/crates/curp/src/members.rs index ce2045451..5682268f1 100644 --- a/crates/curp/src/members.rs +++ b/crates/curp/src/members.rs @@ -439,8 +439,6 @@ pub async fn get_cluster_info_from_remote( let peers = init_cluster_info.peers_addrs(); let self_client_urls = init_cluster_info.self_client_urls(); let connects = rpc::connects(peers, tls_config) - .await - .ok()? .map(|pair| pair.1) .collect_vec(); let mut futs = connects diff --git a/crates/curp/src/rpc/connect.rs b/crates/curp/src/rpc/connect.rs index d438b6c28..c62b37d31 100644 --- a/crates/curp/src/rpc/connect.rs +++ b/crates/curp/src/rpc/connect.rs @@ -11,7 +11,7 @@ use async_trait::async_trait; use bytes::BytesMut; use clippy_utilities::NumericCast; use engine::SnapshotApi; -use futures::{stream::FuturesUnordered, Stream}; +use futures::Stream; #[cfg(test)] use mockall::automock; use tokio::sync::Mutex; @@ -42,6 +42,7 @@ use crate::{ use super::{ proto::commandpb::{ReadIndexRequest, ReadIndexResponse}, + reconnect::Reconnect, OpResponse, RecordRequest, RecordResponse, }; @@ -69,85 +70,79 @@ impl FromTonicChannel for InnerProtocolClient { } } -/// Connect to a server -async fn connect_to( +/// Creates a new connection +fn connect_to( id: ServerId, addrs: Vec, tls_config: Option, -) -> Result>, tonic::transport::Error> { - let (channel, change_tx) = Channel::balance_channel(DEFAULT_BUFFER_SIZE); +) -> Connect { + let (channel, change_tx) = Channel::balance_channel(DEFAULT_BUFFER_SIZE.max(addrs.len())); for addr in &addrs { - let endpoint = build_endpoint(addr, tls_config.as_ref())?; - let _ig = change_tx - .send(tower::discover::Change::Insert(addr.clone(), endpoint)) - .await; + let endpoint = build_endpoint(addr, tls_config.as_ref()) + .unwrap_or_else(|_| unreachable!("address is ill-formatted")); + change_tx + .try_send(tower::discover::Change::Insert(addr.clone(), endpoint)) + .unwrap_or_else(|_| unreachable!("unknown channel tx send error")); } let client = Client::from_channel(channel); - let connect = Arc::new(Connect { + Connect { id, rpc_connect: client, change_tx, addrs: Mutex::new(addrs), tls_config, - }); - Ok(connect) + } } -/// Connect to a map of members -async fn connect_all( - members: HashMap>, - tls_config: Option<&ClientTlsConfig>, -) -> Result>)>, tonic::transport::Error> { - let conns_to: FuturesUnordered<_> = members - .into_iter() - .map(|(id, addrs)| async move { - connect_to::(id, addrs, tls_config.cloned()) - .await - .map(|conn| (id, conn)) - }) - .collect(); - futures::StreamExt::collect::>(conns_to) - .await - .into_iter() - .collect::, _>>() +/// Creates a new connection with auto reconnect +fn new_reconnect( + id: ServerId, + addrs: Vec, + tls_config: Option, +) -> Reconnect>> { + Reconnect::new(Box::new(move || { + connect_to(id, addrs.clone(), tls_config.clone()) + })) } /// A wrapper of [`connect_to`], hide the detailed [`Connect`] -pub(crate) async fn connect( +pub(crate) fn connect( id: ServerId, addrs: Vec, tls_config: Option, -) -> Result, tonic::transport::Error> { - let conn = connect_to::>(id, addrs, tls_config).await?; - Ok(conn) +) -> Arc { + let conn = new_reconnect(id, addrs, tls_config); + Arc::new(conn) } /// Wrapper of [`connect_all`], hide the details of [`Connect`] -pub(crate) async fn connects( +pub(crate) fn connects( members: HashMap>, tls_config: Option<&ClientTlsConfig>, -) -> Result)>, tonic::transport::Error> { - // It seems that casting high-rank types cannot be inferred, so we allow trivial_casts to cast manually - #[allow(trivial_casts)] - #[allow(clippy::as_conversions)] - let conns = connect_all(members, tls_config) - .await? +) -> impl Iterator)> { + let tls_config = tls_config.cloned(); + members .into_iter() - .map(|(id, conn)| (id, conn as Arc)); - Ok(conns) + .map(move |(id, addrs)| (id, connect(id, addrs, tls_config.clone()))) } /// Wrapper of [`connect_all`], hide the details of [`Connect`] -pub(crate) async fn inner_connects( +pub(crate) fn inner_connects( members: HashMap>, tls_config: Option<&ClientTlsConfig>, -) -> Result, tonic::transport::Error> { - let conns = connect_all(members, tls_config) - .await? - .into_iter() - .map(|(id, conn)| (id, InnerConnectApiWrapper::new_from_arc(conn))); - Ok(conns) +) -> impl Iterator { + let tls_config = tls_config.cloned(); + members.into_iter().map(move |(id, addrs)| { + ( + id, + InnerConnectApiWrapper::new_from_arc(Arc::new(connect_to::< + InnerProtocolClient, + >( + id, addrs, tls_config.clone() + ))), + ) + }) } /// Connect interface between server and clients @@ -282,13 +277,13 @@ impl InnerConnectApiWrapper { } /// Create a new `InnerConnectApiWrapper` from id and addrs - pub(crate) async fn connect( + pub(crate) fn connect( id: ServerId, addrs: Vec, tls_config: Option, - ) -> Result { - let conn = connect_to::>(id, addrs, tls_config).await?; - Ok(InnerConnectApiWrapper::new_from_arc(conn)) + ) -> Self { + let conn = connect_to::>(id, addrs, tls_config); + InnerConnectApiWrapper::new_from_arc(Arc::new(conn)) } } diff --git a/crates/curp/src/rpc/mod.rs b/crates/curp/src/rpc/mod.rs index c064c3bb0..10c56fa99 100644 --- a/crates/curp/src/rpc/mod.rs +++ b/crates/curp/src/rpc/mod.rs @@ -67,6 +67,9 @@ mod metrics; pub(crate) mod connect; pub(crate) use connect::{connect, connects, inner_connects}; +/// Auto reconnect connection +mod reconnect; + // Skip for generated code #[allow( clippy::all, diff --git a/crates/curp/src/rpc/reconnect.rs b/crates/curp/src/rpc/reconnect.rs new file mode 100644 index 000000000..e392db38a --- /dev/null +++ b/crates/curp/src/rpc/reconnect.rs @@ -0,0 +1,198 @@ +use std::{ + sync::{atomic::AtomicU64, Arc}, + time::Duration, +}; + +use async_trait::async_trait; +use event_listener::Event; +use futures::Stream; + +use crate::{ + members::ServerId, + rpc::{ + connect::ConnectApi, CurpError, FetchClusterRequest, FetchClusterResponse, + FetchReadStateRequest, FetchReadStateResponse, MoveLeaderRequest, MoveLeaderResponse, + OpResponse, ProposeConfChangeRequest, ProposeConfChangeResponse, ProposeRequest, + PublishRequest, PublishResponse, ReadIndexResponse, RecordRequest, RecordResponse, + ShutdownRequest, ShutdownResponse, + }, +}; + +/// Auto reconnect of a connection +pub(super) struct Reconnect { + /// Connect id + id: ServerId, + /// The connection + connect: tokio::sync::RwLock>, + /// The connect builder + builder: Box C + Send + Sync + 'static>, + /// Signal to abort heartbeat + event: Event, +} + +impl Reconnect { + /// Creates a new `Reconnect` + pub(crate) fn new(builder: Box C + Send + Sync + 'static>) -> Self { + let init_connect = builder(); + Self { + id: init_connect.id(), + connect: tokio::sync::RwLock::new(Some(init_connect)), + builder, + event: Event::new(), + } + } + + /// Creating a new connection to replace the current + async fn reconnect(&self) { + let new_connect = (self.builder)(); + // Cancel the leader keep alive loop task because it hold a read lock + let _cancel = self.event.notify(1); + let _ignore = self.connect.write().await.replace(new_connect); + // After connection is updated, notify to start the keep alive loop + let _continue = self.event.notify(1); + } + + /// Try to reconnect if the result is `Err` + async fn try_reconnect(&self, result: Result) -> Result { + // TODO: use `tonic::Status` instead of `CurpError`, we can't tell + // if a reconnect is required from `CurpError`. + if matches!( + result, + Err(CurpError::RpcTransport(()) | CurpError::Internal(_)) + ) { + tracing::info!("client reconnecting"); + self.reconnect().await; + } + result + } +} + +/// Execute with reconnect +macro_rules! execute_with_reconnect { + ($self:expr, $trait_method:path, $($arg:expr),*) => {{ + let result = { + let connect = $self.connect.read().await; + let connect_ref = connect.as_ref().unwrap(); + ($trait_method)(connect_ref, $($arg),*).await + }; + $self.try_reconnect(result).await + }}; +} + +#[allow(clippy::unwrap_used, clippy::unwrap_in_result)] +#[async_trait] +impl ConnectApi for Reconnect { + /// Get server id + fn id(&self) -> ServerId { + self.id + } + + /// Update server addresses, the new addresses will override the old ones + async fn update_addrs(&self, addrs: Vec) -> Result<(), tonic::transport::Error> { + let connect = self.connect.read().await; + connect.as_ref().unwrap().update_addrs(addrs).await + } + + /// Send `ProposeRequest` + async fn propose_stream( + &self, + request: ProposeRequest, + token: Option, + timeout: Duration, + ) -> Result< + tonic::Response> + Send>>, + CurpError, + > { + execute_with_reconnect!(self, ConnectApi::propose_stream, request, token, timeout) + } + + /// Send `RecordRequest` + async fn record( + &self, + request: RecordRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::record, request, timeout) + } + + /// Send `ReadIndexRequest` + async fn read_index( + &self, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::read_index, timeout) + } + + /// Send `ProposeRequest` + async fn propose_conf_change( + &self, + request: ProposeConfChangeRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::propose_conf_change, request, timeout) + } + + /// Send `PublishRequest` + async fn publish( + &self, + request: PublishRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::publish, request, timeout) + } + + /// Send `ShutdownRequest` + async fn shutdown( + &self, + request: ShutdownRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::shutdown, request, timeout) + } + + /// Send `FetchClusterRequest` + async fn fetch_cluster( + &self, + request: FetchClusterRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::fetch_cluster, request, timeout) + } + + /// Send `FetchReadStateRequest` + async fn fetch_read_state( + &self, + request: FetchReadStateRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::fetch_read_state, request, timeout) + } + + /// Send `MoveLeaderRequest` + async fn move_leader( + &self, + request: MoveLeaderRequest, + timeout: Duration, + ) -> Result, CurpError> { + execute_with_reconnect!(self, ConnectApi::move_leader, request, timeout) + } + + /// Keep send lease keep alive to server and mutate the client id + async fn lease_keep_alive(&self, client_id: Arc, interval: Duration) -> CurpError { + loop { + let connect = self.connect.read().await; + let connect_ref = connect.as_ref().unwrap(); + tokio::select! { + err = connect_ref.lease_keep_alive(Arc::clone(&client_id), interval) => { + return err; + } + _empty = self.event.listen() => {}, + } + // Creates the listener before dropping the read lock. + // This prevents us from losting the event. + let listener = self.event.listen(); + drop(connect); + let _connection_updated = listener.await; + } + } +} diff --git a/crates/curp/src/server/curp_node.rs b/crates/curp/src/server/curp_node.rs index f2ce80c16..18ff5b4e7 100644 --- a/crates/curp/src/server/curp_node.rs +++ b/crates/curp/src/server/curp_node.rs @@ -690,19 +690,11 @@ impl, RC: RoleChange> CurpNode { }; match change.change_type() { ConfChangeType::Add | ConfChangeType::AddLearner => { - let connect = match InnerConnectApiWrapper::connect( + let connect = InnerConnectApiWrapper::connect( change.node_id, change.address, curp.client_tls_config().cloned(), - ) - .await - { - Ok(connect) => connect, - Err(e) => { - error!("connect to {} failed, {}", change.node_id, e); - continue; - } - }; + ); curp.insert_connect(connect.clone()); let sync_event = curp.sync_event(change.node_id); let remove_event = Arc::new(Event::new()); @@ -842,7 +834,8 @@ impl, RC: RoleChange> CurpNode { /// Create a new server instance #[inline] #[allow(clippy::too_many_arguments)] // TODO: refactor this use builder pattern - pub(super) async fn new( + #[allow(clippy::needless_pass_by_value)] // The value should be consumed + pub(super) fn new( cluster_info: Arc, is_leader: bool, cmd_executor: Arc, @@ -860,10 +853,8 @@ impl, RC: RoleChange> CurpNode { .into_iter() .map(|server_id| (server_id, Arc::new(Event::new()))) .collect(); - let connects = rpc::inner_connects(cluster_info.peers_addrs(), client_tls_config.as_ref()) - .await - .map_err(|e| CurpError::internal(format!("parse peers addresses failed, err {e:?}")))? - .collect(); + let connects = + rpc::inner_connects(cluster_info.peers_addrs(), client_tls_config.as_ref()).collect(); let cmd_board = Arc::new(RwLock::new(CommandBoard::new())); let lease_manager = Arc::new(RwLock::new(LeaseManager::new())); let last_applied = cmd_executor diff --git a/crates/curp/src/server/mod.rs b/crates/curp/src/server/mod.rs index 2a10c8925..b4e012f62 100644 --- a/crates/curp/src/server/mod.rs +++ b/crates/curp/src/server/mod.rs @@ -263,7 +263,7 @@ impl, RC: RoleChange> Rpc { /// Panic if storage creation failed #[inline] #[allow(clippy::too_many_arguments)] // TODO: refactor this use builder pattern - pub async fn new( + pub fn new( cluster_info: Arc, is_leader: bool, executor: Arc, @@ -289,9 +289,7 @@ impl, RC: RoleChange> Rpc { client_tls_config, sps, ucps, - ) - .await - { + ) { Ok(n) => n, Err(err) => { panic!("failed to create curp service, {err:?}"); @@ -345,8 +343,7 @@ impl, RC: RoleChange> Rpc { client_tls_config, sps, ucps, - ) - .await; + ); tonic::transport::Server::builder() .add_service(ProtocolServer::new(server.clone())) diff --git a/crates/curp/tests/it/common/curp_group.rs b/crates/curp/tests/it/common/curp_group.rs index e7ee921ec..45a511735 100644 --- a/crates/curp/tests/it/common/curp_group.rs +++ b/crates/curp/tests/it/common/curp_group.rs @@ -136,22 +136,19 @@ impl CurpGroup { let role_change_cb = TestRoleChange::default(); let role_change_arc = role_change_cb.get_inner_arc(); let curp_storage = Arc::new(DB::open(&config.engine_cfg).unwrap()); - let server = Arc::new( - Rpc::new( - cluster_info, - name == leader_name, - ce, - snapshot_allocator, - role_change_cb, - config, - curp_storage, - Arc::clone(&task_manager), - client_tls_config.clone(), - vec![Box::::default()], - vec![Box::::default()], - ) - .await, - ); + let server = Arc::new(Rpc::new( + cluster_info, + name == leader_name, + ce, + snapshot_allocator, + role_change_cb, + config, + curp_storage, + Arc::clone(&task_manager), + client_tls_config.clone(), + vec![Box::::default()], + vec![Box::::default()], + )); task_manager.spawn(TaskName::TonicServer, |n| async move { let ig = Self::run(server, listener, n).await; }); @@ -268,22 +265,19 @@ impl CurpGroup { let role_change_cb = TestRoleChange::default(); let role_change_arc = role_change_cb.get_inner_arc(); let curp_storage = Arc::new(DB::open(&config.engine_cfg).unwrap()); - let server = Arc::new( - Rpc::new( - cluster_info, - false, - ce, - snapshot_allocator, - role_change_cb, - config, - curp_storage, - Arc::clone(&task_manager), - self.client_tls_config.clone(), - vec![], - vec![], - ) - .await, - ); + let server = Arc::new(Rpc::new( + cluster_info, + false, + ce, + snapshot_allocator, + role_change_cb, + config, + curp_storage, + Arc::clone(&task_manager), + self.client_tls_config.clone(), + vec![], + vec![], + )); task_manager.spawn(TaskName::TonicServer, |n| async move { let _ig = Self::run(server, listener, n).await; }); @@ -329,7 +323,6 @@ impl CurpGroup { .await .unwrap() .build() - .await .unwrap() } diff --git a/crates/curp/tests/it/server.rs b/crates/curp/tests/it/server.rs index 4d00cc1ce..bbb869186 100644 --- a/crates/curp/tests/it/server.rs +++ b/crates/curp/tests/it/server.rs @@ -455,7 +455,6 @@ async fn shutdown_rpc_should_shutdown_the_cluster_when_client_has_wrong_leader() .leader_state(follower_id, 0) .all_members(group.all_addrs_map()) .build::() - .await .unwrap(); client.propose_shutdown().await.unwrap(); @@ -477,7 +476,6 @@ async fn propose_conf_change_to_follower() { .leader_state(follower_id, 0) .all_members(group.all_addrs_map()) .build::() - .await .unwrap(); let node_id = group.nodes.keys().next().copied().unwrap(); diff --git a/crates/simulation/src/curp_group.rs b/crates/simulation/src/curp_group.rs index 1d87524f3..8c8b8dff8 100644 --- a/crates/simulation/src/curp_group.rs +++ b/crates/simulation/src/curp_group.rs @@ -195,10 +195,8 @@ impl CurpGroup { ClientBuilder::new(config, true) .all_members(all_members) .build_with_client_id() - .await }) .await - .unwrap() .unwrap(); SimClient { inner: Arc::new(client), diff --git a/crates/simulation/tests/it/curp/server_recovery.rs b/crates/simulation/tests/it/curp/server_recovery.rs index 7e8a88ccf..3e8c85125 100644 --- a/crates/simulation/tests/it/curp/server_recovery.rs +++ b/crates/simulation/tests/it/curp/server_recovery.rs @@ -457,6 +457,13 @@ async fn recovery_after_compaction() { async fn overwritten_config_should_fallback() { init_logger(); let group = CurpGroup::new(5).await; + let client = group.new_client().await; + // A workaround for dedup. The client will lazily acquire an id from the leader during a + // propose. + let _wait_for_client_id = client + .propose(TestCommand::new_put(vec![0], 0), false) + .await; + let client_id = client.client_id(); let leader1 = group.get_leader().await.0; for node in group.nodes.values().filter(|node| node.id != leader1) { group.disable_node(node.id); @@ -468,13 +475,13 @@ async fn overwritten_config_should_fallback() { let node_id = 123; let address = vec!["127.0.0.1:4567".to_owned()]; let changes = vec![ConfChange::add(node_id, address)]; - let client = group.new_client().await; let res = leader_conn .propose_conf_change( ProposeConfChangeRequest { propose_id: Some(PbProposeId { - client_id: client.client_id(), - seq_num: 0, + client_id, + // start from 1 as we already propose an put with seq_num = 0 + seq_num: 1, }), changes, cluster_version: cluster.cluster_version, diff --git a/crates/xline-client/src/lib.rs b/crates/xline-client/src/lib.rs index 77e42cb78..fc8d1f1da 100644 --- a/crates/xline-client/src/lib.rs +++ b/crates/xline-client/src/lib.rs @@ -244,8 +244,7 @@ impl Client { .tls_config(options.tls_config) .discover_from(addrs) .await? - .build::() - .await?, + .build::()?, ) as Arc; let id_gen = Arc::new(lease_gen::LeaseIdGenerator::new()); diff --git a/crates/xline/src/server/xline_server.rs b/crates/xline/src/server/xline_server.rs index b940c8489..20761c575 100644 --- a/crates/xline/src/server/xline_server.rs +++ b/crates/xline/src/server/xline_server.rs @@ -518,8 +518,7 @@ impl XlineServer { self.client_tls_config.clone(), XlineSpeculativePools::new(Arc::clone(&lease_collection)).into_inner(), XlineUncommittedPools::new(lease_collection).into_inner(), - ) - .await; + ); let client = Arc::new( CurpClientBuilder::new(*self.cluster_config.client_config(), false) @@ -527,8 +526,7 @@ impl XlineServer { .cluster_version(self.cluster_info.cluster_version()) .all_members(self.cluster_info.all_members_peer_urls()) .bypass(self.cluster_info.self_id(), curp_server.clone()) - .build::() - .await?, + .build::()?, ) as Arc; if let Some(compactor) = auto_compactor_c { diff --git a/crates/xline/tests/it/lock_test.rs b/crates/xline/tests/it/lock_test.rs index d89231f03..29dc9a19b 100644 --- a/crates/xline/tests/it/lock_test.rs +++ b/crates/xline/tests/it/lock_test.rs @@ -1,4 +1,4 @@ -use std::{error::Error, time::Duration}; +use std::{error::Error, sync::Arc, time::Duration}; use test_macros::abort_on_panic; use tokio::time::{sleep, Instant}; @@ -11,17 +11,20 @@ async fn test_lock() -> Result<(), Box> { cluster.start().await; let client = cluster.client().await; let lock_client = client.lock_client(); + let event = Arc::new(event_listener::Event::new()); let lock_handle = tokio::spawn({ let c = lock_client.clone(); + let event = Arc::clone(&event); async move { let mut xutex = Xutex::new(c, "test", None, None).await.unwrap(); let _lock = xutex.lock_unsafe().await.unwrap(); - sleep(Duration::from_secs(3)).await; + let _notified = event.notify(1); + sleep(Duration::from_secs(2)).await; } }); - sleep(Duration::from_secs(1)).await; + event.listen().await; let now = Instant::now(); let mut xutex = Xutex::new(lock_client, "test", None, None).await?; From 1be16f86c7f27ffc030856544d8d85b2f2452a12 Mon Sep 17 00:00:00 2001 From: bsbds <69835502+bsbds@users.noreply.github.com> Date: Tue, 3 Dec 2024 13:51:06 +0800 Subject: [PATCH 5/9] chore: bump tonic and rustls version Signed-off-by: bsbds <69835502+bsbds@users.noreply.github.com> --- Cargo.lock | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index dcc2fc051..b1e55a724 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2490,9 +2490,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.14" +version = "0.23.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "415d9944693cb90382053259f89fbb077ea730ad7273047ec63b19bc9b160ba8" +checksum = "934b404430bb06b3fae2cba809eb45a1ab1aecd64491213d7c3301b88393f8d1" dependencies = [ "log", "once_cell", @@ -2514,9 +2514,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.9.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e696e35370c65c9c541198af4543ccd580cf17fc25d8e05c5a242b202488c55" +checksum = "16f1201b3c9a7ee8039bcadc17b7e605e2945b27eee7631788c1bd2b0643674b" [[package]] name = "rustls-webpki" From 608163528d806ff6b49e2b1a0adf284203c8505f Mon Sep 17 00:00:00 2001 From: Alan Tang Date: Mon, 9 Dec 2024 12:46:44 +0800 Subject: [PATCH 6/9] refactor: Refactor the new method using the builder pattern Signed-off-by: Alan Tang --- crates/benchmark/src/runner.rs | 20 +- crates/utils/src/config/auth.rs | 44 +++- crates/utils/src/config/client.rs | 144 ++++++++++--- crates/utils/src/config/cluster.rs | 4 +- crates/utils/src/config/compact.rs | 60 +++++- crates/utils/src/config/curp.rs | 9 + crates/utils/src/config/log.rs | 56 ++++- crates/utils/src/config/metrics.rs | 105 ++++++++-- crates/utils/src/config/server.rs | 326 ++++++++++++++++++++--------- crates/utils/src/config/storage.rs | 43 +++- crates/utils/src/config/tls.rs | 105 ++++++++-- crates/utils/src/config/trace.rs | 73 +++++-- crates/xline-test-utils/src/lib.rs | 36 ++-- crates/xline/src/utils/args.rs | 153 +++++++++----- crates/xline/tests/it/auth_test.rs | 25 ++- crates/xline/tests/it/tls_test.rs | 52 ++--- crates/xlinectl/src/main.rs | 30 ++- 17 files changed, 968 insertions(+), 317 deletions(-) diff --git a/crates/benchmark/src/runner.rs b/crates/benchmark/src/runner.rs index d60997aa6..166a6659e 100644 --- a/crates/benchmark/src/runner.rs +++ b/crates/benchmark/src/runner.rs @@ -159,15 +159,17 @@ impl CommandRunner { /// Create clients async fn create_clients(&self) -> Result> { - let client_options = ClientOptions::default().with_client_config(ClientConfig::new( - Duration::from_secs(10), - Duration::from_secs(5), - Duration::from_millis(250), - Duration::from_millis(10_000), - 3, - true, - Duration::from_secs(1), - )); + let client_options = ClientOptions::default().with_client_config( + ClientConfig::builder() + .wait_synced_timeout(Duration::from_secs(10)) + .propose_timeout(Duration::from_secs(5)) + .initial_retry_timeout(Duration::from_millis(250)) + .max_retry_timeout(Duration::from_millis(10_000)) + .retry_count(3) + .fixed_backoff(true) + .keep_alive_interval(Duration::from_secs(1)) + .build(), + ); let addrs = self .args .endpoints diff --git a/crates/utils/src/config/auth.rs b/crates/utils/src/config/auth.rs index 1c3c8e587..fbb830028 100644 --- a/crates/utils/src/config/auth.rs +++ b/crates/utils/src/config/auth.rs @@ -15,13 +15,47 @@ pub struct AuthConfig { } impl AuthConfig { - /// Generate a new `AuthConfig` object + /// Create a builder for `AuthConfig` + #[inline] #[must_use] + pub fn builder() -> Builder { + Builder::default() + } +} + +/// Builder for `AuthConfig` +#[derive(Default, Debug)] +pub struct Builder { + /// The public key file + auth_public_key: Option, + /// The private key file + auth_private_key: Option, +} + +impl Builder { + /// Set the public key file #[inline] - pub fn new(auth_public_key: Option, auth_private_key: Option) -> Self { - Self { - auth_public_key, - auth_private_key, + #[must_use] + pub fn auth_public_key(mut self, path: Option) -> Self { + self.auth_public_key = path; + self + } + + /// Set the private key file + #[inline] + #[must_use] + pub fn auth_private_key(mut self, path: Option) -> Self { + self.auth_private_key = path; + self + } + + /// Build the `AuthConfig` + #[inline] + #[must_use] + pub fn build(self) -> AuthConfig { + AuthConfig { + auth_public_key: self.auth_public_key, + auth_private_key: self.auth_private_key, } } } diff --git a/crates/utils/src/config/client.rs b/crates/utils/src/config/client.rs index 111966298..a429ee58e 100644 --- a/crates/utils/src/config/client.rs +++ b/crates/utils/src/config/client.rs @@ -57,35 +57,11 @@ pub struct ClientConfig { } impl ClientConfig { - /// Create a new client timeout - /// - /// # Panics - /// - /// Panics if `initial_retry_timeout` is larger than `max_retry_timeout` - #[must_use] + /// Create a builder for `ClientConfig` #[inline] - pub fn new( - wait_synced_timeout: Duration, - propose_timeout: Duration, - initial_retry_timeout: Duration, - max_retry_timeout: Duration, - retry_count: usize, - fixed_backoff: bool, - keep_alive_interval: Duration, - ) -> Self { - assert!( - initial_retry_timeout <= max_retry_timeout, - "`initial_retry_timeout` should less or equal to `max_retry_timeout`" - ); - Self { - wait_synced_timeout, - propose_timeout, - initial_retry_timeout, - max_retry_timeout, - retry_count, - fixed_backoff, - keep_alive_interval, - } + #[must_use] + pub fn builder() -> Builder { + Builder::default() } } @@ -103,3 +79,115 @@ impl Default for ClientConfig { } } } + +/// Builder for `ClientConfig` +#[derive(Default, Debug, Clone, Copy)] +pub struct Builder { + /// Curp client wait sync timeout + wait_synced_timeout: Option, + /// Curp client propose request timeout + propose_timeout: Option, + /// Curp client initial retry interval + initial_retry_timeout: Option, + /// Curp client max retry interval + max_retry_timeout: Option, + /// Curp client retry interval + retry_count: Option, + /// Whether to use exponential backoff in retries + fixed_backoff: Option, + /// Curp client keep client id alive interval + keep_alive_interval: Option, +} + +impl Builder { + /// Set the wait sync timeout + #[inline] + #[must_use] + pub fn wait_synced_timeout(mut self, timeout: Duration) -> Self { + self.wait_synced_timeout = Some(timeout); + self + } + + /// Set the propose timeout + #[inline] + #[must_use] + pub fn propose_timeout(mut self, timeout: Duration) -> Self { + self.propose_timeout = Some(timeout); + self + } + + /// Set the initial retry timeout + #[inline] + #[must_use] + pub fn initial_retry_timeout(mut self, timeout: Duration) -> Self { + self.initial_retry_timeout = Some(timeout); + self + } + + /// Set the max retry timeout + #[inline] + #[must_use] + pub fn max_retry_timeout(mut self, timeout: Duration) -> Self { + self.max_retry_timeout = Some(timeout); + self + } + + /// Set the retry count + #[inline] + #[must_use] + pub fn retry_count(mut self, count: usize) -> Self { + self.retry_count = Some(count); + self + } + + /// Set whether to use fixed backoff + #[inline] + #[must_use] + pub fn fixed_backoff(mut self, use_backoff: bool) -> Self { + self.fixed_backoff = Some(use_backoff); + self + } + + /// Set the keep alive interval + #[inline] + #[must_use] + pub fn keep_alive_interval(mut self, interval: Duration) -> Self { + self.keep_alive_interval = Some(interval); + self + } + + /// # Panics + /// + /// Panics if `initial_retry_timeout` is larger than `max_retry_timeout` + /// Build the `ClientConfig` and validate it + #[inline] + #[must_use] + pub fn build(self) -> ClientConfig { + let initial_retry_timeout = self + .initial_retry_timeout + .unwrap_or_else(default_initial_retry_timeout); + let max_retry_timeout = self + .max_retry_timeout + .unwrap_or_else(default_max_retry_timeout); + + // Assert that `initial_retry_timeout <= max_retry_timeout` + assert!( + initial_retry_timeout <= max_retry_timeout, + "`initial_retry_timeout` should be less than or equal to `max_retry_timeout`" + ); + + ClientConfig { + wait_synced_timeout: self + .wait_synced_timeout + .unwrap_or_else(default_client_wait_synced_timeout), + propose_timeout: self.propose_timeout.unwrap_or_else(default_propose_timeout), + initial_retry_timeout, + max_retry_timeout, + retry_count: self.retry_count.unwrap_or_else(default_retry_count), + fixed_backoff: self.fixed_backoff.unwrap_or_else(default_fixed_backoff), + keep_alive_interval: self + .keep_alive_interval + .unwrap_or_else(default_client_id_keep_alive_interval), + } + } +} diff --git a/crates/utils/src/config/cluster.rs b/crates/utils/src/config/cluster.rs index 4b1a60c25..b3aa09db5 100644 --- a/crates/utils/src/config/cluster.rs +++ b/crates/utils/src/config/cluster.rs @@ -1,3 +1,4 @@ +use derive_builder::Builder; use getset::Getters; use serde::Deserialize; use std::collections::HashMap; @@ -6,7 +7,8 @@ use super::prelude::{ClientConfig, CurpConfig, XlineServerTimeout}; /// Cluster configuration object, including cluster relevant configuration fields #[allow(clippy::module_name_repetitions)] -#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters)] +#[derive(Clone, Debug, Deserialize, PartialEq, Eq, Getters, Builder)] +#[builder(pattern = "owned", default)] pub struct ClusterConfig { /// Get xline server name #[getset(get = "pub")] diff --git a/crates/utils/src/config/compact.rs b/crates/utils/src/config/compact.rs index 9def42083..ecc16cc49 100644 --- a/crates/utils/src/config/compact.rs +++ b/crates/utils/src/config/compact.rs @@ -33,18 +33,58 @@ impl Default for CompactConfig { } impl CompactConfig { - /// Create a new compact config + /// Create a builder for `CompactConfig` + #[inline] #[must_use] + pub fn builder() -> Builder { + Builder::default() + } +} + +/// Builder for `CompactConfig` +#[derive(Default, Debug, Clone, Copy)] +pub struct Builder { + /// The max number of historical versions processed in a single compact operation + compact_batch_size: Option, + /// The interval between two compaction batches + compact_sleep_interval: Option, + /// The auto compactor config + auto_compact_config: Option, +} + +impl Builder { + /// Set the compact batch size #[inline] - pub fn new( - compact_batch_size: usize, - compact_sleep_interval: Duration, - auto_compact_config: Option, - ) -> Self { - Self { - compact_batch_size, - compact_sleep_interval, - auto_compact_config, + #[must_use] + pub fn compact_batch_size(mut self, size: usize) -> Self { + self.compact_batch_size = Some(size); + self + } + + /// Set the compact sleep interval + #[inline] + #[must_use] + pub fn compact_sleep_interval(mut self, interval: Duration) -> Self { + self.compact_sleep_interval = Some(interval); + self + } + + /// Set the auto compactor config + #[inline] + #[must_use] + pub fn auto_compact_config(mut self, config: Option) -> Self { + self.auto_compact_config = config; + self + } + + /// Build the `CompactConfig` + #[inline] + #[must_use] + pub fn build(self) -> CompactConfig { + CompactConfig { + compact_batch_size: self.compact_batch_size.unwrap_or_default(), + compact_sleep_interval: self.compact_sleep_interval.unwrap_or_default(), + auto_compact_config: self.auto_compact_config, } } } diff --git a/crates/utils/src/config/curp.rs b/crates/utils/src/config/curp.rs index 16baeca4e..d32d1a968 100644 --- a/crates/utils/src/config/curp.rs +++ b/crates/utils/src/config/curp.rs @@ -83,6 +83,15 @@ pub struct CurpConfig { pub log_entries_cap: usize, } +impl CurpConfig { + /// Create a new `CurpConfig` with a builder + #[must_use] + #[inline] + pub fn builder() -> CurpConfigBuilder { + CurpConfigBuilder::default() + } +} + /// default heartbeat interval #[must_use] #[inline] diff --git a/crates/utils/src/config/log.rs b/crates/utils/src/config/log.rs index 14426fc50..23ee3b714 100644 --- a/crates/utils/src/config/log.rs +++ b/crates/utils/src/config/log.rs @@ -64,14 +64,58 @@ pub const fn default_log_level() -> LevelConfig { } impl LogConfig { - /// Generate a new `LogConfig` object + /// Create a builder for `LogConfig` #[must_use] #[inline] - pub fn new(path: Option, rotation: RotationConfig, level: LevelConfig) -> Self { - Self { - path, - rotation, - level, + pub fn builder() -> Builder { + Builder::default() + } +} + +/// Builder for `LogConfig` +#[derive(Default, Debug)] +pub struct Builder { + /// Log file path + path: Option, + /// Log rotation strategy + rotation: Option, + /// Log verbosity level + level: Option, +} + +impl Builder { + /// Set the log file path + #[inline] + #[must_use] + pub fn path(mut self, path: Option) -> Self { + self.path = path; + self + } + + /// Set the log rotation strategy + #[inline] + #[must_use] + pub fn rotation(mut self, rotation: RotationConfig) -> Self { + self.rotation = Some(rotation); + self + } + + /// Set the log verbosity level + #[inline] + #[must_use] + pub fn level(mut self, level: LevelConfig) -> Self { + self.level = Some(level); + self + } + + /// Build the `LogConfig` and apply defaults where needed + #[inline] + #[must_use] + pub fn build(self) -> LogConfig { + LogConfig { + path: self.path, + rotation: self.rotation.unwrap_or_else(default_rotation), + level: self.level.unwrap_or_else(default_log_level), } } } diff --git a/crates/utils/src/config/metrics.rs b/crates/utils/src/config/metrics.rs index e2a3f6435..04f78499a 100644 --- a/crates/utils/src/config/metrics.rs +++ b/crates/utils/src/config/metrics.rs @@ -50,25 +50,11 @@ pub struct MetricsConfig { } impl MetricsConfig { - /// Create a new `MetricsConfig` + /// Create a new `MetricsConfig` builder #[must_use] #[inline] - pub fn new( - enable: bool, - port: u16, - path: String, - push: bool, - push_endpoint: String, - push_protocol: PushProtocol, - ) -> Self { - Self { - enable, - port, - path, - push, - push_endpoint, - push_protocol, - } + pub fn builder() -> Builder { + Builder::default() } } @@ -86,6 +72,91 @@ impl Default for MetricsConfig { } } +/// Builder for `MetricsConfig` +#[derive(Debug, Default)] +pub struct Builder { + /// Enable or not + enable: Option, + /// The http port to expose + port: Option, + /// The http path to expose + path: Option, + /// Enable push or not + push: Option, + /// Push endpoint + push_endpoint: Option, + /// Push protocol + push_protocol: Option, +} + +impl Builder { + /// Set the `enable` flag + #[must_use] + #[inline] + pub fn enable(mut self, enable: bool) -> Self { + self.enable = Some(enable); + self + } + + /// Set the `port` + #[must_use] + #[inline] + pub fn port(mut self, port: u16) -> Self { + self.port = Some(port); + self + } + + /// Set the `path` + #[must_use] + #[inline] + pub fn path>(mut self, path: S) -> Self { + self.path = Some(path.into()); + self + } + + /// Set the `push` flag + #[must_use] + #[inline] + pub fn push(mut self, push: bool) -> Self { + self.push = Some(push); + self + } + + /// Set the `push_endpoint` + #[must_use] + #[inline] + pub fn push_endpoint>(mut self, push_endpoint: S) -> Self { + self.push_endpoint = Some(push_endpoint.into()); + self + } + + /// Set the `push_protocol` + #[must_use] + #[inline] + pub fn push_protocol(mut self, push_protocol: PushProtocol) -> Self { + self.push_protocol = Some(push_protocol); + self + } + + /// Build the `MetricsConfig` + #[must_use] + #[inline] + pub fn build(self) -> MetricsConfig { + MetricsConfig { + enable: self.enable.unwrap_or_else(default_metrics_enable), + port: self.port.unwrap_or_else(default_metrics_port), + path: self.path.unwrap_or_else(default_metrics_path), + push: self.push.unwrap_or_else(default_metrics_push), + push_endpoint: self + .push_endpoint + .unwrap_or_else(default_metrics_push_endpoint), + push_protocol: self + .push_protocol + .unwrap_or_else(default_metrics_push_protocol), + } + } +} + /// Default metrics enable #[must_use] #[inline] diff --git a/crates/utils/src/config/server.rs b/crates/utils/src/config/server.rs index 54fc6f69e..fd748f346 100644 --- a/crates/utils/src/config/server.rs +++ b/crates/utils/src/config/server.rs @@ -46,26 +46,110 @@ impl XlineServerConfig { /// Generates a new `XlineServerConfig` object #[must_use] #[inline] - #[allow(clippy::too_many_arguments)] - pub fn new( - cluster: ClusterConfig, - storage: StorageConfig, - log: LogConfig, - trace: TraceConfig, - auth: AuthConfig, - compact: CompactConfig, - tls: TlsConfig, - metrics: MetricsConfig, - ) -> Self { - Self { - cluster, - storage, - log, - trace, - auth, - compact, - tls, - metrics, + pub fn builder() -> Builder { + Builder::default() + } +} + +/// Builder for `XlineServerConfig` +#[derive(Debug, Default)] +pub struct Builder { + /// cluster configuration object + cluster: Option, + /// xline storage configuration object + storage: Option, + /// log configuration object + log: Option, + /// trace configuration object + trace: Option, + /// auth configuration object + auth: Option, + /// compactor configuration object + compact: Option, + /// tls configuration object + tls: Option, + /// Metrics config + metrics: Option, +} + +impl Builder { + /// set the cluster object + #[must_use] + #[inline] + pub fn cluster(mut self, cluster: ClusterConfig) -> Self { + self.cluster = Some(cluster); + self + } + + /// set the storage object + #[must_use] + #[inline] + pub fn storage(mut self, storage: StorageConfig) -> Self { + self.storage = Some(storage); + self + } + + /// set the log object + #[must_use] + #[inline] + pub fn log(mut self, log: LogConfig) -> Self { + self.log = Some(log); + self + } + + /// set the trace object + #[must_use] + #[inline] + pub fn trace(mut self, trace: TraceConfig) -> Self { + self.trace = Some(trace); + self + } + + /// set the trace object + #[must_use] + #[inline] + pub fn auth(mut self, auth: AuthConfig) -> Self { + self.auth = Some(auth); + self + } + + /// set the compact object + #[must_use] + #[inline] + pub fn compact(mut self, compact: CompactConfig) -> Self { + self.compact = Some(compact); + self + } + + /// set the compact object + #[must_use] + #[inline] + pub fn tls(mut self, tls: TlsConfig) -> Self { + self.tls = Some(tls); + self + } + + /// set the compact object + #[must_use] + #[inline] + pub fn metrics(mut self, metrics: MetricsConfig) -> Self { + self.metrics = Some(metrics); + self + } + + /// Build the `XlineServerConfig` + #[must_use] + #[inline] + pub fn build(self) -> XlineServerConfig { + XlineServerConfig { + cluster: self.cluster.unwrap_or_default(), + storage: self.storage.unwrap_or_default(), + log: self.log.unwrap_or_default(), + trace: self.trace.unwrap_or_default(), + auth: self.auth.unwrap_or_default(), + compact: self.compact.unwrap_or_default(), + tls: self.tls.unwrap_or_default(), + metrics: self.metrics.unwrap_or_default(), } } } @@ -95,21 +179,11 @@ pub struct XlineServerTimeout { } impl XlineServerTimeout { - /// Create a new server timeout + /// Create a builder for `XlineServerTimeout` #[must_use] #[inline] - pub fn new( - range_retry_timeout: Duration, - compact_timeout: Duration, - sync_victims_interval: Duration, - watch_progress_notify_interval: Duration, - ) -> Self { - Self { - range_retry_timeout, - compact_timeout, - sync_victims_interval, - watch_progress_notify_interval, - } + pub fn builder() -> XlineServerTimeoutBuilder { + XlineServerTimeoutBuilder::default() } } @@ -125,6 +199,65 @@ impl Default for XlineServerTimeout { } } +/// Builder for `XlineServerTimeout` +#[derive(Debug, Default, Clone, Copy)] +pub struct XlineServerTimeoutBuilder { + /// Range request retry timeout settings + range_retry_timeout: Option, + /// Range request retry timeout settings + compact_timeout: Option, + /// Sync victims interval + sync_victims_interval: Option, + /// Watch progress notify interval settings + watch_progress_notify_interval: Option, +} + +impl XlineServerTimeoutBuilder { + /// Set the range retry timeout + #[must_use] + #[inline] + pub fn range_retry_timeout(mut self, timeout: Duration) -> Self { + self.range_retry_timeout = Some(timeout); + self + } + + /// Set the compact timeout + #[must_use] + #[inline] + pub fn compact_timeout(mut self, timeout: Duration) -> Self { + self.compact_timeout = Some(timeout); + self + } + + /// Set the sync victims interval + #[must_use] + #[inline] + pub fn sync_victims_interval(mut self, interval: Duration) -> Self { + self.sync_victims_interval = Some(interval); + self + } + + /// Set the watch progress notify interval + #[must_use] + #[inline] + pub fn watch_progress_notify_interval(mut self, interval: Duration) -> Self { + self.watch_progress_notify_interval = Some(interval); + self + } + + /// Build the `XlineServerTimeout` instance + #[must_use] + #[inline] + pub fn build(self) -> XlineServerTimeout { + XlineServerTimeout { + range_retry_timeout: self.range_retry_timeout.unwrap_or_default(), + compact_timeout: self.compact_timeout.unwrap_or_default(), + sync_victims_interval: self.sync_victims_interval.unwrap_or_default(), + watch_progress_notify_interval: self.watch_progress_notify_interval.unwrap_or_default(), + } + } +} + #[cfg(test)] mod tests { use std::{collections::HashMap, path::PathBuf, time::Duration}; @@ -214,22 +347,22 @@ mod tests { .build() .unwrap(); - let client_config = ClientConfig::new( - default_client_wait_synced_timeout(), - default_propose_timeout(), - Duration::from_secs(5), - Duration::from_secs(50), - default_retry_count(), - default_fixed_backoff(), - default_client_id_keep_alive_interval(), - ); - - let server_timeout = XlineServerTimeout::new( - Duration::from_secs(3), - Duration::from_secs(5), - Duration::from_millis(20), - Duration::from_secs(1), - ); + let client_config = ClientConfig::builder() + .wait_synced_timeout(default_client_wait_synced_timeout()) + .propose_timeout(default_propose_timeout()) + .initial_retry_timeout(Duration::from_secs(5)) + .max_retry_timeout(Duration::from_secs(50)) + .retry_count(default_retry_count()) + .fixed_backoff(default_fixed_backoff()) + .keep_alive_interval(default_client_id_keep_alive_interval()) + .build(); + + let server_timeout = XlineServerTimeout::builder() + .range_retry_timeout(Duration::from_secs(3)) + .compact_timeout(Duration::from_secs(5)) + .sync_victims_interval(Duration::from_millis(20)) + .watch_progress_notify_interval(Duration::from_secs(1)) + .build(); assert_eq!( *config.cluster(), @@ -257,44 +390,47 @@ mod tests { assert_eq!( *config.storage(), - StorageConfig::new(EngineConfig::Memory, default_quota()) + StorageConfig::builder() + .engine(EngineConfig::Memory) + .quota(default_quota()) + .build() ); assert_eq!( *config.log(), - LogConfig::new( - Some(PathBuf::from("/var/log/xline")), - RotationConfig::Daily, - LevelConfig::INFO - ) + LogConfig::builder() + .path(Some(PathBuf::from("/var/log/xline"))) + .rotation(RotationConfig::Daily) + .level(LevelConfig::INFO) + .build() ); assert_eq!( *config.trace(), - TraceConfig::new( - false, - false, - PathBuf::from("./jaeger_jsons"), - LevelConfig::INFO - ) + TraceConfig::builder() + .jaeger_online(false) + .jaeger_offline(false) + .jaeger_output_dir(PathBuf::from("./jaeger_jsons")) + .jaeger_level(LevelConfig::INFO) + .build() ); assert_eq!( *config.compact(), - CompactConfig::new( - 123, - Duration::from_millis(5), - Some(AutoCompactConfig::Periodic(Duration::from_secs( + CompactConfig::builder() + .compact_batch_size(123) + .compact_sleep_interval(Duration::from_millis(5)) + .auto_compact_config(Some(AutoCompactConfig::Periodic(Duration::from_secs( 10 * 60 * 60 - ))) - ) + )))) + .build() ); assert_eq!( *config.auth(), - AuthConfig::new( - Some(PathBuf::from("./public_key.pem")), - Some(PathBuf::from("./private_key.pem")) - ) + AuthConfig::builder() + .auth_public_key(Some(PathBuf::from("./public_key.pem"))) + .auth_private_key(Some(PathBuf::from("./private_key.pem"))) + .build() ); assert_eq!( @@ -309,14 +445,14 @@ mod tests { assert_eq!( *config.metrics(), - MetricsConfig::new( - true, - 9100, - "/metrics".to_owned(), - true, - "http://some-endpoint.com:4396".to_owned(), - PushProtocol::HTTP - ), + MetricsConfig::builder() + .enable(true) + .port(9100) + .path("/metrics".to_owned()) + .push(true) + .push_endpoint("http://some-endpoint.com:4396".to_owned()) + .push_protocol(PushProtocol::HTTP) + .build(), ); } @@ -388,20 +524,20 @@ mod tests { assert_eq!( *config.log(), - LogConfig::new( - Some(PathBuf::from("/var/log/xline")), - RotationConfig::Never, - LevelConfig::INFO - ) + LogConfig::builder() + .path(Some(PathBuf::from("/var/log/xline"))) + .rotation(RotationConfig::Never) + .level(LevelConfig::INFO) + .build() ); assert_eq!( *config.trace(), - TraceConfig::new( - false, - false, - PathBuf::from("./jaeger_jsons"), - LevelConfig::INFO - ) + TraceConfig::builder() + .jaeger_online(false) + .jaeger_offline(false) + .jaeger_output_dir(PathBuf::from("./jaeger_jsons")) + .jaeger_level(LevelConfig::INFO) + .build() ); assert_eq!(*config.compact(), CompactConfig::default()); assert_eq!(*config.auth(), AuthConfig::default()); @@ -454,11 +590,11 @@ mod tests { assert_eq!( *config.compact(), - CompactConfig::new( - default_compact_batch_size(), - default_compact_sleep_interval(), - Some(AutoCompactConfig::Revision(10000)) - ) + CompactConfig::builder() + .compact_batch_size(default_compact_batch_size()) + .compact_sleep_interval(default_compact_sleep_interval()) + .auto_compact_config(Some(AutoCompactConfig::Revision(10000))) + .build() ); } } diff --git a/crates/utils/src/config/storage.rs b/crates/utils/src/config/storage.rs index d0ebf45b7..55382ec5a 100644 --- a/crates/utils/src/config/storage.rs +++ b/crates/utils/src/config/storage.rs @@ -19,11 +19,11 @@ pub struct StorageConfig { } impl StorageConfig { - /// Create a new storage config + /// Create a builder for `StorageConfig` #[inline] #[must_use] - pub fn new(engine: EngineConfig, quota: u64) -> Self { - Self { engine, quota } + pub fn builder() -> Builder { + Builder::default() } } @@ -37,6 +37,43 @@ impl Default for StorageConfig { } } +/// Builder for `StorageConfig` +#[derive(Default, Debug)] +pub struct Builder { + /// Engine Configuration + engine: Option, + /// Quota + quota: Option, +} + +impl Builder { + /// Set the engine configuration + #[inline] + #[must_use] + pub fn engine(mut self, engine: EngineConfig) -> Self { + self.engine = Some(engine); + self + } + + /// Set the quota + #[inline] + #[must_use] + pub fn quota(mut self, quota: u64) -> Self { + self.quota = Some(quota); + self + } + + /// Build the `StorageConfig` and apply defaults where needed + #[inline] + #[must_use] + pub fn build(self) -> StorageConfig { + StorageConfig { + engine: self.engine.unwrap_or_default(), + quota: self.quota.unwrap_or_else(default_quota), + } + } +} + /// Default quota: 8GB #[inline] #[must_use] diff --git a/crates/utils/src/config/tls.rs b/crates/utils/src/config/tls.rs index de56d1bc3..d4e134e4b 100644 --- a/crates/utils/src/config/tls.rs +++ b/crates/utils/src/config/tls.rs @@ -29,25 +29,11 @@ pub struct TlsConfig { } impl TlsConfig { - /// Create a new `TlsConfig` object - #[must_use] - #[inline] - pub fn new( - peer_ca_cert_path: Option, - peer_cert_path: Option, - peer_key_path: Option, - client_ca_cert_path: Option, - client_cert_path: Option, - client_key_path: Option, - ) -> Self { - Self { - peer_ca_cert_path, - peer_cert_path, - peer_key_path, - client_ca_cert_path, - client_cert_path, - client_key_path, - } + /// Create a builder for `TlsConfig` + #[inline] + #[must_use] + pub fn builder() -> Builder { + Builder::default() } /// Whether the server tls is enabled @@ -57,3 +43,84 @@ impl TlsConfig { self.peer_cert_path.is_some() && self.peer_key_path.is_some() } } + +/// Builder for `TlsConfig` +#[derive(Default, Debug)] +pub struct Builder { + /// The CA certificate file used by peer to verify client certificates + peer_ca_cert_path: Option, + /// The public key file used by peer + peer_cert_path: Option, + /// The private key file used by peer + peer_key_path: Option, + /// The CA certificate file used by client to verify peer c + client_ca_cert_path: Option, + /// The public key file used by client + client_cert_path: Option, + /// The private key file used by client + client_key_path: Option, +} + +impl Builder { + /// Set the `peer_ca_cert_path` + #[inline] + #[must_use] + pub fn peer_ca_cert_path(mut self, path: Option) -> Self { + self.peer_ca_cert_path = path; + self + } + + /// Set the `peer_cert_path` + #[inline] + #[must_use] + pub fn peer_cert_path(mut self, path: Option) -> Self { + self.peer_cert_path = path; + self + } + + /// Set the `peer_key_path` + #[inline] + #[must_use] + pub fn peer_key_path(mut self, path: Option) -> Self { + self.peer_key_path = path; + self + } + + /// Set the `client_ca_cert_path` + #[inline] + #[must_use] + pub fn client_ca_cert_path(mut self, path: Option) -> Self { + self.client_ca_cert_path = path; + self + } + + /// Set the `client_cert_path` + #[inline] + #[must_use] + pub fn client_cert_path(mut self, path: Option) -> Self { + self.client_cert_path = path; + self + } + + /// Set the `client_key_path` + #[inline] + #[must_use] + pub fn client_key_path(mut self, path: Option) -> Self { + self.client_key_path = path; + self + } + + /// Build the `TlsConfig` object + #[inline] + #[must_use] + pub fn build(self) -> TlsConfig { + TlsConfig { + peer_ca_cert_path: self.peer_ca_cert_path, + peer_cert_path: self.peer_cert_path, + peer_key_path: self.peer_key_path, + client_ca_cert_path: self.client_ca_cert_path, + client_cert_path: self.client_cert_path, + client_key_path: self.client_key_path, + } + } +} diff --git a/crates/utils/src/config/trace.rs b/crates/utils/src/config/trace.rs index 935f366ce..76ca45b7b 100644 --- a/crates/utils/src/config/trace.rs +++ b/crates/utils/src/config/trace.rs @@ -37,20 +37,69 @@ impl Default for TraceConfig { } impl TraceConfig { - /// Generate a new `TraceConfig` object + /// Create a builder for `TraceConfig` + #[inline] #[must_use] + pub fn builder() -> Builder { + Builder::default() + } +} + +/// Builder for `TraceConfig` +#[derive(Default, Debug)] +pub struct Builder { + /// Open jaeger online, sending data to jaeger agent directly + jaeger_online: Option, + /// Open jaeger offline, saving data to the `jaeger_output_dir` + jaeger_offline: Option, + /// The dir path to save the data when `jaeger_offline` is on + jaeger_output_dir: Option, + /// The verbosity level of tracing + jaeger_level: Option, +} + +impl Builder { + /// Set `jaeger_online` #[inline] - pub fn new( - jaeger_online: bool, - jaeger_offline: bool, - jaeger_output_dir: PathBuf, - jaeger_level: LevelConfig, - ) -> Self { - Self { - jaeger_online, - jaeger_offline, - jaeger_output_dir, - jaeger_level, + #[must_use] + pub fn jaeger_online(mut self, value: bool) -> Self { + self.jaeger_online = Some(value); + self + } + + /// Set `jaeger_offline` + #[inline] + #[must_use] + pub fn jaeger_offline(mut self, value: bool) -> Self { + self.jaeger_offline = Some(value); + self + } + + /// Set `jaeger_output_dir` + #[inline] + #[must_use] + pub fn jaeger_output_dir(mut self, value: PathBuf) -> Self { + self.jaeger_output_dir = Some(value); + self + } + + /// Set `jaeger_level` + #[inline] + #[must_use] + pub fn jaeger_level(mut self, value: LevelConfig) -> Self { + self.jaeger_level = Some(value); + self + } + + /// Build the `TraceConfig` object + #[inline] + #[must_use] + pub fn build(self) -> TraceConfig { + TraceConfig { + jaeger_online: self.jaeger_online.unwrap_or(false), + jaeger_offline: self.jaeger_offline.unwrap_or(false), + jaeger_output_dir: self.jaeger_output_dir.unwrap_or_else(|| "".into()), + jaeger_level: self.jaeger_level.unwrap_or_else(default_log_level), } } } diff --git a/crates/xline-test-utils/src/lib.rs b/crates/xline-test-utils/src/lib.rs index d86a3a3fb..2741a56e0 100644 --- a/crates/xline-test-utils/src/lib.rs +++ b/crates/xline-test-utils/src/lib.rs @@ -241,14 +241,26 @@ impl Cluster { quota: u64, ) -> XlineServerConfig { let cluster = ClusterConfig::default(); - let storage = StorageConfig::new(EngineConfig::RocksDB(path), quota); + let storage = StorageConfig::builder() + .engine(EngineConfig::RocksDB(path)) + .quota(quota) + .build(); let log = LogConfig::default(); let trace = TraceConfig::default(); let auth = AuthConfig::default(); let compact = CompactConfig::default(); let tls = TlsConfig::default(); let metrics = MetricsConfig::default(); - XlineServerConfig::new(cluster, storage, log, trace, auth, compact, tls, metrics) + XlineServerConfig::builder() + .cluster(cluster) + .storage(storage) + .log(log) + .trace(trace) + .auth(auth) + .compact(compact) + .tls(tls) + .metrics(metrics) + .build() } pub fn default_rocks_config_with_path(path: PathBuf) -> XlineServerConfig { @@ -288,16 +300,16 @@ impl Cluster { *old_cluster.server_timeout(), initial_cluster_state, ); - XlineServerConfig::new( - new_cluster, - base_config.storage().clone(), - base_config.log().clone(), - base_config.trace().clone(), - base_config.auth().clone(), - *base_config.compact(), - base_config.tls().clone(), - base_config.metrics().clone(), - ) + XlineServerConfig::builder() + .cluster(new_cluster) + .storage(base_config.storage().clone()) + .log(base_config.log().clone()) + .trace(base_config.trace().clone()) + .auth(base_config.auth().clone()) + .compact(*base_config.compact()) + .tls(base_config.tls().clone()) + .metrics(base_config.metrics().clone()) + .build() } } diff --git a/crates/xline/src/utils/args.rs b/crates/xline/src/utils/args.rs index 5a9da207f..b901811e0 100644 --- a/crates/xline/src/utils/args.rs +++ b/crates/xline/src/utils/args.rs @@ -233,7 +233,10 @@ impl From for XlineServerConfig { &_ => unreachable!("xline only supports memory and rocksdb engine"), }; - let storage = StorageConfig::new(engine, args.quota.unwrap_or_else(default_quota)); + let storage = StorageConfig::builder() + .engine(engine) + .quota(args.quota.unwrap_or_else(default_quota)) + .build(); let Ok(curp_config) = CurpConfigBuilder::default() .heartbeat_interval( args.heartbeat_interval @@ -255,29 +258,48 @@ impl From for XlineServerConfig { else { panic!("failed to create curp config") }; - let client_config = ClientConfig::new( - args.client_wait_synced_timeout - .unwrap_or_else(default_client_wait_synced_timeout), - args.client_propose_timeout - .unwrap_or_else(default_propose_timeout), - args.client_initial_retry_timeout - .unwrap_or_else(default_initial_retry_timeout), - args.client_max_retry_timeout - .unwrap_or_else(default_max_retry_timeout), - args.retry_count.unwrap_or_else(default_retry_count), - args.client_fixed_backoff, - args.client_keep_alive_interval - .unwrap_or_else(default_client_id_keep_alive_interval), - ); - let server_timeout = XlineServerTimeout::new( - args.range_retry_timeout - .unwrap_or_else(default_range_retry_timeout), - args.compact_timeout.unwrap_or_else(default_compact_timeout), - args.sync_victims_interval - .unwrap_or_else(default_sync_victims_interval), - args.watch_progress_notify_interval - .unwrap_or_else(default_watch_progress_notify_interval), - ); + + let client_config = ClientConfig::builder() + .wait_synced_timeout( + args.client_wait_synced_timeout + .unwrap_or_else(default_client_wait_synced_timeout), + ) + .propose_timeout( + args.client_propose_timeout + .unwrap_or_else(default_propose_timeout), + ) + .initial_retry_timeout( + args.client_initial_retry_timeout + .unwrap_or_else(default_initial_retry_timeout), + ) + .max_retry_timeout( + args.client_max_retry_timeout + .unwrap_or_else(default_max_retry_timeout), + ) + .retry_count(args.retry_count.unwrap_or_else(default_retry_count)) + .fixed_backoff(args.client_fixed_backoff) + .keep_alive_interval( + args.client_keep_alive_interval + .unwrap_or_else(default_client_id_keep_alive_interval), + ) + .build(); + + let server_timeout = XlineServerTimeout::builder() + .range_retry_timeout( + args.range_retry_timeout + .unwrap_or_else(default_range_retry_timeout), + ) + .compact_timeout(args.compact_timeout.unwrap_or_else(default_compact_timeout)) + .sync_victims_interval( + args.sync_victims_interval + .unwrap_or_else(default_sync_victims_interval), + ) + .watch_progress_notify_interval( + args.watch_progress_notify_interval + .unwrap_or_else(default_watch_progress_notify_interval), + ) + .build(); + let initial_cluster_state = args.initial_cluster_state.unwrap_or_default(); let cluster = ClusterConfig::new( args.name, @@ -292,14 +314,22 @@ impl From for XlineServerConfig { server_timeout, initial_cluster_state, ); - let log = LogConfig::new(args.log_file, args.log_rotate, args.log_level); - let trace = TraceConfig::new( - args.jaeger_online, - args.jaeger_offline, - args.jaeger_output_dir, - args.jaeger_level, - ); - let auth = AuthConfig::new(args.auth_public_key, args.auth_private_key); + let log = LogConfig::builder() + .path(args.log_file) + .rotation(args.log_rotate) + .level(args.log_level) + .build(); + + let trace = TraceConfig::builder() + .jaeger_online(args.jaeger_online) + .jaeger_offline(args.jaeger_offline) + .jaeger_output_dir(args.jaeger_output_dir) + .jaeger_level(args.jaeger_level) + .build(); + let auth = AuthConfig::builder() + .auth_public_key(args.auth_public_key) + .auth_private_key(args.auth_private_key) + .build(); let auto_compactor_cfg = if let Some(mode) = args.auto_compact_mode { match mode.as_str() { "periodic" => { @@ -321,29 +351,42 @@ impl From for XlineServerConfig { } else { None }; - let compact = CompactConfig::new( - args.compact_batch_size, - args.compact_sleep_interval - .unwrap_or_else(default_compact_sleep_interval), - auto_compactor_cfg, - ); - let tls = TlsConfig::new( - args.peer_ca_cert_path, - args.peer_cert_path, - args.peer_key_path, - args.client_ca_cert_path, - args.client_cert_path, - args.client_key_path, - ); - let metrics = MetricsConfig::new( - args.metrics_enable, - args.metrics_port, - args.metrics_path, - args.metrics_push, - args.metrics_push_endpoint, - args.metrics_push_protocol, - ); - XlineServerConfig::new(cluster, storage, log, trace, auth, compact, tls, metrics) + let compact = CompactConfig::builder() + .compact_batch_size(args.compact_batch_size) + .compact_sleep_interval( + args.compact_sleep_interval + .unwrap_or_else(default_compact_sleep_interval), + ) + .auto_compact_config(auto_compactor_cfg) + .build(); + let tls = TlsConfig::builder() + .peer_ca_cert_path(args.peer_ca_cert_path) + .peer_cert_path(args.peer_cert_path) + .peer_key_path(args.peer_key_path) + .client_ca_cert_path(args.client_ca_cert_path) + .client_cert_path(args.client_cert_path) + .client_key_path(args.client_key_path) + .build(); + + let metrics = MetricsConfig::builder() + .enable(args.metrics_enable) + .port(args.metrics_port) + .path(args.metrics_path) + .push(args.metrics_push) + .push_endpoint(args.metrics_push_endpoint) + .push_protocol(args.metrics_push_protocol) + .build(); + + XlineServerConfig::builder() + .cluster(cluster) + .storage(storage) + .log(log) + .trace(trace) + .auth(auth) + .compact(compact) + .tls(tls) + .metrics(metrics) + .build() } } diff --git a/crates/xline/tests/it/auth_test.rs b/crates/xline/tests/it/auth_test.rs index ddba12298..476372d44 100644 --- a/crates/xline/tests/it/auth_test.rs +++ b/crates/xline/tests/it/auth_test.rs @@ -217,16 +217,21 @@ fn configs_with_auth(size: usize) -> Vec { ) }) .map(|(auth_public_key, auth_private_key)| { - XlineServerConfig::new( - ClusterConfig::default(), - StorageConfig::default(), - LogConfig::default(), - TraceConfig::default(), - AuthConfig::new(auth_public_key, auth_private_key), - CompactConfig::default(), - TlsConfig::default(), - MetricsConfig::default(), - ) + XlineServerConfig::builder() + .cluster(ClusterConfig::default()) + .storage(StorageConfig::default()) + .log(LogConfig::default()) + .trace(TraceConfig::default()) + .auth( + AuthConfig::builder() + .auth_public_key(auth_public_key) + .auth_private_key(auth_private_key) + .build(), + ) + .compact(CompactConfig::default()) + .tls(TlsConfig::default()) + .metrics(MetricsConfig::default()) + .build() }) .take(size) .collect() diff --git a/crates/xline/tests/it/tls_test.rs b/crates/xline/tests/it/tls_test.rs index d1024ce65..5e8f1a01c 100644 --- a/crates/xline/tests/it/tls_test.rs +++ b/crates/xline/tests/it/tls_test.rs @@ -77,16 +77,16 @@ async fn test_certificate_authenticate() { fn configs_with_tls_config(size: usize, tls_config: TlsConfig) -> Vec { iter::repeat(tls_config) .map(|tls_config| { - XlineServerConfig::new( - ClusterConfig::default(), - StorageConfig::default(), - LogConfig::default(), - TraceConfig::default(), - AuthConfig::default(), - CompactConfig::default(), - tls_config, - MetricsConfig::default(), - ) + XlineServerConfig::builder() + .cluster(ClusterConfig::default()) + .storage(StorageConfig::default()) + .log(LogConfig::default()) + .trace(TraceConfig::default()) + .auth(AuthConfig::default()) + .compact(CompactConfig::default()) + .tls(tls_config) + .metrics(MetricsConfig::default()) + .build() }) .take(size) .collect() @@ -101,14 +101,14 @@ fn basic_tls_client_config() -> ClientTlsConfig { fn basic_tls_configs(size: usize) -> Vec { configs_with_tls_config( size, - TlsConfig::new( - None, - Some(PathBuf::from("../../fixtures/server.crt")), - Some(PathBuf::from("../../fixtures/server.key")), - Some(PathBuf::from("../../fixtures/ca.crt")), - None, - None, - ), + TlsConfig::builder() + .peer_ca_cert_path(None) + .peer_cert_path(Some(PathBuf::from("../../fixtures/server.crt"))) + .peer_key_path(Some(PathBuf::from("../../fixtures/server.key"))) + .peer_ca_cert_path(Some(PathBuf::from("../../fixtures/ca.crt"))) + .client_cert_path(None) + .client_key_path(None) + .build(), ) } @@ -126,13 +126,13 @@ fn mtls_client_config(name: &str) -> ClientTlsConfig { fn mtls_configs(size: usize) -> Vec { configs_with_tls_config( size, - TlsConfig::new( - Some(PathBuf::from("../../fixtures/ca.crt")), - Some(PathBuf::from("../../fixtures/server.crt")), - Some(PathBuf::from("../../fixtures/server.key")), - Some(PathBuf::from("../../fixtures/ca.crt")), - Some(PathBuf::from("../../fixtures/root_client.crt")), - Some(PathBuf::from("../../fixtures/root_client.key")), - ), + TlsConfig::builder() + .peer_ca_cert_path(Some(PathBuf::from("../../fixtures/ca.crt"))) + .peer_cert_path(Some(PathBuf::from("../../fixtures/server.crt"))) + .peer_key_path(Some(PathBuf::from("../../fixtures/server.key"))) + .client_ca_cert_path(Some(PathBuf::from("../../fixtures/ca.crt"))) + .client_cert_path(Some(PathBuf::from("../../fixtures/root_client.crt"))) + .client_key_path(Some(PathBuf::from("../../fixtures/root_client.key"))) + .build(), ) } diff --git a/crates/xlinectl/src/main.rs b/crates/xlinectl/src/main.rs index 4eaea1928..52d16fd6d 100644 --- a/crates/xlinectl/src/main.rs +++ b/crates/xlinectl/src/main.rs @@ -268,15 +268,27 @@ async fn main() -> Result<()> { let matches = cli().get_matches(); let user_opt = parse_user(&matches)?; let endpoints = matches.get_many::("endpoints").expect("Required"); - let client_config = ClientConfig::new( - Duration::from_secs(*matches.get_one("wait_synced_timeout").expect("Required")), - Duration::from_secs(*matches.get_one("propose_timeout").expect("Required")), - Duration::from_millis(*matches.get_one("initial_retry_timeout").expect("Required")), - Duration::from_millis(*matches.get_one("max_retry_timeout").expect("Required")), - *matches.get_one("retry_count").expect("Required"), - true, - Duration::from_millis(*matches.get_one("keep_alive_interval").expect("Required")), - ); + + let client_config = ClientConfig::builder() + .wait_synced_timeout(Duration::from_secs( + *matches.get_one("wait_synced_timeout").expect("Required"), + )) + .propose_timeout(Duration::from_secs( + *matches.get_one("propose_timeout").expect("Required"), + )) + .initial_retry_timeout(Duration::from_millis( + *matches.get_one("initial_retry_timeout").expect("Required"), + )) + .max_retry_timeout(Duration::from_millis( + *matches.get_one("max_retry_timeout").expect("Required"), + )) + .retry_count(*matches.get_one("retry_count").expect("Required")) + .fixed_backoff(true) + .keep_alive_interval(Duration::from_millis( + *matches.get_one("keep_alive_interval").expect("Required"), + )) + .build(); + let ca_path: Option = matches.get_one("ca_cert_pem_path").cloned(); let tls_config = match ca_path { Some(path) => { From 5402ae475fca3c9f6405ccb4ad50c428472d26e3 Mon Sep 17 00:00:00 2001 From: Alan Tang Date: Mon, 9 Dec 2024 16:11:36 +0800 Subject: [PATCH 7/9] chore: bump hashbrown version Signed-off-by: Alan Tang --- Cargo.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b1e55a724..74adbd3c9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1209,9 +1209,9 @@ checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" [[package]] name = "hashbrown" -version = "0.15.0" +version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e087f84d4f86bf4b218b927129862374b72199ae7d8657835f1e89000eea4fb" +checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" [[package]] name = "heck" @@ -1395,7 +1395,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" dependencies = [ "equivalent", - "hashbrown 0.15.0", + "hashbrown 0.15.2", ] [[package]] From 157afd0992b715e314bbef22dab1cbaf4fcdfad6 Mon Sep 17 00:00:00 2001 From: Alan Tang Date: Tue, 10 Dec 2024 11:11:39 +0800 Subject: [PATCH 8/9] fix: inconsistent parameters passed for modifying TLS configuration Signed-off-by: Alan Tang --- crates/xline/tests/it/tls_test.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/xline/tests/it/tls_test.rs b/crates/xline/tests/it/tls_test.rs index 5e8f1a01c..9516e8a52 100644 --- a/crates/xline/tests/it/tls_test.rs +++ b/crates/xline/tests/it/tls_test.rs @@ -105,7 +105,7 @@ fn basic_tls_configs(size: usize) -> Vec { .peer_ca_cert_path(None) .peer_cert_path(Some(PathBuf::from("../../fixtures/server.crt"))) .peer_key_path(Some(PathBuf::from("../../fixtures/server.key"))) - .peer_ca_cert_path(Some(PathBuf::from("../../fixtures/ca.crt"))) + .client_ca_cert_path(Some(PathBuf::from("../../fixtures/ca.crt"))) .client_cert_path(None) .client_key_path(None) .build(), From e8ab1c7543260de2ec5e888dd7debc3104aa8876 Mon Sep 17 00:00:00 2001 From: Alan Tang Date: Tue, 10 Dec 2024 12:43:25 +0800 Subject: [PATCH 9/9] chore: bump reqwest and url version Signed-off-by: Alan Tang --- Cargo.lock | 300 +++++++++++++++++++++++++++++++++----- workspace-hack/Cargo.toml | 2 +- 2 files changed, 268 insertions(+), 34 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 74adbd3c9..7336817fa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -879,6 +879,17 @@ dependencies = [ "subtle", ] +[[package]] +name = "displaydoc" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "doc-comment" version = "0.3.3" @@ -1362,6 +1373,124 @@ dependencies = [ "cc", ] +[[package]] +name = "icu_collections" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locid" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_locid_transform" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_locid_transform_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_locid_transform_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" + +[[package]] +name = "icu_normalizer" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_normalizer_data", + "icu_properties", + "icu_provider", + "smallvec", + "utf16_iter", + "utf8_iter", + "write16", + "zerovec", +] + +[[package]] +name = "icu_normalizer_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" + +[[package]] +name = "icu_properties" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93d6020766cfc6302c15dbbc9c8778c37e62c14427cb7f6e601d849e092aeef5" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_locid_transform", + "icu_properties_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_properties_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" + +[[package]] +name = "icu_provider" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_provider_macros", + "stable_deref_trait", + "tinystr", + "writeable", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_provider_macros" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "ident_case" version = "1.0.1" @@ -1370,12 +1499,23 @@ checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] name = "idna" -version = "0.5.0" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +dependencies = [ + "idna_adapter", + "smallvec", + "utf8_iter", +] + +[[package]] +name = "idna_adapter" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" +checksum = "daca1df1c957320b2cf139ac61e7bd64fed304c5040df000a745aa1de3b4ef71" dependencies = [ - "unicode-bidi", - "unicode-normalization", + "icu_normalizer", + "icu_properties", ] [[package]] @@ -1550,6 +1690,12 @@ version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" +[[package]] +name = "litemap" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" + [[package]] name = "lock_api" version = "0.4.12" @@ -2395,9 +2541,9 @@ checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" [[package]] name = "reqwest" -version = "0.12.8" +version = "0.12.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f713147fbe92361e52392c73b8c9e48c04c6625bce969ef54dc901e58e042a7b" +checksum = "a77c62af46e79de0a562e1a9849205ffcb7fc1238876e9bd743357570e04046f" dependencies = [ "base64 0.22.1", "bytes", @@ -2742,6 +2888,12 @@ dependencies = [ "der", ] +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + [[package]] name = "strsim" version = "0.10.0" @@ -2816,6 +2968,17 @@ dependencies = [ "futures-core", ] +[[package]] +name = "synstructure" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "tempfile" version = "3.13.0" @@ -2908,20 +3071,15 @@ dependencies = [ ] [[package]] -name = "tinyvec" -version = "1.8.0" +name = "tinystr" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" +checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" dependencies = [ - "tinyvec_macros", + "displaydoc", + "zerovec", ] -[[package]] -name = "tinyvec_macros" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" - [[package]] name = "tokio" version = "1.40.0" @@ -3262,27 +3420,12 @@ version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" -[[package]] -name = "unicode-bidi" -version = "0.3.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ab17db44d7388991a428b2ee655ce0c212e862eff1768a455c58f9aad6e7893" - [[package]] name = "unicode-ident" version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e91b56cd4cadaeb79bbf1a5645f6b4f8dc5bde8834ad5894a8db35fda9efa1fe" -[[package]] -name = "unicode-normalization" -version = "0.1.24" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5033c97c4262335cded6d6fc3e5c18ab755e1a3dc96376350f3d8e9f009ad956" -dependencies = [ - "tinyvec", -] - [[package]] name = "unicode-width" version = "0.1.14" @@ -3297,15 +3440,27 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.5.2" +version = "2.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c" +checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" dependencies = [ "form_urlencoded", "idna", "percent-encoding", ] +[[package]] +name = "utf16_iter" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" + +[[package]] +name = "utf8_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" + [[package]] name = "utf8parse" version = "0.2.2" @@ -3677,6 +3832,18 @@ dependencies = [ "zeroize", ] +[[package]] +name = "write16" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" + +[[package]] +name = "writeable" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" + [[package]] name = "x509-certificate" version = "0.23.1" @@ -3856,6 +4023,30 @@ dependencies = [ "xline", ] +[[package]] +name = "yoke" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" +dependencies = [ + "serde", + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", + "synstructure", +] + [[package]] name = "zerocopy" version = "0.7.35" @@ -3877,6 +4068,27 @@ dependencies = [ "syn 2.0.79", ] +[[package]] +name = "zerofrom" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", + "synstructure", +] + [[package]] name = "zeroize" version = "1.8.1" @@ -3897,6 +4109,28 @@ dependencies = [ "syn 2.0.79", ] +[[package]] +name = "zerovec" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" +dependencies = [ + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "zstd-sys" version = "2.0.13+zstd.1.5.6" diff --git a/workspace-hack/Cargo.toml b/workspace-hack/Cargo.toml index 54f15db06..9224486ff 100644 --- a/workspace-hack/Cargo.toml +++ b/workspace-hack/Cargo.toml @@ -58,7 +58,7 @@ libc = { version = "0.2", features = ["extra_traits"] } log = { version = "0.4", default-features = false, features = ["std"] } memchr = { version = "2" } predicates = { version = "3", default-features = false, features = ["diff"] } -syn = { version = "2", features = ["extra-traits", "full", "visit", "visit-mut"] } +syn = { version = "2", features = ["extra-traits", "fold", "full", "visit", "visit-mut"] } tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "rt-multi-thread", "signal", "sync", "time"] } ### END HAKARI SECTION