Documentation ¶
Index ¶
- Variables
- func AcceptMessage(iface distsys.ArchetypeInterface) tla.Value
- func AppendEntriesRequest(iface distsys.ArchetypeInterface) tla.Value
- func AppendEntriesResponse(iface distsys.ArchetypeInterface) tla.Value
- func Candidate(iface distsys.ArchetypeInterface) tla.Value
- func FindMaxAgreeIndex(iface distsys.ArchetypeInterface, logLocal tla.Value, i tla.Value, ...) tla.Value
- func FindMaxAgreeIndexRec(iface distsys.ArchetypeInterface, logLocal0 tla.Value, i0 tla.Value, ...) tla.Value
- func Follower(iface distsys.ArchetypeInterface) tla.Value
- func IsQuorum(iface distsys.ArchetypeInterface, s3 tla.Value) tla.Value
- func Key1(iface distsys.ArchetypeInterface) tla.Value
- func Key2(iface distsys.ArchetypeInterface) tla.Value
- func KeySet(iface distsys.ArchetypeInterface) tla.Value
- func LastTerm(iface distsys.ArchetypeInterface, xlog tla.Value) tla.Value
- func Leader(iface distsys.ArchetypeInterface) tla.Value
- func Max(iface distsys.ArchetypeInterface, s1 tla.Value) tla.Value
- func MaxAcc(iface distsys.ArchetypeInterface, s2 tla.Value, e10 tla.Value) tla.Value
- func Min(iface distsys.ArchetypeInterface, s tla.Value) tla.Value
- func MinAcc(iface distsys.ArchetypeInterface, s0 tla.Value, e1 tla.Value) tla.Value
- func NewCustomInChan(ch <-chan tla.Value, timeout time.Duration) distsys.ArchetypeResource
- func NewPersistentLog(name string, db *badger.DB) distsys.ArchetypeResource
- func NewTimerResource(timeout time.Duration, offset time.Duration) distsys.ArchetypeResource
- func Nil(iface distsys.ArchetypeInterface) tla.Value
- func NodeSet(iface distsys.ArchetypeInterface) tla.Value
- func ProposeMessage(iface distsys.ArchetypeInterface) tla.Value
- func RequestVoteRequest(iface distsys.ArchetypeInterface) tla.Value
- func RequestVoteResponse(iface distsys.ArchetypeInterface) tla.Value
- func ServerAdvanceCommitIndexSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerAppendEntriesSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerBecomeLeaderSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerCrasherSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerFollowerAdvanceCommitIndexSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerNetListenerSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerPropChListenerSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerRequestVoteSet(iface distsys.ArchetypeInterface) tla.Value
- func ServerSet(iface distsys.ArchetypeInterface) tla.Value
- func Value1(iface distsys.ArchetypeInterface) tla.Value
- type CustomInChan
- type ImmutableResource
- func (res *ImmutableResource) Abort() chan struct{}
- func (res *ImmutableResource) Close() error
- func (res *ImmutableResource) Commit() chan struct{}
- func (res *ImmutableResource) PreCommit() chan error
- func (res *ImmutableResource) ReadValue() (tla.Value, error)
- func (res *ImmutableResource) WriteValue(value tla.Value) error
- type PersistentLog
- func (res *PersistentLog) Abort() chan struct{}
- func (res *PersistentLog) Close() error
- func (res *PersistentLog) Commit() chan struct{}
- func (res *PersistentLog) Index(index tla.Value) (distsys.ArchetypeResource, error)
- func (res *PersistentLog) PreCommit() chan error
- func (res *PersistentLog) ReadValue() (tla.Value, error)
- func (res *PersistentLog) VClockHint(archClock trace.VClock) trace.VClock
- func (res *PersistentLog) WriteValue(value tla.Value) error
- type TimerResource
- func (res *TimerResource) Abort() chan struct{}
- func (res *TimerResource) Close() error
- func (res *TimerResource) Commit() chan struct{}
- func (res *TimerResource) PreCommit() chan error
- func (res *TimerResource) ReadValue() (tla.Value, error)
- func (res *TimerResource) WriteValue(value tla.Value) error
Constants ¶
This section is empty.
Variables ¶
var AProposer = distsys.MPCalArchetype{ Name: "AProposer", Label: "AProposer.sndReq", RequiredRefParams: []string{"AProposer.propCh"}, RequiredValParams: []string{}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { }, }
var AServerAdvanceCommitIndex = distsys.MPCalArchetype{ Name: "AServerAdvanceCommitIndex", Label: "AServerAdvanceCommitIndex.serverAdvanceCommitIndexLoop", RequiredRefParams: []string{"AServerAdvanceCommitIndex.net", "AServerAdvanceCommitIndex.netLen", "AServerAdvanceCommitIndex.netEnabled", "AServerAdvanceCommitIndex.fd", "AServerAdvanceCommitIndex.state", "AServerAdvanceCommitIndex.currentTerm", "AServerAdvanceCommitIndex.log", "AServerAdvanceCommitIndex.plog", "AServerAdvanceCommitIndex.commitIndex", "AServerAdvanceCommitIndex.nextIndex", "AServerAdvanceCommitIndex.matchIndex", "AServerAdvanceCommitIndex.votedFor", "AServerAdvanceCommitIndex.votesResponded", "AServerAdvanceCommitIndex.votesGranted", "AServerAdvanceCommitIndex.leader", "AServerAdvanceCommitIndex.propCh", "AServerAdvanceCommitIndex.acctCh", "AServerAdvanceCommitIndex.leaderTimeout", "AServerAdvanceCommitIndex.appendEntriesCh", "AServerAdvanceCommitIndex.becomeLeaderCh", "AServerAdvanceCommitIndex.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerAdvanceCommitIndex.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { iface.EnsureArchetypeResourceLocal("AServerAdvanceCommitIndex.newCommitIndex", tla.MakeNumber(0)) }, }
var AServerAppendEntries = distsys.MPCalArchetype{ Name: "AServerAppendEntries", Label: "AServerAppendEntries.serverAppendEntriesLoop", RequiredRefParams: []string{"AServerAppendEntries.net", "AServerAppendEntries.netLen", "AServerAppendEntries.netEnabled", "AServerAppendEntries.fd", "AServerAppendEntries.state", "AServerAppendEntries.currentTerm", "AServerAppendEntries.log", "AServerAppendEntries.plog", "AServerAppendEntries.commitIndex", "AServerAppendEntries.nextIndex", "AServerAppendEntries.matchIndex", "AServerAppendEntries.votedFor", "AServerAppendEntries.votesResponded", "AServerAppendEntries.votesGranted", "AServerAppendEntries.leader", "AServerAppendEntries.propCh", "AServerAppendEntries.acctCh", "AServerAppendEntries.leaderTimeout", "AServerAppendEntries.appendEntriesCh", "AServerAppendEntries.becomeLeaderCh", "AServerAppendEntries.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerAppendEntries.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { iface.EnsureArchetypeResourceLocal("AServerAppendEntries.idx", tla.Value{}) }, }
var AServerBecomeLeader = distsys.MPCalArchetype{ Name: "AServerBecomeLeader", Label: "AServerBecomeLeader.serverBecomeLeaderLoop", RequiredRefParams: []string{"AServerBecomeLeader.net", "AServerBecomeLeader.netLen", "AServerBecomeLeader.netEnabled", "AServerBecomeLeader.fd", "AServerBecomeLeader.state", "AServerBecomeLeader.currentTerm", "AServerBecomeLeader.log", "AServerBecomeLeader.plog", "AServerBecomeLeader.commitIndex", "AServerBecomeLeader.nextIndex", "AServerBecomeLeader.matchIndex", "AServerBecomeLeader.votedFor", "AServerBecomeLeader.votesResponded", "AServerBecomeLeader.votesGranted", "AServerBecomeLeader.leader", "AServerBecomeLeader.propCh", "AServerBecomeLeader.acctCh", "AServerBecomeLeader.leaderTimeout", "AServerBecomeLeader.appendEntriesCh", "AServerBecomeLeader.becomeLeaderCh", "AServerBecomeLeader.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerBecomeLeader.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { }, }
var AServerCrasher = distsys.MPCalArchetype{ Name: "AServerCrasher", Label: "AServerCrasher.serverCrash", RequiredRefParams: []string{"AServerCrasher.netEnabled", "AServerCrasher.fd"}, RequiredValParams: []string{"AServerCrasher.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { }, }
var AServerFollowerAdvanceCommitIndex = distsys.MPCalArchetype{ Name: "AServerFollowerAdvanceCommitIndex", Label: "AServerFollowerAdvanceCommitIndex.serverFollowerAdvanceCommitIndexLoop", RequiredRefParams: []string{"AServerFollowerAdvanceCommitIndex.net", "AServerFollowerAdvanceCommitIndex.netLen", "AServerFollowerAdvanceCommitIndex.netEnabled", "AServerFollowerAdvanceCommitIndex.fd", "AServerFollowerAdvanceCommitIndex.state", "AServerFollowerAdvanceCommitIndex.currentTerm", "AServerFollowerAdvanceCommitIndex.log", "AServerFollowerAdvanceCommitIndex.plog", "AServerFollowerAdvanceCommitIndex.commitIndex", "AServerFollowerAdvanceCommitIndex.nextIndex", "AServerFollowerAdvanceCommitIndex.matchIndex", "AServerFollowerAdvanceCommitIndex.votedFor", "AServerFollowerAdvanceCommitIndex.votesResponded", "AServerFollowerAdvanceCommitIndex.votesGranted", "AServerFollowerAdvanceCommitIndex.leader", "AServerFollowerAdvanceCommitIndex.propCh", "AServerFollowerAdvanceCommitIndex.acctCh", "AServerFollowerAdvanceCommitIndex.leaderTimeout", "AServerFollowerAdvanceCommitIndex.appendEntriesCh", "AServerFollowerAdvanceCommitIndex.becomeLeaderCh", "AServerFollowerAdvanceCommitIndex.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerFollowerAdvanceCommitIndex.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { iface.EnsureArchetypeResourceLocal("AServerFollowerAdvanceCommitIndex.m", tla.Value{}) }, }
var AServerNetListener = distsys.MPCalArchetype{ Name: "AServerNetListener", Label: "AServerNetListener.serverLoop", RequiredRefParams: []string{"AServerNetListener.net", "AServerNetListener.netLen", "AServerNetListener.netEnabled", "AServerNetListener.fd", "AServerNetListener.state", "AServerNetListener.currentTerm", "AServerNetListener.log", "AServerNetListener.plog", "AServerNetListener.commitIndex", "AServerNetListener.nextIndex", "AServerNetListener.matchIndex", "AServerNetListener.votedFor", "AServerNetListener.votesResponded", "AServerNetListener.votesGranted", "AServerNetListener.leader", "AServerNetListener.propCh", "AServerNetListener.acctCh", "AServerNetListener.leaderTimeout", "AServerNetListener.appendEntriesCh", "AServerNetListener.becomeLeaderCh", "AServerNetListener.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerNetListener.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { iface.EnsureArchetypeResourceLocal("AServerNetListener.idx", tla.MakeNumber(1)) iface.EnsureArchetypeResourceLocal("AServerNetListener.m", tla.Value{}) }, }
var AServerPropChListener = distsys.MPCalArchetype{ Name: "AServerPropChListener", Label: "AServerPropChListener.serverLoop", RequiredRefParams: []string{"AServerPropChListener.net", "AServerPropChListener.netLen", "AServerPropChListener.netEnabled", "AServerPropChListener.fd", "AServerPropChListener.state", "AServerPropChListener.currentTerm", "AServerPropChListener.log", "AServerPropChListener.plog", "AServerPropChListener.commitIndex", "AServerPropChListener.nextIndex", "AServerPropChListener.matchIndex", "AServerPropChListener.votedFor", "AServerPropChListener.votesResponded", "AServerPropChListener.votesGranted", "AServerPropChListener.leader", "AServerPropChListener.propCh", "AServerPropChListener.acctCh", "AServerPropChListener.leaderTimeout", "AServerPropChListener.appendEntriesCh", "AServerPropChListener.becomeLeaderCh", "AServerPropChListener.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerPropChListener.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { iface.EnsureArchetypeResourceLocal("AServerPropChListener.idx", tla.MakeNumber(1)) iface.EnsureArchetypeResourceLocal("AServerPropChListener.m", tla.Value{}) }, }
var AServerRequestVote = distsys.MPCalArchetype{ Name: "AServerRequestVote", Label: "AServerRequestVote.serverRequestVoteLoop", RequiredRefParams: []string{"AServerRequestVote.net", "AServerRequestVote.netLen", "AServerRequestVote.netEnabled", "AServerRequestVote.fd", "AServerRequestVote.state", "AServerRequestVote.currentTerm", "AServerRequestVote.log", "AServerRequestVote.plog", "AServerRequestVote.commitIndex", "AServerRequestVote.nextIndex", "AServerRequestVote.matchIndex", "AServerRequestVote.votedFor", "AServerRequestVote.votesResponded", "AServerRequestVote.votesGranted", "AServerRequestVote.leader", "AServerRequestVote.propCh", "AServerRequestVote.acctCh", "AServerRequestVote.leaderTimeout", "AServerRequestVote.appendEntriesCh", "AServerRequestVote.becomeLeaderCh", "AServerRequestVote.fAdvCommitIdxCh"}, RequiredValParams: []string{"AServerRequestVote.srvId"}, JumpTable: jumpTable, ProcTable: procTable, PreAmble: func(iface distsys.ArchetypeInterface) { iface.EnsureArchetypeResourceLocal("AServerRequestVote.idx", tla.MakeNumber(1)) }, }
var LeaderTimeoutConstantDefs = distsys.EnsureMPCalContextConfigs( distsys.DefineConstantValue("LeaderTimeoutReset", tla.MakeBool(true)), )
var PersistentLogConstantDefs = distsys.EnsureMPCalContextConfigs( distsys.DefineConstantValue("LogConcat", logConcat), distsys.DefineConstantValue("LogPop", logPop), )
Functions ¶
func AcceptMessage ¶
func AcceptMessage(iface distsys.ArchetypeInterface) tla.Value
func AppendEntriesRequest ¶
func AppendEntriesRequest(iface distsys.ArchetypeInterface) tla.Value
func AppendEntriesResponse ¶
func AppendEntriesResponse(iface distsys.ArchetypeInterface) tla.Value
func FindMaxAgreeIndex ¶
func FindMaxAgreeIndexRec ¶
func NewCustomInChan ¶
func NewPersistentLog ¶
func NewPersistentLog(name string, db *badger.DB) distsys.ArchetypeResource
func NewTimerResource ¶
func ProposeMessage ¶
func ProposeMessage(iface distsys.ArchetypeInterface) tla.Value
func RequestVoteRequest ¶
func RequestVoteRequest(iface distsys.ArchetypeInterface) tla.Value
func RequestVoteResponse ¶
func RequestVoteResponse(iface distsys.ArchetypeInterface) tla.Value
func ServerAdvanceCommitIndexSet ¶
func ServerAdvanceCommitIndexSet(iface distsys.ArchetypeInterface) tla.Value
func ServerAppendEntriesSet ¶
func ServerAppendEntriesSet(iface distsys.ArchetypeInterface) tla.Value
func ServerBecomeLeaderSet ¶
func ServerBecomeLeaderSet(iface distsys.ArchetypeInterface) tla.Value
func ServerCrasherSet ¶
func ServerCrasherSet(iface distsys.ArchetypeInterface) tla.Value
func ServerFollowerAdvanceCommitIndexSet ¶
func ServerFollowerAdvanceCommitIndexSet(iface distsys.ArchetypeInterface) tla.Value
func ServerNetListenerSet ¶
func ServerNetListenerSet(iface distsys.ArchetypeInterface) tla.Value
func ServerPropChListenerSet ¶
func ServerPropChListenerSet(iface distsys.ArchetypeInterface) tla.Value
func ServerRequestVoteSet ¶
func ServerRequestVoteSet(iface distsys.ArchetypeInterface) tla.Value
Types ¶
type CustomInChan ¶
type CustomInChan struct { distsys.ArchetypeResourceLeafMixin // contains filtered or unexported fields }
CustomInChan is similar resources.InputChannel, however, after a timeout it returns a default value instead of aborting the critical section. It used in implementing periodic sending of AppendEntries request. In some cases, the request should be sent immediately, for example, when the server just becomes a leader. In this case, the input channel signals.
func (*CustomInChan) Abort ¶
func (res *CustomInChan) Abort() chan struct{}
func (*CustomInChan) Close ¶
func (res *CustomInChan) Close() error
func (*CustomInChan) Commit ¶
func (res *CustomInChan) Commit() chan struct{}
func (*CustomInChan) PreCommit ¶
func (res *CustomInChan) PreCommit() chan error
func (*CustomInChan) WriteValue ¶
func (res *CustomInChan) WriteValue(value tla.Value) error
type ImmutableResource ¶
type ImmutableResource struct { distsys.ArchetypeResourceLeafMixin // contains filtered or unexported fields }
func (*ImmutableResource) Abort ¶
func (res *ImmutableResource) Abort() chan struct{}
func (*ImmutableResource) Close ¶
func (res *ImmutableResource) Close() error
func (*ImmutableResource) Commit ¶
func (res *ImmutableResource) Commit() chan struct{}
func (*ImmutableResource) PreCommit ¶
func (res *ImmutableResource) PreCommit() chan error
func (*ImmutableResource) WriteValue ¶
func (res *ImmutableResource) WriteValue(value tla.Value) error
type PersistentLog ¶
type PersistentLog struct {
// contains filtered or unexported fields
}
PersistentLog is a distsys.ArchetypeResource that implements Raft's persistent log behavior.
func (*PersistentLog) Abort ¶
func (res *PersistentLog) Abort() chan struct{}
func (*PersistentLog) Close ¶
func (res *PersistentLog) Close() error
func (*PersistentLog) Commit ¶
func (res *PersistentLog) Commit() chan struct{}
func (*PersistentLog) Index ¶
func (res *PersistentLog) Index(index tla.Value) (distsys.ArchetypeResource, error)
func (*PersistentLog) PreCommit ¶
func (res *PersistentLog) PreCommit() chan error
func (*PersistentLog) VClockHint ¶
func (res *PersistentLog) VClockHint(archClock trace.VClock) trace.VClock
func (*PersistentLog) WriteValue ¶
func (res *PersistentLog) WriteValue(value tla.Value) error
type TimerResource ¶
type TimerResource struct { distsys.ArchetypeResourceLeafMixin // contains filtered or unexported fields }
TimerResource is used to implement randomized timeout in the Raft leader election. It measures the time since the last call to Read and if it's greater than some random timeout, then it returns true; otherwise, returns false. Also, it supports timer reset through write calls.
func (*TimerResource) Abort ¶
func (res *TimerResource) Abort() chan struct{}
func (*TimerResource) Close ¶
func (res *TimerResource) Close() error
func (*TimerResource) Commit ¶
func (res *TimerResource) Commit() chan struct{}
func (*TimerResource) PreCommit ¶
func (res *TimerResource) PreCommit() chan error
func (*TimerResource) WriteValue ¶
func (res *TimerResource) WriteValue(value tla.Value) error