From 7f4cd2a26bdbf56fcd8e2ce32be0bb96a531d82c Mon Sep 17 00:00:00 2001 From: ZhouJianMS Date: Tue, 20 Feb 2024 10:34:10 +0800 Subject: [PATCH 1/2] Extend Raft algorithm to support share witness >> > >> > >> Co-authored-by: Joshua Zhang >> Co-authored-by: Jian Zhou Signed-off-by: ZhouJianMS --- bootstrap.go | 16 +- confchange/confchange.go | 86 ++- confchange/quick_test.go | 3 +- confchange/restore.go | 33 +- log.go | 12 + node.go | 72 +- node_test.go | 29 +- quorum/joint.go | 18 + quorum/majority.go | 82 +++ raft.go | 232 +++++- raft_test.go | 7 +- raftpb/confchange.go | 4 + raftpb/raft.pb.go | 681 ++++++++++++++++-- raftpb/raft.proto | 14 + raftpb/raft_test.go | 8 +- ...action_env_handler_process_apply_thread.go | 13 +- rawnode.go | 10 +- rawnode_test.go | 56 +- storage.go | 1 + testdata/async_storage_writes.txt | 6 +- .../async_storage_writes_append_aba_race.txt | 20 +- testdata/campaign.txt | 6 +- testdata/campaign_learner_must_vote.txt | 62 +- testdata/confchange_v1_add_single.txt | 42 +- testdata/confchange_v1_remove_leader.txt | 81 ++- testdata/confchange_v2_add_double_auto.txt | 325 +++++---- .../confchange_v2_add_double_implicit.txt | 78 +- testdata/confchange_v2_add_single_auto.txt | 42 +- .../confchange_v2_add_single_explicit.txt | 154 ++-- testdata/confchange_v2_replace_leader.txt | 313 +++++--- testdata/probe_and_replicate.txt | 14 +- testdata/single_node.txt | 4 +- tracker/progress.go | 6 + tracker/tracker.go | 236 +++++- util.go | 24 +- witness.go | 280 +++++++ 36 files changed, 2504 insertions(+), 566 deletions(-) create mode 100644 witness.go diff --git a/bootstrap.go b/bootstrap.go index 2a61aa23..5f227e91 100644 --- a/bootstrap.go +++ b/bootstrap.go @@ -50,7 +50,13 @@ func (rn *RawNode) Bootstrap(peers []Peer) error { rn.raft.becomeFollower(1, None) ents := make([]pb.Entry, len(peers)) for i, peer := range peers { - cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context} + var cct pb.ConfChangeType + if peer.IsWitness { + cct = pb.ConfChangeAddWitness + } else { + cct = pb.ConfChangeAddNode + } + cc := pb.ConfChange{Type: cct, NodeID: peer.ID, Context: peer.Context} data, err := cc.Marshal() if err != nil { return err @@ -74,7 +80,13 @@ func (rn *RawNode) Bootstrap(peers []Peer) error { // the invariant that committed < unstable? rn.raft.raftLog.committed = uint64(len(ents)) for _, peer := range peers { - rn.raft.applyConfChange(pb.ConfChange{NodeID: peer.ID, Type: pb.ConfChangeAddNode}.AsV2()) + var cct pb.ConfChangeType + if peer.IsWitness { + cct = pb.ConfChangeAddWitness + } else { + cct = pb.ConfChangeAddNode + } + rn.raft.applyConfChange(pb.ConfChange{NodeID: peer.ID, Type: cct}.AsV2()) } return nil } diff --git a/confchange/confchange.go b/confchange/confchange.go index 55db16a8..d5fab618 100644 --- a/confchange/confchange.go +++ b/confchange/confchange.go @@ -69,6 +69,7 @@ func (c Changer) EnterJoint(autoLeave bool, ccs ...pb.ConfChangeSingle) (tracker for id := range incoming(cfg.Voters) { outgoing(cfg.Voters)[id] = struct{}{} } + cfg.Witnesses[1] = cfg.Witnesses[0] if err := c.apply(&cfg, trk, ccs...); err != nil { return c.err(err) @@ -109,12 +110,14 @@ func (c Changer) LeaveJoint() (tracker.Config, tracker.ProgressMap, error) { for id := range outgoing(cfg.Voters) { _, isVoter := incoming(cfg.Voters)[id] _, isLearner := cfg.Learners[id] + isWitness := cfg.Witnesses[0] == id - if !isVoter && !isLearner { + if !isVoter && !isLearner && !isWitness { delete(trk, id) } } *outgoingPtr(&cfg.Voters) = nil + cfg.Witnesses[1] = 0 cfg.AutoLeave = false return checkAndReturn(cfg, trk) @@ -155,17 +158,25 @@ func (c Changer) apply(cfg *tracker.Config, trk tracker.ProgressMap, ccs ...pb.C // here to ignore these. continue } + + var err error switch cc.Type { case pb.ConfChangeAddNode: - c.makeVoter(cfg, trk, cc.NodeID) + err = c.makeVoter(cfg, trk, cc.NodeID) case pb.ConfChangeAddLearnerNode: - c.makeLearner(cfg, trk, cc.NodeID) + err = c.makeLearner(cfg, trk, cc.NodeID) case pb.ConfChangeRemoveNode: c.remove(cfg, trk, cc.NodeID) case pb.ConfChangeUpdateNode: + case pb.ConfChangeAddWitness: + err = c.makeWitness(cfg, trk, cc.NodeID) default: return fmt.Errorf("unexpected conf type %d", cc.Type) } + + if err != nil { + return err + } } if len(incoming(cfg.Voters)) == 0 { return errors.New("removed all voters") @@ -175,17 +186,23 @@ func (c Changer) apply(cfg *tracker.Config, trk tracker.ProgressMap, ccs ...pb.C // makeVoter adds or promotes the given ID to be a voter in the incoming // majority config. -func (c Changer) makeVoter(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) { +func (c Changer) makeVoter(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) error { pr := trk[id] if pr == nil { - c.initProgress(cfg, trk, id, false /* isLearner */) - return + c.initProgress(cfg, trk, id, false /* isLearner */, false /* isWitness */) + return nil + } + + if pr.IsWitness { + return fmt.Errorf("cannot change witness to non-witness voter") } pr.IsLearner = false nilAwareDelete(&cfg.Learners, id) nilAwareDelete(&cfg.LearnersNext, id) incoming(cfg.Voters)[id] = struct{}{} + + return nil } // makeLearner makes the given ID a learner or stages it to be a learner once @@ -201,14 +218,17 @@ func (c Changer) makeVoter(cfg *tracker.Config, trk tracker.ProgressMap, id uint // simultaneously. Instead, we add the learner to LearnersNext, so that it will // be added to Learners the moment the outgoing config is removed by // LeaveJoint(). -func (c Changer) makeLearner(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) { +func (c Changer) makeLearner(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) error { pr := trk[id] if pr == nil { - c.initProgress(cfg, trk, id, true /* isLearner */) - return + c.initProgress(cfg, trk, id, true /* isLearner */, false /* isWitness */) + return nil } if pr.IsLearner { - return + return nil + } + if pr.IsWitness { + return fmt.Errorf("cannot change witness to non-witness learner") } // Remove any existing voter in the incoming config... c.remove(cfg, trk, id) @@ -223,11 +243,35 @@ func (c Changer) makeLearner(cfg *tracker.Config, trk tracker.ProgressMap, id ui nilAwareAdd(&cfg.LearnersNext, id) } else { pr.IsLearner = true + pr.IsWitness = false nilAwareAdd(&cfg.Learners, id) } + + return nil } -// remove this peer as a voter or learner from the incoming config. +// makeWitness adds the given ID to be a witness in the incoming +// majority config. +func (c Changer) makeWitness(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) error { + if cfg.Witnesses[0] > 0 && cfg.Witnesses[0] != id { + return fmt.Errorf("cannot have more than one witness in incoming") + } + + // now either incoming does not have any witness or it already has this witness + pr := trk[id] + if pr == nil { + c.initProgress(cfg, trk, id, false /* isLearner */, true /* isWitness */) + return nil + } + + if !pr.IsWitness { + return fmt.Errorf("cannot change non-witness voter/learner to witness") + } + + return nil +} + +// remove this peer as a voter or learner or witness from the incoming config. func (c Changer) remove(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) { if _, ok := trk[id]; !ok { return @@ -236,6 +280,9 @@ func (c Changer) remove(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) delete(incoming(cfg.Voters), id) nilAwareDelete(&cfg.Learners, id) nilAwareDelete(&cfg.LearnersNext, id) + if id == cfg.Witnesses[0] { + cfg.Witnesses[0] = 0 + } // If the peer is still a voter in the outgoing config, keep the Progress. if _, onRight := outgoing(cfg.Voters)[id]; !onRight { @@ -243,10 +290,13 @@ func (c Changer) remove(cfg *tracker.Config, trk tracker.ProgressMap, id uint64) } } -// initProgress initializes a new progress for the given node or learner. -func (c Changer) initProgress(cfg *tracker.Config, trk tracker.ProgressMap, id uint64, isLearner bool) { +// initProgress initializes a new progress for the given node or learner or witness. +func (c Changer) initProgress(cfg *tracker.Config, trk tracker.ProgressMap, id uint64, isLearner bool, isWitness bool) { if !isLearner { incoming(cfg.Voters)[id] = struct{}{} + if isWitness { + cfg.Witnesses[0] = id + } } else { nilAwareAdd(&cfg.Learners, id) } @@ -263,6 +313,7 @@ func (c Changer) initProgress(cfg *tracker.Config, trk tracker.ProgressMap, id u Match: 0, Inflights: tracker.NewInflights(c.Tracker.MaxInflight, c.Tracker.MaxInflightBytes), IsLearner: isLearner, + IsWitness: isWitness, // When a node is first added, we should mark it as recently active. // Otherwise, CheckQuorum may cause us to step down if it is invoked // before the added node has had a chance to communicate with us. @@ -328,6 +379,15 @@ func checkInvariants(cfg tracker.Config, trk tracker.ProgressMap) error { } } + for id, voters := range cfg.Voters { + w := cfg.Witnesses[id] + if w > 0 { + if _, ok := voters[w]; !ok { + return fmt.Errorf("%d is in Witnesses[%d] but not in Voters[%d]", w, id, id) + } + } + } + return nil } diff --git a/confchange/quick_test.go b/confchange/quick_test.go index 06784298..b8da5ed7 100644 --- a/confchange/quick_test.go +++ b/confchange/quick_test.go @@ -162,7 +162,8 @@ func (confChanges) Generate(rand *rand.Rand, _ int) reflect.Value { return 1 + uint64(num()) } typ := func() pb.ConfChangeType { - return pb.ConfChangeType(rand.Intn(len(pb.ConfChangeType_name))) + // exclude witness from the test case + return pb.ConfChangeType(rand.Intn(len(pb.ConfChangeType_name) - 1)) } return reflect.ValueOf(genCC(num, id, typ)) } diff --git a/confchange/restore.go b/confchange/restore.go index 68ef029a..b73f9ec1 100644 --- a/confchange/restore.go +++ b/confchange/restore.go @@ -51,14 +51,20 @@ func toConfChangeSingle(cs pb.ConfState) (out []pb.ConfChangeSingle, in []pb.Con // quorum=(1 2 3)&&(1 2 4 6) learners=(5) learners_next=(4) // // as desired. - for _, id := range cs.VotersOutgoing { // If there are outgoing voters, first add them one by one so that the // (non-joint) config has them all. - out = append(out, pb.ConfChangeSingle{ - Type: pb.ConfChangeAddNode, - NodeID: id, - }) + if id == cs.WitnessOutgoing { + out = append(out, pb.ConfChangeSingle{ + Type: pb.ConfChangeAddWitness, + NodeID: cs.WitnessOutgoing, + }) + } else { + out = append(out, pb.ConfChangeSingle{ + Type: pb.ConfChangeAddNode, + NodeID: id, + }) + } } @@ -72,12 +78,20 @@ func toConfChangeSingle(cs pb.ConfState) (out []pb.ConfChangeSingle, in []pb.Con NodeID: id, }) } + // Then we'll add the incoming voters and learners. for _, id := range cs.Voters { - in = append(in, pb.ConfChangeSingle{ - Type: pb.ConfChangeAddNode, - NodeID: id, - }) + if id == cs.Witness { + in = append(in, pb.ConfChangeSingle{ + Type: pb.ConfChangeAddWitness, + NodeID: cs.Witness, + }) + } else { + in = append(in, pb.ConfChangeSingle{ + Type: pb.ConfChangeAddNode, + NodeID: id, + }) + } } for _, id := range cs.Learners { in = append(in, pb.ConfChangeSingle{ @@ -93,6 +107,7 @@ func toConfChangeSingle(cs pb.ConfState) (out []pb.ConfChangeSingle, in []pb.Con NodeID: id, }) } + return out, in } diff --git a/log.go b/log.go index bd7c2feb..a871fd5d 100644 --- a/log.go +++ b/log.go @@ -306,6 +306,18 @@ func (l *raftLog) firstIndex() uint64 { return index } +func (l *raftLog) entry(i uint64) *pb.Entry { + ents, err := l.entries(i, noLimit) + if err != nil { + panic(err) + } + return &ents[0] +} +func (l *raftLog) lastEntry() *pb.Entry { + idx := l.lastIndex() + return l.entry(idx) +} + func (l *raftLog) lastIndex() uint64 { if i, ok := l.unstable.maybeLastIndex(); ok { return i diff --git a/node.go b/node.go index e2a261cd..8688ff42 100644 --- a/node.go +++ b/node.go @@ -112,6 +112,8 @@ type Ready struct { // MustSync indicates whether the HardState and Entries must be durably // written to disk or if a non-durable write is permissible. MustSync bool + + WitnessMessages []WitnessMessage } func isHardStateEqual(a, b pb.HardState) bool { @@ -184,7 +186,7 @@ type Node interface { // // Returns an opaque non-nil ConfState protobuf which must be recorded in // snapshots. - ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState + ApplyConfChange(cc pb.ConfChangeI) (*pb.ConfState, error) // TransferLeadership attempts to transfer leadership to the given transferee. TransferLeadership(ctx context.Context, lead, transferee uint64) @@ -243,8 +245,9 @@ type Node interface { } type Peer struct { - ID uint64 - Context []byte + ID uint64 + Context []byte + IsWitness bool } func setupNode(c *Config, peers []Peer) *node { @@ -293,12 +296,17 @@ type msgWithResult struct { result chan error } +type confStateWithResult struct { + cs *pb.ConfState + result error +} + // node is the canonical implementation of the Node interface type node struct { propc chan msgWithResult recvc chan pb.Message confc chan pb.ConfChangeV2 - confstatec chan pb.ConfState + confstatec chan confStateWithResult readyc chan Ready advancec chan struct{} tickc chan struct{} @@ -314,7 +322,7 @@ func newNode(rn *RawNode) node { propc: make(chan msgWithResult), recvc: make(chan pb.Message), confc: make(chan pb.ConfChangeV2), - confstatec: make(chan pb.ConfState), + confstatec: make(chan confStateWithResult), readyc: make(chan Ready), advancec: make(chan struct{}), // make tickc a buffered chan, so raft node can buffer some ticks when the node @@ -399,35 +407,39 @@ func (n *node) run() { r.Step(m) case cc := <-n.confc: _, okBefore := r.trk.Progress[r.id] - cs := r.applyConfChange(cc) - // If the node was removed, block incoming proposals. Note that we - // only do this if the node was in the config before. Nodes may be - // a member of the group without knowing this (when they're catching - // up on the log and don't have the latest config) and we don't want - // to block the proposal channel in that case. - // - // NB: propc is reset when the leader changes, which, if we learn - // about it, sort of implies that we got readded, maybe? This isn't - // very sound and likely has bugs. - if _, okAfter := r.trk.Progress[r.id]; okBefore && !okAfter { - var found bool - for _, sl := range [][]uint64{cs.Voters, cs.VotersOutgoing} { - for _, id := range sl { - if id == r.id { - found = true + cs, err := r.applyConfChange(cc) + if err != nil { + r.logger.Warningf("raft.node: %x failed to apply conf change. Error: %s", r.id, err.Error()) + } else { + // If the node was removed, block incoming proposals. Note that we + // only do this if the node was in the config before. Nodes may be + // a member of the group without knowing this (when they're catching + // up on the log and don't have the latest config) and we don't want + // to block the proposal channel in that case. + // + // NB: propc is reset when the leader changes, which, if we learn + // about it, sort of implies that we got readded, maybe? This isn't + // very sound and likely has bugs. + if _, okAfter := r.trk.Progress[r.id]; okBefore && !okAfter { + var found bool + for _, sl := range [][]uint64{cs.Voters, cs.VotersOutgoing} { + for _, id := range sl { + if id == r.id { + found = true + break + } + } + if found { break } } - if found { - break + if !found { + propc = nil } } - if !found { - propc = nil - } } select { - case n.confstatec <- cs: + case n.confstatec <- confStateWithResult{cs: cs, result: err}: case <-n.done: } case <-n.tickc: @@ -553,8 +565,8 @@ func (n *node) Advance() { } } -func (n *node) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState { - var cs pb.ConfState +func (n *node) ApplyConfChange(cc pb.ConfChangeI) (*pb.ConfState, error) { + var cs confStateWithResult select { case n.confc <- cc.AsV2(): case <-n.done: @@ -563,7 +575,7 @@ func (n *node) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState { case cs = <-n.confstatec: case <-n.done: } - return &cs + return cs.cs, cs.result } func (n *node) Status() Status { diff --git a/node_test.go b/node_test.go index 281ada19..edf8fd02 100644 --- a/node_test.go +++ b/node_test.go @@ -380,13 +380,13 @@ func TestNodeProposeAddDuplicateNode(t *testing.T) { cancel() <-goroutineStopped - if len(allCommittedEntries) != 4 { - t.Errorf("len(entry) = %d, want %d, %v\n", len(allCommittedEntries), 4, allCommittedEntries) + if len(allCommittedEntries) != 6 { + t.Errorf("len(entry) = %d, want %d, %v\n", len(allCommittedEntries), 6, allCommittedEntries) } if !bytes.Equal(allCommittedEntries[1].Data, ccdata1) { t.Errorf("data = %v, want %v", allCommittedEntries[1].Data, ccdata1) } - if !bytes.Equal(allCommittedEntries[3].Data, ccdata2) { + if !bytes.Equal(allCommittedEntries[5].Data, ccdata2) { t.Errorf("data = %v, want %v", allCommittedEntries[3].Data, ccdata2) } } @@ -816,18 +816,21 @@ func TestNodeProposeAddLearnerNode(t *testing.T) { } var cc raftpb.ConfChange cc.Unmarshal(ent.Data) - state := n.ApplyConfChange(cc) - if len(state.Learners) == 0 || - state.Learners[0] != cc.NodeID || - cc.NodeID != 2 { - t.Errorf("apply conf change should return new added learner: %v", state.String()) + state, err := n.ApplyConfChange(cc) + if err == nil { + if len(state.Learners) == 0 || + state.Learners[0] != cc.NodeID || + cc.NodeID != 2 { + t.Errorf("apply conf change should return new added learner: %v", state.String()) + } + + if len(state.Voters) != 1 { + t.Errorf("add learner should not change the nodes: %v", state.String()) + } + t.Logf("apply raft conf %v changed to: %v", cc, state.String()) + applyConfChan <- struct{}{} } - if len(state.Voters) != 1 { - t.Errorf("add learner should not change the nodes: %v", state.String()) - } - t.Logf("apply raft conf %v changed to: %v", cc, state.String()) - applyConfChan <- struct{}{} } n.Advance() } diff --git a/quorum/joint.go b/quorum/joint.go index e3741e0b..3c0a05e1 100644 --- a/quorum/joint.go +++ b/quorum/joint.go @@ -73,3 +73,21 @@ func (c JointConfig) VoteResult(votes map[uint64]bool) VoteResult { // One side won, the other one is pending, so the whole outcome is. return VotePending } + +func (c JointConfig) VoteResultWithDifference(votes map[uint64]bool) (VoteResult, [2]int) { + r1, v1 := c[0].VoteResultWithDifference(votes) + r2, v2 := c[1].VoteResultWithDifference(votes) + + votesToWin := [2]int{v1, v2} + + if r1 == r2 { + // If they agree, return the agreed state. + return r1, votesToWin + } + if r1 == VoteLost || r2 == VoteLost { + // If either config has lost, loss is the only possible outcome. + return VoteLost, votesToWin + } + // One side won, the other one is pending, so the whole outcome is. + return VotePending, votesToWin +} diff --git a/quorum/majority.go b/quorum/majority.go index 12766137..ad82918a 100644 --- a/quorum/majority.go +++ b/quorum/majority.go @@ -171,6 +171,54 @@ func (c MajorityConfig) CommittedIndex(l AckedIndexer) Index { return Index(srt[pos]) } +func (c MajorityConfig) OneLessThanQuorum(l AckedIndexer) Index { + n := len(c) + if n == 0 { + // This plays well with joint quorums which, when one half is the zero + // MajorityConfig, should behave like the other half. + return math.MaxUint64 + } + + // Use an on-stack slice to collect the committed indexes when n <= 7 + // (otherwise we alloc). The alternative is to stash a slice on + // MajorityConfig, but this impairs usability (as is, MajorityConfig is just + // a map, and that's nice). The assumption is that running with a + // replication factor of >7 is rare, and in cases in which it happens + // performance is a lesser concern (additionally the performance + // implications of an allocation here are far from drastic). + var stk [7]uint64 + var srt []uint64 + if len(stk) >= n { + srt = stk[:n] + } else { + srt = make([]uint64, n) + } + + { + // Fill the slice with the indexes observed. Any unused slots will be + // left as zero; these correspond to voters that may report in, but + // haven't yet. We fill from the right (since the zeroes will end up on + // the left after sorting below anyway). + i := n - 1 + for id := range c { + if idx, ok := l.AckedIndex(id); ok { + srt[i] = uint64(idx) + i-- + } + } + } + + // Sort by index. Use a bespoke algorithm (copied from the stdlib's sort + // package) to keep srt on the stack. + insertionSort(srt) + + // The smallest index into the array for which the value is acked by (quorum - 1). + // In other words, from the end of the slice, move n/2+1+1 to the + // left (accounting for zero-indexing). + pos := n - n/2 + return Index(srt[pos]) +} + // VoteResult takes a mapping of voters to yes/no (true/false) votes and returns // a result indicating whether the vote is pending (i.e. neither a quorum of // yes/no has been reached), won (a quorum of yes has been reached), or lost (a @@ -205,3 +253,37 @@ func (c MajorityConfig) VoteResult(votes map[uint64]bool) VoteResult { } return VoteLost } + +func (c MajorityConfig) VoteResultWithDifference(votes map[uint64]bool) (VoteResult, int) { + if len(c) == 0 { + // By convention, the elections on an empty config win. This comes in + // handy with joint quorums because it'll make a half-populated joint + // quorum behave like a majority quorum. + return VoteWon, 0 + } + + ny := [2]int{} // vote counts for no and yes, respectively + + var missing int + for id := range c { + v, ok := votes[id] + if !ok { + missing++ + continue + } + if v { + ny[1]++ + } else { + ny[0]++ + } + } + + q := len(c)/2 + 1 + if ny[1] >= q { + return VoteWon, 0 + } + if ny[1]+missing >= q { + return VotePending, q - ny[1] + } + return VoteLost, 0 +} diff --git a/raft.go b/raft.go index 3357ae45..ffa55867 100644 --- a/raft.go +++ b/raft.go @@ -335,6 +335,19 @@ func (c *Config) validate() error { return nil } +type WitnessMessage struct { + Type pb.MessageType + From uint64 + To uint64 + Term uint64 + LastLogTerm uint64 + LastLogSubterm uint64 + LastLogIndex uint64 + ReplicationSet [2][]uint64 + Votes map[uint64]bool + Context []byte +} + type raft struct { id uint64 @@ -427,6 +440,8 @@ type raft struct { // current term. Those will be handled as fast as first log is committed in // current term. pendingReadIndexMessages []pb.Message + + witnessMsgs []WitnessMessage } func newRaft(c *Config) *raft { @@ -601,7 +616,7 @@ func (r *raft) sendAppend(to uint64) { // are undesirable when we're sending multiple messages in a batch). func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool { pr := r.trk.Progress[to] - if pr.IsPaused() { + if pr.IsPaused() || pr.IsWitness { return false } @@ -673,6 +688,96 @@ func (r *raft) maybeSendSnapshot(to uint64, pr *tracker.Progress) bool { return true } +func (r *raft) sendHeartbeatToWitness(id uint64, ctx []byte) { + r.witnessMsgs = append(r.witnessMsgs, WitnessMessage{ + Context: ctx, + From: r.id, + To: id, + Type: pb.MsgHeartbeat, + Term: r.Term, + }) +} + +func (r *raft) getWitnessVoteRequestReadiness(votesToWin [2]int) map[uint64]bool { + v1 := votesToWin[0] + v2 := votesToWin[1] + if v1 > 1 && v2 > 1 { + return nil + } + + w1 := r.trk.Witnesses[0] + w2 := r.trk.Witnesses[1] + if w1 == 0 && w2 == 0 { + return nil + } + + w := map[uint64]bool{} + + if w1 == w2 { + w[w1] = (v1 == 1 && v2 <= 1) || (v1 <= 1 && v2 == 1) + } else { + if w1 > 0 { + w[w1] = v1 == 1 + } + if w2 > 0 { + w[w2] = v2 == 1 + } + } + + return w +} + +func (r *raft) sendRequestVoteToWitness(ctx []byte, witnessID uint64, voteType pb.MessageType, term uint64) { + lastEntry := r.raftLog.lastEntry() + votes := r.trk.Votes + v := make(map[uint64]bool) + for id := range votes { + v[id] = votes[id] + } + + r.witnessMsgs = append(r.witnessMsgs, WitnessMessage{ + Context: ctx, + From: r.id, + To: witnessID, + Type: voteType, + Term: term, + LastLogTerm: lastEntry.Term, + LastLogSubterm: lastEntry.Subterm, + LastLogIndex: lastEntry.Index, + Votes: v, + }) + + r.logger.Infof("%x [logterm: %d, logsubterm: %d] sent %s request to witness %x at term %d. Message term: %d", + r.id, lastEntry.Term, lastEntry.Subterm, voteType, witnessID, r.Term, term) +} + +func (r *raft) sendAppendToWitness(witnessID uint64, index uint64) { + epoch := r.trk.Epoch + if epoch == nil { + return + } + + entries, err := r.raftLog.entries(index, 0) + if err != nil { + return + } + + msg := WitnessMessage{ + From: r.id, + To: witnessID, + Type: pb.MsgApp, + Term: r.Term, + LastLogTerm: entries[0].Term, + LastLogSubterm: entries[0].Subterm, + LastLogIndex: entries[0].Index, + ReplicationSet: [2][]uint64{epoch.ReplicationSet[0].GetNonWitnessVoterSlice(), epoch.ReplicationSet[1].GetNonWitnessVoterSlice()}, + } + + r.witnessMsgs = append(r.witnessMsgs, msg) + r.logger.Infof("%x [logterm: %d, logsubterm: %d] sent %s request to witness %x at term %d", + r.id, entries[0].Term, entries[0].Subterm, pb.MsgApp, witnessID, r.Term) +} + // sendHeartbeat sends a heartbeat RPC to the given peer. func (r *raft) sendHeartbeat(to uint64, ctx []byte) { // Attach the commit as min(to.matched, r.committed). @@ -714,11 +819,15 @@ func (r *raft) bcastHeartbeat() { } func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { - r.trk.Visit(func(id uint64, _ *tracker.Progress) { + r.trk.Visit(func(id uint64, pr *tracker.Progress) { if id == r.id { return } - r.sendHeartbeat(id, ctx) + if pr.IsWitness { + r.sendHeartbeatToWitness(id, ctx) + } else { + r.sendHeartbeat(id, ctx) + } }) } @@ -761,6 +870,18 @@ func (r *raft) appliedSnap(snap *pb.Snapshot) { // index changed (in which case the caller should call r.bcastAppend). This can // only be called in StateLeader. func (r *raft) maybeCommit() bool { + r.trk.Committed() + // check if we can commit on q-1 acks in current set + idxMap := r.trk.OneLessThanQuorumInReplicationSet() + for w, wci := range idxMap { + if wci > 0 { + e := r.raftLog.entry(uint64(wci)) + if e.Term == r.Term && e.Subterm == r.trk.Epoch.Subterm { + r.sendAppendToWitness(w, uint64(wci)) + } + } + } + return r.raftLog.maybeCommit(entryID{term: r.Term, index: r.trk.Committed()}) } @@ -769,6 +890,7 @@ func (r *raft) reset(term uint64) { r.Term = term r.Vote = None } + r.lead = None r.electionElapsed = 0 @@ -784,6 +906,7 @@ func (r *raft) reset(term uint64) { Next: r.raftLog.lastIndex() + 1, Inflights: tracker.NewInflights(r.trk.MaxInflight, r.trk.MaxInflightBytes), IsLearner: pr.IsLearner, + IsWitness: pr.IsWitness, } if id == r.id { pr.Match = r.raftLog.lastIndex() @@ -797,9 +920,11 @@ func (r *raft) reset(term uint64) { func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) { li := r.raftLog.lastIndex() + epoch := r.trk.Epoch for i := range es { es[i].Term = r.Term es[i].Index = li + 1 + uint64(i) + es[i].Subterm = epoch.Subterm } // Track the size of this uncommitted proposal. if !r.increaseUncommittedSize(es) { @@ -933,11 +1058,8 @@ func (r *raft) becomeLeader() { // could be expensive. r.pendingConfIndex = r.raftLog.lastIndex() - emptyEnt := pb.Entry{Data: nil} - if !r.appendEntry(emptyEnt) { - // This won't happen because we just called reset() above. - r.logger.Panic("empty entry was dropped") - } + r.maybeStartNewSubterm(true, false) + // The payloadSize of an empty entry is 0 (see TestPayloadSizeOfEmptyEntry), // so the preceding log append does not count against the uncommitted log // quota of the new leader. In other words, after the call to appendEntry, @@ -1003,6 +1125,7 @@ func (r *raft) campaign(t CampaignType) { // better safe than sorry. r.logger.Warningf("%x is unpromotable; campaign() should have been called", r.id) } + var term uint64 var voteMsg pb.MessageType if t == campaignPreElection { @@ -1034,6 +1157,13 @@ func (r *raft) campaign(t CampaignType) { r.send(pb.Message{To: id, Term: term, Type: voteRespMsgType(voteMsg)}) continue } + + if id == r.trk.Witnesses[0] || id == r.trk.Witnesses[1] { + // Witness will vote for candidate only after candidate gets at least + // quorum - 1 votes from other servers. + continue + } + // TODO(pav-kv): it should be ok to simply print %+v for the lastEntryID. last := r.raftLog.lastEntryID() r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d", @@ -1057,6 +1187,16 @@ func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int, rejected return r.trk.TallyVotes() } +func (r *raft) pollAndReportDiff(id uint64, t pb.MessageType, v bool) (granted int, rejected int, result quorum.VoteResult, votesToWin [2]int) { + if v { + r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term) + } else { + r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term) + } + r.trk.RecordVote(id, v) + return r.trk.TallyVotesWithDifference() +} + func (r *raft) Step(m pb.Message) error { // Handle the message term, which may result in our stepping down to a follower. switch { @@ -1247,6 +1387,8 @@ func stepLeader(r *raft, m pb.Message) error { if !r.trk.QuorumActive() { r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id) r.becomeFollower(r.Term, None) + } else { + r.maybeStartNewSubterm(false, false) } // Mark everyone (but ourselves) as inactive in preparation for the next // CheckQuorum. @@ -1599,11 +1741,16 @@ func stepLeader(r *raft, m pb.Message) error { pr.BecomeProbe() } r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr) + case pb.MsgTransferLeader: if pr.IsLearner { r.logger.Debugf("%x is learner. Ignored transferring leadership", r.id) return nil } + if pr.IsWitness { + r.logger.Debugf("%x is witness. Ignored transferring leadership", r.id) + return nil + } leadTransferee := m.From lastLeadTransferee := r.leadTransferee if lastLeadTransferee != None { @@ -1660,8 +1807,8 @@ func stepCandidate(r *raft, m pb.Message) error { r.becomeFollower(m.Term, m.From) // always m.Term == r.Term r.handleSnapshot(m) case myVoteRespType: - gr, rj, res := r.poll(m.From, m.Type, !m.Reject) - r.logger.Infof("%x has received %d %s votes and %d vote rejections", r.id, gr, m.Type, rj) + gr, rj, res, rv := r.pollAndReportDiff(m.From, m.Type, !m.Reject) + r.logger.Infof("%x has received %d %s votes and %d vote rejections. It needs %d votes to win", r.id, gr, m.Type, rj, rv) switch res { case quorum.VoteWon: if r.state == StatePreCandidate { @@ -1674,6 +1821,24 @@ func stepCandidate(r *raft, m pb.Message) error { // pb.MsgPreVoteResp contains future term of pre-candidate // m.Term > r.Term; reuse r.Term r.becomeFollower(r.Term, None) + + case quorum.VotePending: + witnessReadyToVote := r.getWitnessVoteRequestReadiness(rv) + for witnessID, ready := range witnessReadyToVote { + if ready { + // try to get vote from witness + var myVoteType pb.MessageType + var term uint64 + if r.state == StatePreCandidate { + myVoteType = pb.MsgPreVote + term = r.Term + 1 + } else { + myVoteType = pb.MsgVote + term = r.Term + } + r.sendRequestVoteToWitness(nil, witnessID, myVoteType, term) + } + } } case pb.MsgTimeoutNow: r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From) @@ -1915,10 +2080,10 @@ func (r *raft) restore(s pb.Snapshot) bool { // which is true when its own id is in progress list. func (r *raft) promotable() bool { pr := r.trk.Progress[r.id] - return pr != nil && !pr.IsLearner && !r.raftLog.hasNextOrInProgressSnapshot() + return pr != nil && !pr.IsLearner && !pr.IsWitness && !r.raftLog.hasNextOrInProgressSnapshot() } -func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState { +func (r *raft) applyConfChange(cc pb.ConfChangeV2) (*pb.ConfState, error) { cfg, trk, err := func() (tracker.Config, tracker.ProgressMap, error) { changer := confchange.Changer{ Tracker: r.trk, @@ -1933,11 +2098,12 @@ func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState { }() if err != nil { - // TODO(tbg): return the error to the caller. - panic(err) + return nil, err } - return r.switchToConfig(cfg, trk) + c := r.switchToConfig(cfg, trk) + + return &c, nil } // switchToConfig reconfigures this node to use the provided configuration. It @@ -1950,6 +2116,10 @@ func (r *raft) switchToConfig(cfg tracker.Config, trk tracker.ProgressMap) pb.Co r.trk.Config = cfg r.trk.Progress = trk + if r.state == StateLeader { + r.maybeStartNewSubterm(false, true) + } + r.logger.Infof("%x switched to configuration %s", r.id, r.trk.Config) cs := r.trk.ConfState() pr, ok := r.trk.Progress[r.id] @@ -2092,6 +2262,38 @@ func (r *raft) reduceUncommittedSize(s entryPayloadSize) { } } +func (r *raft) maybeStartNewSubterm(newTerm bool, confChange bool) bool { + if newTerm || confChange { + r.trk.ResetReplicationSet(newTerm) + r.logger.Infof("%x resets replication set to %s, %s", r.id, r.trk.Epoch.ReplicationSet[0].String(), r.trk.Epoch.ReplicationSet[1].String()) + } else { + if r.trk.ChangeReplicationSet() { + r.logger.Infof("%x changes replication set to %s, %s", r.id, r.trk.Epoch.ReplicationSet[0].String(), r.trk.Epoch.ReplicationSet[1].String()) + } else { + return false + } + } + + emptyEnt := pb.Entry{Data: nil} + if !r.appendEntry(emptyEnt) { + // This won't happen because we just called reset() above. + r.logger.Panic("empty entry was dropped") + } + + r.logger.Infof("%x starts new subterm. Term: %d, Subterm: %d", r.id, r.Term, r.trk.Epoch.Subterm) + return true +} + +func numOfPendingConf(ents []pb.Entry) int { + n := 0 + for i := range ents { + if ents[i].Type == pb.EntryConfChange || ents[i].Type == pb.EntryConfChangeV2 { + n++ + } + } + return n +} + func releasePendingReadIndexMessages(r *raft) { if len(r.pendingReadIndexMessages) == 0 { // Fast path for the common case to avoid a call to storage.LastIndex() diff --git a/raft_test.go b/raft_test.go index 5a258e56..6fb66486 100644 --- a/raft_test.go +++ b/raft_test.go @@ -429,6 +429,9 @@ func TestLearnerPromotion(t *testing.T) { nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat}) n1.applyConfChange(pb.ConfChange{NodeID: 2, Type: pb.ConfChangeAddNode}.AsV2()) + n1.advanceMessagesAfterAppend() // empty entry will be added in leader log after config change + nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat}) + n2.applyConfChange(pb.ConfChange{NodeID: 2, Type: pb.ConfChangeAddNode}.AsV2()) if n2.isLearner { t.Error("peer 2 is learner, want not") @@ -3572,8 +3575,8 @@ func TestCommitAfterRemoveNode(t *testing.T) { // pending command can now commit. r.applyConfChange(cc.AsV2()) ents = nextEnts(r, s) - if len(ents) != 1 || ents[0].Type != pb.EntryNormal || - string(ents[0].Data) != "hello" { + if len(ents) != 2 || ents[0].Type != pb.EntryNormal || ents[1].Type != pb.EntryNormal || + string(ents[0].Data) != "hello" || len(ents[1].Data) != 0 { t.Fatalf("expected one committed EntryNormal, got %v", ents) } } diff --git a/raftpb/confchange.go b/raftpb/confchange.go index a3ddff62..681b60cb 100644 --- a/raftpb/confchange.go +++ b/raftpb/confchange.go @@ -138,6 +138,8 @@ func ConfChangesFromString(s string) ([]ConfChangeSingle, error) { cc.Type = ConfChangeRemoveNode case 'u': cc.Type = ConfChangeUpdateNode + case 'w': + cc.Type = ConfChangeAddWitness default: return nil, fmt.Errorf("unknown input: %s", tok) } @@ -167,6 +169,8 @@ func ConfChangesToString(ccs []ConfChangeSingle) string { buf.WriteByte('r') case ConfChangeUpdateNode: buf.WriteByte('u') + case ConfChangeAddWitness: + buf.WriteByte('w') default: buf.WriteString("unknown") } diff --git a/raftpb/raft.pb.go b/raftpb/raft.pb.go index 7dcdef0c..3d71b948 100644 --- a/raftpb/raft.pb.go +++ b/raftpb/raft.pb.go @@ -239,6 +239,7 @@ const ( ConfChangeRemoveNode ConfChangeType = 1 ConfChangeUpdateNode ConfChangeType = 2 ConfChangeAddLearnerNode ConfChangeType = 3 + ConfChangeAddWitness ConfChangeType = 4 ) var ConfChangeType_name = map[int32]string{ @@ -246,6 +247,7 @@ var ConfChangeType_name = map[int32]string{ 1: "ConfChangeRemoveNode", 2: "ConfChangeUpdateNode", 3: "ConfChangeAddLearnerNode", + 4: "ConfChangeAddWitness", } var ConfChangeType_value = map[string]int32{ @@ -253,6 +255,7 @@ var ConfChangeType_value = map[string]int32{ "ConfChangeRemoveNode": 1, "ConfChangeUpdateNode": 2, "ConfChangeAddLearnerNode": 3, + "ConfChangeAddWitness": 4, } func (x ConfChangeType) Enum() *ConfChangeType { @@ -279,10 +282,11 @@ func (ConfChangeType) EnumDescriptor() ([]byte, []int) { } type Entry struct { - Term uint64 `protobuf:"varint,2,opt,name=Term" json:"Term"` - Index uint64 `protobuf:"varint,3,opt,name=Index" json:"Index"` - Type EntryType `protobuf:"varint,1,opt,name=Type,enum=raftpb.EntryType" json:"Type"` - Data []byte `protobuf:"bytes,4,opt,name=Data" json:"Data,omitempty"` + Term uint64 `protobuf:"varint,2,opt,name=Term" json:"Term"` + Index uint64 `protobuf:"varint,3,opt,name=Index" json:"Index"` + Type EntryType `protobuf:"varint,1,opt,name=Type,enum=raftpb.EntryType" json:"Type"` + Data []byte `protobuf:"bytes,4,opt,name=Data" json:"Data,omitempty"` + Subterm uint64 `protobuf:"varint,5,opt,name=Subterm" json:"Subterm"` } func (m *Entry) Reset() { *m = Entry{} } @@ -506,6 +510,49 @@ func (m *HardState) XXX_DiscardUnknown() { var xxx_messageInfo_HardState proto.InternalMessageInfo +type WitnessHardState struct { + State HardState `protobuf:"bytes,1,opt,name=state" json:"state"` + LastLogIndex uint64 `protobuf:"varint,2,opt,name=lastLogIndex" json:"lastLogIndex"` + LastLogTerm uint64 `protobuf:"varint,3,opt,name=lastLogTerm" json:"lastLogTerm"` + LastLogSubterm uint64 `protobuf:"varint,4,opt,name=lastLogSubterm" json:"lastLogSubterm"` + Lead uint64 `protobuf:"varint,5,opt,name=lead" json:"lead"` + ReplicationSet []uint64 `protobuf:"varint,6,rep,name=replicationSet" json:"replicationSet,omitempty"` + ReplicationSetOutgoing []uint64 `protobuf:"varint,7,rep,name=replicationSetOutgoing" json:"replicationSetOutgoing,omitempty"` +} + +func (m *WitnessHardState) Reset() { *m = WitnessHardState{} } +func (m *WitnessHardState) String() string { return proto.CompactTextString(m) } +func (*WitnessHardState) ProtoMessage() {} +func (*WitnessHardState) Descriptor() ([]byte, []int) { + return fileDescriptor_b042552c306ae59b, []int{5} +} +func (m *WitnessHardState) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WitnessHardState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_WitnessHardState.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *WitnessHardState) XXX_Merge(src proto.Message) { + xxx_messageInfo_WitnessHardState.Merge(m, src) +} +func (m *WitnessHardState) XXX_Size() int { + return m.Size() +} +func (m *WitnessHardState) XXX_DiscardUnknown() { + xxx_messageInfo_WitnessHardState.DiscardUnknown(m) +} + +var xxx_messageInfo_WitnessHardState proto.InternalMessageInfo + type ConfState struct { // The voters in the incoming config. (If the configuration is not joint, // then the outgoing config is empty). @@ -520,14 +567,16 @@ type ConfState struct { LearnersNext []uint64 `protobuf:"varint,4,rep,name=learners_next,json=learnersNext" json:"learners_next,omitempty"` // If set, the config is joint and Raft will automatically transition into // the final config (i.e. remove the outgoing config) when this is safe. - AutoLeave bool `protobuf:"varint,5,opt,name=auto_leave,json=autoLeave" json:"auto_leave"` + AutoLeave bool `protobuf:"varint,5,opt,name=auto_leave,json=autoLeave" json:"auto_leave"` + Witness uint64 `protobuf:"varint,6,opt,name=witness" json:"witness"` + WitnessOutgoing uint64 `protobuf:"varint,7,opt,name=witness_outgoing,json=witnessOutgoing" json:"witness_outgoing"` } func (m *ConfState) Reset() { *m = ConfState{} } func (m *ConfState) String() string { return proto.CompactTextString(m) } func (*ConfState) ProtoMessage() {} func (*ConfState) Descriptor() ([]byte, []int) { - return fileDescriptor_b042552c306ae59b, []int{5} + return fileDescriptor_b042552c306ae59b, []int{6} } func (m *ConfState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -570,7 +619,7 @@ func (m *ConfChange) Reset() { *m = ConfChange{} } func (m *ConfChange) String() string { return proto.CompactTextString(m) } func (*ConfChange) ProtoMessage() {} func (*ConfChange) Descriptor() ([]byte, []int) { - return fileDescriptor_b042552c306ae59b, []int{6} + return fileDescriptor_b042552c306ae59b, []int{7} } func (m *ConfChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -610,7 +659,7 @@ func (m *ConfChangeSingle) Reset() { *m = ConfChangeSingle{} } func (m *ConfChangeSingle) String() string { return proto.CompactTextString(m) } func (*ConfChangeSingle) ProtoMessage() {} func (*ConfChangeSingle) Descriptor() ([]byte, []int) { - return fileDescriptor_b042552c306ae59b, []int{7} + return fileDescriptor_b042552c306ae59b, []int{8} } func (m *ConfChangeSingle) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -681,7 +730,7 @@ func (m *ConfChangeV2) Reset() { *m = ConfChangeV2{} } func (m *ConfChangeV2) String() string { return proto.CompactTextString(m) } func (*ConfChangeV2) ProtoMessage() {} func (*ConfChangeV2) Descriptor() ([]byte, []int) { - return fileDescriptor_b042552c306ae59b, []int{8} + return fileDescriptor_b042552c306ae59b, []int{9} } func (m *ConfChangeV2) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,6 +769,7 @@ func init() { proto.RegisterType((*Snapshot)(nil), "raftpb.Snapshot") proto.RegisterType((*Message)(nil), "raftpb.Message") proto.RegisterType((*HardState)(nil), "raftpb.HardState") + proto.RegisterType((*WitnessHardState)(nil), "raftpb.WitnessHardState") proto.RegisterType((*ConfState)(nil), "raftpb.ConfState") proto.RegisterType((*ConfChange)(nil), "raftpb.ConfChange") proto.RegisterType((*ConfChangeSingle)(nil), "raftpb.ConfChangeSingle") @@ -729,76 +779,85 @@ func init() { func init() { proto.RegisterFile("raft.proto", fileDescriptor_b042552c306ae59b) } var fileDescriptor_b042552c306ae59b = []byte{ - // 1102 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcb, 0x6e, 0x23, 0x45, - 0x14, 0xed, 0x6e, 0x77, 0xfc, 0xb8, 0x76, 0x9c, 0x4a, 0xc5, 0x33, 0xd3, 0x8a, 0x22, 0x8f, 0xf1, - 0x0c, 0x1a, 0x2b, 0x68, 0x02, 0x32, 0x12, 0x42, 0xec, 0xf2, 0x18, 0x94, 0xa0, 0x38, 0x0c, 0x4e, - 0x26, 0x0b, 0x24, 0x14, 0x55, 0xdc, 0x95, 0x4e, 0x83, 0x5d, 0xd5, 0xaa, 0x2e, 0x87, 0x64, 0x83, - 0x10, 0x5f, 0xc0, 0x92, 0x0d, 0x5b, 0x3e, 0x80, 0x8f, 0x40, 0x59, 0x66, 0xc9, 0x6a, 0xc4, 0x24, - 0x7f, 0xc0, 0x17, 0xa0, 0xaa, 0xae, 0x7e, 0xd8, 0x89, 0x66, 0xc1, 0xae, 0xea, 0xdc, 0x53, 0xf7, - 0x9e, 0x7b, 0x6e, 0x57, 0x35, 0x80, 0x20, 0x67, 0x72, 0x23, 0x12, 0x5c, 0x72, 0x5c, 0x56, 0xeb, - 0xe8, 0x74, 0xb5, 0x15, 0xf0, 0x80, 0x6b, 0xe8, 0x63, 0xb5, 0x4a, 0xa2, 0xdd, 0x9f, 0x60, 0xe1, - 0x15, 0x93, 0xe2, 0x0a, 0x7b, 0xe0, 0x1e, 0x51, 0x31, 0xf1, 0x9c, 0x8e, 0xdd, 0x73, 0xb7, 0xdc, - 0xeb, 0xb7, 0x4f, 0xad, 0xa1, 0x46, 0xf0, 0x2a, 0x2c, 0xec, 0x31, 0x9f, 0x5e, 0x7a, 0xa5, 0x42, - 0x28, 0x81, 0xf0, 0x47, 0xe0, 0x1e, 0x5d, 0x45, 0xd4, 0xb3, 0x3b, 0x76, 0xaf, 0xd9, 0x5f, 0xde, - 0x48, 0x6a, 0x6d, 0xe8, 0x94, 0x2a, 0x90, 0x25, 0xba, 0x8a, 0x28, 0xc6, 0xe0, 0xee, 0x10, 0x49, - 0x3c, 0xb7, 0x63, 0xf7, 0x1a, 0x43, 0xbd, 0xee, 0xfe, 0x6c, 0x03, 0x3a, 0x64, 0x24, 0x8a, 0xcf, - 0xb9, 0x1c, 0x50, 0x49, 0x7c, 0x22, 0x09, 0xfe, 0x0c, 0x60, 0xc4, 0xd9, 0xd9, 0x49, 0x2c, 0x89, - 0x4c, 0x72, 0xd7, 0xf3, 0xdc, 0xdb, 0x9c, 0x9d, 0x1d, 0xaa, 0x80, 0xc9, 0x5d, 0x1b, 0xa5, 0x80, - 0x52, 0x1a, 0x6a, 0xa5, 0xc5, 0x26, 0x12, 0x48, 0xf5, 0x27, 0x55, 0x7f, 0xc5, 0x26, 0x34, 0xd2, - 0xfd, 0x16, 0xaa, 0xa9, 0x02, 0x25, 0x51, 0x29, 0xd0, 0x35, 0x1b, 0x43, 0xbd, 0xc6, 0x5f, 0x40, - 0x75, 0x62, 0x94, 0xe9, 0xc4, 0xf5, 0xbe, 0x97, 0x6a, 0x99, 0x57, 0x6e, 0xf2, 0x66, 0xfc, 0xee, - 0xbf, 0x25, 0xa8, 0x0c, 0x68, 0x1c, 0x93, 0x80, 0xe2, 0x97, 0xe0, 0xca, 0xdc, 0xab, 0x95, 0x34, - 0x87, 0x09, 0x17, 0xdd, 0x52, 0x34, 0xdc, 0x02, 0x47, 0xf2, 0x99, 0x4e, 0x1c, 0xc9, 0x55, 0x1b, - 0x67, 0x82, 0xcf, 0xb5, 0xa1, 0x90, 0xac, 0x41, 0x77, 0xbe, 0x41, 0xdc, 0x86, 0xca, 0x98, 0x07, - 0x7a, 0xba, 0x0b, 0x85, 0x60, 0x0a, 0xe6, 0xb6, 0x95, 0xef, 0xdb, 0xf6, 0x12, 0x2a, 0x94, 0x49, - 0x11, 0xd2, 0xd8, 0xab, 0x74, 0x4a, 0xbd, 0x7a, 0x7f, 0x71, 0x66, 0xc6, 0x69, 0x2a, 0xc3, 0xc1, - 0x6b, 0x50, 0x1e, 0xf1, 0xc9, 0x24, 0x94, 0x5e, 0xb5, 0x90, 0xcb, 0x60, 0x4a, 0xe2, 0x05, 0x97, - 0xd4, 0x5b, 0x2c, 0x4a, 0x54, 0x08, 0xee, 0x43, 0x35, 0x36, 0x5e, 0x7a, 0x35, 0xed, 0x31, 0x9a, - 0xf7, 0x58, 0xf3, 0xed, 0x61, 0xc6, 0x53, 0xb5, 0x04, 0xfd, 0x9e, 0x8e, 0xa4, 0x07, 0x1d, 0xbb, - 0x57, 0x4d, 0x6b, 0x25, 0x18, 0x7e, 0x0e, 0x90, 0xac, 0x76, 0x43, 0x26, 0xbd, 0x7a, 0xa1, 0x62, - 0x01, 0x57, 0xd6, 0x8c, 0x38, 0x93, 0xf4, 0x52, 0x7a, 0x0d, 0x35, 0x72, 0x53, 0x24, 0x05, 0xf1, - 0xa7, 0x50, 0x13, 0x34, 0x8e, 0x38, 0x8b, 0x69, 0xec, 0x35, 0xb5, 0x01, 0x4b, 0x73, 0x83, 0x4b, - 0x3f, 0xc3, 0x8c, 0xd7, 0xfd, 0x0e, 0x6a, 0xbb, 0x44, 0xf8, 0xc9, 0x37, 0x99, 0x8e, 0xc5, 0xbe, - 0x37, 0x96, 0xd4, 0x0d, 0xe7, 0x9e, 0x1b, 0xb9, 0x8b, 0xa5, 0xfb, 0x2e, 0x76, 0xff, 0xb4, 0xa1, - 0x96, 0x5d, 0x02, 0xfc, 0x18, 0xca, 0xea, 0x8c, 0x88, 0x3d, 0xbb, 0x53, 0xea, 0xb9, 0x43, 0xb3, - 0xc3, 0xab, 0x50, 0x1d, 0x53, 0x22, 0x98, 0x8a, 0x38, 0x3a, 0x92, 0xed, 0xf1, 0x0b, 0x58, 0x4a, - 0x58, 0x27, 0x7c, 0x2a, 0x03, 0x1e, 0xb2, 0xc0, 0x2b, 0x69, 0x4a, 0x33, 0x81, 0xbf, 0x36, 0x28, - 0x7e, 0x06, 0x8b, 0xe9, 0xa1, 0x13, 0xa6, 0x4c, 0x72, 0x35, 0xad, 0x91, 0x82, 0x07, 0xca, 0xa3, - 0x67, 0x00, 0x64, 0x2a, 0xf9, 0xc9, 0x98, 0x92, 0x0b, 0xaa, 0xbf, 0xb0, 0x74, 0x16, 0x35, 0x85, - 0xef, 0x2b, 0xb8, 0xfb, 0xbb, 0x0d, 0xa0, 0x44, 0x6f, 0x9f, 0x13, 0x16, 0x50, 0xfc, 0x89, 0xb9, - 0x0b, 0x8e, 0xbe, 0x0b, 0x8f, 0x8b, 0x77, 0x3b, 0x61, 0xdc, 0xbb, 0x0e, 0x2f, 0xa0, 0xc2, 0xb8, - 0x4f, 0x4f, 0x42, 0xdf, 0x98, 0xd2, 0x54, 0xc1, 0xdb, 0xb7, 0x4f, 0xcb, 0x07, 0xdc, 0xa7, 0x7b, - 0x3b, 0xc3, 0xb2, 0x0a, 0xef, 0xf9, 0xd8, 0xcb, 0x47, 0x9a, 0x3c, 0x34, 0xd9, 0x30, 0x57, 0xc1, - 0x09, 0x7d, 0x33, 0x08, 0x30, 0xa7, 0x9d, 0xbd, 0x9d, 0xa1, 0x13, 0xfa, 0xdd, 0x09, 0xa0, 0xbc, - 0xf8, 0x61, 0xc8, 0x82, 0x71, 0x2e, 0xd2, 0xfe, 0x3f, 0x22, 0x9d, 0xf7, 0x89, 0xec, 0xfe, 0x61, - 0x43, 0x23, 0xcf, 0x73, 0xdc, 0xc7, 0x5b, 0x00, 0x52, 0x10, 0x16, 0x87, 0x32, 0xe4, 0xcc, 0x54, - 0x5c, 0x7b, 0xa0, 0x62, 0xc6, 0x49, 0x3f, 0xe6, 0xfc, 0x14, 0xfe, 0x1c, 0x2a, 0x23, 0xcd, 0x4a, - 0x26, 0x5e, 0x78, 0xa7, 0xe6, 0x5b, 0x4b, 0xaf, 0xad, 0xa1, 0x17, 0x3d, 0x2b, 0xcd, 0x78, 0xb6, - 0xbe, 0x0b, 0xb5, 0xec, 0x31, 0xc7, 0x4b, 0x50, 0xd7, 0x9b, 0x03, 0x2e, 0x26, 0x64, 0x8c, 0x2c, - 0xbc, 0x02, 0x4b, 0x1a, 0xc8, 0xf3, 0x23, 0x1b, 0x3f, 0x82, 0xe5, 0x39, 0xf0, 0xb8, 0x8f, 0x9c, - 0xf5, 0xbf, 0x4a, 0x50, 0x2f, 0xbc, 0x75, 0x18, 0xa0, 0x3c, 0x88, 0x83, 0xdd, 0x69, 0x84, 0x2c, - 0x5c, 0x87, 0xca, 0x20, 0x0e, 0xb6, 0x28, 0x91, 0xc8, 0x36, 0x9b, 0xd7, 0x82, 0x47, 0xc8, 0x31, - 0xac, 0xcd, 0x28, 0x42, 0x25, 0xdc, 0x04, 0x48, 0xd6, 0x43, 0x1a, 0x47, 0xc8, 0x35, 0xc4, 0x63, - 0x2e, 0x29, 0x5a, 0x50, 0xda, 0xcc, 0x46, 0x47, 0xcb, 0x26, 0xaa, 0x5e, 0x0f, 0x54, 0xc1, 0x08, - 0x1a, 0xaa, 0x18, 0x25, 0x42, 0x9e, 0xaa, 0x2a, 0x55, 0xdc, 0x02, 0x54, 0x44, 0xf4, 0xa1, 0x1a, - 0xc6, 0xd0, 0x1c, 0xc4, 0xc1, 0x1b, 0x26, 0x28, 0x19, 0x9d, 0x93, 0xd3, 0x31, 0x45, 0x80, 0x97, - 0x61, 0xd1, 0x24, 0x52, 0x37, 0x6e, 0x1a, 0xa3, 0xba, 0xa1, 0x6d, 0x9f, 0xd3, 0xd1, 0x0f, 0xdf, - 0x4c, 0xb9, 0x98, 0x4e, 0x50, 0x43, 0xb5, 0x3d, 0x88, 0x03, 0x3d, 0xa0, 0x33, 0x2a, 0xf6, 0x29, - 0xf1, 0xa9, 0x40, 0x8b, 0xe6, 0xf4, 0x51, 0x38, 0xa1, 0x7c, 0x2a, 0x0f, 0xf8, 0x8f, 0xa8, 0x69, - 0xc4, 0x0c, 0x29, 0xf1, 0xf5, 0x4f, 0x14, 0x2d, 0x19, 0x31, 0x19, 0xa2, 0xc5, 0x20, 0xd3, 0xef, - 0x6b, 0x41, 0x75, 0x8b, 0xcb, 0xa6, 0xaa, 0xd9, 0x6b, 0x0e, 0x36, 0x27, 0x0f, 0x25, 0x17, 0x24, - 0xa0, 0x9b, 0x51, 0x44, 0x99, 0x8f, 0x56, 0xb0, 0x07, 0xad, 0x79, 0x54, 0xf3, 0x5b, 0x6a, 0x62, - 0x33, 0x91, 0xf1, 0x15, 0x7a, 0x84, 0x9f, 0xc0, 0xca, 0x1c, 0xa8, 0xd9, 0x8f, 0x0d, 0xfb, 0x4b, - 0x2e, 0x02, 0x2a, 0x4d, 0x47, 0x4f, 0xd6, 0x7f, 0xb1, 0xa1, 0xf5, 0xd0, 0x17, 0x89, 0xd7, 0xc0, - 0x7b, 0x08, 0xdf, 0x9c, 0x4a, 0x8e, 0x2c, 0xfc, 0x21, 0x7c, 0xf0, 0x50, 0xf4, 0x2b, 0x1e, 0x32, - 0xb9, 0x37, 0x89, 0xc6, 0xe1, 0x28, 0x54, 0xd3, 0x7f, 0x1f, 0xed, 0xd5, 0xa5, 0xa1, 0x39, 0xeb, - 0x57, 0xd0, 0x9c, 0xbd, 0x87, 0xca, 0xff, 0x1c, 0xd9, 0xf4, 0x7d, 0x75, 0xe3, 0x90, 0xa5, 0xac, - 0xc8, 0xe1, 0x21, 0x9d, 0xf0, 0x0b, 0xaa, 0x23, 0xf6, 0x6c, 0xe4, 0x4d, 0xe4, 0x13, 0x99, 0x44, - 0x9c, 0xd9, 0x46, 0x36, 0x7d, 0x7f, 0x3f, 0x79, 0xee, 0x74, 0xb4, 0xb4, 0xf5, 0xfc, 0xfa, 0x5d, - 0xdb, 0xba, 0x79, 0xd7, 0xb6, 0xae, 0x6f, 0xdb, 0xf6, 0xcd, 0x6d, 0xdb, 0xfe, 0xe7, 0xb6, 0x6d, - 0xff, 0x7a, 0xd7, 0xb6, 0x7e, 0xbb, 0x6b, 0x5b, 0x37, 0x77, 0x6d, 0xeb, 0xef, 0xbb, 0xb6, 0xf5, - 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x67, 0x2b, 0x47, 0x0c, 0x83, 0x09, 0x00, 0x00, + // 1239 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0x4d, 0x6f, 0x1b, 0x45, + 0x18, 0xf6, 0xae, 0x37, 0xfe, 0x78, 0xed, 0xd8, 0x93, 0x89, 0x9b, 0xae, 0xa2, 0xca, 0x35, 0x6e, + 0x69, 0xad, 0x40, 0x5b, 0x64, 0xa4, 0x0a, 0x71, 0x73, 0xda, 0xa2, 0x18, 0xc5, 0xa1, 0xd8, 0x69, + 0x90, 0x90, 0x50, 0x34, 0xf1, 0x4e, 0x36, 0x0b, 0xf6, 0xce, 0x6a, 0x77, 0x9c, 0x26, 0x37, 0x04, + 0x7f, 0x00, 0x89, 0x0b, 0x17, 0x38, 0xf2, 0x27, 0xb8, 0xa3, 0x1c, 0x73, 0xe4, 0x54, 0xd1, 0xe4, + 0x1f, 0xf0, 0x0b, 0xd0, 0xcc, 0xce, 0x7e, 0x3a, 0xf4, 0xc0, 0x6d, 0xe6, 0x79, 0x9e, 0x79, 0xbf, + 0x67, 0x06, 0xc0, 0x27, 0xc7, 0xfc, 0xb1, 0xe7, 0x33, 0xce, 0x70, 0x49, 0xac, 0xbd, 0xa3, 0xcd, + 0x96, 0xcd, 0x6c, 0x26, 0xa1, 0x27, 0x62, 0x15, 0xb2, 0xdd, 0xdf, 0x34, 0x58, 0x79, 0xe1, 0x72, + 0xff, 0x1c, 0x9b, 0x60, 0xec, 0x53, 0x7f, 0x6e, 0xea, 0x1d, 0xad, 0x67, 0x6c, 0x1b, 0x17, 0x6f, + 0xee, 0x16, 0xc6, 0x12, 0xc1, 0x9b, 0xb0, 0x32, 0x74, 0x2d, 0x7a, 0x66, 0x16, 0x53, 0x54, 0x08, + 0xe1, 0x0f, 0xc0, 0xd8, 0x3f, 0xf7, 0xa8, 0xa9, 0x75, 0xb4, 0x5e, 0xa3, 0xbf, 0xf6, 0x38, 0x74, + 0xf6, 0x58, 0x9a, 0x14, 0x44, 0x6c, 0xe8, 0xdc, 0xa3, 0x18, 0x83, 0xf1, 0x9c, 0x70, 0x62, 0x1a, + 0x1d, 0xad, 0x57, 0x1f, 0xcb, 0x35, 0x6e, 0x43, 0x79, 0xb2, 0x38, 0xe2, 0xc2, 0xf3, 0x4a, 0xca, + 0x7c, 0x04, 0x76, 0xbf, 0xd7, 0x00, 0x4d, 0x5c, 0xe2, 0x05, 0x27, 0x8c, 0x8f, 0x28, 0x27, 0x96, + 0x38, 0xf4, 0x14, 0x60, 0xca, 0xdc, 0xe3, 0xc3, 0x80, 0x13, 0x1e, 0xfa, 0xae, 0x25, 0xbe, 0x9f, + 0x31, 0xf7, 0x78, 0x22, 0x08, 0x65, 0xaa, 0x3a, 0x8d, 0x00, 0x91, 0x89, 0x23, 0x33, 0x49, 0x27, + 0x19, 0x42, 0x22, 0x7f, 0x19, 0x45, 0x3a, 0x49, 0x89, 0x74, 0xbf, 0x86, 0x4a, 0x14, 0x81, 0x48, + 0x41, 0x44, 0x20, 0x7d, 0xd6, 0xc7, 0x72, 0x8d, 0x3f, 0x85, 0xca, 0x5c, 0x45, 0x26, 0x0d, 0xd7, + 0xfa, 0x66, 0x14, 0x4b, 0x3e, 0x72, 0x65, 0x37, 0xd6, 0x77, 0xff, 0x29, 0x42, 0x79, 0x44, 0x83, + 0x80, 0xd8, 0x14, 0x3f, 0x02, 0x83, 0x27, 0xb5, 0x5c, 0x8f, 0x6c, 0x28, 0x3a, 0x5d, 0x4d, 0x21, + 0xc3, 0x2d, 0xd0, 0x39, 0xcb, 0x64, 0xa2, 0x73, 0x26, 0xd2, 0x38, 0xf6, 0x59, 0x2e, 0x0d, 0x81, + 0xc4, 0x09, 0x1a, 0xf9, 0x04, 0x45, 0x0f, 0x66, 0xcc, 0xde, 0x5f, 0xea, 0x81, 0x02, 0x93, 0xb2, + 0x95, 0x96, 0xcb, 0xf6, 0x08, 0xca, 0xd4, 0xe5, 0xbe, 0x43, 0x03, 0xb3, 0xdc, 0x29, 0xf6, 0x6a, + 0xfd, 0xd5, 0xcc, 0x0c, 0x44, 0xa6, 0x94, 0x06, 0xdf, 0x81, 0xd2, 0x94, 0xcd, 0xe7, 0x0e, 0x37, + 0x2b, 0x29, 0x5b, 0x0a, 0x13, 0x21, 0x9e, 0x32, 0x4e, 0xcd, 0xd5, 0x74, 0x88, 0x02, 0xc1, 0x7d, + 0xa8, 0x04, 0xaa, 0x96, 0x66, 0x55, 0xd6, 0x18, 0xe5, 0x6b, 0x2c, 0xf5, 0xda, 0x38, 0xd6, 0x09, + 0x5f, 0x3e, 0xfd, 0x96, 0x4e, 0xb9, 0x09, 0x1d, 0xad, 0x57, 0x89, 0x7c, 0x85, 0x18, 0xbe, 0x0f, + 0x10, 0xae, 0x76, 0x1c, 0x97, 0x9b, 0xb5, 0x94, 0xc7, 0x14, 0x2e, 0x4a, 0x33, 0x65, 0x2e, 0xa7, + 0x67, 0xdc, 0xac, 0x8b, 0x96, 0x2b, 0x27, 0x11, 0x88, 0x3f, 0x86, 0xaa, 0x4f, 0x03, 0x8f, 0xb9, + 0x01, 0x0d, 0xcc, 0x86, 0x2c, 0x40, 0x33, 0xd7, 0xb8, 0x68, 0x0c, 0x63, 0x5d, 0xf7, 0x1b, 0xa8, + 0xee, 0x10, 0xdf, 0x0a, 0x67, 0x32, 0x6a, 0x8b, 0xb6, 0xd4, 0x96, 0xa8, 0x1a, 0xfa, 0x52, 0x35, + 0x92, 0x2a, 0x16, 0x97, 0xab, 0xd8, 0xfd, 0x43, 0x07, 0xf4, 0x95, 0xc3, 0x5d, 0x1a, 0x04, 0x89, + 0x9b, 0x47, 0xb0, 0x72, 0xe3, 0x6d, 0x89, 0x15, 0x51, 0x5b, 0xa5, 0x0a, 0xf7, 0xa0, 0x3e, 0x23, + 0x01, 0xdf, 0x65, 0xf6, 0x70, 0xe9, 0xc2, 0x64, 0x18, 0xfc, 0x00, 0x6a, 0x6a, 0xbf, 0x9f, 0xbf, + 0x3e, 0x69, 0x02, 0x7f, 0x08, 0x0d, 0xb5, 0x8d, 0xee, 0x7b, 0x7a, 0x10, 0x73, 0x9c, 0xc8, 0x7d, + 0x46, 0x89, 0x95, 0x99, 0x47, 0x89, 0xe0, 0x07, 0xd0, 0xf0, 0xa9, 0x37, 0x73, 0xa6, 0x84, 0x3b, + 0xcc, 0x9d, 0x50, 0x6e, 0x96, 0x3a, 0xc5, 0x9e, 0x31, 0xce, 0xa1, 0xf8, 0x29, 0x6c, 0x64, 0x91, + 0x2f, 0x16, 0xdc, 0x66, 0x8e, 0x6b, 0xcb, 0x39, 0x35, 0xc6, 0xff, 0xc1, 0x76, 0x7f, 0xd4, 0xa1, + 0x1a, 0x3f, 0x21, 0x78, 0x03, 0x4a, 0xa2, 0xe2, 0x7e, 0x60, 0x6a, 0xf2, 0x94, 0xda, 0xe1, 0x4d, + 0xa8, 0xcc, 0x28, 0xf1, 0x5d, 0xc1, 0xe8, 0x92, 0x89, 0xf7, 0xf8, 0x21, 0x34, 0x43, 0xd5, 0x21, + 0x8b, 0x5c, 0x16, 0xc3, 0x10, 0x43, 0x38, 0x72, 0x85, 0xef, 0xc1, 0x6a, 0x74, 0xe8, 0xd0, 0x15, + 0x23, 0x66, 0x48, 0x59, 0x3d, 0x02, 0xf7, 0xc4, 0x84, 0xdd, 0x03, 0x20, 0x0b, 0xce, 0x0e, 0x67, + 0x94, 0x9c, 0x52, 0x59, 0x8f, 0x68, 0x92, 0xab, 0x02, 0xdf, 0x15, 0xb0, 0x18, 0xd3, 0xd7, 0x61, + 0xc7, 0x33, 0x77, 0x34, 0x02, 0xf1, 0x13, 0x40, 0x6a, 0x99, 0xc4, 0x54, 0x4e, 0x09, 0x9b, 0x8a, + 0x8d, 0xab, 0xf0, 0xab, 0x06, 0x20, 0xaa, 0xf0, 0xec, 0x84, 0xb8, 0x36, 0xc5, 0x1f, 0xa9, 0xa7, + 0x49, 0x97, 0x4f, 0xd3, 0x46, 0xfa, 0xa9, 0x0d, 0x15, 0x4b, 0xaf, 0xd3, 0x43, 0x28, 0xbb, 0xcc, + 0xa2, 0x87, 0x8e, 0xa5, 0x46, 0xa2, 0x21, 0xc8, 0xab, 0x37, 0x77, 0x4b, 0x7b, 0xcc, 0xa2, 0xc3, + 0xe7, 0xe3, 0x92, 0xa0, 0x87, 0x16, 0x36, 0x93, 0x1b, 0x16, 0xfe, 0x0b, 0xf1, 0xdd, 0xda, 0x04, + 0xdd, 0xb1, 0xd4, 0xbd, 0x00, 0x75, 0x5a, 0x1f, 0x3e, 0x1f, 0xeb, 0x8e, 0xd5, 0x9d, 0x03, 0x4a, + 0x9c, 0x4f, 0x1c, 0xd7, 0x9e, 0x25, 0x41, 0x6a, 0xff, 0x27, 0x48, 0xfd, 0x5d, 0x41, 0x76, 0x7f, + 0xd7, 0xa0, 0x9e, 0xd8, 0x39, 0xe8, 0xe3, 0x6d, 0x00, 0xee, 0x13, 0x37, 0x70, 0xc4, 0xf8, 0x28, + 0x8f, 0x77, 0x6e, 0xf0, 0x18, 0x6b, 0xa2, 0xb7, 0x25, 0x39, 0x85, 0x3f, 0x81, 0xf2, 0x54, 0xaa, + 0xc2, 0x11, 0x4a, 0x7d, 0x1b, 0xf9, 0xd4, 0xa2, 0x76, 0x2a, 0x79, 0xba, 0x66, 0xc5, 0x4c, 0xcd, + 0xb6, 0x76, 0xa0, 0x1a, 0xff, 0xbd, 0xb8, 0x09, 0x35, 0xb9, 0xd9, 0x63, 0xfe, 0x9c, 0xcc, 0x50, + 0x01, 0xaf, 0x43, 0x53, 0x02, 0x89, 0x7d, 0xa4, 0xe1, 0x5b, 0xb0, 0x96, 0x03, 0x0f, 0xfa, 0x48, + 0xdf, 0xfa, 0xb3, 0x08, 0xb5, 0xd4, 0xd7, 0x83, 0x01, 0x4a, 0xa3, 0xc0, 0xde, 0x59, 0x78, 0xa8, + 0x80, 0x6b, 0x50, 0x1e, 0x05, 0xf6, 0x36, 0x25, 0x1c, 0x69, 0x6a, 0xf3, 0xd2, 0x67, 0x1e, 0xd2, + 0x95, 0x6a, 0xe0, 0x79, 0xa8, 0x88, 0x1b, 0x00, 0xe1, 0x7a, 0x4c, 0x03, 0x0f, 0x19, 0x4a, 0x78, + 0xc0, 0x38, 0x45, 0x2b, 0x22, 0x36, 0xb5, 0x91, 0x6c, 0x49, 0xb1, 0xe2, 0x31, 0x47, 0x65, 0x8c, + 0xa0, 0x2e, 0x9c, 0x51, 0xe2, 0xf3, 0x23, 0xe1, 0xa5, 0x82, 0x5b, 0x80, 0xd2, 0x88, 0x3c, 0x54, + 0xc5, 0x18, 0x1a, 0xa3, 0xc0, 0x7e, 0xe5, 0xfa, 0x94, 0x4c, 0x4f, 0xc8, 0xd1, 0x8c, 0x22, 0xc0, + 0x6b, 0xb0, 0xaa, 0x0c, 0x89, 0x2b, 0xbc, 0x08, 0x50, 0x4d, 0xc9, 0x9e, 0x9d, 0xd0, 0xe9, 0x77, + 0x5f, 0x2e, 0x98, 0xbf, 0x98, 0xa3, 0xba, 0x48, 0x7b, 0x14, 0xd8, 0xb2, 0x41, 0xc7, 0xd4, 0xdf, + 0xa5, 0xc4, 0xa2, 0x3e, 0x5a, 0x55, 0xa7, 0xf7, 0x9d, 0x39, 0x65, 0x0b, 0xbe, 0xc7, 0x5e, 0xa3, + 0x86, 0x0a, 0x66, 0x4c, 0x89, 0x25, 0x5f, 0x3c, 0xd4, 0x54, 0xc1, 0xc4, 0x88, 0x0c, 0x06, 0xa9, + 0x7c, 0x5f, 0xfa, 0x54, 0xa6, 0xb8, 0xa6, 0xbc, 0xaa, 0xbd, 0xd4, 0x60, 0x75, 0x72, 0xc2, 0x99, + 0x4f, 0x6c, 0x3a, 0xf0, 0x3c, 0xea, 0x5a, 0x68, 0x1d, 0x9b, 0xd0, 0xca, 0xa3, 0x52, 0xdf, 0x12, + 0x1d, 0xcb, 0x30, 0xb3, 0x73, 0x74, 0x0b, 0xdf, 0x86, 0xf5, 0x1c, 0x28, 0xd5, 0x1b, 0x4a, 0xfd, + 0x19, 0xf3, 0x6d, 0xca, 0x55, 0x46, 0xb7, 0xb7, 0x7e, 0xd0, 0xa0, 0x75, 0xd3, 0x44, 0xe2, 0x3b, + 0x60, 0xde, 0x84, 0x0f, 0x16, 0x9c, 0xa1, 0x02, 0x7e, 0x1f, 0xde, 0xbb, 0x89, 0xfd, 0x9c, 0x39, + 0x2e, 0x1f, 0xce, 0xc5, 0xd3, 0xe9, 0x88, 0xee, 0xbf, 0x4b, 0xf6, 0xe2, 0x4c, 0xc9, 0xf4, 0xad, + 0x9f, 0x35, 0x68, 0x64, 0x2f, 0xa2, 0x68, 0x40, 0x82, 0x0c, 0x2c, 0x4b, 0x5c, 0x39, 0x54, 0x10, + 0xb5, 0x48, 0xe0, 0x31, 0x9d, 0xb3, 0x53, 0x2a, 0x19, 0x2d, 0xcb, 0xbc, 0xf2, 0x2c, 0xc2, 0x43, + 0x46, 0xcf, 0x66, 0x32, 0xb0, 0xac, 0xdd, 0xf0, 0x01, 0x95, 0x6c, 0x31, 0x7b, 0x6e, 0x60, 0x59, + 0xea, 0x6f, 0x44, 0xc6, 0xf6, 0xfd, 0x8b, 0xb7, 0xed, 0xc2, 0xe5, 0xdb, 0x76, 0xe1, 0xe2, 0xaa, + 0xad, 0x5d, 0x5e, 0xb5, 0xb5, 0xbf, 0xaf, 0xda, 0xda, 0x4f, 0xd7, 0xed, 0xc2, 0x2f, 0xd7, 0xed, + 0xc2, 0xe5, 0x75, 0xbb, 0xf0, 0xd7, 0x75, 0xbb, 0xf0, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4b, + 0xf8, 0x64, 0xa3, 0x4e, 0x0b, 0x00, 0x00, } func (m *Entry) Marshal() (dAtA []byte, err error) { @@ -821,6 +880,9 @@ func (m *Entry) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + i = encodeVarintRaft(dAtA, i, uint64(m.Subterm)) + i-- + dAtA[i] = 0x28 if m.Data != nil { i -= len(m.Data) copy(dAtA[i:], m.Data) @@ -1056,6 +1118,65 @@ func (m *HardState) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *WitnessHardState) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *WitnessHardState) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *WitnessHardState) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.ReplicationSetOutgoing) > 0 { + for iNdEx := len(m.ReplicationSetOutgoing) - 1; iNdEx >= 0; iNdEx-- { + i = encodeVarintRaft(dAtA, i, uint64(m.ReplicationSetOutgoing[iNdEx])) + i-- + dAtA[i] = 0x38 + } + } + if len(m.ReplicationSet) > 0 { + for iNdEx := len(m.ReplicationSet) - 1; iNdEx >= 0; iNdEx-- { + i = encodeVarintRaft(dAtA, i, uint64(m.ReplicationSet[iNdEx])) + i-- + dAtA[i] = 0x30 + } + } + i = encodeVarintRaft(dAtA, i, uint64(m.Lead)) + i-- + dAtA[i] = 0x28 + i = encodeVarintRaft(dAtA, i, uint64(m.LastLogSubterm)) + i-- + dAtA[i] = 0x20 + i = encodeVarintRaft(dAtA, i, uint64(m.LastLogTerm)) + i-- + dAtA[i] = 0x18 + i = encodeVarintRaft(dAtA, i, uint64(m.LastLogIndex)) + i-- + dAtA[i] = 0x10 + { + size, err := m.State.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRaft(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + func (m *ConfState) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1076,6 +1197,12 @@ func (m *ConfState) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + i = encodeVarintRaft(dAtA, i, uint64(m.WitnessOutgoing)) + i-- + dAtA[i] = 0x38 + i = encodeVarintRaft(dAtA, i, uint64(m.Witness)) + i-- + dAtA[i] = 0x30 i-- if m.AutoLeave { dAtA[i] = 1 @@ -1254,6 +1381,7 @@ func (m *Entry) Size() (n int) { l = len(m.Data) n += 1 + l + sovRaft(uint64(l)) } + n += 1 + sovRaft(uint64(m.Subterm)) return n } @@ -1336,6 +1464,31 @@ func (m *HardState) Size() (n int) { return n } +func (m *WitnessHardState) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.State.Size() + n += 1 + l + sovRaft(uint64(l)) + n += 1 + sovRaft(uint64(m.LastLogIndex)) + n += 1 + sovRaft(uint64(m.LastLogTerm)) + n += 1 + sovRaft(uint64(m.LastLogSubterm)) + n += 1 + sovRaft(uint64(m.Lead)) + if len(m.ReplicationSet) > 0 { + for _, e := range m.ReplicationSet { + n += 1 + sovRaft(uint64(e)) + } + } + if len(m.ReplicationSetOutgoing) > 0 { + for _, e := range m.ReplicationSetOutgoing { + n += 1 + sovRaft(uint64(e)) + } + } + return n +} + func (m *ConfState) Size() (n int) { if m == nil { return 0 @@ -1363,6 +1516,8 @@ func (m *ConfState) Size() (n int) { } } n += 2 + n += 1 + sovRaft(uint64(m.Witness)) + n += 1 + sovRaft(uint64(m.WitnessOutgoing)) return n } @@ -1539,6 +1694,25 @@ func (m *Entry) Unmarshal(dAtA []byte) error { m.Data = []byte{} } iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Subterm", wireType) + } + m.Subterm = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Subterm |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) @@ -2284,6 +2458,317 @@ func (m *HardState) Unmarshal(dAtA []byte) error { } return nil } +func (m *WitnessHardState) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: WitnessHardState: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WitnessHardState: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRaft + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.State.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LastLogIndex", wireType) + } + m.LastLogIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LastLogIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LastLogTerm", wireType) + } + m.LastLogTerm = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LastLogTerm |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LastLogSubterm", wireType) + } + m.LastLogSubterm = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LastLogSubterm |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Lead", wireType) + } + m.Lead = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Lead |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ReplicationSet = append(m.ReplicationSet, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthRaft + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ReplicationSet) == 0 { + m.ReplicationSet = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ReplicationSet = append(m.ReplicationSet, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ReplicationSet", wireType) + } + case 7: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ReplicationSetOutgoing = append(m.ReplicationSetOutgoing, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthRaft + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ReplicationSetOutgoing) == 0 { + m.ReplicationSetOutgoing = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ReplicationSetOutgoing = append(m.ReplicationSetOutgoing, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ReplicationSetOutgoing", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipRaft(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRaft + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ConfState) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -2637,6 +3122,44 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { } } m.AutoLeave = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Witness", wireType) + } + m.Witness = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Witness |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WitnessOutgoing", wireType) + } + m.WitnessOutgoing = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.WitnessOutgoing |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) diff --git a/raftpb/raft.proto b/raftpb/raft.proto index a8598ee5..7968ffd5 100644 --- a/raftpb/raft.proto +++ b/raftpb/raft.proto @@ -23,6 +23,7 @@ message Entry { optional uint64 Index = 3 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations optional EntryType Type = 1 [(gogoproto.nullable) = false]; optional bytes Data = 4; + optional uint64 Subterm = 5 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations } message SnapshotMetadata { @@ -113,6 +114,16 @@ message HardState { optional uint64 commit = 3 [(gogoproto.nullable) = false]; } +message WitnessHardState { + optional HardState state = 1 [(gogoproto.nullable) = false]; + optional uint64 lastLogIndex = 2 [(gogoproto.nullable) = false]; + optional uint64 lastLogTerm = 3 [(gogoproto.nullable) = false]; + optional uint64 lastLogSubterm = 4 [(gogoproto.nullable) = false]; + optional uint64 lead = 5 [(gogoproto.nullable) = false]; + repeated uint64 replicationSet = 6; + repeated uint64 replicationSetOutgoing = 7; +} + // ConfChangeTransition specifies the behavior of a configuration change with // respect to joint consensus. enum ConfChangeTransition { @@ -148,6 +159,8 @@ message ConfState { // If set, the config is joint and Raft will automatically transition into // the final config (i.e. remove the outgoing config) when this is safe. optional bool auto_leave = 5 [(gogoproto.nullable) = false]; + optional uint64 witness = 6 [(gogoproto.nullable) = false]; + optional uint64 witness_outgoing = 7 [(gogoproto.nullable) = false]; } enum ConfChangeType { @@ -155,6 +168,7 @@ enum ConfChangeType { ConfChangeRemoveNode = 1; ConfChangeUpdateNode = 2; ConfChangeAddLearnerNode = 3; + ConfChangeAddWitness = 4; } message ConfChange { diff --git a/raftpb/raft_test.go b/raftpb/raft_test.go index 704ff94f..6a626092 100644 --- a/raftpb/raft_test.go +++ b/raftpb/raft_test.go @@ -36,13 +36,13 @@ func TestProtoMemorySizes(t *testing.T) { } var e Entry - assert(unsafe.Sizeof(e), if64Bit(48, 32), "Entry") + assert(unsafe.Sizeof(e), if64Bit(56, 40), "Entry") var sm SnapshotMetadata - assert(unsafe.Sizeof(sm), if64Bit(120, 68), "SnapshotMetadata") + assert(unsafe.Sizeof(sm), if64Bit(136, 84), "SnapshotMetadata") var s Snapshot - assert(unsafe.Sizeof(s), if64Bit(144, 80), "Snapshot") + assert(unsafe.Sizeof(s), if64Bit(160, 96), "Snapshot") var m Message assert(unsafe.Sizeof(m), if64Bit(160, 112), "Message") @@ -51,7 +51,7 @@ func TestProtoMemorySizes(t *testing.T) { assert(unsafe.Sizeof(hs), 24, "HardState") var cs ConfState - assert(unsafe.Sizeof(cs), if64Bit(104, 52), "ConfState") + assert(unsafe.Sizeof(cs), if64Bit(120, 68), "ConfState") var cc ConfChange assert(unsafe.Sizeof(cc), if64Bit(48, 32), "ConfChange") diff --git a/rafttest/interaction_env_handler_process_apply_thread.go b/rafttest/interaction_env_handler_process_apply_thread.go index d21317e0..8fd077a6 100644 --- a/rafttest/interaction_env_handler_process_apply_thread.go +++ b/rafttest/interaction_env_handler_process_apply_thread.go @@ -72,25 +72,30 @@ func processApply(n *Node, ents []raftpb.Entry) error { for _, ent := range ents { var update []byte var cs *raftpb.ConfState + var err error switch ent.Type { case raftpb.EntryConfChange: var cc raftpb.ConfChange - if err := cc.Unmarshal(ent.Data); err != nil { + if err = cc.Unmarshal(ent.Data); err != nil { return err } update = cc.Context - cs = n.RawNode.ApplyConfChange(cc) + cs, err = n.RawNode.ApplyConfChange(cc) case raftpb.EntryConfChangeV2: var cc raftpb.ConfChangeV2 - if err := cc.Unmarshal(ent.Data); err != nil { + if err = cc.Unmarshal(ent.Data); err != nil { return err } - cs = n.RawNode.ApplyConfChange(cc) + cs, err = n.RawNode.ApplyConfChange(cc) update = cc.Context default: update = ent.Data } + if err != nil { + return err + } + // Record the new state by starting with the current state and applying // the command. lastSnap := n.History[len(n.History)-1] diff --git a/rawnode.go b/rawnode.go index 428ef519..4a0c8e9b 100644 --- a/rawnode.go +++ b/rawnode.go @@ -109,9 +109,9 @@ func (rn *RawNode) ProposeConfChange(cc pb.ConfChangeI) error { // ApplyConfChange applies a config change to the local node. The app must call // this when it applies a configuration change, except when it decides to reject // the configuration change, in which case no call must take place. -func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState { - cs := rn.raft.applyConfChange(cc.AsV2()) - return &cs +func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) (*pb.ConfState, error) { + cs, err := rn.raft.applyConfChange(cc.AsV2()) + return cs, err } // Step advances the state machine using the given message. @@ -145,6 +145,7 @@ func (rn *RawNode) readyWithoutAccept() Ready { Entries: r.raftLog.nextUnstableEnts(), CommittedEntries: r.raftLog.nextCommittedEnts(rn.applyUnstableEntries()), Messages: r.msgs, + WitnessMessages: r.witnessMsgs, } if softSt := r.softState(); !softSt.equal(rn.prevSoftSt) { // Allocate only when SoftState changes. @@ -431,6 +432,7 @@ func (rn *RawNode) acceptReady(rd Ready) { } } rn.raft.msgs = nil + rn.raft.witnessMsgs = nil rn.raft.msgsAfterAppend = nil rn.raft.raftLog.acceptUnstable() if len(rd.CommittedEntries) > 0 { @@ -460,7 +462,7 @@ func (rn *RawNode) HasReady() bool { if r.raftLog.hasNextUnstableSnapshot() { return true } - if len(r.msgs) > 0 || len(r.msgsAfterAppend) > 0 { + if len(r.msgs) > 0 || len(r.msgsAfterAppend) > 0 || len(r.witnessMsgs) > 0 { return true } if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(rn.applyUnstableEntries()) { diff --git a/rawnode_test.go b/rawnode_test.go index bca5e64c..ba0f07b9 100644 --- a/rawnode_test.go +++ b/rawnode_test.go @@ -260,7 +260,10 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { cc = ccc } if cc != nil { - cs = rawNode.ApplyConfChange(cc) + cs, err = rawNode.ApplyConfChange(cc) // n.b. applying conf change will cause subterm change and appending an empty entry + if err != nil { + t.Fatal(err) + } } } rawNode.Advance(rd) @@ -321,17 +324,18 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs) } - var maybePlusOne uint64 + var maybePlusMore uint64 if autoLeave, ok := tc.cc.AsV2().EnterJoint(); ok && autoLeave { // If this is an auto-leaving joint conf change, it will have // appended the entry that auto-leaves, so add one to the last // index that forms the basis of our expectations on // pendingConfIndex. (Recall that lastIndex was taken from stable // storage, but this auto-leaving entry isn't on stable storage - // yet). - maybePlusOne = 1 + // yet). Also note that there is an empty entry in unstable which + // was added after applying conf change. + maybePlusMore = 2 } - if exp, act := lastIndex+maybePlusOne, rawNode.raft.pendingConfIndex; exp != act { + if exp, act := lastIndex+maybePlusMore, rawNode.raft.pendingConfIndex; exp != act { t.Fatalf("pendingConfIndex: expected %d, got %d", exp, act) } @@ -342,9 +346,10 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { rd := rawNode.Ready() var context []byte if !tc.exp.AutoLeave { - if len(rd.Entries) > 0 { - t.Fatal("expected no more entries") + if len(rd.Entries) != 1 || rd.Entries[0].Type != pb.EntryNormal || len(rd.Entries[0].Data) != 0 { + t.Fatal("expected no more entries other than an empty entry") } + s.Append(rd.Entries) rawNode.Advance(rd) if tc.exp2 == nil { return @@ -358,11 +363,11 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { } // Check that the right ConfChange comes out. - if len(rd.Entries) != 1 || rd.Entries[0].Type != pb.EntryConfChangeV2 { + if rd.Entries[len(rd.Entries)-1].Type != pb.EntryConfChangeV2 { t.Fatalf("expected exactly one more entry, got %+v", rd) } var cc pb.ConfChangeV2 - if err := cc.Unmarshal(rd.Entries[0].Data); err != nil { + if err := cc.Unmarshal(rd.Entries[len(rd.Entries)-1].Data); err != nil { t.Fatal(err) } if !reflect.DeepEqual(cc, pb.ConfChangeV2{Context: context}) { @@ -370,7 +375,10 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { } // Lie and pretend the ConfChange applied. It won't do so because now // we require the joint quorum and we're only running one node. - cs = rawNode.ApplyConfChange(cc) + cs, err = rawNode.ApplyConfChange(cc) + if err != nil { + t.Fatal(err) + } if exp := tc.exp2; !reflect.DeepEqual(exp, cs) { t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs) } @@ -423,7 +431,10 @@ func TestRawNodeJointAutoLeave(t *testing.T) { if cc != nil { // Force it step down. rawNode.Step(pb.Message{Type: pb.MsgHeartbeatResp, From: 1, Term: rawNode.raft.Term + 1}) - cs = rawNode.ApplyConfChange(cc) + cs, err = rawNode.ApplyConfChange(cc) + if err != nil { + t.Fatal(err) + } } } rawNode.Advance(rd) @@ -512,7 +523,10 @@ func TestRawNodeJointAutoLeave(t *testing.T) { } // Lie and pretend the ConfChange applied. It won't do so because now // we require the joint quorum and we're only running one node. - cs = rawNode.ApplyConfChange(cc) + cs, err = rawNode.ApplyConfChange(cc) + if err != nil { + t.Fatal(err) + } if exp := exp2Cs; !reflect.DeepEqual(&exp, cs) { t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs) } @@ -578,19 +592,25 @@ func TestRawNodeProposeAddDuplicateNode(t *testing.T) { t.Fatal(err) } - // the last three entries should be: ConfChange cc1, cc1, cc2 - entries, err := s.Entries(lastIndex-2, lastIndex+1, noLimit) + // the last 4 entries should be: ConfChange cc1, empty (ignored conf change), empty (by applying cc1), cc2 + entries, err := s.Entries(lastIndex-3, lastIndex+1, noLimit) if err != nil { t.Fatal(err) } - if len(entries) != 3 { - t.Fatalf("len(entries) = %d, want %d", len(entries), 3) + if len(entries) != 4 { + t.Fatalf("len(entries) = %d, want %d", len(entries), 4) } if !bytes.Equal(entries[0].Data, ccdata1) { t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, ccdata1) } - if !bytes.Equal(entries[2].Data, ccdata2) { - t.Errorf("entries[2].Data = %v, want %v", entries[2].Data, ccdata2) + if len(entries[1].Data) != 0 { + t.Errorf("entries[1] is not empty") + } + if len(entries[2].Data) != 0 { + t.Errorf("entries[2] is not empty") + } + if !bytes.Equal(entries[3].Data, ccdata2) { + t.Errorf("entries[3].Data = %v, want %v", entries[3].Data, ccdata2) } } diff --git a/storage.go b/storage.go index f616c315..de4ea633 100644 --- a/storage.go +++ b/storage.go @@ -266,6 +266,7 @@ func (ms *MemoryStorage) Compact(compactIndex uint64) error { ents := make([]pb.Entry, 1, uint64(len(ms.ents))-i) ents[0].Index = ms.ents[i].Index ents[0].Term = ms.ents[i].Term + ents[0].Subterm = ms.ents[i].Subterm ents = append(ents, ms.ents[i+1:]...) ms.ents = ents return nil diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt index 02e59e5a..618cd89c 100644 --- a/testdata/async_storage_writes.txt +++ b/testdata/async_storage_writes.txt @@ -64,7 +64,7 @@ stabilize > 1 receiving messages 1->1 MsgVoteResp Term:1 Log:0/0 INFO 1 received MsgVoteResp from 1 at term 1 - INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 @@ -78,7 +78,9 @@ stabilize > 1 receiving messages 2->1 MsgVoteResp Term:1 Log:0/0 INFO 1 received MsgVoteResp from 2 at term 1 - INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 1 resets replication set to {1,2,3}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 3->1 MsgVoteResp Term:1 Log:0/0 > 1 handling Ready diff --git a/testdata/async_storage_writes_append_aba_race.txt b/testdata/async_storage_writes_append_aba_race.txt index 83964fe7..6b1f651e 100644 --- a/testdata/async_storage_writes_append_aba_race.txt +++ b/testdata/async_storage_writes_append_aba_race.txt @@ -167,16 +167,18 @@ deliver-msgs 3 ---- 3->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 3 at term 2 -INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [3 0] votes to win 4->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 4 at term 2 -INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections +INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [2 0] votes to win 5->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 5 at term 2 -INFO 3 has received 3 MsgVoteResp votes and 0 vote rejections +INFO 3 has received 3 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win 6->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 6 at term 2 -INFO 3 has received 4 MsgVoteResp votes and 0 vote rejections +INFO 3 has received 4 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 3 resets replication set to {1,2,3,4,5,6,7}, {} +INFO 3 starts new subterm. Term: 2, Subterm: 0 INFO 3 became leader at term 2 # Step 5: node 3 proposes some log entries and node 1 receives these entries, @@ -329,16 +331,18 @@ deliver-msgs 4 ---- 4->4 MsgVoteResp Term:3 Log:0/0 INFO 4 received MsgVoteResp from 4 at term 3 -INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [3 0] votes to win 5->4 MsgVoteResp Term:3 Log:0/0 INFO 4 received MsgVoteResp from 5 at term 3 -INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections +INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [2 0] votes to win 6->4 MsgVoteResp Term:3 Log:0/0 INFO 4 received MsgVoteResp from 6 at term 3 -INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections +INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win 7->4 MsgVoteResp Term:3 Log:0/0 INFO 4 received MsgVoteResp from 7 at term 3 -INFO 4 has received 4 MsgVoteResp votes and 0 vote rejections +INFO 4 has received 4 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 4 resets replication set to {1,2,3,4,5,6,7}, {} +INFO 4 starts new subterm. Term: 3, Subterm: 0 INFO 4 became leader at term 3 process-ready 4 diff --git a/testdata/campaign.txt b/testdata/campaign.txt index 4eefab36..6c6907a7 100644 --- a/testdata/campaign.txt +++ b/testdata/campaign.txt @@ -31,7 +31,7 @@ stabilize 1->2 MsgVote Term:1 Log:1/2 1->3 MsgVote Term:1 Log:1/2 INFO 1 received MsgVoteResp from 1 at term 1 - INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 2 receiving messages 1->2 MsgVote Term:1 Log:1/2 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] @@ -55,7 +55,9 @@ stabilize > 1 receiving messages 2->1 MsgVoteResp Term:1 Log:0/0 INFO 1 received MsgVoteResp from 2 at term 1 - INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 1 resets replication set to {1,2,3}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 3->1 MsgVoteResp Term:1 Log:0/0 > 1 handling Ready diff --git a/testdata/campaign_learner_must_vote.txt b/testdata/campaign_learner_must_vote.txt index d05de374..533b7eb5 100644 --- a/testdata/campaign_learner_must_vote.txt +++ b/testdata/campaign_learner_must_vote.txt @@ -54,20 +54,20 @@ campaign 2 ---- INFO 2 is starting a new election at term 1 INFO 2 became candidate at term 2 -INFO 2 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2 -INFO 2 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 +INFO 2 [logterm: 1, index: 5] sent MsgVote request to 1 at term 2 +INFO 2 [logterm: 1, index: 5] sent MsgVote request to 3 at term 2 # Send out the MsgVote requests. process-ready 2 ---- Ready MustSync=true: Lead:0 State:StateCandidate -HardState Term:2 Vote:2 Commit:4 +HardState Term:2 Vote:2 Commit:5 Messages: -2->1 MsgVote Term:2 Log:1/4 -2->3 MsgVote Term:2 Log:1/4 +2->1 MsgVote Term:2 Log:1/5 +2->3 MsgVote Term:2 Log:1/5 INFO 2 received MsgVoteResp from 2 at term 2 -INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win # n2 is now campaigning while n1 is down (does not respond). The latest config # has n3 as a voter, but n3 doesn't even have the corresponding conf change in @@ -76,10 +76,10 @@ INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections stabilize 3 ---- > 3 receiving messages - 2->3 MsgVote Term:2 Log:1/4 + 2->3 MsgVote Term:2 Log:1/5 INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2] INFO 3 became follower at term 2 - INFO 3 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 4] at term 2 + INFO 3 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 5] at term 2 > 3 handling Ready Ready MustSync=true: Lead:0 State:StateFollower @@ -92,27 +92,29 @@ stabilize 2 3 > 2 receiving messages 3->2 MsgVoteResp Term:2 Log:0/0 INFO 2 received MsgVoteResp from 3 at term 2 - INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 2 resets replication set to {1,2,3}, {} + INFO 2 starts new subterm. Term: 2, Subterm: 0 INFO 2 became leader at term 2 > 2 handling Ready Ready MustSync=true: Lead:2 State:StateLeader Entries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 2->1 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] - 2->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] + 2->1 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] + 2->3 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] > 3 receiving messages - 2->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] - DEBUG 3 [logterm: 0, index: 4] rejected MsgApp [logterm: 1, index: 4] from 2 + 2->3 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] + DEBUG 3 [logterm: 0, index: 5] rejected MsgApp [logterm: 1, index: 5] from 2 > 3 handling Ready Ready MustSync=false: Lead:2 State:StateFollower Messages: - 3->2 MsgAppResp Term:2 Log:1/4 Rejected (Hint: 3) + 3->2 MsgAppResp Term:2 Log:1/5 Rejected (Hint: 3) > 2 receiving messages - 3->2 MsgAppResp Term:2 Log:1/4 Rejected (Hint: 3) - DEBUG 2 received MsgAppResp(rejected, hint: (index 3, term 1)) from 3 for index 4 + 3->2 MsgAppResp Term:2 Log:1/5 Rejected (Hint: 3) + DEBUG 2 received MsgAppResp(rejected, hint: (index 3, term 1)) from 3 for index 5 DEBUG 2 decreased progress of 3 to [StateProbe match=0 next=4] > 2 handling Ready Ready MustSync=false: @@ -128,32 +130,34 @@ stabilize 2 3 ] > 3 handling Ready Ready MustSync=true: - HardState Term:2 Vote:2 Commit:4 + HardState Term:2 Vote:2 Commit:5 Entries: 1/4 EntryConfChangeV2 v3 - 2/5 EntryNormal "" + 1/5 EntryNormal "" + 2/6 EntryNormal "" CommittedEntries: 1/4 EntryConfChangeV2 v3 + 1/5 EntryNormal "" Messages: - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/6 INFO 3 switched to configuration voters=(1 2 3) > 2 receiving messages - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/6 > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:5 + HardState Term:2 Vote:2 Commit:6 CommittedEntries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 2->3 MsgApp Term:2 Log:2/5 Commit:5 + 2->3 MsgApp Term:2 Log:2/6 Commit:6 > 3 receiving messages - 2->3 MsgApp Term:2 Log:2/5 Commit:5 + 2->3 MsgApp Term:2 Log:2/6 Commit:6 > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:5 + HardState Term:2 Vote:2 Commit:6 CommittedEntries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/6 > 2 receiving messages - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/6 diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index e54a183f..1cc9fb10 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -18,7 +18,9 @@ Ready MustSync=true: Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 1 resets replication set to {1}, {} +INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 # Add v2 (with an auto transition). @@ -51,13 +53,17 @@ stabilize CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChange v2 + INFO 1 resets replication set to {1,2}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(1 2) > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2, 1/5 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2, 1/5 EntryNormal ""] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -97,12 +103,32 @@ stabilize > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/5 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/5 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 + 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 2 handling Ready Ready MustSync=false: + HardState Term:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/5 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/5 diff --git a/testdata/confchange_v1_remove_leader.txt b/testdata/confchange_v1_remove_leader.txt index cc91508a..6257a100 100644 --- a/testdata/confchange_v1_remove_leader.txt +++ b/testdata/confchange_v1_remove_leader.txt @@ -105,7 +105,14 @@ stabilize 1 1->3 MsgApp Term:1 Log:1/6 Commit:4 1->2 MsgApp Term:1 Log:1/6 Commit:5 1->3 MsgApp Term:1 Log:1/6 Commit:5 + INFO 1 resets replication set to {2,3}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(2 3) +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/7 EntryNormal "" + DEBUG 1 no progress available for 1 raft-state ---- @@ -141,16 +148,28 @@ stabilize 1 2->1 MsgAppResp Term:1 Log:0/6 2->1 MsgAppResp Term:1 Log:0/6 2->1 MsgAppResp Term:1 Log:0/6 +> 1 handling Ready + Ready MustSync=false: + Messages: + 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal ""] # When n3 responds, quorum is reached and everything falls into place. stabilize ---- +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal ""] > 3 receiving messages 1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1] 1->3 MsgApp Term:1 Log:1/4 Commit:3 Entries:[1/5 EntryNormal "foo"] 1->3 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"] 1->3 MsgApp Term:1 Log:1/6 Commit:4 1->3 MsgApp Term:1 Log:1/6 Commit:5 +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/7 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/7 > 3 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:5 @@ -169,6 +188,7 @@ stabilize 3->1 MsgAppResp Term:1 Log:0/6 INFO 3 switched to configuration voters=(2 3) > 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/7 3->1 MsgAppResp Term:1 Log:0/4 3->1 MsgAppResp Term:1 Log:0/5 3->1 MsgAppResp Term:1 Log:0/6 @@ -180,29 +200,64 @@ stabilize CommittedEntries: 1/6 EntryNormal "bar" Messages: - 1->2 MsgApp Term:1 Log:1/6 Commit:6 - 1->3 MsgApp Term:1 Log:1/6 Commit:6 + 1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/7 Commit:6 + 1->3 MsgApp Term:1 Log:1/7 Commit:6 > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/6 Commit:6 + 1->2 MsgApp Term:1 Log:1/7 Commit:6 > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/6 Commit:6 + 1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/7 Commit:6 > 2 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:6 CommittedEntries: 1/6 EntryNormal "bar" Messages: - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/7 > 3 handling Ready - Ready MustSync=false: + Ready MustSync=true: HardState Term:1 Vote:1 Commit:6 + Entries: + 1/7 EntryNormal "" CommittedEntries: 1/6 EntryNormal "bar" Messages: - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/7 + 3->1 MsgAppResp Term:1 Log:0/7 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/7 + 3->1 MsgAppResp Term:1 Log:0/7 + 3->1 MsgAppResp Term:1 Log:0/7 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/7 Commit:7 + 1->3 MsgApp Term:1 Log:1/7 Commit:7 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/7 Commit:7 +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/7 Commit:7 +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/7 +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 3->1 MsgAppResp Term:1 Log:0/7 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/7 + 3->1 MsgAppResp Term:1 Log:0/7 # However not all is well. n1 is still leader but unconditionally drops all # proposals on the floor, so we're effectively stuck if it still heartbeats @@ -223,12 +278,12 @@ stabilize > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:6 - 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:6 + 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:7 + 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:7 > 2 receiving messages - 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:6 + 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:7 > 3 receiving messages - 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:6 + 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:7 > 2 handling Ready Ready MustSync=false: Messages: diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index bf1dfcbe..0cf15765 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -20,7 +20,9 @@ Ready MustSync=true: Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 1 resets replication set to {1}, {} +INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 propose-conf-change 1 transition=auto @@ -58,6 +60,8 @@ HardState Term:1 Vote:1 Commit:4 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 v3 +INFO 1 resets replication set to {1,2,3}, {1} +INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(1 2 3)&&(1) autoleave INFO initiating automatic transition out of joint configuration voters=(1 2 3)&&(1) autoleave @@ -67,10 +71,11 @@ stabilize 1 > 1 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3] - 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] # First, play out the whole interaction between n1 and n2. We see n1's probe to # n2 get rejected (since n2 needs a snapshot); the snapshot is delivered at which @@ -80,7 +85,7 @@ stabilize 1 stabilize 1 2 ---- > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -120,43 +125,82 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/5 Commit:5 + 1->2 MsgApp Term:1 Log:1/6 Commit:6 + INFO 1 resets replication set to {1,2,3}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 2 INFO 1 switched to configuration voters=(1 2 3) > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/5 Commit:5 + 1->2 MsgApp Term:1 Log:1/6 Commit:6 +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/7 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 INFO 2 switched to configuration voters=(1 2 3) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""] +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/7 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/7 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/7 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/7 Commit:7 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/7 Commit:7 +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/7 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/7 # n3 immediately receives a snapshot in the final configuration. stabilize 1 3 ---- > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3] + 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] INFO 3 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 3 became follower at term 1 DEBUG 3 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -170,8 +214,8 @@ stabilize 1 3 3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 3 for index 3 DEBUG 1 decreased progress of 3 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 5] sent snapshot[index: 5, term: 1] to 3 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=1 paused pendingSnap=5] + DEBUG 1 [firstindex: 3, commit: 7] sent snapshot[index: 7, term: 1] to 3 [StateProbe match=0 next=1] + DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=1 paused pendingSnap=7] > 1 handling Ready Ready MustSync=false: Messages: @@ -182,29 +226,29 @@ stabilize 1 3 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] INFO 3 switched to configuration voters=(1 2 3) - INFO 3 [commit: 5, lastindex: 5, lastterm: 1] restored snapshot [index: 5, term: 1] - INFO 3 [commit: 5] restored snapshot [index: 5, term: 1] + INFO 3 [commit: 7, lastindex: 7, lastterm: 1] restored snapshot [index: 7, term: 1] + INFO 3 [commit: 7] restored snapshot [index: 7, term: 1] > 3 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 - Snapshot Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + HardState Term:1 Commit:7 + Snapshot Index:7 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/7 > 1 receiving messages - 3->1 MsgAppResp Term:1 Log:0/5 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=5 next=6 paused pendingSnap=5] + 3->1 MsgAppResp Term:1 Log:0/7 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=7 next=8 paused pendingSnap=7] > 1 handling Ready Ready MustSync=false: Messages: - 1->3 MsgApp Term:1 Log:1/5 Commit:5 + 1->3 MsgApp Term:1 Log:1/7 Commit:7 > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/5 Commit:5 + 1->3 MsgApp Term:1 Log:1/7 Commit:7 > 3 handling Ready Ready MustSync=false: Messages: - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/7 > 1 receiving messages - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/7 # Nothing else happens. stabilize @@ -225,30 +269,30 @@ stabilize 1 > 1 handling Ready Ready MustSync=true: Entries: - 1/6 EntryConfChangeV2 r2 r3 + 1/8 EntryConfChangeV2 r2 r3 Messages: - 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3] - 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3] + 1->2 MsgApp Term:1 Log:1/7 Commit:7 Entries:[1/8 EntryConfChangeV2 r2 r3] + 1->3 MsgApp Term:1 Log:1/7 Commit:7 Entries:[1/8 EntryConfChangeV2 r2 r3] # n2, n3 ack them. stabilize 2 3 ---- > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3] + 1->2 MsgApp Term:1 Log:1/7 Commit:7 Entries:[1/8 EntryConfChangeV2 r2 r3] > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3] + 1->3 MsgApp Term:1 Log:1/7 Commit:7 Entries:[1/8 EntryConfChangeV2 r2 r3] > 2 handling Ready Ready MustSync=true: Entries: - 1/6 EntryConfChangeV2 r2 r3 + 1/8 EntryConfChangeV2 r2 r3 Messages: - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/8 > 3 handling Ready Ready MustSync=true: Entries: - 1/6 EntryConfChangeV2 r2 r3 + 1/8 EntryConfChangeV2 r2 r3 Messages: - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/8 # n1 gets some more proposals. This is part of a regression test: There used to # be a bug in which these proposals would prompt the leader to transition out of @@ -268,76 +312,87 @@ stabilize 1 > 1 handling Ready Ready MustSync=true: Entries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" + 1/9 EntryNormal "foo" + 1/10 EntryNormal "bar" Messages: - 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] - 1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] + 1->2 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryNormal "foo"] + 1->3 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryNormal "foo"] + 1->2 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryNormal "bar"] + 1->3 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryNormal "bar"] > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/8 + 3->1 MsgAppResp Term:1 Log:0/8 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:8 CommittedEntries: - 1/6 EntryConfChangeV2 r2 r3 + 1/8 EntryConfChangeV2 r2 r3 Messages: - 1->2 MsgApp Term:1 Log:1/8 Commit:6 - 1->3 MsgApp Term:1 Log:1/8 Commit:6 + 1->2 MsgApp Term:1 Log:1/10 Commit:8 + 1->3 MsgApp Term:1 Log:1/10 Commit:8 + INFO 1 resets replication set to {1}, {1,2,3} + INFO 1 starts new subterm. Term: 1, Subterm: 3 INFO 1 switched to configuration voters=(1)&&(1 2 3) autoleave INFO initiating automatic transition out of joint configuration voters=(1)&&(1 2 3) autoleave > 1 handling Ready Ready MustSync=true: Entries: - 1/9 EntryConfChangeV2 + 1/11 EntryNormal "" + 1/12 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] - 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/10 Commit:8 Entries:[1/11 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/10 Commit:8 Entries:[1/11 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/11 Commit:8 Entries:[1/12 EntryConfChangeV2] + 1->3 MsgApp Term:1 Log:1/11 Commit:8 Entries:[1/12 EntryConfChangeV2] # n2 and n3 also switch to the joint config, and ack the transition out of it. stabilize 2 3 ---- > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] - 1->2 MsgApp Term:1 Log:1/8 Commit:6 - 1->2 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryNormal "foo"] + 1->2 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryNormal "bar"] + 1->2 MsgApp Term:1 Log:1/10 Commit:8 + 1->2 MsgApp Term:1 Log:1/10 Commit:8 Entries:[1/11 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/11 Commit:8 Entries:[1/12 EntryConfChangeV2] > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] - 1->3 MsgApp Term:1 Log:1/8 Commit:6 - 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] + 1->3 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryNormal "foo"] + 1->3 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryNormal "bar"] + 1->3 MsgApp Term:1 Log:1/10 Commit:8 + 1->3 MsgApp Term:1 Log:1/10 Commit:8 Entries:[1/11 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/11 Commit:8 Entries:[1/12 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: - HardState Term:1 Commit:6 + HardState Term:1 Commit:8 Entries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/9 EntryNormal "foo" + 1/10 EntryNormal "bar" + 1/11 EntryNormal "" + 1/12 EntryConfChangeV2 CommittedEntries: - 1/6 EntryConfChangeV2 r2 r3 + 1/8 EntryConfChangeV2 r2 r3 Messages: - 2->1 MsgAppResp Term:1 Log:0/7 - 2->1 MsgAppResp Term:1 Log:0/8 - 2->1 MsgAppResp Term:1 Log:0/8 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/10 + 2->1 MsgAppResp Term:1 Log:0/10 + 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/12 INFO 2 switched to configuration voters=(1)&&(1 2 3) autoleave > 3 handling Ready Ready MustSync=true: - HardState Term:1 Commit:6 + HardState Term:1 Commit:8 Entries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/9 EntryNormal "foo" + 1/10 EntryNormal "bar" + 1/11 EntryNormal "" + 1/12 EntryConfChangeV2 CommittedEntries: - 1/6 EntryConfChangeV2 r2 r3 + 1/8 EntryConfChangeV2 r2 r3 Messages: - 3->1 MsgAppResp Term:1 Log:0/7 - 3->1 MsgAppResp Term:1 Log:0/8 - 3->1 MsgAppResp Term:1 Log:0/8 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/10 + 3->1 MsgAppResp Term:1 Log:0/10 + 3->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/12 INFO 3 switched to configuration voters=(1)&&(1 2 3) autoleave # n2 and n3 also leave the joint config and the dust settles. We see at the very @@ -346,71 +401,97 @@ stabilize 2 3 stabilize ---- > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 - 2->1 MsgAppResp Term:1 Log:0/8 - 2->1 MsgAppResp Term:1 Log:0/8 2->1 MsgAppResp Term:1 Log:0/9 - 3->1 MsgAppResp Term:1 Log:0/7 - 3->1 MsgAppResp Term:1 Log:0/8 - 3->1 MsgAppResp Term:1 Log:0/8 + 2->1 MsgAppResp Term:1 Log:0/10 + 2->1 MsgAppResp Term:1 Log:0/10 + 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/12 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/10 + 3->1 MsgAppResp Term:1 Log:0/10 + 3->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/12 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:9 + HardState Term:1 Vote:1 Commit:12 CommittedEntries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/9 EntryNormal "foo" + 1/10 EntryNormal "bar" + 1/11 EntryNormal "" + 1/12 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/9 Commit:7 - 1->3 MsgApp Term:1 Log:1/9 Commit:7 - 1->2 MsgApp Term:1 Log:1/9 Commit:8 - 1->3 MsgApp Term:1 Log:1/9 Commit:8 - 1->2 MsgApp Term:1 Log:1/9 Commit:9 - 1->3 MsgApp Term:1 Log:1/9 Commit:9 + 1->2 MsgApp Term:1 Log:1/12 Commit:9 + 1->3 MsgApp Term:1 Log:1/12 Commit:9 + 1->2 MsgApp Term:1 Log:1/12 Commit:10 + 1->3 MsgApp Term:1 Log:1/12 Commit:10 + 1->2 MsgApp Term:1 Log:1/12 Commit:11 + 1->3 MsgApp Term:1 Log:1/12 Commit:11 + 1->2 MsgApp Term:1 Log:1/12 Commit:12 + 1->3 MsgApp Term:1 Log:1/12 Commit:12 + INFO 1 resets replication set to {1}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 4 INFO 1 switched to configuration voters=(1) > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/9 Commit:7 - 1->2 MsgApp Term:1 Log:1/9 Commit:8 - 1->2 MsgApp Term:1 Log:1/9 Commit:9 + 1->2 MsgApp Term:1 Log:1/12 Commit:9 + 1->2 MsgApp Term:1 Log:1/12 Commit:10 + 1->2 MsgApp Term:1 Log:1/12 Commit:11 + 1->2 MsgApp Term:1 Log:1/12 Commit:12 > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/9 Commit:7 - 1->3 MsgApp Term:1 Log:1/9 Commit:8 - 1->3 MsgApp Term:1 Log:1/9 Commit:9 + 1->3 MsgApp Term:1 Log:1/12 Commit:9 + 1->3 MsgApp Term:1 Log:1/12 Commit:10 + 1->3 MsgApp Term:1 Log:1/12 Commit:11 + 1->3 MsgApp Term:1 Log:1/12 Commit:12 +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "" > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:9 + HardState Term:1 Commit:12 CommittedEntries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/9 EntryNormal "foo" + 1/10 EntryNormal "bar" + 1/11 EntryNormal "" + 1/12 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/9 - 2->1 MsgAppResp Term:1 Log:0/9 - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 INFO 2 switched to configuration voters=(1) > 3 handling Ready Ready MustSync=false: - HardState Term:1 Commit:9 + HardState Term:1 Commit:12 CommittedEntries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/9 EntryNormal "foo" + 1/10 EntryNormal "bar" + 1/11 EntryNormal "" + 1/12 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/9 - 3->1 MsgAppResp Term:1 Log:0/9 - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/12 + 3->1 MsgAppResp Term:1 Log:0/12 + 3->1 MsgAppResp Term:1 Log:0/12 + 3->1 MsgAppResp Term:1 Log:0/12 INFO 3 switched to configuration voters=(1) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/12 raft: cannot step as peer not found - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/12 raft: cannot step as peer not found - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/12 raft: cannot step as peer not found - 3->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/12 raft: cannot step as peer not found - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/12 raft: cannot step as peer not found - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/12 + raft: cannot step as peer not found + 3->1 MsgAppResp Term:1 Log:0/12 raft: cannot step as peer not found + 3->1 MsgAppResp Term:1 Log:0/12 + raft: cannot step as peer not found +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:13 + CommittedEntries: + 1/13 EntryNormal "" diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index 536d66b8..6671e11d 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -22,7 +22,9 @@ Ready MustSync=true: Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 1 resets replication set to {1}, {} +INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 propose-conf-change 1 transition=implicit @@ -54,16 +56,19 @@ stabilize 1 2 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 + INFO 1 resets replication set to {1,2}, {1} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(1 2)&&(1) autoleave INFO initiating automatic transition out of joint configuration voters=(1 2)&&(1) autoleave > 1 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -103,34 +108,73 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/5 Commit:5 + 1->2 MsgApp Term:1 Log:1/6 Commit:6 + INFO 1 resets replication set to {1,2}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 2 INFO 1 switched to configuration voters=(1 2) > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/5 Commit:5 + 1->2 MsgApp Term:1 Log:1/6 Commit:6 +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/7 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 INFO 2 switched to configuration voters=(1 2) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""] +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/7 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/7 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/7 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/7 Commit:7 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/7 Commit:7 +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Commit:7 + CommittedEntries: + 1/7 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/7 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/7 diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index 1c487da8..9df33fab 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -20,7 +20,9 @@ Ready MustSync=true: Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 1 resets replication set to {1}, {} +INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 # Add v2 (with an auto transition). @@ -52,13 +54,17 @@ stabilize CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 + INFO 1 resets replication set to {1,2}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(1 2) > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -98,12 +104,32 @@ stabilize > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/5 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/5 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 + 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 2 handling Ready Ready MustSync=false: + HardState Term:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/5 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/5 diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index 123cd17a..ae53123f 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -20,7 +20,9 @@ Ready MustSync=true: Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win +INFO 1 resets replication set to {1}, {} +INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 # Add v2 with an explicit transition. @@ -52,13 +54,17 @@ stabilize 1 2 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 + INFO 1 resets replication set to {1,2}, {1} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(1 2)&&(1) > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -98,15 +104,35 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] > 2 handling Ready + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/5 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/5 +> 1 handling Ready Ready MustSync=false: + HardState Term:1 Vote:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/4 + 1->2 MsgApp Term:1 Log:1/5 Commit:5 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/5 Commit:5 +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/5 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/5 # Check that we're not allowed to change membership again while in the joint state. # This leads to an empty entry being proposed instead (index 5 in the stabilize block @@ -127,51 +153,87 @@ stabilize > 1 handling Ready Ready MustSync=true: Entries: - 1/5 EntryNormal "" - 1/6 EntryConfChangeV2 + 1/6 EntryNormal "" + 1/7 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] - 1->2 MsgApp Term:1 Log:1/5 Commit:4 Entries:[1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryConfChangeV2] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] - 1->2 MsgApp Term:1 Log:1/5 Commit:4 Entries:[1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: Entries: - 1/5 EntryNormal "" - 1/6 EntryConfChangeV2 + 1/6 EntryNormal "" + 1/7 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/7 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/7 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:7 CommittedEntries: - 1/5 EntryNormal "" - 1/6 EntryConfChangeV2 + 1/6 EntryNormal "" + 1/7 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/6 Commit:5 - 1->2 MsgApp Term:1 Log:1/6 Commit:6 + 1->2 MsgApp Term:1 Log:1/7 Commit:6 + 1->2 MsgApp Term:1 Log:1/7 Commit:7 + INFO 1 resets replication set to {1,2}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 2 INFO 1 switched to configuration voters=(1 2) > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/6 Commit:5 - 1->2 MsgApp Term:1 Log:1/6 Commit:6 + 1->2 MsgApp Term:1 Log:1/7 Commit:6 + 1->2 MsgApp Term:1 Log:1/7 Commit:7 +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/8 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/7 Commit:7 Entries:[1/8 EntryNormal ""] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:6 + HardState Term:1 Commit:7 CommittedEntries: - 1/5 EntryNormal "" - 1/6 EntryConfChangeV2 + 1/6 EntryNormal "" + 1/7 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/7 INFO 2 switched to configuration voters=(1 2) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/7 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/7 Commit:7 Entries:[1/8 EntryNormal ""] +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/8 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/8 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/8 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:8 + CommittedEntries: + 1/8 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/8 Commit:8 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/8 Commit:8 +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Commit:8 + CommittedEntries: + 1/8 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/8 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/8 # Check that trying to transition out again won't do anything. propose-conf-change 1 @@ -184,34 +246,34 @@ stabilize > 1 handling Ready Ready MustSync=true: Entries: - 1/7 EntryNormal "" + 1/9 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/8 Commit:8 Entries:[1/9 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/8 Commit:8 Entries:[1/9 EntryNormal ""] > 2 handling Ready Ready MustSync=true: Entries: - 1/7 EntryNormal "" + 1/9 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/9 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/9 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:7 + HardState Term:1 Vote:1 Commit:9 CommittedEntries: - 1/7 EntryNormal "" + 1/9 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/7 Commit:7 + 1->2 MsgApp Term:1 Log:1/9 Commit:9 > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/7 Commit:7 + 1->2 MsgApp Term:1 Log:1/9 Commit:9 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:7 + HardState Term:1 Commit:9 CommittedEntries: - 1/7 EntryNormal "" + 1/9 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/9 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/9 diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index ae43ce21..daea413f 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -83,15 +83,21 @@ stabilize Messages: 1->2 MsgApp Term:1 Log:1/4 Commit:4 1->3 MsgApp Term:1 Log:1/4 Commit:4 + INFO 1 resets replication set to {2,3,4}, {1,2,3} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(2 3 4)&&(1 2 3) > 2 receiving messages 1->2 MsgApp Term:1 Log:1/4 Commit:4 > 3 receiving messages 1->3 MsgApp Term:1 Log:1/4 Commit:4 > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" Messages: - 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] + 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4, 1/5 EntryNormal ""] > 2 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:4 @@ -111,10 +117,26 @@ stabilize > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/4 3->1 MsgAppResp Term:1 Log:0/4 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] > 4 receiving messages - 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4] + 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4, 1/5 EntryNormal ""] INFO 4 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 4 became follower at term 1 +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/5 +> 3 handling Ready + Ready MustSync=true: + Entries: + 1/5 EntryNormal "" + Messages: + 3->1 MsgAppResp Term:1 Log:0/5 > 4 handling Ready Ready MustSync=true: Lead:1 State:StateFollower @@ -122,9 +144,14 @@ stabilize Messages: 4->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/5 4->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 handling Ready Ready MustSync=false: + HardState Term:1 Vote:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" Messages: 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false @@ -135,6 +162,20 @@ stabilize INFO 4 switched to configuration voters=(2 3 4)&&(1 2 3) INFO 4 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 4 [commit: 4] restored snapshot [index: 4, term: 1] +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" + Messages: + 2->1 MsgAppResp Term:1 Log:0/5 +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:5 + CommittedEntries: + 1/5 EntryNormal "" + Messages: + 3->1 MsgAppResp Term:1 Log:0/5 > 4 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 @@ -142,19 +183,26 @@ stabilize Messages: 4->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/5 4->1 MsgAppResp Term:1 Log:0/4 > 1 handling Ready Ready MustSync=false: Messages: - 1->4 MsgApp Term:1 Log:1/4 Commit:4 + 1->4 MsgApp Term:1 Log:1/4 Commit:5 Entries:[1/5 EntryNormal ""] > 4 receiving messages - 1->4 MsgApp Term:1 Log:1/4 Commit:4 + 1->4 MsgApp Term:1 Log:1/4 Commit:5 Entries:[1/5 EntryNormal ""] > 4 handling Ready - Ready MustSync=false: + Ready MustSync=true: + HardState Term:1 Commit:5 + Entries: + 1/5 EntryNormal "" + CommittedEntries: + 1/5 EntryNormal "" Messages: - 4->1 MsgAppResp Term:1 Log:0/4 + 4->1 MsgAppResp Term:1 Log:0/5 > 1 receiving messages - 4->1 MsgAppResp Term:1 Log:0/4 + 4->1 MsgAppResp Term:1 Log:0/5 # Transfer leadership while in the joint config. @@ -183,145 +231,147 @@ stabilize INFO 4 [term 1] received MsgTimeoutNow from 1 and starts an election to get leadership. INFO 4 is starting a new election at term 1 INFO 4 became candidate at term 2 - INFO 4 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2 - INFO 4 [logterm: 1, index: 4] sent MsgVote request to 2 at term 2 - INFO 4 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 + INFO 4 [logterm: 1, index: 5] sent MsgVote request to 1 at term 2 + INFO 4 [logterm: 1, index: 5] sent MsgVote request to 2 at term 2 + INFO 4 [logterm: 1, index: 5] sent MsgVote request to 3 at term 2 > 4 handling Ready Ready MustSync=true: Lead:0 State:StateCandidate - HardState Term:2 Vote:4 Commit:4 + HardState Term:2 Vote:4 Commit:5 Messages: - 4->1 MsgVote Term:2 Log:1/4 - 4->2 MsgVote Term:2 Log:1/4 - 4->3 MsgVote Term:2 Log:1/4 + 4->1 MsgVote Term:2 Log:1/5 + 4->2 MsgVote Term:2 Log:1/5 + 4->3 MsgVote Term:2 Log:1/5 INFO 4 received MsgVoteResp from 4 at term 2 - INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 2] votes to win > 1 receiving messages - 4->1 MsgVote Term:2 Log:1/4 + 4->1 MsgVote Term:2 Log:1/5 INFO 1 [term: 1] received a MsgVote message with higher term from 4 [term: 2] INFO 1 became follower at term 2 - INFO 1 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2 + INFO 1 [logterm: 1, index: 5, vote: 0] cast MsgVote for 4 [logterm: 1, index: 5] at term 2 > 2 receiving messages - 4->2 MsgVote Term:2 Log:1/4 + 4->2 MsgVote Term:2 Log:1/5 INFO 2 [term: 1] received a MsgVote message with higher term from 4 [term: 2] INFO 2 became follower at term 2 - INFO 2 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2 + INFO 2 [logterm: 1, index: 5, vote: 0] cast MsgVote for 4 [logterm: 1, index: 5] at term 2 > 3 receiving messages - 4->3 MsgVote Term:2 Log:1/4 + 4->3 MsgVote Term:2 Log:1/5 INFO 3 [term: 1] received a MsgVote message with higher term from 4 [term: 2] INFO 3 became follower at term 2 - INFO 3 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2 + INFO 3 [logterm: 1, index: 5, vote: 0] cast MsgVote for 4 [logterm: 1, index: 5] at term 2 > 1 handling Ready Ready MustSync=true: Lead:0 State:StateFollower - HardState Term:2 Vote:4 Commit:4 + HardState Term:2 Vote:4 Commit:5 Messages: 1->4 MsgVoteResp Term:2 Log:0/0 > 2 handling Ready Ready MustSync=true: Lead:0 State:StateFollower - HardState Term:2 Vote:4 Commit:4 + HardState Term:2 Vote:4 Commit:5 Messages: 2->4 MsgVoteResp Term:2 Log:0/0 > 3 handling Ready Ready MustSync=true: Lead:0 State:StateFollower - HardState Term:2 Vote:4 Commit:4 + HardState Term:2 Vote:4 Commit:5 Messages: 3->4 MsgVoteResp Term:2 Log:0/0 > 4 receiving messages 1->4 MsgVoteResp Term:2 Log:0/0 INFO 4 received MsgVoteResp from 1 at term 2 - INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [1 1] votes to win 2->4 MsgVoteResp Term:2 Log:0/0 INFO 4 received MsgVoteResp from 2 at term 2 - INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections + INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 4 resets replication set to {2,3,4}, {1,2,3} + INFO 4 starts new subterm. Term: 2, Subterm: 0 INFO 4 became leader at term 2 3->4 MsgVoteResp Term:2 Log:0/0 > 4 handling Ready Ready MustSync=true: Lead:4 State:StateLeader Entries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 4->1 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] - 4->2 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] - 4->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] + 4->1 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] + 4->2 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] + 4->3 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] > 1 receiving messages - 4->1 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] + 4->1 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] > 2 receiving messages - 4->2 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] + 4->2 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] > 3 receiving messages - 4->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] + 4->3 MsgApp Term:2 Log:1/5 Commit:5 Entries:[2/6 EntryNormal ""] > 1 handling Ready Ready MustSync=true: Lead:4 State:StateFollower Entries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 1->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/6 > 2 handling Ready Ready MustSync=true: Lead:4 State:StateFollower Entries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 2->4 MsgAppResp Term:2 Log:0/5 + 2->4 MsgAppResp Term:2 Log:0/6 > 3 handling Ready Ready MustSync=true: Lead:4 State:StateFollower Entries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 3->4 MsgAppResp Term:2 Log:0/5 + 3->4 MsgAppResp Term:2 Log:0/6 > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/5 - 2->4 MsgAppResp Term:2 Log:0/5 - 3->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/6 + 3->4 MsgAppResp Term:2 Log:0/6 > 4 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:6 CommittedEntries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 4->1 MsgApp Term:2 Log:2/5 Commit:4 - 4->1 MsgApp Term:2 Log:2/5 Commit:5 - 4->2 MsgApp Term:2 Log:2/5 Commit:5 - 4->3 MsgApp Term:2 Log:2/5 Commit:5 + 4->1 MsgApp Term:2 Log:2/6 Commit:5 + 4->1 MsgApp Term:2 Log:2/6 Commit:6 + 4->2 MsgApp Term:2 Log:2/6 Commit:6 + 4->3 MsgApp Term:2 Log:2/6 Commit:6 > 1 receiving messages - 4->1 MsgApp Term:2 Log:2/5 Commit:4 - 4->1 MsgApp Term:2 Log:2/5 Commit:5 + 4->1 MsgApp Term:2 Log:2/6 Commit:5 + 4->1 MsgApp Term:2 Log:2/6 Commit:6 > 2 receiving messages - 4->2 MsgApp Term:2 Log:2/5 Commit:5 + 4->2 MsgApp Term:2 Log:2/6 Commit:6 > 3 receiving messages - 4->3 MsgApp Term:2 Log:2/5 Commit:5 + 4->3 MsgApp Term:2 Log:2/6 Commit:6 > 1 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:6 CommittedEntries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 1->4 MsgAppResp Term:2 Log:0/5 - 1->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/6 > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:6 CommittedEntries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 2->4 MsgAppResp Term:2 Log:0/5 + 2->4 MsgAppResp Term:2 Log:0/6 > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:6 CommittedEntries: - 2/5 EntryNormal "" + 2/6 EntryNormal "" Messages: - 3->4 MsgAppResp Term:2 Log:0/5 + 3->4 MsgAppResp Term:2 Log:0/6 > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/5 - 1->4 MsgAppResp Term:2 Log:0/5 - 2->4 MsgAppResp Term:2 Log:0/5 - 3->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/6 + 3->4 MsgAppResp Term:2 Log:0/6 # Leadership transfer succeeded. raft-state @@ -342,84 +392,141 @@ stabilize > 4 handling Ready Ready MustSync=true: Entries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 4->1 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2] - 4->2 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2] - 4->3 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2] + 4->1 MsgApp Term:2 Log:2/6 Commit:6 Entries:[2/7 EntryConfChangeV2] + 4->2 MsgApp Term:2 Log:2/6 Commit:6 Entries:[2/7 EntryConfChangeV2] + 4->3 MsgApp Term:2 Log:2/6 Commit:6 Entries:[2/7 EntryConfChangeV2] > 1 receiving messages - 4->1 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2] + 4->1 MsgApp Term:2 Log:2/6 Commit:6 Entries:[2/7 EntryConfChangeV2] > 2 receiving messages - 4->2 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2] + 4->2 MsgApp Term:2 Log:2/6 Commit:6 Entries:[2/7 EntryConfChangeV2] > 3 receiving messages - 4->3 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2] + 4->3 MsgApp Term:2 Log:2/6 Commit:6 Entries:[2/7 EntryConfChangeV2] > 1 handling Ready Ready MustSync=true: Entries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/7 > 2 handling Ready Ready MustSync=true: Entries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 2->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/7 > 3 handling Ready Ready MustSync=true: Entries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 3->4 MsgAppResp Term:2 Log:0/6 + 3->4 MsgAppResp Term:2 Log:0/7 > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/6 - 2->4 MsgAppResp Term:2 Log:0/6 - 3->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/7 + 2->4 MsgAppResp Term:2 Log:0/7 + 3->4 MsgAppResp Term:2 Log:0/7 > 4 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:7 CommittedEntries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 4->1 MsgApp Term:2 Log:2/6 Commit:6 - 4->2 MsgApp Term:2 Log:2/6 Commit:6 - 4->3 MsgApp Term:2 Log:2/6 Commit:6 + 4->1 MsgApp Term:2 Log:2/7 Commit:7 + 4->2 MsgApp Term:2 Log:2/7 Commit:7 + 4->3 MsgApp Term:2 Log:2/7 Commit:7 + INFO 4 resets replication set to {2,3,4}, {} + INFO 4 starts new subterm. Term: 2, Subterm: 1 INFO 4 switched to configuration voters=(2 3 4) > 1 receiving messages - 4->1 MsgApp Term:2 Log:2/6 Commit:6 + 4->1 MsgApp Term:2 Log:2/7 Commit:7 > 2 receiving messages - 4->2 MsgApp Term:2 Log:2/6 Commit:6 + 4->2 MsgApp Term:2 Log:2/7 Commit:7 > 3 receiving messages - 4->3 MsgApp Term:2 Log:2/6 Commit:6 + 4->3 MsgApp Term:2 Log:2/7 Commit:7 > 1 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:7 CommittedEntries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/7 INFO 1 switched to configuration voters=(2 3 4) > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:7 CommittedEntries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 2->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/7 INFO 2 switched to configuration voters=(2 3 4) > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:7 CommittedEntries: - 2/6 EntryConfChangeV2 + 2/7 EntryConfChangeV2 Messages: - 3->4 MsgAppResp Term:2 Log:0/6 + 3->4 MsgAppResp Term:2 Log:0/7 INFO 3 switched to configuration voters=(2 3 4) +> 4 handling Ready + Ready MustSync=true: + Entries: + 2/8 EntryNormal "" + Messages: + 4->2 MsgApp Term:2 Log:2/7 Commit:7 Entries:[2/8 EntryNormal ""] + 4->3 MsgApp Term:2 Log:2/7 Commit:7 Entries:[2/8 EntryNormal ""] +> 2 receiving messages + 4->2 MsgApp Term:2 Log:2/7 Commit:7 Entries:[2/8 EntryNormal ""] +> 3 receiving messages + 4->3 MsgApp Term:2 Log:2/7 Commit:7 Entries:[2/8 EntryNormal ""] > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/7 raft: cannot step as peer not found - 2->4 MsgAppResp Term:2 Log:0/6 - 3->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/7 + 3->4 MsgAppResp Term:2 Log:0/7 +> 2 handling Ready + Ready MustSync=true: + Entries: + 2/8 EntryNormal "" + Messages: + 2->4 MsgAppResp Term:2 Log:0/8 +> 3 handling Ready + Ready MustSync=true: + Entries: + 2/8 EntryNormal "" + Messages: + 3->4 MsgAppResp Term:2 Log:0/8 +> 4 receiving messages + 2->4 MsgAppResp Term:2 Log:0/8 + 3->4 MsgAppResp Term:2 Log:0/8 +> 4 handling Ready + Ready MustSync=false: + HardState Term:2 Vote:4 Commit:8 + CommittedEntries: + 2/8 EntryNormal "" + Messages: + 4->2 MsgApp Term:2 Log:2/8 Commit:8 + 4->3 MsgApp Term:2 Log:2/8 Commit:8 +> 2 receiving messages + 4->2 MsgApp Term:2 Log:2/8 Commit:8 +> 3 receiving messages + 4->3 MsgApp Term:2 Log:2/8 Commit:8 +> 2 handling Ready + Ready MustSync=false: + HardState Term:2 Vote:4 Commit:8 + CommittedEntries: + 2/8 EntryNormal "" + Messages: + 2->4 MsgAppResp Term:2 Log:0/8 +> 3 handling Ready + Ready MustSync=false: + HardState Term:2 Vote:4 Commit:8 + CommittedEntries: + 2/8 EntryNormal "" + Messages: + 3->4 MsgAppResp Term:2 Log:0/8 +> 4 receiving messages + 2->4 MsgAppResp Term:2 Log:0/8 + 3->4 MsgAppResp Term:2 Log:0/8 # n1 is out of the configuration. raft-state diff --git a/testdata/probe_and_replicate.txt b/testdata/probe_and_replicate.txt index c4100e97..90886890 100644 --- a/testdata/probe_and_replicate.txt +++ b/testdata/probe_and_replicate.txt @@ -379,7 +379,7 @@ stabilize 1 1->6 MsgVote Term:8 Log:6/20 1->7 MsgVote Term:8 Log:6/20 INFO 1 received MsgVoteResp from 1 at term 8 - INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [3 0] votes to win stabilize 2 3 4 5 6 7 ---- @@ -452,19 +452,21 @@ stabilize 1 > 1 receiving messages 2->1 MsgVoteResp Term:8 Log:0/0 INFO 1 received MsgVoteResp from 2 at term 8 - INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [2 0] votes to win 3->1 MsgVoteResp Term:8 Log:0/0 INFO 1 received MsgVoteResp from 3 at term 8 - INFO 1 has received 3 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 3 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win 4->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0) INFO 1 received MsgVoteResp rejection from 4 at term 8 - INFO 1 has received 3 MsgVoteResp votes and 1 vote rejections + INFO 1 has received 3 MsgVoteResp votes and 1 vote rejections. It needs [1 0] votes to win 5->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0) INFO 1 received MsgVoteResp rejection from 5 at term 8 - INFO 1 has received 3 MsgVoteResp votes and 2 vote rejections + INFO 1 has received 3 MsgVoteResp votes and 2 vote rejections. It needs [1 0] votes to win 6->1 MsgVoteResp Term:8 Log:0/0 INFO 1 received MsgVoteResp from 6 at term 8 - INFO 1 has received 4 MsgVoteResp votes and 2 vote rejections + INFO 1 has received 4 MsgVoteResp votes and 2 vote rejections. It needs [0 0] votes to win + INFO 1 resets replication set to {1,2,3,4,5,6,7}, {} + INFO 1 starts new subterm. Term: 8, Subterm: 0 INFO 1 became leader at term 8 7->1 MsgVoteResp Term:8 Log:0/0 > 1 handling Ready diff --git a/testdata/single_node.txt b/testdata/single_node.txt index f6aceb71..4507a71e 100644 --- a/testdata/single_node.txt +++ b/testdata/single_node.txt @@ -20,7 +20,9 @@ stabilize Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:3 INFO 1 received MsgVoteResp from 1 at term 1 - INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 1 resets replication set to {1}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 0 INFO 1 became leader at term 1 > 1 handling Ready Ready MustSync=true: diff --git a/tracker/progress.go b/tracker/progress.go index 32c5ee24..58a1ec0b 100644 --- a/tracker/progress.go +++ b/tracker/progress.go @@ -95,6 +95,9 @@ type Progress struct { // IsLearner is true if this progress is tracked for a learner. IsLearner bool + + // IsWitness is true if this progress is tracked for a witness. + IsWitness bool } // ResetState moves the Progress into the specified State, resetting MsgAppFlowPaused, @@ -239,6 +242,9 @@ func (pr *Progress) String() string { if pr.IsLearner { fmt.Fprint(&buf, " learner") } + if pr.IsWitness { + fmt.Fprint(&buf, " witness") + } if pr.IsPaused() { fmt.Fprint(&buf, " paused") } diff --git a/tracker/tracker.go b/tracker/tracker.go index 2710a7a4..b4cc52f1 100644 --- a/tracker/tracker.go +++ b/tracker/tracker.go @@ -17,6 +17,7 @@ package tracker import ( "fmt" "sort" + "strconv" "strings" "go.etcd.io/raft/v3/quorum" @@ -75,11 +76,19 @@ type Config struct { // right away when entering the joint configuration, so that it is caught up // as soon as possible. LearnersNext map[uint64]struct{} + Witnesses [2]uint64 } func (c Config) String() string { var buf strings.Builder fmt.Fprintf(&buf, "voters=%s", c.Voters) + if c.Witnesses[0] > 0 && c.Witnesses[1] > 0 { + fmt.Fprintf(&buf, " witness={%d,%d}", c.Witnesses[0], c.Witnesses[1]) + } else if c.Witnesses[0] > 0 { + fmt.Fprintf(&buf, " witness=%d", c.Witnesses[0]) + } else if c.Witnesses[1] > 0 { + fmt.Fprintf(&buf, " witness=%d", c.Witnesses[1]) + } if c.Learners != nil { fmt.Fprintf(&buf, " learners=%s", quorum.MajorityConfig(c.Learners).String()) } @@ -108,7 +117,61 @@ func (c *Config) Clone() Config { Voters: quorum.JointConfig{clone(c.Voters[0]), clone(c.Voters[1])}, Learners: clone(c.Learners), LearnersNext: clone(c.LearnersNext), + Witnesses: [2]uint64{c.Witnesses[0], c.Witnesses[1]}, + } +} + +type ReplicationSet struct { + Witness uint64 + Excluded uint64 + NonWitnessVoters map[uint64]struct{} +} + +func (r *ReplicationSet) Clone() ReplicationSet { + result := ReplicationSet{ + Witness: r.Witness, + Excluded: r.Excluded, + } + return result +} + +func (r *ReplicationSet) GetNonWitnessVoterSlice() []uint64 { + results := []uint64{} + for v := range r.NonWitnessVoters { + results = append(results, v) + } + return results +} + +func (r *ReplicationSet) String() string { + ids := []uint64{} + for id := range r.NonWitnessVoters { + ids = append(ids, id) } + if r.Excluded != r.Witness { + ids = append(ids, r.Witness) + } + + sort.Slice(ids, func(i, j int) bool { return ids[i] < ids[j] }) + + replSet := make([]string, 0, 2) + for _, id := range ids { + replSet = append(replSet, strconv.FormatUint(id, 16)) + } + return fmt.Sprintf("{%s}", strings.Join(replSet, ",")) +} + +type Epoch struct { + Subterm uint64 + ReplicationSet [2]ReplicationSet +} + +func (e *Epoch) HasWitness() bool { + return e.ReplicationSet[0].Witness != 0 || e.ReplicationSet[1].Witness != 0 +} + +func (e *Epoch) ReplicateToWitness() (bool, bool) { + return (e.ReplicationSet[0].Witness != 0 && e.ReplicationSet[0].Excluded != e.ReplicationSet[0].Witness), (e.ReplicationSet[1].Witness != 0 && e.ReplicationSet[1].Excluded != e.ReplicationSet[1].Witness) } // ProgressTracker tracks the currently active configuration and the information @@ -123,6 +186,8 @@ type ProgressTracker struct { MaxInflight int MaxInflightBytes uint64 + + Epoch *Epoch } // MakeProgressTracker initializes a ProgressTracker. @@ -137,9 +202,14 @@ func MakeProgressTracker(maxInflight int, maxBytes uint64) ProgressTracker { }, Learners: nil, // only populated when used LearnersNext: nil, // only populated when used + Witnesses: [2]uint64{0, 0}, }, Votes: map[uint64]bool{}, Progress: map[uint64]*Progress{}, + Epoch: &Epoch{ + Subterm: 0, + ReplicationSet: [2]ReplicationSet{}, + }, } return p } @@ -147,11 +217,13 @@ func MakeProgressTracker(maxInflight int, maxBytes uint64) ProgressTracker { // ConfState returns a ConfState representing the active configuration. func (p *ProgressTracker) ConfState() pb.ConfState { return pb.ConfState{ - Voters: p.Voters[0].Slice(), - VotersOutgoing: p.Voters[1].Slice(), - Learners: quorum.MajorityConfig(p.Learners).Slice(), - LearnersNext: quorum.MajorityConfig(p.LearnersNext).Slice(), - AutoLeave: p.AutoLeave, + Voters: p.Voters[0].Slice(), + VotersOutgoing: p.Voters[1].Slice(), + Learners: quorum.MajorityConfig(p.Learners).Slice(), + LearnersNext: quorum.MajorityConfig(p.LearnersNext).Slice(), + AutoLeave: p.AutoLeave, + Witness: p.Witnesses[0], + WitnessOutgoing: p.Witnesses[1], } } @@ -174,12 +246,48 @@ func (l matchAckIndexer) AckedIndex(id uint64) (quorum.Index, bool) { return quorum.Index(pr.Match), true } +type scopedAckIndexer struct { + indexer quorum.AckedIndexer + scope map[uint64]struct{} +} + +func (s scopedAckIndexer) AckedIndex(id uint64) (quorum.Index, bool) { + if _, ok := s.scope[id]; !ok { + return 0, false + } + return s.indexer.AckedIndex(id) +} + // Committed returns the largest log index known to be committed based on what // the voting members of the group have acknowledged. func (p *ProgressTracker) Committed() uint64 { return uint64(p.Voters.CommittedIndex(matchAckIndexer(p.Progress))) } +func (p *ProgressTracker) OneLessThanQuorumInReplicationSet() map[uint64]quorum.Index { + w1, w2 := p.Epoch.ReplicateToWitness() + if !w1 && !w2 { + return nil + } + + epoch := p.Epoch + indexer := scopedAckIndexer{indexer: matchAckIndexer(p.Progress), scope: epoch.ReplicationSet[0].NonWitnessVoters} + result := map[uint64]quorum.Index{} + + if w1 { + result[epoch.ReplicationSet[0].Witness] = p.Voters[0].OneLessThanQuorum(indexer) + } + if w2 { + indexer.scope = epoch.ReplicationSet[1].NonWitnessVoters + idx := p.Voters[1].OneLessThanQuorum(indexer) + if idx1, ok := result[epoch.ReplicationSet[1].Witness]; !ok || idx < idx1 { + result[epoch.ReplicationSet[1].Witness] = idx + } + } + + return result +} + func insertionSort(sl []uint64) { a, b := 0, len(sl) for i := a + 1; i < b; i++ { @@ -288,3 +396,121 @@ func (p *ProgressTracker) TallyVotes() (granted int, rejected int, _ quorum.Vote result := p.Voters.VoteResult(p.Votes) return granted, rejected, result } + +func (p *ProgressTracker) TallyVotesWithDifference() (granted int, rejected int, _ quorum.VoteResult, votesToWin [2]int) { + // Make sure to populate granted/rejected correctly even if the Votes slice + // contains members no longer part of the configuration. This doesn't really + // matter in the way the numbers are used (they're informational), but might + // as well get it right. + for id, pr := range p.Progress { + if pr.IsLearner { + continue + } + v, voted := p.Votes[id] + if !voted { + continue + } + if v { + granted++ + } else { + rejected++ + } + } + result, votesToWin := p.Voters.VoteResultWithDifference(p.Votes) + return granted, rejected, result, votesToWin +} + +func (p *ProgressTracker) ResetReplicationSet(resetSubterm bool) { + epoch := &Epoch{ + Subterm: 0, + ReplicationSet: [2]ReplicationSet{}, + } + if !resetSubterm { + epoch.Subterm = p.Epoch.Subterm + 1 + } + + for i, c := range p.Voters { + set := &epoch.ReplicationSet[i] + set.Excluded = 0 + set.Witness = 0 + set.NonWitnessVoters = map[uint64]struct{}{} + for id := range c { + if p.Witnesses[i] == id { + set.Excluded = id + set.Witness = id + } else { + set.NonWitnessVoters[id] = struct{}{} + } + } + } + + p.Epoch = epoch +} + +func (p *ProgressTracker) ChangeReplicationSet() bool { + epoch := p.Epoch + if epoch == nil { + return false + } + + result := &Epoch{ + Subterm: p.Epoch.Subterm + 1, + ReplicationSet: [2]ReplicationSet{epoch.ReplicationSet[0], epoch.ReplicationSet[1]}, + } + changed := false + for i, set := range epoch.ReplicationSet { + if set.Excluded == 0 { + continue + } + + isExcludedReady := + p.Progress[set.Excluded].RecentActive && + (set.Excluded == set.Witness || p.Progress[set.Excluded].State == StateReplicate) + if !isExcludedReady { + continue + } + + inactiveID := uint64(0) + for id, pr := range p.Progress { + if !pr.RecentActive { + if _, ok := set.NonWitnessVoters[id]; ok || id == set.Witness { + inactiveID = id + break + } + } + } + if inactiveID == 0 && set.Excluded == set.Witness { + continue + } + + changed = true + + if inactiveID > 0 { + result.ReplicationSet[i] = ReplicationSet{ + Witness: p.Epoch.ReplicationSet[i].Witness, + Excluded: inactiveID, + NonWitnessVoters: map[uint64]struct{}{}, + } + } else { + result.ReplicationSet[i] = ReplicationSet{ + Witness: p.Epoch.ReplicationSet[i].Witness, + Excluded: p.Epoch.ReplicationSet[i].Witness, + NonWitnessVoters: map[uint64]struct{}{}, + } + } + for id := range set.NonWitnessVoters { + if id != result.ReplicationSet[i].Excluded { + result.ReplicationSet[i].NonWitnessVoters[id] = struct{}{} + } + } + if set.Excluded != result.ReplicationSet[i].Excluded && set.Excluded != set.Witness { + result.ReplicationSet[i].NonWitnessVoters[set.Excluded] = struct{}{} + } + } + + if changed { + p.Epoch = result + } + + return changed +} diff --git a/util.go b/util.go index 8f781788..c5f7eed8 100644 --- a/util.go +++ b/util.go @@ -94,8 +94,8 @@ func DescribeSoftState(ss SoftState) string { func DescribeConfState(state pb.ConfState) string { return fmt.Sprintf( - "Voters:%v VotersOutgoing:%v Learners:%v LearnersNext:%v AutoLeave:%v", - state.Voters, state.VotersOutgoing, state.Learners, state.LearnersNext, state.AutoLeave, + "Voters:%v VotersOutgoing:%v Learners:%v LearnersNext:%v Witness:%v WitnessOutgoing:%v AutoLeave:%v", + state.Voters, state.VotersOutgoing, state.Learners, state.LearnersNext, state.Witness, state.WitnessOutgoing, state.AutoLeave, ) } @@ -188,6 +188,26 @@ func describeMessageWithIndent(indent string, m pb.Message, f EntryFormatter) st return buf.String() } +func DescribeWitnessMessage(m WitnessMessage, f EntryFormatter) string { + var buf bytes.Buffer + fmt.Fprintf(&buf, "%s->%s %v Term:%d Log:%d/%d/%d", + describeTarget(m.From), describeTarget(m.To), m.Type, m.Term, m.LastLogTerm, m.LastLogSubterm, m.LastLogIndex) + for i, rs := range m.ReplicationSet { + if len(rs) > 0 { + fmt.Fprintf(&buf, " ReplicationSet[%d]:[", i) + for j, r := range rs { + if j != 0 { + buf.WriteString(", ") + } + buf.WriteString(describeTarget(r)) + } + fmt.Fprint(&buf, "]") + } + } + + return buf.String() +} + func describeTarget(id uint64) string { switch id { case None: diff --git a/witness.go b/witness.go new file mode 100644 index 00000000..bb1a6201 --- /dev/null +++ b/witness.go @@ -0,0 +1,280 @@ +package raft + +import ( + "errors" + + "go.etcd.io/raft/v3/raftpb" +) + +var ErrWitnessStateModified = errors.New("witness state was modified") + +type WitnessStorage interface { + GetWitnessState() (WitnessState, error) + Close() +} + +type WitnessState interface { + GetState() raftpb.WitnessHardState + Update(st *raftpb.WitnessHardState) error +} + +type WitnessConfig struct { + ID uint64 + Storage WitnessStorage + CheckQuorum bool + PreVote bool +} + +type Witness interface { + Process(m WitnessMessage) (*raftpb.Message, error) + Close() +} + +type witness struct { + id uint64 + storage WitnessStorage + preVote bool + checkQuorum bool + state raftpb.WitnessHardState + lastError error +} + +func NewWitness(cfg WitnessConfig) Witness { + return &witness{ + id: cfg.ID, + storage: cfg.Storage, + preVote: cfg.PreVote, + checkQuorum: cfg.CheckQuorum, + } +} + +func (w *witness) Close() { + w.storage.Close() +} + +func (w *witness) Process(m WitnessMessage) (*raftpb.Message, error) { + switch m.Type { + case raftpb.MsgApp, raftpb.MsgVote, raftpb.MsgPreVote, raftpb.MsgHeartbeat: + default: + return nil, nil + } + + resp, handled := w.processNoUpdate(m) + if handled { + return resp, nil + } + + for { + state, err := w.storage.GetWitnessState() + w.lastError = err + if err != nil { + return nil, err + } + w.state = state.GetState() + + resp, handled = w.processNoUpdate(m) + if handled { + return resp, nil + } + + st := w.cloneState() + if m.Term > st.State.Term { + w.processMessageWithNewerTerm(m, &st) + } + switch m.Type { + case raftpb.MsgHeartbeat: + resp = w.handleHeartbeat(m) + case raftpb.MsgApp: + resp = w.handleAppend(m, &st) + case raftpb.MsgPreVote, raftpb.MsgVote: + resp = w.handleVote(m, &st) + default: + return nil, nil + } + + err = state.Update(&st) + if err != ErrWitnessStateModified { + w.lastError = err + if err != nil { + return nil, err + } + return resp, nil + } + } +} + +func (w *witness) isHealthy() bool { + return w.lastError == nil +} + +func (w *witness) processNoUpdate(m WitnessMessage) (*raftpb.Message, bool) { + if !w.isHealthy() && m.Type == raftpb.MsgHeartbeat { + return nil, true + } + + if m.Term < w.state.State.Term { + resp := w.processMessageWithOlderTerm(m) + return resp, true + } + + var resp *raftpb.Message + handled := false + if m.Term == w.state.State.Term { + switch m.Type { + case raftpb.MsgHeartbeat: + resp = w.handleHeartbeat(m) + handled = true + case raftpb.MsgApp: + resp, handled = w.handleAppendNoUpdate(m) + } + } + + return resp, handled +} + +func (w *witness) processMessageWithOlderTerm(m WitnessMessage) *raftpb.Message { + var resp *raftpb.Message + if (w.checkQuorum || w.preVote) && (m.Type == raftpb.MsgHeartbeat || m.Type == raftpb.MsgApp) { + resp = &raftpb.Message{To: m.From, Type: raftpb.MsgAppResp} + } else if m.Type == raftpb.MsgPreVote { + resp = &raftpb.Message{To: m.From, Term: w.state.State.Term, Type: raftpb.MsgPreVoteResp, Reject: true} + } else { + resp = nil + } + + return resp +} + +func (w *witness) processMessageWithNewerTerm(m WitnessMessage, st *raftpb.WitnessHardState) { + switch { + case m.Type == raftpb.MsgPreVote: + default: + if m.Type == raftpb.MsgApp || m.Type == raftpb.MsgHeartbeat || m.Type == raftpb.MsgSnap { + st.State.Term = m.Term + st.State.Vote = None + st.Lead = m.From + } else { + st.State.Term = m.Term + st.State.Vote = None + st.Lead = None + } + } +} + +func (w *witness) cloneState() raftpb.WitnessHardState { + st := raftpb.WitnessHardState{ + State: raftpb.HardState{ + Term: w.state.State.Term, + Vote: w.state.State.Vote, + Commit: w.state.State.Commit, + }, + LastLogIndex: w.state.LastLogIndex, + LastLogTerm: w.state.LastLogTerm, + LastLogSubterm: w.state.LastLogSubterm, + Lead: w.state.Lead, + ReplicationSet: make([]uint64, len(w.state.ReplicationSet)), + ReplicationSetOutgoing: make([]uint64, len(w.state.ReplicationSetOutgoing)), + } + copy(st.ReplicationSet, w.state.ReplicationSet) + copy(st.ReplicationSetOutgoing, w.state.ReplicationSetOutgoing) + + return st +} + +func (w *witness) handleHeartbeat(m WitnessMessage) *raftpb.Message { + return &raftpb.Message{ + Type: raftpb.MsgHeartbeatResp, + Term: w.state.State.Term, + From: w.id, + To: m.From, + Context: m.Context, + } +} + +func (w *witness) handleAppendNoUpdate(m WitnessMessage) (*raftpb.Message, bool) { + resp := &raftpb.Message{ + Type: raftpb.MsgAppResp, + To: m.From, + From: w.id, + Term: w.state.State.Term, + } + + handled := true + if m.LastLogTerm == w.state.LastLogTerm && m.LastLogSubterm == w.state.LastLogSubterm { + // leader commits upon quorum-1 acks if the replicated entry is of same subterm as witness + resp.Index = m.LastLogIndex + } else if m.LastLogTerm < w.state.LastLogTerm || (m.LastLogTerm == w.state.LastLogTerm && m.LastLogSubterm < w.state.LastLogSubterm) { + // previous replicated entry + resp.Index = w.state.LastLogIndex + } else { + handled = false + } + + return resp, handled +} + +func (w *witness) handleAppend(m WitnessMessage, st *raftpb.WitnessHardState) *raftpb.Message { + resp, handled := w.handleAppendNoUpdate(m) + if handled { + return resp + } + + st.LastLogTerm = m.LastLogTerm + st.LastLogSubterm = m.LastLogSubterm + st.LastLogIndex = m.LastLogIndex + st.ReplicationSet = m.ReplicationSet[0] + st.ReplicationSetOutgoing = m.ReplicationSet[1] + + return &raftpb.Message{ + Type: raftpb.MsgAppResp, + To: m.From, + From: w.id, + Term: w.state.State.Term, + Index: st.LastLogIndex, + } +} + +func (w *witness) handleVote(m WitnessMessage, st *raftpb.WitnessHardState) *raftpb.Message { + var msgType raftpb.MessageType + if m.Type == raftpb.MsgPreVote { + msgType = raftpb.MsgPreVoteResp + } else { + msgType = raftpb.MsgVoteResp + } + resp := &raftpb.Message{ + Type: msgType, + To: m.From, + From: w.id, + } + + canVote := st.State.Vote == m.From || (st.State.Vote == 0 && st.Lead == 0) || (m.Type == raftpb.MsgPreVote && m.Term > st.State.Term) + reject := true + if canVote { + reject = false + if m.LastLogTerm < st.LastLogTerm || (m.LastLogTerm == st.LastLogTerm && m.LastLogSubterm < st.LastLogSubterm) { + reject = true + } else if m.LastLogTerm == st.LastLogTerm && m.LastLogSubterm == st.LastLogSubterm { + votesInSet := 0 + for _, x := range st.ReplicationSet { + if _, ok := m.Votes[x]; ok { + votesInSet++ + } + } + reject = votesInSet != len(m.Votes) + } + } + + if m.Type == raftpb.MsgVote && !reject { + st.State.Vote = m.From + } + + resp.Reject = reject + + if resp.Reject { + resp.Term = st.State.Term + } else { + resp.Term = m.Term + } + + return resp +} From 43f19c68a57c2317805c76d90351c07327c89ffa Mon Sep 17 00:00:00 2001 From: ZhouJianMS Date: Tue, 20 Feb 2024 15:20:06 +0800 Subject: [PATCH 2/2] Fix test and lint Signed-off-by: ZhouJianMS --- raft.go | 20 ----- raft_test.go | 24 +++--- testdata/campaign_learner_must_vote.txt | 10 ++- testdata/checkquorum.txt | 8 +- testdata/confchange_disable_validation.txt | 30 ++++++- testdata/confchange_v1_add_single.txt | 16 ++-- .../confchange_v1_remove_leader_stepdown.txt | 6 +- testdata/confchange_v2_add_double_auto.txt | 44 +++++++--- .../confchange_v2_add_double_implicit.txt | 26 ++++-- testdata/confchange_v2_add_single_auto.txt | 16 ++-- .../confchange_v2_add_single_explicit.txt | 16 ++-- testdata/confchange_v2_replace_leader.txt | 22 +++-- .../confchange_v2_replace_leader_stepdown.txt | 82 ++++++++++--------- .../forget_leader_prevote_checkquorum.txt | 10 +-- testdata/prevote.txt | 12 +-- testdata/prevote_checkquorum.txt | 24 +++--- testdata/snapshot_succeed_via_app_resp.txt | 6 +- .../snapshot_succeed_via_app_resp_behind.txt | 10 +-- util.go | 2 +- 19 files changed, 231 insertions(+), 153 deletions(-) diff --git a/raft.go b/raft.go index ffa55867..cadbc19c 100644 --- a/raft.go +++ b/raft.go @@ -1177,16 +1177,6 @@ func (r *raft) campaign(t CampaignType) { } } -func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int, rejected int, result quorum.VoteResult) { - if v { - r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term) - } else { - r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term) - } - r.trk.RecordVote(id, v) - return r.trk.TallyVotes() -} - func (r *raft) pollAndReportDiff(id uint64, t pb.MessageType, v bool) (granted int, rejected int, result quorum.VoteResult, votesToWin [2]int) { if v { r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term) @@ -2284,16 +2274,6 @@ func (r *raft) maybeStartNewSubterm(newTerm bool, confChange bool) bool { return true } -func numOfPendingConf(ents []pb.Entry) int { - n := 0 - for i := range ents { - if ents[i].Type == pb.EntryConfChange || ents[i].Type == pb.EntryConfChangeV2 { - n++ - } - } - return n -} - func releasePendingReadIndexMessages(r *raft) { if len(r.pendingReadIndexMessages) == 0 { // Fast path for the common case to avoid a call to storage.LastIndex() diff --git a/raft_test.go b/raft_test.go index 6fb66486..35d2ee68 100644 --- a/raft_test.go +++ b/raft_test.go @@ -3409,13 +3409,11 @@ func TestRemoveNode(t *testing.T) { t.Errorf("nodes = %v, want %v", g, w) } - // Removing the remaining voter will panic. - defer func() { - if r := recover(); r == nil { - t.Error("did not panic") - } - }() - r.applyConfChange(pb.ConfChange{NodeID: 1, Type: pb.ConfChangeRemoveNode}.AsV2()) + _, err := r.applyConfChange(pb.ConfChange{NodeID: 1, Type: pb.ConfChangeRemoveNode}.AsV2()) + // Removing the remaining voter will return error "removed all voters". + if err.Error() != "removed all voters" { + t.Errorf("Removing the remaining voter will return error") + } } // TestRemoveLearner tests that removeNode could update nodes and @@ -3433,13 +3431,11 @@ func TestRemoveLearner(t *testing.T) { t.Errorf("nodes = %v, want %v", g, w) } - // Removing the remaining voter will panic. - defer func() { - if r := recover(); r == nil { - t.Error("did not panic") - } - }() - r.applyConfChange(pb.ConfChange{NodeID: 1, Type: pb.ConfChangeRemoveNode}.AsV2()) + _, err := r.applyConfChange(pb.ConfChange{NodeID: 1, Type: pb.ConfChangeRemoveNode}.AsV2()) + // Removing the remaining voter will return error "removed all voters". + if err.Error() != "removed all voters" { + t.Errorf("Removing the remaining voter will return error") + } } func TestPromotable(t *testing.T) { diff --git a/testdata/campaign_learner_must_vote.txt b/testdata/campaign_learner_must_vote.txt index 533b7eb5..379bd6cd 100644 --- a/testdata/campaign_learner_must_vote.txt +++ b/testdata/campaign_learner_must_vote.txt @@ -119,14 +119,16 @@ stabilize 2 3 > 2 handling Ready Ready MustSync=false: Messages: - 2->3 MsgApp Term:2 Log:1/3 Commit:4 Entries:[ + 2->3 MsgApp Term:2 Log:1/3 Commit:5 Entries:[ 1/4 EntryConfChangeV2 v3 - 2/5 EntryNormal "" + 1/5 EntryNormal "" + 2/6 EntryNormal "" ] > 3 receiving messages - 2->3 MsgApp Term:2 Log:1/3 Commit:4 Entries:[ + 2->3 MsgApp Term:2 Log:1/3 Commit:5 Entries:[ 1/4 EntryConfChangeV2 v3 - 2/5 EntryNormal "" + 1/5 EntryNormal "" + 2/6 EntryNormal "" ] > 3 handling Ready Ready MustSync=true: diff --git a/testdata/checkquorum.txt b/testdata/checkquorum.txt index b25c1e63..ae606329 100644 --- a/testdata/checkquorum.txt +++ b/testdata/checkquorum.txt @@ -42,7 +42,7 @@ stabilize 2->1 MsgVote Term:2 Log:1/11 2->3 MsgVote Term:2 Log:1/11 INFO 2 received MsgVoteResp from 2 at term 2 - INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 2->1 MsgVote Term:2 Log:1/11 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgVote from 2 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3) @@ -147,7 +147,7 @@ Messages: 2->1 MsgVote Term:3 Log:1/11 2->3 MsgVote Term:3 Log:1/11 INFO 2 received MsgVoteResp from 2 at term 3 -INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections +INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win deliver-msgs 1 ---- @@ -171,7 +171,9 @@ stabilize > 2 receiving messages 1->2 MsgVoteResp Term:3 Log:0/0 INFO 2 received MsgVoteResp from 1 at term 3 - INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 2 resets replication set to {1,2,3}, {} + INFO 2 starts new subterm. Term: 3, Subterm: 0 INFO 2 became leader at term 3 > 2 handling Ready Ready MustSync=true: diff --git a/testdata/confchange_disable_validation.txt b/testdata/confchange_disable_validation.txt index 1a2bc4fd..d69dc1f9 100644 --- a/testdata/confchange_disable_validation.txt +++ b/testdata/confchange_disable_validation.txt @@ -64,13 +64,37 @@ stabilize 1/6 EntryConfChangeV2 CommittedEntries: 1/5 EntryConfChangeV2 l2 l3 + INFO 1 resets replication set to {1}, {1} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(1)&&(1) learners=(2 3) > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: HardState Term:1 Vote:1 Commit:6 + Entries: + 1/7 EntryNormal "" CommittedEntries: 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] - 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[ + 1/6 EntryConfChangeV2 + 1/7 EntryNormal "" + ] + 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[ + 1/6 EntryConfChangeV2 + 1/7 EntryNormal "" + ] + INFO 1 resets replication set to {1}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 2 INFO 1 switched to configuration voters=(1) learners=(2 3) +> 1 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:7 + Entries: + 1/8 EntryNormal "" + CommittedEntries: + 1/7 EntryNormal "" +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:8 + CommittedEntries: + 1/8 EntryNormal "" diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index 1cc9fb10..d3b7c67f 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -61,9 +61,15 @@ stabilize Entries: 1/5 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChange v2 + 1/5 EntryNormal "" + ] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChange v2 + 1/5 EntryNormal "" + ] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -83,10 +89,10 @@ stabilize Ready MustSync=false: Messages: 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] @@ -94,7 +100,7 @@ stabilize > 2 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 - Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages diff --git a/testdata/confchange_v1_remove_leader_stepdown.txt b/testdata/confchange_v1_remove_leader_stepdown.txt index fe397650..a5556308 100644 --- a/testdata/confchange_v1_remove_leader_stepdown.txt +++ b/testdata/confchange_v1_remove_leader_stepdown.txt @@ -104,11 +104,15 @@ stabilize 1 1->3 MsgApp Term:1 Log:1/6 Commit:4 1->2 MsgApp Term:1 Log:1/6 Commit:5 1->3 MsgApp Term:1 Log:1/6 Commit:5 + INFO 1 resets replication set to {2,3}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 1 INFO 1 switched to configuration voters=(2 3) INFO 1 became follower at term 1 > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: Lead:0 State:StateFollower + Entries: + 1/7 EntryNormal "" raft-state ---- diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index 0cf15765..7d3cdefa 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -74,8 +74,14 @@ stabilize 1 1/5 EntryNormal "" 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] - 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 v3 + 1/5 EntryNormal "" + ] + 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 v3 + 1/5 EntryNormal "" + ] # First, play out the whole interaction between n1 and n2. We see n1's probe to # n2 get rejected (since n2 needs a snapshot); the snapshot is delivered at which @@ -85,7 +91,10 @@ stabilize 1 stabilize 1 2 ---- > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 v3 + 1/5 EntryNormal "" + ] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -105,10 +114,10 @@ stabilize 1 2 Ready MustSync=false: Messages: 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:true INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2 3)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] @@ -116,7 +125,7 @@ stabilize 1 2 > 2 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 - Snapshot Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true + Snapshot Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:true Messages: 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages @@ -125,9 +134,15 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[ + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 + ] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[ + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 + ] > 2 handling Ready Ready MustSync=true: Entries: @@ -200,7 +215,10 @@ stabilize 1 2 stabilize 1 3 ---- > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3, 1/5 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 v3 + 1/5 EntryNormal "" + ] INFO 3 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 3 became follower at term 1 DEBUG 3 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -220,18 +238,18 @@ stabilize 1 3 Ready MustSync=false: Messages: 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:7 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] + Snapshot: Index:7 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 7, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 7, lastindex: 7, lastterm: 1] restored snapshot [index: 7, term: 1] INFO 3 [commit: 7] restored snapshot [index: 7, term: 1] > 3 handling Ready Ready MustSync=false: HardState Term:1 Commit:7 - Snapshot Index:7 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:7 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/7 > 1 receiving messages diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index 6671e11d..acc5b0d4 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -66,9 +66,15 @@ stabilize 1 2 1/5 EntryNormal "" 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 + 1/5 EntryNormal "" + ] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 + 1/5 EntryNormal "" + ] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -88,10 +94,10 @@ stabilize 1 2 Ready MustSync=false: Messages: 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:true INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] @@ -99,7 +105,7 @@ stabilize 1 2 > 2 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 - Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true + Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:true Messages: 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages @@ -108,9 +114,15 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[ + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 + ] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal "", 1/6 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[ + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 + ] > 2 handling Ready Ready MustSync=true: Entries: diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index 9df33fab..60710adf 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -62,9 +62,15 @@ stabilize Entries: 1/5 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 + 1/5 EntryNormal "" + ] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 + 1/5 EntryNormal "" + ] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -84,10 +90,10 @@ stabilize Ready MustSync=false: Messages: 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] @@ -95,7 +101,7 @@ stabilize > 2 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 - Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index ae53123f..34d378ae 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -62,9 +62,15 @@ stabilize 1 2 Entries: 1/5 EntryNormal "" Messages: - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 + 1/5 EntryNormal "" + ] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2, 1/5 EntryNormal ""] + 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 v2 + 1/5 EntryNormal "" + ] INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 2 became follower at term 1 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 @@ -84,10 +90,10 @@ stabilize 1 2 Ready MustSync=false: Messages: 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] @@ -95,7 +101,7 @@ stabilize 1 2 > 2 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 - Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index daea413f..9ed60ec8 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -97,7 +97,10 @@ stabilize Messages: 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] 1->3 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] - 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4, 1/5 EntryNormal ""] + 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 r1 v4 + 1/5 EntryNormal "" + ] > 2 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:4 @@ -122,7 +125,10 @@ stabilize > 3 receiving messages 1->3 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] > 4 receiving messages - 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4, 1/5 EntryNormal ""] + 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ + 1/4 EntryConfChangeV2 r1 v4 + 1/5 EntryNormal "" + ] INFO 4 [term: 0] received a MsgApp message with higher term from 1 [term: 1] INFO 4 became follower at term 1 > 2 handling Ready @@ -153,11 +159,17 @@ stabilize CommittedEntries: 1/5 EntryNormal "" Messages: + 1->2 MsgApp Term:1 Log:1/5 Commit:5 + 1->3 MsgApp Term:1 Log:1/5 Commit:5 1->4 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/5 Commit:5 +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/5 Commit:5 > 4 receiving messages 1->4 MsgSnap Term:1 Log:0/0 - Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 4 switched to configuration voters=(2 3 4)&&(1 2 3) INFO 4 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] @@ -179,7 +191,7 @@ stabilize > 4 handling Ready Ready MustSync=false: HardState Term:1 Commit:4 - Snapshot Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 4->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages diff --git a/testdata/confchange_v2_replace_leader_stepdown.txt b/testdata/confchange_v2_replace_leader_stepdown.txt index 62d01d23..16d639ed 100644 --- a/testdata/confchange_v2_replace_leader_stepdown.txt +++ b/testdata/confchange_v2_replace_leader_stepdown.txt @@ -77,87 +77,91 @@ stabilize > 1 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] - 1->3 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] - 1->4 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] + 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] + 1->4 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] > 4 receiving messages - 1->4 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->4 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 > 3 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/6 > 4 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 4->1 MsgAppResp Term:1 Log:0/5 + 4->1 MsgAppResp Term:1 Log:0/6 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/5 - 4->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/6 + 4->1 MsgAppResp Term:1 Log:0/6 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/5 Commit:5 - 1->3 MsgApp Term:1 Log:1/5 Commit:5 - 1->4 MsgApp Term:1 Log:1/5 Commit:5 + 1->2 MsgApp Term:1 Log:1/6 Commit:6 + 1->3 MsgApp Term:1 Log:1/6 Commit:6 + 1->4 MsgApp Term:1 Log:1/6 Commit:6 + INFO 1 resets replication set to {2,3,4}, {} + INFO 1 starts new subterm. Term: 1, Subterm: 2 INFO 1 switched to configuration voters=(2 3 4) INFO 1 became follower at term 1 > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/5 Commit:5 + 1->2 MsgApp Term:1 Log:1/6 Commit:6 > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/5 Commit:5 + 1->3 MsgApp Term:1 Log:1/6 Commit:6 > 4 receiving messages - 1->4 MsgApp Term:1 Log:1/5 Commit:5 + 1->4 MsgApp Term:1 Log:1/6 Commit:6 > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: Lead:0 State:StateFollower + Entries: + 1/7 EntryNormal "" > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 INFO 2 switched to configuration voters=(2 3 4) > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/6 INFO 3 switched to configuration voters=(2 3 4) > 4 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:6 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/6 EntryConfChangeV2 Messages: - 4->1 MsgAppResp Term:1 Log:0/5 + 4->1 MsgAppResp Term:1 Log:0/6 INFO 4 switched to configuration voters=(2 3 4) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/5 - 4->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/6 + 4->1 MsgAppResp Term:1 Log:0/6 # n1 is out of the configuration. raft-state @@ -177,8 +181,8 @@ campaign 2 ---- INFO 2 is starting a new election at term 1 INFO 2 became candidate at term 2 -INFO 2 [logterm: 1, index: 5] sent MsgVote request to 3 at term 2 -INFO 2 [logterm: 1, index: 5] sent MsgVote request to 4 at term 2 +INFO 2 [logterm: 1, index: 6] sent MsgVote request to 3 at term 2 +INFO 2 [logterm: 1, index: 6] sent MsgVote request to 4 at term 2 stabilize log-level=none ---- diff --git a/testdata/forget_leader_prevote_checkquorum.txt b/testdata/forget_leader_prevote_checkquorum.txt index 9b3b80ff..c5c5bf05 100644 --- a/testdata/forget_leader_prevote_checkquorum.txt +++ b/testdata/forget_leader_prevote_checkquorum.txt @@ -40,7 +40,7 @@ stabilize 3 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 INFO 3 received MsgPreVoteResp from 3 at term 1 - INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections + INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win deliver-msgs 1 2 ---- @@ -113,7 +113,7 @@ stabilize 3 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 INFO 3 received MsgPreVoteResp from 3 at term 1 - INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections + INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win stabilize 2 ---- @@ -133,7 +133,7 @@ stabilize 3 > 3 receiving messages 2->3 MsgPreVoteResp Term:2 Log:0/0 INFO 3 received MsgPreVoteResp from 2 at term 1 - INFO 3 has received 2 MsgPreVoteResp votes and 0 vote rejections + INFO 3 has received 2 MsgPreVoteResp votes and 0 vote rejections. It needs [0 0] votes to win INFO 3 became candidate at term 2 INFO 3 [logterm: 1, index: 11] sent MsgVote request to 1 at term 2 INFO 3 [logterm: 1, index: 11] sent MsgVote request to 2 at term 2 @@ -145,7 +145,7 @@ stabilize 3 3->1 MsgVote Term:2 Log:1/11 3->2 MsgVote Term:2 Log:1/11 INFO 3 received MsgVoteResp from 3 at term 2 - INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win stabilize log-level=none ---- @@ -209,7 +209,7 @@ Messages: 1->2 MsgPreVote Term:3 Log:2/12 1->3 MsgPreVote Term:3 Log:2/12 INFO 1 received MsgPreVoteResp from 1 at term 2 -INFO 1 has received 1 MsgPreVoteResp votes and 0 vote rejections +INFO 1 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win stabilize 2 ---- diff --git a/testdata/prevote.txt b/testdata/prevote.txt index db763d35..d5135f0e 100644 --- a/testdata/prevote.txt +++ b/testdata/prevote.txt @@ -71,7 +71,7 @@ Messages: 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 INFO 3 received MsgPreVoteResp from 3 at term 1 -INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections +INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win deliver-msgs 1 2 ---- @@ -145,7 +145,7 @@ stabilize 2->1 MsgPreVote Term:2 Log:1/12 2->3 MsgPreVote Term:2 Log:1/12 INFO 2 received MsgPreVoteResp from 2 at term 1 - INFO 2 has received 1 MsgPreVoteResp votes and 0 vote rejections + INFO 2 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 2->1 MsgPreVote Term:2 Log:1/12 INFO 1 [logterm: 1, index: 12, vote: 1] cast MsgPreVote for 2 [logterm: 1, index: 12] at term 1 @@ -163,7 +163,7 @@ stabilize > 2 receiving messages 1->2 MsgPreVoteResp Term:2 Log:0/0 INFO 2 received MsgPreVoteResp from 1 at term 1 - INFO 2 has received 2 MsgPreVoteResp votes and 0 vote rejections + INFO 2 has received 2 MsgPreVoteResp votes and 0 vote rejections. It needs [0 0] votes to win INFO 2 became candidate at term 2 INFO 2 [logterm: 1, index: 12] sent MsgVote request to 1 at term 2 INFO 2 [logterm: 1, index: 12] sent MsgVote request to 3 at term 2 @@ -176,7 +176,7 @@ stabilize 2->1 MsgVote Term:2 Log:1/12 2->3 MsgVote Term:2 Log:1/12 INFO 2 received MsgVoteResp from 2 at term 2 - INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 2->1 MsgVote Term:2 Log:1/12 INFO 1 [term: 1] received a MsgVote message with higher term from 2 [term: 2] @@ -202,7 +202,9 @@ stabilize > 2 receiving messages 1->2 MsgVoteResp Term:2 Log:0/0 INFO 2 received MsgVoteResp from 1 at term 2 - INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 2 resets replication set to {1,2,3}, {} + INFO 2 starts new subterm. Term: 2, Subterm: 0 INFO 2 became leader at term 2 3->2 MsgVoteResp Term:2 Log:0/0 > 2 handling Ready diff --git a/testdata/prevote_checkquorum.txt b/testdata/prevote_checkquorum.txt index 6db6662b..6387f151 100644 --- a/testdata/prevote_checkquorum.txt +++ b/testdata/prevote_checkquorum.txt @@ -40,7 +40,7 @@ stabilize 2->1 MsgPreVote Term:2 Log:1/11 2->3 MsgPreVote Term:2 Log:1/11 INFO 2 received MsgPreVoteResp from 2 at term 1 - INFO 2 has received 1 MsgPreVoteResp votes and 0 vote rejections + INFO 2 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 2->1 MsgPreVote Term:2 Log:1/11 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgPreVote from 2 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3) @@ -73,7 +73,7 @@ Messages: 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 INFO 3 received MsgPreVoteResp from 3 at term 1 -INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections +INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win deliver-msgs 2 ---- @@ -94,7 +94,7 @@ stabilize > 3 receiving messages 2->3 MsgPreVoteResp Term:2 Log:0/0 INFO 3 received MsgPreVoteResp from 2 at term 1 - INFO 3 has received 2 MsgPreVoteResp votes and 0 vote rejections + INFO 3 has received 2 MsgPreVoteResp votes and 0 vote rejections. It needs [0 0] votes to win INFO 3 became candidate at term 2 INFO 3 [logterm: 1, index: 11] sent MsgVote request to 1 at term 2 INFO 3 [logterm: 1, index: 11] sent MsgVote request to 2 at term 2 @@ -106,7 +106,7 @@ stabilize 3->1 MsgVote Term:2 Log:1/11 3->2 MsgVote Term:2 Log:1/11 INFO 3 received MsgVoteResp from 3 at term 2 - INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 3->1 MsgVote Term:2 Log:1/11 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgVote from 3 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3) @@ -124,7 +124,9 @@ stabilize > 3 receiving messages 2->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 2 at term 2 - INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 3 resets replication set to {1,2,3}, {} + INFO 3 starts new subterm. Term: 2, Subterm: 0 INFO 3 became leader at term 2 > 3 handling Ready Ready MustSync=true: @@ -212,7 +214,7 @@ stabilize 1->2 MsgPreVote Term:3 Log:2/12 1->3 MsgPreVote Term:3 Log:2/12 INFO 1 received MsgPreVoteResp from 1 at term 2 - INFO 1 has received 1 MsgPreVoteResp votes and 0 vote rejections + INFO 1 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 2 receiving messages 1->2 MsgPreVote Term:3 Log:2/12 INFO 2 [logterm: 2, index: 12, vote: 3] ignored MsgPreVote from 1 [logterm: 2, index: 12] at term 2: lease is not expired (remaining ticks: 3) @@ -236,7 +238,7 @@ stabilize 2->1 MsgPreVote Term:3 Log:2/12 2->3 MsgPreVote Term:3 Log:2/12 INFO 2 received MsgPreVoteResp from 2 at term 2 - INFO 2 has received 1 MsgPreVoteResp votes and 0 vote rejections + INFO 2 has received 1 MsgPreVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 2->1 MsgPreVote Term:3 Log:2/12 INFO 1 [logterm: 2, index: 12, vote: 0] cast MsgPreVote for 2 [logterm: 2, index: 12] at term 2 @@ -250,7 +252,7 @@ stabilize > 2 receiving messages 1->2 MsgPreVoteResp Term:3 Log:0/0 INFO 2 received MsgPreVoteResp from 1 at term 2 - INFO 2 has received 2 MsgPreVoteResp votes and 0 vote rejections + INFO 2 has received 2 MsgPreVoteResp votes and 0 vote rejections. It needs [0 0] votes to win INFO 2 became candidate at term 3 INFO 2 [logterm: 2, index: 12] sent MsgVote request to 1 at term 3 INFO 2 [logterm: 2, index: 12] sent MsgVote request to 3 at term 3 @@ -262,7 +264,7 @@ stabilize 2->1 MsgVote Term:3 Log:2/12 2->3 MsgVote Term:3 Log:2/12 INFO 2 received MsgVoteResp from 2 at term 3 - INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections. It needs [1 0] votes to win > 1 receiving messages 2->1 MsgVote Term:3 Log:2/12 INFO 1 [term: 2] received a MsgVote message with higher term from 2 [term: 3] @@ -280,7 +282,9 @@ stabilize > 2 receiving messages 1->2 MsgVoteResp Term:3 Log:0/0 INFO 2 received MsgVoteResp from 1 at term 3 - INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections. It needs [0 0] votes to win + INFO 2 resets replication set to {1,2,3}, {} + INFO 2 starts new subterm. Term: 3, Subterm: 0 INFO 2 became leader at term 3 > 2 handling Ready Ready MustSync=true: diff --git a/testdata/snapshot_succeed_via_app_resp.txt b/testdata/snapshot_succeed_via_app_resp.txt index 5c4b0c61..26a9945f 100644 --- a/testdata/snapshot_succeed_via_app_resp.txt +++ b/testdata/snapshot_succeed_via_app_resp.txt @@ -92,7 +92,7 @@ stabilize 1 Ready MustSync=false: Messages: 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false status 1 ---- @@ -108,7 +108,7 @@ stabilize 3 ---- > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 11, lastindex: 11, lastterm: 1] restored snapshot [index: 11, term: 1] @@ -116,7 +116,7 @@ stabilize 3 > 3 handling Ready Ready MustSync=false: HardState Term:1 Commit:11 - Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/11 diff --git a/testdata/snapshot_succeed_via_app_resp_behind.txt b/testdata/snapshot_succeed_via_app_resp_behind.txt index b0f5883b..4a1f1f53 100644 --- a/testdata/snapshot_succeed_via_app_resp_behind.txt +++ b/testdata/snapshot_succeed_via_app_resp_behind.txt @@ -61,7 +61,7 @@ log is empty: first index=6, last index=5 send-snapshot 1 3 ---- 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false # Propose and commit an additional entry, which makes the leader's # last index 12, beyond the snapshot it sent at index 11. @@ -108,7 +108,7 @@ Messages: deliver-msgs 3 ---- 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false INFO log [committed=5, applied=5, applying=5, unstable.offset=6, unstable.offsetInProgress=6, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 11, lastindex: 11, lastterm: 1] restored snapshot [index: 11, term: 1] @@ -129,13 +129,13 @@ stabilize 1 Ready MustSync=false: Messages: 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:12 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:12 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false # Drop the extra MsgSnap(index=12) that 1 just sent, to keep the test tidy. deliver-msgs drop=(3) ---- dropped: 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:12 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:12 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false # 3 sends the affirmative MsgAppResp that resulted from applying the snapshot # at index 11. @@ -144,7 +144,7 @@ stabilize 3 > 3 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:11 - Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] Witness:0 WitnessOutgoing:0 AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/11 diff --git a/util.go b/util.go index c5f7eed8..3bcb3f06 100644 --- a/util.go +++ b/util.go @@ -188,7 +188,7 @@ func describeMessageWithIndent(indent string, m pb.Message, f EntryFormatter) st return buf.String() } -func DescribeWitnessMessage(m WitnessMessage, f EntryFormatter) string { +func DescribeWitnessMessage(m WitnessMessage) string { var buf bytes.Buffer fmt.Fprintf(&buf, "%s->%s %v Term:%d Log:%d/%d/%d", describeTarget(m.From), describeTarget(m.To), m.Type, m.Term, m.LastLogTerm, m.LastLogSubterm, m.LastLogIndex)