Updates to joint consensus, membership & Raft communication layer.

The joint consensus system has been updated significantly. Per some
community feedback and discussion, the old pattern was not spec
compliant and could lead to Raft safety violations. It has been updated
to conform to a more direct and simple implementation of the joint
consensus system. The non-voter sync stage now happens before joint
consensus is ever entered.

Refactor of Raft<->RaftCore communication layer. Using a single channel
with an enum message type. This keeps the code footprint a bit smaller
and will be easier to make updates in the future.

Voting & commitment during joint consensus have also been updated to
follow the Raft spec more precisely.

Added initial github actions.
This commit is contained in:
Anthony Dodd 2020-07-28 22:04:20 -05:00
parent aa92dbc889
commit 99ac1911f3
No known key found for this signature in database
GPG Key ID: 6E0613E0F653DBC0
18 changed files with 715 additions and 582 deletions

42
.github/workflows/ci.yaml vendored Normal file
View File

@ -0,0 +1,42 @@
on: [push]
name: ci
jobs:
build-async-raft:
name: build async-raft
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v2
- uses: actions-rs/toolchain@v1
with:
toolchain: stable
- uses: actions-rs/cargo@v1
with:
command: build
args: -p async-raft --release
build-async-raft-nightly:
name: build async-raft nightly
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v2
- uses: actions-rs/toolchain@v1
with:
toolchain: nightly
override: true
- uses: actions-rs/cargo@v1
with:
command: build
args: -p async-raft --release --all-features
build-memstore:
name: build memstore
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v2
- uses: actions-rs/toolchain@v1
with:
toolchain: stable
- uses: actions-rs/cargo@v1
with:
command: build
args: -p memstore --release

5
Cargo.toml Normal file
View File

@ -0,0 +1,5 @@
[workspace]
members = [
"async-raft",
"memstore",
]

View File

@ -1,5 +1,12 @@
async raft
==========
<h1 align="center">async raft</h1>
<div align="center">
<strong>
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 />
<div align="center">
[![Build Status](https://travis-ci.com/railgun-rs/actix-raft.svg?branch=master)](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)
@ -9,7 +16,10 @@ async raft
[![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)]()
An implementation of the [Raft distributed consensus protocol](https://raft.github.io/) using [the Tokio framework](https://tokio.rs/). 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). Please ⭐ on [github](https://github.com/railgun-rs/actix-raft)!
</div>
</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).
[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,124 +1,154 @@
use futures::future::{Future, FutureExt, TryFutureExt};
use std::collections::HashSet;
use futures::future::{FutureExt, TryFutureExt};
use tokio::sync::oneshot;
use crate::{AppData, AppDataResponse, AppError, RaftNetwork, RaftStorage};
use crate::error::{InitWithConfigError, ProposeConfigChangeError, RaftError};
use crate::raft::{ClientRequest, ClientResponse, InitWithConfig, MembershipConfig, ProposeConfigChange};
use crate::core::{ConsensusState, LeaderState, NonVoterState, ReplicationState, TargetState, UpdateCurrentLeader};
use crate::{AppData, AppDataResponse, AppError, NodeId, RaftNetwork, RaftStorage};
use crate::error::{InitializeError, ChangeConfigError, RaftError};
use crate::raft::{ChangeMembershipTx, ClientRequest, ClientResponse, MembershipConfig};
use crate::core::{ConsensusState, LeaderState, NonVoterReplicationState, NonVoterState, State, UpdateCurrentLeader};
use crate::core::client::ClientRequestEntry;
use crate::metrics::State;
use crate::replication::{RaftEvent, ReplicationStream};
use crate::replication::RaftEvent;
impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> NonVoterState<'a, D, R, E, N, S> {
/// Handle the admin `init_with_config` command.
#[tracing::instrument(level="debug", skip(self, req))]
pub(super) async fn handle_init_with_config(&mut self, mut req: InitWithConfig) -> Result<(), InitWithConfigError<E>> {
#[tracing::instrument(level="debug", skip(self))]
pub(super) async fn handle_init_with_config(&mut self, mut members: HashSet<NodeId>) -> Result<(), InitializeError<E>> {
if self.core.last_log_index != 0 || self.core.current_term != 0 {
tracing::error!({self.core.last_log_index, self.core.current_term}, "rejecting init_with_config request as last_log_index or current_term is 0");
return Err(InitWithConfigError::NotAllowed);
return Err(InitializeError::NotAllowed);
}
// Ensure given config is normalized and ready for use in the cluster.
req = normalize_init_config(req);
if !req.members.contains(&self.core.id) {
req.members.push(self.core.id);
// Ensure given config contains this nodes ID as well.
if !members.contains(&self.core.id) {
members.insert(self.core.id);
}
// Build a new membership config from given init data & assign it as the new cluster
// membership config in memory only.
self.core.membership = MembershipConfig{is_in_joint_consensus: false, members: req.members, non_voters: vec![], removing: vec![]};
self.core.membership = MembershipConfig{members, members_after_consensus: None};
// Become a candidate and start campaigning for leadership. If this node is the only node
// in the cluster, then become leader without holding an election.
if self.core.membership.members.len() == 1 && &self.core.membership.members[0] == &self.core.id {
// in the cluster, then become leader without holding an election. If members len == 1, we
// know it is our ID due to the above code where we ensure our own ID is present.
if self.core.membership.members.len() == 1 {
self.core.current_term += 1;
self.core.voted_for = Some(self.core.id);
self.core.set_target_state(TargetState::Leader);
self.core.set_target_state(State::Leader);
self.core.save_hard_state().await?;
} else {
self.core.set_target_state(TargetState::Candidate);
self.core.set_target_state(State::Candidate);
}
Ok(())
}
}
impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> LeaderState<'a, D, R, E, N, S> {
/// An admin message handler invoked to trigger dynamic cluster configuration changes. See §6.
#[tracing::instrument(level="debug", skip(self, msg))]
pub(super) async fn handle_propose_config_change(
&mut self, msg: ProposeConfigChange,
) -> Result<impl Future<Output=Result<(), ProposeConfigChangeError<E>>> + Send + Sync + 'static, ProposeConfigChangeError<E>> {
// Normalize the proposed config to ensure everything is valid.
let mut msg = normalize_proposed_config(msg, &self.core.membership)?;
/// Add a new node to the cluster as a non-voter, bringing it up-to-speed, and then responding
/// on the given channel.
#[tracing::instrument(level="debug", skip(self, tx))]
pub(super) fn add_member(&mut self, target: NodeId, tx: oneshot::Sender<Result<(), ChangeConfigError<E>>>) {
// Ensure the node doesn't already exist in the current config, in the set of new nodes
// alreading being synced, or in the nodes being removed.
if self.core.membership.members.contains(&target)
|| self.core.membership.members_after_consensus.as_ref().map(|new| new.contains(&target)).unwrap_or(false)
|| self.non_voters.contains_key(&target) {
tracing::debug!("target node is already a cluster member or is being synced");
let _ = tx.send(Err(ChangeConfigError::Noop));
return;
}
// Spawn a replication stream for the new member. Track state as a non-voter so that it
// can be updated to be added to the cluster config once it has been brought up-to-date.
let state = self.spawn_replication_stream(target);
self.non_voters.insert(target, NonVoterReplicationState{state, is_ready_to_join: false, tx: Some(tx)});
}
#[tracing::instrument(level="debug", skip(self, tx))]
pub(super) async fn change_membership(&mut self, members: HashSet<NodeId>, tx: ChangeMembershipTx<E>) {
// Ensure cluster will have at least one node.
if members.is_empty() {
let _ = tx.send(Err(ChangeConfigError::InoperableConfig));
return;
}
// Only allow config updates when currently in a uniform consensus state.
match &mut self.consensus_state {
ConsensusState::Joint{..} => return Err(ProposeConfigChangeError::AlreadyInJointConsensus),
_ => self.consensus_state = ConsensusState::Joint{
new_nodes_being_synced: msg.add_members.clone(),
is_committed: false,
match &self.consensus_state {
ConsensusState::Uniform => (),
ConsensusState::NonVoterSync{..} | ConsensusState::Joint{..} => {
let _ = tx.send(Err(ChangeConfigError::ConfigChangeInProgress));
return;
},
}
// Update current config.
self.core.membership.is_in_joint_consensus = true;
self.core.membership.non_voters.append(&mut msg.add_members);
self.core.membership.removing.append(&mut msg.remove_members);
// Spawn new replication streams for new members. Track state as non voters so that they
// can be updated to be normal members once all non-voters have been brought up-to-date.
for target in msg.add_members {
// Build & spawn a replication stream for the target member.
tracing::debug!({target}, "spawning replication stream");
let replstream = ReplicationStream::new(
self.core.id, target, self.core.current_term, self.core.config.clone(),
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(),
);
let state = ReplicationState{match_index: self.core.last_log_index, is_at_line_rate: true, replstream, remove_after_commit: None};
self.nodes.insert(target, state);
}
// For any nodes being removed which are currently non-voters, immediately remove them.
for node in msg.remove_members {
tracing::debug!({target=node}, "removing target node from replication pool");
if let Some((idx, _)) = self.core.membership.non_voters.iter().enumerate().find(|(_, e)| *e == &node) {
if let Some(node) = self.nodes.remove(&node) {
let _ = node.replstream.repltx.send(RaftEvent::Terminate);
// Check the proposed config for any new nodes. If ALL new nodes already have replication
// streams AND are ready to join, then we can immediately proceed with entering joint
// consensus. Else, new nodes need to first be brought up-to-speed.
//
// Here, all we do is check to see which nodes still need to be synced, which determines
// we can proceed.
let diff = members.difference(&self.core.membership.members).cloned().collect::<Vec<_>>();
let awaiting = diff.into_iter()
.filter(|new_node| match self.non_voters.get(&new_node) {
Some(node) if node.is_ready_to_join => false,
Some(_) => true,
None => {
// Spawn a replication stream for the new member. Track state as a non-voter so that it
// can be updated to be added to the cluster config once it has been brought up-to-date.
let state = self.spawn_replication_stream(*new_node);
self.non_voters.insert(*new_node, NonVoterReplicationState{state, is_ready_to_join: false, tx: None});
true
}
self.core.membership.non_voters.remove(idx);
}
})
.collect::<HashSet<_>>();
// If there are new nodes which need to sync, then we need to wait until they are synced.
// Once they've finished, this routine will be called again to progress further.
if !awaiting.is_empty() {
self.consensus_state = ConsensusState::NonVoterSync{awaiting, members, tx};
return;
}
self.core.report_metrics(State::Leader);
// Enter into joint consensus if we are not awaiting any new nodes.
if !members.contains(&self.core.id) {
self.is_stepping_down = true;
}
self.consensus_state = ConsensusState::Joint{is_committed: false};
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 (tx_joint, rx_join) = oneshot::channel();
let entry = self.append_payload_to_log(payload.entry).await?;
let entry = match self.append_payload_to_log(payload.entry).await {
Ok(entry) => entry,
Err(err) => {
let _ = tx.send(Err(err.into()));
return;
}
};
let cr_entry = ClientRequestEntry::from_entry(entry, payload.response_mode, tx_joint);
self.replicate_client_request(cr_entry).await;
self.core.report_metrics();
// Setup channels for eventual response to the 2-phase config change.
let (tx_cfg_change, rx_cfg_change) = oneshot::channel();
self.propose_config_change_cb = Some(tx_cfg_change); // Once the entire process is done, this is our response channel.
self.joint_consensus_cb.push(rx_join); // Receiver for when the joint consensus is committed.
Ok(rx_cfg_change
.map_err(|_| RaftError::ShuttingDown)
.into_future()
.then(|res| futures::future::ready(match res {
Ok(ok) => match ok {
Ok(ok) => Ok(ok),
Err(err) => Err(ProposeConfigChangeError::from(err)),
},
Err(err) => Err(ProposeConfigChangeError::from(err)),
}))
)
tokio::spawn(async move {
rx_cfg_change
.map_err(|_| RaftError::ShuttingDown)
.into_future()
.then(|res| futures::future::ready(match res {
Ok(ok) => match ok {
Ok(ok) => Ok(ok),
Err(err) => Err(ChangeConfigError::from(err)),
},
Err(err) => Err(ChangeConfigError::from(err)),
}))
});
}
/// Handle the committment of a joint consensus cluster configuration.
/// Handle the commitment of a joint consensus cluster configuration.
#[tracing::instrument(level="debug", skip(self))]
pub(super) async fn handle_joint_consensus_committed(&mut self, _: ClientResponse<R>) -> Result<(), RaftError<E>> {
match &mut self.consensus_state {
@ -143,23 +173,21 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
return Ok(());
}
// Update current config to prepare for exiting joint consensus.
for node in self.core.membership.non_voters.drain(..) {
self.core.membership.members.push(node);
// Cut the cluster config over to the new membership config.
if let Some(new_members) = self.core.membership.members_after_consensus.take() {
self.core.membership.members = new_members;
}
for node in self.core.membership.removing.drain(..) {
// Remember, the leader still has replication streams to these nodes. Just because they
// have been removed from the config here does not mean they are not replicated to. Removed
// nodes will still receive the final config. Once they have committed that config, then the
// leader closes its replication stream to that node, which will have gone into non-voter state.
if let Some((idx, _)) = self.core.membership.members.iter().enumerate().find(|(_, e)| *e == &node) {
self.core.membership.members.remove(idx);
}
}
self.core.membership.is_in_joint_consensus = false;
self.consensus_state = ConsensusState::Uniform;
self.core.report_metrics(State::Leader);
// NOTE WELL: this implementation uses replication streams (src/replication/**) to replicate
// entries. Nodes which do not exist in the new config will still have an active replication
// stream until the current leader determines that they have replicated the config entry which
// removes them from the cluster. At that point in time, the node will revert to non-voter state.
//
// HOWEVER, if an election takes place, the new leader will not have the old nodes in its config
// and the old nodes may not revert to non-voter state using the above mechanism. That is fine.
// The Raft spec accounts for this using the 3rd safety measure of cluster configuration changes
// 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());
@ -167,19 +195,20 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
let entry = self.append_payload_to_log(payload.entry).await?;
let cr_entry = ClientRequestEntry::from_entry(entry, payload.response_mode, tx_uniform);
self.replicate_client_request(cr_entry).await;
self.core.report_metrics();
// Setup channel for eventual committment of the uniform consensus config.
// Setup channel for eventual commitment of the uniform consensus config.
self.uniform_consensus_cb.push(rx_uniform); // Receiver for when the uniform consensus is committed.
Ok(())
}
/// Handle the committment of a uniform consensus cluster configuration.
/// Handle the commitment of a uniform consensus cluster configuration.
#[tracing::instrument(level="debug", skip(self, res))]
pub(super) async fn handle_uniform_consensus_committed(&mut self, res: ClientResponse<R>) -> Result<(), RaftError<E>> {
// Step down if needed.
if !self.core.membership.contains(&self.core.id) {
if self.is_stepping_down {
tracing::debug!("raft node is stepping down");
self.core.set_target_state(TargetState::NonVoter);
self.core.set_target_state(State::NonVoter);
self.core.update_current_leader(UpdateCurrentLeader::Unknown);
return Ok(());
}
@ -191,10 +220,10 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
let nodes_to_remove: Vec<_> = self.nodes.iter_mut()
.filter(|(id, _)| !membership.contains(id))
.filter_map(|(idx, replstate)| {
if replstate.match_index >= res.index() {
if &replstate.match_index >= res.index() {
Some(idx.clone())
} else {
replstate.remove_after_commit = Some(res.index());
replstate.remove_after_commit = Some(*res.index());
None
}
}).collect();
@ -204,63 +233,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
let _ = node.replstream.repltx.send(RaftEvent::Terminate);
}
}
self.core.report_metrics();
Ok(())
}
}
//////////////////////////////////////////////////////////////////////////////////////////////////
// Utilities /////////////////////////////////////////////////////////////////////////////////////
// Ensure given config is normalized and ready for use in the cluster.
fn normalize_init_config(msg: InitWithConfig) -> InitWithConfig {
let mut nodes = vec![];
for node in msg.members {
if !nodes.contains(&node) {
nodes.push(node);
}
}
InitWithConfig{members: nodes}
}
/// Check the proposed config changes with the current config to ensure changes are valid.
///
/// See the documentation on on `ProposeConfigChangeError` for the conditions which will cause
/// errors to be returned.
fn normalize_proposed_config<E: AppError>(mut msg: ProposeConfigChange, current: &MembershipConfig) -> Result<ProposeConfigChange, ProposeConfigChangeError<E>> {
// Ensure no duplicates in adding new nodes & ensure the new
// node is not also be requested for removal.
let mut new_nodes = vec![];
for node in msg.add_members {
if !current.contains(&node) && !msg.remove_members.contains(&node) {
new_nodes.push(node);
}
}
// Ensure targets to remove exist in current config.
let mut remove_nodes = vec![];
for node in msg.remove_members {
if current.contains(&node) && !current.removing.contains(&node) {
remove_nodes.push(node);
}
}
// Account for noop.
if (new_nodes.len() == 0) && (remove_nodes.len() == 0) {
return Err(ProposeConfigChangeError::Noop);
}
// Ensure cluster will have at least two nodes.
let total_removing = current.removing.len() + remove_nodes.len();
let count = current.members.len() + current.non_voters.len() + new_nodes.len();
if total_removing >= count {
return Err(ProposeConfigChangeError::InoperableConfig);
} else if (count - total_removing) < 2 {
return Err(ProposeConfigChangeError::InoperableConfig);
}
msg.add_members = new_nodes;
msg.remove_members = remove_nodes;
Ok(msg)
}

View File

@ -1,7 +1,7 @@
use crate::{AppData, AppDataResponse, AppError, RaftNetwork, RaftStorage};
use crate::error::RaftResult;
use crate::raft::{AppendEntriesRequest, AppendEntriesResponse, ConflictOpt, Entry, EntryPayload};
use crate::core::{RaftCore, TargetState, UpdateCurrentLeader};
use crate::core::{RaftCore, State, UpdateCurrentLeader};
impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> RaftCore<D, R, E, N, S> {
/// An RPC invoked by the leader to replicate log entries (§5.3); also used as heartbeat (§5.2).
@ -31,7 +31,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
// Transition to follower state if needed.
if !self.target_state.is_follower() && !self.target_state.is_non_voter() {
self.set_target_state(TargetState::Follower);
self.set_target_state(State::Follower);
}
// Apply any outstanding logs to state machine based on `msg.leader_commit`.
@ -59,6 +59,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
let msg_index_and_term_match = (&msg.prev_log_index == &self.last_log_index) && (&msg.prev_log_term == &self.last_log_term);
if msg_prev_index_is_min || msg_index_and_term_match {
self.append_log_entries(&msg.entries).await?;
self.report_metrics();
return Ok(AppendEntriesResponse{term: self.current_term, success: true, conflict_opt: None});
}
@ -103,6 +104,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
tracing::trace!("end log consistency check");
self.append_log_entries(&msg.entries).await?;
self.report_metrics();
Ok(AppendEntriesResponse{term: self.current_term, success: true, conflict_opt: None})
}

View File

@ -23,6 +23,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
// Apply this entry to the state machine and return its data response.
let res = self.storage.apply_entry_to_state_machine(entry).await.map_err(|err| self.map_fatal_storage_result(err))?;
self.last_applied = entry.index;
self.report_metrics();
Ok(res)
}
}

View File

@ -5,8 +5,7 @@ use tokio::sync::oneshot;
use crate::{AppData, AppDataResponse, AppError, RaftNetwork, RaftStorage};
use crate::core::LeaderState;
use crate::error::{ClientError, RaftError, RaftResult};
use crate::raft::TxClientResponse;
use crate::raft::{ClientRequest, ClientResponse, Entry, EntryPayload, ResponseMode};
use crate::raft::{ClientRequest, ClientResponse, ClientResponseTx, Entry, EntryPayload, ResponseMode};
use crate::replication::RaftEvent;
/// A wrapper around a ClientRequest which has been transformed into an Entry, along with its ResponseMode & channel.
@ -19,12 +18,12 @@ pub(super) struct ClientRequestEntry<D: AppData, R: AppDataResponse, E: AppError
/// The response mode of the request.
pub response_mode: ResponseMode,
/// The response channel for the request.
pub tx: TxClientResponse<D, R, E>,
pub tx: ClientResponseTx<D, R, E>,
}
impl<D: AppData, R: AppDataResponse, E: AppError> ClientRequestEntry<D, R, E> {
/// Create a new instance from the raw components of a client request.
pub(crate) fn from_entry(entry: Entry<D>, response_mode: ResponseMode, tx: TxClientResponse<D, R, E>) -> Self {
pub(crate) fn from_entry(entry: Entry<D>, response_mode: ResponseMode, tx: ClientResponseTx<D, R, E>) -> Self {
Self{entry: Arc::new(entry), response_mode, tx}
}
}
@ -42,7 +41,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
};
// Check to see if we have any config change logs newer than our commit index. If so, then
// we need to drive the committment of the config change to the cluster.
// we need to drive the commitment of the config change to the cluster.
let mut pending_config = None; // The inner bool represents `is_in_join_consensus`.
if &self.core.last_log_index > &self.core.commit_index {
let (stale_logs_start, stale_logs_stop) = (self.core.commit_index + 1, self.core.last_log_index + 1);
@ -51,7 +50,8 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
// Find the most recent config change.
.iter().rev()
.filter_map(|entry| match &entry.payload {
EntryPayload::ConfigChange(cfg) => Some(cfg.membership.is_in_joint_consensus),
EntryPayload::ConfigChange(cfg) => Some(cfg.membership.is_in_joint_consensus()),
EntryPayload::SnapshotPointer(cfg) => Some(cfg.membership.is_in_joint_consensus()),
_ => None,
})
.nth(0);
@ -63,6 +63,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
self.core.last_log_term = self.core.current_term; // This only ever needs to be updated once per term.
let cr_entry = ClientRequestEntry::from_entry(entry, req.response_mode, tx_payload_committed);
self.replicate_client_request(cr_entry).await;
self.core.report_metrics();
// Setup any callbacks needed for responding to commitment of a pending config.
if let Some(is_in_join_consensus) = pending_config {
@ -77,7 +78,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
/// Handle client requests.
#[tracing::instrument(level="trace", skip(self, rpc, tx))]
pub(super) async fn handle_client_request(&mut self, rpc: ClientRequest<D>, tx: TxClientResponse<D, R, E>) {
pub(super) async fn handle_client_request(&mut self, rpc: ClientRequest<D>, tx: ClientResponseTx<D, R, E>) {
let entry = match self.append_payload_to_log(rpc.entry).await {
Ok(entry) => ClientRequestEntry::from_entry(entry, rpc.response_mode, tx),
Err(err) => {
@ -106,8 +107,8 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
pub(super) async fn replicate_client_request(&mut self, req: ClientRequestEntry<D, R, E>) {
// Replicate the request if there are other cluster members. The client response will be
// returned elsewhere after the entry has been committed to the cluster.
if self.nodes.len() > 0 {
let entry_arc = req.entry.clone();
let entry_arc = req.entry.clone();
if !self.nodes.is_empty() {
self.awaiting_committed.push(req);
for node in self.nodes.values() {
let _ = node.replstream.repltx.send(RaftEvent::Replicate{
@ -115,12 +116,22 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
commit_index: self.core.commit_index,
});
}
return;
} else {
// Else, there are no voting nodes for replication, so the payload is now committed.
self.core.commit_index = entry_arc.index;
self.core.report_metrics();
self.client_request_post_commit(req).await;
}
// Else, there are no nodes for replication, so the payload is now committed.
self.core.commit_index = req.entry.index;
self.client_request_post_commit(req).await;
// Replicate to non-voters.
if !self.non_voters.is_empty() {
for node in self.non_voters.values() {
let _ = node.state.replstream.repltx.send(RaftEvent::Replicate{
entry: entry_arc.clone(),
commit_index: self.core.commit_index,
});
}
}
}
/// Handle the post-commit logic for a client request.

View File

@ -5,7 +5,7 @@ use tokio::io::{AsyncSeekExt, AsyncWriteExt};
use crate::{AppData, AppDataResponse, AppError, RaftNetwork, RaftStorage};
use crate::error::RaftResult;
use crate::raft::{InstallSnapshotRequest, InstallSnapshotResponse};
use crate::core::{TargetState, RaftCore, SnapshotState, UpdateCurrentLeader};
use crate::core::{State, RaftCore, SnapshotState, UpdateCurrentLeader};
impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> RaftCore<D, R, E, N, S> {
/// Invoked by leader to send chunks of a snapshot to a follower (§7).
@ -24,19 +24,26 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
self.update_next_election_timeout();
// Update current term if needed.
let mut report_metrics = false;
if &self.current_term != &req.term {
self.update_current_term(req.term, None);
self.save_hard_state().await?;
report_metrics = true;
}
// Update current leader if needed.
if self.current_leader.as_ref() != Some(&req.leader_id) {
self.update_current_leader(UpdateCurrentLeader::OtherNode(req.leader_id));
report_metrics = true;
}
// If not follower, become follower.
if !self.target_state.is_follower() && !self.target_state.is_non_voter() {
self.set_target_state(TargetState::Follower);
self.set_target_state(State::Follower); // State update will emit metrics.
}
if report_metrics {
self.report_metrics();
}
// Compare current snapshot state with received RPC and handle as needed.

View File

@ -8,7 +8,7 @@ mod install_snapshot;
pub(crate) mod replication;
mod vote;
use std::collections::BTreeMap;
use std::collections::{BTreeMap, HashSet};
use std::sync::Arc;
use std::sync::atomic::{AtomicBool, Ordering};
@ -21,14 +21,11 @@ use tokio::time::{Instant, Duration, delay_until};
use tracing_futures::Instrument;
use crate::{AppData, AppDataResponse, AppError, RaftNetwork, RaftStorage, NodeId};
use crate::error::{ClientError, InitWithConfigError, ProposeConfigChangeError, RaftResult};
use crate::config::{Config, SnapshotPolicy};
use crate::raft::MembershipConfig;
use crate::metrics::{RaftMetrics, State};
use crate::core::client::ClientRequestEntry;
use crate::raft::{RxChanAppendEntries, RxChanVote, RxChanInstallSnapshot, RxChanClient, RxChanInit, RxChanPropose};
use crate::raft::{ClientRequest, ClientResponse, InitWithConfig, ProposeConfigChange};
use crate::raft::{TxClientResponse, TxInitResponse, TxProposeResponse};
use crate::error::{ClientError, InitializeError, ChangeConfigError, RaftResult};
use crate::metrics::RaftMetrics;
use crate::raft::{ChangeMembershipTx, ClientRequest, ClientResponse, ClientResponseTx, RaftMsg, MembershipConfig};
use crate::replication::{RaftEvent, ReplicationStream, ReplicaEvent};
use crate::storage::HardState;
@ -46,7 +43,7 @@ pub struct RaftCore<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<
storage: Arc<S>,
/// The target state of the system.
target_state: TargetState,
target_state: State,
/// The index of the highest log entry known to be committed cluster-wide.
///
@ -102,40 +99,28 @@ pub struct RaftCore<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<
tx_compaction: mpsc::Sender<SnapshotUpdate>,
rx_compaction: mpsc::Receiver<SnapshotUpdate>,
rx_append_entries: RxChanAppendEntries<D, E>,
rx_vote: RxChanVote<E>,
rx_install_snapshot: RxChanInstallSnapshot<E>,
rx_client: RxChanClient<D, R, E>,
rx_init: RxChanInit<E>,
rx_propose: RxChanPropose<E>,
rx_api: mpsc::UnboundedReceiver<RaftMsg<D, R, E>>,
tx_metrics: watch::Sender<RaftMetrics>,
}
impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> RaftCore<D, R, E, N, S> {
pub fn spawn(
pub(crate) fn spawn(
id: NodeId, config: Config, network: Arc<N>, storage: Arc<S>,
rx_append_entries: RxChanAppendEntries<D, E>,
rx_vote: RxChanVote<E>,
rx_install_snapshot: RxChanInstallSnapshot<E>,
rx_client: RxChanClient<D, R, E>,
rx_init: RxChanInit<E>,
rx_propose: RxChanPropose<E>,
rx_api: mpsc::UnboundedReceiver<RaftMsg<D, R, E>>,
tx_metrics: watch::Sender<RaftMetrics>,
needs_shutdown: Arc<AtomicBool>,
) -> JoinHandle<RaftResult<(), E>> {
let config = Arc::new(config);
let membership = MembershipConfig{is_in_joint_consensus: false, members: vec![id], non_voters: vec![], removing: vec![]};
let membership = MembershipConfig::new_initial(id); // This is updated from storage in the main loop.
let (tx_compaction, rx_compaction) = mpsc::channel(1);
let this = Self{
id, config, membership, network, storage,
target_state: TargetState::Follower,
target_state: State::Follower,
commit_index: 0, last_applied: 0, current_term: 0, current_leader: None, voted_for: None,
last_log_index: 0, last_log_term: 0,
snapshot_state: None, snapshot_index: 0,
last_heartbeat: None, next_election_timeout: None,
tx_compaction, rx_compaction,
rx_append_entries, rx_vote, rx_install_snapshot, rx_client,
rx_init, rx_propose, tx_metrics,
tx_compaction, rx_compaction, rx_api, tx_metrics,
needs_shutdown,
};
tokio::spawn(this.main())
@ -166,15 +151,15 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
// 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() {
self.target_state = TargetState::Leader;
self.target_state = State::Leader;
}
// Else if there are other members, that can only mean that state was recovered. Become follower.
else if !is_only_configured_member {
self.target_state = TargetState::Follower;
self.target_state = State::Follower;
}
// Else, for any other condition, stay non-voter.
else {
self.target_state = TargetState::NonVoter;
self.target_state = State::NonVoter;
}
// This is central loop of the system. The Raft core assumes a few different roles based
@ -183,20 +168,22 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
// if some error has been encountered, or if a state change is required.
loop {
match &self.target_state {
TargetState::Leader => LeaderState::new(&mut self).run().await?,
TargetState::Candidate => CandidateState::new(&mut self).run().await?,
TargetState::Follower => FollowerState::new(&mut self).run().await?,
TargetState::NonVoter => NonVoterState::new(&mut self).run().await?,
TargetState::Shutdown => return Ok(()),
State::Leader => LeaderState::new(&mut self).run().await?,
State::Candidate => CandidateState::new(&mut self).run().await?,
State::Follower => FollowerState::new(&mut self).run().await?,
State::NonVoter => NonVoterState::new(&mut self).run().await?,
State::Shutdown => return Ok(()),
}
}
}
/// Report a metrics payload on the current state of the Raft node.
#[tracing::instrument(level="debug", skip(self, state))]
fn report_metrics(&mut self, state: State) {
#[tracing::instrument(level="debug", skip(self))]
fn report_metrics(&mut self) {
let res = self.tx_metrics.broadcast(RaftMetrics{
id: self.id, state, current_term: self.current_term,
id: self.id,
state: self.target_state,
current_term: self.current_term,
last_log_index: self.last_log_index,
last_applied: self.last_applied,
current_leader: self.current_leader,
@ -216,9 +203,9 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// Update core's target state, ensuring all invariants are upheld.
#[tracing::instrument(level="debug", skip(self))]
fn set_target_state(&mut self, target_state: TargetState) {
if &target_state == &TargetState::Follower && self.membership.non_voters.contains(&self.id) {
self.target_state = TargetState::NonVoter;
fn set_target_state(&mut self, target_state: State) {
if &target_state == &State::Follower && !self.membership.contains(&self.id) {
self.target_state = State::NonVoter;
}
self.target_state = target_state;
}
@ -275,7 +262,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
#[tracing::instrument(level="debug", skip(self))]
fn map_fatal_storage_result<Err: std::error::Error>(&mut self, err: Err) -> Err {
tracing::error!({error=%err, id=self.id}, "fatal storage error, shutting down");
self.set_target_state(TargetState::Shutdown);
self.set_target_state(State::Shutdown);
err
}
@ -291,11 +278,11 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
// being removed.
self.membership = cfg;
if !self.membership.contains(&self.id) {
self.set_target_state(TargetState::NonVoter);
} else if &self.target_state == &TargetState::NonVoter && self.membership.members.contains(&self.id) {
self.set_target_state(State::NonVoter);
} else if &self.target_state == &State::NonVoter && self.membership.members.contains(&self.id) {
// The node is a NonVoter and the new config has it configured as a normal member.
// Transition to follower.
self.set_target_state(TargetState::Follower);
self.set_target_state(State::Follower);
}
Ok(self.save_hard_state().await?)
}
@ -355,19 +342,19 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// Reject an init config request due to the Raft node being in a state which prohibits the request.
#[tracing::instrument(level="trace", skip(self, tx))]
fn reject_init_with_config(&self, _: InitWithConfig, tx: TxInitResponse<E>) {
let _ = tx.send(Err(InitWithConfigError::NotAllowed));
fn reject_init_with_config(&self, tx: oneshot::Sender<Result<(), InitializeError<E>>>) {
let _ = tx.send(Err(InitializeError::NotAllowed));
}
/// Reject a proposed config change request due to the Raft node being in a state which prohibits the request.
#[tracing::instrument(level="trace", skip(self, tx))]
fn reject_propose_config_change(&self, _: ProposeConfigChange, tx: TxProposeResponse<E>) {
let _ = tx.send(Err(ProposeConfigChangeError::NodeNotLeader));
fn reject_config_change_not_leader(&self, tx: oneshot::Sender<Result<(), ChangeConfigError<E>>>) {
let _ = tx.send(Err(ChangeConfigError::NodeNotLeader));
}
/// Forward the given client request to the leader.
#[tracing::instrument(level="trace", skip(self, req, tx))]
fn forward_client_request(&self, req: ClientRequest<D>, tx: TxClientResponse<D, R, E>) {
fn forward_client_request(&self, req: ClientRequest<D>, tx: ClientResponseTx<D, R, E>) {
let _ = tx.send(Err(ClientError::ForwardToLeader(req, self.voted_for.clone())));
}
}
@ -414,17 +401,22 @@ pub(self) enum SnapshotUpdate {
///////////////////////////////////////////////////////////////////////////////////////////////////
///////////////////////////////////////////////////////////////////////////////////////////////////
/// The desired target state of a Raft node.
/// All possible states of a Raft node.
#[derive(Clone, Copy, Debug, PartialEq, Eq)]
pub(crate) enum TargetState {
pub enum State {
/// The node is completely passive; replicating entries, but neither voting nor timing out.
NonVoter,
/// The node is replicating logs from the leader.
Follower,
/// The node is campaigning to become the cluster leader.
Candidate,
/// The node is the Raft cluster leader.
Leader,
/// The Raft node is shutting down.
Shutdown,
}
impl TargetState {
impl State {
/// Check if currently in non-voter state.
pub fn is_non_voter(&self) -> bool {
if let Self::NonVoter = self { true } else { false }
@ -450,12 +442,15 @@ impl TargetState {
///////////////////////////////////////////////////////////////////////////////////////////////////
/// Volatile state specific to the Raft leader.
///
/// This state is reinitialized after an election.
struct LeaderState<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> {
pub(super) core: &'a mut RaftCore<D, R, E, N, S>,
/// A mapping of node IDs the replication state of the target node.
pub(super) nodes: BTreeMap<NodeId, ReplicationState<D>>,
/// A mapping of new nodes (non-voters) which are being synced in order to join the cluster.
pub(super) non_voters: BTreeMap<NodeId, NonVoterReplicationState<D, E>>,
/// A bool indicating if this node will be stepping down after committing the current config change.
pub(super) is_stepping_down: bool,
/// The stream of events coming from replication streams.
pub(super) replicationrx: mpsc::UnboundedReceiver<ReplicaEvent<S::Snapshot>>,
/// The clonable sender channel for replication stream events.
@ -463,7 +458,8 @@ struct LeaderState<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwo
/// A buffer of client requests which have been appended locally and are awaiting to be committed to the cluster.
pub(super) awaiting_committed: Vec<ClientRequestEntry<D, R, E>>,
/// A field tracking the cluster's current consensus state, which is used for dynamic membership.
pub(super) consensus_state: ConsensusState,
pub(super) consensus_state: ConsensusState<E>,
/// An optional response channel for when a config change has been proposed, and is awaiting a response.
pub(super) propose_config_change_cb: Option<oneshot::Sender<Result<(), ClientError<D, E>>>>,
/// An optional receiver for when a joint consensus config is committed.
@ -475,19 +471,17 @@ struct LeaderState<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwo
impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> LeaderState<'a, D, R, E, N, S> {
/// Create a new instance.
pub(self) fn new(core: &'a mut RaftCore<D, R, E, N, S>) -> Self {
let consensus_state = if core.membership.is_in_joint_consensus {
ConsensusState::Joint{
new_nodes_being_synced: core.membership.non_voters.clone(),
is_committed: false,
}
let consensus_state = if core.membership.is_in_joint_consensus() {
ConsensusState::Joint{is_committed: false}
} else {
ConsensusState::Uniform
};
let (replicationtx, replicationrx) = mpsc::unbounded_channel();
Self{
core, nodes: BTreeMap::new(), awaiting_committed: Vec::new(), consensus_state,
propose_config_change_cb: None, joint_consensus_cb: FuturesOrdered::new(), uniform_consensus_cb: FuturesOrdered::new(),
replicationtx, replicationrx,
core, nodes: BTreeMap::new(), non_voters: BTreeMap::new(), is_stepping_down: false,
replicationtx, replicationrx, consensus_state, awaiting_committed: Vec::new(),
propose_config_change_cb: None, joint_consensus_cb: FuturesOrdered::new(),
uniform_consensus_cb: FuturesOrdered::new(),
}
}
@ -495,26 +489,19 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
#[tracing::instrument(level="debug", skip(self), fields(id=self.core.id, raft_state="leader"))]
pub(self) async fn run(mut self) -> RaftResult<(), E> {
// Spawn replication streams.
let targets = self.core.membership.members.iter()
.filter(|elem| *elem != &self.core.id)
.chain(self.core.membership.non_voters.iter())
let targets = self.core.membership.all_nodes().into_iter()
.filter(|elem| elem != &self.core.id)
.collect::<Vec<_>>();
for target in targets {
// Build & spawn a replication stream for the target member.
let replstream = ReplicationStream::new(
self.core.id, *target, self.core.current_term, self.core.config.clone(),
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(),
);
let state = ReplicationState{match_index: self.core.last_log_index, is_at_line_rate: true, replstream, remove_after_commit: None};
self.nodes.insert(*target, state);
let state = self.spawn_replication_stream(target);
self.nodes.insert(target, state);
}
// Setup state as leader.
self.core.last_heartbeat = None;
self.core.next_election_timeout = None;
self.core.update_current_leader(UpdateCurrentLeader::ThisNode);
self.core.report_metrics(State::Leader);
self.core.report_metrics();
// Per §8, commit an initial entry as part of becoming the cluster leader.
self.commit_initial_leader_entry().await?;
@ -527,31 +514,30 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
return Ok(());
}
tokio::select!{
Some((rpc, tx)) = self.core.rx_append_entries.next() => {
let res = self.core.handle_append_entries_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_vote.next() => {
let res = self.core.handle_vote_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_install_snapshot.next() => {
let res = self.core.handle_install_snapshot_request(rpc).await;
let _ = tx.send(res);
}
Some(msg) = self.core.rx_api.next() => match msg {
RaftMsg::AppendEntries{rpc, tx} => {
let _ = tx.send(self.core.handle_append_entries_request(rpc).await);
}
RaftMsg::RequestVote{rpc, tx} => {
let _ = tx.send(self.core.handle_vote_request(rpc).await);
}
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::Initialize{tx, ..} => {
self.core.reject_init_with_config(tx);
}
RaftMsg::AddNonVoter{id, tx} => {
self.add_member(id, tx);
}
RaftMsg::ChangeMembership{members, tx} => {
self.change_membership(members, tx).await;
}
},
Some(update) = self.core.rx_compaction.next() => self.core.update_snapshot_state(update),
Some((rpc, tx)) = self.core.rx_client.next() => self.handle_client_request(rpc, tx).await,
Some((rpc, tx)) = self.core.rx_init.next() => self.core.reject_init_with_config(rpc, tx),
Some((rpc, tx)) = self.core.rx_propose.next() => {
let res = match self.handle_propose_config_change(rpc).await {
Ok(res) => match res.await {
Ok(_) => Ok(()),
Err(err) => Err(err),
}
Err(err) => Err(err),
};
let _ = tx.send(res);
}
Some(Ok(res)) = self.joint_consensus_cb.next() => {
match res {
Ok(clientres) => self.handle_joint_consensus_committed(clientres).await?,
@ -587,32 +573,49 @@ struct ReplicationState<D: AppData> {
pub replstream: ReplicationStream<D>,
}
pub enum ConsensusState {
/// The cluster consensus is uniform; not in a joint consensus state.
Uniform,
/// The same as `ReplicationState`, except for non-voters.
struct NonVoterReplicationState<D: AppData, E: AppError> {
/// The replication stream state.
pub state: ReplicationState<D>,
/// A bool indicating if this non-voters is ready to join the cluster.
pub is_ready_to_join: bool,
/// The response channel to use for when this node has successfully joined the cluster.
pub tx: Option<oneshot::Sender<Result<(), ChangeConfigError<E>>>>,
}
/// A state enum used by Raft leaders to navigate the joint consensus protocol.
pub enum ConsensusState<E: AppError> {
/// The cluster is preparring to go into joint consensus, but the leader is still syncing
/// some non-voters to prepare them for cluster membership.
NonVoterSync {
/// The set of non-voters nodes which are still being synced.
awaiting: HashSet<NodeId>,
/// The full membership change which has been proposed.
members: HashSet<NodeId>,
/// The response channel to use once the consensus state is back into uniform state.
tx: ChangeMembershipTx<E>,
},
/// The cluster is in a joint consensus state and is syncing new nodes.
Joint {
/// The new nodes which are being synced.
new_nodes_being_synced: Vec<NodeId>,
/// A bool indicating if the associated config which started this join consensus has yet been comitted.
/// A bool indicating if the associated config which started this joint consensus has yet been comitted.
///
/// NOTE: when a new leader is elected, it will initialize this value to false, and then
/// update this value to true once the new leader's blank payload has been committed.
is_committed: bool,
}
},
/// The cluster consensus is uniform; not in a joint consensus state.
Uniform,
}
impl ConsensusState {
impl<E: AppError> ConsensusState<E> {
/// Check the current state to determine if it is in joint consensus, and if it is safe to finalize the joint consensus.
///
/// The return value will be true if:
/// 1. this object currently represents a joint consensus state.
/// 2. the corresponding config for this consensus state has been committed to the cluster.
/// 3. all new nodes being added to the cluster have been synced.
pub fn is_joint_consensus_safe_to_finalize(&self) -> bool {
match self {
ConsensusState::Joint{is_committed, new_nodes_being_synced}
if *is_committed && new_nodes_being_synced.len() == 0 => true,
ConsensusState::Joint{is_committed} => *is_committed,
_ => false,
}
}
@ -624,15 +627,19 @@ impl ConsensusState {
/// Volatile state specific to a Raft node in candidate state.
struct CandidateState<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> {
core: &'a mut RaftCore<D, R, E, N, S>,
/// The number of votes which have been granted by peer nodes.
votes_granted: u64,
/// The number of votes needed in order to become the Raft leader.
votes_needed: u64,
/// The number of votes which have been granted by peer nodes of the old (current) config group.
votes_granted_old: u64,
/// The number of votes needed from the old (current) config group in order to become the Raft leader.
votes_needed_old: u64,
/// The number of votes which have been granted by peer nodes of the new config group (if applicable).
votes_granted_new: u64,
/// The number of votes needed from the new config group in order to become the Raft leader (if applicable).
votes_needed_new: u64,
}
impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> CandidateState<'a, D, R, E, N, S> {
pub(self) fn new(core: &'a mut RaftCore<D, R, E, N, S>) -> Self {
Self{core, votes_granted: 0, votes_needed: 0}
Self{core, votes_granted_old: 0, votes_needed_old: 0, votes_granted_new: 0, votes_needed_new: 0}
}
/// Run the candidate loop.
@ -641,8 +648,12 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
// Each iteration of the outer loop represents a new term.
loop {
// Setup initial state per term.
self.votes_granted = 1; // We must vote for ourselves per the Raft spec.
self.votes_needed = ((self.core.membership.members.len() / 2) + 1) as u64; // Just need a majority.
self.votes_granted_old = 1; // We must vote for ourselves per the Raft spec.
self.votes_needed_old = ((self.core.membership.members.len() / 2) + 1) as u64; // Just need a majority.
if let Some(nodes) = &self.core.membership.members_after_consensus {
self.votes_granted_new = 1; // We must vote for ourselves per the Raft spec.
self.votes_needed_new = ((nodes.len() / 2) + 1) as u64; // Just need a majority.
}
// Setup new term.
self.core.update_next_election_timeout(); // Generates a new rand value within range.
@ -650,7 +661,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
self.core.voted_for = Some(self.core.id);
self.core.update_current_leader(UpdateCurrentLeader::Unknown);
self.core.save_hard_state().await?;
self.core.report_metrics(State::Candidate);
self.core.report_metrics();
// Send RPCs to all members in parallel.
let mut pending_votes = self.spawn_parallel_vote_requests();
@ -664,23 +675,31 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
let mut timeout_fut = delay_until(self.core.get_next_election_timeout());
tokio::select!{
_ = &mut timeout_fut => break, // This election has timed-out. Break to outer loop, which starts a new term.
Some((rpc, tx)) = self.core.rx_append_entries.next() => {
let res = self.core.handle_append_entries_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_vote.next() => {
let res = self.core.handle_vote_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_install_snapshot.next() => {
let res = self.core.handle_install_snapshot_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_client.next() => self.core.forward_client_request(rpc, tx),
Some((rpc, tx)) = self.core.rx_init.next() => self.core.reject_init_with_config(rpc, tx),
Some((rpc, tx)) = self.core.rx_propose.next() => self.core.reject_propose_config_change(rpc, tx),
Some(update) = self.core.rx_compaction.next() => self.core.update_snapshot_state(update),
Some((res, peer)) = pending_votes.recv() => self.handle_vote_response(res, peer).await?,
Some(msg) = self.core.rx_api.next() => match msg {
RaftMsg::AppendEntries{rpc, tx} => {
let _ = tx.send(self.core.handle_append_entries_request(rpc).await);
}
RaftMsg::RequestVote{rpc, tx} => {
let _ = tx.send(self.core.handle_vote_request(rpc).await);
}
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::Initialize{tx, ..} => {
self.core.reject_init_with_config(tx);
}
RaftMsg::AddNonVoter{tx, ..} => {
self.core.reject_config_change_not_leader(tx);
}
RaftMsg::ChangeMembership{tx, ..} => {
self.core.reject_config_change_not_leader(tx);
}
},
Some(update) = self.core.rx_compaction.next() => self.core.update_snapshot_state(update),
}
}
}
@ -703,7 +722,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
/// Run the follower loop.
#[tracing::instrument(level="debug", skip(self), fields(id=self.core.id, raft_state="follower"))]
pub(self) async fn run(self) -> RaftResult<(), E> {
self.core.report_metrics(State::Follower);
self.core.report_metrics();
loop {
if !self.core.target_state.is_follower() || self.core.needs_shutdown.load(Ordering::SeqCst) {
return Ok(());
@ -712,23 +731,31 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
let mut election_timeout = delay_until(self.core.get_next_election_timeout()); // Value is updated as heartbeats are received.
tokio::select!{
// If an election timeout is hit, then we need to transition to candidate.
_ = &mut election_timeout => self.core.set_target_state(TargetState::Candidate),
Some((rpc, tx)) = self.core.rx_append_entries.next() => {
let res = self.core.handle_append_entries_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_vote.next() => {
let res = self.core.handle_vote_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_install_snapshot.next() => {
let res = self.core.handle_install_snapshot_request(rpc).await;
let _ = tx.send(res);
}
_ = &mut election_timeout => self.core.set_target_state(State::Candidate),
Some(msg) = self.core.rx_api.next() => match msg {
RaftMsg::AppendEntries{rpc, tx} => {
let _ = tx.send(self.core.handle_append_entries_request(rpc).await);
}
RaftMsg::RequestVote{rpc, tx} => {
let _ = tx.send(self.core.handle_vote_request(rpc).await);
}
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::Initialize{tx, ..} => {
self.core.reject_init_with_config(tx);
}
RaftMsg::AddNonVoter{tx, ..} => {
self.core.reject_config_change_not_leader(tx);
}
RaftMsg::ChangeMembership{tx, ..} => {
self.core.reject_config_change_not_leader(tx);
}
},
Some(update) = self.core.rx_compaction.next() => self.core.update_snapshot_state(update),
Some((rpc, tx)) = self.core.rx_client.next() => self.core.forward_client_request(rpc, tx),
Some((rpc, tx)) = self.core.rx_init.next() => self.core.reject_init_with_config(rpc, tx),
Some((rpc, tx)) = self.core.rx_propose.next() => self.core.reject_propose_config_change(rpc, tx),
}
}
}
@ -750,31 +777,36 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
/// Run the non-voter loop.
#[tracing::instrument(level="debug", skip(self), fields(id=self.core.id, raft_state="non-voter"))]
pub(self) async fn run(mut self) -> RaftResult<(), E> {
self.core.report_metrics(State::NonVoter);
self.core.report_metrics();
loop {
if !self.core.target_state.is_non_voter() || self.core.needs_shutdown.load(Ordering::SeqCst) {
return Ok(());
}
tokio::select!{
Some((rpc, tx)) = self.core.rx_append_entries.next() => {
let res = self.core.handle_append_entries_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_vote.next() => {
let res = self.core.handle_vote_request(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_install_snapshot.next() => {
let res = self.core.handle_install_snapshot_request(rpc).await;
let _ = tx.send(res);
}
Some(msg) = self.core.rx_api.next() => match msg {
RaftMsg::AppendEntries{rpc, tx} => {
let _ = tx.send(self.core.handle_append_entries_request(rpc).await);
}
RaftMsg::RequestVote{rpc, tx} => {
let _ = tx.send(self.core.handle_vote_request(rpc).await);
}
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::Initialize{members, tx} => {
let _ = tx.send(self.handle_init_with_config(members).await);
}
RaftMsg::AddNonVoter{tx, ..} => {
self.core.reject_config_change_not_leader(tx);
}
RaftMsg::ChangeMembership{tx, ..} => {
self.core.reject_config_change_not_leader(tx);
}
},
Some(update) = self.core.rx_compaction.next() => self.core.update_snapshot_state(update),
Some((rpc, tx)) = self.core.rx_client.next() => self.core.forward_client_request(rpc, tx),
Some((rpc, tx)) = self.core.rx_init.next() => {
let res = self.handle_init_with_config(rpc).await;
let _ = tx.send(res);
}
Some((rpc, tx)) = self.core.rx_propose.next() => self.core.reject_propose_config_change(rpc, tx),
}
}
}

View File

@ -3,11 +3,23 @@ use tokio::sync::oneshot;
use crate::{AppData, AppDataResponse, AppError, NodeId, RaftNetwork, RaftStorage};
use crate::config::SnapshotPolicy;
use crate::error::RaftResult;
use crate::core::{ConsensusState, LeaderState, SnapshotState, TargetState, UpdateCurrentLeader};
use crate::replication::{RaftEvent, ReplicaEvent};
use crate::core::{ConsensusState, LeaderState, ReplicationState, SnapshotState, State, UpdateCurrentLeader};
use crate::replication::{RaftEvent, ReplicaEvent, ReplicationStream};
use crate::storage::CurrentSnapshotData;
impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> LeaderState<'a, D, R, E, N, S> {
/// Spawn a new replication stream returning its replication state handle.
#[tracing::instrument(level="debug", skip(self))]
pub(super) fn spawn_replication_stream(&self, target: NodeId) -> ReplicationState<D> {
let replstream = ReplicationStream::new(
self.core.id, target, self.core.current_term, self.core.config.clone(),
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}
}
/// Handle a replication event coming from one of the replication streams.
#[tracing::instrument(level="trace", skip(self, event))]
pub(super) async fn handle_replica_event(&mut self, event: ReplicaEvent<S::Snapshot>) {
let res = match event {
@ -15,6 +27,10 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
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::NeedsSnapshot{target, tx} => self.handle_needs_snapshot(target, tx).await,
ReplicaEvent::Shutdown => {
self.core.set_target_state(State::Shutdown);
return;
}
};
if let Err(err) = res {
tracing::error!({error=%err}, "error while processing event from replication stream");
@ -25,23 +41,34 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
#[tracing::instrument(level="trace", skip(self, target, is_line_rate))]
async fn handle_rate_update(&mut self, target: NodeId, is_line_rate: bool) -> RaftResult<(), E> {
// Get a handle the target's replication stat & update it as needed.
let repl_state = match self.nodes.get_mut(&target) {
Some(repl_state) => repl_state,
_ => return Ok(()),
};
repl_state.is_at_line_rate = is_line_rate;
// If in joint consensus, and the target node was one of the new nodes, update
// the joint consensus state to indicate that the target is up-to-date.
if let ConsensusState::Joint{new_nodes_being_synced, ..} = &mut self.consensus_state {
if let Some((idx, _)) = new_nodes_being_synced.iter().enumerate().find(|(_, e)| e == &&target) {
new_nodes_being_synced.remove(idx);
if let Some(state) = self.nodes.get_mut(&target) {
state.is_at_line_rate = is_line_rate;
return Ok(());
}
// Else, if this is a non-voter, then update as needed.
if let Some(state) = self.non_voters.get_mut(&target) {
state.state.is_at_line_rate = is_line_rate;
state.is_ready_to_join = true;
// Issue a response on the non-voters response channel if needed.
if let Some(tx) = state.tx.take() {
let _ = tx.send(Ok(()));
}
// If there are no remaining nodes to sync, then finalize this joint consensus.
if self.consensus_state.is_joint_consensus_safe_to_finalize() {
self.finalize_joint_consensus().await?;
// If we are in NonVoterSync state, and this is one of the nodes being awaiting, then update.
match std::mem::replace(&mut self.consensus_state, ConsensusState::Uniform) {
ConsensusState::NonVoterSync{mut awaiting, members, tx} => {
awaiting.remove(&target);
if awaiting.is_empty() {
// We are ready to move forward with entering joint consensus.
self.consensus_state = ConsensusState::Uniform;
self.change_membership(members, tx).await;
} else {
// We are still awaiting additional nodes, so replace our original state.
self.consensus_state = ConsensusState::NonVoterSync{awaiting, members, tx};
}
}
other => self.consensus_state = other, // Set the original value back to what it was.
}
}
Ok(())
}
@ -52,7 +79,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
self.core.update_current_term(term, None);
self.core.save_hard_state().await?;
self.core.update_current_leader(UpdateCurrentLeader::Unknown);
self.core.set_target_state(TargetState::Follower);
self.core.set_target_state(State::Follower);
}
Ok(())
}
@ -60,12 +87,18 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: 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<(), E> {
// 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;
return Ok(());
}
// Update target's match index & check if it is awaiting removal.
let mut needs_removal = false;
match self.nodes.get_mut(&target) {
Some(replstate) => {
replstate.match_index = match_index;
if let Some(threshold) = &replstate.remove_after_commit {
Some(state) => {
state.match_index = match_index;
if let Some(threshold) = &state.remove_after_commit {
if &match_index >= threshold {
needs_removal = true;
}
@ -81,26 +114,43 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
}
}
// Parse through each targets' match index, and update the value of `commit_index` based
// on the highest value which has been replicated to a majority of the cluster
// including the leader which created the entry.
let mut indices: Vec<_> = self.nodes.values().map(|elem| elem.match_index).collect();
indices.push(self.core.last_log_index);
let new_commit_index = calculate_new_commit_index(indices, self.core.commit_index);
let has_new_commit_index = &new_commit_index > &self.core.commit_index;
// Determine the new commit index of the current membership config nodes.
let mut indices_c0 = self.nodes.iter()
.filter(|(id, _)| self.core.membership.members.contains(id))
.map(|(_, node)| node.match_index)
.collect::<Vec<_>>();
if !self.is_stepping_down {
indices_c0.push(self.core.last_log_index);
}
let commit_index_c0 = calculate_new_commit_index(indices_c0, self.core.commit_index);
// If a new commit index has been determined, update a few needed elements.
// If we are in joint consensus, then calculate the new commit index of the new membership config nodes.
let mut commit_index_c1 = commit_index_c0; // Defaults to just matching C0.
if let Some(members) = &self.core.membership.members_after_consensus {
let indices_c1 = self.nodes.iter()
.filter(|(id, _)| members.contains(id))
.map(|(_, node)| node.match_index)
.collect();
commit_index_c1 = calculate_new_commit_index(indices_c1, self.core.commit_index);
}
// Determine if we have a new commit index, accounting for joint consensus.
// If a new commit index has been established, then update a few needed elements.
let has_new_commit_index = &commit_index_c0 > &self.core.commit_index && &commit_index_c1 > &self.core.commit_index;
if has_new_commit_index {
self.core.commit_index = new_commit_index;
self.core.commit_index = std::cmp::min(commit_index_c0, commit_index_c1);
// Update all replication streams based on new commit index.
for node in self.nodes.values() {
let _ = node.replstream.repltx.send(RaftEvent::UpdateCommitIndex{commit_index: new_commit_index});
let _ = node.replstream.repltx.send(RaftEvent::UpdateCommitIndex{commit_index: self.core.commit_index});
}
for node in self.non_voters.values() {
let _ = node.state.replstream.repltx.send(RaftEvent::UpdateCommitIndex{commit_index: self.core.commit_index});
}
// Check if there are any pending requests which need to be processed.
let filter = self.awaiting_committed.iter().enumerate()
.take_while(|(_idx, elem)| &elem.entry.index <= &new_commit_index)
.take_while(|(_idx, elem)| &elem.entry.index <= &self.core.commit_index)
.last()
.map(|(idx, _)| idx);
if let Some(offset) = filter {
@ -109,6 +159,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
self.client_request_post_commit(request).await;
}
}
self.core.report_metrics();
}
Ok(())
}
@ -163,8 +214,8 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
/// Determine the value for `current_commit` based on all known indicies of the cluster members.
///
/// - `entries`: is a vector of all of the highest known index to be replicated on a target node,
/// one per node of the cluster, including the leader.
/// - `entries`: is a vector of all of the highest known indices to be replicated on a target node,
/// one per node of the cluster, including the leader as long as the leader is not stepping down.
/// - `current_commit`: is the Raft node's `current_commit` value before invoking this function.
/// The output of this function will never be less than this value.
///

View File

@ -4,7 +4,7 @@ use tracing_futures::Instrument;
use crate::{AppData, AppDataResponse, AppError, NodeId, RaftNetwork, RaftStorage};
use crate::error::RaftResult;
use crate::core::{CandidateState, RaftCore, TargetState, UpdateCurrentLeader};
use crate::core::{CandidateState, RaftCore, State, UpdateCurrentLeader};
use crate::raft::{VoteRequest, VoteResponse};
impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> RaftCore<D, R, E, N, S> {
@ -13,16 +13,10 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// See `receiver implementation: RequestVote RPC` in raft-essentials.md in this repo.
#[tracing::instrument(level="trace", skip(self, msg))]
pub(super) async fn handle_vote_request(&mut self, msg: VoteRequest) -> RaftResult<VoteResponse, E> {
// Don't accept vote requests from unknown-cluster members.
if !self.membership.contains(&msg.candidate_id) {
tracing::trace!({candidate=msg.candidate_id}, "RequestVote RPC received from an unknown node");
return Ok(VoteResponse{term: self.current_term, vote_granted: false, is_candidate_unknown: true});
}
// If candidate's current term is less than this nodes current term, reject.
if &msg.term < &self.current_term {
tracing::trace!({candidate=msg.candidate_id, self.current_term, rpc_term=msg.term}, "RequestVote RPC term is less than current term");
return Ok(VoteResponse{term: self.current_term, vote_granted: false, is_candidate_unknown: false});
return Ok(VoteResponse{term: self.current_term, vote_granted: false});
}
// Do not respond to the request if we've received a heartbeat within the election timeout minimum.
@ -31,7 +25,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
let delta = now.duration_since(*inst);
if self.config.election_timeout_min >= (delta.as_millis() as u64) {
tracing::trace!({candidate=msg.candidate_id}, "rejecting vote request received within election timeout minimum");
return Ok(VoteResponse{term: self.current_term, vote_granted: false, is_candidate_unknown: false});
return Ok(VoteResponse{term: self.current_term, vote_granted: false});
}
}
@ -40,7 +34,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
if &msg.term > &self.current_term {
self.update_current_term(msg.term, None);
self.update_next_election_timeout();
self.set_target_state(TargetState::Follower);
self.set_target_state(State::Follower);
self.save_hard_state().await?;
}
@ -49,25 +43,25 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
let client_is_uptodate = (&msg.last_log_term >= &self.last_log_term) && (&msg.last_log_index >= &self.last_log_index);
if !client_is_uptodate {
tracing::trace!({candidate=msg.candidate_id}, "rejecting vote request as candidate's log is not up-to-date");
return Ok(VoteResponse{term: self.current_term, vote_granted: false, is_candidate_unknown: false});
return Ok(VoteResponse{term: self.current_term, vote_granted: false});
}
// Candidate's log is up-to-date so handle voting conditions.
match &self.voted_for {
// This node has already voted for the candidate.
Some(candidate_id) if candidate_id == &msg.candidate_id => {
Ok(VoteResponse{term: self.current_term, vote_granted: true, is_candidate_unknown: false})
Ok(VoteResponse{term: self.current_term, vote_granted: true})
}
// This node has already voted for a different candidate.
Some(_) => Ok(VoteResponse{term: self.current_term, vote_granted: false, is_candidate_unknown: false}),
Some(_) => Ok(VoteResponse{term: self.current_term, vote_granted: false}),
// This node has not yet voted for the current term, so vote for the candidate.
None => {
self.voted_for = Some(msg.candidate_id);
self.set_target_state(TargetState::Follower);
self.set_target_state(State::Follower);
self.update_next_election_timeout();
self.save_hard_state().await?;
tracing::trace!({candidate=msg.candidate_id, msg.term}, "voted for candidate");
Ok(VoteResponse{term: self.current_term, vote_granted: true, is_candidate_unknown: false})
Ok(VoteResponse{term: self.current_term, vote_granted: true})
},
}
}
@ -77,19 +71,11 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
/// Handle response from a vote request sent to a peer.
#[tracing::instrument(level="trace", skip(self, res, target))]
pub(super) async fn handle_vote_response(&mut self, res: VoteResponse, target: NodeId) -> RaftResult<(), E> {
// If responding node sees this node as being unknown to the cluster, and this node has been active,
// then go into NonVoter state, as this typically means this node is being removed from the cluster.
if res.is_candidate_unknown && self.core.last_log_index > 0 {
tracing::trace!({target=target}, "target node considers this node to be unknown to the cluster, transitioning to non-voter");
self.core.set_target_state(TargetState::NonVoter);
return Ok(());
}
// If peer's term is greater than current term, revert to follower state.
if res.term > self.core.current_term {
self.core.update_current_term(res.term, None);
self.core.update_current_leader(UpdateCurrentLeader::Unknown);
self.core.set_target_state(TargetState::Follower);
self.core.set_target_state(State::Follower);
self.core.save_hard_state().await?;
tracing::trace!("reverting to follower state due to greater term observed in RequestVote RPC response");
return Ok(());
@ -97,11 +83,18 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
// If peer granted vote, then update campaign state.
if res.vote_granted {
self.votes_granted += 1;
if self.votes_granted >= self.votes_needed {
// If the campaign was successful, go into leader state.
// Handle vote responses from the C0 config group.
if self.core.membership.members.contains(&target) {
self.votes_granted_old += 1;
}
// Handle vote responses from members of C1 config group.
if self.core.membership.members_after_consensus.as_ref().map(|members| members.contains(&target)).unwrap_or(false) {
self.votes_granted_new += 1;
}
// If we've received enough votes from both config groups, then transition to leader state`.
if &self.votes_granted_old >= &self.votes_needed_old && &self.votes_granted_new >= &self.votes_needed_new {
tracing::trace!("transitioning to leader state as minimum number of votes have been received");
self.core.set_target_state(TargetState::Leader);
self.core.set_target_state(State::Leader);
return Ok(());
}
}
@ -113,8 +106,9 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
/// Build a future of vote requests sent to all peers.
#[tracing::instrument(level="trace", skip(self))]
pub(super) fn spawn_parallel_vote_requests(&self) -> mpsc::Receiver<(VoteResponse, NodeId)> {
let (tx, rx) = mpsc::channel(self.core.membership.members.len());
for member in self.core.membership.members.iter().cloned().filter(|member| member != &self.core.id) {
let all_members = self.core.membership.all_nodes();
let (tx, rx) = mpsc::channel(all_members.len());
for member in all_members.into_iter().filter(|member| member != &self.core.id) {
let rpc = VoteRequest::new(self.core.current_term, self.core.id, self.core.last_log_index, self.core.last_log_term);
let (network, mut tx_inner) = (self.core.network.clone(), tx.clone());
let _ = tokio::spawn(async move {

View File

@ -49,9 +49,9 @@ pub enum ConfigError {
MaxPayloadEntriesTooSmall,
}
/// The set of errors which may take place when requesting to propose a config change.
/// The set of errors which may take place when initializing a pristine Raft node.
#[derive(Debug, Error)]
pub enum InitWithConfigError<E: AppError> {
pub enum InitializeError<E: AppError> {
/// An internal error has taken place.
#[error("{0}")]
RaftError(#[from] RaftError<E>),
@ -62,7 +62,7 @@ pub enum InitWithConfigError<E: AppError> {
/// The set of errors which may take place when requesting to propose a config change.
#[derive(Debug, Error)]
pub enum ProposeConfigChangeError<E: AppError> {
pub enum ChangeConfigError<E: AppError> {
/// An error related to the processing of the config change request.
///
/// Errors of this type will only come about from the internals of applying the config change
@ -71,11 +71,11 @@ pub enum ProposeConfigChangeError<E: AppError> {
RaftError(#[from] RaftError<E>),
/// The cluster is already undergoing a configuration change.
#[error("the cluster is already undergoing a configuration change")]
AlreadyInJointConsensus,
ConfigChangeInProgress,
/// The given config would leave the cluster in an inoperable state.
///
/// This error will be returned if the full set of changes, once fully applied, would leave
/// the cluster with less than two members.
/// the cluster in an inoperable state.
#[error("the given config would leave the cluster in an inoperable state")]
InoperableConfig,
/// The node the config change proposal was sent to was not the leader of the cluster.
@ -84,15 +84,11 @@ pub enum ProposeConfigChangeError<E: AppError> {
/// The proposed config changes would make no difference to the current config.
///
/// This takes into account a current joint consensus and the end result of the config.
///
/// This error will be returned if the proposed add & remove elements are empty; all of the
/// entries to be added already exist in the current config and/or all of the entries to be
/// removed have already been scheduled for removal and/or do not exist in the current config.
#[error("the proposed config change would have no effect, this is a no-op")]
Noop,
}
impl<D: AppData, E: AppError> From<ClientError<D, E>> for ProposeConfigChangeError<E> {
impl<D: AppData, E: AppError> From<ClientError<D, E>> for ChangeConfigError<E> {
fn from(src: ClientError<D, E>) -> Self {
match src {
ClientError::RaftError(err) => Self::RaftError(err),

View File

@ -1,5 +1,5 @@
#![cfg_attr(feature="docinclude", feature(external_doc))]
#![cfg_attr(feature="docinclude", doc(include="../README.md"))]
#![cfg_attr(feature="docinclude", doc(include="../../README.md"))]
pub mod config;
mod core;
@ -18,7 +18,7 @@ use serde::{Serialize, de::DeserializeOwned};
// Top-level exports.
pub use crate::{
config::{Config, ConfigBuilder, SnapshotPolicy},
error::{ConfigError, InitWithConfigError, ProposeConfigChangeError, RaftError},
error::{ConfigError, InitializeError, ChangeConfigError, RaftError},
metrics::RaftMetrics,
network::RaftNetwork,
raft::Raft,

View File

@ -8,21 +8,9 @@
//! return a stream of metrics.
use crate::NodeId;
use crate::core::State;
use crate::raft::MembershipConfig;
/// All possible states of a Raft node.
#[derive(Clone, Copy, Debug, PartialEq, Eq)]
pub enum State {
/// The node is completely passive; replicating entries, but neither voting nor timing out.
NonVoter,
/// The node is replicating logs from the leader.
Follower,
/// The node is campaigning to become the cluster leader.
Candidate,
/// The node is the Raft cluster leader.
Leader,
}
/// Baseline metrics of the current state of the subject Raft node.
///
/// See the [module level documentation](TODO:)

View File

@ -1,5 +1,6 @@
//! Public Raft interface and data types.
use std::collections::HashSet;
use std::sync::Arc;
use std::sync::atomic::{AtomicBool, Ordering};
@ -9,31 +10,10 @@ use tokio::task::JoinHandle;
use crate::{AppData, AppDataResponse, AppError, NodeId, RaftNetwork, RaftStorage};
use crate::config::Config;
use crate::error::{ClientError, InitWithConfigError, ProposeConfigChangeError, RaftError, RaftResult};
use crate::error::{ClientError, InitializeError, ChangeConfigError, RaftError, RaftResult};
use crate::metrics::RaftMetrics;
use crate::core::RaftCore;
pub(crate) type TxAppendEntriesResponse<E> = oneshot::Sender<RaftResult<AppendEntriesResponse, E>>;
pub(crate) type TxVoteResponse<E> = oneshot::Sender<RaftResult<VoteResponse, E>>;
pub(crate) type TxInstallSnapshotResponse<E> = oneshot::Sender<RaftResult<InstallSnapshotResponse, E>>;
pub(crate) type TxClientResponse<D, R, E> = oneshot::Sender<Result<ClientResponse<R>, ClientError<D, E>>>;
pub(crate) type TxInitResponse<E> = oneshot::Sender<Result<(), InitWithConfigError<E>>>;
pub(crate) type TxProposeResponse<E> = oneshot::Sender<Result<(), ProposeConfigChangeError<E>>>;
pub(crate) type TxChanAppendEntries<D, E> = mpsc::UnboundedSender<(AppendEntriesRequest<D>, TxAppendEntriesResponse<E>)>;
pub(crate) type TxChanVote<E> = mpsc::UnboundedSender<(VoteRequest, TxVoteResponse<E>)>;
pub(crate) type TxChanInstallSnapshot<E> = mpsc::UnboundedSender<(InstallSnapshotRequest, TxInstallSnapshotResponse<E>)>;
pub(crate) type TxChanClient<D, R, E> = mpsc::UnboundedSender<(ClientRequest<D>, TxClientResponse<D, R, E>)>;
pub(crate) type TxChanInit<E> = mpsc::UnboundedSender<(InitWithConfig, TxInitResponse<E>)>;
pub(crate) type TxChanPropose<E> = mpsc::UnboundedSender<(ProposeConfigChange, TxProposeResponse<E>)>;
pub(crate) type RxChanAppendEntries<D, E> = mpsc::UnboundedReceiver<(AppendEntriesRequest<D>, TxAppendEntriesResponse<E>)>;
pub(crate) type RxChanVote<E> = mpsc::UnboundedReceiver<(VoteRequest, TxVoteResponse<E>)>;
pub(crate) type RxChanInstallSnapshot<E> = mpsc::UnboundedReceiver<(InstallSnapshotRequest, TxInstallSnapshotResponse<E>)>;
pub(crate) type RxChanClient<D, R, E> = mpsc::UnboundedReceiver<(ClientRequest<D>, TxClientResponse<D, R, E>)>;
pub(crate) type RxChanInit<E> = mpsc::UnboundedReceiver<(InitWithConfig, TxInitResponse<E>)>;
pub(crate) type RxChanPropose<E> = mpsc::UnboundedReceiver<(ProposeConfigChange, TxProposeResponse<E>)>;
/// The Raft API.
///
/// This type implements the full Raft spec, and is the interface into a running Raft node.
@ -53,17 +33,12 @@ pub(crate) type RxChanPropose<E> = mpsc::UnboundedReceiver<(ProposeConfigChange,
/// 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, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> {
tx_append_entries: TxChanAppendEntries<D, E>,
tx_vote: TxChanVote<E>,
tx_install_snapshot: TxChanInstallSnapshot<E>,
tx_client: TxChanClient<D, R, E>,
tx_init_with_config: TxChanInit<E>,
tx_propose_config_change: TxChanPropose<E>,
tx_api: mpsc::UnboundedSender<RaftMsg<D, R, E>>,
rx_metrics: watch::Receiver<RaftMetrics>,
raft_handle: JoinHandle<RaftResult<(), E>>,
needs_shutdown: Arc<AtomicBool>,
marker_n: std::marker::PhantomData<N>,
marker_s: std::marker::PhantomData<S>,
needs_shutdown: Arc<AtomicBool>,
}
impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftStorage<D, R, E>> Raft<D, R, E, N, S> {
@ -86,25 +61,17 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// An implementation of the `RaftStorage` trait which will be used by Raft for data storage.
/// See the docs on the `RaftStorage` trait for more details.
pub fn new(id: NodeId, config: Config, network: Arc<N>, storage: Arc<S>) -> Self {
let (tx_append_entries, rx_append_entries) = mpsc::unbounded_channel();
let (tx_vote, rx_vote) = mpsc::unbounded_channel();
let (tx_install_snapshot, rx_install_snapshot) = mpsc::unbounded_channel();
let (tx_client, rx_client) = mpsc::unbounded_channel();
let (tx_init_with_config, rx_init_with_config) = mpsc::unbounded_channel();
let (tx_propose_config_change, rx_propose_config_change) = mpsc::unbounded_channel();
let (tx_api, rx_api) = mpsc::unbounded_channel();
let (tx_metrics, rx_metrics) = watch::channel(RaftMetrics::new_initial(id));
let needs_shutdown = Arc::new(AtomicBool::new(false));
let raft_handle = RaftCore::spawn(
id, config, network, storage,
rx_append_entries, rx_vote, rx_install_snapshot, rx_client,
rx_init_with_config, rx_propose_config_change, tx_metrics,
rx_api, tx_metrics,
needs_shutdown.clone(),
);
Self{
tx_append_entries, tx_vote, tx_install_snapshot, tx_client,
tx_init_with_config, tx_propose_config_change, rx_metrics, raft_handle,
tx_api, rx_metrics, raft_handle, needs_shutdown,
marker_n: std::marker::PhantomData, marker_s: std::marker::PhantomData,
needs_shutdown,
}
}
@ -116,10 +83,10 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// Applications are responsible for implementing a network layer which can receive the RPCs
/// 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, msg))]
pub async fn append_entries(&self, msg: AppendEntriesRequest<D>) -> RaftResult<AppendEntriesResponse, E> {
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn append_entries(&self, rpc: AppendEntriesRequest<D>) -> RaftResult<AppendEntriesResponse, E> {
let (tx, rx) = oneshot::channel();
self.tx_append_entries.send((msg, tx)).map_err(|_| RaftError::ShuttingDown)?;
self.tx_api.send(RaftMsg::AppendEntries{rpc, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| RaftError::ShuttingDown).and_then(|res| res)?)
}
@ -130,10 +97,10 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// Applications are responsible for implementing a network layer which can receive the RPCs
/// 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, msg))]
pub async fn vote(&self, msg: VoteRequest) -> RaftResult<VoteResponse, E> {
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn vote(&self, rpc: VoteRequest) -> RaftResult<VoteResponse, E> {
let (tx, rx) = oneshot::channel();
self.tx_vote.send((msg, tx)).map_err(|_| RaftError::ShuttingDown)?;
self.tx_api.send(RaftMsg::RequestVote{rpc, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| RaftError::ShuttingDown).and_then(|res| res)?)
}
@ -145,10 +112,10 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
/// Applications are responsible for implementing a network layer which can receive the RPCs
/// 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, msg))]
pub async fn install_snapshot(&self, msg: InstallSnapshotRequest) -> RaftResult<InstallSnapshotResponse, E> {
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn install_snapshot(&self, rpc: InstallSnapshotRequest) -> RaftResult<InstallSnapshotResponse, E> {
let (tx, rx) = oneshot::channel();
self.tx_install_snapshot.send((msg, tx)).map_err(|_| RaftError::ShuttingDown)?;
self.tx_api.send(RaftMsg::InstallSnapshot{rpc, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| RaftError::ShuttingDown).and_then(|res| res)?)
}
@ -168,10 +135,10 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
///
/// 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, msg))]
pub async fn client(&self, msg: ClientRequest<D>) -> Result<ClientResponse<R>, ClientError<D, E>> {
#[tracing::instrument(level="debug", skip(self, rpc))]
pub async fn client(&self, rpc: ClientRequest<D>) -> Result<ClientResponse<R>, ClientError<D, E>> {
let (tx, rx) = oneshot::channel();
self.tx_client.send((msg, tx)).map_err(|_| ClientError::RaftError(RaftError::ShuttingDown))?;
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)?)
}
@ -179,7 +146,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
///
/// This command should be called on pristine nodes — where the log index is 0 and the node is
/// in NonVoter state — as if either of those constraints are false, it indicates that the
/// cluster is already formed and in motion. If `InitWithConfigError::NotAllowed` is returned
/// cluster is already formed and in motion. If `InitializeError::NotAllowed` is returned
/// from this function, it is safe to ignore, as it simply indicates that the cluster is
/// already up and running, which is ultimately the goal of this function.
///
@ -205,31 +172,48 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
///
/// 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, msg))]
pub async fn init_with_config(&self, msg: InitWithConfig) -> Result<(), InitWithConfigError<E>> {
#[tracing::instrument(level="debug", skip(self))]
pub async fn initialize(&self, members: HashSet<NodeId>) -> Result<(), InitializeError<E>> {
let (tx, rx) = oneshot::channel();
self.tx_init_with_config.send((msg, tx)).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| InitWithConfigError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
self.tx_api.send(RaftMsg::Initialize{members, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| InitializeError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
}
/// Synchronize a new Raft node, bringing it up-to-speed (§6).
///
/// Applications built on top of Raft will typically have some peer discovery mechanism for
/// detecting when new nodes come online and need to be added to the cluster. This API
/// facilitates the ability to request that a new node be synchronized with the leader, so
/// that it is up-to-date and ready to be added to the cluster.
///
/// 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.
///
/// If this Raft node is not the cluster leader, then this call will fail.
#[tracing::instrument(level="debug", skip(self))]
pub async fn add_non_voter(&self, id: NodeId) -> Result<(), ChangeConfigError<E>> {
let (tx, rx) = oneshot::channel();
self.tx_api.send(RaftMsg::AddNonVoter{id, tx}).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| ChangeConfigError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
}
/// Propose a cluster configuration change (§6).
///
/// Applications built on top of Raft will typically have some peer discovery mechanism for
/// detecting when new nodes are being added to the cluster, or when a node has been offline
/// for some amount of time and should be removed from the cluster (cluster self-healing). Other
/// applications may not have a dynamic membership system, but instead may have a manual method
/// of updating a cluster's membership.
///
/// An all of the above cases (as well as other cases not discussed), this method is used for
/// proposing changes to the cluster's membership configuration.
/// 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
/// 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.
///
/// If this Raft node is not the cluster leader, then the proposed configuration change will be
/// rejected.
#[tracing::instrument(level="debug", skip(self, msg))]
pub async fn propose_config_change(&self, msg: ProposeConfigChange) -> Result<(), ProposeConfigChangeError<E>> {
#[tracing::instrument(level="debug", skip(self))]
pub async fn change_config(&self, members: HashSet<NodeId>) -> Result<(), ChangeConfigError<E>> {
let (tx, rx) = oneshot::channel();
self.tx_propose_config_change.send((msg, tx)).map_err(|_| RaftError::ShuttingDown)?;
Ok(rx.await.map_err(|_| ProposeConfigChangeError::RaftError(RaftError::ShuttingDown)).and_then(|res| res)?)
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)?)
}
/// Get a handle to the metrics channel.
@ -244,6 +228,41 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
}
}
pub(crate) type ClientResponseTx<D, R, E> = oneshot::Sender<Result<ClientResponse<R>, ClientError<D, E>>>;
pub(crate) type ChangeMembershipTx<E> = oneshot::Sender<Result<(), ChangeConfigError<E>>>;
/// A message coming from the Raft API.
pub(crate) enum RaftMsg<D: AppData, R: AppDataResponse, E: AppError> {
AppendEntries{
rpc: AppendEntriesRequest<D>,
tx: oneshot::Sender<Result<AppendEntriesResponse, RaftError<E>>>,
},
RequestVote{
rpc: VoteRequest,
tx: oneshot::Sender<Result<VoteResponse, RaftError<E>>>,
},
InstallSnapshot{
rpc: InstallSnapshotRequest,
tx: oneshot::Sender<Result<InstallSnapshotResponse, RaftError<E>>>,
},
ClientRequest{
rpc: ClientRequest<D>,
tx: ClientResponseTx<D, R, E>,
},
Initialize{
members: HashSet<NodeId>,
tx: oneshot::Sender<Result<(), InitializeError<E>>>,
},
AddNonVoter{
id: NodeId,
tx: ChangeMembershipTx<E>,
},
ChangeMembership{
members: HashSet<NodeId>,
tx: ChangeMembershipTx<E>,
},
}
//////////////////////////////////////////////////////////////////////////////////////////////////
//////////////////////////////////////////////////////////////////////////////////////////////////
@ -311,8 +330,18 @@ pub struct Entry<D: AppData> {
impl<D: AppData> Entry<D> {
/// Create a new snapshot pointer from the given data.
pub fn new_snapshot_pointer(pointer: EntrySnapshotPointer, index: u64, term: u64) -> Self {
Entry{term, index, payload: EntryPayload::SnapshotPointer(pointer)}
///
/// ### index & term
/// The index and term of the entry being replaced by this snapshot pointer entry.
///
/// ### id
/// The ID of the associated snapshot.
///
/// ### membership
/// The cluster membership config which is contained in the snapshot, which will always be the
/// latest membership covered by the snapshot.
pub fn new_snapshot_pointer(index: u64, term: u64, id: String, membership: MembershipConfig) -> Self {
Entry{term, index, payload: EntryPayload::SnapshotPointer(EntrySnapshotPointer{id, membership})}
}
}
@ -354,6 +383,8 @@ pub struct EntryConfigChange {
pub struct EntrySnapshotPointer {
/// The ID of the snapshot, which is application specific, and probably only meaningful to the storage layer.
pub id: String,
/// The cluster's membership config covered by this snapshot.
pub membership: MembershipConfig,
}
//////////////////////////////////////////////////////////////////////////////////////////////////
@ -362,40 +393,52 @@ pub struct EntrySnapshotPointer {
/// A model of the membership configuration of the cluster.
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct MembershipConfig {
/// A flag indicating if the system is currently in a joint consensus state.
pub is_in_joint_consensus: bool,
/// Voting members of the Raft cluster.
pub members: Vec<NodeId>,
/// Non-voting members of the cluster.
/// All members of the Raft cluster.
pub members: HashSet<NodeId>,
/// All members of the Raft cluster after joint consensus is finalized.
///
/// These nodes are being brought up-to-speed by the leader and will be transitioned over to
/// being standard members once they are up-to-date.
pub non_voters: Vec<NodeId>,
/// The set of nodes which are to be removed after joint consensus is complete.
pub removing: Vec<NodeId>,
/// The presence of a value here indicates that the config is in joint consensus.
pub members_after_consensus: Option<HashSet<NodeId>>,
}
impl MembershipConfig {
/// Create a new initial config containing only the given node ID.
pub fn new_initial(id: NodeId) -> Self {
Self{is_in_joint_consensus: false, members: vec![id], non_voters: vec![], removing: vec![]}
/// Get an iterator over all nodes in the current config.
pub fn all_nodes(&self) -> HashSet<u64> {
let mut all = self.members.clone();
if let Some(members) = &self.members_after_consensus {
all.extend(members);
}
all
}
/// Check if the given NodeId exists in this membership config.
///
/// This checks only the contents of `members` & `non_voters`.
/// When in joint consensus, this will check both config groups.
pub fn contains(&self, x: &NodeId) -> bool {
self.members.contains(x) || self.non_voters.contains(x)
self.members.contains(x) || if let Some(members) = &self.members_after_consensus {
members.contains(x)
} else {
false
}
}
/// Get an iterator over all nodes in the current config.
pub fn all_nodes(&self) -> impl Iterator<Item=&NodeId> {
self.members.iter().chain(self.non_voters.iter())
/// Check to see if the config is currently in joint consensus.
pub fn is_in_joint_consensus(&self) -> bool {
self.members_after_consensus.is_some()
}
/// Get the length of the members & non_voters vectors.
/// 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() + self.non_voters.len()
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();
members.insert(id);
Self{members, members_after_consensus: None}
}
}
@ -429,11 +472,6 @@ pub struct VoteResponse {
pub term: u64,
/// Will be true if the candidate received a vote from the responder.
pub vote_granted: bool,
/// Will be true if the candidate is unknown to the responding node's config.
///
/// If this field is true, and the sender's (the candidate's) index is greater than 0, then it
/// should revert to the NonVoter state; if the sender's index is 0, then resume campaigning.
pub is_candidate_unknown: bool,
}
//////////////////////////////////////////////////////////////////////////////////////////////////
@ -551,10 +589,10 @@ pub enum ClientResponse<R: AppDataResponse> {
impl<R: AppDataResponse> ClientResponse<R> {
/// The index of the log entry corresponding to this response object.
pub fn index(&self) -> u64 {
pub fn index(&self) -> &u64 {
match self {
Self::Committed{index} => *index,
Self::Applied{index, ..} => *index,
Self::Committed{index} => index,
Self::Applied{index, ..} => index,
}
}
@ -566,48 +604,3 @@ impl<R: AppDataResponse> ClientResponse<R> {
}
}
}
//////////////////////////////////////////////////////////////////////////////////////////////////
//////////////////////////////////////////////////////////////////////////////////////////////////
/// The data model used for initializing a pristine Raft node.
pub struct InitWithConfig {
/// All currently known members to initialize the new cluster with.
///
/// If the ID of the node this command is being submitted to is not present, it will be added.
/// If there are duplicates, they will be filtered out to ensure config is proper.
pub(crate) members: Vec<NodeId>,
}
impl InitWithConfig {
/// Construct a new instance.
pub fn new(members: Vec<NodeId>) -> Self {
Self{members}
}
}
//////////////////////////////////////////////////////////////////////////////////////////////////
//////////////////////////////////////////////////////////////////////////////////////////////////
/// Propose a new membership config change to a running cluster.
///
/// There are a few invariants which must be upheld here:
///
/// - if the node this command is sent to is not the leader of the cluster, it will be rejected.
/// - if the given changes would leave the cluster in an inoperable state, it will be rejected.
pub struct ProposeConfigChange {
/// New members to be added to the cluster.
pub(crate) add_members: Vec<NodeId>,
/// Members to be removed from the cluster.
pub(crate) remove_members: Vec<NodeId>,
}
impl ProposeConfigChange {
/// Create a new instance.
///
/// If there are duplicates in either of the givenn vectors, they will be filtered out to
/// ensure config is proper.
pub fn new(add_members: Vec<NodeId>, remove_members: Vec<NodeId>) -> Self {
Self{add_members, remove_members}
}
}

View File

@ -137,7 +137,7 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
let this = Self{
id, target, term, network, storage, config, max_payload_entries,
marker_r: std::marker::PhantomData, marker_e: std::marker::PhantomData,
target_state: TargetReplState::LineRate, last_log_index, commit_index,
target_state: TargetReplState::Lagging, last_log_index, commit_index,
next_index: last_log_index + 1, match_index: last_log_index, match_term: last_log_term,
rafttx, raftrx, heartbeat: interval(heartbeat_timeout), heartbeat_timeout,
replication_buffer: Vec::new(), outbound_buffer: Vec::new(),
@ -148,6 +148,10 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
#[tracing::instrument(level="trace", skip(self), fields(id=self.id, target=self.target))]
async fn main(mut self) {
// Perform an initial heartbeat.
self.send_append_entries().await;
// Proceed to the replication stream's inner loop.
loop {
match &self.target_state {
TargetReplState::LineRate => LineRateState::new(&mut self).run().await,
@ -251,6 +255,8 @@ impl<D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: RaftS
}
Err(err) => {
tracing::error!({error=%err}, "error fetching log entry due to returned AppendEntries RPC conflict_opt");
let _ = self.rafttx.send(ReplicaEvent::Shutdown);
self.target_state = TargetReplState::Shutdown;
return;
}
};
@ -348,7 +354,7 @@ impl<D: AppData> AsRef<Entry<D>> for OutboundEntry<D> {
enum TargetReplState {
/// The replication stream is running at line rate.
LineRate,
/// The replication stream is lagging behind due to the target node.
/// The replication stream is lagging behind.
Lagging,
/// The replication stream is streaming a snapshot over to the target node.
Snapshotting,
@ -411,6 +417,8 @@ pub(crate) enum ReplicaEvent<S>
/// The response channel for delivering the snapshot data.
tx: oneshot::Sender<CurrentSnapshotData<S>>,
},
/// Some critical error has taken place, and Raft needs to shutdown.
Shutdown,
}
//////////////////////////////////////////////////////////////////////////////////////////////////
@ -443,7 +451,10 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
}
tokio::select!{
_ = self.core.heartbeat.next() => self.core.send_append_entries().await,
Some(event) = self.core.raftrx.next() => self.core.drain_raftrx(event),
event = self.core.raftrx.next() => match event {
Some(event) => self.core.drain_raftrx(event),
None => self.core.target_state = TargetReplState::Shutdown,
}
}
}
}
@ -528,6 +539,8 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
Ok(entries) => entries,
Err(err) => {
tracing::error!({error=%err}, "error fetching logs from storage");
let _ = self.core.rafttx.send(ReplicaEvent::Shutdown);
self.core.target_state = TargetReplState::Shutdown;
return;
}
};
@ -604,7 +617,13 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
loop {
tokio::select!{
_ = self.core.heartbeat.next() => self.core.send_append_entries().await,
Some(event) = self.core.raftrx.next() => self.core.drain_raftrx(event),
event = self.core.raftrx.next() => match event {
Some(event) => self.core.drain_raftrx(event),
None => {
self.core.target_state = TargetReplState::Shutdown;
return;
}
},
res = &mut rx => {
match res {
Ok(snapshot) => {
@ -613,7 +632,7 @@ impl<'a, D: AppData, R: AppDataResponse, E: AppError, N: RaftNetwork<D, E>, S: R
}
Err(_) => return, // Channels may close for various acceptable reasons.
}
}
},
}
}
}

View File

@ -6,7 +6,7 @@ use std::io::Cursor;
use async_raft::async_trait;
use async_raft::{AppData, AppDataResponse, AppError, NodeId, RaftStorage};
use async_raft::raft::{Entry, EntryPayload, EntrySnapshotPointer, MembershipConfig};
use async_raft::raft::{Entry, EntryPayload, MembershipConfig};
use async_raft::storage::{CurrentSnapshotData, HardState, InitialState};
use serde::{Serialize, Deserialize};
use thiserror::Error;
@ -243,7 +243,7 @@ impl RaftStorage<MemStoreData, MemStoreDataResponse, MemStoreError> for MemStore
let mut current_snapshot = self.current_snapshot.write().await;
term = log.get(&through).map(|entry| entry.term).ok_or_else(|| MemStoreError::InconsistentLog)?;
*log = log.split_off(&through);
log.insert(through, Entry::new_snapshot_pointer(EntrySnapshotPointer{id: "".into()}, through, term));
log.insert(through, Entry::new_snapshot_pointer(through, term, "".into(), membership_config.clone()));
snapshot = MemStoreSnapshot{index: through, term, membership: membership_config.clone(), data};
*current_snapshot = Some(snapshot.clone());
@ -267,14 +267,23 @@ impl RaftStorage<MemStoreData, MemStoreDataResponse, MemStoreError> for MemStore
.map_err(|err| MemStoreError::SnapshotCborError(err.to_string()))?;
// Update log.
{
// Go backwards through the log to find the most recent membership config <= the `through` index.
let mut log = self.log.write().await;
let membership_config = log.values().rev()
.skip_while(|entry| &entry.index > &index)
.find_map(|entry| match &entry.payload {
EntryPayload::ConfigChange(cfg) => Some(cfg.membership.clone()),
_ => None,
})
.unwrap_or_else(|| MembershipConfig::new_initial(self.id));
match &delete_through {
Some(through) => {
*log = log.split_off(&(through + 1));
}
None => log.clear(),
}
log.insert(index, Entry::new_snapshot_pointer(EntrySnapshotPointer{id}, index, term));
log.insert(index, Entry::new_snapshot_pointer(index, term, id, membership_config));
}
// Update current snapshot.

View File

@ -168,5 +168,5 @@ The Raft spec states explicitly that the index is a monotonically increasing val
Sticking with a uniform, monotonically increasing index value, which does not reset per term, is the recommended approach.
### does the definition of committment include the leader itself?
### does the definition of commitment include the leader itself?
The leader is included in the calculation of majorities, except in the case of joint consensus for the specific case where the leader is not part of the new configuration.