Expand description
Creating a Raft node
You can use RawNode::new
to create the Raft node. To create the Raft node, you need to
provide a Storage
component, and a Config
to the RawNode::new
function.
use raft::{
Config,
storage::MemStorage,
raw_node::RawNode,
};
use slog::{Drain, o};
// Select some defaults, then change what we need.
let config = Config {
id: 1,
..Default::default()
};
// Initialize logger.
let logger = slog::Logger::root(slog_stdlog::StdLog.fuse(), o!());
// ... Make any configuration changes.
// After, make sure it's valid!
config.validate().unwrap();
// We'll use the built-in `MemStorage`, but you will likely want your own.
// Finally, create our Raft node!
let storage = MemStorage::new_with_conf_state((vec![1], vec![]));
let mut node = RawNode::new(&config, storage, &logger).unwrap();
Ticking the Raft node
Use a timer to tick the Raft node at regular intervals. See the following example using Rust
channel recv_timeout
to drive the Raft node at least every 100ms, calling
tick()
each time.
use std::{sync::mpsc::{channel, RecvTimeoutError}, time::{Instant, Duration}};
// We're using a channel, but this could be any stream of events.
let (tx, rx) = channel();
let timeout = Duration::from_millis(100);
let mut remaining_timeout = timeout;
// Send the `tx` somewhere else...
loop {
let now = Instant::now();
match rx.recv_timeout(remaining_timeout) {
Ok(()) => {
// Let's save this for later.
unimplemented!()
},
Err(RecvTimeoutError::Timeout) => (),
Err(RecvTimeoutError::Disconnected) => unimplemented!(),
}
let elapsed = now.elapsed();
if elapsed >= remaining_timeout {
remaining_timeout = timeout;
// We drive Raft every 100ms.
node.tick();
} else {
remaining_timeout -= elapsed;
}
}
Proposing to, and stepping the Raft node
Using the propose
function you can drive the Raft node when the client sends a request to the
Raft server. You can call propose
to add the request to the Raft log explicitly.
In most cases, the client needs to wait for a response for the request. For example, if the client writes a value to a key and wants to know whether the write succeeds or not, but the write flow is asynchronous in Raft, so the write log entry must be replicated to other followers, then committed and at last applied to the state machine, so here we need a way to notify the client after the write is finished.
One simple way is to use a unique ID for the client request, and save the associated callback function in a hash map. When the log entry is applied, we can get the ID from the decoded entry, call the corresponding callback, and notify the client.
You can call the step
function when you receive the Raft messages from other nodes.
Here is a simple example to use propose
and step
:
enum Msg {
Propose {
id: u8,
callback: Box<dyn Fn() + Send>,
},
Raft(Message),
}
// Simulate a message coming down the stream.
tx.send(Msg::Propose { id: 1, callback: Box::new(|| ()) });
let mut cbs = HashMap::new();
loop {
let now = Instant::now();
match rx.recv_timeout(remaining_timeout) {
Ok(Msg::Propose { id, callback }) => {
cbs.insert(id, callback);
node.propose(vec![], vec![id]).unwrap();
}
Ok(Msg::Raft(m)) => node.step(m).unwrap(),
Err(RecvTimeoutError::Timeout) => (),
Err(RecvTimeoutError::Disconnected) => unimplemented!(),
}
let elapsed = now.elapsed();
if elapsed >= remaining_timeout {
remaining_timeout = timeout;
// We drive Raft every 100ms.
node.tick();
} else {
remaining_timeout -= elapsed;
}
break;
}
In the above example, we use a channel to receive the propose
and step
messages. We only
propose the request ID to the Raft log. In your own practice, you can embed the ID in your request
and propose the encoded binary request data.
Processing the Ready
State
When your Raft node is ticked and running, Raft should enter a Ready
state. You need to first use
has_ready
to check whether Raft is ready. If yes, use the ready
function to get a Ready
state:
if !node.has_ready() {
return;
}
// The Raft is ready, we can do something now.
let mut ready = node.ready();
The Ready
state contains quite a bit of information, and you need to check and process them one
by one:
-
Check whether
messages
is empty or not. If not, it means that the node will send messages to other nodes:if !ready.messages().is_empty() { for msg in ready.take_messages() { // Send messages to other peers. } }
-
Check whether
snapshot
is empty or not. If not empty, it means that the Raft node has received a Raft snapshot from the leader and we must apply the snapshot:if !ready.snapshot().is_empty() { // This is a snapshot, we need to apply the snapshot at first. node.mut_store() .wl() .apply_snapshot(ready.snapshot().clone()) .unwrap(); }
-
Check whether
committed_entries
is empty or not. If not, it means that there are some newly committed log entries which you must apply to the state machine. Of course, after applying, you need to update the applied index and resumeapply
later:let mut _last_apply_index = 0; for entry in ready.take_committed_entries() { // Mostly, you need to save the last apply index to resume applying // after restart. Here we just ignore this because we use a Memory storage. _last_apply_index = entry.index; if entry.data.is_empty() { // Emtpy entry, when the peer becomes Leader it will send an empty entry. continue; } match entry.get_entry_type() { EntryType::EntryNormal => handle_normal(entry), // It's recommended to always use `EntryType::EntryConfChangeV2. EntryType::EntryConfChange => handle_conf_change(entry), EntryType::EntryConfChangeV2 => handle_conf_change_v2(entry), } }
Note, although Raft guarentees only persisted committed entries will be applied, but it doesn’t guarentee commit index is persisted before being applied. For example, if application is restarted after applying committed entries before persisting commit index, apply index can be larger than commit index and cause panic. To solve the problem, persisting commit index with or before applying entries. You can also always assign commit index to the
max(commit_index, applied_index)
after restarting, it may work but potential log loss may also be ignored silently. -
Check whether
entries
is empty or not. If not empty, it means that there are newly added entries but have not been committed yet, we must append the entries to the Raft log:if !ready.entries().is_empty() { // Append entries to the Raft log node.mut_store().wl().append(ready.entries()).unwrap(); }
-
Check whether
hs
is empty or not. If not empty, it means that theHardState
of the node has changed. For example, the node may vote for a new leader, or the commit index has been increased. We must persist the changedHardState
:if let Some(hs) = ready.hs() { // Raft HardState changed, and we need to persist it. node.mut_store().wl().set_hardstate(hs.clone()); }
-
Check whether
persisted_messages
is empty or not. If not, it means that the node will send messages to other nodes after persisting hardstate, entries and snapshot:if !ready.persisted_messages().is_empty() { for msg in ready.take_persisted_messages() { // Send persisted messages to other peers. } }
-
Call
advance
to notify that the previous work is completed. Get the return valueLightReady
and handle itsmessages
andcommitted_entries
like step 1 and step 3 does. Then calladvance_apply
to advance the applied index inside.let mut light_rd = node.advance(ready); // Like step 1 and 3, you can use functions to make them behave the same. handle_messages(light_rd.take_messages()); handle_committed_entries(light_rd.take_committed_entries()); node.advance_apply();
For more information, check out an example.
Sometimes it’s better not to block the raft machine in IO operation, so that latency of read/write can be more predictable and the fsync frequencey can be controlled. The crate supports async ready to offload the IO operation to other thread. The usage is the same as above except:
- All writes are not required to be persisted immediately, they can be written into memory caches;
- Persisted messages should be sent after all coresponding writes are persisted;
advance_append_async
is used when all writes are finished instead ofadvance/advance_append
.- Only persisted entries can be committed and applied, so to make progress, all writes should be persisted at some point.
Arbitrary Membership Changes
When building a resilient, scalable distributed system there is a strong need to be able to change the membership of a peer group dynamically, without downtime. This Raft crate supports this via Joint Consensus (Raft paper, section 6).
It permits resilient arbitrary dynamic membership changes. A membership change can do any or all of the following:
- Add peer (learner or voter) n to the group.
- Remove a learner n from the group.
- Promote a learner to a voter.
- Demote a voter back to learner.
- Replace a node n with another node m.
For example to promote a learner 4 and demote an existing voter 3:
let steps = vec![
raft_proto::new_conf_change_single(4, ConfChangeType::AddNode),
raft_proto::new_conf_change_single(3, ConfChangeType::RemoveNode),
];
let mut cc = ConfChangeV2::default();
cc.set_changes(steps.into());
node.propose_conf_change(vec![], cc).unwrap();
// After the log is committed and applied
// node.apply_conf_change(&cc).unwrap();
This process is a two-phase process, during the midst of it the peer group’s leader is managing two independent, possibly overlapping peer sets.
Note: In order to maintain resiliency guarantees (progress while a majority of both peer sets is active), it is recommended to wait until the entire peer group has exited the transition phase before taking old, removed peers offline.
Re-exports
pub use raft_log::NO_LIMIT;
pub use raw_node::is_empty_snap;
Deprecatedpub use raw_node::LightReady;
pub use raw_node::Peer;
pub use raw_node::RawNode;
pub use raw_node::Ready;
pub use raw_node::SnapshotStatus;
pub use storage::GetEntriesContext;
pub use storage::RaftState;
pub use storage::Storage;
pub use util::majority;
Modules
- Generated file from
eraftpb.proto
- A “prelude” for crates using the
raft
crate. - The raw node of the raft module.
- Represents the storage trait and example implementation.
- This module contains a collection of various tools to use to manipulate and control messages and data associated with raft.
Structs
- Changer facilitates configuration changes. It exposes methods to handle simple and joint consensus while performing the proper validation that allows refusing invalid configuration changes before they affect the active configuration.
- Config contains the parameters to start a raft.
- A buffer of inflight messages.
- A configuration of two groups of (possibly overlapping) majority configurations. Decisions require the support of both majorities.
- A set of IDs that uses majority quorums to make decisions.
- The progress of catching up from a restart.
ProgressTracker
contains severalProgress
es, which could beLeader
,Follower
andLearner
.- A struct that represents the raft consensus itself. Stores details concerning the current and possible state the system can take.
- Raft log implementation
- ReadState provides state for read only query. It’s caller’s responsibility to send MsgReadIndex first before getting this state from ready. It’s also caller’s duty to differentiate if this state is what it requests through request_ctx, e.g. given a unique id as request_ctx.
- SoftState provides state that is useful for logging and debugging. The state is volatile and does not need to be persisted to the WAL.
- Represents the current status of the raft
- The
unstable.entries[i]
has raft log positioni+unstable.offset
. Note thatunstable.offset
may be less than the highest log position in storage; this means that the next write to storage might need to truncate the log before persisting unstable.entries.
Enums
- The base error type for raft
- The state of the progress.
- Determines the relative safety of and consistency of read only requests.
- The role of the node.
- An error with the storage.
Constants
- A constant represents invalid id of raft.
- A constant represents invalid index of raft log.
Functions
- The default logger we fall back to when passed
None
in external facing constructors. - Maps vote and pre_vote message types to their correspond responses.
Type Definitions
- Changes made by
Changer
. - A result type that wraps up the raft errors.