|
|
@@ -16,6 +16,7 @@ use crossbeam_utils::sync::WaitGroup;
|
|
|
use parking_lot::{Condvar, Mutex};
|
|
|
use rand::{thread_rng, Rng};
|
|
|
|
|
|
+use crate::install_snapshot::InstallSnapshotArgs;
|
|
|
use crate::persister::PersistedRaftState;
|
|
|
pub use crate::persister::Persister;
|
|
|
pub(crate) use crate::raft_state::RaftState;
|
|
|
@@ -320,6 +321,12 @@ where
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+enum SyncLogEntryOperation<Command> {
|
|
|
+ AppendEntries(AppendEntriesArgs<Command>),
|
|
|
+ InstallSnapshot(InstallSnapshotArgs),
|
|
|
+ None,
|
|
|
+}
|
|
|
+
|
|
|
// Command must be
|
|
|
// 0. 'static: Raft<Command> must be 'static, it is moved to another thread.
|
|
|
// 1. clone: they are copied to the persister.
|
|
|
@@ -685,13 +692,26 @@ where
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- let args = match Self::build_append_entries(&rf, peer_index) {
|
|
|
- Some(args) => args,
|
|
|
- None => return,
|
|
|
+ let operation = Self::build_sync_log_entry(&rf, peer_index);
|
|
|
+ let (term, match_index, succeeded) = match operation {
|
|
|
+ SyncLogEntryOperation::AppendEntries(args) => {
|
|
|
+ let term = args.term;
|
|
|
+ let match_index = args.prev_log_index + args.entries.len();
|
|
|
+ let succeeded = Self::append_entries(&rpc_client, args).await;
|
|
|
+
|
|
|
+ (term, match_index, succeeded)
|
|
|
+ }
|
|
|
+ SyncLogEntryOperation::InstallSnapshot(args) => {
|
|
|
+ let term = args.term;
|
|
|
+ let match_index = args.last_included_index;
|
|
|
+ let succeeded =
|
|
|
+ Self::send_install_snapshot(&rpc_client, args).await;
|
|
|
+
|
|
|
+ (term, match_index, succeeded)
|
|
|
+ }
|
|
|
+ SyncLogEntryOperation::None => return,
|
|
|
};
|
|
|
- let term = args.term;
|
|
|
- let match_index = args.prev_log_index + args.entries.len();
|
|
|
- let succeeded = Self::append_entries(&rpc_client, args).await;
|
|
|
+
|
|
|
match succeeded {
|
|
|
Ok(Some(true)) => {
|
|
|
let mut rf = rf.lock();
|
|
|
@@ -750,24 +770,43 @@ where
|
|
|
};
|
|
|
}
|
|
|
|
|
|
- fn build_append_entries(
|
|
|
+ fn build_sync_log_entry(
|
|
|
rf: &Mutex<RaftState<Command>>,
|
|
|
peer_index: usize,
|
|
|
- ) -> Option<AppendEntriesArgs<Command>> {
|
|
|
+ ) -> SyncLogEntryOperation<Command> {
|
|
|
let rf = rf.lock();
|
|
|
if !rf.is_leader() {
|
|
|
- return None;
|
|
|
+ return SyncLogEntryOperation::None;
|
|
|
}
|
|
|
+
|
|
|
+ // To send AppendEntries request, next_index must be strictly larger
|
|
|
+ // than start(). Otherwise we won't be able to know the log term of the
|
|
|
+ // entry right before next_index.
|
|
|
+ return if rf.next_index[peer_index] > rf.log.start() {
|
|
|
+ SyncLogEntryOperation::AppendEntries(Self::build_append_entries(
|
|
|
+ &rf, peer_index,
|
|
|
+ ))
|
|
|
+ } else {
|
|
|
+ SyncLogEntryOperation::InstallSnapshot(
|
|
|
+ Self::build_install_snapshot(&rf),
|
|
|
+ )
|
|
|
+ };
|
|
|
+ }
|
|
|
+
|
|
|
+ fn build_append_entries(
|
|
|
+ rf: &RaftState<Command>,
|
|
|
+ peer_index: usize,
|
|
|
+ ) -> AppendEntriesArgs<Command> {
|
|
|
let prev_log_index = rf.next_index[peer_index] - 1;
|
|
|
let prev_log_term = rf.log[prev_log_index].term;
|
|
|
- Some(AppendEntriesArgs {
|
|
|
+ AppendEntriesArgs {
|
|
|
term: rf.current_term,
|
|
|
leader_id: rf.leader_id,
|
|
|
prev_log_index,
|
|
|
prev_log_term,
|
|
|
entries: rf.log.after(rf.next_index[peer_index]).to_vec(),
|
|
|
leader_commit: rf.commit_index,
|
|
|
- })
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
const APPEND_ENTRIES_RETRY: usize = 1;
|