diff --git a/tla/consensus/MCccfraft.cfg b/tla/consensus/MCccfraft.cfg index db57df0718b6..feba850add95 100644 --- a/tla/consensus/MCccfraft.cfg +++ b/tla/consensus/MCccfraft.cfg @@ -15,7 +15,6 @@ CONSTANTS ClientRequest <- MCClientRequest SignCommittableMessages <- MCSignCommittableMessages ChangeConfigurationInt <- MCChangeConfigurationInt - NotifyCommit <- MCNotifyCommit Nil = Nil @@ -29,7 +28,6 @@ CONSTANTS RequestVoteResponse = RequestVoteResponse AppendEntriesRequest = AppendEntriesRequest AppendEntriesResponse = AppendEntriesResponse - NotifyCommitMessage = NotifyCommitMessage ProposeVoteRequest = ProposeVoteRequest OrderedNoDup = OrderedNoDup diff --git a/tla/consensus/MCccfraft.tla b/tla/consensus/MCccfraft.tla index a1890a759340..b18450c1df74 100644 --- a/tla/consensus/MCccfraft.tla +++ b/tla/consensus/MCccfraft.tla @@ -88,13 +88,6 @@ MCSend(msg) == /\ n.type = AppendEntriesResponse /\ CCF!Send(msg) -\* CCF: Limit the number of times a RetiredLeader server sends commit -\* notifications per commit Index and server -MCNotifyCommit(i,j) == - /\ \/ commitsNotified[i][1] < commitIndex[i] - \/ commitsNotified[i][2] < MaxCommitsNotified - /\ CCF!NotifyCommit(i,j) - \* Limit max number of simultaneous candidates MCInMaxSimultaneousCandidates(i) == Cardinality({ s \in GetServerSetForIndex(i, commitIndex[i]) : state[s] = Candidate}) < 1 @@ -124,7 +117,7 @@ mc_spec == Symmetry == Permutations(Servers) \* Include all variables in the view, which is similar to defining no view. -View == << reconfigurationVars, <>, serverVars, candidateVars, leaderVars, logVars >> +View == << reconfigurationVars, <>, serverVars, candidateVars, leaderVars, logVars >> ---- diff --git a/tla/consensus/MCccfraftAtomicReconfig.cfg b/tla/consensus/MCccfraftAtomicReconfig.cfg index 64f27fbfcade..b01a0a3d2290 100644 --- a/tla/consensus/MCccfraftAtomicReconfig.cfg +++ b/tla/consensus/MCccfraftAtomicReconfig.cfg @@ -15,7 +15,6 @@ CONSTANTS ClientRequest <- MCClientRequest SignCommittableMessages <- MCSignCommittableMessages ChangeConfigurationInt <- MCChangeConfigurationInt - NotifyCommit <- MCNotifyCommit Nil = Nil @@ -29,7 +28,6 @@ CONSTANTS RequestVoteResponse = RequestVoteResponse AppendEntriesRequest = AppendEntriesRequest AppendEntriesResponse = AppendEntriesResponse - NotifyCommitMessage = NotifyCommitMessage ProposeVoteRequest = ProposeVoteRequest OrderedNoDup = OrderedNoDup diff --git a/tla/consensus/MCccfraftWithReconfig.cfg b/tla/consensus/MCccfraftWithReconfig.cfg index 21663b2df412..a4a9ade74893 100644 --- a/tla/consensus/MCccfraftWithReconfig.cfg +++ b/tla/consensus/MCccfraftWithReconfig.cfg @@ -15,7 +15,6 @@ CONSTANTS ClientRequest <- MCClientRequest SignCommittableMessages <- MCSignCommittableMessages ChangeConfigurationInt <- MCChangeConfigurationInt - NotifyCommit <- MCNotifyCommit Nil = Nil @@ -29,7 +28,6 @@ CONSTANTS RequestVoteResponse = RequestVoteResponse AppendEntriesRequest = AppendEntriesRequest AppendEntriesResponse = AppendEntriesResponse - NotifyCommitMessage = NotifyCommitMessage ProposeVoteRequest = ProposeVoteRequest OrderedNoDup = OrderedNoDup diff --git a/tla/consensus/SIMCoverageccfraft.cfg b/tla/consensus/SIMCoverageccfraft.cfg index 4c204ee17962..118cb833dea0 100644 --- a/tla/consensus/SIMCoverageccfraft.cfg +++ b/tla/consensus/SIMCoverageccfraft.cfg @@ -15,7 +15,6 @@ CONSTANTS RequestVoteResponse = RequestVoteResponse AppendEntriesRequest = AppendEntriesRequest AppendEntriesResponse = AppendEntriesResponse - NotifyCommitMessage = NotifyCommitMessage TypeEntry = Entry TypeSignature = Signature diff --git a/tla/consensus/SIMCoverageccfraft.tla b/tla/consensus/SIMCoverageccfraft.tla index 17e5aa628036..fbfaabb55833 100644 --- a/tla/consensus/SIMCoverageccfraft.tla +++ b/tla/consensus/SIMCoverageccfraft.tla @@ -37,7 +37,7 @@ SIMCoverageSpec == CSVFile == "SIMCoverageccfraft_S" \o ToString(Cardinality(Servers)) \o ".csv" CSVColumnHeaders == - "Spec#P#Q#R#reconfigurationCount#committedLog#clientRequests#commitsNotified11#commitsNotified12#currentTerm#state#node" + "Spec#P#Q#R#reconfigurationCount#committedLog#clientRequests#currentTerm#state#node" ASSUME CSVRecords(CSVFile) = 0 => @@ -45,10 +45,9 @@ ASSUME StatisticsStateConstraint == (TLCGet("level") > TLCGet("config").depth) => - TLCDefer(\A srv \in Servers : CSVWrite("%1$s#%2$s#%3$s#%4$s#%5$s#%6$s#%7$s#%8$s#%9$s#%10$s#%11$s#%12$s", + TLCDefer(\A srv \in Servers : CSVWrite("%1$s#%2$s#%3$s#%4$s#%5$s#%6$s#%7$s#%8$s#%9$s#%10$s", << conf[1], Cardinality(conf[2]), Cardinality(conf[3]), Cardinality(conf[4]), reconfigurationCount, committedLog.index, clientRequests, - commitsNotified[srv][1], commitsNotified[srv][2], currentTerm[srv], state[srv], srv>>, CSVFile)) ============================================================================= diff --git a/tla/consensus/SIMccfraft.cfg b/tla/consensus/SIMccfraft.cfg index 9f82642d1841..473f388cdaa8 100644 --- a/tla/consensus/SIMccfraft.cfg +++ b/tla/consensus/SIMccfraft.cfg @@ -15,7 +15,6 @@ CONSTANTS RequestVoteResponse = RequestVoteResponse AppendEntriesRequest = AppendEntriesRequest AppendEntriesResponse = AppendEntriesResponse - NotifyCommitMessage = NotifyCommitMessage ProposeVoteRequest = ProposeVoteRequest OrderedNoDup = OrderedNoDup diff --git a/tla/consensus/SIMccfraft.tla b/tla/consensus/SIMccfraft.tla index 494a7d17a2a0..384cdece4228 100644 --- a/tla/consensus/SIMccfraft.tla +++ b/tla/consensus/SIMccfraft.tla @@ -23,7 +23,6 @@ Forward == \/ \E i \in Servers : BecomeLeader(i) \/ \E i \in Servers : ClientRequest(i) \/ \E i \in Servers : SignCommittableMessages(i) - \/ \E i, j \in Servers : NotifyCommit(i,j) \/ \E i \in Servers : AdvanceCommitIndex(i) \/ \E i, j \in Servers : AppendEntries(i, j) \/ \E i, j \in Servers : Receive(i, j) diff --git a/tla/consensus/Traceccfraft.cfg b/tla/consensus/Traceccfraft.cfg index 762e51385b46..203fe98f979d 100644 --- a/tla/consensus/Traceccfraft.cfg +++ b/tla/consensus/Traceccfraft.cfg @@ -57,7 +57,6 @@ CONSTANTS RequestVoteResponse = RequestVoteResponse AppendEntriesRequest = AppendEntriesRequest AppendEntriesResponse = AppendEntriesResponse - NotifyCommitMessage = NotifyCommitMessage ProposeVoteRequest = ProposeVoteRequest OrderedNoDup = OrderedNoDup diff --git a/tla/consensus/Traceccfraft.tla b/tla/consensus/Traceccfraft.tla index ea1679e37639..a1417bdcc731 100644 --- a/tla/consensus/Traceccfraft.tla +++ b/tla/consensus/Traceccfraft.tla @@ -141,7 +141,7 @@ logline == \* accepting that lost messages remain in messages. DropMessages == /\ l \in 1..Len(TraceLog) - /\ UNCHANGED <> + /\ UNCHANGED <> /\ UNCHANGED <> /\ Network!DropMessages(logline.msg.state.node_id) @@ -456,7 +456,6 @@ TraceDifferentialInv == \* /\ d.removedFromConfiguration = removedFromConfiguration \* /\ d.configurations = configurations \* /\ d.messages = messages - \* /\ d.commitsNotified = commitsNotified \* /\ d.currentTerm = currentTerm \* /\ d.state = state \* /\ d.votedFor = votedFor @@ -484,7 +483,6 @@ TraceAlias == \* ClientRequest |-> [ i \in Servers |-> ENABLED ClientRequest(i) ], \* SignCommittableMessages |-> [ i \in Servers |-> ENABLED SignCommittableMessages(i) ], \* ChangeConfiguration |-> [ i \in Servers |-> ENABLED ChangeConfiguration(i) ], - \* NotifyCommit |-> [ i,j \in Servers |-> ENABLED NotifyCommit(i,j) ], \* AdvanceCommitIndex |-> [ i \in Servers |-> ENABLED AdvanceCommitIndex(i) ], \* AppendEntries |-> [ i,j \in Servers |-> ENABLED AppendEntries(i, j) ], \* CheckQuorum |-> [ i \in Servers |-> ENABLED CheckQuorum(i) ], diff --git a/tla/consensus/ccfraft.tla b/tla/consensus/ccfraft.tla index c45e55e6122c..5bb28c7e98e5 100644 --- a/tla/consensus/ccfraft.tla +++ b/tla/consensus/ccfraft.tla @@ -69,7 +69,6 @@ CONSTANTS RequestVoteResponse, AppendEntriesRequest, AppendEntriesResponse, - NotifyCommitMessage, ProposeVoteRequest \* CCF: Content types (Normal entry or a signature that signs @@ -180,10 +179,6 @@ RequestVoteResponseTypeOK(m) == /\ m.type = RequestVoteResponse /\ m.voteGranted \in BOOLEAN -NotifyCommitMessageTypeOK(m) == - /\ m.type = NotifyCommitMessage - /\ m.commitIndex \in Nat - ProposeVoteRequestTypeOK(m) == /\ m.type = ProposeVoteRequest /\ m.term \in Nat @@ -198,28 +193,14 @@ MessagesTypeInv == \/ AppendEntriesResponseTypeOK(m) \/ RequestVoteRequestTypeOK(m) \/ RequestVoteResponseTypeOK(m) - \/ NotifyCommitMessageTypeOK(m) \/ ProposeVoteRequestTypeOK(m) -\* CCF: After reconfiguration, a RetiredLeader leader may need to notify servers -\* of the current commit level to ensure that no deadlock is reached through -\* leaving the network after retirement (as that would lead to endless leader -\* re-elects and drop-outs until f is reached and network fails). -VARIABLE commitsNotified - -CommitsNotifiedTypeInv == - \A i \in Servers : - /\ commitsNotified[i][1] \in Nat - /\ commitsNotified[i][2] \in Nat - messageVars == << - messages, - commitsNotified + messages >> MessageVarsTypeInv == /\ MessagesTypeInv - /\ CommitsNotifiedTypeInv ------------------------------------------------------------------------------ \* The following variables are all per server (functions with domain Servers). @@ -491,7 +472,6 @@ InitReconfigurationVars == InitMessagesVars == /\ Network!InitMessageVar - /\ commitsNotified = [i \in Servers |-> <<0,0>>] \* i.e., <> InitServerVars == /\ currentTerm = [i \in Servers |-> 0] @@ -558,7 +538,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) == @@ -593,7 +573,7 @@ AppendEntries(i, j) == \* Record the most recent index we have sent to this node. \* (see https://github.com/microsoft/CCF/blob/9fbde45bf5ab856ca7bcf655e8811dc7baf1e8a3/src/consensus/aft/raft.h#L935-L936) /\ nextIndex' = [nextIndex EXCEPT ![i][j] = @ + Len(m.entries)] - /\ UNCHANGED <> + /\ UNCHANGED <> \* Candidate i transitions to leader. BecomeLeader(i) == @@ -731,27 +711,7 @@ AdvanceCommitIndex(i) == ELSE UNCHANGED <> \* Otherwise, Configuration and states remain unchanged ELSE UNCHANGED <> - /\ UNCHANGED <> - -\* CCF: RetiredLeader server i notifies the current commit level to server j -\* This allows to retire gracefully instead of deadlocking the system through removing itself from the network. -NotifyCommit(i,j) == - \* Only RetiredLeader servers send these commit messages - /\ state[i] = RetiredLeader - \* Only send notifications of commit to servers in the server set - /\ IsInServerSetForIndex(j, i, commitIndex[i]) - /\ commitsNotified[i][1] < commitIndex[i] - /\ LET new_notified == IF commitsNotified[i][1] = commitIndex[i] - THEN <> - ELSE <> - IN commitsNotified' = [commitsNotified EXCEPT ![i] = new_notified] - /\ LET msg == [type |-> NotifyCommitMessage, - commitIndex |-> commitIndex[i], - term |-> currentTerm[i], - source |-> i, - dest |-> j] - IN Send(msg) - /\ UNCHANGED <> + /\ UNCHANGED <> \* CCF supports checkQuorum which enables a leader to choose to abdicate leadership. CheckQuorum(i) == @@ -783,7 +743,7 @@ HandleRequestVoteRequest(i, j, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Server i receives a RequestVote response from server j with \* m.term = currentTerm[i]. @@ -796,7 +756,7 @@ HandleRequestVoteResponse(i, j, m) == \/ /\ ~m.voteGranted /\ UNCHANGED votesGranted /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Server i receives a RequestVote request from server j with \* m.term < currentTerm[i]. @@ -833,13 +793,13 @@ RejectAppendEntriesRequest(i, j, m, logOk) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> ReturnToFollowerState(i, m) == /\ m.term = currentTerm[i] /\ state[i] = Candidate /\ state' = [state EXCEPT ![i] = Follower] - /\ UNCHANGED <> + /\ UNCHANGED <> AppendEntriesAlreadyDone(i, j, index, m) == /\ \/ m.entries = << >> @@ -861,7 +821,7 @@ AppendEntriesAlreadyDone(i, j, index, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> ConflictAppendEntriesRequest(i, index, m) == /\ m.entries /= << >> @@ -872,7 +832,7 @@ ConflictAppendEntriesRequest(i, index, m) == /\ committableIndices' = [ committableIndices EXCEPT ![i] = @ \ Len(log'[i])..Len(log[i])] \* Potentially also shorten the configurations if the removed txns contained reconfigurations /\ configurations' = [configurations EXCEPT ![i] = ConfigurationsToIndex(i,Len(new_log))] - /\ UNCHANGED <> + /\ UNCHANGED <> NoConflictAppendEntriesRequest(i, j, m) == /\ m.entries /= << >> @@ -916,7 +876,7 @@ NoConflictAppendEntriesRequest(i, j, m) == source |-> i, dest |-> j], m) - /\ UNCHANGED <> + /\ UNCHANGED <> AcceptAppendEntriesRequest(i, j, logOk, m) == \* accept request @@ -957,7 +917,7 @@ HandleAppendEntriesResponse(i, j, m) == \* "If AppendEntries fails because of log inconsistency: decrement nextIndex and retry" /\ UNCHANGED matchIndex /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Any RPC with a newer term causes the recipient to advance its term first. UpdateTerm(i, j, m) == @@ -977,7 +937,7 @@ UpdateTerm(i, j, m) == DropStaleResponse(i, j, m) == /\ m.term < currentTerm[i] /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> DropResponseWhenNotInState(i, j, m) == \/ /\ m.type = AppendEntriesResponse @@ -985,7 +945,7 @@ DropResponseWhenNotInState(i, j, m) == \/ /\ m.type = RequestVoteResponse /\ state[i] \in States \ { Candidate } /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* Drop messages if they are irrelevant to the node DropIgnoredMessage(i,j,m) == @@ -1003,7 +963,7 @@ DropIgnoredMessage(i,j,m) == \/ /\ state[i] = RetiredLeader /\ m.type /= RequestVoteRequest /\ Discard(m) - /\ UNCHANGED <> + /\ UNCHANGED <> \* RetiredLeader leaders send notify commit messages to update all nodes about the commit level UpdateCommitIndex(i,j,m) == @@ -1014,7 +974,7 @@ UpdateCommitIndex(i,j,m) == IN /\ commitIndex' = [commitIndex EXCEPT ![i] = m.commitIndex] /\ configurations' = [configurations EXCEPT ![i] = new_configurations] - /\ UNCHANGED <> \* Receive a message. @@ -1060,13 +1020,6 @@ RcvAppendEntriesResponse(i, j) == \/ DropResponseWhenNotInState(m.dest, m.source, m) \/ DropStaleResponse(m.dest, m.source, m) -RcvUpdateCommitIndex(i, j) == - \E m \in Network!MessagesTo(i, j) : - /\ j = m.source - /\ m.type = NotifyCommitMessage - /\ UpdateCommitIndex(m.dest, m.source, m) - /\ Discard(m) - RcvProposeVoteRequest(i, j) == \E m \in Network!MessagesTo(i, j) : /\ j = m.source @@ -1082,7 +1035,6 @@ Receive(i, j) == \/ RcvRequestVoteResponse(i, j) \/ RcvAppendEntriesRequest(i, j) \/ RcvAppendEntriesResponse(i, j) - \/ RcvUpdateCommitIndex(i, j) \/ RcvProposeVoteRequest(i, j) \* End of message handlers. @@ -1100,7 +1052,6 @@ Next == \/ \E i \in Servers : ClientRequest(i) \/ \E i \in Servers : SignCommittableMessages(i) \/ \E i \in Servers : ChangeConfiguration(i) - \/ \E i, j \in Servers : NotifyCommit(i,j) \/ \E i \in Servers : AdvanceCommitIndex(i) \/ \E i, j \in Servers : AppendEntries(i, j) \/ \E i \in Servers : CheckQuorum(i) @@ -1118,7 +1069,6 @@ Spec == /\ \A i, j \in Servers : WF_vars(RcvRequestVoteResponse(i, j)) /\ \A i, j \in Servers : WF_vars(RcvAppendEntriesRequest(i, j)) /\ \A i, j \in Servers : WF_vars(RcvAppendEntriesResponse(i, j)) - /\ \A i, j \in Servers : WF_vars(RcvUpdateCommitIndex(i, j)) /\ \A i, j \in Servers : WF_vars(RcvProposeVoteRequest(i, j)) \* Node actions /\ \A s, t \in Servers : WF_vars(AppendEntries(s, t)) @@ -1415,7 +1365,6 @@ DebugAlias == removedFromConfiguration |-> removedFromConfiguration, configurations |-> configurations, messages |-> messages, - commitsNotified |-> commitsNotified, currentTerm |-> currentTerm, state |-> state, votedFor |-> votedFor,