Implement committal
This commit is contained in:
261
Raft/Server.fs
261
Raft/Server.fs
@@ -23,7 +23,7 @@ type LeaderState =
|
||||
/// For each server, index of the next log entry to send to that server
|
||||
NextIndex : int<LogIndex> array
|
||||
/// For each server, index of the highest log entry known to be replicated on that server
|
||||
MatchIndex : int array
|
||||
MatchIndex : int<LogIndex> array
|
||||
}
|
||||
|
||||
static member New (clusterSize : int) (currentIndex : int<LogIndex>) : LeaderState =
|
||||
@@ -55,7 +55,7 @@ type RequestVoteMessage =
|
||||
{
|
||||
CandidateTerm : int<Term>
|
||||
CandidateId : int<ServerId>
|
||||
CandidateLastLogEntry : int<LogIndex> * int<Term>
|
||||
CandidateLastLogEntry : LogEntry option
|
||||
ReplyChannel : RequestVoteReply -> unit
|
||||
}
|
||||
|
||||
@@ -65,21 +65,25 @@ type RequestVoteMessage =
|
||||
/// I, a follower, acknowledge the leader's instruction to add an entry to my log.
|
||||
type AppendEntriesReply =
|
||||
{
|
||||
/// Me, the follower who is replying
|
||||
Follower : int<ServerId>
|
||||
/// The term I, the follower, think it is
|
||||
FollowerTerm : int<Term>
|
||||
/// Reply with failure if the follower thinks the leader is out of date:
|
||||
/// Reply with None if the follower thinks the leader is out of date:
|
||||
/// that is, if the leader's believed term is smaller than the follower's,
|
||||
/// or if the leader's declared previous log entry doesn't exist on the follower.
|
||||
Success : bool
|
||||
}
|
||||
|
||||
type LogEntry =
|
||||
{
|
||||
Index : int<LogIndex>
|
||||
Term : int<Term>
|
||||
/// If instead we accepted the update, this is the current head of the follower's log
|
||||
/// after accepting the update.
|
||||
Success : int<LogIndex> option
|
||||
}
|
||||
|
||||
override this.ToString () =
|
||||
sprintf "Log entry %i at subjective term %i" this.Index this.Term
|
||||
let description =
|
||||
match this.Success with
|
||||
| Some index -> sprintf "successfully applied leader entry, log length %i" index
|
||||
| None -> "did not apply leader entry"
|
||||
|
||||
sprintf "Follower %i %s" this.FollowerTerm description
|
||||
|
||||
/// I am the leader. Followers, update your state as follows.
|
||||
type AppendEntriesMessage<'a> =
|
||||
@@ -89,7 +93,7 @@ type AppendEntriesMessage<'a> =
|
||||
/// I am your leader! This is me! (so everyone knows where to send clients to)
|
||||
LeaderId : int<ServerId>
|
||||
/// The entry immediately preceding the entry I'm sending you, so you can tell if we've got out of sync.
|
||||
PrevLogEntry : LogEntry
|
||||
PrevLogEntry : LogEntry option
|
||||
/// Followers, append this entry to your log. (Or, if None, this is just a heartbeat.)
|
||||
/// It was determined at the given term - recall that we might need to bring a restarted node up to speed
|
||||
/// with what happened during terms that took place while it was down.
|
||||
@@ -127,10 +131,12 @@ type Instruction<'a> =
|
||||
|
||||
type Reply =
|
||||
| RequestVoteReply of RequestVoteReply
|
||||
| AppendEntriesReply of AppendEntriesReply
|
||||
|
||||
override this.ToString () =
|
||||
match this with
|
||||
| RequestVoteReply v -> v.ToString ()
|
||||
| AppendEntriesReply r -> r.ToString ()
|
||||
|
||||
type Message<'a> =
|
||||
| Instruction of Instruction<'a>
|
||||
@@ -174,9 +180,33 @@ type ServerStatus =
|
||||
|
||||
type private ServerAction<'a> =
|
||||
| BeginElection
|
||||
| EmitHeartbeat
|
||||
| Receive of Message<'a>
|
||||
| Sync of AsyncReplyChannel<unit>
|
||||
|
||||
[<RequireQualifiedAccess>]
|
||||
module internal ServerUtils =
|
||||
|
||||
/// Return the maximum log index which a quorum has committed.
|
||||
/// Recall that 0 means "nothing committed".
|
||||
let maxLogAQuorumHasCommitted (matchIndex : int<LogIndex>[]) : int<LogIndex> =
|
||||
let numberWhoCommittedIndex = matchIndex |> Array.countBy id
|
||||
|
||||
numberWhoCommittedIndex |> Array.sortInPlaceBy fst
|
||||
|
||||
let rec go (numberCounted : int) (result : int<LogIndex>) (i : int) =
|
||||
if i < 0 then
|
||||
result
|
||||
else
|
||||
let numberCounted = numberCounted + snd numberWhoCommittedIndex.[i]
|
||||
|
||||
if numberCounted > matchIndex.Length / 2 then
|
||||
fst numberWhoCommittedIndex.[i]
|
||||
else
|
||||
go numberCounted (fst numberWhoCommittedIndex.[i]) (i - 1)
|
||||
|
||||
go 0 0<LogIndex> (numberWhoCommittedIndex.Length - 1)
|
||||
|
||||
type Server<'a>
|
||||
(
|
||||
clusterSize : int,
|
||||
@@ -195,46 +225,46 @@ type Server<'a>
|
||||
// We're definitely out of date. Switch to follower mode.
|
||||
currentType <- ServerSpecialisation.Follower
|
||||
persistentState.AdvanceToTerm message.Term
|
||||
// TODO - `DropStaleResponse` suggests we should do this
|
||||
//elif message.Term < persistentState.CurrentTerm then
|
||||
// // Drop the message, it's old.
|
||||
// ()
|
||||
//else
|
||||
|
||||
match message with
|
||||
| RequestVote message ->
|
||||
// This was guaranteed above.
|
||||
assert (message.CandidateTerm <= persistentState.CurrentTerm)
|
||||
|
||||
// The following clauses define either condition under which we accept that the candidate has more data
|
||||
// The following clause defines the condition under which we accept that the candidate has more data
|
||||
// than we do, and so could be a more suitable leader than us.
|
||||
|
||||
// TODO collapse these clauses, it'll be much neater
|
||||
|
||||
let messageSupersedesMe =
|
||||
// Is the candidate advertising a later term than our last-persisted write was made at?
|
||||
// (That would mean it's far in the future of us.)
|
||||
match persistentState.GetLastLogEntry () with
|
||||
| Some (_, (_, ourLastTerm)) -> snd message.CandidateLastLogEntry > ourLastTerm
|
||||
| None ->
|
||||
// We have persisted no history at all!
|
||||
// (This is the `logOk` of the paper.)
|
||||
let candidateSupersedesMe =
|
||||
match persistentState.GetLastLogEntry (), message.CandidateLastLogEntry with
|
||||
| Some (_, ourLastEntry), Some candidateLastLogEntry ->
|
||||
// The candidate wins if:
|
||||
// * it's from so far in the future that an election has happened which we haven't heard about; or
|
||||
// * it's from the same term as us, but it's logged more than we have.
|
||||
candidateLastLogEntry.Term > ourLastEntry.Term
|
||||
|| (candidateLastLogEntry.Term = ourLastEntry.Term
|
||||
&& candidateLastLogEntry.Index >= persistentState.CurrentLogIndex)
|
||||
| Some _, None ->
|
||||
// We've logged something, and they have not. We're ahead, and won't vote for them.
|
||||
false
|
||||
| None, _ ->
|
||||
// We have persisted no history at all.
|
||||
// They take precedence - either they've logged something (in which case they must be ahead of us),
|
||||
// or they haven't logged anything (in which case we'll defer to them for beating us to the
|
||||
// first election).
|
||||
true
|
||||
|
||||
let messageExtendsMe =
|
||||
// Do we agree what the current term is, is the candidate advertising a more advanced log than us?
|
||||
match persistentState.GetLastLogEntry () with
|
||||
| Some (_, (_, ourLastTerm)) ->
|
||||
snd message.CandidateLastLogEntry = ourLastTerm
|
||||
&& fst message.CandidateLastLogEntry >= persistentState.CurrentLogIndex
|
||||
| None ->
|
||||
// We've persisted no history; the candidate needs to also be at the start of history,
|
||||
// or else we'd have already considered them in the `messageSupersedesMe` check.
|
||||
snd message.CandidateLastLogEntry = 0<Term>
|
||||
|
||||
// (This is the `logOk` of the paper. It's true iff the candidate's declaration is ahead of us.)
|
||||
let candidateIsAhead = messageSupersedesMe || messageExtendsMe
|
||||
|
||||
// But just because the candidate is ahead of us, doesn't mean we can vote for them.
|
||||
// We can only vote for one candidate per election.
|
||||
// (We can't rely on our own VotedFor property, because that may have been in a previous election.)
|
||||
|
||||
let shouldVoteFor =
|
||||
if message.CandidateTerm = persistentState.CurrentTerm && candidateIsAhead then
|
||||
if message.CandidateTerm = persistentState.CurrentTerm && candidateSupersedesMe then
|
||||
// We agree on which election we're taking part in, and moreover we agree that the candidate is
|
||||
// suitable.
|
||||
match persistentState.VotedFor with
|
||||
@@ -278,7 +308,8 @@ type Server<'a>
|
||||
// Reject the request: the "leader" is actually outdated, it was only a leader in the past.
|
||||
{
|
||||
FollowerTerm = persistentState.CurrentTerm
|
||||
Success = false
|
||||
Success = None
|
||||
Follower = me
|
||||
}
|
||||
|> message.ReplyChannel
|
||||
|
||||
@@ -295,8 +326,9 @@ type Server<'a>
|
||||
}
|
||||
|
||||
{
|
||||
Success = true
|
||||
Success = Some persistentState.CurrentLogIndex
|
||||
FollowerTerm = persistentState.CurrentTerm
|
||||
Follower = me
|
||||
}
|
||||
|> message.ReplyChannel
|
||||
|
||||
@@ -308,7 +340,10 @@ type Server<'a>
|
||||
|
||||
| Some (toInsert, toInsertTerm) ->
|
||||
|
||||
let desiredLogInsertionPosition = message.PrevLogEntry.Index + 1<LogIndex>
|
||||
let desiredLogInsertionPosition =
|
||||
match message.PrevLogEntry with
|
||||
| None -> 1<LogIndex>
|
||||
| Some entry -> entry.Index + 1<LogIndex>
|
||||
|
||||
match persistentState.GetLogEntry desiredLogInsertionPosition with
|
||||
| Some (_, existingTerm) when toInsertTerm = existingTerm ->
|
||||
@@ -322,8 +357,9 @@ type Server<'a>
|
||||
persistentState.AppendToLog toInsert toInsertTerm
|
||||
|
||||
{
|
||||
Success = true
|
||||
Success = Some desiredLogInsertionPosition
|
||||
FollowerTerm = persistentState.CurrentTerm
|
||||
Follower = me
|
||||
}
|
||||
|> message.ReplyChannel
|
||||
|
||||
@@ -336,25 +372,27 @@ type Server<'a>
|
||||
persistentState.AppendToLog toInsert toInsertTerm
|
||||
|
||||
{
|
||||
Success = true
|
||||
Success = Some desiredLogInsertionPosition
|
||||
FollowerTerm = persistentState.CurrentTerm
|
||||
Follower = me
|
||||
}
|
||||
|> message.ReplyChannel
|
||||
|
||||
let logIsConsistent (message : AppendEntriesMessage<'a>) : bool =
|
||||
if message.PrevLogEntry.Index = 0<LogIndex> then
|
||||
match message.PrevLogEntry with
|
||||
| None ->
|
||||
// The leader advertises that they have no committed history, so certainly it's consistent with
|
||||
// us.
|
||||
true
|
||||
else
|
||||
| Some entry ->
|
||||
|
||||
match persistentState.GetLogEntry message.PrevLogEntry.Index with
|
||||
match persistentState.GetLogEntry entry.Index with
|
||||
| None ->
|
||||
// The leader's advertised commit is ahead of our history.
|
||||
false
|
||||
| Some (_, ourTermForThisEntry) ->
|
||||
// The leader's advertised commit is in our history; do we agree with it?
|
||||
ourTermForThisEntry = message.PrevLogEntry.Term
|
||||
ourTermForThisEntry = entry.Term
|
||||
|
||||
match currentType with
|
||||
| ServerSpecialisation.Leader _ ->
|
||||
@@ -373,7 +411,8 @@ type Server<'a>
|
||||
// Reject the request, it's inconsistent with our history.
|
||||
{
|
||||
FollowerTerm = persistentState.CurrentTerm
|
||||
Success = false
|
||||
Success = None
|
||||
Follower = me
|
||||
}
|
||||
|> message.ReplyChannel
|
||||
|
||||
@@ -388,6 +427,72 @@ type Server<'a>
|
||||
assert (logIsConsistent message)
|
||||
acceptRequest ()
|
||||
|
||||
let divideByTwoRoundingUp (n : int) =
|
||||
if n % 2 = 0 then n / 2 else (n / 2) + 1
|
||||
|
||||
let processReply (r : Reply) : unit =
|
||||
match r with
|
||||
| AppendEntriesReply appendEntriesReply ->
|
||||
match currentType with
|
||||
| ServerSpecialisation.Candidate _
|
||||
| ServerSpecialisation.Follower -> ()
|
||||
| ServerSpecialisation.Leader leaderState ->
|
||||
|
||||
if appendEntriesReply.FollowerTerm = persistentState.CurrentTerm then
|
||||
match appendEntriesReply.Success with
|
||||
| Some matchIndex ->
|
||||
leaderState.MatchIndex.[appendEntriesReply.Follower / 1<ServerId>] <- matchIndex
|
||||
leaderState.NextIndex.[appendEntriesReply.Follower / 1<ServerId>] <- matchIndex + 1<LogIndex>
|
||||
| None ->
|
||||
leaderState.NextIndex.[appendEntriesReply.Follower / 1<ServerId>] <-
|
||||
max
|
||||
(leaderState.NextIndex.[appendEntriesReply.Follower / 1<ServerId>] - 1<LogIndex>)
|
||||
1<LogIndex>
|
||||
// Note that the decision to process this *here* means the algorithm doesn't work in clusters of
|
||||
// only one node, because then there will never be any AppendEntries replies.
|
||||
let maxLogAQuorumHasCommitted =
|
||||
ServerUtils.maxLogAQuorumHasCommitted leaderState.MatchIndex
|
||||
|
||||
if maxLogAQuorumHasCommitted > 0<LogIndex> then
|
||||
let ourLogTerm =
|
||||
match persistentState.GetLogEntry maxLogAQuorumHasCommitted with
|
||||
| None ->
|
||||
failwithf
|
||||
"Invariant violated. The leader has not logged an entry (%i) which it knows a quorum has logged."
|
||||
maxLogAQuorumHasCommitted
|
||||
| Some (_, term) -> term
|
||||
|
||||
if ourLogTerm = persistentState.CurrentTerm then
|
||||
volatileState <-
|
||||
{ volatileState with
|
||||
CommitIndex = maxLogAQuorumHasCommitted
|
||||
}
|
||||
|
||||
| RequestVoteReply requestVoteReply ->
|
||||
match currentType with
|
||||
| ServerSpecialisation.Leader _
|
||||
| ServerSpecialisation.Follower ->
|
||||
// We're not expecting any votes; drop them.
|
||||
()
|
||||
| ServerSpecialisation.Candidate state ->
|
||||
|
||||
if requestVoteReply.VoterTerm = persistentState.CurrentTerm then
|
||||
state.Votes.[requestVoteReply.Voter / 1<ServerId>] <- Some requestVoteReply.VoteGranted
|
||||
|
||||
// Inefficient, but :shrug:
|
||||
if
|
||||
Array.sumBy
|
||||
(function
|
||||
| Some true -> 1
|
||||
| _ -> 0)
|
||||
state.Votes > clusterSize / 2
|
||||
then
|
||||
// Become the leader!
|
||||
currentType <-
|
||||
LeaderState.New clusterSize persistentState.CurrentLogIndex
|
||||
|> ServerSpecialisation.Leader
|
||||
|
||||
|
||||
let mailbox =
|
||||
let rec loop (mailbox : MailboxProcessor<_>) =
|
||||
async {
|
||||
@@ -396,6 +501,26 @@ type Server<'a>
|
||||
//System.Console.WriteLine toPrint
|
||||
|
||||
match m with
|
||||
| ServerAction.EmitHeartbeat ->
|
||||
match currentType with
|
||||
| ServerSpecialisation.Candidate _
|
||||
| ServerSpecialisation.Follower -> ()
|
||||
| ServerSpecialisation.Leader _ ->
|
||||
for i in 0 .. clusterSize - 1 do
|
||||
if i * 1<ServerId> <> me then
|
||||
{
|
||||
LeaderTerm = persistentState.CurrentTerm
|
||||
LeaderId = me
|
||||
PrevLogEntry = persistentState.GetLastLogEntry () |> Option.map snd
|
||||
NewEntry = None
|
||||
LeaderCommitIndex = volatileState.CommitIndex
|
||||
ReplyChannel =
|
||||
fun reply ->
|
||||
messageChannel me (reply |> Reply.AppendEntriesReply |> Message.Reply)
|
||||
}
|
||||
|> Instruction.AppendEntries
|
||||
|> Message.Instruction
|
||||
|> messageChannel (i * 1<ServerId>)
|
||||
| ServerAction.BeginElection ->
|
||||
match currentType with
|
||||
| ServerSpecialisation.Leader _ -> ()
|
||||
@@ -412,44 +537,15 @@ type Server<'a>
|
||||
{
|
||||
CandidateTerm = persistentState.CurrentTerm
|
||||
CandidateId = me
|
||||
CandidateLastLogEntry =
|
||||
match persistentState.GetLastLogEntry () with
|
||||
| Some (index, (_, term)) -> (index, term)
|
||||
| None ->
|
||||
// TODO this is almost certainly not right
|
||||
(0<LogIndex>, 0<Term>)
|
||||
ReplyChannel =
|
||||
// TODO this is bypassing the network - stop it!
|
||||
fun reply -> messageChannel me (RequestVoteReply reply |> Message.Reply)
|
||||
CandidateLastLogEntry = persistentState.GetLastLogEntry () |> Option.map snd
|
||||
ReplyChannel = fun reply -> messageChannel me (RequestVoteReply reply |> Message.Reply)
|
||||
}
|
||||
|> Instruction.RequestVote
|
||||
|> Message.Instruction
|
||||
|> messageChannel (i * 1<ServerId>)
|
||||
| ServerAction.Receive (Instruction m) -> return processMessage m
|
||||
| ServerAction.Sync reply -> reply.Reply ()
|
||||
| ServerAction.Receive (Reply (RequestVoteReply requestVoteReply)) ->
|
||||
match currentType with
|
||||
| ServerSpecialisation.Leader _
|
||||
| ServerSpecialisation.Follower ->
|
||||
// We're not expecting any votes; drop them.
|
||||
()
|
||||
| ServerSpecialisation.Candidate state ->
|
||||
|
||||
if requestVoteReply.VoterTerm = persistentState.CurrentTerm then
|
||||
state.Votes.[requestVoteReply.Voter / 1<ServerId>] <- Some requestVoteReply.VoteGranted
|
||||
|
||||
// Inefficient, but :shrug:
|
||||
if
|
||||
Array.sumBy
|
||||
(function
|
||||
| Some true -> 1
|
||||
| _ -> 0)
|
||||
state.Votes > clusterSize / 2
|
||||
then
|
||||
// Become the leader!
|
||||
currentType <-
|
||||
LeaderState.New clusterSize persistentState.CurrentLogIndex
|
||||
|> ServerSpecialisation.Leader
|
||||
| ServerAction.Receive (Instruction m) -> processMessage m
|
||||
| ServerAction.Receive (Reply r) -> processReply r
|
||||
| ServerAction.Sync replyChannel -> replyChannel.Reply ()
|
||||
|
||||
return! loop mailbox
|
||||
}
|
||||
@@ -458,7 +554,8 @@ type Server<'a>
|
||||
mailbox.Error.Add raise
|
||||
mailbox
|
||||
|
||||
member this.TriggerTimeout () = mailbox.Post ServerAction.BeginElection
|
||||
member this.TriggerInactivityTimeout () = mailbox.Post ServerAction.BeginElection
|
||||
member this.TriggerHeartbeatTimeout () = mailbox.Post ServerAction.EmitHeartbeat
|
||||
|
||||
member this.Message (m : Message<'a>) = mailbox.Post (ServerAction.Receive m)
|
||||
|
||||
|
Reference in New Issue
Block a user