diff --git a/crates/curp-external-api/src/cmd.rs b/crates/curp-external-api/src/cmd.rs index bbec7a288..e16619598 100644 --- a/crates/curp-external-api/src/cmd.rs +++ b/crates/curp-external-api/src/cmd.rs @@ -31,6 +31,7 @@ pub trait Command: pri::Serializable + ConflictCheck + PbCodec { type Error: pri::Serializable + PbCodec + std::error::Error; /// K (key) is used to tell confliction + /// /// The key can be a single key or a key range type K: pri::Serializable + Eq + Hash + ConflictCheck; @@ -52,6 +53,7 @@ pub trait Command: pri::Serializable + ConflictCheck + PbCodec { /// Prepare the command /// /// # Errors + /// /// Return `Self::Error` when `CommandExecutor::prepare` goes wrong #[inline] fn prepare(&self, e: &E) -> Result @@ -64,6 +66,7 @@ pub trait Command: pri::Serializable + ConflictCheck + PbCodec { /// Execute the command according to the executor /// /// # Errors + /// /// Return `Self::Error` when `CommandExecutor::execute` goes wrong #[inline] async fn execute(&self, e: &E) -> Result @@ -76,6 +79,7 @@ pub trait Command: pri::Serializable + ConflictCheck + PbCodec { /// Execute the command after_sync callback /// /// # Errors + /// /// Return `Self::Error` when `CommandExecutor::after_sync` goes wrong #[inline] async fn after_sync( @@ -116,6 +120,7 @@ impl ConflictCheck for u32 { } /// Command executor which actually executes the command. +/// /// It is usually defined by the protocol user. #[async_trait] pub trait CommandExecutor: pri::ThreadSafe @@ -125,18 +130,21 @@ where /// Prepare the command /// /// # Errors + /// /// This function may return an error if there is a problem preparing the command. fn prepare(&self, cmd: &C) -> Result; /// Execute the command /// /// # Errors + /// /// This function may return an error if there is a problem executing the command. async fn execute(&self, cmd: &C) -> Result; /// Execute the after_sync callback /// /// # Errors + /// /// This function may return an error if there is a problem executing the after_sync callback. async fn after_sync( &self, @@ -148,24 +156,28 @@ where /// Set the index of the last log entry that has been successfully applied to the command executor /// /// # Errors + /// /// Returns an error if setting the last applied log entry fails. fn set_last_applied(&self, index: LogIndex) -> Result<(), C::Error>; /// Get the index of the last log entry that has been successfully applied to the command executor /// /// # Errors + /// /// Returns an error if retrieval of the last applied log entry fails. fn last_applied(&self) -> Result; /// Take a snapshot /// /// # Errors + /// /// This function may return an error if there is a problem taking a snapshot. async fn snapshot(&self) -> Result; /// Reset the command executor using the snapshot or to the initial state if None /// /// # Errors + /// /// This function may return an error if there is a problem resetting the command executor. async fn reset(&self, snapshot: Option<(Snapshot, LogIndex)>) -> Result<(), C::Error>; diff --git a/crates/curp/src/client/mod.rs b/crates/curp/src/client/mod.rs index 17ded1a7d..92aa8c4ae 100644 --- a/crates/curp/src/client/mod.rs +++ b/crates/curp/src/client/mod.rs @@ -97,6 +97,7 @@ pub trait ClientApi { /// Send fetch cluster requests to all servers (That's because initially, we didn't /// know who the leader is.) + /// /// Note: The fetched cluster may still be outdated if `linearizable` is false async fn fetch_cluster(&self, linearizable: bool) -> Result; diff --git a/crates/curp/src/client/retry.rs b/crates/curp/src/client/retry.rs index e5cb550b5..9c716341b 100644 --- a/crates/curp/src/client/retry.rs +++ b/crates/curp/src/client/retry.rs @@ -279,6 +279,7 @@ where /// Send fetch cluster requests to all servers (That's because initially, we didn't /// know who the leader is.) + /// /// Note: The fetched cluster may still be outdated if `linearizable` is false async fn fetch_cluster( &self, diff --git a/crates/curp/src/client/tests.rs b/crates/curp/src/client/tests.rs index a50df9ce0..06807203b 100644 --- a/crates/curp/src/client/tests.rs +++ b/crates/curp/src/client/tests.rs @@ -820,6 +820,7 @@ impl ConnectApi for MockedStreamConnectApi { } /// Create mocked stream connects +/// /// The leader is S0 #[allow(trivial_casts)] // cannot be inferred fn init_mocked_stream_connects( diff --git a/crates/curp/src/client/unary.rs b/crates/curp/src/client/unary.rs index fdccfdf61..e13e5284d 100644 --- a/crates/curp/src/client/unary.rs +++ b/crates/curp/src/client/unary.rs @@ -24,6 +24,7 @@ pub(super) struct UnaryConfig { /// The rpc timeout of a propose request propose_timeout: Duration, /// The rpc timeout of a 2-RTT request, usually takes longer than propose timeout + /// /// The recommended the values is within (propose_timeout, 2 * propose_timeout]. wait_synced_timeout: Duration, } @@ -60,9 +61,13 @@ impl Unary { } /// Get a handle `f` and apply to the leader + /// /// NOTICE: + /// /// The leader might be outdate if the local state is stale. + /// /// `map_leader` should never be invoked in [`ClientApi::fetch_cluster`] + /// /// `map_leader` might call `fetch_leader_id`, `fetch_cluster`, finally /// result in stack overflow. async fn map_leader>>( diff --git a/crates/curp/src/members.rs b/crates/curp/src/members.rs index 37a39c115..ce2045451 100644 --- a/crates/curp/src/members.rs +++ b/crates/curp/src/members.rs @@ -98,7 +98,9 @@ impl ClusterInfo { } /// Construct a new `ClusterInfo` from members map + /// /// # Panics + /// /// panic if `all_members` is empty #[inline] #[must_use] @@ -131,7 +133,9 @@ impl ClusterInfo { } /// Construct a new `ClusterInfo` from `FetchClusterResponse` + /// /// # Panics + /// /// panic if `cluster.members` doesn't contain `self_addr` #[inline] #[must_use] @@ -234,7 +238,9 @@ impl ClusterInfo { } /// Get the current member + /// /// # Panics + /// /// panic if self member id is not in members #[allow(clippy::unwrap_used)] // self member id must be in members #[must_use] @@ -257,7 +263,7 @@ impl ClusterInfo { self.self_member().client_urls.clone() } - /// Get the current server id + /// Get the current server name #[must_use] #[inline] pub fn self_name(&self) -> String { diff --git a/crates/curp/src/rpc/connect.rs b/crates/curp/src/rpc/connect.rs index cc4a4d175..0975e3687 100644 --- a/crates/curp/src/rpc/connect.rs +++ b/crates/curp/src/rpc/connect.rs @@ -631,6 +631,7 @@ impl BypassedConnect { const BYPASS_KEY: &str = "bypass"; /// Inject bypassed message into a request's metadata and check if it is a bypassed request. +/// /// A bypass request can skip the check for lease expiration (there will never be a disconnection from oneself). pub(crate) trait Bypass { /// Inject into metadata diff --git a/crates/curp/src/rpc/mod.rs b/crates/curp/src/rpc/mod.rs index e4d570dc1..2cd69a50a 100644 --- a/crates/curp/src/rpc/mod.rs +++ b/crates/curp/src/rpc/mod.rs @@ -157,7 +157,9 @@ impl ProposeRequest { } /// Get command + /// /// # Errors + /// /// Return error if the command can't be decoded #[inline] pub fn cmd(&self) -> Result { @@ -619,6 +621,7 @@ impl PublishRequest { } /// NOTICE: +/// /// Please check test case `test_unary_fast_round_return_early_err` `test_unary_propose_return_early_err` /// `test_retry_propose_return_no_retry_error` `test_retry_propose_return_retry_error` if you added some /// new [`CurpError`] @@ -891,6 +894,7 @@ impl From> for PoolEntryInner { } /// Command Id wrapper, which is used to identify a command +/// /// The underlying data is a tuple of (`client_id`, `seq_num`) #[derive( Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize, Ord, PartialOrd, Default, diff --git a/crates/curp/src/server/cmd_worker/conflict_checked_mpmc.rs b/crates/curp/src/server/cmd_worker/conflict_checked_mpmc.rs index 4946dc479..afdf1b634 100644 --- a/crates/curp/src/server/cmd_worker/conflict_checked_mpmc.rs +++ b/crates/curp/src/server/cmd_worker/conflict_checked_mpmc.rs @@ -25,7 +25,9 @@ use crate::{ /// Cart mod cart { /// Cart is a utility that acts as a temporary container. + /// /// It is usually filled by the provider and consumed by the customer. + /// /// This is useful when we are sure that the provider will fill the cart and the cart will be consumed by the customer /// so that we don't need to check whether there is something in the `Option`. #[derive(Debug)] @@ -188,6 +190,7 @@ enum OnceState { } /// The filter will block any msg if its predecessors(msgs that arrive earlier and conflict with it) haven't finished process +/// /// Internally it maintains a dependency graph of conflicting cmds struct Filter { @@ -318,6 +321,7 @@ impl> Filter { } /// Update the vertex, see if it can progress + /// /// Return true if it can be removed #[allow(clippy::expect_used, clippy::too_many_lines)] // TODO: split this function fn update_vertex(&mut self, vid: u64) -> bool { diff --git a/crates/curp/src/server/curp_node.rs b/crates/curp/src/server/curp_node.rs index c4818da9a..f0a4e5857 100644 --- a/crates/curp/src/server/curp_node.rs +++ b/crates/curp/src/server/curp_node.rs @@ -722,8 +722,11 @@ impl CurpNode { } /// Candidate or pre candidate broadcasts votes - /// Return `Some(vote)` if bcast pre vote and success - /// Return `None` if bcast pre vote and fail or bcast vote + /// + /// # Returns + /// + /// - `Some(vote)` if bcast pre vote and success + /// - `None` if bcast pre vote and fail or bcast vote async fn bcast_vote(curp: &RawCurp, vote: Vote) -> Option { if vote.is_pre_vote { debug!("{} broadcasts pre votes to all servers", curp.id()); diff --git a/crates/curp/src/server/lease_manager.rs b/crates/curp/src/server/lease_manager.rs index c16381cfb..840e2fb07 100644 --- a/crates/curp/src/server/lease_manager.rs +++ b/crates/curp/src/server/lease_manager.rs @@ -13,6 +13,7 @@ const DEFAULT_LEASE_TTL: Duration = Duration::from_secs(8); /// Lease manager pub(crate) struct LeaseManager { /// client_id => expired_at + /// /// expiry queue to check the smallest expired_at pub(super) expiry_queue: PriorityQueue>, } diff --git a/crates/curp/src/server/mod.rs b/crates/curp/src/server/mod.rs index 8ee55a599..29f7b7f84 100644 --- a/crates/curp/src/server/mod.rs +++ b/crates/curp/src/server/mod.rs @@ -59,6 +59,7 @@ mod metrics; pub use storage::{db::DB, StorageApi, StorageError}; /// The Rpc Server to handle rpc requests +/// /// This Wrapper is introduced due to the `MadSim` rpc lib #[derive(Debug)] pub struct Rpc { @@ -231,7 +232,9 @@ impl crate::rpc::InnerProtocol for Rpc { impl Rpc { /// New `Rpc` + /// /// # Panics + /// /// Panic if storage creation failed #[inline] #[allow(clippy::too_many_arguments)] // TODO: refactor this use builder pattern @@ -278,8 +281,9 @@ impl Rpc { /// Run a new rpc server on a specific addr, designed to be used in the tests /// /// # Errors - /// `ServerError::ParsingError` if parsing failed for the local server address - /// `ServerError::RpcError` if any rpc related error met + /// + /// - `ServerError::ParsingError` if parsing failed for the local server address + /// - `ServerError::RpcError` if any rpc related error met #[cfg(madsim)] #[allow(clippy::too_many_arguments)] #[inline] diff --git a/crates/curp/src/server/raw_curp/log.rs b/crates/curp/src/server/raw_curp/log.rs index f7996649f..274d51eb3 100644 --- a/crates/curp/src/server/raw_curp/log.rs +++ b/crates/curp/src/server/raw_curp/log.rs @@ -70,7 +70,9 @@ impl From> for LogRange { } /// Curp logs +/// /// There exists a fake log entry 0 whose term equals 0 +/// /// For the leader, there should never be a gap between snapshot and entries /// /// Examples: diff --git a/crates/curp/src/tracker.rs b/crates/curp/src/tracker.rs index 09ecc18ec..d76edb4dc 100644 --- a/crates/curp/src/tracker.rs +++ b/crates/curp/src/tracker.rs @@ -14,6 +14,7 @@ const USIZE_BITS: usize = std::mem::size_of::() * 8; const DEFAULT_BIT_VEC_QUEUE_CAP: usize = 1024; /// A one-direction bit vector queue +/// /// It use a ring buffer `VecDeque` to store bits /// /// Memory Layout: @@ -161,7 +162,9 @@ impl BitVecQueue { } /// Split this bit vec queue to `at` + /// /// e.g. + /// /// 001100 -> `split_at(2)` -> 1100 fn split_at(&mut self, at: usize) { if self.store.is_empty() { diff --git a/crates/engine/src/api/engine_api.rs b/crates/engine/src/api/engine_api.rs index 4a7058b03..999e56352 100644 --- a/crates/engine/src/api/engine_api.rs +++ b/crates/engine/src/api/engine_api.rs @@ -14,7 +14,9 @@ pub trait StorageEngine: Send + Sync + 'static + std::fmt::Debug { fn transaction(&self) -> Self::Transaction<'_>; /// Get all the values of the given table + /// /// # Errors + /// /// Return `EngineError::TableNotFound` if the given table does not exist /// Return `EngineError` if met some errors #[allow(clippy::type_complexity)] // it's clear that (Vec, Vec) is a key-value pair @@ -23,6 +25,7 @@ pub trait StorageEngine: Send + Sync + 'static + std::fmt::Debug { /// Get a snapshot of the current state of the database /// /// # Errors + /// /// Return `EngineError` if met some errors when creating the snapshot fn get_snapshot( &self, @@ -33,6 +36,7 @@ pub trait StorageEngine: Send + Sync + 'static + std::fmt::Debug { /// Apply a snapshot to the database /// /// # Errors + /// /// Return `EngineError` if met some errors when applying the snapshot async fn apply_snapshot( &self, @@ -46,6 +50,7 @@ pub trait StorageEngine: Send + Sync + 'static + std::fmt::Debug { /// Get the file size of the engine (Measured in bytes) /// /// # Errors + /// /// Return `EngineError` if met some errors when get file size fn file_size(&self) -> Result; } diff --git a/crates/engine/src/metrics.rs b/crates/engine/src/metrics.rs index b52960b1b..17f5d01e2 100644 --- a/crates/engine/src/metrics.rs +++ b/crates/engine/src/metrics.rs @@ -39,7 +39,9 @@ impl Layer { impl Layer { /// Apply snapshot from file, only works for `RocksEngine` + /// /// # Errors + /// /// Return `EngineError` when `RocksDB` returns an error. #[inline] pub async fn apply_snapshot_from_file( @@ -68,9 +70,11 @@ where } /// Get all the values of the given table + /// /// # Errors - /// Return `EngineError::TableNotFound` if the given table does not exist - /// Return `EngineError` if met some errors + /// + /// - Return `EngineError::TableNotFound` if the given table does not exist + /// - Return `EngineError` if met some errors #[allow(clippy::type_complexity)] // it's clear that (Vec, Vec) is a key-value pair fn get_all(&self, table: &str) -> Result, Vec)>, EngineError> { self.engine.get_all(table) @@ -79,6 +83,7 @@ where /// Get a snapshot of the current state of the database /// /// # Errors + /// /// Return `EngineError` if met some errors when creating the snapshot fn get_snapshot( &self, @@ -91,6 +96,7 @@ where /// Apply a snapshot to the database /// /// # Errors + /// /// Return `EngineError` if met some errors when applying the snapshot async fn apply_snapshot( &self, diff --git a/crates/engine/src/mock_rocksdb_engine.rs b/crates/engine/src/mock_rocksdb_engine.rs index c9f433839..30e75ec24 100644 --- a/crates/engine/src/mock_rocksdb_engine.rs +++ b/crates/engine/src/mock_rocksdb_engine.rs @@ -172,7 +172,9 @@ impl RocksSnapshot { } /// Create a new mock snapshot for receiving + /// /// # Errors + /// /// Return `EngineError` when create directory failed. #[inline] #[allow(clippy::unnecessary_wraps)] // the real rocksdb engine need the Result wrap diff --git a/crates/engine/src/proxy.rs b/crates/engine/src/proxy.rs index 838146e18..6952d1667 100644 --- a/crates/engine/src/proxy.rs +++ b/crates/engine/src/proxy.rs @@ -35,7 +35,9 @@ pub enum Engine { impl Engine { /// Create a new `Engine` instance + /// /// # Errors + /// /// Return `EngineError` when DB open failed. #[inline] pub fn new(engine_type: EngineType, tables: &[&'static str]) -> Result { @@ -48,7 +50,9 @@ impl Engine { } /// Apply snapshot from file, only works for `RocksEngine` + /// /// # Errors + /// /// Return `EngineError` when `RocksDB` returns an error. #[inline] pub async fn apply_snapshot_from_file( @@ -258,7 +262,9 @@ pub enum Snapshot { impl Snapshot { /// Create a new `Snapshot` instance + /// /// # Errors + /// /// Return `EngineError` when DB open failed. #[inline] pub fn new_for_receiving(engine_type: EngineType) -> Result { diff --git a/crates/engine/src/rocksdb_engine/mod.rs b/crates/engine/src/rocksdb_engine/mod.rs index 0fdffcbe3..28270bdb3 100644 --- a/crates/engine/src/rocksdb_engine/mod.rs +++ b/crates/engine/src/rocksdb_engine/mod.rs @@ -91,7 +91,9 @@ impl RocksEngine { } /// Get the total sst file size of all tables + /// /// # WARNING + /// /// This method need to flush memtable to disk. it may be slow. do not call it frequently. fn get_db_size, V: AsRef<[T]>>( db: &OptimisticTransactionDB, @@ -455,7 +457,9 @@ impl RocksSnapshot { } /// Create a new snapshot for receiving + /// /// # Errors + /// /// Return `EngineError` when create directory failed. #[inline] pub fn new_for_receiving

(dir: P) -> Result @@ -470,7 +474,9 @@ impl RocksSnapshot { } /// Create a new snapshot for sending + /// /// # Errors + /// /// Return `EngineError` when read directory failed. #[inline] pub fn new_for_sending

(dir: P) -> Result diff --git a/crates/utils/src/config.rs b/crates/utils/src/config.rs index d37b3c088..af947fe08 100644 --- a/crates/utils/src/config.rs +++ b/crates/utils/src/config.rs @@ -293,6 +293,7 @@ pub struct CurpConfig { 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()")] @@ -305,13 +306,16 @@ pub struct CurpConfig { 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")] diff --git a/crates/utils/src/parser.rs b/crates/utils/src/parser.rs index e7e4e8520..15c0d7182 100644 --- a/crates/utils/src/parser.rs +++ b/crates/utils/src/parser.rs @@ -40,7 +40,9 @@ pub enum ConfigFileError { } /// parse members from string like "node1=addr1,addr2,node2=add3,addr4,addr5,node3=addr6" +/// /// # Errors +/// /// Return error when pass wrong args #[inline] pub fn parse_members(s: &str) -> Result>, ConfigParseError> { @@ -69,7 +71,9 @@ pub fn parse_members(s: &str) -> Result>, ConfigPars } /// Parse `ClusterRange` from the given string +/// /// # Errors +/// /// Return error when parsing the given string to `ClusterRange` failed #[inline] pub fn parse_range(s: &str) -> Result { @@ -86,7 +90,9 @@ pub fn parse_range(s: &str) -> Result { } /// Parse `Duration` from string +/// /// # Errors +/// /// Return error when parsing the given string to `Duration` failed #[inline] pub fn parse_duration(s: &str) -> Result { @@ -150,7 +156,9 @@ pub fn parse_duration(s: &str) -> Result { } /// Parse `InitialClusterState` from string +/// /// # Errors +/// /// Return error when parsing the given string to `InitialClusterState` failed #[inline] pub fn parse_state(s: &str) -> Result { @@ -164,7 +172,9 @@ pub fn parse_state(s: &str) -> Result { } /// Parse `LOG_PATH` from string +/// /// # Errors +/// /// Return error when parsing the given string to `PathBuf` failed #[inline] pub fn parse_log_file(s: &str) -> Result { @@ -195,7 +205,9 @@ pub fn parse_log_file(s: &str) -> Result { } /// Parse `LevelConfig` from string +/// /// # Errors +/// /// Return error when parsing the given string to `LevelConfig` failed #[inline] pub fn parse_log_level(s: &str) -> Result { @@ -212,7 +224,9 @@ pub fn parse_log_level(s: &str) -> Result { } /// Parse `RotationConfig` from string +/// /// # Errors +/// /// Return error when parsing the given string to `RotationConfig` failed #[inline] pub fn parse_rotation(s: &str) -> Result { @@ -227,7 +241,9 @@ pub fn parse_rotation(s: &str) -> Result { } /// Parse bytes from string +/// /// # Errors +/// /// Return error when parsing the given string to usize failed #[inline] #[allow(clippy::arithmetic_side_effects)] @@ -264,7 +280,9 @@ pub fn parse_batch_bytes(s: &str) -> Result { } /// Get the metrics push protocol +/// /// # Errors +/// /// Return error when parsing the given string to `MetricsPushProtocol` failed #[inline] pub fn parse_metrics_push_protocol(s: &str) -> Result { diff --git a/crates/xline/src/id_gen.rs b/crates/xline/src/id_gen.rs index a84addc48..980be97e4 100644 --- a/crates/xline/src/id_gen.rs +++ b/crates/xline/src/id_gen.rs @@ -8,6 +8,7 @@ use clippy_utilities::{NumericCast, OverflowArithmetic}; use curp::members::ServerId; /// Generator of unique id +/// /// id format: /// | prefix | suffix | /// | 2 bytes | 5 bytes | 1 byte | diff --git a/crates/xline/src/restore.rs b/crates/xline/src/restore.rs index 2801d5867..dcb6f1818 100644 --- a/crates/xline/src/restore.rs +++ b/crates/xline/src/restore.rs @@ -10,9 +10,11 @@ use utils::table_names::XLINE_TABLES; use crate::server::MAINTENANCE_SNAPSHOT_CHUNK_SIZE; /// Restore snapshot to data dir +/// /// # Errors -/// return `ClientError::IoError` if meet io errors -/// return `ClientError::EngineError` if meet engine errors +/// +/// - return `ClientError::IoError` if meet io errors +/// - return `ClientError::EngineError` if meet engine errors #[inline] #[allow(clippy::indexing_slicing)] // safe operation pub async fn restore, D: Into>( diff --git a/crates/xline/src/server/kv_server.rs b/crates/xline/src/server/kv_server.rs index 0c05da1c3..ff47f7e4b 100644 --- a/crates/xline/src/server/kv_server.rs +++ b/crates/xline/src/server/kv_server.rs @@ -227,6 +227,7 @@ impl Kv for KvServer { } /// Put puts the given key into the key-value store. + /// /// A put request increments the revision of the key-value store /// and generates one event in the event history. #[instrument(skip_all)] @@ -256,6 +257,7 @@ impl Kv for KvServer { } /// DeleteRange deletes the given range from the key-value store. + /// /// A delete request increments the revision of the key-value store /// and generates a delete event in the event history for every deleted key. #[instrument(skip_all)] @@ -285,6 +287,7 @@ impl Kv for KvServer { } /// Txn processes multiple requests in a single transaction. + /// /// A txn request increments the revision of the key-value store /// and generates events with the same revision for every completed request. /// It is not allowed to modify the same key several times within one txn. diff --git a/crates/xline/src/server/xline_server.rs b/crates/xline/src/server/xline_server.rs index 08a68d157..cc8be70c4 100644 --- a/crates/xline/src/server/xline_server.rs +++ b/crates/xline/src/server/xline_server.rs @@ -97,7 +97,9 @@ pub struct XlineServer { impl XlineServer { /// New `XlineServer` + /// /// # Errors + /// /// Return error if init cluster info failed #[inline] pub async fn new( diff --git a/crates/xline/src/storage/db.rs b/crates/xline/src/storage/db.rs index 8d442edf9..c7c47e0df 100644 --- a/crates/xline/src/storage/db.rs +++ b/crates/xline/src/storage/db.rs @@ -42,6 +42,7 @@ impl DB { /// Create a new `DB` /// /// # Errors + /// /// Return `ExecuteError::DbError` when open db failed #[inline] pub fn open(config: &EngineConfig) -> Result, ExecuteError> { diff --git a/crates/xline/src/storage/revision.rs b/crates/xline/src/storage/revision.rs index b1e466b23..ebe9c5766 100644 --- a/crates/xline/src/storage/revision.rs +++ b/crates/xline/src/storage/revision.rs @@ -68,7 +68,9 @@ impl Revision { } /// Decode `Revision` from `&[u8]` + /// /// # Panics + /// /// This function panics if there is not enough remaining data in `buf`. #[must_use] #[inline] diff --git a/crates/xline/src/utils/args.rs b/crates/xline/src/utils/args.rs index f16c55002..f8b6d44c8 100644 --- a/crates/xline/src/utils/args.rs +++ b/crates/xline/src/utils/args.rs @@ -348,7 +348,9 @@ impl From for XlineServerConfig { } /// Parse config from command line arguments or config file +/// /// # Errors +/// /// Return error if parse failed #[inline] pub async fn parse_config() -> Result { diff --git a/crates/xline/src/utils/metrics.rs b/crates/xline/src/utils/metrics.rs index 97a22896d..8d500dca7 100644 --- a/crates/xline/src/utils/metrics.rs +++ b/crates/xline/src/utils/metrics.rs @@ -5,7 +5,9 @@ use tracing::info; use utils::config::{MetricsConfig, MetricsPushProtocol}; /// Start metrics server +/// /// # Errors +/// /// Return error if init failed #[inline] pub fn init_metrics(config: &MetricsConfig) -> anyhow::Result<()> { diff --git a/crates/xlineapi/proto b/crates/xlineapi/proto index 769ab7e09..4b5a0075e 160000 --- a/crates/xlineapi/proto +++ b/crates/xlineapi/proto @@ -1 +1 @@ -Subproject commit 769ab7e09ea3976f5b95ca101326a424b4abd08e +Subproject commit 4b5a0075e144944c0a534580081245b2906085ea diff --git a/crates/xlineapi/src/command.rs b/crates/xlineapi/src/command.rs index ab28daa17..fe057a4ea 100644 --- a/crates/xlineapi/src/command.rs +++ b/crates/xlineapi/src/command.rs @@ -15,6 +15,7 @@ use crate::{ }; /// The curp client trait object on the command of xline +/// /// TODO: use `type CurpClient = impl ClientApi<...>` when `type_alias_impl_trait` stabilized pub type CurpClient = dyn ClientApi + Sync + Send + 'static; @@ -125,6 +126,7 @@ impl KeyRange { } /// Get end of range with prefix + /// /// User will provide a start key when prefix is true, we need calculate the end key of `KeyRange` #[allow(clippy::indexing_slicing)] // end[i] is always valid #[must_use]