├── .gitignore ├── examples └── rmqttraft-warp-memstore │ ├── Cargo.toml │ └── src │ └── main.rs ├── proto └── raft_service.proto ├── LICENSE ├── Cargo.toml ├── src ├── error.rs ├── lib.rs ├── raft_service.rs ├── timeout_recorder.rs ├── storage.rs ├── raft_server.rs ├── message.rs ├── raft.rs └── raft_node.rs └── README.md /.gitignore: -------------------------------------------------------------------------------- 1 | # Default ignored files 2 | Cargo.lock 3 | /target 4 | /.git 5 | /.idea 6 | /.cargo 7 | /examples/rmqttraft-warp-memstore/target 8 | /examples/rmqttraft-warp-memstore/Cargo.lock 9 | /examples/rmqttraft-warp-memstore/.idea/.gitignore 10 | /examples/rmqttraft-warp-memstore/.idea/encodings.xml 11 | /examples/rmqttraft-warp-memstore/.idea/modules.xml 12 | /examples/rmqttraft-warp-memstore/.idea/rmqttraft-warp-memstore.iml 13 | /examples/rmqttraft-warp-memstore/.idea/vcs.xml 14 | -------------------------------------------------------------------------------- /examples/rmqttraft-warp-memstore/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "rmqttraft-warp-memstore" 3 | version = "0.1.0" 4 | authors = ["rmqtt-rs "] 5 | edition = "2018" 6 | 7 | # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html 8 | 9 | [dependencies] 10 | async-trait = "0.1.48" 11 | bincode = "1.3" 12 | log = "0.4" 13 | rmqtt-raft = { path = "../..", features = ["reuse"] } 14 | serde = "1.0" 15 | slog-async = "2" 16 | slog-term = "2" 17 | slog = "2" 18 | slog-stdlog = "4" 19 | slog-scope = "4" 20 | structopt = "0.3" 21 | tokio = { version = "1", features = ["rt-multi-thread", "macros"] , default-features = false} 22 | warp = "0.3" 23 | chrono = { version = "0.4", default-features = false, features = ["clock"] } 24 | once_cell = "1.7.2" 25 | -------------------------------------------------------------------------------- /proto/raft_service.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package raftservice; 3 | 4 | //import "eraftpb.proto"; 5 | 6 | service RaftService { 7 | rpc RequestId(Empty) returns (IdRequestReponse) {} 8 | rpc ChangeConfig(ConfChange) returns (RaftResponse) {} 9 | rpc SendMessage(Message) returns (RaftResponse) {} 10 | rpc SendProposal(Proposal) returns (RaftResponse) {} 11 | rpc SendQuery(Query) returns (RaftResponse) {} 12 | } 13 | 14 | message ConfChange { 15 | bytes inner = 1; 16 | } 17 | 18 | message Message { 19 | bytes inner = 1; 20 | } 21 | 22 | enum ResultCode { 23 | Ok = 0; 24 | Error = 1; 25 | WrongLeader = 2; 26 | } 27 | 28 | message Proposal { 29 | bytes inner = 1; 30 | } 31 | 32 | message Query { 33 | bytes inner = 1; 34 | } 35 | 36 | message IdRequestReponse{ 37 | ResultCode code = 1; 38 | bytes data = 2; 39 | } 40 | 41 | message Empty {} 42 | 43 | message Entry { 44 | uint64 key = 1; 45 | string value = 2; 46 | } 47 | 48 | message RaftResponse { 49 | bytes inner = 2; 50 | } 51 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2022 rmqtt-rs 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "rmqtt-raft" 3 | version = "0.5.3" 4 | authors = ["rmqtt "] 5 | edition = "2021" 6 | license = "MIT OR Apache-2.0" 7 | repository = "https://github.com/rmqtt/rmqtt-raft" 8 | homepage = "https://github.com/rmqtt/rmqtt-raft" 9 | description = "rmqtt-raft - A raft framework, for regular people" 10 | keywords = ["raft", "distributed-systems"] 11 | categories = ["algorithms"] 12 | exclude = ["examples", ".gitignore", ".cargo/config"] 13 | 14 | [package.metadata.docs.rs] 15 | all-features = true 16 | 17 | [features] 18 | default = [] 19 | reuse = ["reuseport", "reuseaddr"] 20 | reuseport = ["socket2", "tokio-stream"] 21 | reuseaddr = ["socket2", "tokio-stream"] 22 | 23 | [dependencies] 24 | tikv-raft = { package = "raft", version = "0.7", features = [ 25 | "prost-codec", 26 | ], default-features = false } 27 | tokio = { version = "1", default-features = false, features = ["macros"] } 28 | socket2 = { version = "0.5", features = ["all"], optional = true } 29 | tokio-stream = { version = "0.1", features = ["net"], optional = true } 30 | tonic = "0.9" 31 | prost = "0.11" 32 | futures = "0.3" 33 | async-trait = "0.1" 34 | bincode = "1.3" 35 | serde = { version = "1.0", features = ["derive"] } 36 | log = { version = "0.4", features = ["std"] } 37 | slog = "2" 38 | thiserror = "1.0" 39 | dashmap = "6.1" 40 | ahash = "0.8" 41 | chrono = { version = "0.4", default-features = false, features = ["clock"] } 42 | anyhow = "1.0" 43 | once_cell = "1" 44 | bytestring = { version = "1.3", features = ["serde"] } 45 | scopeguard = "1" 46 | box-counter = { version = "0.3", features = ["count", "rate"] } 47 | 48 | [build-dependencies] 49 | tonic-build = "0.9" 50 | -------------------------------------------------------------------------------- /src/error.rs: -------------------------------------------------------------------------------- 1 | use thiserror::Error as ThisError; 2 | 3 | pub type Result = std::result::Result; 4 | 5 | #[allow(clippy::result_large_err)] 6 | #[derive(Debug, ThisError)] 7 | pub enum Error { 8 | #[error("raft error: `{0}`")] 9 | RaftError(#[from] tikv_raft::Error), 10 | #[error("Error joining the cluster")] 11 | JoinError, 12 | #[error("gprc error: `{0}`")] 13 | Grpc(#[from] tonic::transport::Error), 14 | #[error("error calling remote procedure: `{0}`")] 15 | RemoteCall(#[from] tonic::Status), 16 | #[error("io error: {0}")] 17 | Io(String), 18 | #[error("unexpected error, {0}")] 19 | Other(#[source] Box), 20 | #[error("unexpected error")] 21 | Unknown, 22 | #[error("too busy")] 23 | Busy, 24 | #[error("leader does not exist")] 25 | LeaderNotExist, 26 | #[error("Not a Leader")] 27 | NotLeader, 28 | #[error("timeout")] 29 | Elapsed, 30 | #[error("{0}")] 31 | Msg(String), 32 | #[error("send error, {0}")] 33 | SendError(String), 34 | #[error("recv error, {0}")] 35 | RecvError(String), 36 | #[error("{0}")] 37 | Anyhow(anyhow::Error), 38 | } 39 | 40 | impl Error { 41 | pub fn boxed(self) -> Box { 42 | Box::new(self) 43 | } 44 | } 45 | 46 | impl From for Error { 47 | fn from(e: prost::DecodeError) -> Self { 48 | Self::Other(Box::new(e)) 49 | } 50 | } 51 | 52 | impl From for Error { 53 | fn from(e: prost::EncodeError) -> Self { 54 | Self::Other(Box::new(e)) 55 | } 56 | } 57 | 58 | impl From for Error { 59 | fn from(e: tokio::io::Error) -> Self { 60 | Self::Io(e.to_string()) 61 | } 62 | } 63 | 64 | impl From for Error { 65 | fn from(e: bincode::Error) -> Self { 66 | Self::Other(e) 67 | } 68 | } 69 | 70 | impl From for Error { 71 | fn from(e: std::string::FromUtf8Error) -> Self { 72 | Self::Other(Box::new(e)) 73 | } 74 | } 75 | 76 | impl From for Error { 77 | fn from(e: String) -> Self { 78 | Self::Msg(e) 79 | } 80 | } 81 | 82 | impl From<&str> for Error { 83 | fn from(e: &str) -> Self { 84 | Self::Msg(e.to_owned()) 85 | } 86 | } 87 | 88 | impl From for Error { 89 | #[inline] 90 | fn from(e: anyhow::Error) -> Self { 91 | Error::Anyhow(e) 92 | } 93 | } 94 | impl From for Error { 95 | #[inline] 96 | fn from(_: tokio::time::error::Elapsed) -> Self { 97 | Error::Elapsed 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | #![allow(clippy::result_large_err)] 2 | 3 | use std::time::Duration; 4 | 5 | // Re-exporting necessary types and modules for external use. 6 | pub use crate::error::{Error, Result}; 7 | pub use crate::message::Status; 8 | pub use crate::raft::{Mailbox, Raft, Store}; 9 | pub use tikv_raft::{ReadOnlyOption, StateRole}; 10 | 11 | // Importing modules for internal use. 12 | mod error; 13 | mod message; 14 | mod raft; 15 | mod raft_node; 16 | mod raft_server; 17 | mod raft_service; 18 | mod storage; 19 | mod timeout_recorder; 20 | 21 | /// Configuration options for the Raft-based system. 22 | #[derive(Clone)] 23 | pub struct Config { 24 | #[cfg(feature = "reuseaddr")] 25 | /// Whether to reuse local addresses. This option is enabled only if the `reuseaddr` feature is active. 26 | pub reuseaddr: bool, 27 | 28 | #[cfg(all( 29 | feature = "reuseport", 30 | not(any(target_os = "solaris", target_os = "illumos")) 31 | ))] 32 | /// Whether to reuse local ports. This option is enabled only if the `reuseport` feature is active 33 | /// and the target OS is not Solaris or Illumos. 34 | pub reuseport: bool, 35 | 36 | /// The timeout duration for gRPC calls. 37 | pub grpc_timeout: Duration, 38 | 39 | /// The maximum number of concurrent gRPC calls. 40 | pub grpc_concurrency_limit: usize, 41 | 42 | /// The maximum size of gRPC messages in bytes. 43 | pub grpc_message_size: usize, 44 | 45 | /// The threshold for the gRPC circuit breaker. If the number of failed requests exceeds this threshold, 46 | /// the circuit breaker will trip. 47 | pub grpc_breaker_threshold: u64, 48 | 49 | /// The interval at which the gRPC circuit breaker will retry after tripping. 50 | pub grpc_breaker_retry_interval: Duration, 51 | 52 | /// The maximum number of proposals to batch together before processing. 53 | pub proposal_batch_size: usize, 54 | 55 | /// The timeout duration for collecting proposals into a batch. If this timeout is reached, 56 | /// the collected proposals will be processed regardless of the batch size. 57 | pub proposal_batch_timeout: Duration, 58 | 59 | /// The interval at which snapshots are generated. 60 | pub snapshot_interval: Duration, 61 | 62 | /// The interval at which heartbeat messages are sent to maintain leader election and cluster health. 63 | pub heartbeat: Duration, 64 | 65 | /// Configuration options for the Raft protocol. 66 | pub raft_cfg: tikv_raft::Config, 67 | } 68 | 69 | impl Default for Config { 70 | /// Provides default values for the `Config` struct. 71 | fn default() -> Self { 72 | Self { 73 | #[cfg(feature = "reuseaddr")] 74 | reuseaddr: false, 75 | 76 | #[cfg(all( 77 | feature = "reuseport", 78 | not(any(target_os = "solaris", target_os = "illumos")) 79 | ))] 80 | reuseport: false, 81 | 82 | grpc_timeout: Duration::from_secs(6), 83 | grpc_concurrency_limit: 200, 84 | grpc_message_size: 50 * 1024 * 1024, // 50 MB 85 | grpc_breaker_threshold: 4, 86 | grpc_breaker_retry_interval: Duration::from_millis(2500), 87 | proposal_batch_size: 50, 88 | proposal_batch_timeout: Duration::from_millis(200), 89 | snapshot_interval: Duration::from_secs(600), 90 | heartbeat: Duration::from_millis(100), 91 | raft_cfg: tikv_raft::Config { 92 | election_tick: 10, 93 | heartbeat_tick: 5, 94 | check_quorum: true, 95 | pre_vote: true, 96 | ..Default::default() 97 | }, 98 | } 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /src/raft_service.rs: -------------------------------------------------------------------------------- 1 | use std::time::Duration; 2 | 3 | use tonic::transport::{Channel, Endpoint}; 4 | 5 | use raft_service_client::RaftServiceClient; 6 | 7 | use crate::error::Result; 8 | 9 | tonic::include_proto!("raftservice"); 10 | 11 | pub(crate) type RaftServiceClientType = RaftServiceClient; 12 | 13 | /// Creates a gRPC `Endpoint` for connecting to a Raft service. 14 | /// 15 | /// This function constructs a gRPC `Endpoint` configured with the specified address, concurrency 16 | /// limit, and timeout settings. The `Endpoint` is used to establish a connection to the Raft 17 | /// service. 18 | /// 19 | /// # Parameters 20 | /// - `saddr`: The server address in the form of a string (e.g., "127.0.0.1:50051"). 21 | /// - `concurrency_limit`: The maximum number of concurrent requests allowed. 22 | /// - `timeout`: The connection timeout duration. 23 | /// 24 | /// # Returns 25 | /// Returns a `Result` containing the configured `Endpoint` on success, or an error if the endpoint 26 | /// creation fails. 27 | #[inline] 28 | pub(crate) fn endpoint( 29 | saddr: &str, 30 | concurrency_limit: usize, 31 | timeout: Duration, 32 | ) -> Result { 33 | let endpoint = Channel::from_shared(format!("http://{saddr}")) 34 | .map(|endpoint| { 35 | endpoint 36 | .concurrency_limit(concurrency_limit) 37 | .connect_timeout(timeout) 38 | .timeout(timeout) 39 | }) 40 | .map_err(anyhow::Error::new)?; 41 | Ok(endpoint) 42 | } 43 | 44 | /// Establishes a connection to the Raft service and returns a client. 45 | /// 46 | /// This asynchronous function creates a new `RaftServiceClient` instance, using the provided 47 | /// address, concurrency limit, message size, and timeout settings. The client is configured with 48 | /// the specified message size for both encoding and decoding. 49 | /// 50 | /// # Parameters 51 | /// - `saddr`: The server address in the form of a string (e.g., "127.0.0.1:50051"). 52 | /// - `concurrency_limit`: The maximum number of concurrent requests allowed. 53 | /// - `message_size`: The maximum size of messages for encoding and decoding. 54 | /// - `timeout`: The connection timeout duration. 55 | /// 56 | /// # Returns 57 | /// Returns a `Result` containing the `RaftServiceClient` instance on success, or an error if the 58 | /// connection fails. 59 | #[inline] 60 | pub(crate) async fn connect( 61 | saddr: &str, 62 | concurrency_limit: usize, 63 | message_size: usize, 64 | timeout: Duration, 65 | ) -> Result { 66 | Ok(RaftServiceClientType::new( 67 | endpoint(saddr, concurrency_limit, timeout)? 68 | .connect() 69 | .await?, 70 | ) 71 | .max_decoding_message_size(message_size) 72 | .max_encoding_message_size(message_size)) 73 | } 74 | 75 | /// Binds a TCP listener to the specified address and returns a `TcpListenerStream`. 76 | /// 77 | /// This function sets up a TCP listener with options for socket reuse and a backlog queue. It 78 | /// returns a `TcpListenerStream` that can be used to accept incoming connections. This is 79 | /// particularly useful for scenarios requiring high-performance and customizable socket options. 80 | /// 81 | /// # Parameters 82 | /// - `laddr`: The local address to bind in the form of `std::net::SocketAddr`. 83 | /// - `backlog`: The maximum number of pending connections in the backlog queue. 84 | /// - `_reuseaddr`: Whether to enable the `SO_REUSEADDR` option on Unix-like systems. 85 | /// - `_reuseport`: Whether to enable the `SO_REUSEPORT` option on Unix-like systems. 86 | /// 87 | /// # Returns 88 | /// Returns a `Result` containing the `TcpListenerStream` on success, or an error if the binding fails. 89 | #[inline] 90 | #[cfg(all(feature = "socket2", feature = "tokio-stream"))] 91 | pub fn bind( 92 | laddr: std::net::SocketAddr, 93 | backlog: i32, 94 | _reuseaddr: bool, 95 | _reuseport: bool, 96 | ) -> anyhow::Result { 97 | use socket2::{Domain, SockAddr, Socket, Type}; 98 | let builder = Socket::new(Domain::for_address(laddr), Type::STREAM, None)?; 99 | builder.set_nonblocking(true)?; 100 | #[cfg(unix)] 101 | #[cfg(feature = "reuseaddr")] 102 | builder.set_reuse_address(_reuseaddr)?; 103 | #[cfg(unix)] 104 | #[cfg(feature = "reuseport")] 105 | builder.set_reuse_port(_reuseport)?; 106 | builder.bind(&SockAddr::from(laddr))?; 107 | builder.listen(backlog)?; 108 | let listener = tokio_stream::wrappers::TcpListenerStream::new( 109 | tokio::net::TcpListener::from_std(std::net::TcpListener::from(builder))?, 110 | ); 111 | Ok(listener) 112 | } 113 | -------------------------------------------------------------------------------- /src/timeout_recorder.rs: -------------------------------------------------------------------------------- 1 | //! A sliding window timeout recorder for counting and calculating event rates. 2 | //! 3 | //! `TimeoutRecorder` splits a fixed time window into multiple equal-sized buckets. 4 | //! Each event is recorded into the corresponding bucket based on the current time. 5 | //! 6 | //! This allows efficient computation of: 7 | //! - Total number of recent events within the window (`get()`) 8 | //! - Average event rate (`rate_per_second()`, `rate_per()`) 9 | //! - Recent activity in the current bucket (`recent_get()`, `recent_bucket_rate_per_second()`) 10 | //! 11 | //! This structure is useful for rate-limiting, monitoring, and analytics where 12 | //! time-based metrics are required without storing every individual event. 13 | //! 14 | //! ## Example 15 | //! ```rust,ignore 16 | //! let mut recorder = TimeoutRecorder::new(std::time::Duration::from_secs(10), 10); 17 | //! recorder.incr(); // record an event 18 | //! let total = recorder.get(); // get total in last 10 seconds 19 | //! let rate = recorder.rate_per_second(); // average rate per second 20 | //! ``` 21 | 22 | use std::time::{Duration, Instant}; 23 | 24 | #[derive(Clone)] 25 | pub struct TimeoutRecorder { 26 | buckets: Vec, 27 | window_size: Duration, 28 | bucket_count: usize, 29 | bucket_duration: Duration, 30 | max: usize, 31 | } 32 | 33 | #[derive(Debug, Clone)] 34 | struct Bucket { 35 | timestamp: Instant, 36 | count: usize, 37 | } 38 | 39 | impl TimeoutRecorder { 40 | /// Creates a new TimeoutRecorder with the given time window and number of buckets. 41 | pub fn new(window_size: Duration, bucket_count: usize) -> Self { 42 | let now = Instant::now(); 43 | let buckets = vec![ 44 | Bucket { 45 | timestamp: now, 46 | count: 0, 47 | }; 48 | bucket_count 49 | ]; 50 | 51 | TimeoutRecorder { 52 | buckets, 53 | window_size, 54 | bucket_count, 55 | bucket_duration: window_size / bucket_count as u32, 56 | max: 0, 57 | } 58 | } 59 | 60 | /// Increments the count in the current bucket. 61 | #[inline] 62 | pub fn incr(&mut self) { 63 | let now = Instant::now(); 64 | let index = self.bucket_index(now); 65 | 66 | let bucket = &mut self.buckets[index]; 67 | let dur = now.duration_since(bucket.timestamp); 68 | if dur >= self.window_size || dur >= self.bucket_duration { 69 | // Reset the bucket if it's expired 70 | bucket.count = 0; 71 | bucket.timestamp = now; 72 | } 73 | 74 | bucket.count += 1; 75 | self.max += 1; 76 | } 77 | 78 | /// Returns the total count of events in the valid time window. 79 | #[inline] 80 | #[allow(dead_code)] 81 | pub fn get(&self) -> usize { 82 | let now = Instant::now(); 83 | self.buckets 84 | .iter() 85 | .filter(|b| now.duration_since(b.timestamp) <= self.window_size) 86 | .map(|b| b.count) 87 | .sum() 88 | } 89 | 90 | /// Returns the maximum total count ever recorded. 91 | #[inline] 92 | pub fn max(&self) -> usize { 93 | self.max 94 | } 95 | 96 | /// Returns the average rate per second over the entire window. 97 | #[allow(dead_code)] 98 | #[inline] 99 | pub fn rate_per_second(&self) -> f64 { 100 | self.rate_per(Duration::from_secs(1)) 101 | } 102 | 103 | /// Returns the average rate per the given duration over the window. 104 | #[allow(dead_code)] 105 | #[inline] 106 | pub fn rate_per(&self, per: Duration) -> f64 { 107 | let count = self.get() as f64; 108 | let window_secs = self.window_size.as_secs_f64(); 109 | if window_secs == 0.0 { 110 | return 0.0; 111 | } 112 | count * (per.as_secs_f64() / window_secs) 113 | } 114 | 115 | /// Returns the rate of the current bucket (converted to per second). 116 | #[inline] 117 | #[allow(dead_code)] 118 | pub fn recent_bucket_rate_per_second(&self) -> f64 { 119 | let now = Instant::now(); 120 | let index = self.bucket_index(now); 121 | let bucket = &self.buckets[index]; 122 | 123 | if now.duration_since(bucket.timestamp) <= self.bucket_duration { 124 | let duration_secs = self.bucket_duration.as_secs_f64(); 125 | if duration_secs > 0.0 { 126 | bucket.count as f64 / duration_secs 127 | } else { 128 | 0.0 129 | } 130 | } else { 131 | // Current bucket is expired 132 | 0.0 133 | } 134 | } 135 | 136 | /// Returns the count in the current (non-expired) bucket. 137 | #[inline] 138 | pub fn recent_get(&self) -> usize { 139 | let now = Instant::now(); 140 | let index = self.bucket_index(now); 141 | let bucket = &self.buckets[index]; 142 | if now.duration_since(bucket.timestamp) <= self.bucket_duration { 143 | bucket.count 144 | } else { 145 | 0 146 | } 147 | } 148 | 149 | /// Computes the index of the bucket for the current time. 150 | #[inline] 151 | fn bucket_index(&self, now: Instant) -> usize { 152 | let elapsed = now.elapsed().as_millis() as u64; 153 | ((elapsed / self.bucket_duration.as_millis() as u64) % self.bucket_count as u64) as usize 154 | } 155 | } 156 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # RmqttRaft - A raft framework, for regular people 2 | 3 | GitHub Release 4 | crates.io 5 | Documentation 6 | 7 | This is an attempt to create a layer on top of 8 | [tikv/raft-rs](https://github.com/tikv/raft-rs), that is easier to use and implement. This is not supposed to be the 9 | most featureful raft, but instead a convenient interface to get started quickly, and have a working raft in no time. 10 | 11 | The interface is strongly inspired by the one used by [canonical/raft](https://github.com/canonical/raft). 12 | 13 | ## Usage 14 | 15 | Add this to your `Cargo.toml`: 16 | 17 | ```toml 18 | [dependencies] 19 | rmqtt-raft = "0.4" 20 | ``` 21 | 22 | ## Getting started 23 | 24 | In order to "raft" storage, we need to implement the `Storage` trait for it. Bellow is an example with `HashStore`, 25 | which is a thread-safe wrapper around an 26 | `HashMap`: 27 | 28 | ```rust 29 | #[derive(Serialize, Deserialize)] 30 | pub enum Message { 31 | Insert { key: String, value: String }, 32 | Get { key: String }, 33 | } 34 | 35 | #[derive(Clone)] 36 | struct HashStore(Arc>>); 37 | 38 | impl HashStore { 39 | fn new() -> Self { 40 | Self(Arc::new(RwLock::new(HashMap::new()))) 41 | } 42 | fn get(&self, key: &str) -> Option { 43 | self.0.read().unwrap().get(key).cloned() 44 | } 45 | } 46 | 47 | #[async_trait] 48 | impl Store for HashStore { 49 | async fn apply(&mut self, message: &[u8]) -> RaftResult> { 50 | let message: Message = deserialize(message).unwrap(); 51 | let message: Vec = match message { 52 | Message::Insert { key, value } => { 53 | let mut db = self.0.write().unwrap(); 54 | let v = serialize(&value).unwrap(); 55 | db.insert(key, value); 56 | v 57 | } 58 | _ => Vec::new(), 59 | }; 60 | Ok(message) 61 | } 62 | 63 | async fn query(&self, query: &[u8]) -> RaftResult> { 64 | let query: Message = deserialize(query).unwrap(); 65 | let data: Vec = match query { 66 | Message::Get { key } => { 67 | if let Some(val) = self.get(&key) { 68 | serialize(&val).unwrap() 69 | } else { 70 | Vec::new() 71 | } 72 | } 73 | _ => Vec::new(), 74 | }; 75 | Ok(data) 76 | } 77 | 78 | async fn snapshot(&self) -> RaftResult> { 79 | Ok(serialize(&self.0.read().unwrap().clone())?) 80 | } 81 | 82 | async fn restore(&mut self, snapshot: &[u8]) -> RaftResult<()> { 83 | let new: HashMap = deserialize(snapshot).unwrap(); 84 | let mut db = self.0.write().unwrap(); 85 | let _ = std::mem::replace(&mut *db, new); 86 | Ok(()) 87 | } 88 | } 89 | 90 | ``` 91 | 92 | Only 4 methods need to be implemented for the Store: 93 | 94 | - `Store::apply`: applies a commited entry to the store. 95 | - `Store::query` query a entry from the store; 96 | - `Store::snapshot`: returns snapshot data for the store. 97 | - `Store::restore`: applies the snapshot passed as argument. 98 | 99 | ### running the raft 100 | 101 | ```rust 102 | #[tokio::main] 103 | async fn main() -> std::result::Result<(), Box> { 104 | let decorator = slog_term::TermDecorator::new().build(); 105 | let drain = slog_term::FullFormat::new(decorator).build().fuse(); 106 | let drain = slog_async::Async::new(drain).build().fuse(); 107 | let logger = slog::Logger::root(drain, slog_o!("version" => env!("CARGO_PKG_VERSION"))); 108 | 109 | // converts log to slog 110 | #[allow(clippy::let_unit_value)] 111 | let _log_guard = slog_stdlog::init().unwrap(); 112 | 113 | let options = Options::from_args(); 114 | let store = HashStore::new(); 115 | info!(logger, "peer_addrs: {:?}", options.peer_addrs); 116 | let cfg = Config { 117 | reuseaddr: true, 118 | reuseport: true, 119 | // grpc_message_size: 50 * 1024 * 1024, 120 | ..Default::default() 121 | }; 122 | let raft = Raft::new( 123 | options.raft_laddr.clone(), 124 | store.clone(), 125 | logger.clone(), 126 | cfg, 127 | )?; 128 | let leader_info = raft.find_leader_info(options.peer_addrs).await?; 129 | info!(logger, "leader_info: {:?}", leader_info); 130 | 131 | let mailbox = Arc::new(raft.mailbox()); 132 | let (raft_handle, mailbox) = match leader_info { 133 | Some((leader_id, leader_addr)) => { 134 | info!(logger, "running in follower mode"); 135 | let handle = tokio::spawn(raft.join( 136 | options.id, 137 | options.raft_laddr, 138 | Some(leader_id), 139 | leader_addr, 140 | )); 141 | (handle, mailbox) 142 | } 143 | None => { 144 | info!(logger, "running in leader mode"); 145 | let handle = tokio::spawn(raft.lead(options.id)); 146 | (handle, mailbox) 147 | } 148 | }; 149 | 150 | tokio::try_join!(raft_handle)?.0?; 151 | Ok(()) 152 | } 153 | ``` 154 | 155 | The `mailbox` gives you a way to interact with the raft, for sending a message, or leaving the cluster for example. 156 | 157 | ## Credit 158 | 159 | This work is based on [riteraft](https://github.com/ritelabs/riteraft), but more adjustments and improvements have been 160 | made to the code . 161 | 162 | ## License 163 | 164 | This library is licensed under either of: 165 | 166 | * MIT license [LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT 167 | * Apache License 2.0 [LICENSE-APACHE](LICENSE-APACHE) or https://opensource.org/licenses/Apache-2.0 168 | 169 | at your option. 170 | 171 | -------------------------------------------------------------------------------- /src/storage.rs: -------------------------------------------------------------------------------- 1 | use tikv_raft::prelude::*; 2 | use tikv_raft::storage::MemStorage as CoreMemStorage; 3 | use tikv_raft::GetEntriesContext; 4 | 5 | use crate::error::Result; 6 | 7 | /// A trait defining operations for a log store in a Raft implementation. 8 | /// 9 | /// The `LogStore` trait extends the `Storage` trait with additional methods to manage Raft log entries, 10 | /// hard state, configuration state, and snapshots. Implementations of this trait should support appending 11 | /// log entries, updating the hard state and configuration state, creating and applying snapshots, and 12 | /// compacting the log. 13 | /// 14 | /// # Methods 15 | /// - `append`: Append a list of log entries to the log store. 16 | /// - `set_hard_state`: Set the hard state for the Raft state machine. 17 | /// - `set_hard_state_comit`: Set the commit index in the hard state. 18 | /// - `set_conf_state`: Set the configuration state for the Raft state machine. 19 | /// - `create_snapshot`: Create a snapshot with the given data. 20 | /// - `apply_snapshot`: Apply a snapshot to the log store. 21 | /// - `compact`: Compact the log store up to the given index. 22 | pub trait LogStore: Storage { 23 | /// Appends a list of log entries to the log store. 24 | fn append(&mut self, entries: &[Entry]) -> Result<()>; 25 | /// Sets the hard state for the Raft state machine. 26 | fn set_hard_state(&mut self, hard_state: &HardState) -> Result<()>; 27 | 28 | /// Sets the commit index in the hard state. 29 | fn set_hard_state_comit(&mut self, comit: u64) -> Result<()>; 30 | 31 | /// Sets the configuration state for the Raft state machine. 32 | fn set_conf_state(&mut self, conf_state: &ConfState) -> Result<()>; 33 | 34 | fn create_snapshot(&mut self, applied_index: u64, term: u64) -> Result; 35 | 36 | fn set_snapshot(&mut self, snapshot: Snapshot); 37 | 38 | /// Applies a snapshot to the log store. 39 | fn apply_snapshot(&mut self, snapshot: Snapshot) -> Result<()>; 40 | 41 | /// Compacts the log store up to the given index. 42 | fn compact(&mut self, index: u64) -> Result<()>; 43 | } 44 | 45 | /// An in-memory implementation of the `LogStore` trait using Tikv's `MemStorage`. 46 | /// 47 | /// The `MemStorage` struct provides an in-memory storage backend for Raft logs and state. It uses Tikv's 48 | /// `CoreMemStorage` as the underlying storage engine and includes additional methods for managing snapshots. 49 | /// 50 | /// # Fields 51 | /// - `core`: The underlying `CoreMemStorage` used for log and state management. 52 | /// - `snapshot`: The currently held snapshot. 53 | pub struct MemStorage { 54 | core: CoreMemStorage, 55 | snapshot: Snapshot, 56 | } 57 | 58 | impl MemStorage { 59 | /// Creates a new `MemStorage` instance with default settings. 60 | /// 61 | /// This function initializes `CoreMemStorage` and sets the `snapshot` to its default value. 62 | /// 63 | /// # Returns 64 | /// Returns a new `MemStorage` instance. 65 | #[inline] 66 | pub fn create() -> Self { 67 | let core = CoreMemStorage::default(); 68 | let snapshot = Default::default(); 69 | Self { core, snapshot } 70 | } 71 | } 72 | 73 | impl LogStore for MemStorage { 74 | /// Appends a list of log entries to the in-memory log store. 75 | /// 76 | /// This method acquires a write lock on the underlying `CoreMemStorage` and appends the provided 77 | /// entries. 78 | /// 79 | /// # Parameters 80 | /// - `entries`: The entries to be appended. 81 | /// 82 | /// # Returns 83 | /// Returns a `Result` indicating success or failure. 84 | #[inline] 85 | fn append(&mut self, entries: &[Entry]) -> Result<()> { 86 | let mut store = self.core.wl(); 87 | store.append(entries)?; 88 | Ok(()) 89 | } 90 | 91 | /// Sets the hard state for the Raft state machine. 92 | /// 93 | /// This method acquires a write lock on the underlying `CoreMemStorage` and updates the hard state. 94 | /// 95 | /// # Parameters 96 | /// - `hard_state`: The new hard state to set. 97 | /// 98 | /// # Returns 99 | /// Returns a `Result` indicating success or failure. 100 | #[inline] 101 | fn set_hard_state(&mut self, hard_state: &HardState) -> Result<()> { 102 | let mut store = self.core.wl(); 103 | store.set_hardstate(hard_state.clone()); 104 | Ok(()) 105 | } 106 | 107 | /// Sets the commit index in the hard state. 108 | /// 109 | /// This method updates the commit index in the hard state by first acquiring a write lock on the 110 | /// underlying `CoreMemStorage`, modifying the commit index, and then setting the updated hard state. 111 | /// 112 | /// # Parameters 113 | /// - `comit`: The commit index to set. 114 | /// 115 | /// # Returns 116 | /// Returns a `Result` indicating success or failure. 117 | #[inline] 118 | fn set_hard_state_comit(&mut self, comit: u64) -> Result<()> { 119 | let mut store = self.core.wl(); 120 | let mut hard_state = store.hard_state().clone(); 121 | hard_state.set_commit(comit); 122 | store.set_hardstate(hard_state); 123 | Ok(()) 124 | } 125 | 126 | /// Sets the configuration state for the Raft state machine. 127 | /// 128 | /// This method acquires a write lock on the underlying `CoreMemStorage` and updates the configuration state. 129 | /// 130 | /// # Parameters 131 | /// - `conf_state`: The new configuration state to set. 132 | /// 133 | /// # Returns 134 | /// Returns a `Result` indicating success or failure. 135 | #[inline] 136 | fn set_conf_state(&mut self, conf_state: &ConfState) -> Result<()> { 137 | let mut store = self.core.wl(); 138 | store.set_conf_state(conf_state.clone()); 139 | Ok(()) 140 | } 141 | 142 | #[inline] 143 | fn create_snapshot(&mut self, applied_index: u64, term: u64) -> Result { 144 | let mut snapshot = self.core.snapshot(applied_index, 0)?; 145 | let mut metadata = snapshot.take_metadata(); 146 | metadata.index = applied_index; 147 | metadata.term = term; 148 | snapshot.set_metadata(metadata); 149 | Ok(snapshot) 150 | } 151 | 152 | #[inline] 153 | fn set_snapshot(&mut self, snapshot: Snapshot) { 154 | self.snapshot = snapshot; 155 | } 156 | 157 | /// Applies a snapshot to the in-memory log store. 158 | /// 159 | /// This method acquires a write lock on the underlying `CoreMemStorage` and applies the provided snapshot. 160 | /// 161 | /// # Parameters 162 | /// - `snapshot`: The snapshot to apply. 163 | /// 164 | /// # Returns 165 | /// Returns a `Result` indicating success or failure. 166 | #[inline] 167 | fn apply_snapshot(&mut self, snapshot: Snapshot) -> Result<()> { 168 | let mut store = self.core.wl(); 169 | self.snapshot = snapshot; 170 | let mut snapshot = Snapshot::default(); 171 | snapshot.set_metadata(self.snapshot.get_metadata().clone()); 172 | store.apply_snapshot(snapshot)?; 173 | Ok(()) 174 | } 175 | 176 | /// Compacts the log store up to the given index. 177 | /// 178 | /// This method acquires a write lock on the underlying `CoreMemStorage` and compacts the log up to the specified index. 179 | /// 180 | /// # Parameters 181 | /// - `index`: The index up to which to compact the log. 182 | /// 183 | /// # Returns 184 | /// Returns a `Result` indicating success or failure. 185 | #[inline] 186 | fn compact(&mut self, index: u64) -> Result<()> { 187 | let mut store = self.core.wl(); 188 | store.compact(index)?; 189 | Ok(()) 190 | } 191 | } 192 | 193 | impl Storage for MemStorage { 194 | /// Retrieves the initial state of the Raft state machine. 195 | /// 196 | /// This method returns the initial state from the underlying `CoreMemStorage`. 197 | /// 198 | /// # Returns 199 | /// Returns a `Result` containing the `RaftState` on success. 200 | #[inline] 201 | fn initial_state(&self) -> tikv_raft::Result { 202 | let raft_state = self.core.initial_state()?; 203 | Ok(raft_state) 204 | } 205 | 206 | /// Retrieves a range of log entries. 207 | /// 208 | /// This method acquires a read lock on the underlying `CoreMemStorage` and returns log entries 209 | /// in the specified range. 210 | /// 211 | /// # Parameters 212 | /// - `low`: The start index of the range (inclusive). 213 | /// - `high`: The end index of the range (exclusive). 214 | /// - `max_size`: The maximum size of the entries to return (optional). 215 | /// - `context`: Additional context for retrieving the entries. 216 | /// 217 | /// # Returns 218 | /// Returns a `Result` containing a vector of `Entry` objects on success. 219 | #[inline] 220 | fn entries( 221 | &self, 222 | low: u64, 223 | high: u64, 224 | max_size: impl Into>, 225 | context: GetEntriesContext, 226 | ) -> tikv_raft::Result> { 227 | let entries = self.core.entries(low, high, max_size, context)?; 228 | Ok(entries) 229 | } 230 | 231 | /// Retrieves the term of the log entry at the specified index. 232 | /// 233 | /// This method returns the term of the log entry from the underlying `CoreMemStorage`. 234 | /// 235 | /// # Parameters 236 | /// - `idx`: The index of the log entry. 237 | /// 238 | /// # Returns 239 | /// Returns a `Result` containing the term of the entry on success. 240 | #[inline] 241 | fn term(&self, idx: u64) -> tikv_raft::Result { 242 | self.core.term(idx) 243 | } 244 | 245 | /// Retrieves the first index of the log. 246 | /// 247 | /// This method returns the first index from the underlying `CoreMemStorage`. 248 | /// 249 | /// # Returns 250 | /// Returns a `Result` containing the first index on success. 251 | #[inline] 252 | fn first_index(&self) -> tikv_raft::Result { 253 | self.core.first_index() 254 | } 255 | 256 | /// Retrieves the last index of the log. 257 | /// 258 | /// This method returns the last index from the underlying `CoreMemStorage`. 259 | /// 260 | /// # Returns 261 | /// Returns a `Result` containing the last index on success. 262 | #[inline] 263 | fn last_index(&self) -> tikv_raft::Result { 264 | self.core.last_index() 265 | } 266 | 267 | /// Retrieves the current snapshot. 268 | /// 269 | /// This method returns a clone of the current snapshot held in the `snapshot` field. 270 | /// 271 | /// # Parameters 272 | /// - `_request_index`: The index for the snapshot request (not used in this implementation). 273 | /// - `_to`: The index up to which the snapshot is requested (not used in this implementation). 274 | /// 275 | /// # Returns 276 | /// Returns a `Result` containing the current `Snapshot` on success. 277 | #[inline] 278 | fn snapshot(&self, _request_index: u64, _to: u64) -> tikv_raft::Result { 279 | log::info!( 280 | "get snapshot, _request_index: {_request_index}, _to: {_to}, snapshot metadata: {:?}, snapshot data len: {}", 281 | self.snapshot.get_metadata(), self.snapshot.get_data().len() 282 | ); 283 | // Ok(self.snapshot.clone()) 284 | let mut snap = self.core.snapshot(_request_index, _to)?; 285 | snap.mut_metadata().index = self.snapshot.get_metadata().index; 286 | snap.mut_metadata().term = self.snapshot.get_metadata().term; 287 | *snap.mut_data() = self.snapshot.get_data().to_vec(); 288 | Ok(snap) 289 | } 290 | } 291 | -------------------------------------------------------------------------------- /src/raft_server.rs: -------------------------------------------------------------------------------- 1 | use std::net::SocketAddr; 2 | use std::sync::Arc; 3 | use std::time::Duration; 4 | 5 | use bincode::serialize; 6 | use futures::channel::{mpsc, oneshot}; 7 | use futures::SinkExt; 8 | use log::{info, warn}; 9 | use prost::Message as _; 10 | use tikv_raft::eraftpb::{ConfChange, Message as RaftMessage}; 11 | use tokio::time::timeout; 12 | use tonic::transport::Server; 13 | use tonic::{Request, Response, Status}; 14 | 15 | use crate::message::{Message, RaftResponse}; 16 | use crate::raft_service::raft_service_server::{RaftService, RaftServiceServer}; 17 | use crate::raft_service::{ 18 | self, ConfChange as RiteraftConfChange, Empty, Message as RiteraftMessage, 19 | }; 20 | use crate::{error, Config}; 21 | 22 | /// A gRPC server that handles Raft-related requests. 23 | pub struct RaftServer { 24 | snd: mpsc::Sender, 25 | laddr: SocketAddr, 26 | timeout: Duration, 27 | cfg: Arc, 28 | } 29 | 30 | impl RaftServer { 31 | /// Creates a new instance of `RaftServer`. 32 | /// 33 | /// This function initializes a new `RaftServer` with the specified parameters. 34 | /// 35 | /// # Parameters 36 | /// - `snd`: A sender for Raft messages. 37 | /// - `laddr`: The local address where the server will listen for incoming requests. 38 | /// - `cfg`: Configuration for the server, including gRPC timeouts and other settings. 39 | /// 40 | /// # Returns 41 | /// Returns a new `RaftServer` instance. 42 | pub fn new(snd: mpsc::Sender, laddr: SocketAddr, cfg: Arc) -> Self { 43 | RaftServer { 44 | snd, 45 | laddr, 46 | timeout: cfg.grpc_timeout, 47 | cfg, 48 | } 49 | } 50 | 51 | /// Starts the gRPC server to handle Raft requests. 52 | /// 53 | /// This function sets up the gRPC server and listens for incoming requests. It uses 54 | /// the `RaftServiceServer` to handle requests and manage configuration options. 55 | /// 56 | /// # Returns 57 | /// Returns a `Result` indicating whether the server started successfully or if an error occurred. 58 | pub async fn run(self) -> error::Result<()> { 59 | let laddr = self.laddr; 60 | let _cfg = self.cfg.clone(); 61 | info!("listening gRPC requests on: {}", laddr); 62 | let svc = RaftServiceServer::new(self) 63 | .max_decoding_message_size(_cfg.grpc_message_size) 64 | .max_encoding_message_size(_cfg.grpc_message_size); 65 | let server = Server::builder().add_service(svc); 66 | 67 | #[cfg(any(feature = "reuseport", feature = "reuseaddr"))] 68 | #[cfg(all(feature = "socket2", feature = "tokio-stream"))] 69 | { 70 | log::info!( 71 | "reuseaddr: {}, reuseport: {}", 72 | _cfg.reuseaddr, 73 | _cfg.reuseport 74 | ); 75 | let listener = raft_service::bind(laddr, 1024, _cfg.reuseaddr, _cfg.reuseport)?; 76 | server.serve_with_incoming(listener).await?; 77 | } 78 | #[cfg(not(any(feature = "reuseport", feature = "reuseaddr")))] 79 | server.serve(laddr).await?; 80 | 81 | info!("server has quit"); 82 | Ok(()) 83 | } 84 | } 85 | 86 | #[tonic::async_trait] 87 | impl RaftService for RaftServer { 88 | /// Handles requests for a new Raft node ID. 89 | /// 90 | /// This method sends a `RequestId` message to the Raft node and waits for a response. 91 | /// It returns the node ID if successful or an error status if not. 92 | /// 93 | /// # Parameters 94 | /// - `req`: The incoming request containing no additional data. 95 | /// 96 | /// # Returns 97 | /// Returns a `Response` containing the node ID or an error status. 98 | async fn request_id( 99 | &self, 100 | _: Request, 101 | ) -> Result, Status> { 102 | let mut sender = self.snd.clone(); 103 | let (tx, rx) = oneshot::channel(); 104 | let _ = sender.send(Message::RequestId { chan: tx }).await; 105 | //let response = rx.await; 106 | let reply = timeout(self.timeout, rx) 107 | .await 108 | .map_err(|_e| Status::unavailable("recv timeout for reply"))? 109 | .map_err(|_e| Status::unavailable("recv canceled for reply"))?; 110 | match reply { 111 | RaftResponse::WrongLeader { 112 | leader_id, 113 | leader_addr, 114 | } => { 115 | warn!("sending wrong leader, leader_id: {leader_id}, leader_addr: {leader_addr:?}"); 116 | Ok(Response::new(raft_service::IdRequestReponse { 117 | code: raft_service::ResultCode::WrongLeader as i32, 118 | data: serialize(&(leader_id, leader_addr)) 119 | .map_err(|e| Status::unavailable(e.to_string()))?, 120 | })) 121 | } 122 | RaftResponse::RequestId { leader_id } => { 123 | Ok(Response::new(raft_service::IdRequestReponse { 124 | code: raft_service::ResultCode::Ok as i32, 125 | data: serialize(&leader_id).map_err(|e| Status::unavailable(e.to_string()))?, 126 | })) 127 | } 128 | _ => unreachable!(), 129 | } 130 | } 131 | 132 | /// Handles configuration change requests. 133 | /// 134 | /// This method processes a configuration change request by sending it to the Raft node 135 | /// and waits for a response. It returns the result of the configuration change operation. 136 | /// 137 | /// # Parameters 138 | /// - `req`: The incoming request containing the configuration change data. 139 | /// 140 | /// # Returns 141 | /// Returns a `Response` containing the result of the configuration change or an error status. 142 | async fn change_config( 143 | &self, 144 | req: Request, 145 | ) -> Result, Status> { 146 | let change = ConfChange::decode(req.into_inner().inner.as_ref()) 147 | .map_err(|e| Status::invalid_argument(e.to_string()))?; 148 | 149 | let mut sender = self.snd.clone(); 150 | 151 | let (tx, rx) = oneshot::channel(); 152 | 153 | let message = Message::ConfigChange { change, chan: tx }; 154 | 155 | match sender.send(message).await { 156 | Ok(_) => (), 157 | Err(_) => warn!("send error"), 158 | } 159 | 160 | let mut reply = raft_service::RaftResponse::default(); 161 | 162 | match timeout(self.timeout, rx).await { 163 | Ok(Ok(raft_response)) => { 164 | reply.inner = 165 | serialize(&raft_response).map_err(|e| Status::unavailable(e.to_string()))?; 166 | } 167 | Ok(_) => (), 168 | Err(e) => { 169 | reply.inner = serialize(&RaftResponse::Error("timeout".into())) 170 | .map_err(|e| Status::unavailable(e.to_string()))?; 171 | warn!("timeout waiting for reply, {:?}", e); 172 | } 173 | } 174 | 175 | Ok(Response::new(reply)) 176 | } 177 | 178 | /// Handles sending Raft messages. 179 | /// 180 | /// This method processes a Raft message by sending it to the Raft node and returns 181 | /// the result of the send operation. 182 | /// 183 | /// # Parameters 184 | /// - `request`: The incoming request containing the Raft message data. 185 | /// 186 | /// # Returns 187 | /// Returns a `Response` indicating success or an error status. 188 | async fn send_message( 189 | &self, 190 | request: Request, 191 | ) -> Result, Status> { 192 | let message = RaftMessage::decode(request.into_inner().inner.as_ref()) 193 | .map_err(|e| Status::invalid_argument(e.to_string()))?; 194 | match self.snd.clone().try_send(Message::Raft(Box::new(message))) { 195 | Ok(()) => { 196 | let response = RaftResponse::Ok; 197 | Ok(Response::new(raft_service::RaftResponse { 198 | inner: serialize(&response).map_err(|e| Status::unavailable(e.to_string()))?, 199 | })) 200 | } 201 | Err(_) => Err(Status::unavailable("error for try send message")), 202 | } 203 | } 204 | 205 | /// Handles sending proposals. 206 | /// 207 | /// This method sends a proposal to the Raft node and waits for a response. It returns 208 | /// the result of the proposal send operation. 209 | /// 210 | /// # Parameters 211 | /// - `req`: The incoming request containing the proposal data. 212 | /// 213 | /// # Returns 214 | /// Returns a `Response` containing the result of the proposal send operation or an error status. 215 | async fn send_proposal( 216 | &self, 217 | req: Request, 218 | ) -> Result, Status> { 219 | let proposal = req.into_inner().inner; 220 | let mut sender = self.snd.clone(); 221 | let (tx, rx) = oneshot::channel(); 222 | let message = Message::Propose { proposal, chan: tx }; 223 | 224 | match sender.try_send(message) { 225 | Ok(()) => match timeout(self.timeout, rx).await { 226 | Ok(Ok(raft_response)) => match serialize(&raft_response) { 227 | Ok(resp) => Ok(Response::new(raft_service::RaftResponse { inner: resp })), 228 | Err(e) => { 229 | warn!("serialize error, {}", e); 230 | Err(Status::unavailable("serialize error")) 231 | } 232 | }, 233 | Ok(Err(e)) => { 234 | warn!("recv error for reply, {}", e); 235 | Err(Status::unavailable("recv error for reply")) 236 | } 237 | Err(e) => { 238 | warn!("timeout waiting for reply, {}", e); 239 | Err(Status::unavailable("timeout waiting for reply")) 240 | } 241 | }, 242 | Err(e) => { 243 | warn!("error for try send message, {}", e); 244 | Err(Status::unavailable("error for try send message")) 245 | } 246 | } 247 | } 248 | 249 | /// Handles sending queries. 250 | /// 251 | /// This method sends a query to the Raft node and waits for a response. It returns 252 | /// the result of the query send operation. 253 | /// 254 | /// # Parameters 255 | /// - `req`: The incoming request containing the query data. 256 | /// 257 | /// # Returns 258 | /// Returns a `Response` containing the result of the query send operation or an error status. 259 | async fn send_query( 260 | &self, 261 | req: Request, 262 | ) -> Result, Status> { 263 | let query = req.into_inner().inner; 264 | let mut sender = self.snd.clone(); 265 | let (tx, rx) = oneshot::channel(); 266 | let message = Message::Query { query, chan: tx }; 267 | let mut reply = raft_service::RaftResponse::default(); 268 | match sender.try_send(message) { 269 | Ok(()) => { 270 | // if we don't receive a response after 2secs, we timeout 271 | match timeout(self.timeout, rx).await { 272 | Ok(Ok(raft_response)) => { 273 | reply.inner = serialize(&raft_response) 274 | .map_err(|e| Status::unavailable(e.to_string()))?; 275 | } 276 | Ok(Err(e)) => { 277 | reply.inner = serialize(&RaftResponse::Error(e.to_string())) 278 | .map_err(|e| Status::unavailable(e.to_string()))?; 279 | warn!("send query error, {}", e); 280 | } 281 | Err(_e) => { 282 | reply.inner = serialize(&RaftResponse::Error("timeout".into())) 283 | .map_err(|e| Status::unavailable(e.to_string()))?; 284 | warn!("timeout waiting for send query reply"); 285 | } 286 | } 287 | } 288 | Err(e) => { 289 | reply.inner = serialize(&RaftResponse::Error(e.to_string())) 290 | .map_err(|e| Status::unavailable(e.to_string()))?; 291 | warn!("send query error, {}", e) 292 | } 293 | } 294 | 295 | Ok(Response::new(reply)) 296 | } 297 | } 298 | -------------------------------------------------------------------------------- /examples/rmqttraft-warp-memstore/src/main.rs: -------------------------------------------------------------------------------- 1 | #[macro_use] 2 | extern crate slog; 3 | extern crate slog_async; 4 | extern crate slog_term; 5 | 6 | use async_trait::async_trait; 7 | use bincode::{deserialize, serialize}; 8 | use rmqtt_raft::{Config, Mailbox, Raft, Result as RaftResult, Store}; 9 | use serde::{Deserialize, Serialize}; 10 | use slog::info; 11 | use slog::Drain; 12 | use slog_async::Async; 13 | use slog_async::OverflowStrategy; 14 | use std::collections::HashMap; 15 | use std::convert::From; 16 | use std::convert::Infallible; 17 | use std::net::SocketAddr; 18 | use std::str::FromStr; 19 | use std::sync::{Arc, RwLock}; 20 | use structopt::StructOpt; 21 | use warp::{reply, Filter}; 22 | 23 | #[derive(Debug, StructOpt)] 24 | struct Options { 25 | #[structopt(long)] 26 | id: u64, 27 | #[structopt(long)] 28 | raft_laddr: String, 29 | #[structopt(name = "peer-addr", long)] 30 | peer_addrs: Vec, 31 | #[structopt(long)] 32 | web_server: Option, 33 | } 34 | 35 | #[derive(Serialize, Deserialize)] 36 | pub enum Message { 37 | Insert { key: String, value: String }, 38 | Get { key: String }, 39 | } 40 | 41 | #[derive(Clone)] 42 | struct HashStore(Arc>>); 43 | 44 | impl HashStore { 45 | fn new() -> Self { 46 | Self(Arc::new(RwLock::new(HashMap::new()))) 47 | } 48 | fn get(&self, key: &str) -> Option { 49 | self.0.read().unwrap().get(key).cloned() 50 | } 51 | } 52 | 53 | #[async_trait] 54 | impl Store for HashStore { 55 | async fn apply(&mut self, message: &[u8]) -> RaftResult> { 56 | let message: Message = deserialize(message).unwrap(); 57 | let message: Vec = match message { 58 | Message::Insert { key, value } => { 59 | let mut db = self.0.write().unwrap(); 60 | let v = serialize(&value).unwrap(); 61 | // log::info!("{key}={value}"); 62 | db.insert(key, value); 63 | v 64 | } 65 | _ => Vec::new(), 66 | }; 67 | Ok(message) 68 | } 69 | 70 | async fn query(&self, query: &[u8]) -> RaftResult> { 71 | let query: Message = deserialize(query).unwrap(); 72 | let data: Vec = match query { 73 | Message::Get { key } => { 74 | if let Some(val) = self.get(&key) { 75 | serialize(&val).unwrap() 76 | } else { 77 | Vec::new() 78 | } 79 | } 80 | _ => Vec::new(), 81 | }; 82 | Ok(data) 83 | } 84 | 85 | async fn snapshot(&self) -> RaftResult> { 86 | let data = serialize(&self.0.read().unwrap().clone())?; 87 | log::info!("snapshot len: {}", data.len()); 88 | Ok(data) 89 | } 90 | 91 | async fn restore(&mut self, snapshot: &[u8]) -> RaftResult<()> { 92 | log::info!("restore len: {}", snapshot.len()); 93 | if !snapshot.is_empty() { 94 | let new: HashMap = deserialize(snapshot).unwrap(); 95 | let mut db = self.0.write().unwrap(); 96 | let _ = std::mem::replace(&mut *db, new); 97 | } 98 | Ok(()) 99 | } 100 | } 101 | 102 | fn with_mailbox( 103 | mailbox: Arc, 104 | ) -> impl Filter,), Error = Infallible> + Clone { 105 | warp::any().map(move || mailbox.clone()) 106 | } 107 | 108 | fn with_store(store: HashStore) -> impl Filter + Clone { 109 | warp::any().map(move || store.clone()) 110 | } 111 | 112 | async fn put( 113 | mailbox: Arc, 114 | key: String, 115 | value: String, 116 | ) -> Result { 117 | let value = format!("{value}-{}", incr_one()); 118 | let message = Message::Insert { key, value }; 119 | let message = serialize(&message).unwrap(); 120 | let result = mailbox.send_proposal(message).await; 121 | match result { 122 | Ok(r) => { 123 | let result: String = deserialize(&r).unwrap(); 124 | Ok(reply::json(&result)) 125 | } 126 | Err(e) => Ok(reply::json(&format!("put error, {e:?}"))), 127 | } 128 | } 129 | 130 | async fn get(store: HashStore, key: String) -> Result { 131 | let response = store.get(&key); 132 | Ok(reply::json(&response)) 133 | } 134 | 135 | async fn leave(mailbox: Arc) -> Result { 136 | mailbox.leave().await.unwrap(); 137 | Ok(reply::json(&"OK".to_string())) 138 | } 139 | 140 | async fn status(mailbox: Arc) -> Result { 141 | match mailbox.status().await { 142 | Err(e) => Ok(reply::json(&e.to_string())), 143 | Ok(response) => Ok(reply::json(&response)), 144 | } 145 | } 146 | 147 | //target\release\rmqttraft-warp-memstore.exe --id 1 --raft-laddr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8081" 148 | //target\release\rmqttraft-warp-memstore.exe --id 2 --raft-laddr "127.0.0.1:5002" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8082" 149 | //target\release\rmqttraft-warp-memstore.exe --id 3 --raft-laddr "127.0.0.1:5003" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --web-server "0.0.0.0:8083" 150 | //target\release\rmqttraft-warp-memstore.exe --id 4 --raft-laddr "127.0.0.1:5004" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --web-server "0.0.0.0:8084" 151 | //target\release\rmqttraft-warp-memstore.exe --id 5 --raft-laddr "127.0.0.1:5005" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --web-server "0.0.0.0:8085" 152 | 153 | //./target/release/rmqttraft-warp-memstore --id 1 --raft-laddr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8081" 154 | //./target/release/rmqttraft-warp-memstore --id 2 --raft-laddr "127.0.0.1:5002" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8082" 155 | //./target/release/rmqttraft-warp-memstore --id 3 --raft-laddr "127.0.0.1:5003" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --web-server "0.0.0.0:8083" 156 | 157 | //target\debug\rmqttraft-warp-memstore.exe --id 1 --raft-laddr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8081" 158 | //target\debug\rmqttraft-warp-memstore.exe --id 2 --raft-laddr "127.0.0.1:5002" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8082" 159 | //target\debug\rmqttraft-warp-memstore.exe --id 3 --raft-laddr "127.0.0.1:5003" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --web-server "0.0.0.0:8083" 160 | 161 | //./target/debug/rmqttraft-warp-memstore --id 1 --raft-laddr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8081" > out_1.log 2>&1 & 162 | //./target/debug/rmqttraft-warp-memstore --id 2 --raft-laddr "127.0.0.1:5002" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5003" --web-server "0.0.0.0:8082" > out_2.log 2>&1 & 163 | //./target/debug/rmqttraft-warp-memstore --id 3 --raft-laddr "127.0.0.1:5003" --peer-addr "127.0.0.1:5001" --peer-addr "127.0.0.1:5002" --web-server "0.0.0.0:8083" > out_3.log 2>&1 & 164 | 165 | // wrk -c 100 -t4 -d60s -H "Connection: keep-alive" "http://127.0.0.1:8081/put/key1/val-1" 166 | // wrk -c 100 -t4 -d60s -H "Connection: keep-alive" "http://127.0.0.1:8082/put/key1/val-2" 167 | // wrk -c 100 -t6 -d60s -H "Connection: keep-alive" "http://127.0.0.1:8083/get/key1" 168 | 169 | // ab -n 5000 -c 20 "http://127.0.0.1:8081/put/key1/val-1" 170 | // ab -n 5000 -c 50 "http://127.0.0.1:8082/put/key1/val-2" 171 | // ab -n 5000 -c 20 "http://127.0.0.1:8083/get/key1" 172 | 173 | // ab -n 50000 -c 1000 "http://127.0.0.1:8081/put/key1/val-1" 174 | // ab -n 50000 -c 1000 "http://127.0.0.1:8082/put/key2/val-1" 175 | // ab -n 50000 -c 1000 "http://127.0.0.1:8083/put/key3/val-1" 176 | 177 | // http://127.0.0.1:8081/status 178 | 179 | #[tokio::main] 180 | async fn main() -> std::result::Result<(), Box> { 181 | let decorator = slog_term::TermDecorator::new().build(); 182 | let drain = slog_term::FullFormat::new(decorator) 183 | .use_custom_timestamp(timestamp) 184 | .use_custom_header_print(custom_header_format) 185 | .build() 186 | .fuse(); 187 | let drain = slog::LevelFilter::new(drain, slog::Level::Info).fuse(); 188 | let drain = Async::new(drain) 189 | .chan_size(1024) 190 | .overflow_strategy(OverflowStrategy::Block) 191 | .build() 192 | .fuse(); 193 | let logger = slog::Logger::root(drain, slog_o!("version" => env!("CARGO_PKG_VERSION"))); 194 | 195 | // converts log to slog 196 | #[allow(clippy::let_unit_value)] 197 | let _guard = slog_scope::set_global_logger(logger.clone()); 198 | slog_stdlog::init_with_level(log::Level::Info)?; 199 | 200 | let options = Options::from_args(); 201 | let store = HashStore::new(); 202 | info!(logger, "peer_addrs: {:?}", options.peer_addrs); 203 | let cfg = Config { 204 | reuseaddr: true, 205 | reuseport: true, 206 | snapshot_interval: Duration::from_secs(60), 207 | // grpc_message_size: 50 * 1024 * 1024, 208 | ..Default::default() 209 | }; 210 | let raft = Raft::new( 211 | options.raft_laddr.clone(), 212 | store.clone(), 213 | logger.clone(), 214 | cfg, 215 | )?; 216 | let leader_info = raft.find_leader_info(options.peer_addrs).await?; 217 | info!(logger, "leader_info: {:?}", leader_info); 218 | 219 | let mailbox = Arc::new(raft.mailbox()); 220 | let (raft_handle, mailbox) = match leader_info { 221 | Some((leader_id, leader_addr)) => { 222 | info!(logger, "running in follower mode"); 223 | let handle = tokio::spawn(raft.join( 224 | options.id, 225 | options.raft_laddr, 226 | Some(leader_id), 227 | leader_addr, 228 | )); 229 | (handle, mailbox) 230 | } 231 | None => { 232 | info!(logger, "running in leader mode"); 233 | let handle = tokio::spawn(raft.lead(options.id)); 234 | (handle, mailbox) 235 | } 236 | }; 237 | 238 | let put_kv = warp::get() 239 | .and(warp::path!("put" / String / String)) 240 | .and(with_mailbox(mailbox.clone())) 241 | .and_then(|key, value, mailbox: Arc| put(mailbox, key, value)); 242 | 243 | let get_kv = warp::get() 244 | .and(warp::path!("get" / String)) 245 | .and(with_store(store.clone())) 246 | .and_then(|key, store: HashStore| get(store, key)); 247 | 248 | let leave_kv = warp::get() 249 | .and(warp::path!("leave")) 250 | .and(with_mailbox(mailbox.clone())) 251 | .and_then(leave); 252 | 253 | let status = warp::get() 254 | .and(warp::path!("status")) 255 | .and(with_mailbox(mailbox.clone())) 256 | .and_then(status); 257 | 258 | let routes = put_kv.or(get_kv).or(leave_kv).or(status); 259 | 260 | if let Some(addr) = options.web_server { 261 | let _server = tokio::spawn(async move { 262 | warp::serve(routes) 263 | .run(SocketAddr::from_str(&addr).unwrap()) 264 | .await; 265 | }); 266 | } 267 | 268 | tokio::try_join!(raft_handle)?.0?; 269 | Ok(()) 270 | } 271 | 272 | use slog::Record; 273 | use slog_term::CountingWriter; 274 | use slog_term::RecordDecorator; 275 | use slog_term::ThreadSafeTimestampFn; 276 | use std::io; 277 | use std::io::Write; 278 | use std::time::Duration; 279 | 280 | fn timestamp(io: &mut dyn std::io::Write) -> std::io::Result<()> { 281 | write!( 282 | io, 283 | "{}", 284 | chrono::Local::now().format("%Y-%m-%d %H:%M:%S.3f") 285 | ) 286 | } 287 | 288 | fn custom_header_format( 289 | fn_timestamp: &dyn ThreadSafeTimestampFn>, 290 | mut rd: &mut dyn RecordDecorator, 291 | record: &Record, 292 | _use_file_location: bool, 293 | ) -> io::Result { 294 | rd.start_timestamp()?; 295 | fn_timestamp(&mut rd)?; 296 | 297 | rd.start_whitespace()?; 298 | write!(rd, " ")?; 299 | 300 | rd.start_level()?; 301 | write!(rd, "{}", record.level().as_short_str())?; 302 | 303 | rd.start_location()?; 304 | if record.function().is_empty() { 305 | write!(rd, " {}.{} | ", record.module(), record.line())?; 306 | } else { 307 | write!( 308 | rd, 309 | " {}::{}.{} | ", 310 | record.module(), 311 | record.function(), 312 | record.line() 313 | )?; 314 | } 315 | 316 | rd.start_msg()?; 317 | let mut count_rd = CountingWriter::new(&mut rd); 318 | write!(count_rd, "{}", record.msg())?; 319 | Ok(count_rd.count() != 0) 320 | } 321 | 322 | use std::sync::atomic::{AtomicUsize, Ordering}; 323 | static GLOBAL_COUNTER: AtomicUsize = AtomicUsize::new(0); 324 | 325 | pub fn increment_counter(by: usize) -> usize { 326 | GLOBAL_COUNTER.fetch_add(by, Ordering::Relaxed) + by 327 | } 328 | 329 | pub fn incr_one() -> usize { 330 | GLOBAL_COUNTER.fetch_add(1, Ordering::Relaxed) + 1 331 | } 332 | 333 | pub fn get_counter() -> usize { 334 | GLOBAL_COUNTER.load(Ordering::Relaxed) 335 | } 336 | -------------------------------------------------------------------------------- /src/message.rs: -------------------------------------------------------------------------------- 1 | use std::collections::HashMap; 2 | use std::time::{Duration, Instant}; 3 | 4 | use bytestring::ByteString; 5 | use serde::de::{self, Deserializer}; 6 | use serde::ser::Serializer; 7 | use serde::{Deserialize, Serialize}; 8 | 9 | use futures::channel::oneshot::Sender; 10 | use tikv_raft::eraftpb::{ConfChange, Message as RaftMessage}; 11 | use tikv_raft::prelude::Snapshot; 12 | use tikv_raft::StateRole; 13 | 14 | /// Enumeration representing various types of responses that can be sent back to clients. 15 | #[derive(Serialize, Deserialize, Debug)] 16 | pub enum RaftResponse { 17 | /// Indicates that the request was sent to the wrong leader. 18 | WrongLeader { 19 | leader_id: u64, 20 | leader_addr: Option, 21 | }, 22 | /// Indicates that a join request was successful. 23 | JoinSuccess { 24 | assigned_id: u64, 25 | peer_addrs: HashMap, 26 | }, 27 | /// Contains the leader ID in response to a request for ID. 28 | RequestId { leader_id: u64 }, 29 | /// Represents an error with a message. 30 | Error(String), 31 | /// Too busy 32 | Busy, 33 | /// Contains arbitrary response data. 34 | Response { data: Vec }, 35 | /// Represents the status of the system. 36 | Status(Status), 37 | /// Represents a successful operation. 38 | Ok, 39 | } 40 | 41 | /// Enumeration representing different types of messages that can be sent within the system. 42 | #[allow(dead_code)] 43 | pub enum Message { 44 | /// A proposal message to be processed. 45 | Propose { 46 | proposal: Vec, 47 | chan: Sender, 48 | }, 49 | /// A query message to be processed. 50 | Query { 51 | query: Vec, 52 | chan: Sender, 53 | }, 54 | /// A configuration change message to be processed. 55 | ConfigChange { 56 | change: ConfChange, 57 | chan: Sender, 58 | }, 59 | /// A request for the leader's ID. 60 | RequestId { chan: Sender }, 61 | /// Report that a node is unreachable. 62 | ReportUnreachable { node_id: u64 }, 63 | /// A Raft message to be processed. 64 | Raft(Box), 65 | /// A request for the status of the system. 66 | Status { chan: Sender }, 67 | /// Snapshot 68 | Snapshot { snapshot: Snapshot }, 69 | } 70 | 71 | #[derive(Serialize, Deserialize, Debug, Clone)] 72 | pub struct PeerState { 73 | pub addr: ByteString, 74 | pub available: bool, 75 | } 76 | 77 | /// Struct representing the status of the system. 78 | #[derive(Serialize, Deserialize, Debug, Clone)] 79 | pub struct Status { 80 | pub id: u64, 81 | pub leader_id: u64, 82 | pub uncommitteds: usize, 83 | pub merger_proposals: usize, 84 | pub sending_raft_messages: isize, 85 | pub timeout_max: isize, 86 | pub timeout_recent_count: isize, 87 | pub propose_count: isize, 88 | pub propose_rate: f64, 89 | pub peers: HashMap>, 90 | #[serde( 91 | serialize_with = "Status::serialize_role", 92 | deserialize_with = "Status::deserialize_role" 93 | )] 94 | pub role: StateRole, 95 | } 96 | 97 | impl Status { 98 | #[inline] 99 | pub fn available(&self) -> bool { 100 | if matches!(self.role, StateRole::Leader) { 101 | //Check if the number of available nodes is greater than or equal to half of the total nodes. 102 | let (all_count, available_count) = self.get_count(); 103 | let available = available_count >= ((all_count / 2) + (all_count % 2)); 104 | log::debug!( 105 | "is Leader, all_count: {}, available_count: {} {}", 106 | all_count, 107 | available_count, 108 | available 109 | ); 110 | available 111 | } else if self.leader_id > 0 { 112 | //As long as a leader exists and is available, the system considers itself in a normal state. 113 | let available = self 114 | .peers 115 | .get(&self.leader_id) 116 | .and_then(|p| p.as_ref().map(|p| p.available)) 117 | .unwrap_or_default(); 118 | log::debug!("has Leader, available: {}", available); 119 | available 120 | } else { 121 | //If there is no Leader, it's still necessary to check whether the number of all other 122 | // available nodes is greater than or equal to half. 123 | let (all_count, available_count) = self.get_count(); 124 | let available = available_count >= ((all_count / 2) + (all_count % 2)); 125 | log::debug!( 126 | "no Leader, all_count: {}, available_count: {} {}", 127 | all_count, 128 | available_count, 129 | available 130 | ); 131 | available 132 | } 133 | } 134 | 135 | #[inline] 136 | fn get_count(&self) -> (usize, usize) { 137 | let available_count = self 138 | .peers 139 | .iter() 140 | .filter(|(_, p)| if let Some(p) = p { p.available } else { false }) 141 | .count(); 142 | if self.peers.contains_key(&self.id) { 143 | (self.peers.len() - 1, available_count - 1) 144 | } else { 145 | (self.peers.len(), available_count) 146 | } 147 | } 148 | 149 | /// Checks if the node has started. 150 | #[inline] 151 | pub fn is_started(&self) -> bool { 152 | self.leader_id > 0 153 | } 154 | 155 | /// Checks if this node is the leader. 156 | #[inline] 157 | pub fn is_leader(&self) -> bool { 158 | self.leader_id == self.id && matches!(self.role, StateRole::Leader) 159 | } 160 | 161 | #[inline] 162 | pub fn deserialize_role<'de, D>(deserializer: D) -> Result 163 | where 164 | D: Deserializer<'de>, 165 | { 166 | let role = match u8::deserialize(deserializer)? { 167 | 1 => StateRole::Follower, 168 | 2 => StateRole::Candidate, 169 | 3 => StateRole::Leader, 170 | 4 => StateRole::PreCandidate, 171 | _ => return Err(de::Error::missing_field("role")), 172 | }; 173 | Ok(role) 174 | } 175 | 176 | #[inline] 177 | pub fn serialize_role(role: &StateRole, s: S) -> std::result::Result 178 | where 179 | S: Serializer, 180 | { 181 | match role { 182 | StateRole::Follower => 1u8, 183 | StateRole::Candidate => 2u8, 184 | StateRole::Leader => 3u8, 185 | StateRole::PreCandidate => 4u8, 186 | } 187 | .serialize(s) 188 | } 189 | } 190 | 191 | #[derive(Clone, Serialize, Deserialize, Debug)] 192 | pub(crate) enum RemoveNodeType { 193 | Normal, 194 | Stale, 195 | } 196 | 197 | /// Enumeration for reply channels which could be single or multiple. 198 | pub(crate) enum ReplyChan { 199 | /// Single reply channel with its timestamp. 200 | One((Sender, Instant)), 201 | /// Multiple reply channels with their timestamps. 202 | More(Vec<(Sender, Instant)>), 203 | } 204 | 205 | /// Enumeration for proposals which could be a single proposal or multiple proposals. 206 | #[derive(Serialize, Deserialize)] 207 | pub(crate) enum Proposals { 208 | /// A single proposal. 209 | One(Vec), 210 | /// Multiple proposals. 211 | More(Vec>), 212 | } 213 | 214 | /// A struct to manage proposal batching and sending. 215 | pub(crate) struct Merger { 216 | proposals: Vec>, 217 | chans: Vec<(Sender, Instant)>, 218 | start_collection_time: i64, 219 | proposal_batch_size: usize, 220 | proposal_batch_timeout: i64, 221 | } 222 | 223 | impl Merger { 224 | /// Creates a new `Merger` instance with the specified batch size and timeout. 225 | /// 226 | /// # Parameters 227 | /// - `proposal_batch_size`: The maximum number of proposals to include in a batch. 228 | /// - `proposal_batch_timeout`: The timeout duration for collecting proposals. 229 | /// 230 | /// # Returns 231 | /// A new `Merger` instance. 232 | pub fn new(proposal_batch_size: usize, proposal_batch_timeout: Duration) -> Self { 233 | Self { 234 | proposals: Vec::new(), 235 | chans: Vec::new(), 236 | start_collection_time: 0, 237 | proposal_batch_size, 238 | proposal_batch_timeout: proposal_batch_timeout.as_millis() as i64, 239 | } 240 | } 241 | 242 | /// Adds a new proposal and its corresponding reply channel to the merger. 243 | /// 244 | /// # Parameters 245 | /// - `proposal`: The proposal data to be added. 246 | /// - `chan`: The reply channel for the proposal. 247 | #[inline] 248 | pub fn add(&mut self, proposal: Vec, chan: Sender) { 249 | self.proposals.push(proposal); 250 | self.chans.push((chan, Instant::now())); 251 | } 252 | 253 | /// Returns the number of proposals currently held by the merger. 254 | /// 255 | /// # Returns 256 | /// The number of proposals. 257 | #[inline] 258 | pub fn len(&self) -> usize { 259 | self.proposals.len() 260 | } 261 | 262 | /// Retrieves a batch of proposals and their corresponding reply channels if the batch size or timeout criteria are met. 263 | /// 264 | /// # Returns 265 | /// An `Option` containing the proposals and reply channels, or `None` if no batch is ready. 266 | #[inline] 267 | pub fn take(&mut self) -> Option<(Proposals, ReplyChan)> { 268 | let max = self.proposal_batch_size; 269 | let len = self.len(); 270 | let len = if len > max { max } else { len }; 271 | if len > 0 && (len == max || self.timeout()) { 272 | let data = if len == 1 { 273 | match (self.proposals.pop(), self.chans.pop()) { 274 | (Some(proposal), Some(chan)) => { 275 | Some((Proposals::One(proposal), ReplyChan::One(chan))) 276 | } 277 | _ => unreachable!(), 278 | } 279 | } else { 280 | let mut proposals = self.proposals.drain(0..len).collect::>(); 281 | let mut chans = self.chans.drain(0..len).collect::>(); 282 | proposals.reverse(); 283 | chans.reverse(); 284 | Some((Proposals::More(proposals), ReplyChan::More(chans))) 285 | }; 286 | self.start_collection_time = chrono::Local::now().timestamp_millis(); 287 | data 288 | } else { 289 | None 290 | } 291 | } 292 | 293 | #[inline] 294 | fn timeout(&self) -> bool { 295 | chrono::Local::now().timestamp_millis() 296 | > (self.start_collection_time + self.proposal_batch_timeout) 297 | } 298 | } 299 | 300 | #[tokio::test] 301 | async fn test_merger() -> std::result::Result<(), Box> { 302 | let mut merger = Merger::new(50, Duration::from_millis(200)); 303 | use futures::channel::oneshot::channel; 304 | use std::time::Duration; 305 | 306 | let add = |merger: &mut Merger| { 307 | let (tx, rx) = channel(); 308 | merger.add(vec![1, 2, 3], tx); 309 | rx 310 | }; 311 | 312 | use std::sync::atomic::{AtomicI64, Ordering}; 313 | use std::sync::Arc; 314 | const MAX: i64 = 111; 315 | let count = Arc::new(AtomicI64::new(0)); 316 | let mut futs = Vec::new(); 317 | for _ in 0..MAX { 318 | let rx = add(&mut merger); 319 | let count1 = count.clone(); 320 | let fut = async move { 321 | let r = tokio::time::timeout(Duration::from_secs(3), rx).await; 322 | match r { 323 | Ok(_) => {} 324 | Err(_) => { 325 | println!("timeout ..."); 326 | } 327 | } 328 | count1.fetch_add(1, Ordering::SeqCst); 329 | }; 330 | 331 | futs.push(fut); 332 | } 333 | 334 | let sends = async { 335 | loop { 336 | if let Some((_data, chan)) = merger.take() { 337 | match chan { 338 | ReplyChan::One((tx, _)) => { 339 | let _ = tx.send(RaftResponse::Ok); 340 | } 341 | ReplyChan::More(txs) => { 342 | for (tx, _) in txs { 343 | let _ = tx.send(RaftResponse::Ok); 344 | } 345 | } 346 | } 347 | } 348 | tokio::time::sleep(Duration::from_millis(100)).await; 349 | if merger.len() == 0 { 350 | break; 351 | } 352 | } 353 | }; 354 | 355 | let count_p = count.clone(); 356 | let count_print = async move { 357 | loop { 358 | tokio::time::sleep(Duration::from_secs(2)).await; 359 | println!("count_p: {}", count_p.load(Ordering::SeqCst)); 360 | if count_p.load(Ordering::SeqCst) >= MAX { 361 | break; 362 | } 363 | } 364 | }; 365 | println!("futs: {}", futs.len()); 366 | futures::future::join3(futures::future::join_all(futs), sends, count_print).await; 367 | 368 | Ok(()) 369 | } 370 | -------------------------------------------------------------------------------- /src/raft.rs: -------------------------------------------------------------------------------- 1 | use std::net::{SocketAddr, ToSocketAddrs}; 2 | use std::sync::Arc; 3 | use std::time::Duration; 4 | 5 | use async_trait::async_trait; 6 | use bincode::{deserialize, serialize}; 7 | use bytestring::ByteString; 8 | use futures::channel::{mpsc, oneshot}; 9 | use futures::future::FutureExt; 10 | use futures::SinkExt; 11 | use log::{debug, info, warn}; 12 | use prost::Message as _; 13 | use tikv_raft::eraftpb::{ConfChange, ConfChangeType}; 14 | use tokio::sync::RwLock; 15 | use tokio::time::timeout; 16 | use tonic::Request; 17 | 18 | use crate::error::{Error, Result}; 19 | use crate::message::{Message, RaftResponse, RemoveNodeType, Status}; 20 | use crate::raft_node::{Peer, RaftNode}; 21 | use crate::raft_server::RaftServer; 22 | use crate::raft_service::connect; 23 | use crate::raft_service::{ConfChange as RiteraftConfChange, Empty, ResultCode}; 24 | use crate::Config; 25 | 26 | type DashMap = dashmap::DashMap; 27 | 28 | #[async_trait] 29 | pub trait Store: Clone + Send + Sync { 30 | async fn apply(&mut self, message: &[u8]) -> Result>; 31 | async fn query(&self, query: &[u8]) -> Result>; 32 | async fn snapshot(&self) -> Result>; 33 | async fn restore(&mut self, snapshot: &[u8]) -> Result<()>; 34 | } 35 | 36 | struct ProposalSender { 37 | proposal: Vec, 38 | client: Peer, 39 | } 40 | 41 | impl ProposalSender { 42 | async fn send(self) -> Result { 43 | match self.client.send_proposal(self.proposal).await { 44 | Ok(reply) => { 45 | let raft_response: RaftResponse = deserialize(&reply)?; 46 | Ok(raft_response) 47 | } 48 | Err(e) => { 49 | warn!("error sending proposal {:?}", e); 50 | Err(e) 51 | } 52 | } 53 | } 54 | } 55 | 56 | #[derive(Clone)] 57 | struct LeaderInfo { 58 | leader: bool, 59 | target_leader_id: u64, 60 | target_leader_addr: Option, 61 | } 62 | 63 | type LeaderInfoError = ByteString; 64 | 65 | /// A mailbox to send messages to a running raft node. 66 | #[derive(Clone)] 67 | pub struct Mailbox { 68 | peers: Arc>, 69 | sender: mpsc::Sender, 70 | grpc_timeout: Duration, 71 | grpc_concurrency_limit: usize, 72 | grpc_message_size: usize, 73 | grpc_breaker_threshold: u64, 74 | grpc_breaker_retry_interval: i64, 75 | #[allow(clippy::type_complexity)] 76 | leader_info: Arc< 77 | RwLock< 78 | Option<( 79 | Option, 80 | Option, 81 | std::time::Instant, 82 | )>, 83 | >, 84 | >, 85 | } 86 | 87 | impl Mailbox { 88 | #[inline] 89 | pub(crate) fn new( 90 | peers: Arc>, 91 | sender: mpsc::Sender, 92 | grpc_timeout: Duration, 93 | grpc_concurrency_limit: usize, 94 | grpc_message_size: usize, 95 | grpc_breaker_threshold: u64, 96 | grpc_breaker_retry_interval: i64, 97 | ) -> Self { 98 | Self { 99 | peers, 100 | sender, 101 | grpc_timeout, 102 | grpc_concurrency_limit, 103 | grpc_message_size, 104 | grpc_breaker_threshold, 105 | grpc_breaker_retry_interval, 106 | leader_info: Arc::new(RwLock::new(None)), 107 | } 108 | } 109 | 110 | /// Retrieves a list of peers with their IDs. 111 | /// This method returns a vector containing tuples of peer IDs and their respective `Peer` objects. 112 | /// It iterates over the internal `peers` map and collects the IDs and cloned `Peer` instances. 113 | #[inline] 114 | pub fn pears(&self) -> Vec<(u64, Peer)> { 115 | self.peers 116 | .iter() 117 | .map(|p| { 118 | let (id, _) = p.key(); 119 | (*id, p.value().clone()) 120 | }) 121 | .collect::>() 122 | } 123 | 124 | #[inline] 125 | async fn peer(&self, leader_id: u64, leader_addr: String) -> Peer { 126 | self.peers 127 | .entry((leader_id, leader_addr.clone())) 128 | .or_insert_with(|| { 129 | Peer::new( 130 | leader_addr, 131 | self.grpc_timeout, 132 | self.grpc_concurrency_limit, 133 | self.grpc_message_size, 134 | self.grpc_breaker_threshold, 135 | self.grpc_breaker_retry_interval, 136 | ) 137 | }) 138 | .clone() 139 | } 140 | 141 | #[inline] 142 | async fn send_to_leader( 143 | &self, 144 | proposal: Vec, 145 | leader_id: u64, 146 | leader_addr: String, 147 | ) -> Result { 148 | let peer = self.peer(leader_id, leader_addr).await; 149 | let proposal_sender = ProposalSender { 150 | proposal, 151 | client: peer, 152 | }; 153 | proposal_sender.send().await 154 | } 155 | 156 | /// Sends a proposal to the leader node. 157 | /// This method first attempts to send the proposal to the local node if it is the leader. 158 | /// If the node is not the leader, it retrieves the leader's address and sends the proposal to the leader node. 159 | /// If the proposal is successfully handled, the method returns a `RaftResponse::Response` with the resulting data. 160 | #[inline] 161 | pub async fn send_proposal(&self, message: Vec) -> Result> { 162 | match self.get_leader_info().await? { 163 | LeaderInfo { leader: true, .. } => { 164 | debug!("this node is leader"); 165 | let (tx, rx) = oneshot::channel(); 166 | let proposal = Message::Propose { 167 | proposal: message.clone(), 168 | chan: tx, 169 | }; 170 | let mut sender = self.sender.clone(); 171 | sender 172 | .send(proposal) 173 | .await //.try_send(proposal) 174 | .map_err(|e| Error::SendError(e.to_string()))?; 175 | let reply = timeout(self.grpc_timeout, rx).await; 176 | let reply = reply 177 | .map_err(|e| Error::RecvError(e.to_string()))? 178 | .map_err(|e| Error::RecvError(e.to_string()))?; 179 | match reply { 180 | RaftResponse::Response { data } => return Ok(data), 181 | RaftResponse::Busy => return Err(Error::Busy), 182 | RaftResponse::Error(e) => return Err(Error::from(e)), 183 | _ => { 184 | warn!("Recv other raft response: {:?}", reply); 185 | return Err(Error::Unknown); 186 | } 187 | } 188 | } 189 | LeaderInfo { 190 | leader: false, 191 | target_leader_id, 192 | target_leader_addr, 193 | .. 194 | } => { 195 | debug!( 196 | "This node not is Leader, leader_id: {:?}, leader_addr: {:?}", 197 | target_leader_id, target_leader_addr 198 | ); 199 | if let Some(target_leader_addr) = target_leader_addr { 200 | if target_leader_id != 0 { 201 | return match self 202 | .send_to_leader(message, target_leader_id, target_leader_addr.clone()) 203 | .await? 204 | { 205 | RaftResponse::Response { data } => return Ok(data), 206 | RaftResponse::WrongLeader { 207 | leader_id, 208 | leader_addr, 209 | } => { 210 | warn!("The target node is not the Leader, target_leader_id: {}, target_leader_addr: {:?}, actual_leader_id: {}, actual_leader_addr: {:?}", 211 | target_leader_id, target_leader_addr, leader_id, leader_addr); 212 | return Err(Error::NotLeader); 213 | } 214 | RaftResponse::Busy => Err(Error::Busy), 215 | RaftResponse::Error(e) => Err(Error::from(e)), 216 | _ => { 217 | warn!("Recv other raft response, target_leader_id: {}, target_leader_addr: {:?}", target_leader_id, target_leader_addr); 218 | return Err(Error::Unknown); 219 | } 220 | }; 221 | } 222 | } 223 | } 224 | } 225 | Err(Error::LeaderNotExist) 226 | } 227 | 228 | /// Deprecated method to send a message, internally calls `send_proposal`. 229 | #[inline] 230 | #[deprecated] 231 | pub async fn send(&self, message: Vec) -> Result> { 232 | self.send_proposal(message).await 233 | } 234 | 235 | /// Sends a query to the Raft node and returns the response data. 236 | /// It sends a `Message::Query` containing the query bytes and waits for a response. 237 | /// On success, it returns the data wrapped in `RaftResponse::Response`. 238 | #[inline] 239 | pub async fn query(&self, query: Vec) -> Result> { 240 | let (tx, rx) = oneshot::channel(); 241 | let mut sender = self.sender.clone(); 242 | match sender.try_send(Message::Query { query, chan: tx }) { 243 | Ok(()) => match timeout(self.grpc_timeout, rx).await { 244 | Ok(Ok(RaftResponse::Response { data })) => Ok(data), 245 | Ok(Ok(RaftResponse::Error(e))) => Err(Error::from(e)), 246 | _ => Err(Error::Unknown), 247 | }, 248 | Err(e) => Err(Error::SendError(e.to_string())), 249 | } 250 | } 251 | 252 | /// Sends a request to leave the Raft cluster. 253 | /// It initiates a `ConfigChange` to remove the node from the cluster and waits for a response. 254 | #[inline] 255 | pub async fn leave(&self) -> Result<()> { 256 | let mut change = ConfChange::default(); 257 | // set node id to 0, the node will set it to self when it receives it. 258 | change.set_node_id(0); 259 | change.set_change_type(ConfChangeType::RemoveNode); 260 | change.set_context(serialize(&RemoveNodeType::Normal)?); 261 | let mut sender = self.sender.clone(); 262 | let (chan, rx) = oneshot::channel(); 263 | match sender.send(Message::ConfigChange { change, chan }).await { 264 | Ok(()) => match rx.await { 265 | Ok(RaftResponse::Ok) => Ok(()), 266 | Ok(RaftResponse::Error(e)) => Err(Error::from(e)), 267 | _ => Err(Error::Unknown), 268 | }, 269 | Err(e) => Err(Error::SendError(e.to_string())), 270 | } 271 | } 272 | 273 | /// Retrieves the current status of the Raft node. 274 | /// Sends a `Message::Status` request and waits for a `RaftResponse::Status` reply, which contains the node's status. 275 | #[inline] 276 | pub async fn status(&self) -> Result { 277 | let (tx, rx) = oneshot::channel(); 278 | let mut sender = self.sender.clone(); 279 | match sender.send(Message::Status { chan: tx }).await { 280 | Ok(_) => match timeout(self.grpc_timeout, rx).await { 281 | Ok(Ok(RaftResponse::Status(status))) => Ok(status), 282 | Ok(Ok(RaftResponse::Error(e))) => Err(Error::from(e)), 283 | _ => Err(Error::Unknown), 284 | }, 285 | Err(e) => Err(Error::SendError(e.to_string())), 286 | } 287 | } 288 | 289 | /// Retrieves leader information, including whether the current node is the leader, the leader ID, and its address. 290 | /// This method sends a `Message::RequestId` and waits for a response with the leader's ID and address. 291 | #[inline] 292 | async fn _get_leader_info(&self) -> std::result::Result { 293 | let (tx, rx) = oneshot::channel(); 294 | let mut sender = self.sender.clone(); 295 | match sender.send(Message::RequestId { chan: tx }).await { 296 | Ok(_) => match timeout(self.grpc_timeout, rx).await { 297 | Ok(Ok(RaftResponse::RequestId { leader_id })) => Ok(LeaderInfo { 298 | leader: true, 299 | target_leader_id: leader_id, 300 | target_leader_addr: None, 301 | }), 302 | Ok(Ok(RaftResponse::WrongLeader { 303 | leader_id, 304 | leader_addr, 305 | })) => Ok(LeaderInfo { 306 | leader: false, 307 | target_leader_id: leader_id, 308 | target_leader_addr: leader_addr, 309 | }), 310 | Ok(Ok(RaftResponse::Error(e))) => Err(LeaderInfoError::from(e)), 311 | _ => Err("Unknown".into()), 312 | }, 313 | Err(e) => Err(LeaderInfoError::from(e.to_string())), 314 | } 315 | } 316 | 317 | #[inline] 318 | async fn get_leader_info(&self) -> Result { 319 | { 320 | let leader_info = self.leader_info.read().await; 321 | if let Some((leader_info, err, inst)) = leader_info.as_ref() { 322 | if inst.elapsed().as_secs() < 5 { 323 | if let Some(leader_info) = leader_info { 324 | return Ok(leader_info.clone()); 325 | } 326 | if let Some(err) = err { 327 | return Err(err.to_string().into()); 328 | } 329 | } 330 | } 331 | } 332 | 333 | let mut write = self.leader_info.write().await; 334 | 335 | return match self._get_leader_info().await { 336 | Ok(leader_info) => { 337 | write.replace((Some(leader_info.clone()), None, std::time::Instant::now())); 338 | Ok(leader_info) 339 | } 340 | Err(e) => { 341 | let err = e.to_string().into(); 342 | write.replace((None, Some(e), std::time::Instant::now())); 343 | Err(err) 344 | } 345 | }; 346 | } 347 | } 348 | 349 | pub struct Raft { 350 | store: S, 351 | tx: mpsc::Sender, 352 | rx: mpsc::Receiver, 353 | laddr: SocketAddr, 354 | logger: slog::Logger, 355 | cfg: Arc, 356 | } 357 | 358 | impl Raft { 359 | /// Creates a new Raft node with the provided address, store, logger, and configuration. 360 | /// The node communicates with other peers using a mailbox. 361 | pub fn new( 362 | laddr: A, 363 | store: S, 364 | logger: slog::Logger, 365 | cfg: Config, 366 | ) -> Result { 367 | let laddr = laddr 368 | .to_socket_addrs()? 369 | .next() 370 | .ok_or_else(|| Error::from("None"))?; 371 | let (tx, rx) = mpsc::channel(100_000); 372 | let cfg = Arc::new(cfg); 373 | Ok(Self { 374 | store, 375 | tx, 376 | rx, 377 | laddr, 378 | logger, 379 | cfg, 380 | }) 381 | } 382 | 383 | /// Returns a `Mailbox` for the Raft node, which facilitates communication with peers. 384 | pub fn mailbox(&self) -> Mailbox { 385 | Mailbox::new( 386 | Arc::new(DashMap::default()), 387 | self.tx.clone(), 388 | self.cfg.grpc_timeout, 389 | self.cfg.grpc_concurrency_limit, 390 | self.cfg.grpc_message_size, 391 | self.cfg.grpc_breaker_threshold, 392 | self.cfg.grpc_breaker_retry_interval.as_millis() as i64, 393 | ) 394 | } 395 | 396 | /// Finds leader information by querying a list of peer addresses. 397 | /// Returns the leader ID and its address if found. 398 | pub async fn find_leader_info(&self, peer_addrs: Vec) -> Result> { 399 | let mut futs = Vec::new(); 400 | for addr in peer_addrs { 401 | let fut = async { 402 | let _addr = addr.clone(); 403 | match self.request_leader(addr).await { 404 | Ok(reply) => Ok(reply), 405 | Err(e) => Err(e), 406 | } 407 | }; 408 | futs.push(fut.boxed()); 409 | } 410 | 411 | let (leader_id, leader_addr) = match futures::future::select_ok(futs).await { 412 | Ok((Some((leader_id, leader_addr)), _)) => (leader_id, leader_addr), 413 | Ok((None, _)) => return Err(Error::LeaderNotExist), 414 | Err(_e) => return Ok(None), 415 | }; 416 | 417 | if leader_id == 0 { 418 | Ok(None) 419 | } else { 420 | Ok(Some((leader_id, leader_addr))) 421 | } 422 | } 423 | 424 | /// Requests the leader information from a specific peer. 425 | /// Sends a `Message::RequestId` to the peer and waits for the response. 426 | async fn request_leader(&self, peer_addr: String) -> Result> { 427 | let (leader_id, leader_addr): (u64, String) = { 428 | let mut client = connect( 429 | &peer_addr, 430 | 1, 431 | self.cfg.grpc_message_size, 432 | self.cfg.grpc_timeout, 433 | ) 434 | .await?; 435 | let response = client 436 | .request_id(Request::new(Empty::default())) 437 | .await? 438 | .into_inner(); 439 | match response.code() { 440 | ResultCode::WrongLeader => { 441 | let (leader_id, addr): (u64, Option) = deserialize(&response.data)?; 442 | if let Some(addr) = addr { 443 | (leader_id, addr) 444 | } else { 445 | return Ok(None); 446 | } 447 | } 448 | ResultCode::Ok => (deserialize(&response.data)?, peer_addr), 449 | ResultCode::Error => return Ok(None), 450 | } 451 | }; 452 | Ok(Some((leader_id, leader_addr))) 453 | } 454 | 455 | /// The `lead` function transitions the current node to the leader role in a Raft cluster. 456 | /// It initializes the leader node and runs both the Raft server and the node concurrently. 457 | /// The function will return once the server or node experiences an error, or when the leader 458 | /// role is relinquished. 459 | /// 460 | /// # Arguments 461 | /// 462 | /// * `node_id` - The unique identifier for the node. 463 | /// 464 | /// # Returns 465 | /// 466 | /// A `Result<()>` indicating success or failure during the process. 467 | pub async fn lead(self, node_id: u64) -> Result<()> { 468 | let node = RaftNode::new_leader( 469 | self.rx, 470 | self.tx.clone(), 471 | node_id, 472 | self.store, 473 | &self.logger, 474 | self.cfg.clone(), 475 | ) 476 | .await?; 477 | 478 | let server = RaftServer::new(self.tx, self.laddr, self.cfg.clone()); 479 | let server_handle = async { 480 | if let Err(e) = server.run().await { 481 | warn!("raft server run error: {:?}", e); 482 | Err(e) 483 | } else { 484 | Ok(()) 485 | } 486 | }; 487 | let node_handle = async { 488 | if let Err(e) = node.run().await { 489 | warn!("node run error: {:?}", e); 490 | Err(e) 491 | } else { 492 | Ok(()) 493 | } 494 | }; 495 | 496 | tokio::try_join!(server_handle, node_handle)?; 497 | info!("leaving leader node"); 498 | 499 | Ok(()) 500 | } 501 | 502 | /// The `join` function is used to make the current node join an existing Raft cluster. 503 | /// It tries to discover the current leader, communicates with the leader to join the cluster, 504 | /// and configures the node as a follower. 505 | /// 506 | /// # Arguments 507 | /// 508 | /// * `node_id` - The unique identifier for the current node. 509 | /// * `node_addr` - The address of the current node. 510 | /// * `leader_id` - The optional leader node's identifier (if already known). 511 | /// * `leader_addr` - The address of the leader node. 512 | /// 513 | /// # Returns 514 | /// 515 | /// A `Result<()>` indicating success or failure during the joining process. 516 | pub async fn join( 517 | self, 518 | node_id: u64, 519 | node_addr: String, 520 | leader_id: Option, 521 | leader_addr: String, 522 | ) -> Result<()> { 523 | // 1. try to discover the leader and obtain an id from it, if leader_id is None. 524 | info!("attempting to join peer cluster at {}", leader_addr); 525 | let (leader_id, leader_addr): (u64, String) = if let Some(leader_id) = leader_id { 526 | (leader_id, leader_addr) 527 | } else { 528 | self.request_leader(leader_addr) 529 | .await? 530 | .ok_or(Error::JoinError)? 531 | }; 532 | 533 | // 2. run server and node to prepare for joining 534 | let mut node = RaftNode::new_follower( 535 | self.rx, 536 | self.tx.clone(), 537 | node_id, 538 | self.store, 539 | &self.logger, 540 | self.cfg.clone(), 541 | )?; 542 | let peer = node.add_peer(&leader_addr, leader_id); 543 | let mut client = peer.client().await?; 544 | let server = RaftServer::new(self.tx, self.laddr, self.cfg.clone()); 545 | let server_handle = async { 546 | if let Err(e) = server.run().await { 547 | warn!("raft server run error: {:?}", e); 548 | Err(e) 549 | } else { 550 | Ok(()) 551 | } 552 | }; 553 | 554 | let node_handle = async { 555 | tokio::time::sleep(Duration::from_millis(1500)).await; 556 | //try remove from the cluster 557 | let mut change_remove = ConfChange::default(); 558 | change_remove.set_node_id(node_id); 559 | change_remove.set_change_type(ConfChangeType::RemoveNode); 560 | change_remove.set_context(serialize(&RemoveNodeType::Stale)?); 561 | let change_remove = RiteraftConfChange { 562 | inner: ConfChange::encode_to_vec(&change_remove), 563 | }; 564 | 565 | let raft_response = client 566 | .change_config(Request::new(change_remove)) 567 | .await? 568 | .into_inner(); 569 | 570 | info!( 571 | "change_remove raft_response: {:?}", 572 | deserialize::(&raft_response.inner)? 573 | ); 574 | 575 | // 3. Join the cluster 576 | // TODO: handle wrong leader 577 | let mut change = ConfChange::default(); 578 | change.set_node_id(node_id); 579 | change.set_change_type(ConfChangeType::AddNode); 580 | change.set_context(serialize(&node_addr)?); 581 | // change.set_context(serialize(&node_addr)?); 582 | 583 | let change = RiteraftConfChange { 584 | inner: ConfChange::encode_to_vec(&change), 585 | }; 586 | let raft_response = client 587 | .change_config(Request::new(change)) 588 | .await? 589 | .into_inner(); 590 | if let RaftResponse::JoinSuccess { 591 | assigned_id, 592 | peer_addrs, 593 | } = deserialize(&raft_response.inner)? 594 | { 595 | info!( 596 | "change_config response.assigned_id: {:?}, peer_addrs: {:?}", 597 | assigned_id, peer_addrs 598 | ); 599 | for (id, addr) in peer_addrs { 600 | if id != assigned_id { 601 | node.add_peer(&addr, id); 602 | } 603 | } 604 | } else { 605 | return Err(Error::JoinError); 606 | } 607 | 608 | if let Err(e) = node.run().await { 609 | warn!("node run error: {:?}", e); 610 | Err(e) 611 | } else { 612 | Ok(()) 613 | } 614 | }; 615 | let _ = tokio::try_join!(server_handle, node_handle)?; 616 | info!("leaving follower node"); 617 | Ok(()) 618 | } 619 | } 620 | -------------------------------------------------------------------------------- /src/raft_node.rs: -------------------------------------------------------------------------------- 1 | use std::collections::HashMap; 2 | use std::ops::{Deref, DerefMut}; 3 | use std::sync::atomic::{AtomicI64, AtomicIsize, AtomicU64, Ordering}; 4 | use std::sync::Arc; 5 | use std::time::{Duration, Instant}; 6 | 7 | use anyhow::anyhow; 8 | use bincode::{deserialize, serialize}; 9 | use box_counter::Counter; 10 | use bytestring::ByteString; 11 | use futures::channel::{mpsc, oneshot}; 12 | use futures::SinkExt; 13 | use futures::StreamExt; 14 | use log::*; 15 | use prost::Message as _; 16 | use scopeguard::guard; 17 | use tikv_raft::eraftpb::{ConfChange, ConfChangeType, Entry, EntryType, Message as RaftMessage}; 18 | use tikv_raft::{prelude::*, raw_node::RawNode, Config as RaftConfig}; 19 | use tokio::sync::RwLock; 20 | use tokio::time::timeout; 21 | use tonic::Request; 22 | 23 | use crate::error::{Error, Result}; 24 | use crate::message::{ 25 | Merger, Message, PeerState, Proposals, RaftResponse, RemoveNodeType, ReplyChan, Status, 26 | }; 27 | use crate::raft::Store; 28 | use crate::raft_service::raft_service_client::RaftServiceClient; 29 | use crate::raft_service::{connect, Message as RraftMessage, Proposal as RraftProposal, Query}; 30 | use crate::storage::{LogStore, MemStorage}; 31 | use crate::timeout_recorder::TimeoutRecorder; 32 | use crate::Config; 33 | 34 | pub type RaftGrpcClient = RaftServiceClient; 35 | 36 | struct MessageSender { 37 | message: RaftMessage, 38 | client: Peer, 39 | client_id: u64, 40 | chan: mpsc::Sender, 41 | max_retries: usize, 42 | timeout: Duration, 43 | sending_raft_messages: Arc, 44 | } 45 | 46 | impl MessageSender { 47 | /// attempt to send a message MessageSender::max_retries times at MessageSender::timeout 48 | /// inteval. 49 | async fn send(self) { 50 | let sending_raft_messages = self.sending_raft_messages.clone(); 51 | sending_raft_messages.fetch_add(1, Ordering::SeqCst); 52 | let _guard = guard((), |_| { 53 | sending_raft_messages.fetch_sub(1, Ordering::SeqCst); 54 | }); 55 | self._send().await 56 | } 57 | 58 | async fn _send(mut self) { 59 | let mut current_retry = 0usize; 60 | loop { 61 | match self.client.send_message(&self.message).await { 62 | Ok(_) => { 63 | return; 64 | } 65 | Err(e) => { 66 | if current_retry < self.max_retries { 67 | current_retry += 1; 68 | tokio::time::sleep(self.timeout).await; 69 | } else { 70 | warn!( 71 | "error sending message after {}/{} retries: {:?}, target addr: {:?}", 72 | current_retry, self.max_retries, e, self.client.addr 73 | ); 74 | if let Err(e) = self 75 | .chan 76 | .send(Message::ReportUnreachable { 77 | node_id: self.client_id, 78 | }) 79 | .await 80 | { 81 | warn!( 82 | "error ReportUnreachable after {}/{} retries: {:?}, target addr: {:?}", 83 | current_retry, self.max_retries, e, self.client.addr 84 | ); 85 | } 86 | return; 87 | } 88 | } 89 | } 90 | } 91 | } 92 | } 93 | 94 | struct QuerySender { 95 | query: Vec, 96 | client: Peer, 97 | chan: oneshot::Sender, 98 | max_retries: usize, 99 | timeout: Duration, 100 | } 101 | 102 | impl QuerySender { 103 | async fn send(self) { 104 | let mut current_retry = 0usize; 105 | 106 | let mut client = match self.client.client().await { 107 | Ok(c) => c, 108 | Err(e) => { 109 | warn!( 110 | "error sending query after, {:?}, target addr: {:?}", 111 | e, self.client.addr 112 | ); 113 | if let Err(e) = self.chan.send(RaftResponse::Error(e.to_string())) { 114 | warn!( 115 | "send_query, Message::Query, RaftResponse send error: {:?}, target addr: {:?}", 116 | e, self.client.addr 117 | ); 118 | } 119 | return; 120 | } 121 | }; 122 | 123 | loop { 124 | let message_request = Request::new(Query { 125 | inner: self.query.clone(), 126 | }); 127 | match client.send_query(message_request).await { 128 | Ok(grpc_response) => { 129 | let raft_response = match deserialize(&grpc_response.into_inner().inner) { 130 | Ok(resp) => resp, 131 | Err(e) => { 132 | warn!( 133 | "send_query, Message::Query, RaftResponse deserialize error: {:?}, target addr: {:?}", 134 | e, self.client.addr 135 | ); 136 | return; 137 | } 138 | }; 139 | if let Err(e) = self.chan.send(raft_response) { 140 | warn!( 141 | "send_query, Message::Query, RaftResponse send error: {:?}, target addr: {:?}", 142 | e, self.client.addr 143 | ); 144 | } 145 | return; 146 | } 147 | Err(e) => { 148 | if current_retry < self.max_retries { 149 | current_retry += 1; 150 | tokio::time::sleep(self.timeout).await; 151 | } else { 152 | warn!( 153 | "error sending query after {} retries: {}, target addr: {:?}", 154 | self.max_retries, e, self.client.addr 155 | ); 156 | if let Err(e) = self.chan.send(RaftResponse::Error(e.to_string())) { 157 | warn!( 158 | "send_query, Message::Query, RaftResponse send error: {:?}, target addr: {:?}", 159 | e, self.client.addr 160 | ); 161 | } 162 | return; 163 | } 164 | } 165 | } 166 | } 167 | } 168 | } 169 | 170 | #[derive(Clone)] 171 | pub struct Peer { 172 | addr: ByteString, 173 | client: Arc>>, 174 | grpc_fails: Arc, 175 | grpc_fail_time: Arc, 176 | crw_timeout: Duration, 177 | concurrency_limit: usize, 178 | grpc_message_size: usize, 179 | grpc_breaker_threshold: u64, 180 | grpc_breaker_retry_interval: i64, 181 | active_tasks: Arc, 182 | } 183 | 184 | impl Peer { 185 | /// Creates a new `Peer` instance with the specified parameters. 186 | /// 187 | /// # Parameters 188 | /// - `addr`: The address of the peer to connect to. 189 | /// - `crw_timeout`: The timeout duration for connection and read/write operations. 190 | /// - `concurrency_limit`: The maximum number of concurrent gRPC requests allowed. 191 | /// - `grpc_message_size`: The maximum size of a gRPC message. 192 | /// - `grpc_breaker_threshold`: The threshold for the number of gRPC failures before breaking the circuit. 193 | /// - `grpc_breaker_retry_interval`: The time interval for retrying after the circuit breaker is tripped. 194 | /// 195 | /// # Returns 196 | /// - A new `Peer` instance with the provided configuration. 197 | /// 198 | /// # Behavior 199 | /// - Initializes internal state, including counters and timeouts. 200 | /// - Logs the connection attempt to the specified address. 201 | pub fn new( 202 | addr: String, 203 | crw_timeout: Duration, 204 | concurrency_limit: usize, 205 | grpc_message_size: usize, 206 | grpc_breaker_threshold: u64, 207 | grpc_breaker_retry_interval: i64, 208 | ) -> Peer { 209 | debug!("connecting to node at {}...", addr); 210 | Peer { 211 | addr: addr.into(), 212 | client: Arc::new(RwLock::new(None)), 213 | grpc_fails: Arc::new(AtomicU64::new(0)), 214 | grpc_fail_time: Arc::new(AtomicI64::new(0)), 215 | crw_timeout, 216 | concurrency_limit, 217 | grpc_message_size, 218 | grpc_breaker_threshold, 219 | grpc_breaker_retry_interval, 220 | active_tasks: Arc::new(AtomicI64::new(0)), 221 | } 222 | } 223 | 224 | /// Returns the number of currently active tasks associated with this peer. 225 | /// 226 | /// # Returns 227 | /// - The number of active tasks as an `i64`. 228 | /// 229 | /// # Behavior 230 | /// - Reads the value of the `active_tasks` counter. 231 | #[inline] 232 | pub fn active_tasks(&self) -> i64 { 233 | self.active_tasks.load(Ordering::SeqCst) 234 | } 235 | 236 | /// Returns the number of gRPC failures encountered by this peer. 237 | /// 238 | /// # Returns 239 | /// - The number of gRPC failures as a `u64`. 240 | /// 241 | /// # Behavior 242 | /// - Reads the value of the `grpc_fails` counter. 243 | #[inline] 244 | pub fn grpc_fails(&self) -> u64 { 245 | self.grpc_fails.load(Ordering::SeqCst) 246 | } 247 | 248 | /// Connects to the peer if not already connected, and returns the gRPC client. 249 | /// 250 | /// # Returns 251 | /// - `Ok(RaftGrpcClient)`: On successful connection, returns the gRPC client. 252 | /// - `Err(Error)`: On failure, returns an error. 253 | /// 254 | /// # Behavior 255 | /// - Checks if the gRPC client is already connected and available. 256 | /// - If not, attempts to establish a new connection and store the client. 257 | #[inline] 258 | async fn connect(&self) -> Result { 259 | if let Some(c) = self.client.read().await.as_ref() { 260 | return Ok(c.clone()); 261 | } 262 | 263 | let mut client = self.client.write().await; 264 | if let Some(c) = client.as_ref() { 265 | return Ok(c.clone()); 266 | } 267 | 268 | let c = connect( 269 | &self.addr, 270 | self.concurrency_limit, 271 | self.grpc_message_size, 272 | self.crw_timeout, 273 | ) 274 | .await?; 275 | client.replace(c.clone()); 276 | Ok(c) 277 | } 278 | 279 | /// Retrieves the gRPC client by establishing a connection if needed. 280 | /// 281 | /// # Returns 282 | /// - `Ok(RaftGrpcClient)`: On successful connection, returns the gRPC client. 283 | /// - `Err(Error)`: On failure, returns an error. 284 | /// 285 | /// # Behavior 286 | /// - Calls `connect` to ensure the client is connected and available. 287 | #[inline] 288 | pub async fn client(&self) -> Result { 289 | self.connect().await 290 | } 291 | 292 | /// Sends a Raft message to the peer and waits for a response. 293 | /// 294 | /// # Parameters 295 | /// - `msg`: The Raft message to be sent. 296 | /// 297 | /// # Returns 298 | /// - `Ok(Vec)`: On successful message send, returns the response data as a byte vector. 299 | /// - `Err(Error)`: On failure, returns an error. 300 | /// 301 | /// # Behavior 302 | /// - Checks if the peer is available for sending messages. 303 | /// - Encodes the message and sends it using the `_send_message` method. 304 | /// - Updates the active task count and records success or failure. 305 | #[inline] 306 | pub async fn send_message(&self, msg: &RaftMessage) -> Result> { 307 | if !self.available() { 308 | return Err(Error::Msg("The gRPC remote service is unavailable".into())); 309 | } 310 | 311 | let msg = RraftMessage { 312 | inner: RaftMessage::encode_to_vec(msg), 313 | }; 314 | self.active_tasks.fetch_add(1, Ordering::SeqCst); 315 | let reply = self._send_message(msg).await; 316 | self.active_tasks.fetch_sub(1, Ordering::SeqCst); 317 | match reply { 318 | Ok(reply) => { 319 | self.record_success(); 320 | Ok(reply) 321 | } 322 | Err(e) => { 323 | self.record_failure(); 324 | Err(e) 325 | } 326 | } 327 | } 328 | 329 | #[inline] 330 | async fn _send_message(&self, msg: RraftMessage) -> Result> { 331 | let c = self.connect().await?; 332 | async fn task(mut c: RaftGrpcClient, msg: RraftMessage) -> Result> { 333 | let message_request = Request::new(msg); 334 | let response = c.send_message(message_request).await?; 335 | let message_reply = response.into_inner(); 336 | Ok(message_reply.inner) 337 | } 338 | 339 | let result = tokio::time::timeout(self.crw_timeout, task(c, msg)).await; 340 | let result = result.map_err(|_| Error::Elapsed)??; 341 | Ok(result) 342 | } 343 | 344 | /// Sends a Raft proposal to the peer and waits for a response. 345 | /// 346 | /// # Parameters 347 | /// - `msg`: The Raft proposal to be sent as a byte vector. 348 | /// 349 | /// # Returns 350 | /// - `Ok(Vec)`: On successful proposal send, returns the response data as a byte vector. 351 | /// - `Err(Error)`: On failure, returns an error. 352 | /// 353 | /// # Behavior 354 | /// - Checks if the peer is available for sending proposals. 355 | /// - Wraps the proposal in a `RraftProposal` and sends it using the `_send_proposal` method. 356 | /// - Updates the active task count and records success or failure. 357 | #[inline] 358 | pub async fn send_proposal(&self, msg: Vec) -> Result> { 359 | if !self.available() { 360 | return Err(Error::Msg("The gRPC remote service is unavailable".into())); 361 | } 362 | 363 | let msg = RraftProposal { inner: msg }; 364 | let _active_tasks = self.active_tasks.fetch_add(1, Ordering::SeqCst); 365 | let reply = self._send_proposal(msg).await; 366 | self.active_tasks.fetch_sub(1, Ordering::SeqCst); 367 | match reply { 368 | Ok(reply) => { 369 | self.record_success(); 370 | Ok(reply) 371 | } 372 | Err(e) => { 373 | self.record_failure(); 374 | Err(e) 375 | } 376 | } 377 | } 378 | 379 | #[inline] 380 | async fn _send_proposal(&self, msg: RraftProposal) -> Result> { 381 | let c = self.connect().await?; 382 | 383 | async fn task(mut c: RaftGrpcClient, msg: RraftProposal) -> Result> { 384 | let message_request = Request::new(msg); 385 | let response = c.send_proposal(message_request).await?; 386 | let message_reply = response.into_inner(); 387 | Ok(message_reply.inner) 388 | } 389 | 390 | let result = tokio::time::timeout(self.crw_timeout, task(c, msg)).await; 391 | let result = result.map_err(|_| Error::Elapsed)??; 392 | Ok(result) 393 | } 394 | 395 | #[inline] 396 | fn record_failure(&self) { 397 | self.grpc_fails.fetch_add(1, Ordering::SeqCst); 398 | self.grpc_fail_time 399 | .store(chrono::Local::now().timestamp_millis(), Ordering::SeqCst); 400 | } 401 | 402 | #[inline] 403 | fn record_success(&self) { 404 | self.grpc_fails.store(0, Ordering::SeqCst); 405 | } 406 | 407 | #[inline] 408 | pub(crate) fn is_unavailable(&self) -> bool { 409 | self.grpc_fails.load(Ordering::SeqCst) >= self.grpc_breaker_threshold 410 | } 411 | 412 | #[inline] 413 | fn available(&self) -> bool { 414 | self.grpc_fails.load(Ordering::SeqCst) < self.grpc_breaker_threshold 415 | || (chrono::Local::now().timestamp_millis() 416 | - self.grpc_fail_time.load(Ordering::SeqCst)) 417 | > self.grpc_breaker_retry_interval 418 | } 419 | } 420 | 421 | pub struct RaftNode { 422 | inner: RawNode, 423 | pub peers: HashMap>, 424 | pub rcv: mpsc::Receiver, 425 | pub snd: mpsc::Sender, 426 | store: S, 427 | uncommitteds: HashMap, 428 | should_quit: bool, 429 | seq: AtomicU64, 430 | sending_raft_messages: Arc, 431 | last_snap_time: Instant, 432 | cfg: Arc, 433 | timeout_recorder: TimeoutRecorder, 434 | propose_counter: Counter, 435 | } 436 | 437 | impl RaftNode { 438 | /// Creates a new leader node for the Raft cluster. 439 | /// 440 | /// This function initializes a new `RaftNode` instance as a leader. It sets up the Raft configuration, 441 | /// applies a default snapshot to initialize the state, and sets the node to be a leader. 442 | /// 443 | /// # Parameters 444 | /// - `rcv`: A receiver for Raft messages. This will be used to receive incoming messages. 445 | /// - `snd`: A sender for Raft messages. This will be used to send outgoing messages. 446 | /// - `id`: The unique identifier for this Raft node. 447 | /// - `store`: The store implementation used for persisting Raft state. 448 | /// - `logger`: A logger instance for logging messages related to the Raft node. 449 | /// - `cfg`: Configuration for the Raft node, including various timeouts and limits. 450 | /// 451 | /// # Returns 452 | /// Returns a `Result` containing either the newly created `RaftNode` or an error if the creation failed. 453 | pub async fn new_leader( 454 | rcv: mpsc::Receiver, 455 | snd: mpsc::Sender, 456 | id: u64, 457 | store: S, 458 | logger: &slog::Logger, 459 | cfg: Arc, 460 | ) -> Result { 461 | let config = Self::new_config(id, &cfg.raft_cfg); 462 | config.validate()?; 463 | 464 | let mut s = Snapshot::default(); 465 | // Because we don't use the same configuration to initialize every node, so we use 466 | // a non-zero index to force new followers catch up logs by snapshot first, which will 467 | // bring all nodes to the same initial state. 468 | s.mut_metadata().index = 1; 469 | s.mut_metadata().term = 1; 470 | s.mut_metadata().mut_conf_state().voters = vec![id]; 471 | s.set_data(store.snapshot().await?); 472 | 473 | let mut storage: MemStorage = MemStorage::create(); 474 | storage.apply_snapshot(s)?; 475 | let mut inner = RawNode::new(&config, storage, logger)?; 476 | let peers = HashMap::new(); 477 | let seq = AtomicU64::new(0); 478 | let sending_raft_messages = Arc::new(AtomicIsize::new(0)); 479 | let last_snap_time = Instant::now(); // + cfg.snapshot_interval; 480 | 481 | inner.raft.become_candidate(); 482 | inner.raft.become_leader(); 483 | 484 | // let msg_tx = Self::start_message_sender(); 485 | let uncommitteds = HashMap::new(); 486 | let node = RaftNode { 487 | inner, 488 | rcv, 489 | peers, 490 | store, 491 | // msg_tx, 492 | uncommitteds, 493 | seq, 494 | snd, 495 | should_quit: false, 496 | sending_raft_messages, 497 | last_snap_time, 498 | cfg, 499 | timeout_recorder: TimeoutRecorder::new(Duration::from_secs(15), 5), 500 | propose_counter: Counter::new(Duration::from_secs(3)), 501 | }; 502 | Ok(node) 503 | } 504 | 505 | /// Creates a new follower node for the Raft cluster. 506 | /// 507 | /// This function initializes a new `RaftNode` instance as a follower. It sets up the Raft configuration 508 | /// and creates a new `RawNode` instance in follower mode. 509 | /// 510 | /// # Parameters 511 | /// - `rcv`: A receiver for Raft messages. This will be used to receive incoming messages. 512 | /// - `snd`: A sender for Raft messages. This will be used to send outgoing messages. 513 | /// - `id`: The unique identifier for this Raft node. 514 | /// - `store`: The store implementation used for persisting Raft state. 515 | /// - `logger`: A logger instance for logging messages related to the Raft node. 516 | /// - `cfg`: Configuration for the Raft node, including various timeouts and limits. 517 | /// 518 | /// # Returns 519 | /// Returns a `Result` containing either the newly created `RaftNode` or an error if the creation failed. 520 | pub fn new_follower( 521 | rcv: mpsc::Receiver, 522 | snd: mpsc::Sender, 523 | id: u64, 524 | store: S, 525 | logger: &slog::Logger, 526 | cfg: Arc, 527 | ) -> Result { 528 | let config = Self::new_config(id, &cfg.raft_cfg); 529 | config.validate()?; 530 | 531 | let storage = MemStorage::create(); 532 | let inner = RawNode::new(&config, storage, logger)?; 533 | let peers = HashMap::new(); 534 | let seq = AtomicU64::new(0); 535 | let sending_raft_messages = Arc::new(AtomicIsize::new(0)); 536 | let last_snap_time = Instant::now(); // + cfg.snapshot_interval; 537 | // let msg_tx = Self::start_message_sender(); 538 | let uncommitteds = HashMap::new(); 539 | Ok(RaftNode { 540 | inner, 541 | rcv, 542 | peers, 543 | store, 544 | // msg_tx, 545 | uncommitteds, 546 | seq, 547 | snd, 548 | should_quit: false, 549 | sending_raft_messages, 550 | last_snap_time, 551 | cfg, 552 | timeout_recorder: TimeoutRecorder::new(Duration::from_secs(10), 5), 553 | propose_counter: Counter::new(Duration::from_secs(3)), 554 | }) 555 | } 556 | 557 | /// Creates a new Raft configuration with the specified node ID. 558 | /// 559 | /// This function clones the provided configuration and sets the node ID. 560 | /// 561 | /// # Parameters 562 | /// - `id`: The unique identifier for the Raft node. 563 | /// - `cfg`: The base Raft configuration to clone and modify. 564 | /// 565 | /// # Returns 566 | /// Returns a `RaftConfig` with the updated node ID. 567 | #[inline] 568 | fn new_config(id: u64, cfg: &RaftConfig) -> RaftConfig { 569 | let mut cfg = cfg.clone(); 570 | cfg.id = id; 571 | cfg 572 | } 573 | 574 | /// Retrieves a peer by its ID. 575 | /// 576 | /// This function looks up a peer in the `peers` map by its ID. 577 | /// 578 | /// # Parameters 579 | /// - `id`: The ID of the peer to retrieve. 580 | /// 581 | /// # Returns 582 | /// Returns an `Option`. If the peer is found, it is returned; otherwise, `None` is returned. 583 | #[inline] 584 | pub fn peer(&self, id: u64) -> Option { 585 | match self.peers.get(&id) { 586 | Some(Some(p)) => Some(p.clone()), 587 | _ => None, 588 | } 589 | } 590 | 591 | /// Checks if the current node is the leader. 592 | /// 593 | /// This function compares the leader ID of the Raft instance with the current node's ID. 594 | /// 595 | /// # Returns 596 | /// Returns `true` if the current node is the leader, otherwise `false`. 597 | #[inline] 598 | pub fn is_leader(&self) -> bool { 599 | self.inner.raft.leader_id == self.inner.raft.id 600 | } 601 | 602 | /// Retrieves the ID of the current node. 603 | /// 604 | /// This function returns the unique identifier of the current Raft node. 605 | /// 606 | /// # Returns 607 | /// Returns the node's ID as a `u64`. 608 | #[inline] 609 | pub fn id(&self) -> u64 { 610 | self.raft.id 611 | } 612 | 613 | /// Adds a new peer to the `peers` map. 614 | /// 615 | /// This function creates a new `Peer` instance with the specified address and configuration, 616 | /// and adds it to the `peers` map. 617 | /// 618 | /// # Parameters 619 | /// - `addr`: The address of the new peer. 620 | /// - `id`: The unique identifier for the new peer. 621 | /// 622 | /// # Returns 623 | /// Returns the newly created `Peer` instance. 624 | #[inline] 625 | pub fn add_peer(&mut self, addr: &str, id: u64) -> Peer { 626 | let peer = Peer::new( 627 | addr.to_string(), 628 | self.cfg.grpc_timeout, 629 | self.cfg.grpc_concurrency_limit, 630 | self.cfg.grpc_message_size, 631 | self.cfg.grpc_breaker_threshold, 632 | self.cfg.grpc_breaker_retry_interval.as_millis() as i64, 633 | ); 634 | self.peers.insert(id, Some(peer.clone())); 635 | peer 636 | } 637 | 638 | #[inline] 639 | fn leader(&self) -> u64 { 640 | self.raft.leader_id 641 | } 642 | 643 | #[inline] 644 | fn has_leader(&self) -> bool { 645 | self.raft.leader_id > 0 646 | } 647 | 648 | #[inline] 649 | fn peer_addrs(&self) -> HashMap { 650 | self.peers 651 | .iter() 652 | .filter_map(|(&id, peer)| { 653 | peer.as_ref() 654 | .map(|Peer { addr, .. }| (id, addr.to_string())) 655 | }) 656 | .collect() 657 | } 658 | 659 | #[inline] 660 | fn peer_states(&self) -> HashMap> { 661 | self.peers 662 | .iter() 663 | .map(|(id, peer)| { 664 | if let Some(p) = peer { 665 | ( 666 | *id, 667 | Some(PeerState { 668 | addr: p.addr.clone(), 669 | available: !p.is_unavailable(), 670 | }), 671 | ) 672 | } else { 673 | (*id, None) 674 | } 675 | }) 676 | .collect() 677 | } 678 | 679 | #[inline] 680 | async fn status(&self, merger_proposals: usize) -> Status { 681 | let role = self.raft.state; 682 | let leader_id = self.raft.leader_id; 683 | let sending_raft_messages = self.sending_raft_messages.load(Ordering::SeqCst); 684 | let timeout_max = self.timeout_recorder.max() as isize; 685 | let timeout_recent_count = self.timeout_recorder.recent_get() as isize; 686 | let propose_count = self.propose_counter.count(); 687 | let propose_rate = self.propose_counter.rate(); 688 | Status { 689 | id: self.inner.raft.id, 690 | leader_id, 691 | uncommitteds: self.uncommitteds.len(), 692 | merger_proposals, 693 | sending_raft_messages, 694 | timeout_max, 695 | timeout_recent_count, 696 | propose_count, 697 | propose_rate, 698 | peers: self.peer_states(), 699 | role, 700 | } 701 | } 702 | 703 | // forward query request to leader 704 | #[inline] 705 | async fn forward_query(&self, query: Vec, chan: oneshot::Sender) { 706 | let id = self.leader(); 707 | let peer = match self.peer(id) { 708 | Some(peer) => peer, 709 | None => { 710 | if let Err(e) = chan.send(RaftResponse::WrongLeader { 711 | leader_id: id, 712 | leader_addr: None, 713 | }) { 714 | warn!( 715 | "forward_query, Message::Query, RaftResponse send error: {:?}", 716 | e 717 | ); 718 | } 719 | return; 720 | } 721 | }; 722 | 723 | let query_sender = QuerySender { 724 | query, 725 | client: peer, 726 | chan, 727 | timeout: Duration::from_millis(1000), 728 | max_retries: 0, 729 | }; 730 | tokio::spawn(query_sender.send()); 731 | } 732 | 733 | #[inline] 734 | async fn send_query(&self, query: &[u8], chan: oneshot::Sender) { 735 | let data = self.store.query(query).await.unwrap_or_default(); 736 | if let Err(e) = chan.send(RaftResponse::Response { data }) { 737 | warn!("Message::Query, RaftResponse send error: {:?}", e); 738 | } 739 | } 740 | 741 | #[inline] 742 | fn send_wrong_leader(&self, from: &str, chan: oneshot::Sender) { 743 | let leader_id = self.leader(); 744 | // leader can't be an empty node 745 | let leader_addr = self 746 | .peers 747 | .get(&leader_id) 748 | .and_then(|peer| peer.as_ref().map(|p| p.addr.to_string())); 749 | let raft_response = RaftResponse::WrongLeader { 750 | leader_id, 751 | leader_addr, 752 | }; 753 | if let Err(e) = chan.send(raft_response) { 754 | warn!( 755 | "send_wrong_leader, from: {}, RaftResponse send error: {:?}", 756 | from, e 757 | ); 758 | } 759 | } 760 | 761 | #[inline] 762 | fn send_is_busy(&self, chan: oneshot::Sender) { 763 | if let Err(e) = chan.send(RaftResponse::Busy) { 764 | warn!("send_is_busy, RaftResponse send error: {:?}", e); 765 | } 766 | } 767 | 768 | #[inline] 769 | fn _send_error(&self, chan: oneshot::Sender, e: String) { 770 | let raft_response = RaftResponse::Error(e); 771 | if let Err(e) = chan.send(raft_response) { 772 | warn!("send_error, RaftResponse send error: {:?}", e); 773 | } 774 | } 775 | 776 | #[inline] 777 | fn send_leader_id(&self, chan: oneshot::Sender) { 778 | if let Err(e) = chan.send(RaftResponse::RequestId { 779 | leader_id: self.leader(), 780 | }) { 781 | warn!("Message::RequestId, RaftResponse send error: {:?}", e); 782 | } 783 | } 784 | 785 | #[inline] 786 | async fn send_status(&self, merger: &Merger, chan: oneshot::Sender) { 787 | if let Err(e) = chan.send(RaftResponse::Status(self.status(merger.len()).await)) { 788 | warn!("Message::Status, RaftResponse send error: {:?}", e); 789 | } 790 | } 791 | 792 | #[inline] 793 | fn _take_and_propose(&mut self, merger: &mut Merger) -> Result<()> { 794 | if let Some((data, reply_chans)) = merger.take() { 795 | let seq = self.seq.fetch_add(1, Ordering::Relaxed); 796 | self.uncommitteds.insert(seq, reply_chans); 797 | let seq = serialize(&seq).map_err(|e| anyhow!(e))?; 798 | let data = serialize(&data).map_err(|e| anyhow!(e))?; 799 | self.propose(seq, data)?; 800 | } 801 | Ok(()) 802 | } 803 | 804 | #[inline] 805 | fn take_and_propose(&mut self, merger: &mut Merger) { 806 | if let Err(e) = self._take_and_propose(merger) { 807 | error!("propose to raft error, {:?}", e); 808 | } 809 | } 810 | 811 | #[inline] 812 | fn is_busy(&self) -> bool { 813 | self.sending_raft_messages.load(Ordering::SeqCst) 814 | > self.cfg.raft_cfg.max_inflight_msgs as isize 815 | || self.timeout_recorder.recent_get() > 0 816 | } 817 | 818 | pub(crate) async fn run(mut self) -> Result<()> { 819 | let mut heartbeat = self.cfg.heartbeat; 820 | let mut now = Instant::now(); 821 | let mut snapshot_received = self.is_leader(); 822 | let mut merger = Merger::new( 823 | self.cfg.proposal_batch_size, 824 | self.cfg.proposal_batch_timeout, 825 | ); 826 | info!("snapshot_received: {:?}", snapshot_received); 827 | info!("has_leader: {:?}", self.has_leader()); 828 | 829 | loop { 830 | if self.should_quit { 831 | warn!("Quitting raft"); 832 | return Ok(()); 833 | } 834 | 835 | match timeout(heartbeat, self.rcv.next()).await { 836 | Ok(Some(Message::ConfigChange { chan, mut change })) => { 837 | info!("change Received, {:?}", change); 838 | // whenever a change id is 0, it's a message to self. 839 | if change.get_node_id() == 0 { 840 | change.set_node_id(self.id()); 841 | } 842 | 843 | if !self.is_leader() { 844 | // wrong leader send client cluster data 845 | // TODO: retry strategy in case of failure 846 | self.send_wrong_leader("ConfigChange", chan); 847 | } else { 848 | // leader assign new id to peer 849 | info!("received request from: {}", change.get_node_id()); 850 | let seq = self.seq.fetch_add(1, Ordering::Relaxed); 851 | self.uncommitteds 852 | .insert(seq, ReplyChan::One((chan, Instant::now()))); 853 | match serialize(&seq) { 854 | Ok(req) => { 855 | if let Err(e) = self.propose_conf_change(req, change) { 856 | warn!("propose_conf_change, error: {:?}", e); 857 | } 858 | } 859 | Err(e) => { 860 | warn!("serialize seq, error: {:?}", e); 861 | } 862 | }; 863 | } 864 | } 865 | Ok(Some(Message::Raft(m))) => { 866 | let msg_type = m.get_msg_type(); 867 | if msg_type != MessageType::MsgHeartbeat 868 | && msg_type != MessageType::MsgHeartbeatResponse 869 | { 870 | debug!( 871 | "raft message: to={} from={} msg_type={:?}, commit={}, {:?}", 872 | self.raft.id, 873 | m.from, 874 | m.msg_type, 875 | m.get_commit(), 876 | m 877 | ); 878 | } 879 | if MessageType::MsgTransferLeader == msg_type { 880 | info!( 881 | "raft message MsgTransferLeader, snapshot_received: {}, raft.leader_id: {}, {:?}", 882 | snapshot_received, self.raft.leader_id, m 883 | ); 884 | } 885 | 886 | if !snapshot_received && msg_type == MessageType::MsgHeartbeat { 887 | debug!( 888 | "raft message, snapshot_received: {}, has_leader: {}, {:?}", 889 | snapshot_received, 890 | self.has_leader(), 891 | m 892 | ); 893 | } else { 894 | if let Err(e) = self.step(*m) { 895 | warn!( 896 | "step error, {:?}, msg_type: {:?}, snapshot_received: {}", 897 | e, msg_type, snapshot_received 898 | ); 899 | } 900 | if msg_type == MessageType::MsgSnapshot { 901 | snapshot_received = true; 902 | } 903 | } 904 | } 905 | Ok(Some(Message::Propose { proposal, chan })) => { 906 | self.propose_counter.inc(); 907 | let now = Instant::now(); 908 | if !self.is_leader() { 909 | debug!("Message::Propose, send_wrong_leader {:?}", proposal); 910 | self.send_wrong_leader("Propose", chan); 911 | } else if self.is_busy() { 912 | self.send_is_busy(chan); 913 | } else { 914 | merger.add(proposal, chan); 915 | self.take_and_propose(&mut merger); 916 | } 917 | if now.elapsed() > self.cfg.heartbeat { 918 | info!("Message::Propose elapsed: {:?}", now.elapsed()); 919 | } 920 | } 921 | 922 | Ok(Some(Message::Query { query, chan })) => { 923 | let now = Instant::now(); 924 | if !self.is_leader() { 925 | debug!("[forward_query] query.len: {:?}", query.len()); 926 | self.forward_query(query, chan).await; 927 | } else { 928 | debug!("Message::Query, {:?}", query); 929 | self.send_query(&query, chan).await; 930 | } 931 | if now.elapsed() > self.cfg.heartbeat { 932 | info!("Message::Query elapsed: {:?}", now.elapsed()); 933 | } 934 | } 935 | 936 | Ok(Some(Message::RequestId { chan })) => { 937 | debug!("requested Id, is_leader: {}", self.is_leader()); 938 | if !self.is_leader() { 939 | self.send_wrong_leader("RequestId", chan); 940 | } else { 941 | self.send_leader_id(chan); 942 | } 943 | } 944 | Ok(Some(Message::Status { chan })) => { 945 | self.send_status(&merger, chan).await; 946 | } 947 | Ok(Some(Message::Snapshot { snapshot })) => { 948 | self.set_snapshot(snapshot); 949 | } 950 | Ok(Some(Message::ReportUnreachable { node_id })) => { 951 | debug!( 952 | "Message::ReportUnreachable, node_id: {}, sending_raft_messages: {}", 953 | node_id, 954 | self.sending_raft_messages.load(Ordering::SeqCst) 955 | ); 956 | self.report_unreachable(node_id); 957 | } 958 | Ok(None) => { 959 | error!("Recv None"); 960 | return Err(Error::RecvError("Recv None".into())); 961 | } 962 | Err(_) => { 963 | self.take_and_propose(&mut merger); 964 | } 965 | } 966 | 967 | let elapsed = now.elapsed(); 968 | now = Instant::now(); 969 | 970 | if elapsed >= heartbeat { 971 | if elapsed > Duration::from_millis(500) { 972 | warn!( 973 | "[run] raft tick elapsed: {:?}, heartbeat: {:?}, uncommitteds: {}, sending_raft_messages: {}", 974 | elapsed, 975 | heartbeat, 976 | self.uncommitteds.len(), 977 | self.sending_raft_messages.load(Ordering::SeqCst), 978 | ); 979 | } 980 | heartbeat = self.cfg.heartbeat; 981 | self.tick(); 982 | } else { 983 | heartbeat -= elapsed; 984 | } 985 | 986 | let on_ready_now = Instant::now(); 987 | if let Err(e) = self.on_ready().await { 988 | error!( 989 | "raft on_ready(..) error: {:?}, elapsed: {:?}", 990 | e, 991 | on_ready_now.elapsed() 992 | ); 993 | return Err(e); 994 | } 995 | if on_ready_now.elapsed() > Duration::from_millis(500) { 996 | warn!( 997 | "[run] raft on_ready(..) uncommitteds: {}, sending_raft_messages: {}, elapsed: {:?}", 998 | self.uncommitteds.len(), 999 | self.sending_raft_messages.load(Ordering::SeqCst), 1000 | on_ready_now.elapsed() 1001 | ); 1002 | } 1003 | } 1004 | } 1005 | 1006 | async fn on_ready(&mut self) -> Result<()> { 1007 | if !self.has_ready() { 1008 | return Ok(()); 1009 | } 1010 | 1011 | let mut ready = self.ready(); 1012 | 1013 | if !ready.messages().is_empty() { 1014 | // Send out the messages. 1015 | self.send_messages(ready.take_messages()); 1016 | } 1017 | 1018 | if *ready.snapshot() != Snapshot::default() { 1019 | let snapshot = ready.snapshot(); 1020 | log::info!( 1021 | "snapshot metadata: {:?}, data len: {}", 1022 | snapshot.get_metadata(), 1023 | snapshot.get_data().len() 1024 | ); 1025 | self.store.restore(snapshot.get_data()).await?; 1026 | let store = self.mut_store(); 1027 | store.apply_snapshot(snapshot.clone())?; 1028 | } 1029 | 1030 | self.handle_committed_entries(ready.take_committed_entries()) 1031 | .await?; 1032 | 1033 | if !ready.entries().is_empty() { 1034 | let entries = ready.entries(); 1035 | let store = self.mut_store(); 1036 | store.append(entries)?; 1037 | } 1038 | 1039 | if let Some(hs) = ready.hs() { 1040 | // Raft HardState changed, and we need to persist it. 1041 | let store = self.mut_store(); 1042 | store.set_hard_state(hs)?; 1043 | } 1044 | 1045 | if !ready.persisted_messages().is_empty() { 1046 | // Send out the persisted messages come from the node. 1047 | self.send_messages(ready.take_persisted_messages()); 1048 | } 1049 | let mut light_rd = self.advance(ready); 1050 | 1051 | if let Some(commit) = light_rd.commit_index() { 1052 | let store = self.mut_store(); 1053 | store.set_hard_state_comit(commit)?; 1054 | } 1055 | // Send out the messages. 1056 | self.send_messages(light_rd.take_messages()); 1057 | // Apply all committed entries. 1058 | self.handle_committed_entries(light_rd.take_committed_entries()) 1059 | .await?; 1060 | self.advance_apply(); 1061 | 1062 | Ok(()) 1063 | } 1064 | 1065 | fn send_messages(&mut self, msgs: Vec) { 1066 | for message in msgs { 1067 | // for message in ready.messages.drain(..) { 1068 | let client_id = message.get_to(); 1069 | let client = match self.peer(client_id) { 1070 | Some(peer) => peer, 1071 | None => continue, 1072 | }; 1073 | 1074 | let message_sender = MessageSender { 1075 | message, 1076 | client, 1077 | client_id, 1078 | chan: self.snd.clone(), 1079 | max_retries: 0, 1080 | timeout: Duration::from_millis(500), 1081 | sending_raft_messages: self.sending_raft_messages.clone(), 1082 | }; 1083 | tokio::spawn(message_sender.send()); 1084 | } 1085 | } 1086 | 1087 | async fn handle_committed_entries(&mut self, committed_entries: Vec) -> Result<()> { 1088 | // Fitler out empty entries produced by new elected leaders. 1089 | let committed_entries_count = committed_entries.len(); 1090 | let now = std::time::Instant::now(); 1091 | for entry in committed_entries { 1092 | if entry.data.is_empty() { 1093 | // From new elected leaders. 1094 | continue; 1095 | } 1096 | if let EntryType::EntryConfChange = entry.get_entry_type() { 1097 | self.handle_config_change(&entry).await?; 1098 | } else { 1099 | self.handle_normal(&entry).await?; 1100 | } 1101 | } 1102 | 1103 | if now.elapsed().as_millis() > 500 { 1104 | log::info!( 1105 | "[handle_committed_entries] uncommitteds.len(): {}, sending_raft_messages: {}, \ 1106 | committed_entries_count: {}, raft.inflight_buffers_size: {}, \ 1107 | raft.msgs: {}, raft.group_commit: {}, raft.pending_read_count: {}, raft.ready_read_count: {}, \ 1108 | raft.soft_state: {:?}, raft.hard_state: {:?}, raft.state: {:?}, raft.heartbeat_elapsed: {}, \ 1109 | self.raft.read_states: {}, raft.heartbeat_timeout: {}, raft.heartbeat_elapsed: {}, \ 1110 | cost time: {:?}", 1111 | self.uncommitteds.len(), self.sending_raft_messages.load(Ordering::SeqCst), 1112 | committed_entries_count, self.raft.inflight_buffers_size(), self.raft.msgs.len(), 1113 | self.raft.group_commit(), self.raft.pending_read_count(), self.raft.ready_read_count(), 1114 | self.raft.soft_state(), self.raft.hard_state(), self.raft.state, self.raft.heartbeat_elapsed(), 1115 | self.raft.read_states.len(), self.raft.heartbeat_timeout(), self.raft.heartbeat_elapsed(), 1116 | now.elapsed() 1117 | ); 1118 | } 1119 | Ok(()) 1120 | } 1121 | 1122 | #[inline] 1123 | async fn handle_config_change(&mut self, entry: &Entry) -> Result<()> { 1124 | info!("handle_config_change, entry: {:?}", entry); 1125 | let seq: u64 = deserialize(entry.get_context())?; 1126 | info!("handle_config_change, seq: {:?}", seq); 1127 | let change = ConfChange::decode(entry.get_data()) 1128 | .map_err(|e| tonic::Status::invalid_argument(e.to_string()))?; 1129 | info!("handle_config_change, change: {:?}", change); 1130 | let id = change.get_node_id(); 1131 | 1132 | let change_type = change.get_change_type(); 1133 | 1134 | match change_type { 1135 | ConfChangeType::AddNode => { 1136 | let addr: String = deserialize(change.get_context())?; 1137 | info!("adding {} ({}) to peers", addr, id); 1138 | self.add_peer(&addr, id); 1139 | } 1140 | ConfChangeType::RemoveNode => { 1141 | let ctx = change.get_context(); 1142 | let typ = if !ctx.is_empty() { 1143 | deserialize(change.get_context())? 1144 | } else { 1145 | RemoveNodeType::Normal 1146 | }; 1147 | info!("removing ({}) to peers, RemoveNodeType: {:?}", id, typ); 1148 | if id == self.id() { 1149 | if !matches!(typ, RemoveNodeType::Stale) { 1150 | self.should_quit = true; 1151 | warn!("quiting the cluster"); 1152 | } 1153 | } else { 1154 | self.peers.remove(&id); 1155 | } 1156 | } 1157 | _ => { 1158 | warn!("unimplemented! change_type: {:?}", change_type); 1159 | } 1160 | } 1161 | 1162 | if let Ok(cs) = self.apply_conf_change(&change) { 1163 | info!("conf state: {cs:?}, id: {id}, this id: {}", self.id()); 1164 | if matches!(change_type, ConfChangeType::AddNode) { 1165 | let store = self.mut_store(); 1166 | store.set_conf_state(&cs)?; 1167 | if id != self.id() { 1168 | let snap = self.generate_snapshot_sync().await?; 1169 | self.set_snapshot(snap); 1170 | } 1171 | } else { 1172 | let store = self.mut_store(); 1173 | store.set_conf_state(&cs)?; 1174 | } 1175 | } 1176 | 1177 | if let Some(sender) = self.uncommitteds.remove(&seq) { 1178 | let response = match change_type { 1179 | ConfChangeType::AddNode => RaftResponse::JoinSuccess { 1180 | assigned_id: id, 1181 | peer_addrs: self.peer_addrs(), 1182 | }, 1183 | ConfChangeType::RemoveNode => RaftResponse::Ok, 1184 | _ => { 1185 | warn!("unimplemented! change_type: {:?}", change_type); 1186 | RaftResponse::Error("unimplemented".into()) 1187 | } 1188 | }; 1189 | if let ReplyChan::One((sender, _)) = sender { 1190 | if sender.send(response).is_err() { 1191 | warn!("error sending response") 1192 | } 1193 | } 1194 | } 1195 | Ok(()) 1196 | } 1197 | 1198 | #[inline] 1199 | async fn handle_normal(&mut self, entry: &Entry) -> Result<()> { 1200 | let seq: u64 = deserialize(entry.get_context())?; 1201 | debug!( 1202 | "[handle_normal] seq:{}, senders.len(): {}", 1203 | seq, 1204 | self.uncommitteds.len() 1205 | ); 1206 | 1207 | match ( 1208 | deserialize::(entry.get_data())?, 1209 | self.uncommitteds.remove(&seq), 1210 | ) { 1211 | (Proposals::One(data), chan) => { 1212 | let apply_start = std::time::Instant::now(); 1213 | let reply = 1214 | tokio::time::timeout(Duration::from_secs(5), self.store.apply(&data)).await; 1215 | if apply_start.elapsed().as_secs() > 2 { 1216 | self.timeout_recorder.incr(); 1217 | log::warn!("apply, cost time: {:?}", apply_start.elapsed()); 1218 | } 1219 | if let Some(ReplyChan::One((chan, inst))) = chan { 1220 | if inst.elapsed().as_secs() > 2 { 1221 | self.timeout_recorder.incr(); 1222 | debug!( 1223 | "[handle_normal] cost time, {:?}, chan is canceled: {}, uncommitteds: {}, sending_raft_messages: {}", 1224 | inst.elapsed(), 1225 | chan.is_canceled(), 1226 | self.uncommitteds.len(), 1227 | self.sending_raft_messages.load(Ordering::SeqCst) 1228 | ); 1229 | } 1230 | if !chan.is_canceled() { 1231 | let reply = reply.unwrap_or_else(|e| Err(Error::from(e))); 1232 | let res = match reply { 1233 | Ok(data) => RaftResponse::Response { data }, 1234 | Err(e) => RaftResponse::Error(e.to_string()), 1235 | }; 1236 | if let Err(_resp) = chan.send(res) { 1237 | warn!( 1238 | "[handle_normal] send RaftResponse error, seq:{}, uncommitteds: {}, sending_raft_messages: {}", 1239 | seq, 1240 | self.uncommitteds.len(), 1241 | self.sending_raft_messages.load(Ordering::SeqCst) 1242 | ); 1243 | } 1244 | } 1245 | } 1246 | } 1247 | (Proposals::More(mut datas), chans) => { 1248 | let mut chans = if let Some(ReplyChan::More(chans)) = chans { 1249 | Some(chans) 1250 | } else { 1251 | None 1252 | }; 1253 | while let Some(data) = datas.pop() { 1254 | let apply_start = std::time::Instant::now(); 1255 | let reply = 1256 | tokio::time::timeout(Duration::from_secs(5), self.store.apply(&data)).await; 1257 | if apply_start.elapsed().as_secs() > 2 { 1258 | self.timeout_recorder.incr(); 1259 | log::warn!("apply, cost time: {:?}", apply_start.elapsed()); 1260 | } 1261 | if let Some((chan, inst)) = chans.as_mut().and_then(|cs| cs.pop()) { 1262 | if inst.elapsed().as_secs() > 2 { 1263 | self.timeout_recorder.incr(); 1264 | debug!( 1265 | "[handle_normal] cost time, {:?}, chan is canceled: {}, uncommitteds: {}, sending_raft_messages: {}", 1266 | inst.elapsed(), 1267 | chan.is_canceled(), 1268 | self.uncommitteds.len(), 1269 | self.sending_raft_messages.load(Ordering::SeqCst) 1270 | ); 1271 | } 1272 | if !chan.is_canceled() { 1273 | let reply = reply.unwrap_or_else(|e| Err(Error::from(e))); 1274 | let res = match reply { 1275 | Ok(data) => RaftResponse::Response { data }, 1276 | Err(e) => RaftResponse::Error(e.to_string()), 1277 | }; 1278 | if let Err(_resp) = chan.send(res) { 1279 | warn!("[handle_normal] send RaftResponse error, seq:{}", seq,); 1280 | } 1281 | } 1282 | } 1283 | } 1284 | } 1285 | } 1286 | 1287 | if Instant::now() > self.last_snap_time + self.cfg.snapshot_interval { 1288 | self.last_snap_time = Instant::now(); 1289 | info!("gen snapshot start"); 1290 | self.generate_snapshot_async()?; 1291 | } 1292 | Ok(()) 1293 | } 1294 | 1295 | async fn generate_snapshot_sync(&mut self) -> Result { 1296 | let last_applied = self.raft.raft_log.applied; 1297 | let last_term = self.raft.raft_log.term(last_applied).unwrap_or(0); 1298 | let mut snapshot = self.mut_store().create_snapshot(last_applied, last_term)?; 1299 | 1300 | let now = Instant::now(); 1301 | let snap = match self.store.snapshot().await { 1302 | Err(e) => { 1303 | log::error!("gen snapshot error, {e:?}"); 1304 | return Err(e); 1305 | } 1306 | Ok(snap) => snap, 1307 | }; 1308 | info!( 1309 | "gen snapshot cost time: {:?}, snapshot len: {}, last_applied: {last_applied}", 1310 | now.elapsed(), 1311 | snap.len() 1312 | ); 1313 | 1314 | snapshot.set_data(snap); 1315 | 1316 | Ok(snapshot) 1317 | } 1318 | 1319 | fn generate_snapshot_async(&mut self) -> Result<()> { 1320 | let store = self.store.clone(); 1321 | let mut tx = self.snd.clone(); 1322 | let last_applied = self.raft.raft_log.applied; 1323 | let last_term = self.raft.raft_log.term(last_applied).unwrap_or(0); 1324 | let mut snapshot = self.mut_store().create_snapshot(last_applied, last_term)?; 1325 | 1326 | tokio::spawn(async move { 1327 | let now = Instant::now(); 1328 | let snap = match store.snapshot().await { 1329 | Err(e) => { 1330 | log::error!("gen snapshot error, {e:?}"); 1331 | return; 1332 | } 1333 | Ok(snap) => snap, 1334 | }; 1335 | info!( 1336 | "gen snapshot cost time: {:?}, snapshot len: {}, last_applied: {last_applied}", 1337 | now.elapsed(), 1338 | snap.len() 1339 | ); 1340 | 1341 | snapshot.set_data(snap); 1342 | 1343 | if let Err(e) = tx.send(Message::Snapshot { snapshot }).await { 1344 | log::error!("send snapshot error, {e:?}"); 1345 | } 1346 | }); 1347 | 1348 | Ok(()) 1349 | } 1350 | 1351 | fn set_snapshot(&mut self, snap: Snapshot) { 1352 | let store = self.mut_store(); 1353 | let last_applied = snap.get_metadata().index; 1354 | store.set_snapshot(snap); 1355 | if let Err(e) = store.compact(last_applied) { 1356 | error!("compact error, {e}"); 1357 | } 1358 | info!("set snapshot,last_applied: {last_applied}"); 1359 | } 1360 | } 1361 | 1362 | impl Deref for RaftNode { 1363 | type Target = RawNode; 1364 | 1365 | fn deref(&self) -> &Self::Target { 1366 | &self.inner 1367 | } 1368 | } 1369 | 1370 | impl DerefMut for RaftNode { 1371 | fn deref_mut(&mut self) -> &mut Self::Target { 1372 | &mut self.inner 1373 | } 1374 | } 1375 | --------------------------------------------------------------------------------