TLA+ spec for safekeeper membership change (#9966)

## Problem

We want to define the algorithm for safekeeper membership change.

## Summary of changes

Add spec for it, several models and logs of checking them.

ref https://github.com/neondatabase/neon/issues/8699
This commit is contained in:
Arseny Sher
2025-01-09 15:26:17 +03:00
committed by GitHub
parent 5baa4e7f0a
commit 030ab1c0e8
13 changed files with 784 additions and 83 deletions

View File

@@ -0,0 +1,41 @@
---- MODULE MCProposerAcceptorReconfig ----
EXTENDS TLC, ProposerAcceptorReconfig
\* Augments the spec with model checking constraints.
\* It slightly duplicates MCProposerAcceptorStatic, but we can't EXTENDS it
\* because it EXTENDS ProposerAcceptorStatic in turn. The duplication isn't big
\* anyway.
\* For model checking.
CONSTANTS
max_entries, \* model constraint: max log entries acceptor/proposer can hold
max_term, \* model constraint: max allowed term
max_generation \* mode constraint: max config generation
ASSUME max_entries \in Nat /\ max_term \in Nat /\ max_generation \in Nat
\* Model space constraint.
StateConstraint == /\ \A p \in proposers:
/\ prop_state[p].term <= max_term
/\ Len(prop_state[p].wal) <= max_entries
/\ conf_store.generation <= max_generation
\* Sets of proposers and acceptors and symmetric because we don't take any
\* actions depending on some concrete proposer/acceptor (like IF p = p1 THEN
\* ...)
ProposerAcceptorSymmetry == Permutations(proposers) \union Permutations(acceptors)
\* enforce order of the vars in the error trace with ALIAS
\* Note that ALIAS is supported only since version 1.8.0 which is pre-release
\* as of writing this.
Alias == [
prop_state |-> prop_state,
prop_conf |-> prop_conf,
acc_state |-> acc_state,
acc_conf |-> acc_conf,
committed |-> committed,
conf_store |-> conf_store
]
====

View File

@@ -3,6 +3,9 @@ EXTENDS TLC, ProposerAcceptorStatic
\* Augments the spec with model checking constraints. \* Augments the spec with model checking constraints.
\* Note that MCProposerAcceptorReconfig duplicates it and might need to
\* be updated as well.
\* For model checking. \* For model checking.
CONSTANTS CONSTANTS
max_entries, \* model constraint: max log entries acceptor/proposer can hold max_entries, \* model constraint: max log entries acceptor/proposer can hold

View File

@@ -0,0 +1,350 @@
---- MODULE ProposerAcceptorReconfig ----
(*
Spec for https://github.com/neondatabase/neon/blob/538e2312a617c65d489d391892c70b2e4d7407b5/docs/rfcs/035-safekeeper-dynamic-membership-change.md
Simplifications:
- The ones inherited from ProposerAcceptorStatic.
- We don't model transient state of the configuration change driver process
(storage controller in the implementation). Its actions StartChange and FinishChange
are taken based on the persistent state of safekeepers and conf store. The
justification for that is the following: once new configuration n is
created (e.g with StartChange or FinishChange), any old configuration
change driver working on older conf < n will never be able to commit
it to the conf store because it is protected by CAS. The
propagation of these older confs is still possible though, and
spec allows to do it through acceptors.
Plus the model is already pretty huge.
- Previous point also means that the FinishChange action is
based only on the current state of safekeepers, not from
the past. That's ok because while individual
acceptor <last_log_term, flush_lsn> may go down,
quorum one never does. So the FinishChange
condition which collects max of the quorum may get
only more strict over time.
The invariants expectedly break if any of FinishChange
required conditions are removed.
*)
EXTENDS Integers, Sequences, FiniteSets, TLC
VARIABLES
\* state which is the same in the static spec
prop_state,
acc_state,
committed,
elected_history,
\* reconfiguration only state
prop_conf, \* prop_conf[p] is current configuration of proposer p
acc_conf, \* acc_conf[a] is current configuration of acceptor a
conf_store \* configuration in the configuration store.
CONSTANT
acceptors,
proposers
CONSTANT NULL
\* Import ProposerAcceptorStatic under PAS.
\*
\* Note that all vars and consts are named the same and thus substituted
\* implicitly.
PAS == INSTANCE ProposerAcceptorStatic
\********************************************************************************
\* Helpers
\********************************************************************************
\********************************************************************************
\* Type assertion
\********************************************************************************
\* Is c a valid config?
IsConfig(c) ==
/\ DOMAIN c = {"generation", "members", "newMembers"}
\* Unique id of the configuration.
/\ c.generation \in Nat
/\ c.members \in SUBSET acceptors
\* newMembers is NULL when it is not a joint conf.
/\ \/ c.newMembers = NULL
\/ c.newMembers \in SUBSET acceptors
TypeOk ==
/\ PAS!TypeOk
/\ \A p \in proposers: IsConfig(prop_conf[p])
/\ \A a \in acceptors: IsConfig(acc_conf[a])
/\ IsConfig(conf_store)
\********************************************************************************
\* Initial
\********************************************************************************
Init ==
/\ PAS!Init
/\ \E init_members \in SUBSET acceptors:
LET init_conf == [generation |-> 1, members |-> init_members, newMembers |-> NULL] IN
\* refer to RestartProposer why it is not NULL
/\ prop_conf = [p \in proposers |-> init_conf]
/\ acc_conf = [a \in acceptors |-> init_conf]
/\ conf_store = init_conf
\* We could start with anything, but to reduce state space state with
\* the most reasonable total acceptors - 1 conf size, which e.g.
\* makes basic {a1} -> {a2} change in {a1, a2} acceptors and {a1, a2,
\* a3} -> {a2, a3, a4} in {a1, a2, a3, a4} acceptors models even in
\* the smallest models with single change.
/\ Cardinality(init_members) = Cardinality(acceptors) - 1
\********************************************************************************
\* Actions
\********************************************************************************
\* Proposer p loses all state, restarting. In the static spec we bump restarted
\* proposer term to max of some quorum + 1 which is a minimal term which can win
\* election. With reconfigurations it's harder to calculate such a term, so keep
\* it simple and take random acceptor one + 1.
\*
\* Also make proposer to adopt configuration of another random acceptor. In the
\* impl proposer starts with NULL configuration until handshake with first
\* acceptor. Removing this NULL special case makes the spec a bit simpler.
RestartProposer(p) ==
/\ \E a \in acceptors: PAS!RestartProposerWithTerm(p, acc_state[a].term + 1)
/\ \E a \in acceptors: prop_conf' = [prop_conf EXCEPT ![p] = acc_conf[a]]
/\ UNCHANGED <<acc_conf, conf_store>>
\* Acceptor a immediately votes for proposer p.
Vote(p, a) ==
\* Configuration must be the same.
/\ prop_conf[p].generation = acc_conf[a].generation
\* And a is expected be a member of it. This is likely redundant as long as
\* becoming leader checks membership (though vote also contributes to max
\* <term, lsn> calculation).
/\ \/ a \in prop_conf[p].members
\/ (prop_conf[p].newMembers /= NULL) /\ (a \in prop_conf[p].newMembers)
/\ PAS!Vote(p, a)
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
\* Proposer p gets elected.
BecomeLeader(p) ==
/\ prop_state[p].state = "campaign"
\* Votes must form quorum in both sets (if the newMembers exists).
/\ PAS!FormsQuorum(DOMAIN prop_state[p].votes, prop_conf[p].members)
/\ \/ prop_conf[p].newMembers = NULL
\* TLA+ disjunction evaluation doesn't short-circuit for a good reason:
\* https://groups.google.com/g/tlaplus/c/U6tOJ4dsjVM/m/UdOznPCVBwAJ
\* so repeat the null check.
\/ (prop_conf[p].newMembers /= NULL) /\ (PAS!FormsQuorum(DOMAIN prop_state[p].votes, prop_conf[p].newMembers))
\* DoBecomeLeader will copy WAL of the highest voter to proposer's WAL, so
\* ensure its conf is still the same. In the impl WAL fetching also has to
\* check the configuration.
/\ prop_conf[p].generation = acc_conf[PAS!MaxVoteAcc(p)].generation
/\ \A a \in DOMAIN prop_state[p].votes: prop_conf[p].generation = acc_conf[a].generation
/\ PAS!DoBecomeLeader(p)
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
UpdateTerm(p, a) ==
/\ PAS!UpdateTerm(p, a)
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
TruncateWal(p, a) ==
/\ prop_state[p].state = "leader"
\* Configuration must be the same.
/\ prop_conf[p].generation = acc_conf[a].generation
/\ PAS!TruncateWal(p, a)
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
NewEntry(p) ==
/\ PAS!NewEntry(p)
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
AppendEntry(p, a) ==
/\ prop_state[p].state = "leader"
\* Configuration must be the same.
/\ prop_conf[p].generation = acc_conf[a].generation
\* And a is member of it. Ignoring this likely wouldn't hurt, but not useful
\* either.
/\ \/ a \in prop_conf[p].members
\/ (prop_conf[p].newMembers /= NULL) /\ (a \in prop_conf[p].newMembers)
/\ PAS!AppendEntry(p, a)
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
\* see PAS!CommitEntries for comments.
CommitEntries(p) ==
/\ prop_state[p].state = "leader"
/\ \E q1 \in PAS!AllMinQuorums(prop_conf[p].members):
LET q1_commit_lsn == PAS!QuorumCommitLsn(p, q1) IN
\* Configuration must be the same.
/\ \A a \in q1: prop_conf[p].generation = acc_conf[a].generation
/\ q1_commit_lsn /= NULL
\* We must collect acks from both quorums, if newMembers is present.
/\ IF prop_conf[p].newMembers = NULL THEN
PAS!DoCommitEntries(p, q1_commit_lsn)
ELSE
\E q2 \in PAS!AllMinQuorums(prop_conf[p].newMembers):
LET q2_commit_lsn == PAS!QuorumCommitLsn(p, q2) IN
\* Configuration must be the same.
/\ \A a \in q1: prop_conf[p].generation = acc_conf[a].generation
/\ q2_commit_lsn /= NULL
/\ PAS!DoCommitEntries(p, PAS!Min(q1_commit_lsn, q2_commit_lsn))
/\ UNCHANGED <<prop_conf, acc_conf, conf_store>>
\* Proposer p adopts higher conf c from conf store or from some acceptor.
ProposerSwitchConf(p) ==
/\ \E c \in ({conf_store} \union {acc_conf[a]: a \in acceptors}):
\* p's conf is lower than c.
/\ (c.generation > prop_conf[p].generation)
\* We allow to bump conf without restart only when wp is already elected.
\* If it isn't, the votes it has already collected are from the previous
\* configuration and can't be used.
\*
\* So if proposer is in 'campaign' in the impl we would restart preserving
\* conf and increasing term. In the spec this transition is already covered
\* by more a generic RestartProposer, so we don't specify it here.
/\ prop_state[p].state = "leader"
/\ prop_conf' = [prop_conf EXCEPT ![p] = c]
/\ UNCHANGED <<prop_state, acc_state, committed, elected_history, acc_conf, conf_store>>
\* Do CAS on the conf store, starting change into the new_members conf.
StartChange(new_members) ==
\* Possible only if we don't already have the change in progress.
/\ conf_store.newMembers = NULL
\* Not necessary, but reduces space a bit.
/\ new_members /= conf_store.members
/\ conf_store' = [generation |-> conf_store.generation + 1, members |-> conf_store.members, newMembers |-> new_members]
/\ UNCHANGED <<prop_state, acc_state, committed, elected_history, prop_conf, acc_conf>>
\* Acceptor's last_log_term.
AccLastLogTerm(acc) ==
PAS!LastLogTerm(PAS!AcceptorTermHistory(acc))
\* Do CAS on the conf store, transferring joint conf into the newMembers only.
FinishChange ==
\* have joint conf
/\ conf_store.newMembers /= NULL
\* The conditions for finishing the change are:
/\ \E qo \in PAS!AllMinQuorums(conf_store.members):
\* 1) Old majority must be aware of the joint conf.
\* Note: generally the driver can't know current acceptor
\* generation, it can only know that it once had been the
\* expected one, but it might have advanced since then.
\* But as explained at the top of the file if acceptor gen
\* advanced, FinishChange will never be able to complete
\* due to CAS anyway. We use strict equality here because
\* that's what makes sense conceptually (old driver should
\* abandon its attempt if it observes that conf has advanced).
/\ \A a \in qo: conf_store.generation = acc_conf[a].generation
\* 2) New member set must have log synced, i.e. some its majority needs
\* to have <last_log_term, lsn> at least as high as max of some
\* old majority.
\* 3) Term must be synced, i.e. some majority of the new set must
\* have term >= than max term of some old majority.
\* This ensures that two leaders are never elected with the same
\* term even after config change (which would be bad unless we treat
\* generation as a part of term which we don't).
\* 4) A majority of the new set must be aware of the joint conf.
\* This allows to safely destoy acceptor state if it is not a
\* member of its current conf (which is useful for cleanup after
\* migration as well as for aborts).
/\ LET sync_pos == PAS!MaxTermLsn({[term |-> AccLastLogTerm(a), lsn |-> PAS!FlushLsn(a)]: a \in qo})
sync_term == PAS!Maximum({acc_state[a].term: a \in qo})
IN
\E qn \in PAS!AllMinQuorums(conf_store.newMembers):
\A a \in qn:
/\ PAS!TermLsnGE([term |-> AccLastLogTerm(a), lsn |-> PAS!FlushLsn(a)], sync_pos)
/\ acc_state[a].term >= sync_term
\* The same note as above about strict equality applies here.
/\ conf_store.generation = acc_conf[a].generation
/\ conf_store' = [generation |-> conf_store.generation + 1, members |-> conf_store.newMembers, newMembers |-> NULL]
/\ UNCHANGED <<prop_state, acc_state, committed, elected_history, prop_conf, acc_conf>>
\* Do CAS on the conf store, aborting the change in progress.
AbortChange ==
\* have joint conf
/\ conf_store.newMembers /= NULL
/\ conf_store' = [generation |-> conf_store.generation + 1, members |-> conf_store.members, newMembers |-> NULL]
/\ UNCHANGED <<prop_state, acc_state, committed, elected_history, prop_conf, acc_conf>>
\* Acceptor a switches to higher configuration from the conf store
\* or from some proposer.
AccSwitchConf(a) ==
/\ \E c \in ({conf_store} \union {prop_conf[p]: p \in proposers}):
/\ acc_conf[a].generation < c.generation
/\ acc_conf' = [acc_conf EXCEPT ![a] = c]
/\ UNCHANGED <<prop_state, acc_state, committed, elected_history, prop_conf, conf_store>>
\* Nuke all acceptor state if it is not a member of its current conf. Models
\* cleanup after migration/abort.
AccReset(a) ==
/\ \/ (acc_conf[a].newMembers = NULL) /\ (a \notin acc_conf[a].members)
\/ (acc_conf[a].newMembers /= NULL) /\ (a \notin (acc_conf[a].members \union acc_conf[a].newMembers))
/\ acc_state' = [acc_state EXCEPT ![a] = PAS!InitAcc]
\* Set nextSendLsn to `a` to NULL everywhere. nextSendLsn serves as a mark
\* that elected proposer performed TruncateWal on the acceptor, which isn't
\* true anymore after state reset. In the impl local deletion is expected to
\* terminate all existing connections.
/\ prop_state' = [p \in proposers |-> [prop_state[p] EXCEPT !.nextSendLsn[a] = NULL]]
/\ UNCHANGED <<committed, elected_history, prop_conf, acc_conf, conf_store>>
\*******************************************************************************
\* Final spec
\*******************************************************************************
Next ==
\/ \E p \in proposers: RestartProposer(p)
\/ \E p \in proposers: \E a \in acceptors: Vote(p, a)
\/ \E p \in proposers: BecomeLeader(p)
\/ \E p \in proposers: \E a \in acceptors: UpdateTerm(p, a)
\/ \E p \in proposers: \E a \in acceptors: TruncateWal(p, a)
\/ \E p \in proposers: NewEntry(p)
\/ \E p \in proposers: \E a \in acceptors: AppendEntry(p, a)
\/ \E p \in proposers: CommitEntries(p)
\/ \E new_members \in SUBSET acceptors: StartChange(new_members)
\/ FinishChange
\/ AbortChange
\/ \E p \in proposers: ProposerSwitchConf(p)
\/ \E a \in acceptors: AccSwitchConf(a)
\/ \E a \in acceptors: AccReset(a)
Spec == Init /\ [][Next]_<<prop_state, acc_state, committed, elected_history, prop_conf, acc_conf, conf_store>>
\********************************************************************************
\* Invariants
\********************************************************************************
AllConfs ==
{conf_store} \union {prop_conf[p]: p \in proposers} \union {acc_conf[a]: a \in acceptors}
\* Fairly trivial (given the conf store) invariant that different configurations
\* with the same generation are never issued.
ConfigSafety ==
\A c1, c2 \in AllConfs:
(c1.generation = c2.generation) => (c1 = c2)
ElectionSafety == PAS!ElectionSafety
ElectionSafetyFull == PAS!ElectionSafetyFull
LogIsMonotonic == PAS!LogIsMonotonic
LogSafety == PAS!LogSafety
\********************************************************************************
\* Invariants which don't need to hold, but useful for playing/debugging.
\********************************************************************************
\* Check that we ever switch into non joint conf.
MaxAccConf == ~ \E a \in acceptors:
/\ acc_conf[a].generation = 3
/\ acc_conf[a].newMembers /= NULL
CommittedNotTruncated == PAS!CommittedNotTruncated
MaxTerm == PAS!MaxTerm
MaxStoreConf == conf_store.generation <= 1
MaxAccWalLen == PAS!MaxAccWalLen
MaxCommitLsn == PAS!MaxCommitLsn
====

View File

@@ -18,7 +18,7 @@
\* - old WAL is immediately copied to proposer on its election, without on-demand fetch later. \* - old WAL is immediately copied to proposer on its election, without on-demand fetch later.
\* Some ideas how to break it to play around to get a feeling: \* Some ideas how to break it to play around to get a feeling:
\* - replace Quorums with BadQuorums. \* - replace Quorum with BadQuorum.
\* - remove 'don't commit entries from previous terms separately' rule in \* - remove 'don't commit entries from previous terms separately' rule in
\* CommitEntries and observe figure 8 from the raft paper. \* CommitEntries and observe figure 8 from the raft paper.
\* With p2a3t4l4 32 steps error was found in 1h on 80 cores. \* With p2a3t4l4 32 steps error was found in 1h on 80 cores.
@@ -69,16 +69,26 @@ Upsert(f, k, v, l(_)) ==
\***************** \*****************
NumAccs == Cardinality(acceptors) \* Does set of acceptors `acc_set` form the quorum in the member set `members`?
\* Acceptors not from `members` are excluded (matters only for reconfig).
FormsQuorum(acc_set, members) ==
Cardinality(acc_set \intersect members) >= (Cardinality(members) \div 2 + 1)
\* does acc_set form the quorum? \* Like FormsQuorum, but for minimal quorum.
Quorum(acc_set) == Cardinality(acc_set) >= (NumAccs \div 2 + 1) FormsMinQuorum(acc_set, members) ==
\* all quorums of acceptors Cardinality(acc_set \intersect members) = (Cardinality(members) \div 2 + 1)
Quorums == {subset \in SUBSET acceptors: Quorum(subset)}
\* For substituting Quorums and seeing what happens. \* All sets of acceptors forming minimal quorums in the member set `members`.
BadQuorum(acc_set) == Cardinality(acc_set) >= (NumAccs \div 2) AllQuorums(members) == {subset \in SUBSET members: FormsQuorum(subset, members)}
BadQuorums == {subset \in SUBSET acceptors: BadQuorum(subset)} AllMinQuorums(members) == {subset \in SUBSET acceptors: FormsMinQuorum(subset, members)}
\* For substituting Quorum and seeing what happens.
FormsBadQuorum(acc_set, members) ==
Cardinality(acc_set \intersect members) >= (Cardinality(members) \div 2)
FormsMinBadQuorum(acc_set, members) ==
Cardinality(acc_set \intersect members) = (Cardinality(members) \div 2)
AllBadQuorums(members) == {subset \in SUBSET acceptors: FormsBadQuorum(subset, members)}
AllMinBadQuorums(members) == {subset \in SUBSET acceptors: FormsMinBadQuorum(subset, members)}
\* flushLsn (end of WAL, i.e. index of next entry) of acceptor a. \* flushLsn (end of WAL, i.e. index of next entry) of acceptor a.
FlushLsn(a) == Len(acc_state[a].wal) + 1 FlushLsn(a) == Len(acc_state[a].wal) + 1
@@ -135,10 +145,11 @@ TypeOk ==
/\ IsWal(prop_state[p].wal) /\ IsWal(prop_state[p].wal)
\* Map of acceptor -> next lsn to send. It is set when truncate_wal is \* Map of acceptor -> next lsn to send. It is set when truncate_wal is
\* done so sending entries is allowed only after that. In the impl TCP \* done so sending entries is allowed only after that. In the impl TCP
\* ensures this ordering. \* ensures this ordering. We use NULL instead of missing value to use
\* EXCEPT in AccReset.
/\ \A a \in DOMAIN prop_state[p].nextSendLsn: /\ \A a \in DOMAIN prop_state[p].nextSendLsn:
/\ a \in acceptors /\ a \in acceptors
/\ prop_state[p].nextSendLsn[a] \in Lsns /\ prop_state[p].nextSendLsn[a] \in Lsns \union {NULL}
/\ \A a \in acceptors: /\ \A a \in acceptors:
/\ DOMAIN acc_state[a] = {"term", "termHistory", "wal"} /\ DOMAIN acc_state[a] = {"term", "termHistory", "wal"}
/\ acc_state[a].term \in Terms /\ acc_state[a].term \in Terms
@@ -167,6 +178,19 @@ TypeOk ==
\* Initial \* Initial
\******************************************************************************** \********************************************************************************
InitAcc ==
[
\* There will be no leader in zero term, 1 is the first
\* real.
term |-> 0,
\* Again, leader in term 0 doesn't exist, but we initialize
\* term histories with it to always have common point in
\* them. Lsn is 1 because TLA+ sequences are indexed from 1
\* (we don't want to truncate WAL out of range).
termHistory |-> << [term |-> 0, lsn |-> 1] >>,
wal |-> << >>
]
Init == Init ==
/\ prop_state = [p \in proposers |-> [ /\ prop_state = [p \in proposers |-> [
state |-> "campaign", state |-> "campaign",
@@ -174,19 +198,9 @@ Init ==
votes |-> EmptyF, votes |-> EmptyF,
termHistory |-> << >>, termHistory |-> << >>,
wal |-> << >>, wal |-> << >>,
nextSendLsn |-> EmptyF nextSendLsn |-> [a \in acceptors |-> NULL]
]] ]]
/\ acc_state = [a \in acceptors |-> [ /\ acc_state = [a \in acceptors |-> InitAcc]
\* There will be no leader in zero term, 1 is the first
\* real.
term |-> 0,
\* Again, leader in term 0 doesn't exist, but we initialize
\* term histories with it to always have common point in
\* them. Lsn is 1 because TLA+ sequences are indexed from 1
\* (we don't want to truncate WAL out of range).
termHistory |-> << [term |-> 0, lsn |-> 1] >>,
wal |-> << >>
]]
/\ committed = {} /\ committed = {}
/\ elected_history = EmptyF /\ elected_history = EmptyF
@@ -195,23 +209,35 @@ Init ==
\* Actions \* Actions
\******************************************************************************** \********************************************************************************
\* Proposer loses all state. RestartProposerWithTerm(p, new_term) ==
/\ prop_state' = [prop_state EXCEPT ![p].state = "campaign",
![p].term = new_term,
![p].votes = EmptyF,
![p].termHistory = << >>,
![p].wal = << >>,
![p].nextSendLsn = [a \in acceptors |-> NULL]]
/\ UNCHANGED <<acc_state, committed, elected_history>>
\* Proposer p loses all state, restarting.
\* For simplicity (and to reduct state space), we assume it immediately gets \* For simplicity (and to reduct state space), we assume it immediately gets
\* current state from quorum q of acceptors determining the term he will request \* current state from quorum q of acceptors determining the term he will request
\* to vote for. \* to vote for.
RestartProposer(p, q) == RestartProposer(p) ==
/\ Quorum(q) \E q \in AllQuorums(acceptors):
/\ LET new_term == Maximum({acc_state[a].term : a \in q}) + 1 IN LET new_term == Maximum({acc_state[a].term : a \in q}) + 1 IN
/\ prop_state' = [prop_state EXCEPT ![p].state = "campaign", RestartProposerWithTerm(p, new_term)
![p].term = new_term,
![p].votes = EmptyF,
![p].termHistory = << >>,
![p].wal = << >>,
![p].nextSendLsn = EmptyF]
/\ UNCHANGED <<acc_state, committed, elected_history>>
\* Term history of acceptor a's WAL: the one saved truncated to contain only <= \* Term history of acceptor a's WAL: the one saved truncated to contain only <=
\* local FlushLsn entries. \* local FlushLsn entries. Note that FlushLsn is the end LSN of the last entry
\* (and begin LSN of the next). The mental model for non strict comparison is
\* that once proposer is elected it immediately writes log record with zero
\* length. This allows leader to commit existing log without writing any new
\* entries. For example, assume acceptor has WAL
\* 1.1, 1.2
\* written by prop with term 1; its current <last_log_term, flush_lsn>
\* is <1, 3>. Now prop with term 2 and max vote from this acc is elected.
\* Once TruncateWAL is done, <last_log_term, flush_lsn> becomes <2, 3>
\* without any new records explicitly written.
AcceptorTermHistory(a) == AcceptorTermHistory(a) ==
SelectSeq(acc_state[a].termHistory, LAMBDA th_entry: th_entry.lsn <= FlushLsn(a)) SelectSeq(acc_state[a].termHistory, LAMBDA th_entry: th_entry.lsn <= FlushLsn(a))
@@ -230,35 +256,52 @@ Vote(p, a) ==
\* Get lastLogTerm from term history th. \* Get lastLogTerm from term history th.
LastLogTerm(th) == th[Len(th)].term LastLogTerm(th) == th[Len(th)].term
\* Compares <term, lsn> pairs: returns true if tl1 >= tl2.
TermLsnGE(tl1, tl2) ==
/\ tl1.term >= tl2.term
/\ (tl1.term = tl2.term => tl1.lsn >= tl2.lsn)
\* Choose max <term, lsn> pair in the non empty set of them.
MaxTermLsn(term_lsn_set) ==
CHOOSE max_tl \in term_lsn_set: \A tl \in term_lsn_set: TermLsnGE(max_tl, tl)
\* Find acceptor with the highest <last_log_term, lsn> vote in proposer p's votes.
MaxVoteAcc(p) ==
CHOOSE a \in DOMAIN prop_state[p].votes:
LET a_vote == prop_state[p].votes[a]
a_vote_term_lsn == [term |-> LastLogTerm(a_vote.termHistory), lsn |-> a_vote.flushLsn]
vote_term_lsns == {[term |-> LastLogTerm(v.termHistory), lsn |-> v.flushLsn]: v \in Range(prop_state[p].votes)}
IN
a_vote_term_lsn = MaxTermLsn(vote_term_lsns)
\* Workhorse for BecomeLeader.
\* Assumes the check prop_state[p] votes is quorum has been done *outside*.
DoBecomeLeader(p) ==
LET
\* Find acceptor with the highest <last_log_term, lsn> vote.
max_vote_acc == MaxVoteAcc(p)
max_vote == prop_state[p].votes[max_vote_acc]
prop_th == Append(max_vote.termHistory, [term |-> prop_state[p].term, lsn |-> max_vote.flushLsn])
IN
\* We copy all log preceding proposer's term from the max vote node so
\* make sure it is still on one term with us. This is a model
\* simplification which can be removed, in impl we fetch WAL on demand
\* from safekeeper which has it later. Note though that in case of on
\* demand fetch we must check on donor not only term match, but that
\* truncate_wal had already been done (if it is not max_vote_acc).
/\ acc_state[max_vote_acc].term = prop_state[p].term
/\ prop_state' = [prop_state EXCEPT ![p].state = "leader",
![p].termHistory = prop_th,
![p].wal = acc_state[max_vote_acc].wal
]
/\ elected_history' = Upsert(elected_history, prop_state[p].term, 1, LAMBDA c: c + 1)
/\ UNCHANGED <<acc_state, committed>>
\* Proposer p gets elected. \* Proposer p gets elected.
BecomeLeader(p) == BecomeLeader(p) ==
/\ prop_state[p].state = "campaign" /\ prop_state[p].state = "campaign"
/\ Quorum(DOMAIN prop_state[p].votes) /\ FormsQuorum(DOMAIN prop_state[p].votes, acceptors)
/\ LET /\ DoBecomeLeader(p)
\* Find acceptor with the highest <last_log_term, lsn> vote.
max_vote_acc ==
CHOOSE a \in DOMAIN prop_state[p].votes:
LET v == prop_state[p].votes[a]
IN \A v2 \in Range(prop_state[p].votes):
/\ LastLogTerm(v.termHistory) >= LastLogTerm(v2.termHistory)
/\ (LastLogTerm(v.termHistory) = LastLogTerm(v2.termHistory) => v.flushLsn >= v2.flushLsn)
max_vote == prop_state[p].votes[max_vote_acc]
prop_th == Append(max_vote.termHistory, [term |-> prop_state[p].term, lsn |-> max_vote.flushLsn])
IN
\* We copy all log preceding proposer's term from the max vote node so
\* make sure it is still on one term with us. This is a model
\* simplification which can be removed, in impl we fetch WAL on demand
\* from safekeeper which has it later. Note though that in case of on
\* demand fetch we must check on donor not only term match, but that
\* truncate_wal had already been done (if it is not max_vote_acc).
/\ acc_state[max_vote_acc].term = prop_state[p].term
/\ prop_state' = [prop_state EXCEPT ![p].state = "leader",
![p].termHistory = prop_th,
![p].wal = acc_state[max_vote_acc].wal
]
/\ elected_history' = Upsert(elected_history, prop_state[p].term, 1, LAMBDA c: c + 1)
/\ UNCHANGED <<acc_state, committed>>
\* Acceptor a learns about elected proposer p's term. In impl it matches to \* Acceptor a learns about elected proposer p's term. In impl it matches to
\* VoteRequest/VoteResponse exchange when leader is already elected and is not \* VoteRequest/VoteResponse exchange when leader is already elected and is not
@@ -287,10 +330,11 @@ FindHighestCommonPoint(prop_th, acc_th, acc_flush_lsn) ==
IN IN
[term |-> last_common_term, lsn |-> Min(acc_common_term_end, prop_common_term_end)] [term |-> last_common_term, lsn |-> Min(acc_common_term_end, prop_common_term_end)]
\* Elected proposer p immediately truncates WAL (and term history) of acceptor a \* Elected proposer p immediately truncates WAL (and sets term history) of
\* before starting streaming. Establishes nextSendLsn for a. \* acceptor a before starting streaming. Establishes nextSendLsn for a.
\* \*
\* In impl this happens at each reconnection, here we also allow to do it multiple times. \* In impl this happens at each reconnection, here we also allow to do it
\* multiple times.
TruncateWal(p, a) == TruncateWal(p, a) ==
/\ prop_state[p].state = "leader" /\ prop_state[p].state = "leader"
/\ acc_state[a].term = prop_state[p].term /\ acc_state[a].term = prop_state[p].term
@@ -321,8 +365,8 @@ NewEntry(p) ==
AppendEntry(p, a) == AppendEntry(p, a) ==
/\ prop_state[p].state = "leader" /\ prop_state[p].state = "leader"
/\ acc_state[a].term = prop_state[p].term /\ acc_state[a].term = prop_state[p].term
/\ a \in DOMAIN prop_state[p].nextSendLsn \* did TruncateWal /\ prop_state[p].nextSendLsn[a] /= NULL \* did TruncateWal
/\ prop_state[p].nextSendLsn[a] <= Len(prop_state[p].wal) \* have smth to send /\ prop_state[p].nextSendLsn[a] <= Len(prop_state[p].wal) \* have smth to send
/\ LET /\ LET
send_lsn == prop_state[p].nextSendLsn[a] send_lsn == prop_state[p].nextSendLsn[a]
entry == prop_state[p].wal[send_lsn] entry == prop_state[p].wal[send_lsn]
@@ -337,41 +381,65 @@ AppendEntry(p, a) ==
PropStartLsn(p) == PropStartLsn(p) ==
IF prop_state[p].state = "leader" THEN prop_state[p].termHistory[Len(prop_state[p].termHistory)].lsn ELSE NULL IF prop_state[p].state = "leader" THEN prop_state[p].termHistory[Len(prop_state[p].termHistory)].lsn ELSE NULL
\* Proposer p commits all entries it can using quorum q. Note that unlike \* LSN which can be committed by proposer p using min quorum q (check that q
\* will62794/logless-reconfig this allows to commit entries from previous terms \* forms quorum must have been done outside). NULL if there is none.
\* (when conditions for that are met). QuorumCommitLsn(p, q) ==
CommitEntries(p, q) == IF
/\ prop_state[p].state = "leader" /\ prop_state[p].state = "leader"
/\ \A a \in q: /\ \A a \in q:
\* Without explicit responses to appends this ensures that append
\* up to FlushLsn has been accepted.
/\ acc_state[a].term = prop_state[p].term /\ acc_state[a].term = prop_state[p].term
\* nextSendLsn existence means TruncateWal has happened, it ensures \* nextSendLsn existence means TruncateWal has happened, it ensures
\* acceptor's WAL (and FlushLsn) are from proper proposer's history. \* acceptor's WAL (and FlushLsn) are from proper proposer's history.
\* Alternatively we could compare LastLogTerm here, but that's closer to \* Alternatively we could compare LastLogTerm here, but that's closer to
\* what we do in the impl (we check flushLsn in AppendResponse, but \* what we do in the impl (we check flushLsn in AppendResponse, but
\* AppendRequest is processed only if HandleElected handling was good). \* AppendRequest is processed only if HandleElected handling was good).
/\ a \in DOMAIN prop_state[p].nextSendLsn /\ prop_state[p].nextSendLsn[a] /= NULL
\* Now find the LSN present on all the quorum. THEN
/\ LET quorum_lsn == Minimum({FlushLsn(a): a \in q}) IN \* Now find the LSN present on all the quorum.
\* This is the basic Raft rule of not committing entries from previous LET quorum_lsn == Minimum({FlushLsn(a): a \in q}) IN
\* terms except along with current term entry (commit them only when \* This is the basic Raft rule of not committing entries from previous
\* quorum recovers, i.e. last_log_term on it reaches leader's term). \* terms except along with current term entry (commit them only when
/\ quorum_lsn >= PropStartLsn(p) \* quorum recovers, i.e. last_log_term on it reaches leader's term).
/\ committed' = committed \cup {[term |-> prop_state[p].wal[lsn], lsn |-> lsn]: lsn \in 1..(quorum_lsn - 1)} IF quorum_lsn >= PropStartLsn(p) THEN
/\ UNCHANGED <<prop_state, acc_state, elected_history>> quorum_lsn
ELSE
NULL
ELSE
NULL
\* Commit all entries on proposer p with record lsn < commit_lsn.
DoCommitEntries(p, commit_lsn) ==
/\ committed' = committed \cup {[term |-> prop_state[p].wal[lsn], lsn |-> lsn]: lsn \in 1..(commit_lsn - 1)}
/\ UNCHANGED <<prop_state, acc_state, elected_history>>
\* Proposer p commits all entries it can using some quorum. Note that unlike
\* will62794/logless-reconfig this allows to commit entries from previous terms
\* (when conditions for that are met).
CommitEntries(p) ==
/\ prop_state[p].state = "leader"
\* Using min quorums here is better because 1) QuorumCommitLsn for
\* simplicity checks min across all accs in q. 2) it probably makes
\* evaluation faster.
/\ \E q \in AllMinQuorums(acceptors):
LET commit_lsn == QuorumCommitLsn(p, q) IN
/\ commit_lsn /= NULL
/\ DoCommitEntries(p, commit_lsn)
\******************************************************************************* \*******************************************************************************
\* Final spec \* Final spec
\******************************************************************************* \*******************************************************************************
Next == Next ==
\/ \E q \in Quorums: \E p \in proposers: RestartProposer(p, q) \/ \E p \in proposers: RestartProposer(p)
\/ \E p \in proposers: \E a \in acceptors: Vote(p, a) \/ \E p \in proposers: \E a \in acceptors: Vote(p, a)
\/ \E p \in proposers: BecomeLeader(p) \/ \E p \in proposers: BecomeLeader(p)
\/ \E p \in proposers: \E a \in acceptors: UpdateTerm(p, a) \/ \E p \in proposers: \E a \in acceptors: UpdateTerm(p, a)
\/ \E p \in proposers: \E a \in acceptors: TruncateWal(p, a) \/ \E p \in proposers: \E a \in acceptors: TruncateWal(p, a)
\/ \E p \in proposers: NewEntry(p) \/ \E p \in proposers: NewEntry(p)
\/ \E p \in proposers: \E a \in acceptors: AppendEntry(p, a) \/ \E p \in proposers: \E a \in acceptors: AppendEntry(p, a)
\/ \E q \in Quorums: \E p \in proposers: CommitEntries(p, q) \/ \E p \in proposers: CommitEntries(p)
Spec == Init /\ [][Next]_<<prop_state, acc_state, committed, elected_history>> Spec == Init /\ [][Next]_<<prop_state, acc_state, committed, elected_history>>

View File

@@ -2,6 +2,7 @@
# Usage: ./modelcheck.sh <config_file> <spec_file>, e.g. # Usage: ./modelcheck.sh <config_file> <spec_file>, e.g.
# ./modelcheck.sh models/MCProposerAcceptorStatic_p2_a3_t3_l3.cfg MCProposerAcceptorStatic.tla # ./modelcheck.sh models/MCProposerAcceptorStatic_p2_a3_t3_l3.cfg MCProposerAcceptorStatic.tla
# ./modelcheck.sh models/MCProposerAcceptorReconfig_p2_a3_t3_l3_c3.cfg MCProposerAcceptorReconfig.tla
CONFIG=$1 CONFIG=$1
SPEC=$2 SPEC=$2
@@ -12,6 +13,7 @@ mkdir -p "tlc-results"
CONFIG_FILE=$(basename -- "$CONFIG") CONFIG_FILE=$(basename -- "$CONFIG")
outfilename="$SPEC-${CONFIG_FILE}-$(date --utc +%Y-%m-%d--%H-%M-%S)".log outfilename="$SPEC-${CONFIG_FILE}-$(date --utc +%Y-%m-%d--%H-%M-%S)".log
outfile="tlc-results/$outfilename" outfile="tlc-results/$outfilename"
echo "saving results to $outfile"
touch $outfile touch $outfile
# Save some info about the run. # Save some info about the run.
@@ -45,5 +47,6 @@ echo "" >> $outfile
# https://docs.tlapl.us/codebase:architecture#fingerprint_sets_fpsets # https://docs.tlapl.us/codebase:architecture#fingerprint_sets_fpsets
# #
# Add -simulate to run in infinite simulation mode. # Add -simulate to run in infinite simulation mode.
# -coverage 1 is useful for profiling (check how many times actions are taken).
java -Xmx$MEM -XX:MaxDirectMemorySize=$MEM -XX:+UseParallelGC -Dtlc2.tool.fp.FPSet.impl=tlc2.tool.fp.OffHeapDiskFPSet \ java -Xmx$MEM -XX:MaxDirectMemorySize=$MEM -XX:+UseParallelGC -Dtlc2.tool.fp.FPSet.impl=tlc2.tool.fp.OffHeapDiskFPSet \
-cp "${TOOLSPATH}" tlc2.TLC $SPEC -config $CONFIG -workers auto -gzip | tee -a $outfile -cp "${TOOLSPATH}" tlc2.TLC $SPEC -config $CONFIG -workers auto -gzip | tee -a $outfile

View File

@@ -0,0 +1,21 @@
CONSTANTS
NULL = NULL
proposers = {p1, p2}
acceptors = {a1, a2}
max_term = 2
max_entries = 2
max_generation = 3
SPECIFICATION Spec
CONSTRAINT StateConstraint
INVARIANT
TypeOk
ConfigSafety
ElectionSafetyFull
LogIsMonotonic
LogSafety
\* As its comment explains generally it is not expected to hold, but
\* in such small model it is true.
CommittedNotTruncated
SYMMETRY ProposerAcceptorSymmetry
CHECK_DEADLOCK FALSE
ALIAS Alias

View File

@@ -0,0 +1,19 @@
CONSTANTS
NULL = NULL
proposers = {p1, p2}
acceptors = {a1, a2}
max_term = 2
max_entries = 2
max_generation = 5
SPECIFICATION Spec
CONSTRAINT StateConstraint
INVARIANT
TypeOk
ConfigSafety
ElectionSafetyFull
LogIsMonotonic
LogSafety
CommittedNotTruncated
SYMMETRY ProposerAcceptorSymmetry
CHECK_DEADLOCK FALSE
ALIAS Alias

View File

@@ -0,0 +1,20 @@
CONSTANTS
NULL = NULL
proposers = {p1, p2}
acceptors = {a1, a2, a3}
max_term = 2
max_entries = 2
max_generation = 3
SPECIFICATION Spec
CONSTRAINT StateConstraint
INVARIANT
TypeOk
ConfigSafety
ElectionSafetyFull
LogIsMonotonic
LogSafety
CommittedNotTruncated
SYMMETRY ProposerAcceptorSymmetry
CHECK_DEADLOCK FALSE
ALIAS Alias

View File

@@ -0,0 +1,19 @@
CONSTANTS
NULL = NULL
proposers = {p1, p2}
acceptors = {a1, a2, a3, a4}
max_term = 2
max_entries = 2
max_generation = 3
SPECIFICATION Spec
CONSTRAINT StateConstraint
INVARIANT
TypeOk
ElectionSafetyFull
LogIsMonotonic
LogSafety
CommittedNotTruncated
SYMMETRY ProposerAcceptorSymmetry
CHECK_DEADLOCK FALSE
ALIAS Alias

View File

@@ -0,0 +1,25 @@
# Print all lines, but thin out lines starting with Progress:
# leave only first and last 5 ones in the beginning, and only 1 of 1440
# of others (once a day).
# Also remove checkpointing logs.
{
lines[NR] = $0
}
$0 ~ /^Progress/ {
++pcount
}
END {
progress_idx = 0
for (i = 1; i <= NR; i++) {
if (lines[i] ~ /^Progress/) {
if (progress_idx < 5 || progress_idx >= pcount - 5 || progress_idx % 1440 == 0) {
print lines[i]
}
progress_idx++
}
else if (lines[i] ~ /^Checkpointing/) {}
else {
print lines[i]
}
}
}

View File

@@ -0,0 +1,3 @@
#!/bin/bash
awk -f remove_interm_progress.awk $1 > $1.thin

View File

@@ -0,0 +1,65 @@
git revision: 9e386917a
Platform: Linux neon-dev-arm64-1 6.8.0-49-generic #49-Ubuntu SMP PREEMPT_DYNAMIC Sun Nov 3 21:21:58 UTC 2024 aarch64 aarch64 aarch64 GNU/Linux
CPU Info Linux: Neoverse-N1
CPU Cores Linux: 80
CPU Info Mac:
CPU Cores Mac:
Spec: MCProposerAcceptorReconfig.tla
Config: models/MCProposerAcceptorReconfig_p2_a2_t2_l2_c3.cfg
----
CONSTANTS
NULL = NULL
proposers = {p1, p2}
acceptors = {a1, a2}
max_term = 2
max_entries = 2
max_generation = 3
SPECIFICATION Spec
CONSTRAINT StateConstraint
INVARIANT
TypeOk
ElectionSafetyFull
LogIsMonotonic
LogSafety
\* CommittedNotTruncated
SYMMETRY ProposerAcceptorSymmetry
CHECK_DEADLOCK FALSE
ALIAS Alias
----
TLC2 Version 2.20 of Day Month 20?? (rev: f68cb71)
Running breadth-first search Model-Checking with fp 99 and seed -9189733667206762985 with 35 workers on 80 cores with 27307MB heap and 30720MB offheap memory [pid: 391272] (Linux 6.8.0-49-generic aarch64, Ubuntu 21.0.5 x86_64, OffHeapDiskFPSet, DiskStateQueue).
Parsing file /home/arseny/neon2/safekeeper/spec/MCProposerAcceptorReconfig.tla
Parsing file /tmp/tlc-3211535543066978921/TLC.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/TLC.tla)
Parsing file /home/arseny/neon2/safekeeper/spec/ProposerAcceptorReconfig.tla
Parsing file /tmp/tlc-3211535543066978921/_TLCTrace.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/_TLCTrace.tla)
Parsing file /tmp/tlc-3211535543066978921/Integers.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/Integers.tla)
Parsing file /tmp/tlc-3211535543066978921/Sequences.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/Sequences.tla)
Parsing file /tmp/tlc-3211535543066978921/FiniteSets.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/FiniteSets.tla)
Parsing file /tmp/tlc-3211535543066978921/Naturals.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/Naturals.tla)
Parsing file /home/arseny/neon2/safekeeper/spec/ProposerAcceptorStatic.tla
Parsing file /tmp/tlc-3211535543066978921/TLCExt.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/TLCExt.tla)
Semantic processing of module Naturals
Semantic processing of module Sequences
Semantic processing of module FiniteSets
Semantic processing of module TLC
Semantic processing of module Integers
Semantic processing of module ProposerAcceptorStatic
Semantic processing of module ProposerAcceptorReconfig
Semantic processing of module TLCExt
Semantic processing of module _TLCTrace
Semantic processing of module MCProposerAcceptorReconfig
Starting... (2024-12-11 04:24:13)
Computing initial states...
Finished computing initial states: 2 states generated, with 1 of them distinct at 2024-12-11 04:24:15.
Progress(16) at 2024-12-11 04:24:18: 1,427,589 states generated (1,427,589 s/min), 142,472 distinct states found (142,472 ds/min), 47,162 states left on queue.
Model checking completed. No error has been found.
Estimates of the probability that TLC did not check all reachable states
because two distinct states had the same fingerprint:
calculated (optimistic): val = 1.0E-6
based on the actual fingerprints: val = 4.2E-8
17746857 states generated, 1121659 distinct states found, 0 states left on queue.
The depth of the complete state graph search is 37.
The average outdegree of the complete state graph is 1 (minimum is 0, the maximum 9 and the 95th percentile is 3).
Finished in 33s at (2024-12-11 04:24:46)

View File

@@ -0,0 +1,64 @@
git revision: 9e386917a
Platform: Linux neon-dev-arm64-1 6.8.0-49-generic #49-Ubuntu SMP PREEMPT_DYNAMIC Sun Nov 3 21:21:58 UTC 2024 aarch64 aarch64 aarch64 GNU/Linux
CPU Info Linux: Neoverse-N1
CPU Cores Linux: 80
CPU Info Mac:
CPU Cores Mac:
Spec: MCProposerAcceptorReconfig.tla
Config: models/MCProposerAcceptorReconfig_p2_a2_t2_l2_c5.cfg
----
CONSTANTS
NULL = NULL
proposers = {p1, p2}
acceptors = {a1, a2}
max_term = 2
max_entries = 2
max_generation = 5
SPECIFICATION Spec
CONSTRAINT StateConstraint
INVARIANT
TypeOk
ElectionSafetyFull
LogIsMonotonic
LogSafety
\* CommittedNotTruncated
SYMMETRY ProposerAcceptorSymmetry
CHECK_DEADLOCK FALSE
ALIAS Alias
----
TLC2 Version 2.20 of Day Month 20?? (rev: f68cb71)
Running breadth-first search Model-Checking with fp 114 and seed -8099467489737745861 with 35 workers on 80 cores with 27307MB heap and 30720MB offheap memory [pid: 392020] (Linux 6.8.0-49-generic aarch64, Ubuntu 21.0.5 x86_64, OffHeapDiskFPSet, DiskStateQueue).
Parsing file /home/arseny/neon2/safekeeper/spec/MCProposerAcceptorReconfig.tla
Parsing file /tmp/tlc-11757875725969857497/TLC.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/TLC.tla)
Parsing file /home/arseny/neon2/safekeeper/spec/ProposerAcceptorReconfig.tla
Parsing file /tmp/tlc-11757875725969857497/_TLCTrace.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/_TLCTrace.tla)
Parsing file /tmp/tlc-11757875725969857497/Integers.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/Integers.tla)
Parsing file /tmp/tlc-11757875725969857497/Sequences.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/Sequences.tla)
Parsing file /tmp/tlc-11757875725969857497/FiniteSets.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/FiniteSets.tla)
Parsing file /tmp/tlc-11757875725969857497/Naturals.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/Naturals.tla)
Parsing file /home/arseny/neon2/safekeeper/spec/ProposerAcceptorStatic.tla
Parsing file /tmp/tlc-11757875725969857497/TLCExt.tla (jar:file:/home/arseny/tla2tools.jar!/tla2sany/StandardModules/TLCExt.tla)
Semantic processing of module Naturals
Semantic processing of module Sequences
Semantic processing of module FiniteSets
Semantic processing of module TLC
Semantic processing of module Integers
Semantic processing of module ProposerAcceptorStatic
Semantic processing of module ProposerAcceptorReconfig
Semantic processing of module TLCExt
Semantic processing of module _TLCTrace
Semantic processing of module MCProposerAcceptorReconfig
Starting... (2024-12-11 04:26:12)
Computing initial states...
Finished computing initial states: 2 states generated, with 1 of them distinct at 2024-12-11 04:26:14.
Progress(14) at 2024-12-11 04:26:17: 1,519,385 states generated (1,519,385 s/min), 231,263 distinct states found (231,263 ds/min), 121,410 states left on queue.
Progress(20) at 2024-12-11 04:27:17: 42,757,204 states generated (41,237,819 s/min), 4,198,386 distinct states found (3,967,123 ds/min), 1,308,109 states left on queue.
Progress(22) at 2024-12-11 04:28:17: 83,613,929 states generated (40,856,725 s/min), 7,499,873 distinct states found (3,301,487 ds/min), 1,929,464 states left on queue.
Progress(23) at 2024-12-11 04:29:17: 124,086,758 states generated (40,472,829 s/min), 10,569,712 distinct states found (3,069,839 ds/min), 2,386,988 states left on queue.
Progress(24) at 2024-12-11 04:30:17: 163,412,538 states generated (39,325,780 s/min), 13,314,303 distinct states found (2,744,591 ds/min), 2,610,637 states left on queue.
Progress(25) at 2024-12-11 04:31:17: 202,643,708 states generated (39,231,170 s/min), 15,960,583 distinct states found (2,646,280 ds/min), 2,759,681 states left on queue.
Progress(26) at 2024-12-11 04:32:17: 240,681,633 states generated (38,037,925 s/min), 18,443,440 distinct states found (2,482,857 ds/min), 2,852,177 states left on queue.
Progress(27) at 2024-12-11 04:33:17: 278,559,134 states generated (37,877,501 s/min), 20,878,067 distinct states found (2,434,627 ds/min), 2,904,400 states left on queue.
Progress(28) at 2024-12-11 04:34:17: 316,699,911 states generated (38,140,777 s/min), 23,212,229 distinct states found (2,334,162 ds/min), 2,864,969 states left on queue.