From 314ac46369bb882e96b42c3972ef7c187f1f5c8f Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Fri, 17 Oct 2025 13:11:23 +0100 Subject: [PATCH 01/17] Maybe done raft tla? --- tla/consensus/ccfraft.tla | 61 ++++++++++++++++++++++++++++++++------- 1 file changed, 50 insertions(+), 11 deletions(-) diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index b77b7e706095..7664239a3191 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -31,12 +31,16 @@ CONSTANT Reordered, Guarantee +CONSTANT + PreVoteEnabled + \* Leadership states CONSTANTS \* See original Raft paper (https://www.usenix.org/system/files/conference/atc14/atc14-paper-ongaro.pdf) \* and comments for leadership_state in ../src/consensus/aft/raft.h for details on the Follower, - \* Candidate, and Leader states. + \* PreVoteCandidate, Candidate, and Leader states. Follower, + PreVoteCandidate, Candidate, Leader, \* Initial state for a joiner node, until it has received a first message @@ -45,6 +49,7 @@ CONSTANTS LeadershipStates == { Follower, + PreVoteCandidate, Candidate, Leader, None @@ -624,11 +629,36 @@ Init == ------------------------------------------------------------------------------ \* Define state transitions -BecomeCandidate(i) == +BecomePreVoteCandidate(i) == + /\ PreVoteEnabled[i] \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates /\ leadershipState[i] \in {Follower, Candidate} + /\ + \* Check that the reconfiguration which added this node is at least committable + \/ \E c \in DOMAIN configurations[i] : + /\ i \in configurations[i][c] + /\ MaxCommittableIndex(log[i]) >= c + \* Or if the node isn't in a configuration, that it is retiring + \/ i \in retirementCompleted[i] + /\ leadershipState' = [leadershipState EXCEPT ![i] = PreVoteCandidate] + /\ votesGranted' = [votesGranted EXCEPT ![i] = {i}] + /\ UNCHANGED <> + /\ UNCHANGED <> + +BecomeCandidate(i) == + \* Only servers that haven't completed retirement can become candidates + /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} + \* Only servers that are followers/candidates can become candidates + /\ \/ /\ ~PreVoteEnabled[i] + /\ leadershipState[i] \in {Follower, Candidate} + \/ /\ PreVoteEnabled[i] + /\ leadershipState[i] \in {PreVoteCandidate} + \* To become a Candidate, the PreVoteCandidate must have received votes from a majority in each active configuration + \* Only votes by nodes part of a given configuration should be tallied against it + /\ \A c \in DOMAIN configurations[i] : + (votesGranted[i] \intersect configurations[i][c]) \in Quorums[configurations[i][c]] /\ \* Check that the reconfiguration which added this node is at least committable \/ \E c \in DOMAIN configurations[i] : @@ -646,12 +676,16 @@ BecomeCandidate(i) == \* Server i times out (becomes candidate) and votes for itself in the election of the next term \* At some point later (non-deterministically), the candidate will request votes from the other nodes. Timeout(i) == - /\ BecomeCandidate(i) + /\ \/ /\ ~PreVoteEnabled[i] + /\ BecomeCandidate(i) + \/ /\ PreVoteEnabled[i] + /\ BecomePreVoteCandidate(i) /\ UNCHANGED messageVars \* Candidate i sends j a RequestVote request. RequestVote(i,j) == LET + isPreVote == leadershipState[i] = PreVoteCandidate msg == [type |-> RequestVoteRequest, term |-> currentTerm[i], \* CCF: Use last signature entry and not last log entry in elections. @@ -659,12 +693,13 @@ RequestVote(i,j) == lastCommittableTerm |-> LastCommittableTerm(i), lastCommittableIndex |-> LastCommittableIndex(i), source |-> i, - dest |-> j] + dest |-> j, + isPreVote |-> isPreVote] IN \* Timeout votes for itself atomically. Thus we do not need to request our own vote. /\ i /= j \* Only requests vote if we are already a candidate (and therefore have not completed retirement) - /\ leadershipState[i] = Candidate + /\ leadershipState[i] \in {PreVoteCandidate, Candidate} \* Reconfiguration: Make sure j is in a configuration of i /\ IsInServerSet(j, i) /\ Send(msg) @@ -922,7 +957,10 @@ HandleRequestVoteRequest(i, j, m) == /\ logOk /\ votedFor[i] \in {Nil, j} IN /\ m.term <= currentTerm[i] - /\ \/ grant /\ votedFor' = [votedFor EXCEPT ![i] = j] + /\ \/ grant \/ /\ ~m.isPreVote + /\ votedFor' = [votedFor EXCEPT ![i] = j] + \/ /\ m.isPreVote + /\ UNCHANGED votedFor \/ ~grant /\ UNCHANGED votedFor /\ Reply([type |-> RequestVoteResponse, term |-> currentTerm[i], @@ -937,8 +975,8 @@ HandleRequestVoteRequest(i, j, m) == \* m.term = currentTerm[i]. HandleRequestVoteResponse(i, j, m) == /\ m.term = currentTerm[i] - \* Only Candidates need to tally votes - /\ leadershipState[i] = Candidate + \* Only PreVoteCandidates and Candidates need to tally votes + /\ leadershipState[i] \in {PreVoteCandidate, Candidate} /\ \/ /\ m.voteGranted /\ votesGranted' = [votesGranted EXCEPT ![i] = votesGranted[i] \cup {j}] @@ -993,7 +1031,7 @@ RejectAppendEntriesRequest(i, j, m, logOk) == \* Must check that m is an AppendEntries message before returning to follower state ReturnToFollowerState(i, m) == /\ m.term = currentTerm[i] - /\ leadershipState[i] = Candidate + /\ leadershipState[i] \in {PreVoteCandidate, Candidate} /\ leadershipState' = [leadershipState EXCEPT ![i] = Follower] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = TRUE] \* Note that the set of messages is unchanged as m is discarded @@ -1139,7 +1177,8 @@ UpdateTerm(i, j, m) == /\ m.term > currentTerm[i] /\ currentTerm' = [currentTerm EXCEPT ![i] = m.term] \* See become_aware_of_new_term() in raft.h:1915 - /\ leadershipState' = [leadershipState EXCEPT ![i] = IF @ \in {Leader, Candidate, None} THEN Follower ELSE @] + /\ leadershipState' = [leadershipState EXCEPT + ![i] = IF @ \in {Leader, Candidate, PreVoteCandidate, None} THEN Follower ELSE @] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = TRUE] /\ votedFor' = [votedFor EXCEPT ![i] = Nil] \* See rollback(last_committable_index()) in raft::become_follower @@ -1164,7 +1203,7 @@ DropResponseWhenNotInState(i, j, m) == \/ /\ m.type = AppendEntriesResponse /\ leadershipState[i] \in LeadershipStates \ { Leader } \/ /\ m.type = RequestVoteResponse - /\ leadershipState[i] \in LeadershipStates \ { Candidate } + /\ leadershipState[i] \in LeadershipStates \ { Candidate, PreVoteCandidate } /\ Discard(m) /\ UNCHANGED <> From 5e3978e68aef9f5d37596cf3e639177d4a6a506a Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Thu, 23 Oct 2025 15:45:55 +0100 Subject: [PATCH 02/17] Basic working --- tla/consensus/SIMccfraft.cfg | 3 +++ tla/consensus/ccfraft.tla | 38 ++++++++++++++++++++---------------- 2 files changed, 24 insertions(+), 17 deletions(-) diff --git a/tla/consensus/SIMccfraft.cfg b/tla/consensus/SIMccfraft.cfg index 26d2b1f51e06..2be54ece18a1 100644 --- a/tla/consensus/SIMccfraft.cfg +++ b/tla/consensus/SIMccfraft.cfg @@ -6,6 +6,7 @@ CONSTANTS Nil = Nil Follower = L_Follower + PreVoteCandidate = L_PreVoteCandidate Candidate = L_Candidate Leader = L_Leader None = L_None @@ -51,6 +52,8 @@ CONSTANTS Extend <- [abs]ABSExtend CopyMaxAndExtend <- [abs]ABSCopyMaxAndExtend + PreVoteEnabled <- TRUE + _PERIODIC Periodically diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 7664239a3191..0d34c6182bf9 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -630,7 +630,7 @@ Init == \* Define state transitions BecomePreVoteCandidate(i) == - /\ PreVoteEnabled[i] + /\ PreVoteEnabled \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates @@ -651,9 +651,9 @@ BecomeCandidate(i) == \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates - /\ \/ /\ ~PreVoteEnabled[i] + /\ \/ /\ ~PreVoteEnabled /\ leadershipState[i] \in {Follower, Candidate} - \/ /\ PreVoteEnabled[i] + \/ /\ PreVoteEnabled /\ leadershipState[i] \in {PreVoteCandidate} \* To become a Candidate, the PreVoteCandidate must have received votes from a majority in each active configuration \* Only votes by nodes part of a given configuration should be tallied against it @@ -676,10 +676,8 @@ BecomeCandidate(i) == \* Server i times out (becomes candidate) and votes for itself in the election of the next term \* At some point later (non-deterministically), the candidate will request votes from the other nodes. Timeout(i) == - /\ \/ /\ ~PreVoteEnabled[i] - /\ BecomeCandidate(i) - \/ /\ PreVoteEnabled[i] - /\ BecomePreVoteCandidate(i) + /\ \/ BecomePreVoteCandidate(i) + \/ BecomeCandidate(i) /\ UNCHANGED messageVars \* Candidate i sends j a RequestVote request. @@ -957,10 +955,10 @@ HandleRequestVoteRequest(i, j, m) == /\ logOk /\ votedFor[i] \in {Nil, j} IN /\ m.term <= currentTerm[i] - /\ \/ grant \/ /\ ~m.isPreVote - /\ votedFor' = [votedFor EXCEPT ![i] = j] - \/ /\ m.isPreVote - /\ UNCHANGED votedFor + /\ \/ grant /\ \/ /\ ~m.isPreVote + /\ votedFor' = [votedFor EXCEPT ![i] = j] + \/ /\ m.isPreVote + /\ UNCHANGED votedFor \/ ~grant /\ UNCHANGED votedFor /\ Reply([type |-> RequestVoteResponse, term |-> currentTerm[i], @@ -1209,6 +1207,9 @@ DropResponseWhenNotInState(i, j, m) == \* Drop messages if they are irrelevant to the node DropIgnoredMessage(i,j,m) == + \* raft.h::recv_request_vote + \* We specifically always respond to request votes + /\ m.type /= RequestVoteRequest \* Drop messages if... /\ \* .. recipient is still None.. @@ -1219,14 +1220,11 @@ DropIgnoredMessage(i,j,m) == \/ /\ leadershipState[i] /= None \* .. and it comes from a server outside of the configuration /\ \lnot IsInServerSet(j, i) - \* raft.h::recv_request_vote - \* We specifically don't ignore RequestVoteRequest messages from unknown nodes - /\ m.type /= RequestVoteRequest \* OR if recipient has completed retirement and this is not a request to vote or append entries request \* This spec requires that a retired node still helps with voting and appending entries to ensure \* the next configurations learns that its retirement has been committed. \/ /\ membershipState[i] = RetiredCommitted - /\ m.type \notin {RequestVoteRequest, AppendEntriesRequest} + /\ m.type /= AppendEntriesRequest /\ Discard(m) /\ UNCHANGED <> @@ -1634,8 +1632,14 @@ PermittedLogChangesProp == StateTransitionsProp == [][\A i \in Servers : /\ leadershipState[i] = None => leadershipState'[i] \in {None, Follower} - /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, Candidate} - /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, Candidate, Leader} + /\ \/ /\ ~PreVoteEnabled + /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, Candidate} + /\ leadershipState[i] /= PreVoteCandidate + /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, Candidate, Leader} + \/ /\ PreVoteEnabled + /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} + /\ leadershipState[i] = PreVoteCandidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} + /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate, Leader} /\ leadershipState[i] = Leader => leadershipState'[i] \in {Follower, Leader} ]_vars From a05f2baf91f290df5bfd73aa348ba0d49be04eff Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Thu, 23 Oct 2025 16:52:30 +0100 Subject: [PATCH 03/17] Make preVoteEnabled configurable for simulation --- tla/consensus/SIMccfraft.cfg | 2 +- tla/consensus/SIMccfraft.tla | 3 +++ tla/consensus/ccfraft.tla | 27 ++++++++++++++++++--------- 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/tla/consensus/SIMccfraft.cfg b/tla/consensus/SIMccfraft.cfg index 2be54ece18a1..a64c141ae40b 100644 --- a/tla/consensus/SIMccfraft.cfg +++ b/tla/consensus/SIMccfraft.cfg @@ -52,7 +52,7 @@ CONSTANTS Extend <- [abs]ABSExtend CopyMaxAndExtend <- [abs]ABSCopyMaxAndExtend - PreVoteEnabled <- TRUE + InitOptionConsts <- SIMInitOptionConsts _PERIODIC Periodically diff --git a/tla/consensus/SIMccfraft.tla b/tla/consensus/SIMccfraft.tla index d9c234e89cf6..694cf303910a 100644 --- a/tla/consensus/SIMccfraft.tla +++ b/tla/consensus/SIMccfraft.tla @@ -16,6 +16,9 @@ SIMInitReconfigurationVars == \* Start with any subset of servers in the active configuration. \/ CCF!InitReconfigurationVars +SIMInitOptionConsts == + /\ preVoteEnabled \in {TRUE, FALSE} + LOCAL R == 1..IF "R" \in DOMAIN IOEnv THEN atoi(IOEnv.R) ELSE 10 diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 0d34c6182bf9..336b34d2b754 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -31,9 +31,6 @@ CONSTANT Reordered, Guarantee -CONSTANT - PreVoteEnabled - \* Leadership states CONSTANTS \* See original Raft paper (https://www.usenix.org/system/files/conference/atc14/atc14-paper-ongaro.pdf) @@ -119,6 +116,12 @@ Nil == ------------------------------------------------------------------------------ \* Global variables +\* Whether PreVote is enabled in this cluster +VARIABLE + preVoteEnabled + +PreVoteEnabledTypeInv == preVoteEnabled \in BOOLEAN + \* A set representing requests and responses sent from one server \* to another. With CCF, we have message integrity and can ensure unique messages. \* Messages only records messages that are currently in-flight, actions should @@ -355,6 +358,7 @@ vars == << \* Invariant to check the type safety of all variables TypeInv == + /\ PreVoteEnabledTypeInv /\ ReconfigurationVarsTypeInv /\ MessageVarsTypeInv /\ ServerVarsTypeInv @@ -620,17 +624,21 @@ InitCandidateVars == InitLeaderVars == /\ matchIndex = [i \in Servers |-> [j \in Servers |-> 0]] +InitOptionConsts == + /\ preVoteEnabled = FALSE + Init == /\ InitReconfigurationVars /\ InitMessagesVars /\ InitCandidateVars /\ InitLeaderVars + /\ InitOptionConsts ------------------------------------------------------------------------------ \* Define state transitions BecomePreVoteCandidate(i) == - /\ PreVoteEnabled + /\ preVoteEnabled \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates @@ -651,9 +659,9 @@ BecomeCandidate(i) == \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates - /\ \/ /\ ~PreVoteEnabled + /\ \/ /\ ~preVoteEnabled /\ leadershipState[i] \in {Follower, Candidate} - \/ /\ PreVoteEnabled + \/ /\ preVoteEnabled /\ leadershipState[i] \in {PreVoteCandidate} \* To become a Candidate, the PreVoteCandidate must have received votes from a majority in each active configuration \* Only votes by nodes part of a given configuration should be tallied against it @@ -1314,7 +1322,8 @@ NextInt(i) == \/ \E j \in Servers : Receive(i, j) Next == - \E i \in Servers: NextInt(i) + /\ \E i \in Servers: NextInt(i) + /\ UNCHANGED preVoteEnabled Fairness == \* Network actions @@ -1632,11 +1641,11 @@ PermittedLogChangesProp == StateTransitionsProp == [][\A i \in Servers : /\ leadershipState[i] = None => leadershipState'[i] \in {None, Follower} - /\ \/ /\ ~PreVoteEnabled + /\ \/ /\ ~preVoteEnabled /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, Candidate} /\ leadershipState[i] /= PreVoteCandidate /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, Candidate, Leader} - \/ /\ PreVoteEnabled + \/ /\ preVoteEnabled /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} /\ leadershipState[i] = PreVoteCandidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate, Leader} From bcb6ff2ed67f0fb02c4264d2ce42801cd8bdad3c Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Thu, 23 Oct 2025 18:12:54 +0100 Subject: [PATCH 04/17] Get model checking working? --- tla/consensus/MCccfraft.cfg | 1 + tla/consensus/MCccfraft.tla | 4 ++++ tla/consensus/ccfraft.tla | 1 + 3 files changed, 6 insertions(+) diff --git a/tla/consensus/MCccfraft.cfg b/tla/consensus/MCccfraft.cfg index 0c5e7093a9b3..f42e7b652309 100644 --- a/tla/consensus/MCccfraft.cfg +++ b/tla/consensus/MCccfraft.cfg @@ -15,6 +15,7 @@ CONSTANTS Nil = Nil Follower = L_Follower + PreVoteCandidate = L_PreVoteCandidate Candidate = L_Candidate Leader = L_Leader None = L_None diff --git a/tla/consensus/MCccfraft.tla b/tla/consensus/MCccfraft.tla index 436cdf23aadf..fb372cf42793 100644 --- a/tla/consensus/MCccfraft.tla +++ b/tla/consensus/MCccfraft.tla @@ -106,10 +106,14 @@ MCSend(msg) == /\ n.type = AppendEntriesResponse /\ CCF!Send(msg) +MCInitOptionConsts == + /\ preVoteEnabled \in {FALSE, TRUE} + MCInit == /\ InitMessagesVars /\ InitCandidateVars /\ InitLeaderVars + /\ MCInitOptionConsts /\ IF Cardinality(Configurations[1]) = 1 \* If the first config is just one node, we can start with a two-tx log and a single config. THEN InitLogConfigServerVars(Configurations[1], StartLog) diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 336b34d2b754..81e5a6b7822f 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -348,6 +348,7 @@ LeaderVarsTypeInv == \* All variables; used for stuttering (asserting state hasn't changed). vars == << + preVoteEnabled, reconfigurationVars, messageVars, serverVars, From 9db0cec0f6d39125568f15298145409e0d7f60f7 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Thu, 23 Oct 2025 18:13:40 +0100 Subject: [PATCH 05/17] REVERT: test that a prevote leader can be elected. --- tla/consensus/MCccfraft.cfg | 1 + tla/consensus/MCccfraft.tla | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/tla/consensus/MCccfraft.cfg b/tla/consensus/MCccfraft.cfg index f42e7b652309..742d1be7c6fd 100644 --- a/tla/consensus/MCccfraft.cfg +++ b/tla/consensus/MCccfraft.cfg @@ -93,3 +93,4 @@ INVARIANTS RetiredCommittedInv RetirementCompletedNotInConfigsInv RetirementCompletedAreRetirementCompletedInv + DebugInvLeaderAfterInit diff --git a/tla/consensus/MCccfraft.tla b/tla/consensus/MCccfraft.tla index fb372cf42793..260f5ae0dd18 100644 --- a/tla/consensus/MCccfraft.tla +++ b/tla/consensus/MCccfraft.tla @@ -144,6 +144,11 @@ DebugNotTooManySigsInv == \A i \in Servers: FoldSeq(LAMBDA e, count: IF e.contentType = TypeSignature THEN count + 1 ELSE count, 0, log[i]) < 8 +DebugInvLeaderAfterInit == + \lnot \E i \in Servers : /\ preVoteEnabled + /\ leadershipState[i] = Leader + /\ currentTerm[i] > 2 + ---- \* Initialize the counters for the Debug invariants to 0. From c674fefe4656554ba18bf3a158d483b901bd8d05 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Thu, 23 Oct 2025 18:15:45 +0100 Subject: [PATCH 06/17] Maybe tracing should also pass now? --- tla/consensus/Traceccfraft.cfg | 1 + 1 file changed, 1 insertion(+) diff --git a/tla/consensus/Traceccfraft.cfg b/tla/consensus/Traceccfraft.cfg index 0abd26c8569d..f9a12f291cfb 100644 --- a/tla/consensus/Traceccfraft.cfg +++ b/tla/consensus/Traceccfraft.cfg @@ -57,6 +57,7 @@ CONSTANTS Nil = Nil Follower = L_Follower + PreVoteCandidate = L_PreVoteCandidate Candidate = L_Candidate Leader = L_Leader None = L_None From 362fae2bb04cc1fee920977691bfb170214fa9c9 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Mon, 27 Oct 2025 12:38:04 +0000 Subject: [PATCH 07/17] Propagate UNCHANGED preVoteEnabled to all relevant points --- tla/consensus/Traceccfraft.tla | 4 +-- tla/consensus/ccfraft.tla | 48 ++++++++++++++++++---------------- 2 files changed, 27 insertions(+), 25 deletions(-) diff --git a/tla/consensus/Traceccfraft.tla b/tla/consensus/Traceccfraft.tla index 79c945aa75e1..4fa1cc36515f 100644 --- a/tla/consensus/Traceccfraft.tla +++ b/tla/consensus/Traceccfraft.tla @@ -149,7 +149,7 @@ IsDropPendingTo == /\ IsEvent("drop_pending_to") /\ Network!DropMessage(logline.msg.to_node_id, LAMBDA msg: IsMessage(msg, logline.msg.to_node_id, logline.msg.from_node_id, logline)) - /\ UNCHANGED <> + /\ UNCHANGED <> IsTimeout == /\ IsEvent("become_candidate") @@ -402,7 +402,7 @@ IsRcvProposeVoteRequest == /\ m.type = ProposeVoteRequest /\ m.term = logline.msg.packet.term /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> /\ Range(logline.msg.state.committable_indices) \subseteq CommittableIndices(logline.msg.state.node_id) /\ commitIndex[logline.msg.state.node_id] = logline.msg.state.commit_idx /\ leadershipState[logline.msg.state.node_id] = ToLeadershipState[logline.msg.state.leadership_state] diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 81e5a6b7822f..5c01ae5baad1 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -122,6 +122,8 @@ VARIABLE PreVoteEnabledTypeInv == preVoteEnabled \in BOOLEAN +configVars == << preVoteEnabled >> + \* A set representing requests and responses sent from one server \* to another. With CCF, we have message integrity and can ensure unique messages. \* Messages only records messages that are currently in-flight, actions should @@ -687,7 +689,7 @@ BecomeCandidate(i) == Timeout(i) == /\ \/ BecomePreVoteCandidate(i) \/ BecomeCandidate(i) - /\ UNCHANGED messageVars + /\ UNCHANGED <> \* Candidate i sends j a RequestVote request. RequestVote(i,j) == @@ -710,7 +712,7 @@ RequestVote(i,j) == \* Reconfiguration: Make sure j is in a configuration of i /\ IsInServerSet(j, i) /\ Send(msg) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Leader i sends j an AppendEntries request AppendEntries(i, j) == @@ -750,7 +752,7 @@ AppendEntries(i, j) == \* Record the most recent index we have sent to this node. \* raft.h::send_append_entries /\ sentIndex' = [sentIndex EXCEPT ![i][j] = @ + Len(m.entries)] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Candidate i transitions to leader. BecomeLeader(i) == @@ -773,7 +775,7 @@ BecomeLeader(i) == \* been rolled back as it was unsigned /\ membershipState' = [membershipState EXCEPT ![i] = IF @ = RetirementOrdered THEN Active ELSE @] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Leader i receives a client request to add to the log. The consensus spec is agnostic to request payloads, \* and does not model or differentiate them. See the consistency spec (tla/consistency/*) for a specification @@ -785,7 +787,7 @@ ClientRequest(i) == /\ membershipState[i] # RetiredCommitted \* Add new request to leader's log /\ log' = [log EXCEPT ![i] = Append(@, [term |-> currentTerm[i], contentType |-> TypeEntry]) ] - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF: Signed commits \* In CCF, the leader periodically signs the latest log prefix. Only these signatures are committable in CCF. @@ -808,7 +810,7 @@ SignCommittableMessages(i) == /\ IF membershipState[i] = RetirementOrdered THEN membershipState' = [membershipState EXCEPT ![i] = RetirementSigned] ELSE UNCHANGED membershipState - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF: Reconfiguration of servers \* In the TLA+ model, a reconfiguration is initiated by the Leader which appends an arbitrary new configuration to its own log. @@ -846,7 +848,7 @@ ChangeConfigurationInt(i, newConfiguration) == /\ IF membershipState[i] = Active /\ i \notin newConfiguration THEN membershipState' = [membershipState EXCEPT ![i] = RetirementOrdered] ELSE UNCHANGED membershipState - /\ UNCHANGED <> + /\ UNCHANGED <> ChangeConfiguration(i) == \* Reconfigure to any *non-empty* subset of servers. ChangeConfigurationInt checks that the new @@ -866,7 +868,7 @@ AppendRetiredCommitted(i) == term |-> currentTerm[i], contentType |-> TypeRetired, retired |-> retire_committed_nodes])] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Leader i advances its commitIndex to the next possible Index. @@ -934,7 +936,7 @@ AdvanceCommitIndex(i) == IN Send(msg) ELSE UNCHANGED <> /\ retirementCompleted' = [retirementCompleted EXCEPT ![i] = NextRetirementCompleted(retirementCompleted[i], configurations[i], log[i], commitIndex'[i], i)] - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF supports checkQuorum which enables a leader to choose to abdicate leadership. CheckQuorum(i) == @@ -945,7 +947,7 @@ CheckQuorum(i) == \E n \in configurations[i][c]: n /= i /\ leadershipState' = [leadershipState EXCEPT ![i] = Follower] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = TRUE] - /\ UNCHANGED <> + /\ UNCHANGED <> ------------------------------------------------------------------------------ \* Message handlers @@ -975,7 +977,7 @@ HandleRequestVoteRequest(i, j, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> \* Server i receives a RequestVote response from server j with @@ -990,7 +992,7 @@ HandleRequestVoteResponse(i, j, m) == \/ /\ ~m.voteGranted /\ UNCHANGED votesGranted /\ Discard(m) - /\ UNCHANGED <> \* Server i replies to a AppendEntries request from server j with a NACK @@ -1032,7 +1034,7 @@ RejectAppendEntriesRequest(i, j, m, logOk) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Candidate i steps down to follower in the same term after receiving a message m from a leader in the current term \* Must check that m is an AppendEntries message before returning to follower state @@ -1042,7 +1044,7 @@ ReturnToFollowerState(i, m) == /\ leadershipState' = [leadershipState EXCEPT ![i] = Follower] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = TRUE] \* Note that the set of messages is unchanged as m is discarded - /\ UNCHANGED <> \* Follower i receives a AppendEntries from leader j for log entries it already has @@ -1070,7 +1072,7 @@ AppendEntriesAlreadyDone(i, j, index, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Follower i receives an AppendEntries request m where it needs to roll back first \* This action rolls back the log and leaves m in messages for further processing @@ -1086,7 +1088,7 @@ ConflictAppendEntriesRequest(i, index, m) == /\ configurations' = [configurations EXCEPT ![i] = ConfigurationsToIndex(i,Len(new_log))] /\ membershipState' = [membershipState EXCEPT ![i] = CalcMembershipState(log'[i], commitIndex[i], i)] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = FALSE] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Follower i receives an AppendEntries request m from leader j for log entries which directly follow its log NoConflictAppendEntriesRequest(i, j, m) == @@ -1130,7 +1132,7 @@ NoConflictAppendEntriesRequest(i, j, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> AcceptAppendEntriesRequest(i, j, logOk, m) == /\ m.term = currentTerm[i] @@ -1173,7 +1175,7 @@ HandleAppendEntriesResponse(i, j, m) == \* "If AppendEntries fails because of log inconsistency: decrement nextIndex (aka sentIndex +1) and retry" /\ UNCHANGED matchIndex /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Any message with a newer term causes the recipient to advance its term first. \* Note that UpdateTerm does not discard message m from the set of messages so this @@ -1197,13 +1199,13 @@ UpdateTerm(i, j, m) == /\ membershipState' = [membershipState EXCEPT ![i] = IF @ = RetirementOrdered THEN Active ELSE @] \* messages is unchanged so m can be processed further. - /\ UNCHANGED <> + /\ UNCHANGED <> \* Responses with stale terms are ignored. DropStaleResponse(i, j, m) == /\ m.term < currentTerm[i] /\ Discard(m) - /\ UNCHANGED <> DropResponseWhenNotInState(i, j, m) == @@ -1212,7 +1214,7 @@ DropResponseWhenNotInState(i, j, m) == \/ /\ m.type = RequestVoteResponse /\ leadershipState[i] \in LeadershipStates \ { Candidate, PreVoteCandidate } /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Drop messages if they are irrelevant to the node DropIgnoredMessage(i,j,m) == @@ -1235,7 +1237,7 @@ DropIgnoredMessage(i,j,m) == \/ /\ membershipState[i] = RetiredCommitted /\ m.type /= AppendEntriesRequest /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Receive a message. @@ -1290,6 +1292,7 @@ RcvProposeVoteRequest(i, j) == /\ m.term = currentTerm[i] /\ BecomeCandidate(m.dest) /\ Discard(m) + /\ UNCHANGED configVars \* Node i receives a message from node j. Receive(i, j) == @@ -1324,7 +1327,6 @@ NextInt(i) == Next == /\ \E i \in Servers: NextInt(i) - /\ UNCHANGED preVoteEnabled Fairness == \* Network actions From ac4a2e35801af4f20fa6d8f4b7176e1d737669bc Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Mon, 27 Oct 2025 18:08:34 +0000 Subject: [PATCH 08/17] Only allow transitions from prevotecandidate to candidate --- tla/consensus/SIMccfraft.tla | 1 + tla/consensus/ccfraft.tla | 15 ++++++++------- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/tla/consensus/SIMccfraft.tla b/tla/consensus/SIMccfraft.tla index 694cf303910a..f715a46b98dd 100644 --- a/tla/consensus/SIMccfraft.tla +++ b/tla/consensus/SIMccfraft.tla @@ -69,6 +69,7 @@ SIMFairness == /\ \A s \in Servers : WF_vars(SignCommittableMessages(s)) /\ \A s \in Servers : WF_vars(AdvanceCommitIndex(s)) /\ \A s \in Servers : WF_vars(AppendRetiredCommitted(s)) + /\ \A s \in Servers : WF_vars(preVoteEnabled /\ BecomeCandidate(s)) /\ \A s \in Servers : WF_vars(BecomeLeader(s)) \* The following fairness conditions reference the original CCF actions \* and, thus, do not include the RandomElement conjunct. diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 5c01ae5baad1..72f0e4aa75dd 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -645,7 +645,7 @@ BecomePreVoteCandidate(i) == \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates - /\ leadershipState[i] \in {Follower, Candidate} + /\ leadershipState[i] = Follower /\ \* Check that the reconfiguration which added this node is at least committable \/ \E c \in DOMAIN configurations[i] : @@ -656,7 +656,7 @@ BecomePreVoteCandidate(i) == /\ leadershipState' = [leadershipState EXCEPT ![i] = PreVoteCandidate] /\ votesGranted' = [votesGranted EXCEPT ![i] = {i}] /\ UNCHANGED <> - /\ UNCHANGED <> + /\ UNCHANGED <> BecomeCandidate(i) == \* Only servers that haven't completed retirement can become candidates @@ -665,7 +665,7 @@ BecomeCandidate(i) == /\ \/ /\ ~preVoteEnabled /\ leadershipState[i] \in {Follower, Candidate} \/ /\ preVoteEnabled - /\ leadershipState[i] \in {PreVoteCandidate} + /\ leadershipState[i] = PreVoteCandidate \* To become a Candidate, the PreVoteCandidate must have received votes from a majority in each active configuration \* Only votes by nodes part of a given configuration should be tallied against it /\ \A c \in DOMAIN configurations[i] : @@ -682,14 +682,13 @@ BecomeCandidate(i) == \* Candidate votes for itself /\ votedFor' = [votedFor EXCEPT ![i] = i] /\ votesGranted' = [votesGranted EXCEPT ![i] = {i}] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Server i times out (becomes candidate) and votes for itself in the election of the next term \* At some point later (non-deterministically), the candidate will request votes from the other nodes. Timeout(i) == - /\ \/ BecomePreVoteCandidate(i) - \/ BecomeCandidate(i) - /\ UNCHANGED <> + \/ preVoteEnabled /\ BecomePreVoteCandidate(i) + \/ ~preVoteEnabled /\ BecomeCandidate(i) \* Candidate i sends j a RequestVote request. RequestVote(i,j) == @@ -1314,6 +1313,7 @@ Receive(i, j) == \* Defines how the variables may transition, given a node i. NextInt(i) == \/ Timeout(i) + \/ (preVoteEnabled /\ BecomeCandidate(i)) \/ BecomeLeader(i) \/ ClientRequest(i) \/ SignCommittableMessages(i) @@ -1343,6 +1343,7 @@ Fairness == /\ \A s \in Servers : WF_vars(SignCommittableMessages(s)) /\ \A s \in Servers : WF_vars(AdvanceCommitIndex(s)) /\ \A s \in Servers : WF_vars(AppendRetiredCommitted(s)) + /\ \A s \in Servers : WF_vars(preVoteEnabled /\ BecomeCandidate(s)) /\ \A s \in Servers : WF_vars(BecomeLeader(s)) /\ \A s \in Servers : WF_vars(Timeout(s)) /\ \A s \in Servers : WF_vars(ChangeConfiguration(s)) From 121bf96133313a70bfde5ae5ab2b4e913210fbcc Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 14:19:54 +0000 Subject: [PATCH 09/17] Rough 3 state prevotestatus for CI --- tla/consensus/MCccfraft.cfg | 4 + tla/consensus/MCccfraft.tla | 5 +- tla/consensus/SIMccfraft.cfg | 6 ++ tla/consensus/SIMccfraft.tla | 5 +- tla/consensus/Traceccfraft.cfg | 4 + tla/consensus/ccfraft.tla | 141 +++++++++++++++++++-------------- 6 files changed, 101 insertions(+), 64 deletions(-) diff --git a/tla/consensus/MCccfraft.cfg b/tla/consensus/MCccfraft.cfg index 742d1be7c6fd..7b658d5abae5 100644 --- a/tla/consensus/MCccfraft.cfg +++ b/tla/consensus/MCccfraft.cfg @@ -20,6 +20,10 @@ CONSTANTS Leader = L_Leader None = L_None + PreVoteDisabled = PV_PreVoteDisabled + PreVoteCapable = PV_PreVoteCapable + PreVoteEnabled = PV_PreVoteEnabled + Active = R_Active RetirementOrdered = R_RetirementOrdered RetirementSigned = R_RetirementSigned diff --git a/tla/consensus/MCccfraft.tla b/tla/consensus/MCccfraft.tla index 260f5ae0dd18..2ea89a8dae0a 100644 --- a/tla/consensus/MCccfraft.tla +++ b/tla/consensus/MCccfraft.tla @@ -106,8 +106,7 @@ MCSend(msg) == /\ n.type = AppendEntriesResponse /\ CCF!Send(msg) -MCInitOptionConsts == - /\ preVoteEnabled \in {FALSE, TRUE} +MCInitOptionConsts == PreVoteStatusTypeInv MCInit == /\ InitMessagesVars @@ -145,7 +144,7 @@ DebugNotTooManySigsInv == FoldSeq(LAMBDA e, count: IF e.contentType = TypeSignature THEN count + 1 ELSE count, 0, log[i]) < 8 DebugInvLeaderAfterInit == - \lnot \E i \in Servers : /\ preVoteEnabled + \lnot \E i \in Servers : /\ PreVoteEnabled \in preVoteStatus /\ leadershipState[i] = Leader /\ currentTerm[i] > 2 diff --git a/tla/consensus/SIMccfraft.cfg b/tla/consensus/SIMccfraft.cfg index a64c141ae40b..ac27bc743e51 100644 --- a/tla/consensus/SIMccfraft.cfg +++ b/tla/consensus/SIMccfraft.cfg @@ -11,6 +11,10 @@ CONSTANTS Leader = L_Leader None = L_None + PreVoteDisabled = PV_PreVoteDisabled + PreVoteCapable = PV_PreVoteCapable + PreVoteEnabled = PV_PreVoteEnabled + Active = R_Active RetirementOrdered = R_RetirementOrdered RetirementSigned = R_RetirementSigned @@ -74,6 +78,8 @@ PROPERTIES LeaderProp LogMatchingProp + VotesGrantedMonotonicProp + \* ALIAS \* \* DebugAlias \* \* DebugActingServerAlias diff --git a/tla/consensus/SIMccfraft.tla b/tla/consensus/SIMccfraft.tla index f715a46b98dd..2b2dcd3db6c7 100644 --- a/tla/consensus/SIMccfraft.tla +++ b/tla/consensus/SIMccfraft.tla @@ -16,8 +16,7 @@ SIMInitReconfigurationVars == \* Start with any subset of servers in the active configuration. \/ CCF!InitReconfigurationVars -SIMInitOptionConsts == - /\ preVoteEnabled \in {TRUE, FALSE} +SIMInitOptionConsts == PreVoteStatusTypeInv LOCAL R == 1..IF "R" \in DOMAIN IOEnv THEN atoi(IOEnv.R) ELSE 10 @@ -69,7 +68,7 @@ SIMFairness == /\ \A s \in Servers : WF_vars(SignCommittableMessages(s)) /\ \A s \in Servers : WF_vars(AdvanceCommitIndex(s)) /\ \A s \in Servers : WF_vars(AppendRetiredCommitted(s)) - /\ \A s \in Servers : WF_vars(preVoteEnabled /\ BecomeCandidate(s)) + /\ \A s \in Servers : WF_vars(PreVoteEnabled \in preVoteStatus /\ BecomeCandidate(s)) /\ \A s \in Servers : WF_vars(BecomeLeader(s)) \* The following fairness conditions reference the original CCF actions \* and, thus, do not include the RandomElement conjunct. diff --git a/tla/consensus/Traceccfraft.cfg b/tla/consensus/Traceccfraft.cfg index f9a12f291cfb..af091437e803 100644 --- a/tla/consensus/Traceccfraft.cfg +++ b/tla/consensus/Traceccfraft.cfg @@ -62,6 +62,10 @@ CONSTANTS Leader = L_Leader None = L_None + PreVoteDisabled = PV_PreVoteDisabled + PreVoteCapable = PV_PreVoteCapable + PreVoteEnabled = PV_PreVoteEnabled + Active = R_Active RetirementOrdered = R_RetirementOrdered RetirementSigned = R_RetirementSigned diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 72f0e4aa75dd..67205f2404a0 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -116,13 +116,20 @@ Nil == ------------------------------------------------------------------------------ \* Global variables -\* Whether PreVote is enabled in this cluster +CONSTANTS + \* The node is ignorant of prevote + PreVoteDisabled, + \* The node is able to respond to prevote, but that is not enabled + PreVoteCapable, + \* The node will become a PreVoteCandidate before becoming a Candidate + PreVoteEnabled VARIABLE - preVoteEnabled - -PreVoteEnabledTypeInv == preVoteEnabled \in BOOLEAN + preVoteStatus -configVars == << preVoteEnabled >> +PreVoteStatusTypeInv == preVoteStatus \in { + {PreVoteDisabled}, + {PreVoteCapable}, + {PreVoteCapable, PreVoteEnabled}} \* A set representing requests and responses sent from one server \* to another. With CCF, we have message integrity and can ensure unique messages. @@ -350,7 +357,7 @@ LeaderVarsTypeInv == \* All variables; used for stuttering (asserting state hasn't changed). vars == << - preVoteEnabled, + preVoteStatus, reconfigurationVars, messageVars, serverVars, @@ -361,7 +368,7 @@ vars == << \* Invariant to check the type safety of all variables TypeInv == - /\ PreVoteEnabledTypeInv + /\ PreVoteStatusTypeInv /\ ReconfigurationVarsTypeInv /\ MessageVarsTypeInv /\ ServerVarsTypeInv @@ -628,7 +635,7 @@ InitLeaderVars == /\ matchIndex = [i \in Servers |-> [j \in Servers |-> 0]] InitOptionConsts == - /\ preVoteEnabled = FALSE + /\ preVoteStatus = {PreVoteDisabled} Init == /\ InitReconfigurationVars @@ -641,11 +648,12 @@ Init == \* Define state transitions BecomePreVoteCandidate(i) == - /\ preVoteEnabled + /\ PreVoteEnabled \in preVoteStatus \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} - \* Only servers that are followers/candidates can become candidates - /\ leadershipState[i] = Follower + \* Only servers that are followers/candidates can become pre-vote-candidates + \* Candidates can time out and become pre-vote-candidates for the next term + /\ leadershipState[i] \in {Follower, Candidate} /\ \* Check that the reconfiguration which added this node is at least committable \/ \E c \in DOMAIN configurations[i] : @@ -656,20 +664,19 @@ BecomePreVoteCandidate(i) == /\ leadershipState' = [leadershipState EXCEPT ![i] = PreVoteCandidate] /\ votesGranted' = [votesGranted EXCEPT ![i] = {i}] /\ UNCHANGED <> - /\ UNCHANGED <> + /\ UNCHANGED <> BecomeCandidate(i) == \* Only servers that haven't completed retirement can become candidates /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} \* Only servers that are followers/candidates can become candidates - /\ \/ /\ ~preVoteEnabled - /\ leadershipState[i] \in {Follower, Candidate} - \/ /\ preVoteEnabled - /\ leadershipState[i] = PreVoteCandidate - \* To become a Candidate, the PreVoteCandidate must have received votes from a majority in each active configuration - \* Only votes by nodes part of a given configuration should be tallied against it - /\ \A c \in DOMAIN configurations[i] : - (votesGranted[i] \intersect configurations[i][c]) \in Quorums[configurations[i][c]] + /\ IF PreVoteEnabled \notin preVoteStatus + THEN leadershipState[i] \in {Follower, Candidate} + ELSE /\ leadershipState[i] = PreVoteCandidate + \* To become a Candidate, the PreVoteCandidate must have received votes from a majority in each active configuration + \* Only votes by nodes part of a given configuration should be tallied against it + /\ \A c \in DOMAIN configurations[i] : + (votesGranted[i] \intersect configurations[i][c]) \in Quorums[configurations[i][c]] /\ \* Check that the reconfiguration which added this node is at least committable \/ \E c \in DOMAIN configurations[i] : @@ -682,13 +689,14 @@ BecomeCandidate(i) == \* Candidate votes for itself /\ votedFor' = [votedFor EXCEPT ![i] = i] /\ votesGranted' = [votesGranted EXCEPT ![i] = {i}] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Server i times out (becomes candidate) and votes for itself in the election of the next term \* At some point later (non-deterministically), the candidate will request votes from the other nodes. Timeout(i) == - \/ preVoteEnabled /\ BecomePreVoteCandidate(i) - \/ ~preVoteEnabled /\ BecomeCandidate(i) + IF PreVoteEnabled \notin preVoteStatus + THEN BecomeCandidate(i) + ELSE BecomePreVoteCandidate(i) \* Candidate i sends j a RequestVote request. RequestVote(i,j) == @@ -711,7 +719,7 @@ RequestVote(i,j) == \* Reconfiguration: Make sure j is in a configuration of i /\ IsInServerSet(j, i) /\ Send(msg) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Leader i sends j an AppendEntries request AppendEntries(i, j) == @@ -751,7 +759,7 @@ AppendEntries(i, j) == \* Record the most recent index we have sent to this node. \* raft.h::send_append_entries /\ sentIndex' = [sentIndex EXCEPT ![i][j] = @ + Len(m.entries)] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Candidate i transitions to leader. BecomeLeader(i) == @@ -774,7 +782,7 @@ BecomeLeader(i) == \* been rolled back as it was unsigned /\ membershipState' = [membershipState EXCEPT ![i] = IF @ = RetirementOrdered THEN Active ELSE @] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Leader i receives a client request to add to the log. The consensus spec is agnostic to request payloads, \* and does not model or differentiate them. See the consistency spec (tla/consistency/*) for a specification @@ -786,7 +794,7 @@ ClientRequest(i) == /\ membershipState[i] # RetiredCommitted \* Add new request to leader's log /\ log' = [log EXCEPT ![i] = Append(@, [term |-> currentTerm[i], contentType |-> TypeEntry]) ] - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF: Signed commits \* In CCF, the leader periodically signs the latest log prefix. Only these signatures are committable in CCF. @@ -809,7 +817,7 @@ SignCommittableMessages(i) == /\ IF membershipState[i] = RetirementOrdered THEN membershipState' = [membershipState EXCEPT ![i] = RetirementSigned] ELSE UNCHANGED membershipState - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF: Reconfiguration of servers \* In the TLA+ model, a reconfiguration is initiated by the Leader which appends an arbitrary new configuration to its own log. @@ -847,7 +855,7 @@ ChangeConfigurationInt(i, newConfiguration) == /\ IF membershipState[i] = Active /\ i \notin newConfiguration THEN membershipState' = [membershipState EXCEPT ![i] = RetirementOrdered] ELSE UNCHANGED membershipState - /\ UNCHANGED <> + /\ UNCHANGED <> ChangeConfiguration(i) == \* Reconfigure to any *non-empty* subset of servers. ChangeConfigurationInt checks that the new @@ -867,7 +875,7 @@ AppendRetiredCommitted(i) == term |-> currentTerm[i], contentType |-> TypeRetired, retired |-> retire_committed_nodes])] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Leader i advances its commitIndex to the next possible Index. @@ -935,7 +943,7 @@ AdvanceCommitIndex(i) == IN Send(msg) ELSE UNCHANGED <> /\ retirementCompleted' = [retirementCompleted EXCEPT ![i] = NextRetirementCompleted(retirementCompleted[i], configurations[i], log[i], commitIndex'[i], i)] - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF supports checkQuorum which enables a leader to choose to abdicate leadership. CheckQuorum(i) == @@ -946,7 +954,7 @@ CheckQuorum(i) == \E n \in configurations[i][c]: n /= i /\ leadershipState' = [leadershipState EXCEPT ![i] = Follower] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = TRUE] - /\ UNCHANGED <> + /\ UNCHANGED <> ------------------------------------------------------------------------------ \* Message handlers @@ -961,9 +969,17 @@ HandleRequestVoteRequest(i, j, m) == \* CCF change: Log is only okay up to signatures, \* not any message in the log /\ m.lastCommittableIndex >= MaxCommittableIndex(log[i]) - grant == /\ m.term = currentTerm[i] - /\ logOk - /\ votedFor[i] \in {Nil, j} + grant_pre_vote_disabled == /\ m.term = currentTerm[i] + /\ logOk + /\ votedFor[i] \in {Nil, j} + grant_pre_vote_capable == /\ logOk + /\ IF ~m.isPreVote + THEN /\ m.term = currentTerm[i] + /\ votedFor[i] \in {Nil, j} + ELSE m.term >= currentTerm[i] + grant == IF PreVoteDisabled \in preVoteStatus + THEN grant_pre_vote_disabled + ELSE grant_pre_vote_capable IN /\ m.term <= currentTerm[i] /\ \/ grant /\ \/ /\ ~m.isPreVote /\ votedFor' = [votedFor EXCEPT ![i] = j] @@ -976,7 +992,7 @@ HandleRequestVoteRequest(i, j, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> \* Server i receives a RequestVote response from server j with @@ -991,7 +1007,7 @@ HandleRequestVoteResponse(i, j, m) == \/ /\ ~m.voteGranted /\ UNCHANGED votesGranted /\ Discard(m) - /\ UNCHANGED <> \* Server i replies to a AppendEntries request from server j with a NACK @@ -1033,7 +1049,7 @@ RejectAppendEntriesRequest(i, j, m, logOk) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Candidate i steps down to follower in the same term after receiving a message m from a leader in the current term \* Must check that m is an AppendEntries message before returning to follower state @@ -1043,7 +1059,7 @@ ReturnToFollowerState(i, m) == /\ leadershipState' = [leadershipState EXCEPT ![i] = Follower] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = TRUE] \* Note that the set of messages is unchanged as m is discarded - /\ UNCHANGED <> \* Follower i receives a AppendEntries from leader j for log entries it already has @@ -1071,7 +1087,7 @@ AppendEntriesAlreadyDone(i, j, index, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Follower i receives an AppendEntries request m where it needs to roll back first \* This action rolls back the log and leaves m in messages for further processing @@ -1087,7 +1103,7 @@ ConflictAppendEntriesRequest(i, index, m) == /\ configurations' = [configurations EXCEPT ![i] = ConfigurationsToIndex(i,Len(new_log))] /\ membershipState' = [membershipState EXCEPT ![i] = CalcMembershipState(log'[i], commitIndex[i], i)] /\ isNewFollower' = [isNewFollower EXCEPT ![i] = FALSE] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Follower i receives an AppendEntries request m from leader j for log entries which directly follow its log NoConflictAppendEntriesRequest(i, j, m) == @@ -1131,7 +1147,7 @@ NoConflictAppendEntriesRequest(i, j, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> AcceptAppendEntriesRequest(i, j, logOk, m) == /\ m.term = currentTerm[i] @@ -1174,7 +1190,7 @@ HandleAppendEntriesResponse(i, j, m) == \* "If AppendEntries fails because of log inconsistency: decrement nextIndex (aka sentIndex +1) and retry" /\ UNCHANGED matchIndex /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Any message with a newer term causes the recipient to advance its term first. \* Note that UpdateTerm does not discard message m from the set of messages so this @@ -1198,13 +1214,13 @@ UpdateTerm(i, j, m) == /\ membershipState' = [membershipState EXCEPT ![i] = IF @ = RetirementOrdered THEN Active ELSE @] \* messages is unchanged so m can be processed further. - /\ UNCHANGED <> + /\ UNCHANGED <> \* Responses with stale terms are ignored. DropStaleResponse(i, j, m) == /\ m.term < currentTerm[i] /\ Discard(m) - /\ UNCHANGED <> DropResponseWhenNotInState(i, j, m) == @@ -1213,7 +1229,7 @@ DropResponseWhenNotInState(i, j, m) == \/ /\ m.type = RequestVoteResponse /\ leadershipState[i] \in LeadershipStates \ { Candidate, PreVoteCandidate } /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Drop messages if they are irrelevant to the node DropIgnoredMessage(i,j,m) == @@ -1236,7 +1252,7 @@ DropIgnoredMessage(i,j,m) == \/ /\ membershipState[i] = RetiredCommitted /\ m.type /= AppendEntriesRequest /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Receive a message. @@ -1291,7 +1307,6 @@ RcvProposeVoteRequest(i, j) == /\ m.term = currentTerm[i] /\ BecomeCandidate(m.dest) /\ Discard(m) - /\ UNCHANGED configVars \* Node i receives a message from node j. Receive(i, j) == @@ -1313,7 +1328,7 @@ Receive(i, j) == \* Defines how the variables may transition, given a node i. NextInt(i) == \/ Timeout(i) - \/ (preVoteEnabled /\ BecomeCandidate(i)) + \/ (PreVoteEnabled \in preVoteStatus /\ BecomeCandidate(i)) \/ BecomeLeader(i) \/ ClientRequest(i) \/ SignCommittableMessages(i) @@ -1343,7 +1358,7 @@ Fairness == /\ \A s \in Servers : WF_vars(SignCommittableMessages(s)) /\ \A s \in Servers : WF_vars(AdvanceCommitIndex(s)) /\ \A s \in Servers : WF_vars(AppendRetiredCommitted(s)) - /\ \A s \in Servers : WF_vars(preVoteEnabled /\ BecomeCandidate(s)) + /\ \A s \in Servers : WF_vars(PreVoteEnabled \in preVoteStatus /\ BecomeCandidate(s)) /\ \A s \in Servers : WF_vars(BecomeLeader(s)) /\ \A s \in Servers : WF_vars(Timeout(s)) /\ \A s \in Servers : WF_vars(ChangeConfiguration(s)) @@ -1645,14 +1660,17 @@ PermittedLogChangesProp == StateTransitionsProp == [][\A i \in Servers : /\ leadershipState[i] = None => leadershipState'[i] \in {None, Follower} - /\ \/ /\ ~preVoteEnabled - /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, Candidate} - /\ leadershipState[i] /= PreVoteCandidate - /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, Candidate, Leader} - \/ /\ preVoteEnabled - /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} - /\ leadershipState[i] = PreVoteCandidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} - /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate, Leader} + /\ IF PreVoteEnabled \notin preVoteStatus + THEN + \* A follower can become a Candidate via a timeout or a ProposeVoteRequest + /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, Candidate} + /\ leadershipState[i] /= PreVoteCandidate + /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, Candidate, Leader} + ELSE + \* A follower can become a PreVoteCandidate via a timeout, or a Candidate via ProposeVoteRequest + /\ leadershipState[i] = Follower => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} + /\ leadershipState[i] = PreVoteCandidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate} + /\ leadershipState[i] = Candidate => leadershipState'[i] \in {Follower, PreVoteCandidate, Candidate, Leader} /\ leadershipState[i] = Leader => leadershipState'[i] \in {Follower, Leader} ]_vars @@ -1683,6 +1701,13 @@ LeaderProp == \* There is repeatedly a non-retired leader. []<><<\E i \in Servers : leadershipState[i] = Leader /\ membershipState[i] # RetiredCommitted>>_vars +VotesGrantedMonotonicProp == + \A i \in Servers: [][ + /\ leadershipState[i] = leadershipState'[i] + /\ currentTerm[i] = currentTerm'[i] + => votesGranted[i] \subseteq votesGranted'[i] + ]_vars + ------------------------------------------------------------------------------ \* Refinement of the more high-level specification abs.tla that abstracts the \* asynchronous network and the message passing between nodes. Instead, any From 7100bb6814e225d0a92467a51095c5da6e986e8c Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 16:19:32 +0000 Subject: [PATCH 10/17] Add debug non-trivial leader --- tla/consensus/SIMccfraft.cfg | 3 ++- tla/consensus/ccfraft.tla | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/tla/consensus/SIMccfraft.cfg b/tla/consensus/SIMccfraft.cfg index ac27bc743e51..6e5c183ecd92 100644 --- a/tla/consensus/SIMccfraft.cfg +++ b/tla/consensus/SIMccfraft.cfg @@ -124,4 +124,5 @@ INVARIANTS \* DebugInvAllMessagesProcessable \* DebugInvRetirementReachable \* DebugInvUpToDepth - \* DebugMoreUpToDateCorrectInv \ No newline at end of file + \* DebugMoreUpToDateCorrectInv + \* DebugNonTrivialLeaderInv \ No newline at end of file diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 67205f2404a0..932746e45684 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -1774,4 +1774,9 @@ DebugCommittedEntriesTermsInv == k <= commitIndex[i] => log[i][k].term >= log[j][k].term +DebugNonTrivialLeaderInv == + ~\E i \in Servers: + /\ leadershipState[i] = Leader + /\ currentTerm[i] > StartTerm + =============================================================================== \ No newline at end of file From 0d2a3863f2be3b04e6fe3dded44272ab930cbc41 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 16:24:08 +0000 Subject: [PATCH 11/17] Remove debug inv from model checking --- tla/consensus/MCccfraft.cfg | 1 - tla/consensus/MCccfraft.tla | 5 ----- 2 files changed, 6 deletions(-) diff --git a/tla/consensus/MCccfraft.cfg b/tla/consensus/MCccfraft.cfg index 7b658d5abae5..f4fe83056c1b 100644 --- a/tla/consensus/MCccfraft.cfg +++ b/tla/consensus/MCccfraft.cfg @@ -97,4 +97,3 @@ INVARIANTS RetiredCommittedInv RetirementCompletedNotInConfigsInv RetirementCompletedAreRetirementCompletedInv - DebugInvLeaderAfterInit diff --git a/tla/consensus/MCccfraft.tla b/tla/consensus/MCccfraft.tla index 2ea89a8dae0a..af168e902c02 100644 --- a/tla/consensus/MCccfraft.tla +++ b/tla/consensus/MCccfraft.tla @@ -143,11 +143,6 @@ DebugNotTooManySigsInv == \A i \in Servers: FoldSeq(LAMBDA e, count: IF e.contentType = TypeSignature THEN count + 1 ELSE count, 0, log[i]) < 8 -DebugInvLeaderAfterInit == - \lnot \E i \in Servers : /\ PreVoteEnabled \in preVoteStatus - /\ leadershipState[i] = Leader - /\ currentTerm[i] > 2 - ---- \* Initialize the counters for the Debug invariants to 0. From e85c3ddcbfaf8a7ce278554618008cd72a5bf5e0 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 16:56:17 +0000 Subject: [PATCH 12/17] Snags --- tla/consensus/MCccfraft.tla | 4 ++-- tla/consensus/SIMccfraft.cfg | 2 +- tla/consensus/SIMccfraft.tla | 2 +- tla/consensus/Traceccfraft.cfg | 2 ++ tla/consensus/Traceccfraft.tla | 2 ++ tla/consensus/ccfraft.tla | 6 +++--- 6 files changed, 11 insertions(+), 7 deletions(-) diff --git a/tla/consensus/MCccfraft.tla b/tla/consensus/MCccfraft.tla index af168e902c02..ea5b7e94ccfd 100644 --- a/tla/consensus/MCccfraft.tla +++ b/tla/consensus/MCccfraft.tla @@ -106,13 +106,13 @@ MCSend(msg) == /\ n.type = AppendEntriesResponse /\ CCF!Send(msg) -MCInitOptionConsts == PreVoteStatusTypeInv +MCInitPreVoteStatus == PreVoteStatusTypeInv MCInit == /\ InitMessagesVars /\ InitCandidateVars /\ InitLeaderVars - /\ MCInitOptionConsts + /\ MCInitPreVoteStatus /\ IF Cardinality(Configurations[1]) = 1 \* If the first config is just one node, we can start with a two-tx log and a single config. THEN InitLogConfigServerVars(Configurations[1], StartLog) diff --git a/tla/consensus/SIMccfraft.cfg b/tla/consensus/SIMccfraft.cfg index 6e5c183ecd92..4a55e1f4e3b6 100644 --- a/tla/consensus/SIMccfraft.cfg +++ b/tla/consensus/SIMccfraft.cfg @@ -56,7 +56,7 @@ CONSTANTS Extend <- [abs]ABSExtend CopyMaxAndExtend <- [abs]ABSCopyMaxAndExtend - InitOptionConsts <- SIMInitOptionConsts + InitPreVoteStatus <- SIMInitPreVoteStatus _PERIODIC Periodically diff --git a/tla/consensus/SIMccfraft.tla b/tla/consensus/SIMccfraft.tla index 2b2dcd3db6c7..a78a5d659c6f 100644 --- a/tla/consensus/SIMccfraft.tla +++ b/tla/consensus/SIMccfraft.tla @@ -16,7 +16,7 @@ SIMInitReconfigurationVars == \* Start with any subset of servers in the active configuration. \/ CCF!InitReconfigurationVars -SIMInitOptionConsts == PreVoteStatusTypeInv +SIMInitPreVoteStatus == PreVoteStatusTypeInv LOCAL R == 1..IF "R" \in DOMAIN IOEnv THEN atoi(IOEnv.R) ELSE 10 diff --git a/tla/consensus/Traceccfraft.cfg b/tla/consensus/Traceccfraft.cfg index af091437e803..0e7cac20ccf5 100644 --- a/tla/consensus/Traceccfraft.cfg +++ b/tla/consensus/Traceccfraft.cfg @@ -54,6 +54,8 @@ CONSTANTS InitReconfigurationVars <- TraceInitReconfigurationVars + InitPreVoteStatus <- TraceInitPreVoteStatus + Nil = Nil Follower = L_Follower diff --git a/tla/consensus/Traceccfraft.tla b/tla/consensus/Traceccfraft.tla index 4fa1cc36515f..5fa0b984eff2 100644 --- a/tla/consensus/Traceccfraft.tla +++ b/tla/consensus/Traceccfraft.tla @@ -111,6 +111,8 @@ TraceAppendEntriesBatchsize(i, j) == TraceInitReconfigurationVars == /\ InitLogConfigServerVars({TraceLog[1].msg.state.node_id}, StartLog) +TraceInitPreVoteStatus == PreVoteStatusTypeInv + ------------------------------------------------------------------------------------- VARIABLE l, ts diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 932746e45684..cb0edc3cde75 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -119,7 +119,7 @@ Nil == CONSTANTS \* The node is ignorant of prevote PreVoteDisabled, - \* The node is able to respond to prevote, but that is not enabled + \* The node is able to respond to prevote messages PreVoteCapable, \* The node will become a PreVoteCandidate before becoming a Candidate PreVoteEnabled @@ -634,7 +634,7 @@ InitCandidateVars == InitLeaderVars == /\ matchIndex = [i \in Servers |-> [j \in Servers |-> 0]] -InitOptionConsts == +InitPreVoteStatus == /\ preVoteStatus = {PreVoteDisabled} Init == @@ -642,7 +642,7 @@ Init == /\ InitMessagesVars /\ InitCandidateVars /\ InitLeaderVars - /\ InitOptionConsts + /\ InitPreVoteStatus ------------------------------------------------------------------------------ \* Define state transitions From c4504ee596ad63850bda6b9f950a9cf629f62d14 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 17:09:25 +0000 Subject: [PATCH 13/17] Clarify membershipState of become.*Candidate --- tla/consensus/ccfraft.tla | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index cb0edc3cde75..cbb1642b3f01 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -650,7 +650,7 @@ Init == BecomePreVoteCandidate(i) == /\ PreVoteEnabled \in preVoteStatus \* Only servers that haven't completed retirement can become candidates - /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} + /\ membershipState[i] \in (MembershipStates \ {RetiredCommitted}) \* Only servers that are followers/candidates can become pre-vote-candidates \* Candidates can time out and become pre-vote-candidates for the next term /\ leadershipState[i] \in {Follower, Candidate} @@ -668,7 +668,7 @@ BecomePreVoteCandidate(i) == BecomeCandidate(i) == \* Only servers that haven't completed retirement can become candidates - /\ membershipState[i] \in {Active, RetirementOrdered, RetirementSigned, RetirementCompleted} + /\ membershipState[i] \in (MembershipStates \ {RetiredCommitted}) \* Only servers that are followers/candidates can become candidates /\ IF PreVoteEnabled \notin preVoteStatus THEN leadershipState[i] \in {Follower, Candidate} From c4891d92ab47a1be1e32031ce9a79c0382e838af Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 17:24:59 +0000 Subject: [PATCH 14/17] Rejig naming --- tla/consensus/ccfraft.tla | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index cbb1642b3f01..925b4fe93004 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -663,8 +663,7 @@ BecomePreVoteCandidate(i) == \/ i \in retirementCompleted[i] /\ leadershipState' = [leadershipState EXCEPT ![i] = PreVoteCandidate] /\ votesGranted' = [votesGranted EXCEPT ![i] = {i}] - /\ UNCHANGED <> - /\ UNCHANGED <> + /\ UNCHANGED <> BecomeCandidate(i) == \* Only servers that haven't completed retirement can become candidates From 70e7819ad038c00e7a3212ffdd0995a9bbacd63d Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Tue, 28 Oct 2025 17:48:16 +0000 Subject: [PATCH 15/17] Constrain to only use pre-vote responses when a pre-vote candidate --- tla/consensus/ccfraft.tla | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index 925b4fe93004..c0ba14e47b45 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -172,6 +172,7 @@ RequestVoteRequestTypeOK(m) == RequestVoteResponseTypeOK(m) == /\ m.type = RequestVoteResponse /\ m.voteGranted \in BOOLEAN + /\ m.isPreVote \in BOOLEAN ProposeVoteRequestTypeOK(m) == /\ m.type = ProposeVoteRequest @@ -988,6 +989,7 @@ HandleRequestVoteRequest(i, j, m) == /\ Reply([type |-> RequestVoteResponse, term |-> currentTerm[i], voteGranted |-> grant, + isPreVote |-> m.isPreVote, source |-> i, dest |-> j], m) @@ -999,7 +1001,8 @@ HandleRequestVoteRequest(i, j, m) == HandleRequestVoteResponse(i, j, m) == /\ m.term = currentTerm[i] \* Only PreVoteCandidates and Candidates need to tally votes - /\ leadershipState[i] \in {PreVoteCandidate, Candidate} + /\ \/ m.isPreVote /\ leadershipState[i] = PreVoteCandidate + \/ ~m.isPreVote /\ leadershipState[i] = Candidate /\ \/ /\ m.voteGranted /\ votesGranted' = [votesGranted EXCEPT ![i] = votesGranted[i] \cup {j}] @@ -1226,7 +1229,8 @@ DropResponseWhenNotInState(i, j, m) == \/ /\ m.type = AppendEntriesResponse /\ leadershipState[i] \in LeadershipStates \ { Leader } \/ /\ m.type = RequestVoteResponse - /\ leadershipState[i] \in LeadershipStates \ { Candidate, PreVoteCandidate } + /\ \/ m.isPreVote /\ leadershipState[i] /= PreVoteCandidate + \/ ~m.isPreVote /\ leadershipState[i] /= Candidate /\ Discard(m) /\ UNCHANGED <> From fd864cbf80650ae260cce65a2dacaf01090b624b Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Wed, 29 Oct 2025 09:14:02 +0000 Subject: [PATCH 16/17] Make trace validation pass --- tla/consensus/Traceccfraft.tla | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tla/consensus/Traceccfraft.tla b/tla/consensus/Traceccfraft.tla index 5fa0b984eff2..8624ffb53b0e 100644 --- a/tla/consensus/Traceccfraft.tla +++ b/tla/consensus/Traceccfraft.tla @@ -151,7 +151,7 @@ IsDropPendingTo == /\ IsEvent("drop_pending_to") /\ Network!DropMessage(logline.msg.to_node_id, LAMBDA msg: IsMessage(msg, logline.msg.to_node_id, logline.msg.from_node_id, logline)) - /\ UNCHANGED <> + /\ UNCHANGED <> IsTimeout == /\ IsEvent("become_candidate") @@ -404,7 +404,7 @@ IsRcvProposeVoteRequest == /\ m.type = ProposeVoteRequest /\ m.term = logline.msg.packet.term /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> /\ Range(logline.msg.state.committable_indices) \subseteq CommittableIndices(logline.msg.state.node_id) /\ commitIndex[logline.msg.state.node_id] = logline.msg.state.commit_idx /\ leadershipState[logline.msg.state.node_id] = ToLeadershipState[logline.msg.state.leadership_state] From 542ac53cf8687dd6c7e726f5853da21fa6dd13f6 Mon Sep 17 00:00:00 2001 From: cjen1-msft Date: Wed, 29 Oct 2025 13:31:21 +0000 Subject: [PATCH 17/17] Snag --- tla/consensus/ccfraft.tla | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index c0ba14e47b45..f949b76109d5 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -1344,7 +1344,7 @@ NextInt(i) == \/ \E j \in Servers : Receive(i, j) Next == - /\ \E i \in Servers: NextInt(i) + \E i \in Servers: NextInt(i) Fairness == \* Network actions