Doc & guide updates.

Added the `client_read` method on the Raft type for guarding against
stale reads.
This commit is contained in:
Anthony Dodd 2020-08-13 21:41:08 -05:00
parent 8a9108cac3
commit 9ac6dcb889
No known key found for this signature in database
GPG Key ID: 6E0613E0F653DBC0
37 changed files with 679 additions and 587 deletions

View File

@ -25,6 +25,10 @@ jobs:
with:
command: test
args: -p async-raft --release --test initialization
- uses: actions-rs/cargo@v1
with:
command: test
args: -p async-raft --release --test client_reads
- uses: actions-rs/cargo@v1
with:
command: test

View File

@ -4,8 +4,8 @@ This is a Rust project, so [rustup](https://rustup.rs/) is the best place to sta
Check out the `.travis.yml` file to get an idea on how to run tests and the like.
### rustfmt & clippy
Haven't added these yet, but I am definitely planning on doing so.
### clippy
Haven't added clippy integration yet, but I am definitely planning on doing so. Don't run rustfmt ...
### the guide
The guide for this project is built using [mdBook](https://rust-lang-nursery.github.io/mdBook/index.html). Review their guide for more details on how to work with mdBook. Here are a few of the pertinents:
@ -20,3 +20,6 @@ mdbook build
# Watch the FS for changes & rebuild.
mdbook watch
```
### release checklist
- Any documentation updates should also be reflected in the guide.

View File

@ -4,10 +4,10 @@
An implementation of the <a href="https://raft.github.io/">Raft distributed consensus protocol</a> using <a href="https://tokio.rs/">the Tokio framework</a>. Please ⭐ on <a href="https://github.com/railgun-rs/actix-raft">github</a>!
</strong>
</div>
<br />
<br/>
<div align="center">
[![Build Status](https://travis-ci.com/railgun-rs/actix-raft.svg?branch=master)](https://travis-ci.com/railgun-rs/actix-raft)
[![Build Status](https://github.com/railgun-rs/actix-raft/workflows/ci/badge.svg?branch=async-raft)](https://travis-ci.com/railgun-rs/actix-raft)
[![Crates.io](https://img.shields.io/crates/v/actix-raft.svg)](https://crates.io/crates/actix-raft)
[![docs.rs](https://docs.rs/actix-raft/badge.svg)](https://docs.rs/actix-raft)
[![License](https://img.shields.io/badge/license-MIT%2FApache--2.0-blue)](LICENSE)
@ -17,9 +17,9 @@
[![GitHub issues closed](https://img.shields.io/github/issues-closed-raw/railgun-rs/actix-raft.svg)]()
</div>
</br>
<br/>
Blazing fast Rust, a modern consensus protocol, and a reliable async runtime — this project intends to provide a consensus backbone for the next generation of distributed data storage systems (SQL, NoSQL, KV, Streaming &c).
Blazing fast Rust, a modern consensus protocol, and a reliable async runtime — this project intends to provide a consensus backbone for the next generation of distributed data storage systems (SQL, NoSQL, KV, Streaming, Graph ... or maybe something more exotic).
[The guide](https://railgun-rs.github.io/actix-raft) is the best place to get started, followed by [the docs](https://docs.rs/actix-raft/latest/actix_raft/) for more in-depth details.
@ -27,8 +27,8 @@ This crate differs from other Raft implementations in that:
- It is fully reactive and embraces the async ecosystem. It is driven by actual Raft related events taking place in the system as opposed to being driven by a `tick` operation. Batching of messages during replication is still used whenever possible for maximum throughput.
- Storage and network integration is well defined via two traits `RaftStorage` & `RaftNetwork`. This provides applications maximum flexibility in being able to choose their storage and networking mediums. See the [storage](https://railgun-rs.github.io/actix-raft/storage.html) & [network](https://railgun-rs.github.io/actix-raft/network.html) chapters of the guide for more details.
- All interaction with the Raft node is well defined via a single public `Raft` type, which is used to spawn the Raft async task, and to interact with that task. The API for this system is clear and concise. See the [raft](https://railgun-rs.github.io/actix-raft/raft.html) chapter in the guide.
- It fully supports dynamic cluster membership changes according to the Raft spec. See the [`dynamic membership`](https://railgun-rs.github.io/actix-raft/dynamic-membership.html) chapter in the guide.
- Details on initial cluster formation, and how to effectively do so from an application level perspective, are discussed in the [cluster formation](https://railgun-rs.github.io/actix-raft/cluster-formation.html) chapter in the guide.
- It fully supports dynamic cluster membership changes according to the Raft spec. See the [`dynamic membership`](https://railgun-rs.github.io/actix-raft/dynamic-membership.html) chapter in the guide. With full support for leader stepdown, and non-voter syncing.
- Details on initial cluster formation, and how to effectively do so from an application's perspective, are discussed in the [cluster formation](https://railgun-rs.github.io/actix-raft/cluster-formation.html) chapter in the guide.
- Automatic log compaction with snapshots, as well as snapshot streaming from the leader node to follower nodes is fully supported and configurable.
- The entire code base is [instrumented with tracing](https://docs.rs/tracing/). This can be used for [standard logging](https://docs.rs/tracing/latest/tracing/index.html#log-compatibility), or for [distributed tracing](https://docs.rs/tracing/latest/tracing/index.html#related-crates), and the verbosity can be [statically configured at compile time](https://docs.rs/tracing/latest/tracing/level_filters/index.html) to completely remove all instrumentation below the configured level.

View File

@ -1,6 +1,4 @@
//! The Raft configuration module.
use std::time::Duration;
//! Raft runtime configuration.
use rand::{thread_rng, Rng};
@ -18,12 +16,10 @@ pub const DEFAULT_LOGS_SINCE_LAST: u64 = 5000;
pub const DEFAULT_MAX_PAYLOAD_ENTRIES: u64 = 300;
/// Default replication lag threshold.
pub const DEFAULT_REPLICATION_LAG_THRESHOLD: u64 = 1000;
/// Default metrics rate.
pub const DEFAULT_METRICS_RATE: Duration = Duration::from_millis(5000);
/// Default snapshot chunksize.
pub const DEFAULT_SNAPSHOT_CHUNKSIZE: u64 = 1024 * 1024 * 3;
/// Raft log snapshot policy.
/// Log compaction and snapshot policy.
///
/// This governs when periodic snapshots will be taken, and also governs the conditions which
/// would cause a leader to send an `InstallSnapshot` RPC to a follower based on replication lag.
@ -44,6 +40,11 @@ impl Default for SnapshotPolicy {
/// The runtime configuration for a Raft node.
///
/// The default values used by this type should generally work well for Raft clusters which will
/// be running with nodes in multiple datacenter availability zones with low latency between
/// zones. These values should typically be made configurable from the perspective of the
/// application which is being built on top of Raft.
///
/// When building the Raft configuration for your application, remember this inequality from the
/// Raft spec: `broadcastTime ≪ electionTimeout ≪ MTBF`.
///
@ -71,7 +72,7 @@ pub struct Config {
pub election_timeout_min: u64,
/// The maximum election timeout in milliseconds.
pub election_timeout_max: u64,
/// The heartbeat interval at which leaders will send heartbeats to followers.
/// The heartbeat interval in milliseconds at which leaders will send heartbeats to followers.
///
/// Defaults to 50 milliseconds.
///
@ -127,7 +128,7 @@ impl Config {
}
}
/// A configuration builder to ensure that the Raft's runtime config is valid.
/// A configuration builder to ensure that runtime config is valid.
///
/// For election timeout config & heartbeat interval configuration, it is recommended that §5.6 of
/// the Raft spec is considered in order to set the appropriate values.

View File

@ -5,7 +5,7 @@ use tokio::sync::oneshot;
use crate::{AppData, AppDataResponse, NodeId, RaftNetwork, RaftStorage};
use crate::error::{InitializeError, ChangeConfigError, RaftError};
use crate::raft::{ChangeMembershipTx, ClientRequest, MembershipConfig};
use crate::raft::{ChangeMembershipTx, ClientWriteRequest, MembershipConfig};
use crate::core::{ConsensusState, LeaderState, NonVoterReplicationState, NonVoterState, State, UpdateCurrentLeader};
use crate::core::client::ClientRequestEntry;
use crate::replication::RaftEvent;
@ -117,7 +117,7 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
self.core.membership.members_after_consensus = Some(members);
// Propagate the command as any other client request.
let payload = ClientRequest::<D>::new_config(self.core.membership.clone());
let payload = ClientWriteRequest::<D>::new_config(self.core.membership.clone());
let (tx_joint, rx_join) = oneshot::channel();
let entry = match self.append_payload_to_log(payload.entry).await {
Ok(entry) => entry,
@ -192,7 +192,7 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
// described at the very end of §6. This measure is already implemented and in place.
// Propagate the next command as any other client request.
let payload = ClientRequest::<D>::new_config(self.core.membership.clone());
let payload = ClientWriteRequest::<D>::new_config(self.core.membership.clone());
let (tx_uniform, rx_uniform) = oneshot::channel();
let entry = self.append_payload_to_log(payload.entry).await?;
let cr_entry = ClientRequestEntry::from_entry(entry, tx_uniform);

View File

@ -1,11 +1,17 @@
use std::sync::Arc;
use anyhow::anyhow;
use futures::stream::FuturesUnordered;
use futures::future::TryFutureExt;
use tokio::stream::StreamExt;
use tokio::sync::oneshot;
use tokio::time::{Duration, timeout};
use crate::{AppData, AppDataResponse, RaftNetwork, RaftStorage};
use crate::core::{LeaderState, State};
use crate::error::{ClientError, RaftError, RaftResult};
use crate::raft::{ClientRequest, ClientResponse, ClientResponseTx, Entry, EntryPayload};
use crate::error::{ClientReadError, ClientWriteError, RaftError, RaftResult};
use crate::raft::{ClientWriteRequest, ClientWriteResponse, ClientReadResponseTx, ClientWriteResponseTx, Entry, EntryPayload};
use crate::raft::{AppendEntriesRequest};
use crate::replication::RaftEvent;
/// A wrapper around a ClientRequest which has been transformed into an Entry, along with its response channel.
@ -29,7 +35,7 @@ impl<D: AppData, R: AppDataResponse> ClientRequestEntry<D, R> {
/// An enum type wrapping either a client response channel or an internal Raft response channel.
#[derive(derive_more::From)]
pub enum ClientOrInternalResponseTx<D: AppData, R: AppDataResponse> {
Client(ClientResponseTx<D, R>),
Client(ClientWriteResponseTx<D, R>),
Internal(oneshot::Sender<Result<u64, RaftError>>),
}
@ -39,10 +45,10 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
pub(super) async fn commit_initial_leader_entry(&mut self) -> RaftResult<()> {
// If the cluster has just formed, and the current index is 0, then commit the current
// config, else a blank payload.
let req: ClientRequest<D> = if self.core.last_log_index == 0 {
ClientRequest::new_config(self.core.membership.clone())
let req: ClientWriteRequest<D> = if self.core.last_log_index == 0 {
ClientWriteRequest::new_config(self.core.membership.clone())
} else {
ClientRequest::new_blank_payload()
ClientWriteRequest::new_blank_payload()
};
// Check to see if we have any config change logs newer than our commit index. If so, then
@ -81,13 +87,106 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
Ok(())
}
/// Handle client requests.
/// Handle client read requests.
///
/// Spawn requests to all members of the cluster, include members being added in joint
/// consensus. Each request will have a timeout, and we respond once we have a majority
/// agreement from each config group. Most of the time, we will have a single uniform
/// config group.
#[tracing::instrument(level="trace", skip(self, tx))]
pub(super) async fn handle_client_read_request(&mut self, tx: ClientReadResponseTx) {
// Setup sentinel values to track when we've received majority confirmation of leadership.
let len_members = self.core.membership.members.len();
let mut c0_confirmed = 0usize;
let c0_needed: usize = if (len_members % 2) == 0 { (len_members/2)-1 } else { len_members/2 };
let mut c1_confirmed = 0usize;
let mut c1_needed = 0usize;
if let Some(joint_members) = &self.core.membership.members_after_consensus {
let len = joint_members.len(); // Will never be zero, as we don't allow it when proposing config changes.
c1_needed = if (len % 2) == 0 { (len/2)-1 } else { len/2 };
}
// As long as we are not about to step down, then increment for our vote.
if !self.is_stepping_down {
if self.core.membership.members.contains(&self.core.id) {
c0_confirmed += 1;
}
if self.core.membership.members_after_consensus.as_ref().map(|members| members.contains(&self.core.id)).unwrap_or(false) {
c1_confirmed += 1;
}
}
// Spawn parallel requests, all with the standard timeout for heartbeats.
let mut pending = FuturesUnordered::new();
for (id, node) in self.nodes.iter() {
let rpc = AppendEntriesRequest{
term: self.core.current_term,
leader_id: self.core.id,
prev_log_index: node.match_index,
prev_log_term: node.match_term,
entries: vec![],
leader_commit: self.core.commit_index,
};
let target = id.clone();
let network = self.core.network.clone();
let ttl = Duration::from_millis(self.core.config.heartbeat_interval);
let task = tokio::spawn(async move {
match timeout(ttl, network.append_entries(target, rpc)).await {
Ok(Ok(data)) => Ok((target, data)),
Ok(Err(err)) => Err((target, err)),
Err(_timeout) => Err((target, anyhow!("timeout waiting for leadership confirmation"))),
}
}).map_err(move |err| (*id, err));
pending.push(task);
}
// Handle responses as they return.
while let Some(res) = pending.next().await {
let (target, data) = match res {
Ok(Ok(res)) => res,
Ok(Err((target, err))) => {
tracing::error!({target, error=%err}, "timeout while confirming leadership for read request");
continue;
}
Err((target, err)) => {
tracing::error!({target}, "{}", err);
continue;
}
};
// If we receive a response with a greater term, then revert to follower and abort this request.
if &data.term != &self.core.current_term {
self.core.update_current_term(data.term, None);
self.core.set_target_state(State::Follower);
}
// If the term is the same, then it means we are still the leader.
if self.core.membership.members.contains(&target) {
c0_confirmed += 1;
}
if self.core.membership.members_after_consensus.as_ref().map(|members| members.contains(&target)).unwrap_or(false) {
c1_confirmed += 1;
}
if &c0_confirmed >= &c0_needed && &c1_confirmed >= &c1_needed {
let _ = tx.send(Ok(()));
return;
}
}
// If we've hit this location, then we've failed to gather needed confirmations due to
// request failures.
let _ = tx.send(Err(ClientReadError::RaftError(
RaftError::RaftNetwork(anyhow!("too many requests failed, could not confirm leadership"))
)));
}
/// Handle client write requests.
#[tracing::instrument(level="trace", skip(self, rpc, tx))]
pub(super) async fn handle_client_request(&mut self, rpc: ClientRequest<D>, tx: ClientResponseTx<D, R>) {
pub(super) async fn handle_client_write_request(&mut self, rpc: ClientWriteRequest<D>, tx: ClientWriteResponseTx<D, R>) {
let entry = match self.append_payload_to_log(rpc.entry).await {
Ok(entry) => ClientRequestEntry::from_entry(entry, tx),
Err(err) => {
let _ = tx.send(Err(ClientError::RaftError(err)));
let _ = tx.send(Err(ClientWriteError::RaftError(err)));
return;
}
};
@ -148,10 +247,10 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
EntryPayload::Normal(inner) => {
match self.apply_entry_to_state_machine(&req.entry.index, &inner.data).await {
Ok(data) => {
let _ = tx.send(Ok(ClientResponse{index: req.entry.index, data}));
let _ = tx.send(Ok(ClientWriteResponse{index: req.entry.index, data}));
}
Err(err) => {
let _ = tx.send(Err(ClientError::RaftError(RaftError::from(err))));
let _ = tx.send(Err(ClientWriteError::RaftError(RaftError::from(err))));
}
}
}

View File

@ -22,9 +22,9 @@ use tracing_futures::Instrument;
use crate::{AppData, AppDataResponse, RaftNetwork, RaftStorage, NodeId};
use crate::config::{Config, SnapshotPolicy};
use crate::core::client::ClientRequestEntry;
use crate::error::{ClientError, ChangeConfigError, InitializeError, RaftError, RaftResult};
use crate::error::{ClientReadError, ClientWriteError, ChangeConfigError, InitializeError, RaftError, RaftResult};
use crate::metrics::RaftMetrics;
use crate::raft::{ChangeMembershipTx, ClientRequest, ClientResponseTx, RaftMsg, MembershipConfig};
use crate::raft::{ChangeMembershipTx, ClientWriteRequest, ClientReadResponseTx, ClientWriteResponseTx, RaftMsg, MembershipConfig};
use crate::replication::{RaftEvent, ReplicationStream, ReplicaEvent};
use crate::storage::HardState;
@ -125,7 +125,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
}
/// The main loop of the Raft protocol.
#[tracing::instrument(level="trace", skip(self), fields(id=self.id))]
#[tracing::instrument(level="trace", skip(self), fields(id=self.id, cluster=%self.config.cluster_name))]
async fn main(mut self) -> RaftResult<()> {
tracing::trace!("raft node is initializing");
let state = self.storage.get_initial_state().await.map_err(|err| self.map_fatal_storage_error(err))?;
@ -146,7 +146,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
}
// Set initial state based on state recovered from disk.
let is_only_configured_member = self.membership.len() == 1 && self.membership.contains(&self.id);
let is_only_configured_member = self.membership.members.len() == 1 && self.membership.contains(&self.id);
// If this is the only configured member and there is live state, then this is
// a single-node cluster. Become leader.
if is_only_configured_member && &self.last_log_index != &u64::min_value() {
@ -356,10 +356,16 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
let _ = tx.send(Err(ChangeConfigError::NodeNotLeader));
}
/// Forward the given client request to the leader.
/// Forward the given client write request to the leader.
#[tracing::instrument(level="trace", skip(self, req, tx))]
fn forward_client_request(&self, req: ClientRequest<D>, tx: ClientResponseTx<D, R>) {
let _ = tx.send(Err(ClientError::ForwardToLeader(req, self.voted_for.clone())));
fn forward_client_write_request(&self, req: ClientWriteRequest<D>, tx: ClientWriteResponseTx<D, R>) {
let _ = tx.send(Err(ClientWriteError::ForwardToLeader(req, self.current_leader.clone())));
}
/// Forward the given client read request to the leader.
#[tracing::instrument(level="trace", skip(self, tx))]
fn forward_client_read_request(&self, tx: ClientReadResponseTx) {
let _ = tx.send(Err(ClientReadError::ForwardToLeader(self.current_leader.clone())));
}
}
@ -531,8 +537,11 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
RaftMsg::InstallSnapshot{rpc, tx} => {
let _ = tx.send(self.core.handle_install_snapshot_request(rpc).await);
}
RaftMsg::ClientRequest{rpc, tx} => {
self.handle_client_request(rpc, tx).await;
RaftMsg::ClientReadRequest{tx} => {
self.handle_client_read_request(tx).await;
}
RaftMsg::ClientWriteRequest{rpc, tx} => {
self.handle_client_write_request(rpc, tx).await;
}
RaftMsg::Initialize{tx, ..} => {
self.core.reject_init_with_config(tx);
@ -575,6 +584,7 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
/// A struct tracking the state of a replication stream from the perspective of the Raft actor.
struct ReplicationState<D: AppData> {
pub match_index: u64,
pub match_term: u64,
pub is_at_line_rate: bool,
pub remove_after_commit: Option<u64>,
pub replstream: ReplicationStream<D>,
@ -693,8 +703,11 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
RaftMsg::InstallSnapshot{rpc, tx} => {
let _ = tx.send(self.core.handle_install_snapshot_request(rpc).await);
}
RaftMsg::ClientRequest{rpc, tx} => {
self.core.forward_client_request(rpc, tx);
RaftMsg::ClientReadRequest{tx} => {
self.core.forward_client_read_request(tx);
}
RaftMsg::ClientWriteRequest{rpc, tx} => {
self.core.forward_client_write_request(rpc, tx);
}
RaftMsg::Initialize{tx, ..} => {
self.core.reject_init_with_config(tx);
@ -749,8 +762,11 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
RaftMsg::InstallSnapshot{rpc, tx} => {
let _ = tx.send(self.core.handle_install_snapshot_request(rpc).await);
}
RaftMsg::ClientRequest{rpc, tx} => {
self.core.forward_client_request(rpc, tx);
RaftMsg::ClientReadRequest{tx} => {
self.core.forward_client_read_request(tx);
}
RaftMsg::ClientWriteRequest{rpc, tx} => {
self.core.forward_client_write_request(rpc, tx);
}
RaftMsg::Initialize{tx, ..} => {
self.core.reject_init_with_config(tx);
@ -800,8 +816,11 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
RaftMsg::InstallSnapshot{rpc, tx} => {
let _ = tx.send(self.core.handle_install_snapshot_request(rpc).await);
}
RaftMsg::ClientRequest{rpc, tx} => {
self.core.forward_client_request(rpc, tx);
RaftMsg::ClientReadRequest{tx} => {
self.core.forward_client_read_request(tx);
}
RaftMsg::ClientWriteRequest{rpc, tx} => {
self.core.forward_client_write_request(rpc, tx);
}
RaftMsg::Initialize{members, tx} => {
let _ = tx.send(self.handle_init_with_config(members).await);

View File

@ -16,7 +16,13 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
self.core.last_log_index, self.core.last_log_term, self.core.commit_index,
self.core.network.clone(), self.core.storage.clone(), self.replicationtx.clone(),
);
ReplicationState{match_index: self.core.last_log_index, is_at_line_rate: false, replstream, remove_after_commit: None}
ReplicationState{
match_index: self.core.last_log_index,
match_term: self.core.current_term,
is_at_line_rate: false,
replstream,
remove_after_commit: None,
}
}
/// Handle a replication event coming from one of the replication streams.
@ -25,7 +31,7 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
let res = match event {
ReplicaEvent::RateUpdate{target, is_line_rate} => self.handle_rate_update(target, is_line_rate).await,
ReplicaEvent::RevertToFollower{target, term} => self.handle_revert_to_follower(target, term).await,
ReplicaEvent::UpdateMatchIndex{target, match_index} => self.handle_update_match_index(target, match_index).await,
ReplicaEvent::UpdateMatchIndex{target, match_index, match_term} => self.handle_update_match_index(target, match_index, match_term).await,
ReplicaEvent::NeedsSnapshot{target, tx} => self.handle_needs_snapshot(target, tx).await,
ReplicaEvent::Shutdown => {
self.core.set_target_state(State::Shutdown);
@ -88,10 +94,11 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
/// Handle events from a replication stream which updates the target node's match index.
#[tracing::instrument(level="trace", skip(self, target, match_index))]
async fn handle_update_match_index(&mut self, target: NodeId, match_index: u64) -> RaftResult<()> {
async fn handle_update_match_index(&mut self, target: NodeId, match_index: u64, match_term: u64) -> RaftResult<()> {
// If this is a non-voter, then update and return.
if let Some(state) = self.non_voters.get_mut(&target) {
state.state.match_index = match_index;
state.state.match_term = match_term;
return Ok(());
}
@ -100,6 +107,7 @@ impl<'a, D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>
match self.nodes.get_mut(&target) {
Some(state) => {
state.match_index = match_index;
state.match_term = match_term;
if let Some(threshold) = &state.remove_after_commit {
if &match_index >= threshold {
needs_removal = true;

View File

@ -3,7 +3,7 @@
use thiserror::Error;
use crate::{AppData, NodeId};
use crate::raft::ClientRequest;
use crate::raft::ClientWriteRequest;
/// A result type where the error variant is always a `RaftError`.
pub type RaftResult<T> = std::result::Result<T, RaftError>;
@ -29,15 +29,26 @@ impl From<tokio::io::Error> for RaftError {
}
}
/// An error related to a client request.
/// An error related to a client read request.
#[derive(Debug, Error)]
pub enum ClientError<D: AppData> {
pub enum ClientReadError {
/// A Raft error.
#[error("{0}")]
RaftError(#[from] RaftError),
/// The client request must be forwarded to the cluster leader.
#[error("the client request must be forwarded to the cluster leader")]
ForwardToLeader(ClientRequest<D>, Option<NodeId>),
/// The client read request must be forwarded to the cluster leader.
#[error("the client read request must be forwarded to the cluster leader")]
ForwardToLeader(Option<NodeId>),
}
/// An error related to a client write request.
#[derive(Debug, Error)]
pub enum ClientWriteError<D: AppData> {
/// A Raft error.
#[error("{0}")]
RaftError(#[from] RaftError),
/// The client write request must be forwarded to the cluster leader.
#[error("the client write request must be forwarded to the cluster leader")]
ForwardToLeader(ClientWriteRequest<D>, Option<NodeId>),
}
/// Error variants related to configuration.
@ -54,6 +65,7 @@ pub enum ConfigError {
/// The set of errors which may take place when initializing a pristine Raft node.
#[derive(Debug, Error)]
#[non_exhaustive]
pub enum InitializeError {
/// An internal error has taken place.
#[error("{0}")]
@ -65,6 +77,7 @@ pub enum InitializeError {
/// The set of errors which may take place when requesting to propose a config change.
#[derive(Debug, Error)]
#[non_exhaustive]
pub enum ChangeConfigError {
/// An error related to the processing of the config change request.
///
@ -91,11 +104,11 @@ pub enum ChangeConfigError {
Noop,
}
impl<D: AppData> From<ClientError<D>> for ChangeConfigError {
fn from(src: ClientError<D>) -> Self {
impl<D: AppData> From<ClientWriteError<D>> for ChangeConfigError {
fn from(src: ClientWriteError<D>) -> Self {
match src {
ClientError::RaftError(err) => Self::RaftError(err),
ClientError::ForwardToLeader(_, _) => Self::NodeNotLeader,
ClientWriteError::RaftError(err) => Self::RaftError(err),
ClientWriteError::ForwardToLeader(_, _) => Self::NodeNotLeader,
}
}
}

View File

@ -14,19 +14,16 @@ use std::fmt::Debug;
use serde::{Serialize, de::DeserializeOwned};
// Top-level exports.
pub use crate::{
config::{Config, ConfigBuilder, SnapshotPolicy},
core::State,
error::{ClientError, ConfigError, InitializeError, ChangeConfigError, RaftError},
error::{ClientWriteError, ConfigError, InitializeError, ChangeConfigError, RaftError},
metrics::RaftMetrics,
network::RaftNetwork,
raft::Raft,
storage::RaftStorage,
};
// Re-exports.
pub use async_trait::async_trait;
pub use async_trait;
/// A Raft node's ID.
pub type NodeId = u64;
@ -45,13 +42,13 @@ pub trait AppData: Clone + Debug + Send + Sync + Serialize + DeserializeOwned +
///
/// The intention of this trait is that applications which are using this crate will be able to
/// use their own concrete data types for returning response data from the storage layer when an
/// entry is successfully applied to the state machine as part of a client request (this is not
/// used during replication). This allows applications to seamlessly return application specific
/// data from their storage layer, up through Raft, and back into their application for returning
/// entry is applied to the state machine as part of a client request (this is not used during
/// replication). This allows applications to seamlessly return application specific data from
/// their storage layer, up through Raft, and back into their application for returning
/// data to clients.
///
/// This type must encapsulate both success and error responses, as application specific logic
/// related to the success or failure of a client request, application specific validation logic,
/// enforcing of data constraints, and anything of that nature are expressly out of the realm of
/// related to the success or failure of a client request application specific validation logic,
/// enforcing of data constraints, and anything of that nature are expressly out of the realm of
/// the Raft consensus protocol.
pub trait AppDataResponse: Clone + Debug + Send + Sync + Serialize + DeserializeOwned + 'static {}

View File

@ -11,7 +11,7 @@ use crate::NodeId;
use crate::core::State;
use crate::raft::MembershipConfig;
/// Baseline metrics of the current state of the subject Raft node.
/// A set of metrics describing the current state of a Raft node.
///
/// See the [module level documentation](TODO:)
/// for more details.

View File

@ -3,7 +3,7 @@
use anyhow::Result;
use async_trait::async_trait;
use crate::AppData;
use crate::{AppData, NodeId};
use crate::raft::{AppendEntriesRequest, AppendEntriesResponse};
use crate::raft::{InstallSnapshotRequest, InstallSnapshotResponse};
use crate::raft::{VoteRequest, VoteResponse};
@ -18,11 +18,11 @@ pub trait RaftNetwork<D>: Send + Sync + 'static
D: AppData,
{
/// Send an AppendEntries RPC to the target Raft node (§5).
async fn append_entries(&self, target: u64, rpc: AppendEntriesRequest<D>) -> Result<AppendEntriesResponse>;
async fn append_entries(&self, target: NodeId, rpc: AppendEntriesRequest<D>) -> Result<AppendEntriesResponse>;
/// Send an InstallSnapshot RPC to the target Raft node (§7).
async fn install_snapshot(&self, target: u64, rpc: InstallSnapshotRequest) -> Result<InstallSnapshotResponse>;
async fn install_snapshot(&self, target: NodeId, rpc: InstallSnapshotRequest) -> Result<InstallSnapshotResponse>;
/// Send a RequestVote RPC to the target Raft node (§5).
async fn vote(&self, target: u64, rpc: VoteRequest) -> Result<VoteResponse>;
async fn vote(&self, target: NodeId, rpc: VoteRequest) -> Result<VoteResponse>;
}

View File

@ -10,26 +10,22 @@ use tokio::task::JoinHandle;
use crate::{AppData, AppDataResponse, NodeId, RaftNetwork, RaftStorage};
use crate::config::Config;
use crate::error::{ClientError, InitializeError, ChangeConfigError, RaftError, RaftResult};
use crate::error::{ClientReadError, ClientWriteError, ChangeConfigError, InitializeError, RaftError, RaftResult};
use crate::metrics::RaftMetrics;
use crate::core::RaftCore;
/// The Raft API.
///
/// This type implements the full Raft spec, and is the interface into a running Raft node.
/// This type implements the full Raft spec, and is the interface to a running Raft node.
/// Applications building on top of Raft will use this to spawn a Raft task and interact with
/// the spawned task.
///
/// For more information on the Raft protocol, see the specification here:
/// https://raft.github.io/raft.pdf (**pdf warning**).
///
/// The beginning of §5, the spec has a condensed summary of the Raft consensus algorithm. This
/// crate, and especially this actor, attempts to follow the terminology and nomenclature used
/// there as precisely as possible to aid in understanding this system.
/// For more information on the Raft protocol, see
/// [the specification here](https://raft.github.io/raft.pdf) (**pdf warning**).
///
/// ### shutting down
/// If any of the interfaces returns a `RaftError::ShuttingDown`, this indicates that the Raft node
/// is shutting down (probably for data safety reasons due to a storage error), and the `shutdown`
/// is shutting down (potentially for data safety reasons due to a storage error), and the `shutdown`
/// method should be called on this type to await the shutdown of the node. If the parent
/// application needs to shutdown the Raft node for any reason, calling `shutdown` will do the trick.
pub struct Raft<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> {
@ -51,7 +47,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// state machine. This ensures that restarts of the node will yield the same ID every time.
///
/// ### `config`
/// The runtime config Raft. See the docs on the `Config` object for more details.
/// Raft's runtime config. See the docs on the `Config` object for more details.
///
/// ### `network`
/// An implementation of the `RaftNetwork` trait which will be used by Raft for sending RPCs to
@ -84,7 +80,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// sent by Raft nodes via their `RaftNetwork` implementation. See the [networking section](TODO:)
/// in the guide for more details.
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn append_entries(&self, rpc: AppendEntriesRequest<D>) -> RaftResult<AppendEntriesResponse> {
pub async fn append_entries(&self, rpc: AppendEntriesRequest<D>) -> Result<AppendEntriesResponse, RaftError> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::AppendEntries{rpc, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| RaftError::ShuttingDown).and_then(|res| res)?)
@ -98,7 +94,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// sent by Raft nodes via their `RaftNetwork` implementation. See the [networking section](TODO:)
/// in the guide for more details.
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn vote(&self, rpc: VoteRequest) -> RaftResult<VoteResponse> {
pub async fn vote(&self, rpc: VoteRequest) -> Result<VoteResponse, RaftError> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::RequestVote{rpc, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| RaftError::ShuttingDown).and_then(|res| res)?)
@ -113,33 +109,45 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// sent by Raft nodes via their `RaftNetwork` implementation. See the [networking section](TODO:)
/// in the guide for more details.
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn install_snapshot(&self, rpc: InstallSnapshotRequest) -> RaftResult<InstallSnapshotResponse> {
pub async fn install_snapshot(&self, rpc: InstallSnapshotRequest) -> Result<InstallSnapshotResponse, RaftError> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::InstallSnapshot{rpc, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| RaftError::ShuttingDown).and_then(|res| res)?)
}
/// Submit a client request to this Raft node to update the state of the system (§5.1).
/// Check to ensure this node is still the cluster leader, in order to guard against stale reads (§8).
///
/// Client requests are application specific and should contain whatever type of data is needed
/// by the application itself.
/// The actual read operation itself is up to the application, this method just ensures that
/// the read will not be stale.
#[tracing::instrument(level="debug", skip(self))]
pub async fn client_read(&self) -> Result<(), ClientReadError> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::ClientReadRequest{tx}).map_err(|_| ClientReadError::RaftError(RaftError::ShuttingDown))?;
Ok(rx.await.map_err(|_| ClientReadError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
}
/// Submit a mutating client request to Raft to update the state of the system (§5.1).
///
/// It will be appended to the log, committed to the cluster, and then applied to the
/// application state machine. The result of applying the request to the state machine will
/// be returned as the response from this method.
///
/// Our goal for Raft is to implement linearizable semantics. If the leader crashes after committing
/// a log entry but before responding to the client, the client may retry the command with a new
/// leader, causing it to be executed a second time.
///
/// The solution is for clients to assign unique serial numbers to every command. Then, the state
/// machine tracks the latest serial number processed for each client, along with the associated
/// response. If it receives a command whose serial number has already been executed, it responds
/// immediately without reexecuting the request (§8).
/// leader, causing it to be executed a second time. As such, clients should assign unique serial
/// numbers to every command. Then, the state machine should track the latest serial number
/// processed for each client, along with the associated response. If it receives a command whose
/// serial number has already been executed, it responds immediately without reexecuting the
/// request (§8). The `RaftStorage::apply_entry_to_state_machine` method is the perfect place
/// to implement this.
///
/// These are application specific requirements, and must be implemented by the application which is
/// being built on top of Raft.
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn client(&self, rpc: ClientRequest<D>) -> Result<ClientResponse<R>, ClientError<D>> {
pub async fn client_write(&self, rpc: ClientWriteRequest<D>) -> Result<ClientWriteResponse<R>, ClientWriteError<D>> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::ClientRequest{rpc, tx}).map_err(|_| ClientError::RaftError(RaftError::ShuttingDown))?;
Ok(rx.await.map_err(|_| ClientError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
self.tx_api.send(RaftMsg::ClientWriteRequest{rpc, tx}).map_err(|_| ClientWriteError::RaftError(RaftError::ShuttingDown))?;
Ok(rx.await.map_err(|_| ClientWriteError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
}
/// Initialize a pristine Raft node with the given config.
@ -153,7 +161,8 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// This command will work for single-node or multi-node cluster formation. This command
/// should be called with all discovered nodes which need to be part of cluster, and as such
/// it is recommended that applications be configured with an initial cluster formation delay
/// which will allow time for the initial members of the cluster to be discovered for this call.
/// which will allow time for the initial members of the cluster to be discovered (by the
/// parent application) for this call.
///
/// If successful, this routine will set the given config as the active config, only in memory,
/// and will start an election.
@ -167,11 +176,8 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// entry (instead of the normal blank entry created by new leaders).
///
/// Every member of the cluster should perform these actions. This routine is race-condition
/// free, and Raft guarantees that the first node to become the cluster leader will propage
/// free, and Raft guarantees that the first node to become the cluster leader will propagate
/// only its own config.
///
/// Once a cluster is up and running, the `propose_config_change` routine should be used to
/// update the cluster's membership config.
#[tracing::instrument(level="debug", skip(self))]
pub async fn initialize(&self, members: HashSet<NodeId>) -> Result<(), InitializeError> {
let (tx, rx) = oneshot::channel();
@ -188,7 +194,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
///
/// Calling this API will add the target node as a non-voter, starting the syncing process.
/// Once the node is up-to-speed, this function will return. It is the responsibility of the
/// application to then call `change_config` once all of the new nodes are synced.
/// application to then call `change_membership` once all of the new nodes are synced.
///
/// If this Raft node is not the cluster leader, then this call will fail.
#[tracing::instrument(level="debug", skip(self))]
@ -202,7 +208,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
///
/// This will cause the leader to begin a cluster membership configuration change. If there
/// are new nodes in the proposed config which are not already registered as non-voters — from
/// an earlier call to `add_non_voter` — then the new nodes first be synced as non-voters
/// an earlier call to `add_non_voter` — then the new nodes will first be synced as non-voters
/// before moving the cluster into joint consensus. As this process may take some time, it is
/// recommended that `add_non_voter` be called first for new nodes, and then once all new nodes
/// have been synchronized, call this method to start reconfiguration.
@ -210,7 +216,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
/// If this Raft node is not the cluster leader, then the proposed configuration change will be
/// rejected.
#[tracing::instrument(level="debug", skip(self))]
pub async fn change_config(&self, members: HashSet<NodeId>) -> Result<(), ChangeConfigError> {
pub async fn change_membership(&self, members: HashSet<NodeId>) -> Result<(), ChangeConfigError> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::ChangeMembership{members, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| ChangeConfigError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
@ -228,36 +234,40 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Ra
}
}
pub(crate) type ClientResponseTx<D, R> = oneshot::Sender<Result<ClientResponse<R>, ClientError<D>>>;
pub(crate) type ClientWriteResponseTx<D, R> = oneshot::Sender<Result<ClientWriteResponse<R>, ClientWriteError<D>>>;
pub(crate) type ClientReadResponseTx = oneshot::Sender<Result<(), ClientReadError>>;
pub(crate) type ChangeMembershipTx = oneshot::Sender<Result<(), ChangeConfigError>>;
/// A message coming from the Raft API.
pub(crate) enum RaftMsg<D: AppData, R: AppDataResponse> {
AppendEntries{
AppendEntries {
rpc: AppendEntriesRequest<D>,
tx: oneshot::Sender<Result<AppendEntriesResponse, RaftError>>,
},
RequestVote{
RequestVote {
rpc: VoteRequest,
tx: oneshot::Sender<Result<VoteResponse, RaftError>>,
},
InstallSnapshot{
InstallSnapshot {
rpc: InstallSnapshotRequest,
tx: oneshot::Sender<Result<InstallSnapshotResponse, RaftError>>,
},
ClientRequest{
rpc: ClientRequest<D>,
tx: ClientResponseTx<D, R>,
ClientWriteRequest {
rpc: ClientWriteRequest<D>,
tx: ClientWriteResponseTx<D, R>,
},
Initialize{
ClientReadRequest {
tx: ClientReadResponseTx,
},
Initialize {
members: HashSet<NodeId>,
tx: oneshot::Sender<Result<(), InitializeError>>,
},
AddNonVoter{
AddNonVoter {
id: NodeId,
tx: ChangeMembershipTx,
},
ChangeMembership{
ChangeMembership {
members: HashSet<NodeId>,
tx: ChangeMembershipTx,
},
@ -427,13 +437,6 @@ impl MembershipConfig {
self.members_after_consensus.is_some()
}
/// Get the length of the `members` vec.
///
/// This does not consider the length of the config after joint consensus, if applicable.
pub fn len(&self) -> usize {
self.members.len()
}
/// Create a new initial config containing only the given node ID.
pub fn new_initial(id: NodeId) -> Self {
let mut members = HashSet::new();
@ -508,16 +511,16 @@ pub struct InstallSnapshotResponse {
/// An application specific client request to update the state of the system (§5.1).
///
/// The entries of this payload will be appended to the Raft log and then applied to the Raft state
/// The entry of this payload will be appended to the Raft log and then applied to the Raft state
/// machine according to the Raft protocol.
#[derive(Debug, Serialize, Deserialize)]
pub struct ClientRequest<D: AppData> {
pub struct ClientWriteRequest<D: AppData> {
/// The application specific contents of this client request.
#[serde(bound="D: AppData")]
pub(crate) entry: EntryPayload<D>,
}
impl<D: AppData> ClientRequest<D> {
impl<D: AppData> ClientWriteRequest<D> {
/// Create a new client payload instance with a normal entry type.
pub fn new(entry: D) -> Self {
Self::new_base(EntryPayload::Normal(EntryNormal{data: entry}))
@ -543,7 +546,7 @@ impl<D: AppData> ClientRequest<D> {
/// The response to a `ClientRequest`.
#[derive(Debug, Serialize, Deserialize)]
pub struct ClientResponse<R: AppDataResponse> {
pub struct ClientWriteResponse<R: AppDataResponse> {
/// The log index of the successfully processed client request.
pub index: u64,
/// Application specific response data.

View File

@ -145,7 +145,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Re
ReplicationStream{handle, repltx: raftrx_tx}
}
#[tracing::instrument(level="trace", skip(self), fields(id=self.id, target=self.target))]
#[tracing::instrument(level="trace", skip(self), fields(id=self.id, target=self.target, cluster=%self.config.cluster_name))]
async fn main(mut self) {
// Perform an initial heartbeat.
self.send_append_entries().await;
@ -213,7 +213,7 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Re
self.next_index = index + 1; // This should always be the next expected index.
self.match_index = index;
self.match_term = term;
let _ = self.rafttx.send(ReplicaEvent::UpdateMatchIndex{target: self.target, match_index: index});
let _ = self.rafttx.send(ReplicaEvent::UpdateMatchIndex{target: self.target, match_index: index, match_term: term});
// If running at line rate, and our buffered outbound requests have accumulated too
// much, we need to purge and transition to a lagging state. The target is not able to
@ -249,6 +249,9 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Re
// it will never exist. So instead, we just return, and accept the conflict data.
if &conflict.index == &0 {
self.target_state = TargetReplState::Lagging;
let _ = self.rafttx.send(ReplicaEvent::UpdateMatchIndex{
target: self.target, match_index: self.match_index, match_term: self.match_term,
});
return;
}
@ -261,6 +264,9 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Re
// This condition would only ever be reached if the log has been removed due to
// log compaction (barring critical storage failure), so transition to snapshotting.
self.target_state = TargetReplState::Snapshotting;
let _ = self.rafttx.send(ReplicaEvent::UpdateMatchIndex{
target: self.target, match_index: self.match_index, match_term: self.match_term,
});
return;
}
Err(err) => {
@ -272,6 +278,9 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Re
};
// Check snapshot policy and handle conflict as needed.
let _ = self.rafttx.send(ReplicaEvent::UpdateMatchIndex{
target: self.target, match_index: self.match_index, match_term: self.match_term,
});
match &self.config.snapshot_policy {
SnapshotPolicy::LogsSinceLast(threshold) => {
let diff = &self.last_log_index - &conflict.index; // NOTE WELL: underflow is guarded against above.
@ -286,10 +295,6 @@ impl<D: AppData, R: AppDataResponse, N: RaftNetwork<D>, S: RaftStorage<D, R>> Re
}
}
}
self.match_index = if self.match_index > 0 { self.match_index - 1} else { 0 };
self.next_index = self.match_index + 1;
self.target_state = TargetReplState::Lagging;
}
/// Perform a check to see if this replication stream is lagging behind far enough that a
@ -422,6 +427,8 @@ pub(crate) enum ReplicaEvent<S>
target: NodeId,
/// The index of the most recent log known to have been successfully replicated on the target.
match_index: u64,
/// The term of the most recent log known to have been successfully replicated on the target.
match_term: u64,
},
/// An event indicating that the Raft node needs to revert to follower state.
RevertToFollower{

View File

@ -90,7 +90,7 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// the node's ID so that it is consistent across restarts.
async fn get_membership_config(&self) -> Result<MembershipConfig>;
/// A request from Raft to get Raft's state information from storage.
/// Get Raft's state information from storage.
///
/// When the Raft node is first started, it will call this interface on the storage system to
/// fetch the last known state from stable storage. If no such entry exists due to being the
@ -102,10 +102,10 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// state record; and the index of the last log applied to the state machine.
async fn get_initial_state(&self) -> Result<InitialState>;
/// A request from Raft to save its hard state.
/// Save Raft's hard-state.
async fn save_hard_state(&self, hs: &HardState) -> Result<()>;
/// A request from Raft to get a series of log entries from storage.
/// Get a series of log entries from storage.
///
/// The start value is inclusive in the search and the stop value is non-inclusive: `[start, stop)`.
async fn get_log_entries(&self, start: u64, stop: u64) -> Result<Vec<Entry<D>>>;
@ -114,22 +114,16 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// of the log if `stop` is `None`.
async fn delete_logs_from(&self, start: u64, stop: Option<u64>) -> Result<()>;
/// A request from Raft to append a new entry to the log.
/// Append a new entry to the log.
async fn append_entry_to_log(&self, entry: &Entry<D>) -> Result<()>;
/// A request from Raft to replicate a payload of entries to the log.
///
/// These requests come about via the Raft leader's replication process. An error coming from this
/// interface will cause Raft to shutdown, as this is not where application logic should be
/// returning application specific errors. Application specific constraints may only be enforced
/// in the `AppendEntryToLog` handler.
/// Replicate a payload of entries to the log.
///
/// Though the entries will always be presented in order, each entry's index should be used to
/// determine its location to be written in the log, as logs may need to be overwritten under
/// some circumstances.
/// determine its location to be written in the log.
async fn replicate_to_log(&self, entries: &[Entry<D>]) -> Result<()>;
/// A request from Raft to apply the given log entry to the state machine.
/// Apply the given log entry to the state machine.
///
/// The Raft protocol guarantees that only logs which have been _committed_, that is, logs which
/// have been replicated to a majority of the cluster, will be applied to the state machine.
@ -147,13 +141,13 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// and error variants encoded in the type, perhaps using an inner `Result` type.
async fn apply_entry_to_state_machine(&self, index: &u64, data: &D) -> Result<R>;
/// A request from Raft to apply the given payload of entries to the state machine, as part of replication.
/// Apply the given payload of entries to the state machine, as part of replication.
///
/// The Raft protocol guarantees that only logs which have been _committed_, that is, logs which
/// have been replicated to a majority of the cluster, will be applied to the state machine.
async fn replicate_to_state_machine(&self, entries: &[(&u64, &D)]) -> Result<()>;
/// A request from Raft to perform log compaction, returning a handle to the generated snapshot.
/// Perform log compaction, returning a handle to the generated snapshot.
///
/// ### `through`
/// The log should be compacted starting from entry `0` and should cover all entries through the
@ -165,7 +159,8 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// for details on how to implement this handler.
async fn do_log_compaction(&self, through: u64) -> Result<CurrentSnapshotData<Self::Snapshot>>;
/// Create a new snapshot returning a writable handle to the snapshot object along with the ID of the snapshot.
/// Create a new blank snapshot, returning a writable handle to the snapshot object along with
/// the ID of the snapshot.
///
/// ### implementation guide
/// See the [storage chapter of the guide]()
@ -174,11 +169,12 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// Finalize the installation of a snapshot which has finished streaming from the cluster leader.
///
/// Delete all entries in the log, stopping at `delete_through`, unless `None`, in which case
/// Delete all entries in the log through `delete_through`, unless `None`, in which case
/// all entries of the log are to be deleted.
///
/// Write a new snapshot pointer to the log at the given `index`. The snapshot pointer should be
/// constructed via the `Entry::new_snapshot_pointer` constructor.
/// constructed via the `Entry::new_snapshot_pointer` constructor and the other parameters
/// provided to this method.
///
/// All other snapshots should be deleted at this point.
///
@ -192,7 +188,7 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
id: String, snapshot: Box<Self::Snapshot>,
) -> Result<()>;
/// A request from Raft to get a readable handle to the current snapshot, along with its metadata.
/// Get a readable handle to the current snapshot, along with its metadata.
///
/// ### implementation algorithm
/// Implementing this method should be straightforward. Check the configured snapshot
@ -202,6 +198,6 @@ pub trait RaftStorage<D, R>: Send + Sync + 'static
/// the current live snapshot, and any new snapshot which is being created.
///
/// A proper snapshot implementation will store the term, index and membership config as part
/// of the snapshot as well, which can be decoded for creating this method's response.
/// of the snapshot, which should be decoded for creating this method's response data.
async fn get_current_snapshot(&self) -> Result<Option<CurrentSnapshotData<Self::Snapshot>>>;
}

View File

@ -0,0 +1,50 @@
mod fixtures;
use std::sync::Arc;
use std::time::Duration;
use anyhow::Result;
use async_raft::Config;
use tokio::time::delay_for;
use fixtures::RaftRouter;
/// Client read tests.
///
/// What does this test do?
///
/// - create a stable 3-node cluster.
/// - call the client_read interface on the leader, and assert success.
/// - call the client_read interface on the followers, and assert failure.
///
/// RUST_LOG=async_raft,memstore,client_reads=trace cargo test -p async-raft --test client_reads
#[tokio::test(core_threads=4)]
async fn client_reads() -> Result<()> {
fixtures::init_tracing();
// Setup test dependencies.
let config = Arc::new(Config::build("test".into()).validate().expect("failed to build Raft config"));
let router = Arc::new(RaftRouter::new(config.clone()));
router.new_raft_node(0).await;
router.new_raft_node(1).await;
router.new_raft_node(2).await;
// Assert all nodes are in non-voter state & have no entries.
delay_for(Duration::from_secs(10)).await;
router.assert_pristine_cluster().await;
// Initialize the cluster, then assert that a stable cluster was formed & held.
tracing::info!("--- initializing cluster");
router.initialize_from_single_node(0).await?;
delay_for(Duration::from_secs(10)).await;
router.assert_stable_cluster(Some(1), Some(1)).await;
// Get the ID of the leader, and assert that client_read succeeds.
let leader = router.leader().await.expect("leader not found");
assert_eq!(leader, 0, "expected leader to be node 0, got {}", leader);
router.client_read(leader).await.expect(&format!("expected client_read to succeed for cluster leader {}", leader));
router.client_read(1).await.expect_err("expected client_read on follower node 1 to fail");
router.client_read(2).await.expect_err("expected client_read on follower node 2 to fail");
Ok(())
}

View File

@ -51,7 +51,7 @@ async fn compaction() -> Result<()> {
// Add a new node and assert that it received the same snapshot.
router.new_raft_node(1).await;
router.add_non_voter(0, 1).await.expect("failed to add new node as non-voter");
router.change_config(0, hashset![0, 1]).await.expect("failed to modify cluster membership");
router.change_membership(0, hashset![0, 1]).await.expect("failed to modify cluster membership");
delay_for(Duration::from_secs(5)).await; // Wait to ensure metrics are updated (this is way more than enough).
router.assert_stable_cluster(Some(1), Some(502)).await; // We expect index to be 500 + 2 (joint & uniform config change entries).
router.assert_storage_state(1, 502, None, 500, Some((500.into(), 1, MembershipConfig{members: hashset![0u64], members_after_consensus: None}))).await;

View File

@ -56,7 +56,7 @@ async fn dynamic_membership() -> Result<()> {
inner?;
}
tracing::info!("--- changing cluster config");
router.change_config(0, hashset![0, 1, 2, 3, 4]).await?;
router.change_membership(0, hashset![0, 1, 2, 3, 4]).await?;
delay_for(Duration::from_secs(5)).await;
router.assert_stable_cluster(Some(1), Some(3)).await; // Still in term 1, so leader is still node 0.

View File

@ -6,13 +6,13 @@ use std::collections::{BTreeMap, HashSet};
use std::sync::Arc;
use anyhow::{anyhow, Result};
use async_raft::async_trait;
use async_raft::async_trait::async_trait;
use async_raft::{Config, NodeId, Raft, RaftMetrics, RaftNetwork, State};
use async_raft::error::{ChangeConfigError, ClientError};
use async_raft::error::{ChangeConfigError, ClientReadError, ClientWriteError};
use async_raft::raft::{AppendEntriesRequest, AppendEntriesResponse};
use async_raft::raft::{InstallSnapshotRequest, InstallSnapshotResponse};
use async_raft::raft::{VoteRequest, VoteResponse};
use async_raft::raft::ClientRequest;
use async_raft::raft::ClientWriteRequest;
use async_raft::raft::MembershipConfig;
use async_raft::storage::RaftStorage;
use memstore::{MemStore, ClientRequest as MemClientRequest, ClientResponse as MemClientResponse};
@ -124,10 +124,17 @@ impl RaftRouter {
node.0.add_non_voter(target).await
}
pub async fn change_config(&self, leader: NodeId, members: HashSet<NodeId>) -> Result<(), ChangeConfigError> {
pub async fn change_membership(&self, leader: NodeId, members: HashSet<NodeId>) -> Result<(), ChangeConfigError> {
let rt = self.routing_table.read().await;
let node = rt.get(&leader).expect(&format!("node with ID {} does not exist", leader));
node.0.change_config(members).await
node.0.change_membership(members).await
}
/// Send a client read request to the target node.
pub async fn client_read(&self, target: NodeId) -> Result<(), ClientReadError> {
let rt = self.routing_table.read().await;
let node = rt.get(&target).expect(&format!("node with ID {} does not exist", target));
node.0.client_read().await
}
/// Send a client request to the target node, causing test failure on error.
@ -146,10 +153,10 @@ impl RaftRouter {
}
}
async fn send_client_request(&self, target: NodeId, req: MemClientRequest) -> std::result::Result<MemClientResponse, ClientError<MemClientRequest>> {
async fn send_client_request(&self, target: NodeId, req: MemClientRequest) -> std::result::Result<MemClientResponse, ClientWriteError<MemClientRequest>> {
let rt = self.routing_table.read().await;
let node = rt.get(&target).expect(&format!("node '{}' does not exist in routing table", target));
node.0.client(ClientRequest::new(req)).await.map(|res| res.data)
node.0.client_write(ClientWriteRequest::new(req)).await.map(|res| res.data)
}
//////////////////////////////////////////////////////////////////////////////////////////////

View File

@ -45,7 +45,7 @@ async fn stepdown() -> Result<()> {
assert_eq!(0, orig_leader, "expected original leader to be node 0");
router.new_raft_node(2).await;
router.new_raft_node(3).await;
router.change_config(orig_leader, hashset![1, 2, 3]).await?;
router.change_membership(orig_leader, hashset![1, 2, 3]).await?;
delay_for(Duration::from_secs(1)).await; // Give time for step down metrics to flow through.
// Assert on the state of the old leader.

View File

@ -3,8 +3,8 @@ authors = ["Anthony Dodd"]
language = "en"
multilingual = false
src = "guide/src"
title = "Actix Raft"
description = "The ActixRaft user guide."
title = "Async Raft"
description = "The async-raft user guide."
[build]
build-dir = "guide/book"

View File

@ -1,10 +1,11 @@
Actix Raft
Async Raft
==========
[Overview](./overview.md)
- [Getting Started](./getting-started.md)
- [Raft](./raft.md)
- [Network](./network.md)
- [Storage](./storage.md)
- [Raft API](./raft.md)
- [Putting It All Together](./putting-it-all-together.md)
- [Cluster Controls](./cluster-controls.md)
- [Cluster Formation](./cluster-formation.md)
- [Dynamic Membership](./dynamic-membership.md)

View File

@ -1,6 +1,6 @@
Cluster Controls
================
Raft nodes may be controlled in various ways outside of the normal flow of the Raft protocol using the `admin` message types. This allows the parent application — within which the Raft node is running — to influence the Raft node's behavior based on application level needs.
Raft nodes may be controlled in various ways outside of the normal flow of the Raft protocol using some of the API methods of the `Raft` type. This allows the parent application — within which the Raft node is running — to influence the Raft node's behavior based on application level needs.
### concepts
In the world of Raft consensus, there are a few aspects of a Raft node's lifecycle which are not directly dictated in the Raft spec. Cluster formation and the preliminary details of what would lead to dynamic cluster membership changes are a few examples of concepts not directly detailed in the spec. This implementation of Raft offers as much flexibility as possible to deal with such details in a way which is safe according to the Raft specification, but also in a way which preserves flexibility for the many different types of applications which may be implemented using Raft.

View File

@ -1,18 +1,14 @@
Cluster Formation
=================
All Raft nodes, when they first come online in a pristine state, will enter into the `NonVoter` state, which is a completely passive state. This gives the parent application the ability to issue admin commands to the node based on the intention of the parent application.
All Raft nodes, when they first come online in a pristine state, will enter into the `NonVoter` state, which is a completely passive state.
To form a new cluster, all application nodes must issue the `InitWithConfig` command to their embedded Raft nodes with the IDs of all discovered nodes which are to be part of the cluster (including the ID of the running node). Or if the application is to run in a standalone / single-node manner, it may issue the command with only its own ID.
To form a new cluster, application nodes must call the `Raft.initialize` method with the IDs of all discovered nodes which are to be part of the cluster (including the ID of the running node). Or if the application is to run in a standalone / single-node manner, it may issue the command with only its own ID.
#### `InitWithConfig`
This command is used exclusively for the formation of new clusters. This command will fail if the node is not in the `NonVoter` state, or if the node's log index is not `0`.
#### `Raft.initialize`
This method is used exclusively for the formation of new clusters. This command will fail if the node is not in the `NonVoter` state, or if the node's log index is greater than `0`.
This will cause the Raft node to hold the given configuration in memory and then immediately perform the election protocol. For single-node clusters, the node will immediately become leader, for multi-node clusters it will submit `RequestVote` RPCs to all of the nodes in the given config list. **NOTE WELL that EVERY node in the cluster MUST perform this action** when a new cluster is being formed. It is safe for all nodes to issue this command in parallel. Once this process has been completed, the newly elected leader will append the given membership config data to the log, ensuring that the new configuration will be reckoned as the initial cluster configuration moving forward throughout the life of the cluster.
This will cause the Raft node to hold the given configuration in memory and then immediately perform the election protocol. For single-node clusters, the node will immediately become leader, for multi-node clusters it will submit `RequestVote` RPCs to all of the nodes in the given config list. **NOTE WELL that it is safe for EVERY node in the cluster to perform this action in parallel** when a new cluster is being formed. Once this process has been completed, the newly elected leader will append the given membership config data to the log, ensuring that the new configuration will be reckoned as the initial cluster configuration moving forward throughout the life of the cluster.
However, in order to ensure that multiple independent clusters aren't formed by prematurely issuing the `InitWithConfig` command before all peers are discovered, it would be prudent to have all discovered node's exchange some information during their handshake protocol. This will allow the parent application to make informed decisions as to whether the `InitWithConfig` should be called and how early it should be called when starting a new cluster. An application level configuration for this facet is recommended.
In order to ensure that multiple independent clusters aren't formed by prematurely calling the `Raft.initialize` method before all peers are discovered, it is recommended that applications adopt a configurable `cluster_formation_delay` setting. The value for such a configuration should simply be a few orders of magnitude greater than the amount of time it takes for all the nodes of a new cluster to come online and discover each other. There are alternative patterns which may be used. Ultimately, this is subject to the design of the application.
Generally speaking, an application config like `initial_cluster_formation_delay` (or the like), which configures the application to wait for the specifed amount of time before issuing an `InitWithConfig` command, should do the trick. The value for such a configuration should simply be a few orders of magnitude greater than the amount of time it takes for all the nodes of a new cluster to come online and discover each other.
As a rule of thumb, when new nodes come online, the leader of an existing Raft cluster will eventually discover the node (via the application's discovery system), and in such cases, the application should submit a new `ProposeConfigChange` to the leader to add it to the cluster. The same goes for removing nodes from the cluster.
**For single-node clusters**, scaling up the cluster by adding new nodes via the `ProposeConfigChange` command should work as expected, but there is one invariant which must be upheld: the original node of the cluster must remain online until at least half of the other new nodes have been brough up-to-date, otherwise the Raft cluster will not be able to make progress. After the other nodes have been brought up-to-date, everything should run normally according to the Raft spec.
As a rule of thumb, when new nodes come online, the leader of an existing Raft cluster will eventually discover the node (via the application's discovery system), and in such cases, the application could call the `Raft.add_non_voter` method to begin syncing the new node with the cluster. Once it is finished syncing, then applications should call the `Raft.change_membership` method to add the new node as a voting member of the cluster. For removing nodes from the cluster, the leader should call `Raft.change_membership` with the updated config, no preliminary steps are needed. See the next section for more details on this subject.

View File

@ -1,10 +1,13 @@
Dynamic Membership
==================
Throughout the lifecycle of a Raft cluster, various nodes may need to go offline for various reasons. They may experience hardware or software errors which cause them to go offline when unintended, or perhaps a cluster had too many nodes and it needs to downsize. New nodes may be added to clusters as well in order to replace old nodes, nodes going offline for maintenence, or simply to increase the size of a cluster. Applications may control such events using the `ProposeConfigChange` command. This command allows for nodes to be safely added and removed form a running Raft cluster.
Throughout the lifecycle of a Raft cluster, nodes will come and go. New nodes may need to be added to the cluster for various application specific reasons. Nodes may experience hardware failure and end up going offline. This implementation of Raft offers two mechanisms for controlling these lifecycle events.
#### `ProposeConfigChange`
This command will propose a new config change to a running cluster. This command will fail if the Raft node to which this command was submitted is not the Raft leader, and the outcome of the proposed config change must not leave the cluster in a state where it will have less than two functioning nodes, as the cluster would no longer be able to make progress in a safe manner. Once the leader receives this command, the new configuration will be appended to the log and the Raft dynamic configuration change protocol will begin. For more details on how this is implemented, see §6 of the Raft spec.
#### `Raft.add_non_voter`
This method will add a new non-voter to the cluster and will immediately begin syncing the node with the leader. This method may be called multiple times as needed. The `Future` returned by calling this method will resolve once the node is up-to-date and is ready to be added as a voting member of the cluster.
Cluster auto-healing, where cluster members which have been offline for some period of time are automatically removed, is an application specific behavior, but is fully supported via this dynamic cluster membership system.
#### `Raft.change_membership`
This method will start a cluster membership change. If there are any new nodes in the given config which were not previously added as non-voters from an earlier call to `Raft.add_non_voter`, then those nodes will begin the sync process. It is recommended that applications always call `Raft.add_non_voter` first when adding new nodes to the cluster, as this offers a bit more flexibility. Once `Raft.change_membership` is called, it can not be called again until the reconfiguration process is complete (which is typically quite fast).
Likewise, dynamically adding new nodes to a running cluster based on an application's discovery system is also fully supported by this system.
Cluster auto-healing — where cluster members which have been offline for some period of time are automatically removed — is an application specific behavior, but is fully supported via this dynamic cluster membership system. Simply call `Raft.change_membership` with the dead node removed from the membership set.
Cluster leader stepdown is also fully supported. Nothing special needs to take place. Simply call `Raft.change_membership` with the ID of the leader removed from the membership set. The leader will recognize that it is being removed from the cluster, and will stepdown once it has committed the config change to the cluster according to the safety protocols defined in the Raft spec.

View File

@ -1,9 +1,9 @@
Get To It
=========
In my very humble opinion, Rust is an outstanding language for building new databases, data stores, messaging systems and all of the other things. Books have already been written about this, so I will just say in conclusion:
- We don't need to keep allocating all of the memory of all of the clouds to the JVM.
Rust is an outstanding language for building stateful systems.
- We don't need to continue allocating all of the memory of all of the clouds to the JVM.
- We ABSOLUTELY need to move away from all of the memory safety issues which have compromised countless systems.
- Raft is a robust and powerful consensus protocol.
- Huge companies have invested a lot of money in creating actor-based languages specifically for creating database systems (think FoundationDB), and we already have Actix (thanks Nikolay)!
- Raft is a robust and powerful consensus protocol, and this implementation offers a fairly robust API to build against.
There are plenty of awesome things to build! Let's do this!

View File

@ -1,202 +1,62 @@
Getting Started
===============
This crate's `Raft` type is an Actix actor which is intended to run within some parent application, which traditionally will be some sort of data storage system (SQL, NoSQL, KV store, AMQP, Streaming, whatever). Inasmuch as the `Raft` instance is an actor, it is expected that the parent application is also built upon the Actix actor framework, though that is not technically required.
Raft is a distributed consensus protocol designed to manage a replicated log containing state machine commands from clients. Why use Raft? Among other things, it provides data storage systems with fault-tolerance, strong consistency and linearizability.
To use this crate, applications must also implement the `RaftStorage` & `RaftNetwork` traits. See the [storage](https://railgun-rs.github.io/actix-raft/storage.html) & [network](https://railgun-rs.github.io/actix-raft/network.html) chapters for details on what these traits represent and how to implement them. In brief, the implementing types must be actors which can handle specific message types which correspond to everything needed for Raft storage and networking.
A visual depiction of how Raft works (taken from the spec) can be seen below.
### deep dive
To get started, applications can define a type alias which declares the types which are going to be used for the application's data, errors, `RaftNetwork` impl & `RaftStorage` impl.
<p>
<img style="max-width:600px;" src="./images/raft-overview.png"/>
</p>
First, let's define the new application's main data type & a response type. This is the data which will be inside of Raft's normal log entries and the response type which the storage engine will return after applying them to the state machine.
Raft is intended to run within some parent application, which traditionally will be some sort of data storage system (SQL, NoSQL, KV store, AMQP, Streaming, Graph, whatever). You can do whatever you want with your application, Raft will provide you with the consensus module.
## first steps
In order to start using Raft, you will need to declare the data types you will use for client requests and client responses. Let's do that now. Throughout this guide, we will be using the `memstore` crate, which is an in-memory implementation of the `RaftStorage` trait for demo and testing purposes (part of the same repo). This will give us a concrete set of examples to work with, which also happen to be used for all of the integration tests of `async-raft` itself.
### `async_raft::AppData`
This marker trait is used to declare an application's data type. It has the following constraints: `Clone + Debug + Send + Sync + Serialize + DeserializeOwned + 'static`. Your data type represents the requests which will be sent to your application to create, update and delete data. Requests to read data should not be sent through Raft, only mutating requests. More on linearizable reads, and how to avoid stale reads, is discussed in the [Raft API chapter](TODO:).
The intention of this trait is that applications which are using this crate will be able to use their own concrete data types throughout their application without having to serialize and deserialize their data as it goes through Raft. Instead, applications can present their data models as-is to Raft, Raft will present it to the application's `RaftStorage` impl when ready, and the application may then deal with the data directly in the storage engine without having to do a preliminary deserialization.
##### impl
Finishing up this step is easy, just `impl AppData for YourData {}` ... and in most cases, that's it. You'll need to be sure that the aforementioned constraints are satisfied on `YourData`. The following derivation should do the trick `#[derive(Clone, Debug, Serialize, Deserialize)]`.
In the `memstore` crate, here is a snippet of what the code looks like:
```rust
use actix_raft::{AppData, AppDataResponse};
use serde::{Serialize, Deserialize};
/// The application's data type.
/// The application data request type which the `MemStore` works with.
///
/// Enum types are recommended as typically there will be different types of data mutating
/// requests which will be submitted by application clients.
/// Conceptually, for demo purposes, this represents an update to a client's status info,
/// returning the previously recorded status.
#[derive(Clone, Debug, Serialize, Deserialize)]
enum Data {
// Your data variants go here.
pub struct ClientRequest {
/* fields omitted */
}
/// The application's data response types.
///
/// Enum types are recommended as typically there will be multiple response types which can be
/// returned from the storage layer.
impl AppData for ClientRequest {}
```
### `async_raft::AppDataResponse`
This marker trait is used to declare an application's response data. It has the following constraints: `Clone + Debug + Send + Sync + Serialize + DeserializeOwned + 'static`.
The intention of this trait is that applications which are using this crate will be able to use their own concrete data types for returning response data from the storage layer when an entry is applied to the state machine as part of a client request (this is not used during replication). This allows applications to seamlessly return application specific data from their storage layer, up through Raft, and back into their application for returning data to clients.
This type must encapsulate both success and error responses, as application specific logic related to the success or failure of a client request — application specific validation logic, enforcing of data constraints, and anything of that nature — are expressly out of the realm of the Raft consensus protocol.
##### impl
Finishing up this step is also easy: `impl AppDataResponse for YourDataResponse {}`. The aforementioned derivation applies here as well.
In the `memstore` crate, here is a snippet of what the code looks like:
```rust
/// The application data response type which the `MemStore` works with.
#[derive(Clone, Debug, Serialize, Deserialize)]
enum DataResponse {
// Your response variants go here.
}
pub struct ClientResponse(Result<Option<String>, ClientError>);
/// This also has a `'static` lifetime constraint, so no `&` references at this time.
/// The new futures & async/await should help out with this quite a lot, so
/// hopefully this constraint will be removed in actix as well.
impl AppData for Data {}
/// This also has a `'static` lifetime constraint, so no `&` references at this time.
impl AppDataResponse for DataResponse {}
impl AppDataResponse for ClientResponse {}
```
Now we'll define the application's error type.
---
```rust
use actix_raft::AppError;
use serde::{Serialize, Deserialize};
/// The application's error struct. This could be an enum as well.
///
/// NOTE: the below impls for Display & Error can be
/// derived using crates like `Failure` &c.
#[derive(Debug, Serialize, Deserialize)]
pub struct Error;
impl std::fmt::Display for Error {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
// ... snip ...
}
}
impl std::error::Error for Error {}
// Mark this type for use as an `actix_raft::AppError`.
impl AppError for Error {}
```
Now for the two big parts. `RaftNetwork` & `RaftStorage`. Here, we will only look at the skeleton for these types. See the [network](https://railgun-rs.github.io/actix-raft/network.html) & [storage](https://railgun-rs.github.io/actix-raft/storage.html) chapters for more details on how to actually implement these types. First, let's cover the network impl.
```rust
use actix::{Actor, Context, ResponseActFuture};
use actix_raft::{RaftNetwork, messages};
/// Your application's network interface actor.
struct AppNetwork {/* ... snip ... */}
impl Actor for AppNetwork {
type Context = Context<Self>;
// ... snip ... other actix methods can be implemented here as needed.
}
// Ensure you impl this over your application's data type. Here, it is `Data`.
impl RaftNetwork<Data> for AppNetwork {}
// Then you just implement the various message handlers.
// See the network chapter for details.
impl Handler<messages::AppendEntriesRequest<Data>> for AppNetwork {
type Result = ResponseActFuture<Self, messages::AppendEntriesResponse, ()>;
fn handle(&mut self, _msg: messages::AppendEntriesRequest<Data>, _ctx: &mut Self::Context) -> Self::Result {
// ... snip ...
# actix::fut::err(())
}
}
// Impl handlers on `AppNetwork` for the other `actix_raft::messages` message types.
```
Now for the storage impl. We'll use an `actix::Context` here (which is async), but you could also use an `actix::SyncContext`.
```rust
use actix::{Actor, Context, ResponseActFuture};
use actix_raft::{NodeId, RaftStorage, storage};
/// Your application's storage interface actor.
struct AppStorage {/* ... snip ... */}
// Ensure you impl this over your application's data, data response & error types.
impl RaftStorage<Data, DataResponse, Error> for AppStorage {
type Actor = Self;
type Context = Context<Self>;
}
impl Actor for AppStorage {
type Context = Context<Self>;
// ... snip ... other actix methods can be implemented here as needed.
}
// Then you just implement the various message handlers.
// See the storage chapter for details.
impl Handler<storage::GetInitialState<Error>> for AppStorage {
type Result = ResponseActFuture<Self, storage::InitialState, Error>;
fn handle(&mut self, _msg: storage::GetInitialState<Error>, _ctx: &mut Self::Context) -> Self::Result {
// ... snip ...
# actix::fut::err(())
}
}
// Impl handlers on `AppStorage` for the other `actix_raft::storage` message types.
```
In order for Raft to expose metrics on how it is doing, we will need a type which can receive `RaftMetrics` messages. Application's can do whatever they want with this info. Expose integrations with Prometheus & Influx, trigger events, whatever is needed. Here we will keep it simple.
```rust
use actix::{Actor, Context};
use actix_raft::RaftMetrics;
/// Your application's metrics interface actor.
struct AppMetrics {/* ... snip ... */}
impl Actor for AppMetrics {
type Context = Context<Self>;
// ... snip ... other actix methods can be implemented here as needed.
}
impl Handler<RaftMetrics> for AppMetrics {
type Result = ();
fn handle(&mut self, _msg: RaftMetrics, _ctx: &mut Context<Self>) -> Self::Result {
// ... snip ...
}
}
```
And finally, a simple type alias which ties everything together. This type alias can then be used throughout the application's code base without the need to specify the various types being used for data, errors, network & storage.
```rust
use actix_raft::Raft;
/// A type alias used to define an application's concrete Raft type.
type AppRaft = Raft<Data, DataResponse, Error, AppNetwork, AppStorage>;
```
### booting up the system
Now that the various needed types are in place, the actix system will need to be started, the various actor types we've defined above will need to be started, and then we're off to the races.
```rust
use actix;
use actix_raft::{Config, ConfigBuilder, SnapshotPolicy};
fn main() {
// Build the actix system.
let sys = actix::System::new("my-awesome-app");
// Build the needed runtime config for Raft specifying where
// snapshots will be stored. See the storage chapter for more details.
let config = Config::build(String::from("/app/snapshots")).validate().unwrap();
// Start off with just a single node in the cluster. Applications
// should implement their own discovery system. See the cluster
// formation chapter for more details.
let members = vec![1];
// Start the various actor types and hold on to their addrs.
let network = AppNetwork.start();
let storage = AppStorage.start();
let metrics = AppMetrics.start();
let app_raft = AppRaft::new(1, config, network, storage, metrics).start();
// Run the actix system. Unix signals for termination &
// graceful shutdown are automatically handled.
let _ = sys.run();
}
```
----
You've already ascended to the next level of AWESOME! There is a lot more to cover, we're just getting started. Next, let's take a look at the `Raft` type in more detail.
Woot woot! Onward to the networking layer.

Binary file not shown.

After

Width:  |  Height:  |  Size: 96 KiB

Binary file not shown.

Before

Width:  |  Height:  |  Size: 47 KiB

View File

@ -1,9 +1,5 @@
Metrics
=======
`Raft` exports metrics on a regular interval in order to facilitate maximum observability and integration with metrics aggregations tools (eg, prometheus, influx &c) using the [`RaftMetrics`](https://docs.rs/actix-raft/latest/actix_raft/metrics/struct.RaftMetrics.html) type.
`Raft` exports metrics on its internal state via the `Raft.metrics` method, which returns a stream of [`RaftMetrics`](TODO:). The metrics themselves describe the state of the Raft node, its current role in the cluster, its current membership config, as well as information on the Raft log and the last index to be applied to the state machine.
The `Raft` instance constructor expects a `Recipient<RaftMetrics>` (an [actix::Recipient](https://docs.rs/actix/latest/actix/struct.Recipient.html)) to be supplied, and will use this recipient to export its metrics. The `RaftMetrics` type holds the baseline metrics on the state of the Raft node the metrics are coming from, its current role in the cluster, its current membership config, as well as information on the Raft log and the last index to be applied to the state machine.
Applications may use this data in whatever way is needed. The obvious use cases are to expose these metrics to a metrics collection system. Applications may also use this data to trigger events within higher levels of the parent application.
Metrics will be exported at a regular interval according to the [Config.metrics_rate](https://docs.rs/actix-raft/latest/actix_raft/config/struct.Config.html#structfield.metrics_rate) value, but will also emit a new metrics record any time the `state` of the Raft node changes, the `membership_config` changes, or the `current_leader` changes.
Applications may use this data in whatever way is needed. The obvious use cases are to expose these metrics to a metrics collection system, such as Prometheus, TimescaleDB, Influx &c. Applications may also use this data to trigger events within higher levels of the application itself.

View File

@ -2,42 +2,30 @@ Network
=======
Raft is a distributed consensus protocol, so the ability to send and receive data over a network is integral to the proper functionality of nodes within a Raft cluster.
The network capabilities required by this system are broken up into two parts: the application network & the `RaftNetwork` trait.
The network capabilities required by this system are broken up into two parts: the `RaftNetwork` trait & the application network.
### `RaftNetwork`
Raft uses the `RaftNetwork` trait for sending Raft RPCs. This trait says nothing about how those requests should be received on the other end. There is a lot of flexibility with this trait. Maybe you want to use [Tonic gRPC](https://github.com/hyperium/tonic), or perhaps some other [HTTP-based protocol](https://github.com/seanmonstar/reqwest). One could use WebSockets, a raw TCP socket, UDP, HTTP3 ... in the end, this depends on the application's needs. Whichever option is chosen, the fundamental requirement is that an implementor of the `RaftNetwork` trait must be able to reliably transmit data over the network.
All of the methods to be implemented are similar in structure. Take this one for example:
```rust
/// Send an AppendEntries RPC to the target Raft node (§5).
async fn append_entries(&self, target: NodeId, rpc: AppendEntriesRequest<D>) -> Result<AppendEntriesResponse>;
```
The implementing type should use the given `NodeId` (just a `u64`) to identify the target Raft node to which the given `rpc` must be sent. For applications using a single Raft cluster, this is quite simple. If using a multi-Raft setup, cluster information could be embedded in the `RaftNetwork` implementing type, and network requests could be enriched with that cluster information before being transmitted over the network to ensure that the receiving server can pass the received `rpc` to the correct Raft cluster.
The excellent [`async_trait`](https://docs.rs/async-trait/) crate is re-exported by this crate to make implementation as easy as possible. Please see the documentation on how to use this macro to creating an async trait implementation.
### Application Network
The main role of the application network, in this context, is to handle client requests and then feed them into Raft. There are a few other important things that it will probably need to do as well, depending on the application's needs, here are a few other common networking roles:
- **discovery:** a component which allows the members of an application cluster (its nodes) to discover and communicate with each other. This is not provided by this crate. There are lots of solutions out there to solve this problem. Applications can build their own discovery system by way of DNS, they could use other systems like etcd or consul. The important thing to note here is that once a peer is discovered, it would be prudent for application nodes to maintain a connection with that peer, as heartbeats are very regular, and building network connections is not free.
The main role of the application network, in this context, is to handle RPCs from Raft peers and client requests coming from application clients, and then feed them into Raft. This is essentially the receiving end of the `RaftNetwork` trait, however this project does not enforce any specific interface on how this is to be implemented. The only requirement is that it work with the `RaftNetwork` trait implementation. There are a few other important things that it will probably need to do as well, depending on the application's needs, here are a few other common networking roles:
- **discovery:** a component which allows the members of an application cluster (its nodes) to discover and communicate with each other. This is not provided by this crate. There are lots of solutions out there to solve this problem. Applications can build their own discovery system by way of DNS, they could use other systems like etcd or consul. The important thing to note here is that once a peer is discovered, it would be prudent for application nodes to maintain a connection with that peer, as heartbeats are very regular, and building new network connections is not free.
- **data format:** the way that data is serialized and sent accross the networking medium. Popular data formats include protobuf, capnproto, flatbuffers, message pack, JSON &c. Applications are responsible for serializing and deserializing the various message types used in this crate for network transmission. Serde is used throughout this system to aid on this front.
Applications must be able to facilitate message exchange between nodes reliably.
### `trait RaftNetwork`
This trait defines the requirement of an application's ability to send and receive Raft RPCs.
```rust
pub trait RaftNetwork<D>
where
D: AppData,
Self: Actor<Context=Context<Self>>,
Self: Handler<AppendEntriesRequest<D>>,
Self::Context: ToEnvelope<Self, AppendEntriesRequest<D>>,
Self: Handler<InstallSnapshotRequest>,
Self::Context: ToEnvelope<Self, InstallSnapshotRequest>,
Self: Handler<VoteRequest>,
Self::Context: ToEnvelope<Self, VoteRequest>,
{}
```
Stated simply, all this trait requires is that the implementing type be an Actix [`Actor`](https://docs.rs/actix/latest/actix/trait.Actor.html) & that it implement handlers for the following message types:
- `AppendEntriesRequest`
- `InstallSnapshotRequest`
- `VoteRequest`
The type used to implement `RaftNetwork` could be the same type used to provide the other networking capabilities of an application, or it could be an independent type. The requirement is that the implementing type must be able to transmit the RPCs it receives on its handlers to the target Raft nodes identified in the RPCs. This trait is used directly by the `Raft` actor to send heartbeats to other nodes in the Raft cluster to maintain leadership, replicate entries, request votes when an election takes place, and to install snapshots.
----
Now that we've got a solid taste for the network requirements, the next logic topic to understand is [Raft storage](https://railgun-rs.github.io/actix-raft/storage.html).
Now that we've got a solid taste for the network requirements, let's move on to Raft storage.

View File

@ -0,0 +1,131 @@
Putting It All Together
=======================
In previous chapters we've seen how to define our application's data types which will be used for interacting with Raft, we've seen how to implement the `RaftNetwork` trait, we've seen how to implement the `RaftStorage` trait, and we've reviewed the `Raft` API itself. Now its time to put all of these components together. Let's do this.
For this chapter, we're going to use snippets of the code found in the `memstore` crate, which is an in-memory implementation of the `RaftStorage` trait for demo and testing purposes, which also happens to be used for all of the integration tests of `async-raft` itself.
### Recap On Our Data Types
As we've seen earlier, here are our `AppData` and `AppDataResponse` types/impls.
```rust
/// The application data request type which the `MemStore` works with.
///
/// Conceptually, for demo purposes, this represents an update to a client's status info,
/// returning the previously recorded status.
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct ClientRequest {
/// The ID of the client which has sent the request.
pub client: String,
/// The serial number of this request.
pub serial: u64,
/// A string describing the status of the client. For a real application, this should probably
/// be an enum representing all of the various types of requests / operations which a client
/// can perform.
pub status: String,
}
impl AppData for ClientRequest {}
/// The application data response type which the `MemStore` works with.
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct ClientResponse(Result<Option<String>, ClientError>);
impl AppDataResponse for ClientResponse {}
```
### RaftNetwork impl
We've already discussed the `RaftNetwork` trait in a previous chapter. Here is an abbreviated snippet of what the `RaftNetwork` impl looks like in the `async-raft` integration test suite.
```rust
// We use anyhow::Result in our impl below.
use anyhow::Result;
/// A type which emulates a network transport and implements the `RaftNetwork` trait.
pub struct RaftRouter {
// ... some internal state ...
}
#[async_trait]
impl RaftNetwork<ClientRequest> for RaftRouter {
/// Send an AppendEntries RPC to the target Raft node (§5).
async fn append_entries(&self, target: u64, rpc: AppendEntriesRequest<ClientRequest>) -> Result<AppendEntriesResponse> {
// ... snip ...
}
/// Send an InstallSnapshot RPC to the target Raft node (§7).
async fn install_snapshot(&self, target: u64, rpc: InstallSnapshotRequest) -> Result<InstallSnapshotResponse> {
// ... snip ...
}
/// Send a RequestVote RPC to the target Raft node (§5).
async fn vote(&self, target: u64, rpc: VoteRequest) -> Result<VoteResponse> {
// ... snip ...
}
}
```
### RaftStorage impl
We've already got a `RaftStorage` impl to work with from the `memstore` crate. Here is an abbreviated snippet of the code.
```rust
// We use anyhow::Result in our impl below.
use anyhow::Result;
#[async_trait]
impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
type Snapshot = Cursor<Vec<u8>>;
async fn get_membership_config(&self) -> Result<MembershipConfig> {
// ... snip ...
}
async fn get_initial_state(&self) -> Result<InitialState> {
// ... snip ...
}
// The remainder of our methods are implemented below.
// ... snip ...
}
```
### Raft Type Alias
For better readability in your application's code, it would be beneficial to define a type alias which fully qualifies all of the types which your Raft instance will be using. This is quite simple. The example below is taken directly from this project's integration test suite, which uses the `memstore` crate and a specialized `RaftNetwork` impl designed specifically for testing.
```rust
/// A concrete Raft type used during testing.
pub type MemRaft = Raft<ClientRequest, ClientResponse, RaftRouter, MemStore>;
```
### Give It The Boot
Though applications will be much more complex than this contrived example, booting a Raft node is dead simple. Even if your application uses a multi-Raft pattern for managing different segments / shards of data, the same principal applies. Boot a Raft node, and retain its instance for API usage.
```rust
//! This code assumes the code samples above.
#[tokio::main]
async fn main() {
// Get our node's ID from stable storage.
let node_id = get_id_from_storage().await;
// Build our Raft runtime config, then instantiate our
// RaftNetwork & RaftStorage impls.
let config = Arc::new(Config::build("primary-raft-group".into())
.validate()
.expect("failed to build Raft config"));
let network = Arc::new(RaftRouter::new(config.clone()));
let storage = Arc::new(MemStore::new(node_id));
// Create a new Raft node, which spawns an async task which
// runs the Raft core logic. Keep this Raft instance around
// for calling API methods based on events in your app.
let raft = Raft::new(node_id, config, network, storage);
run_app(raft).await; // This is subjective. Do it your own way.
// Just run your app, feeding Raft & client
// RPCs into the Raft node as they arrive.
}
```
----
You've officially ascended to the next level of AWESOME! Next, let's take a look at cluster lifecycle controls, dynamic membership, and the like.

View File

@ -1,54 +1,56 @@
Raft
====
The central most type of this crate is the `Raft` type. It is a highly generic actor with the signature:
Raft API
========
The `Raft` type represents the singular API of this crate, and is the interface to a running Raft node. It is highly generic, which allows your application's data types to be known at compile, for maximum performance and type-safety. Users of this Raft implementation get to choose the exact types to be used throughout the system, and get to work with their application's data types directly without the overhead of serializing and deserializing the data as it moves through the `Raft` system.
```
Raft<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D>, S: RaftStorage<D, R, E>>
```
In previous chapters, we've defined our `AppData`, `AppDataResponse`, `RaftNetwork` and `RaftStorage` types. These four types are used as part of a concrete `Raft` definition, and applications may find it beneficial to define an alias covering all of these types for easier reference. Something like the following:
The generics here allow `Raft` to use statically known types, defined in the parent application using this crate, for maximum performance and type-safety. Users of this Raft implementation get to choose the exact types they want to use for application specific error handling coming from the storage layer, and also get to work with their application's data types directly without the overhead of serializing and deserializing the data as it moves through the `Raft` system.
```rust
/// Your Raft type alias.
type YourRaft = Raft<YourData, YourDataResponse, YourRaftNetwork, YourRaftStorage>;
```
### API
As the `Raft` type is an Actix [`Actor`](https://docs.rs/actix/latest/actix/trait.Actor.html), all interaction with `Raft` is handled via message passing. All pertinent message types derive the serde traits for easier integration with other data serialization formats in the Rust ecosystem, providing maximum flexibility for applications using this crate.
The API of the `Raft` type is broken up into 4 sections: Client Requests, Raft RPCs, Admin Commands & Utility Methods.
All message types are sent to a `Raft` node via the actor's [`Addr`](https://docs.rs/actix/latest/actix/struct.Addr.html). Applications using this crate are expected to have networking capabilities for cluster communication & client interaction. Applications are responsible for handling client requests & Raft RPCs coming from their network layer, and must send them to the `Raft` actor returning the response. More details on this topic can be found in the [network chapter](https://railgun-rs.github.io/actix-raft/network.html).
#### Client Requests
The application level interface for clients is 100% at the discression of the application being built. However, once a client read or write operation is ready to be processed, the below methods provide the read/write functionality for Raft interaction.
The public API of the `Raft` type is broken up into 3 sections: Client Requests, Raft RPCs & Admin Commands.
- [`async fn client_read(...) -> Result<...>`](TODO:): Check to ensure this node is still the cluster leader, in order to guard against stale reads. The actual read operation itself is up to the application, this method just ensures that the read will not be stale.
- [`async fn client_write(...) -> Result<...>`](TODO:): Submit a mutating client request to Raft to update the state of the system (§5.1). It will be appended to the log, committed to the cluster, and then applied to the application state machine. The result of applying the request to the state machine will be returned as the response from this method.
##### Client Requests
- [ClientPayload](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.ClientPayload.html): a payload of data which needs to be committed to the Raft cluster. Typically, this will be data coming from application clients.
#### Raft RPCs
These methods directly correspond to the `RaftNetwork` trait described in earlier chapters. The application is responsible for implementing its own network layer which can receive these RPCs coming from Raft peers, and should then pass them into the Raft node using the following methods.
##### Raft RPCs
- [AppendEntriesRequest](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.AppendEntriesRequest.html): An RPC invoked by the leader to replicate log entries (§5.3); also used as heartbeat (§5.2).
- [VoteRequest](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.VoteRequest.html): An RPC invoked by candidates to gather votes (§5.2).
- [InstallSnapshotRequest](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.InstallSnapshotRequest.html): Invoked by the Raft leader to send chunks of a snapshot to a follower (§7).
- [`async fn append_entries(...) -> Result<...>`](TODO:): An RPC invoked by the leader to replicate log entries (§5.3); also used as heartbeat (§5.2).
- [`async fn vote(...) -> Result<...>`](TODO:): An RPC invoked by candidates to gather votes (§5.2).
- [`async fn install_snapshot(...) -> Result<...>`](TODO:): Invoked by the Raft leader to send chunks of a snapshot to a follower (§7).
##### Admin Commands
- [InitWithConfig](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.InitWithConfig.html): Initialize a pristine Raft node with the given config & start a campaign to become leader.
- [ProposeConfigChange](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.ProposeConfigChange.html): Propose a new membership config change to a running cluster.
#### Admin Commands
All of these methods are intended for use directly by the parent application for managing various lifecycles of the cluster. Each of these lifecycles are discussed in more detail in the [Cluster Controls](TODO:) chapter.
- [`async fn initialize(...) -> Result<...>`](TODO:): Initialize a pristine Raft node with the given config & start a campaign to become leader.
- [`async fn add_non_voter(...) -> Result<...>`](TODO:): Add a new node to the cluster as a non-voter, which will sync the node with the master so that it can later join the cluster as a voting member.
- [`async fn change_membership(...) -> Result<...>`](TODO:): Propose a new membership config change to a running cluster.
### client requests diagram
The following diagram shows how client requests are presented to Raft from within an application, how the data is stored, replicated and ultimately applied to the application's state machine.
#### Utility Methods
- [`fn metrics(&self) -> watch::Receiver<RaftMetrics>`](TODO:): Get a stream of all metrics coming from the Raft node.
- [`fn shutdown(self) -> tokio::task::JoinHandle<RaftResult<()>>`](TODO:): Send a shutdown signal to the Raft node, and get a `JoinHandle` which can be used to await the full shutdown of the node. If the node is already in shutdown, this routine will allow you to await its full shutdown.
<p>
<img src="./images/raft-workflow-client-requests.png"/>
</p>
### Reading & Writing Data
What does the Raft spec have to say about reading and writing data?
The numbered elements represent segments of the workflow.
1. The parent application has received a client request, and presents the payload to `Raft` using the `ClientPayload` type.
2. `Raft` will present the payload to the `RaftStorage` impl via the `AppendEntryToLog` type. This is the one location where the `RaftStorage` impl may return an application specific error. This could be for validation logic, enforcing unique indices, data/schema validation; whatever application level rules the application enforces, this is where they should be enforced. Close to the data, just before it hits the `Raft` log.
3. The `RaftStorage` impl responds to the `Raft` actor. If it is successful, go to step 4, else the error response will be sent back to the caller immediately. The error response is a statically known type defined by the parent application.
4. `Raft` uses the `RaftNetwork` impl to communicate with the peers of the cluster.
5. `Raft` uses the `RaftNetwork` impl to replicate the entry to all other nodes in the cluster.
6. Follower nodes in the cluster respond upon successful replication.
7. Once the entry has been replicated to a majority of nodes in the cluster — known as a "committed" entry in the Raft spec — it is ready to be applied to the application's state machine.
8. `Raft` will apply the entry to the application's state machine via the `ApplyEntryToStateMachine` type.
9. The `RaftStorage` impl responds to the `Raft` actor.
10. The success response is returned to the caller.
> Clients of Raft send all of their requests to the leader. When a client first starts up, it connects to a randomly-chosen server. If the clients first choice is not the leader, that server will reject the clients request and supply information about the most recent leader it has heard from. If the leader crashes, client requests will timeout; clients then try again with randomly-chosen servers.
**NOTE:** this implementation of Raft offers the option for client requests to receive a response once its entry has been committed, and before it is applied to the state machine. This is controlled by the `ClientPayload.response_type` field, which is an instance of the `ResponseMode` enum which may be either `Committed` or `Applied`. Application's may use either depending on their needs.
The `Raft.metrics` method, discussed above, provides a stream of data on the Raft node's internals, and should be used in order to determine the cluster leader, which should only need to be performed once when the client connection is first established.
> Our goal for Raft is to implement linearizable semantics (each operation appears to execute instantaneously, exactly once, at some point between its invocation and its response). [...] if the leader crashes after committing the log entry but before responding to the client, the client [may] retry the command with a new leader, causing it to be executed a second time. The solution is for clients to assign unique serial numbers to every command. Then, the state machine tracks the latest serial number processed for each client, along with the associated response. If it receives a command whose serial number has already been executed, it responds immediately without re-executing the request.
As described in the quote above, applications will need to have their clients assign unique serial numbers to every command sent to the application servers. Then, within the application specific code implemented inside of `RaftStorage::apply_entry_to_state_machine`, if the application detects that the serial number has already been executed for the requesting client, then the response should be immediately returned without re-executing the request. Much of this will be application specific, but these principals can help with design.
> Read-only operations can be handled without writing anything into the log. However, with no additional measures, this would run the risk of returning stale data, since the leader responding to the request might have been superseded by a newer leader of which it is unaware. [...] a leader must check whether it has been deposed before processing a read-only request (its information may be stale if a more recent leader has been elected). Raft handles this by having the leader exchange heartbeat messages with a majority of the cluster before responding to read-only requests.
The `Raft.client_read` method should be used to ensure that the callee Raft node is still the cluster leader.
----
The API is simple enough, but there is more to learn about `Raft` than just feeding it messages. The next logical topic to understand is [Raft networking](https://railgun-rs.github.io/actix-raft/network.html).
The API is simple enough, now its time to put everything together.

View File

@ -1,143 +1,22 @@
Storage
=======
The way that data is stored and represented is an integral part of every data storage system. Whether it is a SQL or NoSQL database, a columner store, a KV store, or anything which stores data, control over the storage technology and technique is critical.
The way that data is stored and represented is an integral part of every data storage system. Whether it is a SQL or NoSQL database, a KV store, an AMQP / Streaming / Eventing system, a Graph database, or anything which stores data — control over the storage technology and technique is critical. This implementation of Raft uses the `RaftStorage` trait to define the behavior needed of an application's storage layer to work with Raft.
This implementation of Raft uses the `RaftStorage` trait to define the behavior needed of an application's storage layer to work with Raft. This is definitely the most complex looking trait in this crate. Ultimately the implementing type must be an Actix [`Actor`](https://docs.rs/actix/latest/actix) and it must implement handlers for a specific set of message types.
### implementation
There are a few important decisions which need to be made in order to implement the `RaftStorage` trait.
When creatinga new `RaftStorage` instance, it would be logical to supply the ID of the parent Raft node as well as the node's snapshot directory. Such information is needed when booting a node for the first time.
1. **How do you plan on storing your snapshots?** The `RaftStorage::Snapshot` associated type must declare the type your application uses for dealing with the raw bytes of a snapshot. For most applications, it stands to reason that a simple on-disk file is what will be used. As such, take a look at [Tokio's fs::File](https://docs.rs/tokio/latest/tokio/fs/struct.File.html). It satisfies all of the trait bounds for the `Snapshot` associated type.
2. **How do you plan on storing your data?** A majority of the methods of your `RaftStorage` impl will involve reading and writing data. Rust has a few data storage crates available to choose from which will satisfy these requirements. Have a look at [Sled](https://docs.rs/sled/latest/sled/), or [RocksDB](https://docs.rs/rocksdb/latest/rocksdb/). There are others to choose from, but these may be a solid starting point. Or you could always roll your own.
```rust
pub trait RaftStorage<D, R, E>: 'static
where
D: AppData,
R: AppDataResponse,
E: AppError,
{
/// The type to use as the storage actor. Should just be Self.
type Actor: Actor<Context=Self::Context> +
Handler<GetInitialState<E>> +
Handler<SaveHardState<E>> +
Handler<GetLogEntries<D, E>> +
Handler<AppendEntryToLog<D, E>> +
Handler<ReplicateToLog<D, E>> +
Handler<ApplyEntryToStateMachine<D, R, E>> +
Handler<ReplicateToStateMachine<D, E>> +
Handler<CreateSnapshot<E>> +
Handler<InstallSnapshot<E>> +
Handler<GetCurrentSnapshot<E>>;
Once you're ready to begin with your implementation, be sure to adhere to the documentation of the `RaftStorage` methods themselves. There are plenty of data safety requirements to uphold in order for your application to work properly overall, and to work properly with Raft.
/// The type to use as the storage actor's context. Should be `Context<Self>` or `SyncContext<Self>`.
type Context: ActorContext +
ToEnvelope<Self::Actor, GetInitialState<E>> +
ToEnvelope<Self::Actor, SaveHardState<E>> +
ToEnvelope<Self::Actor, GetLogEntries<D, E>> +
ToEnvelope<Self::Actor, AppendEntryToLog<D, E>> +
ToEnvelope<Self::Actor, ReplicateToLog<D, E>> +
ToEnvelope<Self::Actor, ApplyEntryToStateMachine<D, R, E>> +
ToEnvelope<Self::Actor, ReplicateToStateMachine<D, E>> +
ToEnvelope<Self::Actor, CreateSnapshot<E>> +
ToEnvelope<Self::Actor, InstallSnapshot<E>> +
ToEnvelope<Self::Actor, GetCurrentSnapshot<E>>;
}
```
For inspiration, have a look at this [repo's `memstore` project](https://github.com/railgun-rs/actix-raft/tree/master/memstore). It is an in-memory implementation of the `RaftStorage` trait, intended for demo and testing purposes.
Actix handlers must be implemented for the following types, all of which are found in the `storage` module:
- [GetInitialState](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.GetInitialState.html): A request from Raft to get Raft's state information from storage.
- [SaveHardState](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.SaveHardState.html): A request from Raft to save its HardState.
- [GetLogEntries](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.GetLogEntries.html): A request from Raft to get a series of log entries from storage.
- [AppendEntryToLog](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.AppendEntryToLog.html): A request from Raft to append a new entry to the log.
- [ReplicateToLog](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.ReplicateToLog.html): A request from Raft to replicate a payload of entries to the log.
- [ApplyEntryToStateMachine](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.ApplyEntryToStateMachine.html): A request from Raft to apply the given log entry to the state machine.
- [ReplicateToStateMachine](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.ReplicateToStateMachine.html): A request from Raft to apply the given log entries to the state machine, as part of replication.
- [CreateSnapshot](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.CreateSnapshot.html): A request from Raft to have a new snapshot created which covers the current breadth of the log.
- [InstallSnapshot](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.InstallSnapshot.html): A request from Raft to have a new snapshot written to disk and installed.
- [GetCurrentSnapshot](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.GetCurrentSnapshot.html): A request from Raft to get metadata of the current snapshot.
### compaction / snapshots
This implementation of Raft automatically triggers log compaction based on runtime configuration, using the `RaftStorage::do_log_compaction` method. Additionally, the Raft leader may stream a snapshot over to other nodes if the node is new and needs to be brought up-to-speed, or if a node is lagging behind.
The following sections detail how to implement a safe and correct storage system for Raft using the `RaftStorage` trait. A very important note to keep in mind: data storage, data layout, data representation ... all of that is up to the implementor of the `RaftStorage` trait. That's the whole point. Every application is going to have nuances in terms of what they need to do at the storage layer. This is one of the primary locations where an application can innovate and differentiate.
### state
This pertains to implementing the `GetInitialState` & `SaveHardState` handlers.
##### `GetInitialState`
When the storage system comes online, it should check for any state currently on disk. Based on how the storage layer is persisting data, it may have to look in a few locations to get all of the needed data. Once the [`InitialState`](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.InitialState.html) data has been collected, respond.
##### `SaveHardState`
This handler will be called periodically based on different events happening in Raft. Primarily, membership changes and elections will cause this to be called. Implementation is simple. Persist the data in the given [`HardState`](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.HardState.html) to disk, ensure that it can be accurately retrieved even after a node failure, and respond.
### log & state machine
This pertains to implementing the `GetLogEntries`, `AppendEntryToLog`, `ReplicateToLog`, `ApplyEntryToStateMachine` & `ReplicateToStateMachine` handlers.
Traditionally, there are a few different terms used to refer to the log of mutations which are to be applied to a data storage system. Write-ahead log (WAL), op-log, there are a few different terms, sometimes with different nuances. In Raft, this is known simply as the log. A log entry describes the "type" of mutation to be applied to the state machine, and the state machine is the actual business-logic representation of all applied log entries.
##### `GetLogEntries`
This will be called at various times to fetch a range of entries from the log. The `start` field is inclusive, the `stop` field is non-inclusive. Simply fetch the specified range of logs from the storage medium, and return them.
##### `AppendEntryToLog`
Called as the direct result of a client request and will only be called on the Raft leader node. **THIS IS THE ONE AND ONLY** `RaftStorage` handler which is allowed to return errors which will not cause the Raft node to terminate. Reveiw the docs on the [`AppendEntryToLog`](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.AppendEntryToLog.html) type, and you will see that its message response type is the `AppError` type, which is a statically known error type chosen by the implementor (which was reviewed earlier in the [raft overview chapter](https://railgun-rs.github.io/actix-raft/raft.html)).
This is where an application may enforce business-logic rules, such as unique indices, relational constraints, type validation, whatever is needed by the application. If everything checks out, insert the entry at its specified index in the log. **Don't just blindly append,** use the entry's index. There are times when log entries must be overwritten, and Raft guarantees the safety of such operations.
**Another very important note:** per the Raft spec in §8, to ensure that client requests are not applied > 1 due to a failure scenario and the client issuing a retry, the Raft spec recommends that applications track client IDs and use serial numbers on each request. This handler may then use that information to reject duplicate request using an application specific error. The application's client may observe this error and treat it as an overall success. This is an application level responsibility, Raft simply provides the mechanism to be able to implement it.
##### `ReplicateToLog`
This is similar to `AppendEntryToLog` except that this handler is only called on followers, and they should never perform validation or falible operations. If this handler returns an error, the Raft node will terminate in order to guard against data corruption. As mentioned previously, there are times when log entries must be overwritten. Raft guarantees the safety of these operations. **Use the index of each entry when inserting into the log.**
##### `ApplyEntryToStateMachine`
Once a log entry is known to be committed (it has been replicated to a majority of nodes in the cluster), the leader will call this handler to apply the entry to the application's state machine. Committed entries will never be removed or overwritten in the log, which is why it is safe to apply the entry to the state machine. To implement this handler, apply the contents of the entry to the application's state machine in whatever way is needed. This handler is allowed to return an application specific response type, which allows the application to return arbitrary information about the process of applying the entry.
For example, if building a SQL database, and the entry calls for inserting a new record and the full row of data needs to be returned to the client, this handler may return such data in its response.
Raft, as a protocol, guarantees strict linearizability. Entries will never be re-applied. The only case where data is removed from the state machine is during some cases of snapshotting where the entire state machine needs to be rebuilt. Read on for more details.
**NOTE WELL:** there are times when Raft needs to append blank entries to the log which will end up being applied to the state machine. See §8 for more details. Application's should handle this with a "no-op" variant of their `AppDataResponse` type.
##### `ReplicateToStateMachine`
This is similar to `ApplyEntryToStateMachine` except that this handler is only called on followers as part of replication, and are not allowed to return response data (as there is nothing to return response data to during replication).
### snapshots & log compaction
This pertains to implementing the `CreateSnapshot`, `InstallSnapshot` & `GetCurrentSnapshot`.
The snapshot and log compaction capabilities defined in the Raft spec are fully supported by this implementation. The storage layer is left to the application which uses this Raft implementation, but all snapshot behavior defined in the Raft spec is supported. Additionally, this implemention supports:
- Configurable snapshot policies. This allows nodes to perform log compacation at configurable intervals.
- Leader based `InstallSnapshot` RPC support. This allows the Raft leader to make determinations on when a new member (or a slow member) should receive a snapshot in order to come up-to-date faster.
For clarity, **it is emphasized** that implementing the log compaction & snapshot creation behavior is up to the `RaftStorage` implementor. This guide is here to help, and §7 of the Raft spec is dedicated to the subject.
##### `CreateSnapshot`
This handler is called when the Raft node determines that a snapshot is needed based on the cluster's configured snapshot policy. `Raft` guarantees that this interface will never be called multiple overlapping times, and it will not be called when an `InstallSnapshot` operation is in progress.
**It is critical to note** that the newly created snapshot must be able to be used to completely and accurately create a state machine. In addition to saving space on disk (log compaction), snapshots are used to bring new Raft nodes and slow Raft nodes up-to-speed with the cluster leader.
**implementation algorithm:**
- The generated snapshot should include all log entries starting from entry `0` up through the index specified by `CreateSnapshot.through`. This will include any snapshot which may already exist. If a snapshot does already exist, the new log compaction process should be able to just load the old snapshot first, and resume processing from its last entry.
- The newly generated snapshot should be written to the configured snapshot directory.
- All previous entries in the log should be deleted up to the entry specified at index `through`.
- The entry at index `through` should be replaced with a new entry created from calling [`Entry::new_snapshot_pointer(...)`](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.Entry.html).
- Any old snapshot will no longer have representation in the log, and should be deleted.
- Return a [`CurrentSnapshotData`](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.CurrentSnapshotData.html) struct which contains all metadata pertinent to the snapshot.
##### `InstallSnapshot`
This handler is called when the leader of the Raft cluster has determined that the subject node needs to receive a new snapshot. This is typically the case when new nodes are added to a running cluster, or if a node has gone offline for some amount of time without being removed from the cluster, or the node is VERY slow.
This message holds an `UnboundedReceiver` which will stream in new chunks of data as they are received from the Raft leader. See the docs on the [InstallSnapshotChunk](https://docs.rs/actix-raft/latest/actix_raft/storage/struct.InstallSnapshotChunk.html) for more info.
**implementation algorithm:**
- Upon receiving the request, a new snapshot file should be created on disk.
- Every new chunk of data received should be written to the new snapshot file starting at the `offset` specified in the chunk. Once the chunk has been successfully written, the `InstallSnapshotChunk.cb` (a `oneshot::Sender`) should be called to indicate that the storage engine has finished writing the chunk.
- If the receiver is dropped, the snapshot which was being created should be removed from disk, and a success response should be returned.
Once a chunk is received which is the final chunk of the snapshot (`InstallSnapshotChunk.done`), after writing the chunk's data, there are a few important steps to take:
- Create a new entry in the log via the [`Entry::new_snapshot_pointer(...)`](https://docs.rs/actix-raft/latest/actix_raft/messages/struct.Entry.html) constructor. Insert the new entry into the log at the specified `index` of the original `InstallSnapshot` payload.
- If there are any logs older than `index`, remove them.
- If there are any other snapshots in the configured snapshot dir, remove them.
- If an existing log entry has same index and term as snapshot's last included entry, retain log entries following it, then return.
- Else, discard the entire log leaving only the new snapshot pointer. **The state machine must be rebuilt from the new snapshot.** Return once the state machine has been brought up-to-date.
##### `GetCurrentSnapshot`
A request to get information on the current snapshot. `RaftStorage` implementations must take care to ensure that there is only ever one active snapshot, old snapshots should be deleted as part of `CreateSnapshot` and `InstallSnapshot` requests, and the snapshot information should be able to be retrieved efficiently. Having to load and parse the entire snapshot on each `GetCurrentSnapshot` request may not be such a great idea! Snapshots can be quite large.
Compaction / snapshotting are not optional in this system. It is an integral component of the Raft spec, and `RaftStorage` implementations should be careful to implement the compaction / snapshotting related methods carefully according to the trait's documentation.
----
Woot woot! Made it through the hard part! There is more to learn, so let's keep going.
There is more to learn, so let's keep going. Time to learn about the most central API of this project.

View File

@ -15,3 +15,9 @@ thiserror = "1.0.20"
tokio = { version="0.2.22", default-features=false, features=["sync"] }
tracing = "0.1.17"
tracing-futures = "0.2.4"
[features]
docinclude = [] # Used only for activating `doc(include="...")` on nightly.
[package.metadata.docs.rs]
features = ["docinclude"] # Activate `docinclude` during docs.rs build.

View File

@ -1,3 +1,22 @@
memstore
========
TODO: fill this out.
<h1 align="center">memstore</h1>
<div align="center">
<strong>
An in-memory storage system implementing the <code>async_raft::RaftStorage</code> trait. Please ⭐ on <a href="https://github.com/railgun-rs/actix-raft">github</a>!
</strong>
</div>
<br />
<div align="center">
[![Build Status](https://github.com/railgun-rs/actix-raft/workflows/ci/badge.svg?branch=async-raft)](https://travis-ci.com/railgun-rs/actix-raft)
[![Crates.io](https://img.shields.io/crates/v/actix-raft.svg)](https://crates.io/crates/actix-raft)
[![docs.rs](https://docs.rs/actix-raft/badge.svg)](https://docs.rs/actix-raft)
[![License](https://img.shields.io/badge/license-MIT%2FApache--2.0-blue)](LICENSE)
![Crates.io](https://img.shields.io/crates/d/actix-raft.svg)
![Crates.io](https://img.shields.io/crates/dv/actix-raft.svg)
[![GitHub issues open](https://img.shields.io/github/issues-raw/railgun-rs/actix-raft.svg)]()
[![GitHub issues closed](https://img.shields.io/github/issues-closed-raw/railgun-rs/actix-raft.svg)]()
</div>
</br>
[The guide](https://railgun-rs.github.io/actix-raft) is the best place to get started, followed by [the docs](https://docs.rs/actix-raft/latest/actix_raft/) for more in-depth details.

View File

@ -1,3 +1,6 @@
#![cfg_attr(feature="docinclude", feature(external_doc))]
#![cfg_attr(feature="docinclude", doc(include="../README.md"))]
#[cfg(test)]
mod test;
@ -5,7 +8,7 @@ use std::collections::{BTreeMap, HashMap};
use std::io::Cursor;
use anyhow::Result;
use async_raft::async_trait;
use async_raft::async_trait::async_trait;
use async_raft::{AppData, AppDataResponse, NodeId, RaftStorage};
use async_raft::raft::{Entry, EntryPayload, MembershipConfig};
use async_raft::storage::{CurrentSnapshotData, HardState, InitialState};
@ -15,9 +18,10 @@ use tokio::sync::{RwLockReadGuard, RwLockWriteGuard};
const ERR_INCONSISTENT_LOG: &str = "a query was received which was expecting data to be in place which does not exist in the log";
/// A request to update a client's status info, returning the last recorded status.
/// The application data request type which the `MemStore` works with.
///
/// This is the application data type which the `MemStore` works with.
/// Conceptually, for demo purposes, this represents an update to a client's status info,
/// returning the previously recorded status.
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct ClientRequest {
/// The ID of the client which has sent the request.
@ -69,7 +73,7 @@ pub struct MemStoreStateMachine {
pub client_status: HashMap<String, String>,
}
/// An in-memory storage system for demo and testing purposes related to `async-raft`.
/// An in-memory storage system implementing the `async_raft::RaftStorage` trait.
pub struct MemStore {
/// The ID of the Raft node for which this memory storage instances is configured.
id: NodeId,