mirror of
https://github.com/neondatabase/neon.git
synced 2025-12-22 21:59:59 +00:00
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:
41
safekeeper/spec/MCProposerAcceptorReconfig.tla
Normal file
41
safekeeper/spec/MCProposerAcceptorReconfig.tla
Normal 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
|
||||
]
|
||||
|
||||
====
|
||||
@@ -3,6 +3,9 @@ EXTENDS TLC, ProposerAcceptorStatic
|
||||
|
||||
\* Augments the spec with model checking constraints.
|
||||
|
||||
\* Note that MCProposerAcceptorReconfig duplicates it and might need to
|
||||
\* be updated as well.
|
||||
|
||||
\* For model checking.
|
||||
CONSTANTS
|
||||
max_entries, \* model constraint: max log entries acceptor/proposer can hold
|
||||
|
||||
350
safekeeper/spec/ProposerAcceptorReconfig.tla
Normal file
350
safekeeper/spec/ProposerAcceptorReconfig.tla
Normal 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
|
||||
|
||||
====
|
||||
@@ -18,7 +18,7 @@
|
||||
\* - 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:
|
||||
\* - replace Quorums with BadQuorums.
|
||||
\* - replace Quorum with BadQuorum.
|
||||
\* - remove 'don't commit entries from previous terms separately' rule in
|
||||
\* CommitEntries and observe figure 8 from the raft paper.
|
||||
\* 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?
|
||||
Quorum(acc_set) == Cardinality(acc_set) >= (NumAccs \div 2 + 1)
|
||||
\* all quorums of acceptors
|
||||
Quorums == {subset \in SUBSET acceptors: Quorum(subset)}
|
||||
\* Like FormsQuorum, but for minimal quorum.
|
||||
FormsMinQuorum(acc_set, members) ==
|
||||
Cardinality(acc_set \intersect members) = (Cardinality(members) \div 2 + 1)
|
||||
|
||||
\* For substituting Quorums and seeing what happens.
|
||||
BadQuorum(acc_set) == Cardinality(acc_set) >= (NumAccs \div 2)
|
||||
BadQuorums == {subset \in SUBSET acceptors: BadQuorum(subset)}
|
||||
\* All sets of acceptors forming minimal quorums in the member set `members`.
|
||||
AllQuorums(members) == {subset \in SUBSET members: FormsQuorum(subset, members)}
|
||||
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(a) == Len(acc_state[a].wal) + 1
|
||||
@@ -135,10 +145,11 @@ TypeOk ==
|
||||
/\ IsWal(prop_state[p].wal)
|
||||
\* 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
|
||||
\* 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 \in acceptors
|
||||
/\ prop_state[p].nextSendLsn[a] \in Lsns
|
||||
/\ prop_state[p].nextSendLsn[a] \in Lsns \union {NULL}
|
||||
/\ \A a \in acceptors:
|
||||
/\ DOMAIN acc_state[a] = {"term", "termHistory", "wal"}
|
||||
/\ acc_state[a].term \in Terms
|
||||
@@ -167,16 +178,8 @@ TypeOk ==
|
||||
\* Initial
|
||||
\********************************************************************************
|
||||
|
||||
Init ==
|
||||
/\ prop_state = [p \in proposers |-> [
|
||||
state |-> "campaign",
|
||||
term |-> 1,
|
||||
votes |-> EmptyF,
|
||||
termHistory |-> << >>,
|
||||
wal |-> << >>,
|
||||
nextSendLsn |-> EmptyF
|
||||
]]
|
||||
/\ acc_state = [a \in acceptors |-> [
|
||||
InitAcc ==
|
||||
[
|
||||
\* There will be no leader in zero term, 1 is the first
|
||||
\* real.
|
||||
term |-> 0,
|
||||
@@ -186,7 +189,18 @@ Init ==
|
||||
\* (we don't want to truncate WAL out of range).
|
||||
termHistory |-> << [term |-> 0, lsn |-> 1] >>,
|
||||
wal |-> << >>
|
||||
]
|
||||
|
||||
Init ==
|
||||
/\ prop_state = [p \in proposers |-> [
|
||||
state |-> "campaign",
|
||||
term |-> 1,
|
||||
votes |-> EmptyF,
|
||||
termHistory |-> << >>,
|
||||
wal |-> << >>,
|
||||
nextSendLsn |-> [a \in acceptors |-> NULL]
|
||||
]]
|
||||
/\ acc_state = [a \in acceptors |-> InitAcc]
|
||||
/\ committed = {}
|
||||
/\ elected_history = EmptyF
|
||||
|
||||
@@ -195,23 +209,35 @@ Init ==
|
||||
\* Actions
|
||||
\********************************************************************************
|
||||
|
||||
\* Proposer loses all state.
|
||||
\* 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
|
||||
\* to vote for.
|
||||
RestartProposer(p, q) ==
|
||||
/\ Quorum(q)
|
||||
/\ LET new_term == Maximum({acc_state[a].term : a \in q}) + 1 IN
|
||||
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 = EmptyF]
|
||||
![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
|
||||
\* current state from quorum q of acceptors determining the term he will request
|
||||
\* to vote for.
|
||||
RestartProposer(p) ==
|
||||
\E q \in AllQuorums(acceptors):
|
||||
LET new_term == Maximum({acc_state[a].term : a \in q}) + 1 IN
|
||||
RestartProposerWithTerm(p, new_term)
|
||||
|
||||
\* 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) ==
|
||||
SelectSeq(acc_state[a].termHistory, LAMBDA th_entry: th_entry.lsn <= FlushLsn(a))
|
||||
|
||||
@@ -230,18 +256,30 @@ Vote(p, a) ==
|
||||
\* Get lastLogTerm from term history th.
|
||||
LastLogTerm(th) == th[Len(th)].term
|
||||
|
||||
\* Proposer p gets elected.
|
||||
BecomeLeader(p) ==
|
||||
/\ prop_state[p].state = "campaign"
|
||||
/\ Quorum(DOMAIN prop_state[p].votes)
|
||||
/\ LET
|
||||
\* Find acceptor with the highest <last_log_term, lsn> vote.
|
||||
max_vote_acc ==
|
||||
\* 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 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)
|
||||
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
|
||||
@@ -259,6 +297,11 @@ BecomeLeader(p) ==
|
||||
/\ elected_history' = Upsert(elected_history, prop_state[p].term, 1, LAMBDA c: c + 1)
|
||||
/\ UNCHANGED <<acc_state, committed>>
|
||||
|
||||
\* Proposer p gets elected.
|
||||
BecomeLeader(p) ==
|
||||
/\ prop_state[p].state = "campaign"
|
||||
/\ FormsQuorum(DOMAIN prop_state[p].votes, acceptors)
|
||||
/\ DoBecomeLeader(p)
|
||||
|
||||
\* 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
|
||||
@@ -287,10 +330,11 @@ FindHighestCommonPoint(prop_th, acc_th, acc_flush_lsn) ==
|
||||
IN
|
||||
[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
|
||||
\* before starting streaming. Establishes nextSendLsn for a.
|
||||
\* Elected proposer p immediately truncates WAL (and sets term history) of
|
||||
\* 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) ==
|
||||
/\ prop_state[p].state = "leader"
|
||||
/\ acc_state[a].term = prop_state[p].term
|
||||
@@ -321,7 +365,7 @@ NewEntry(p) ==
|
||||
AppendEntry(p, a) ==
|
||||
/\ prop_state[p].state = "leader"
|
||||
/\ 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
|
||||
/\ LET
|
||||
send_lsn == prop_state[p].nextSendLsn[a]
|
||||
@@ -337,41 +381,65 @@ AppendEntry(p, a) ==
|
||||
PropStartLsn(p) ==
|
||||
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
|
||||
\* will62794/logless-reconfig this allows to commit entries from previous terms
|
||||
\* (when conditions for that are met).
|
||||
CommitEntries(p, q) ==
|
||||
\* LSN which can be committed by proposer p using min quorum q (check that q
|
||||
\* forms quorum must have been done outside). NULL if there is none.
|
||||
QuorumCommitLsn(p, q) ==
|
||||
IF
|
||||
/\ prop_state[p].state = "leader"
|
||||
/\ \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
|
||||
\* nextSendLsn existence means TruncateWal has happened, it ensures
|
||||
\* acceptor's WAL (and FlushLsn) are from proper proposer's history.
|
||||
\* Alternatively we could compare LastLogTerm here, but that's closer to
|
||||
\* what we do in the impl (we check flushLsn in AppendResponse, but
|
||||
\* AppendRequest is processed only if HandleElected handling was good).
|
||||
/\ a \in DOMAIN prop_state[p].nextSendLsn
|
||||
/\ prop_state[p].nextSendLsn[a] /= NULL
|
||||
THEN
|
||||
\* Now find the LSN present on all the quorum.
|
||||
/\ LET quorum_lsn == Minimum({FlushLsn(a): a \in q}) IN
|
||||
LET quorum_lsn == Minimum({FlushLsn(a): a \in q}) IN
|
||||
\* This is the basic Raft rule of not committing entries from previous
|
||||
\* terms except along with current term entry (commit them only when
|
||||
\* quorum recovers, i.e. last_log_term on it reaches leader's term).
|
||||
/\ quorum_lsn >= PropStartLsn(p)
|
||||
/\ committed' = committed \cup {[term |-> prop_state[p].wal[lsn], lsn |-> lsn]: lsn \in 1..(quorum_lsn - 1)}
|
||||
IF quorum_lsn >= PropStartLsn(p) THEN
|
||||
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
|
||||
\*******************************************************************************
|
||||
|
||||
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: 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 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>>
|
||||
|
||||
|
||||
@@ -2,6 +2,7 @@
|
||||
|
||||
# Usage: ./modelcheck.sh <config_file> <spec_file>, e.g.
|
||||
# ./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
|
||||
SPEC=$2
|
||||
|
||||
@@ -12,6 +13,7 @@ mkdir -p "tlc-results"
|
||||
CONFIG_FILE=$(basename -- "$CONFIG")
|
||||
outfilename="$SPEC-${CONFIG_FILE}-$(date --utc +%Y-%m-%d--%H-%M-%S)".log
|
||||
outfile="tlc-results/$outfilename"
|
||||
echo "saving results to $outfile"
|
||||
touch $outfile
|
||||
|
||||
# Save some info about the run.
|
||||
@@ -45,5 +47,6 @@ echo "" >> $outfile
|
||||
# https://docs.tlapl.us/codebase:architecture#fingerprint_sets_fpsets
|
||||
#
|
||||
# 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 \
|
||||
-cp "${TOOLSPATH}" tlc2.TLC $SPEC -config $CONFIG -workers auto -gzip | tee -a $outfile
|
||||
|
||||
@@ -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
|
||||
@@ -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
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
25
safekeeper/spec/remove_interm_progress.awk
Normal file
25
safekeeper/spec/remove_interm_progress.awk
Normal 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]
|
||||
}
|
||||
}
|
||||
}
|
||||
3
safekeeper/spec/remove_interm_progress.sh
Executable file
3
safekeeper/spec/remove_interm_progress.sh
Executable file
@@ -0,0 +1,3 @@
|
||||
#!/bin/bash
|
||||
|
||||
awk -f remove_interm_progress.awk $1 > $1.thin
|
||||
@@ -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)
|
||||
@@ -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.
|
||||
Reference in New Issue
Block a user