diff --git a/pkg/raft/raft.go b/pkg/raft/raft.go index cfa8d574818c..4558747e6900 100644 --- a/pkg/raft/raft.go +++ b/pkg/raft/raft.go @@ -486,7 +486,25 @@ func newRaft(c *Config) *raft { if c.Applied > 0 { raftlog.appliedTo(c.Applied, 0 /* size */) } - r.becomeFollower(r.Term, r.lead) + + if r.lead == r.id { + // If we were the leader, we must have waited out the leadMaxSupported. This + // is done in the kvserver layer before reaching this point. Therefore, it + // should be safe to defortify and become a follower while forgetting that + // we were the leader. If we don't forget that we were the leader, it will + // lead to situations where r.id == r.lead but r.state != StateLeader which + // might confuse the layers above raft. + r.deFortify(r.id, r.Term) + r.becomeFollower(r.Term, None) + } else { + // If we weren't the leader, we should NOT forget who the leader is to avoid + // regressing the leadMaxSupported. We can't just forget the leader because + // we might have been fortified which would lead to a case where + // r.lead == None && r.leadEpoch != 0. Also, we can't call r.defortify() + // because we might have a fortified leader, and calling defortify would + // incorrectly allow us to campaign/vote. + r.becomeFollower(r.Term, r.lead) + } var nodesStrs []string for _, n := range r.trk.VoterNodes() { @@ -1039,6 +1057,7 @@ func (r *raft) reset(term uint64) { r.setTerm(term) } + r.lead = None r.electionElapsed = 0 r.heartbeatElapsed = 0 r.resetRandomizedElectionTimeout() @@ -1261,12 +1280,20 @@ func (r *raft) becomeFollower(term uint64, lead pb.PeerID) { r.lead = None lead = None } + r.state = pb.StateFollower r.step = stepFollower - r.reset(term) r.tick = r.tickElection + + // Start de-fortifying eagerly so we don't have to wait out a full heartbeat + // timeout before sending the first de-fortification message. + if r.shouldBcastDeFortify() { + r.bcastDeFortify() + } + + r.reset(term) r.setLead(lead) - r.state = pb.StateFollower r.logger.Infof("%x became follower at term %d", r.id, r.Term) + r.logger.Debugf("%x reset election elapsed to %d", r.id, r.electionElapsed) } func (r *raft) becomeCandidate() { @@ -2167,7 +2194,7 @@ func stepCandidate(r *raft, m pb.Message) error { case quorum.VoteLost: // pb.MsgPreVoteResp contains future term of pre-candidate // m.Term > r.Term; reuse r.Term - r.becomeFollower(r.Term, r.lead) + r.becomeFollower(r.Term, None) } } return nil @@ -2274,14 +2301,7 @@ func (r *raft) checkQuorumActive() { } if !quorumActiveByHeartbeats && !quorumActiveByFortification { r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id) - // NB: Stepping down because of CheckQuorum is a special, in that we know - // the LeadSupportUntil is in the past. This means that the leader can - // safely call a new election or vote for a different peer without - // regressing LeadSupportUntil. We don't need to/want to give this any - // special treatment -- instead, we handle this like the general step down - // case by simply remembering the term/lead information from our stint as - // the leader. - r.becomeFollower(r.Term, r.id) + r.becomeFollower(r.Term, None) } // Mark everyone (but ourselves) as inactive in preparation for the next // CheckQuorum. @@ -2710,9 +2730,12 @@ func (r *raft) switchToConfig(cfg quorum.Config, progressMap tracker.ProgressMap // interruption). This might still drop some proposals but it's better than // nothing. // - // NB: Similar to the CheckQuorum step down case, we must remember our - // prior stint as leader, lest we regress the QSE. - r.becomeFollower(r.Term, r.lead) + // A learner can't campaign or participate in elections, and in order for a + // learner to get promoted to a voter, it needs a new leader to get elected + // and propose that change. Therefore, it should be safe at this point to + // defortify and forget that we were the leader at this term and step down. + r.deFortify(r.id, r.Term) + r.becomeFollower(r.Term, None) return cs } @@ -2778,7 +2801,12 @@ func (r *raft) resetRandomizedElectionTimeout() { func (r *raft) transferLeader(to pb.PeerID) { assertTrue(r.state == pb.StateLeader, "only the leader can transfer leadership") r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow}) - r.becomeFollower(r.Term, r.lead) + // When a leader transfers leadership to another replica, it instructs the + // replica to campaign without performing the campaign checks. Therefore, it + // should be safe to defortify and forget the we were the leader at this term + // when stepping down. + r.deFortify(r.id, r.Term) + r.becomeFollower(r.Term, None) } func (r *raft) abortLeaderTransfer() { diff --git a/pkg/raft/raft_test.go b/pkg/raft/raft_test.go index d1bae66d5289..60069f46a09e 100644 --- a/pkg/raft/raft_test.go +++ b/pkg/raft/raft_test.go @@ -3548,7 +3548,7 @@ func TestLeaderTransferLeaderStepsDownImmediately(t *testing.T) { nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader}) require.Equal(t, uint64(1), lead.Term) - checkLeaderTransferState(t, lead, pb.StateFollower, 1) + checkLeaderTransferState(t, lead, pb.StateFollower, None) // Eventually, the previous leader gives up on waiting and calls an election // to reestablish leadership at the next term. diff --git a/pkg/raft/rawnode_test.go b/pkg/raft/rawnode_test.go index 0c6d8fc3cdd8..bc9588bb4d7e 100644 --- a/pkg/raft/rawnode_test.go +++ b/pkg/raft/rawnode_test.go @@ -594,31 +594,45 @@ func TestRawNodeRestart(t *testing.T) { MustSync: false, } - storage := newTestMemoryStorage(withPeers(1)) + storage := newTestMemoryStorage(withPeers(1, 2)) storage.SetHardState(st) storage.Append(entries) - rawNode, err := NewRawNode(newTestConfig(1, 10, 1, storage)) + rawNode1, err := NewRawNode(newTestConfig(1, 10, 1, storage)) require.NoError(t, err) - rd := rawNode.Ready() + rawNode2, err := NewRawNode(newTestConfig(2, 10, 1, storage)) + require.NoError(t, err) + rd := rawNode1.Ready() assert.Equal(t, want, rd) - rawNode.Advance(rd) - assert.False(t, rawNode.HasReady()) - // Ensure that the HardState was correctly loaded post restart. - assert.Equal(t, uint64(1), rawNode.raft.Term) - assert.Equal(t, uint64(1), rawNode.raft.raftLog.committed) - assert.Equal(t, pb.PeerID(1), rawNode.raft.lead) - assert.True(t, rawNode.raft.state == pb.StateFollower) - assert.Equal(t, pb.Epoch(1), rawNode.raft.leadEpoch) + rawNode1.Advance(rd) + assert.False(t, rawNode1.HasReady()) + + // Ensure that the HardState was correctly loaded post rawNode1 restart. + assert.Equal(t, uint64(1), rawNode1.raft.Term) + assert.Equal(t, uint64(1), rawNode1.raft.raftLog.committed) + assert.True(t, rawNode1.raft.state == pb.StateFollower) + // Since rawNode1 was the leader, it should become a follower while forgetting + // that it was the leader/leadEpoch was for this term. + assert.Equal(t, None, rawNode1.raft.lead) + assert.Equal(t, pb.Epoch(0), rawNode1.raft.leadEpoch) + + // Ensure that the HardState was correctly loaded post rawNode2 restart. + assert.Equal(t, uint64(1), rawNode2.raft.Term) + assert.Equal(t, uint64(1), rawNode2.raft.raftLog.committed) + assert.True(t, rawNode2.raft.state == pb.StateFollower) + // Since rawNode2 was a follower, it should remember who the leader was, and + // what the leadEpoch was. + assert.Equal(t, pb.PeerID(1), rawNode2.raft.lead) + assert.Equal(t, pb.Epoch(1), rawNode2.raft.leadEpoch) // Ensure we campaign after the election timeout has elapsed. - for i := int64(0); i < rawNode.raft.randomizedElectionTimeout; i++ { + for i := int64(0); i < rawNode1.raft.randomizedElectionTimeout; i++ { // TODO(arul): consider getting rid of this hack to reset the epoch so that // we can call an election without panicking. - rawNode.raft.leadEpoch = 0 - rawNode.raft.tick() + rawNode1.raft.leadEpoch = 0 + rawNode1.raft.tick() } - assert.Equal(t, pb.StateCandidate, rawNode.raft.state) - assert.Equal(t, uint64(2), rawNode.raft.Term) // this should in-turn bump the term + assert.Equal(t, pb.StateCandidate, rawNode1.raft.state) + assert.Equal(t, uint64(2), rawNode1.raft.Term) // this should in-turn bump the term } func TestRawNodeRestartFromSnapshot(t *testing.T) { diff --git a/pkg/raft/testdata/async_storage_writes.txt b/pkg/raft/testdata/async_storage_writes.txt index 28188ae50ab7..2fa146be4f41 100644 --- a/pkg/raft/testdata/async_storage_writes.txt +++ b/pkg/raft/testdata/async_storage_writes.txt @@ -5,12 +5,15 @@ add-nodes 3 voters=(1,2,3) index=10 async-storage-writes=true ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] campaign 1 @@ -36,11 +39,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/10 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 1 processing append thread Processing: diff --git a/pkg/raft/testdata/campaign_learner_must_vote.txt b/pkg/raft/testdata/campaign_learner_must_vote.txt index 832b60fcc3b2..a9cefd97595a 100644 --- a/pkg/raft/testdata/campaign_learner_must_vote.txt +++ b/pkg/raft/testdata/campaign_learner_must_vote.txt @@ -84,6 +84,7 @@ stabilize 3 DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 4] at term 2 > 3 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/confchange_disable_validation.txt b/pkg/raft/testdata/confchange_disable_validation.txt index 57d1b49303ab..9bedce8fbab1 100644 --- a/pkg/raft/testdata/confchange_disable_validation.txt +++ b/pkg/raft/testdata/confchange_disable_validation.txt @@ -11,15 +11,19 @@ add-nodes 4 voters=(1) index=2 max-committed-size-per-ready=1 disable-conf-chang ---- INFO 1 switched to configuration voters=(1) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 2 switched to configuration voters=(1) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 3 switched to configuration voters=(1) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 4 switched to configuration voters=(1) INFO 4 became follower at term 0 +DEBUG 4 reset election elapsed to 0 INFO newRaft 4 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 diff --git a/pkg/raft/testdata/confchange_drop_if_unapplied.txt b/pkg/raft/testdata/confchange_drop_if_unapplied.txt index 4fd482ddfa45..981dbb23096f 100644 --- a/pkg/raft/testdata/confchange_drop_if_unapplied.txt +++ b/pkg/raft/testdata/confchange_drop_if_unapplied.txt @@ -7,15 +7,19 @@ add-nodes 4 voters=(1) index=2 disable-conf-change-validation=true ---- INFO 1 switched to configuration voters=(1) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 2 switched to configuration voters=(1) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 3 switched to configuration voters=(1) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 4 switched to configuration voters=(1) INFO 4 became follower at term 0 +DEBUG 4 reset election elapsed to 0 INFO newRaft 4 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 diff --git a/pkg/raft/testdata/confchange_fortification_safety.txt b/pkg/raft/testdata/confchange_fortification_safety.txt index 7f9bc831e93b..9f8eb1f9bda0 100644 --- a/pkg/raft/testdata/confchange_fortification_safety.txt +++ b/pkg/raft/testdata/confchange_fortification_safety.txt @@ -6,15 +6,19 @@ add-nodes 4 voters=(1, 2, 3) index=2 ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 4 switched to configuration voters=(1 2 3) INFO 4 became follower at term 0 +DEBUG 4 reset election elapsed to 0 INFO newRaft 4 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 @@ -145,6 +149,7 @@ stabilize 1 4 1->4 MsgFortifyLeader Term:1 Log:0/0 INFO 4 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 4 became follower at term 1 + DEBUG 4 reset election elapsed to 0 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v4] DEBUG 4 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 4 handling Ready diff --git a/pkg/raft/testdata/confchange_v1_add_single.txt b/pkg/raft/testdata/confchange_v1_add_single.txt index 77022ff5645d..58c93cccc30a 100644 --- a/pkg/raft/testdata/confchange_v1_add_single.txt +++ b/pkg/raft/testdata/confchange_v1_add_single.txt @@ -5,6 +5,7 @@ add-nodes 1 voters=(1) index=2 ---- INFO 1 switched to configuration voters=(1) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 @@ -32,6 +33,7 @@ add-nodes 1 ---- INFO 2 switched to configuration voters=() INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # n1 commits the conf change using itself as commit quorum, immediately transitions into @@ -62,6 +64,7 @@ stabilize 1->2 MsgFortifyLeader Term:1 Log:0/0 INFO 2 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2] DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready diff --git a/pkg/raft/testdata/confchange_v1_remove_leader.txt b/pkg/raft/testdata/confchange_v1_remove_leader.txt index a550efda9ce1..b1c199ec300a 100644 --- a/pkg/raft/testdata/confchange_v1_remove_leader.txt +++ b/pkg/raft/testdata/confchange_v1_remove_leader.txt @@ -138,14 +138,17 @@ stabilize 1 1->2 MsgApp Term:1 Log:1/8 Commit:7 1->3 MsgApp Term:1 Log:1/8 Commit:7 INFO 1 switched to configuration voters=(2 3) learners=(1) + DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 1 + DEBUG 1 reset election elapsed to 0 > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: State:StateFollower + HardState Term:1 Vote:1 Commit:7 Lead:0 LeadEpoch:0 raft-state ---- -1: StateFollower (Non-Voter) Term:1 Lead:0 LeadEpoch:1 +1: StateFollower (Non-Voter) Term:1 Lead:0 LeadEpoch:0 2: StateFollower (Voter) Term:1 Lead:1 LeadEpoch:1 3: StateFollower (Voter) Term:1 Lead:1 LeadEpoch:1 @@ -206,7 +209,7 @@ stabilize # n1 can no longer propose. propose 1 baz ---- -INFO 1 not forwarding to itself at term 1; dropping proposal +INFO 1 no leader at term 1; dropping proposal raft proposal dropped # Nor can it campaign to become leader. diff --git a/pkg/raft/testdata/confchange_v2_add_double_auto.txt b/pkg/raft/testdata/confchange_v2_add_double_auto.txt index f1a78f5e7215..f4674e89df2a 100644 --- a/pkg/raft/testdata/confchange_v2_add_double_auto.txt +++ b/pkg/raft/testdata/confchange_v2_add_double_auto.txt @@ -7,6 +7,7 @@ add-nodes 1 voters=(1) index=2 ---- INFO 1 switched to configuration voters=(1) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 @@ -33,9 +34,11 @@ add-nodes 2 ---- INFO 2 switched to configuration voters=() INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] INFO 3 switched to configuration voters=() INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # Process n1 once, so that it can append the entry. @@ -86,6 +89,7 @@ stabilize 1 2 1->2 MsgFortifyLeader Term:1 Log:0/0 INFO 2 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3] DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready @@ -180,6 +184,7 @@ stabilize 1 3 1->3 MsgFortifyLeader Term:1 Log:0/0 INFO 3 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3] DEBUG 3 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 1->3 MsgFortifyLeader Term:1 Log:0/0 diff --git a/pkg/raft/testdata/confchange_v2_add_single_auto.txt b/pkg/raft/testdata/confchange_v2_add_single_auto.txt index 3fa6ae90fc26..006f376cbe40 100644 --- a/pkg/raft/testdata/confchange_v2_add_single_auto.txt +++ b/pkg/raft/testdata/confchange_v2_add_single_auto.txt @@ -7,6 +7,7 @@ add-nodes 1 voters=(1) index=2 ---- INFO 1 switched to configuration voters=(1) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 @@ -34,6 +35,7 @@ add-nodes 1 ---- INFO 2 switched to configuration voters=() INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # n1 commits the conf change using itself as commit quorum, immediately transitions into @@ -63,6 +65,7 @@ stabilize 1->2 MsgFortifyLeader Term:1 Log:0/0 INFO 2 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready diff --git a/pkg/raft/testdata/confchange_v2_add_single_explicit.txt b/pkg/raft/testdata/confchange_v2_add_single_explicit.txt index 85c2e308f1a4..fda90ba77c57 100644 --- a/pkg/raft/testdata/confchange_v2_add_single_explicit.txt +++ b/pkg/raft/testdata/confchange_v2_add_single_explicit.txt @@ -7,6 +7,7 @@ add-nodes 1 voters=(1) index=2 ---- INFO 1 switched to configuration voters=(1) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 @@ -34,6 +35,7 @@ add-nodes 1 ---- INFO 2 switched to configuration voters=() INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # n1 commits the conf change using itself as commit quorum, then starts catching up n2. @@ -63,6 +65,7 @@ stabilize 1 2 1->2 MsgFortifyLeader Term:1 Log:0/0 INFO 2 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2] DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready diff --git a/pkg/raft/testdata/confchange_v2_add_single_implicit.txt b/pkg/raft/testdata/confchange_v2_add_single_implicit.txt index 2ef2021a965d..c4690a1451d5 100644 --- a/pkg/raft/testdata/confchange_v2_add_single_implicit.txt +++ b/pkg/raft/testdata/confchange_v2_add_single_implicit.txt @@ -11,9 +11,11 @@ add-nodes 2 voters=(1, 2) index=2 ---- INFO 1 switched to configuration voters=(1 2) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] campaign 1 @@ -37,6 +39,7 @@ add-nodes 1 ---- INFO 3 switched to configuration voters=() INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # n1 commits the conf change using itself as commit quorum, then starts catching up n2. @@ -95,6 +98,7 @@ stabilize 1->3 MsgFortifyLeader Term:1 Log:0/0 INFO 3 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v3] DEBUG 3 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready diff --git a/pkg/raft/testdata/confchange_v2_replace_leader.txt b/pkg/raft/testdata/confchange_v2_replace_leader.txt index fb993ebb42a8..d322e3a3bb8c 100644 --- a/pkg/raft/testdata/confchange_v2_replace_leader.txt +++ b/pkg/raft/testdata/confchange_v2_replace_leader.txt @@ -128,8 +128,9 @@ stabilize > 4 receiving messages 1->4 MsgApp Term:1 Log:1/5 Commit:5 > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: State:StateFollower + HardState Term:1 Vote:1 Commit:5 Lead:0 LeadEpoch:0 > 2 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:5 Lead:1 LeadEpoch:1 @@ -162,7 +163,7 @@ stabilize # n1 is out of the configuration. raft-state ---- -1: StateFollower (Non-Voter) Term:1 Lead:0 LeadEpoch:1 +1: StateFollower (Non-Voter) Term:1 Lead:0 LeadEpoch:0 2: StateFollower (Voter) Term:1 Lead:1 LeadEpoch:1 3: StateFollower (Voter) Term:1 Lead:1 LeadEpoch:1 4: StateFollower (Voter) Term:1 Lead:1 LeadEpoch:1 @@ -188,8 +189,7 @@ ok stabilize log-level=debug ---- > 1 handling Ready - Ready MustSync=true: - HardState Term:1 Vote:1 Commit:5 Lead:1 LeadEpoch:0 + Ready MustSync=false: Messages: 1->2 MsgDeFortifyLeader Term:1 Log:0/0 1->3 MsgDeFortifyLeader Term:1 Log:0/0 diff --git a/pkg/raft/testdata/de_fortification_basic.txt b/pkg/raft/testdata/de_fortification_basic.txt index b5eae7423455..b3ecc1088f3a 100644 --- a/pkg/raft/testdata/de_fortification_basic.txt +++ b/pkg/raft/testdata/de_fortification_basic.txt @@ -8,12 +8,15 @@ add-nodes 3 voters=(1,2,3) index=2 async-storage-writes=true ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1] bump-epoch 1 @@ -81,11 +84,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/2 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 2, vote: 0] cast MsgVote for 1 [logterm: 1, index: 2] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/2 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 2, vote: 0] cast MsgVote for 1 [logterm: 1, index: 2] at term 1 > 1 processing append thread Processing: @@ -291,6 +296,7 @@ print-fortification-state 1 step-down 1 ---- INFO 1 became follower at term 1 +DEBUG 1 reset election elapsed to 0 # 3 isn't fortified by 1. However, we'll still send a MsgDeFortifyLeader to it # because the leader can't distinguish between a follower that isn't fortifying @@ -407,11 +413,13 @@ stabilize 2->1 MsgVote Term:2 Log:1/3 INFO 1 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 3] at term 2 > 3 receiving messages 2->3 MsgVote Term:2 Log:1/3 INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 3] at term 2 > 2 processing append thread Processing: @@ -632,6 +640,7 @@ DEBUG 3 ignoring MsgHup due to leader fortification step-down 2 ---- INFO 2 became follower at term 2 +DEBUG 2 reset election elapsed to 0 campaign 3 ---- @@ -756,6 +765,7 @@ stabilize 3->1 MsgVote Term:4 Log:2/4 INFO 1 [term: 2] received a MsgVote message with higher term from 3 [term: 4], advancing term INFO 1 became follower at term 4 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 2, index: 4, vote: 0] cast MsgVote for 3 [logterm: 2, index: 4] at term 4 > 2 receiving messages 3->2 MsgVote Term:4 Log:2/4 @@ -810,6 +820,7 @@ stabilize INFO 2 [term: 2] received a MsgFortifyLeader message with higher term from 3 [term: 4], new leader indicated, advancing term DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 became follower at term 4 + DEBUG 2 reset election elapsed to 0 3->2 MsgApp Term:4 Log:2/4 Commit:4 Entries:[4/5 EntryNormal ""] > 3 processing append thread Processing: @@ -1028,12 +1039,14 @@ stabilize DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 [term: 4] received a MsgVote message with higher term from 2 [term: 5], advancing term INFO 1 became follower at term 5 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 4, index: 5, vote: 0] cast MsgVote for 2 [logterm: 4, index: 5] at term 5 > 3 receiving messages 2->3 MsgVote Term:5 Log:4/5 DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 [term: 4] received a MsgVote message with higher term from 2 [term: 5], advancing term INFO 3 became follower at term 5 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 4, index: 5, vote: 0] cast MsgVote for 2 [logterm: 4, index: 5] at term 5 > 2 processing append thread Processing: @@ -1251,6 +1264,7 @@ raft-state step-down 2 ---- INFO 2 became follower at term 5 +DEBUG 2 reset election elapsed to 0 send-de-fortify 2 1 ---- @@ -1320,6 +1334,7 @@ stabilize DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 [term: 5] received a MsgVote message with higher term from 1 [term: 6], advancing term INFO 2 became follower at term 6 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 5, index: 6, vote: 0] cast MsgVote for 1 [logterm: 5, index: 6] at term 6 > 3 receiving messages 1->3 MsgVote Term:6 Log:5/6 @@ -1373,6 +1388,7 @@ stabilize INFO 3 [term: 5] received a MsgApp message with higher term from 1 [term: 6], new leader indicated, advancing term DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 became follower at term 6 + DEBUG 3 reset election elapsed to 0 > 1 processing append thread Processing: 1->AppendThread MsgStorageAppend Term:6 Log:6/7 Commit:6 Vote:1 Lead:1 LeadEpoch:3 Entries:[6/7 EntryNormal ""] diff --git a/pkg/raft/testdata/de_fortification_checkquorum.txt b/pkg/raft/testdata/de_fortification_checkquorum.txt index 4d677b1e6afd..d0cf7e71c6ef 100644 --- a/pkg/raft/testdata/de_fortification_checkquorum.txt +++ b/pkg/raft/testdata/de_fortification_checkquorum.txt @@ -69,7 +69,9 @@ INFO 1 leader at term 1 does not support itself in the liveness fabric DEBUG 1 has not received messages from a quorum of peers in the last election timeout DEBUG 1 does not have store liveness support from a quorum of peers WARN 1 stepped down to follower since quorum is not active +DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 1 +DEBUG 1 reset election elapsed to 0 # Needs resetting post step-down. set-randomized-election-timeout 1 timeout=3 @@ -81,7 +83,6 @@ ok # which is what we're interested in for this test. tick-election 1 ---- -DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired DEBUG 1 cannot campaign since it's not supported by a quorum in store liveness raft-state 1 @@ -96,7 +97,7 @@ stabilize > 1 handling Ready Ready MustSync=true: State:StateFollower - HardState Term:1 Vote:1 Commit:11 Lead:1 LeadEpoch:0 + HardState Term:1 Vote:1 Commit:11 Lead:0 LeadEpoch:0 Messages: 1->2 MsgDeFortifyLeader Term:1 Log:0/0 1->3 MsgDeFortifyLeader Term:1 Log:0/0 @@ -104,6 +105,8 @@ stabilize 1->3 MsgDeFortifyLeader Term:1 Log:0/0 1->2 MsgDeFortifyLeader Term:1 Log:0/0 1->3 MsgDeFortifyLeader Term:1 Log:0/0 + 1->2 MsgDeFortifyLeader Term:1 Log:0/0 + 1->3 MsgDeFortifyLeader Term:1 Log:0/0 > 2 receiving messages 1->2 MsgDeFortifyLeader Term:1 Log:0/0 DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired @@ -111,6 +114,8 @@ stabilize DEBUG 2 is not fortifying 1; de-fortification is a no-op 1->2 MsgDeFortifyLeader Term:1 Log:0/0 DEBUG 2 is not fortifying 1; de-fortification is a no-op + 1->2 MsgDeFortifyLeader Term:1 Log:0/0 + DEBUG 2 is not fortifying 1; de-fortification is a no-op > 3 receiving messages 1->3 MsgDeFortifyLeader Term:1 Log:0/0 DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired @@ -118,6 +123,8 @@ stabilize DEBUG 3 is not fortifying 1; de-fortification is a no-op 1->3 MsgDeFortifyLeader Term:1 Log:0/0 DEBUG 3 is not fortifying 1; de-fortification is a no-op + 1->3 MsgDeFortifyLeader Term:1 Log:0/0 + DEBUG 3 is not fortifying 1; de-fortification is a no-op > 2 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:11 Lead:1 LeadEpoch:0 @@ -226,7 +233,9 @@ INFO 2 leader at term 2 does not support itself in the liveness fabric DEBUG 2 has not received messages from a quorum of peers in the last election timeout DEBUG 2 does not have store liveness support from a quorum of peers WARN 2 stepped down to follower since quorum is not active +DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 became follower at term 2 +DEBUG 2 reset election elapsed to 0 # Tick 1 once to ensure it grants its prevote. set-randomized-election-timeout 1 timeout=5 @@ -292,6 +301,7 @@ stabilize 1 3 3->1 MsgVote Term:3 Log:2/12 INFO 1 [term: 2] received a MsgVote message with higher term from 3 [term: 3], advancing term INFO 1 became follower at term 3 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 2, index: 12, vote: 0] cast MsgVote for 3 [logterm: 2, index: 12] at term 3 > 1 handling Ready Ready MustSync=true: @@ -361,7 +371,7 @@ dropped: 3->2 MsgApp Term:3 Log:2/12 Commit:12 Entries:[3/13 EntryNormal ""] raft-state ---- 1: StateFollower (Voter) Term:3 Lead:3 LeadEpoch:1 -2: StateFollower (Voter) Term:2 Lead:0 LeadEpoch:1 +2: StateFollower (Voter) Term:2 Lead:0 LeadEpoch:0 3: StateLeader (Voter) Term:3 Lead:3 LeadEpoch:1 raft-log 3 @@ -383,23 +393,29 @@ ok # And as a result, it continues to send out de-fortification requests. tick-heartbeat 2 ---- -DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired +ok stabilize ---- > 2 handling Ready Ready MustSync=true: State:StateFollower - HardState Term:2 Vote:2 Commit:12 Lead:2 LeadEpoch:0 + HardState Term:2 Vote:2 Commit:12 Lead:0 LeadEpoch:0 Messages: 2->1 MsgDeFortifyLeader Term:2 Log:0/0 2->3 MsgDeFortifyLeader Term:2 Log:0/0 + 2->1 MsgDeFortifyLeader Term:2 Log:0/0 + 2->3 MsgDeFortifyLeader Term:2 Log:0/0 > 1 receiving messages 2->1 MsgDeFortifyLeader Term:2 Log:0/0 INFO 1 [term: 3] ignored a MsgDeFortifyLeader message with lower term from 2 [term: 2] + 2->1 MsgDeFortifyLeader Term:2 Log:0/0 + INFO 1 [term: 3] ignored a MsgDeFortifyLeader message with lower term from 2 [term: 2] > 3 receiving messages 2->3 MsgDeFortifyLeader Term:2 Log:0/0 INFO 3 [term: 3] ignored a MsgDeFortifyLeader message with lower term from 2 [term: 2] + 2->3 MsgDeFortifyLeader Term:2 Log:0/0 + INFO 3 [term: 3] ignored a MsgDeFortifyLeader message with lower term from 2 [term: 2] tick-heartbeat 3 ---- @@ -417,6 +433,7 @@ stabilize 3->2 MsgFortifyLeader Term:3 Log:0/0 INFO 2 [term: 2] received a MsgFortifyLeader message with higher term from 3 [term: 3], new leader indicated, advancing term INFO 2 became follower at term 3 + DEBUG 2 reset election elapsed to 0 3->2 MsgApp Term:3 Log:2/12 Commit:13 Entries:[3/13 EntryNormal ""] > 2 handling Ready Ready MustSync=true: @@ -426,9 +443,16 @@ stabilize CommittedEntries: 3/13 EntryNormal "" Messages: + 2->1 MsgDeFortifyLeader Term:2 Log:0/0 + 2->3 MsgDeFortifyLeader Term:2 Log:0/0 2->3 MsgFortifyLeaderResp Term:3 Log:0/0 LeadEpoch:1 2->3 MsgAppResp Term:3 Log:0/13 Commit:13 +> 1 receiving messages + 2->1 MsgDeFortifyLeader Term:2 Log:0/0 + INFO 1 [term: 3] ignored a MsgDeFortifyLeader message with lower term from 2 [term: 2] > 3 receiving messages + 2->3 MsgDeFortifyLeader Term:2 Log:0/0 + INFO 3 [term: 3] ignored a MsgDeFortifyLeader message with lower term from 2 [term: 2] 2->3 MsgFortifyLeaderResp Term:3 Log:0/0 LeadEpoch:1 2->3 MsgAppResp Term:3 Log:0/13 Commit:13 > 3 handling Ready diff --git a/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt b/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt index adebe78b5944..a439625c8671 100644 --- a/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt +++ b/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt @@ -79,6 +79,7 @@ stabilize > 3 receiving messages 1->3 MsgFortifyLeader Term:1 Log:0/0 INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 > 3 handling Ready Ready MustSync=true: State:StateFollower diff --git a/pkg/raft/testdata/fortification_checkquorum.txt b/pkg/raft/testdata/fortification_checkquorum.txt index 80c249bf5505..c67cd8548e4b 100644 --- a/pkg/raft/testdata/fortification_checkquorum.txt +++ b/pkg/raft/testdata/fortification_checkquorum.txt @@ -85,4 +85,6 @@ INFO 1 leader at term 1 does not support itself in the liveness fabric DEBUG 1 has not received messages from a quorum of peers in the last election timeout DEBUG 1 does not have store liveness support from a quorum of peers WARN 1 stepped down to follower since quorum is not active +DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 1 +DEBUG 1 reset election elapsed to 0 diff --git a/pkg/raft/testdata/fortification_followers_dont_call_election.txt b/pkg/raft/testdata/fortification_followers_dont_call_election.txt index c9cef81f1aca..f851bd7222ea 100644 --- a/pkg/raft/testdata/fortification_followers_dont_call_election.txt +++ b/pkg/raft/testdata/fortification_followers_dont_call_election.txt @@ -9,12 +9,15 @@ add-nodes 3 voters=(1,2,3) index=10 ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] campaign 1 @@ -39,11 +42,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/10 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/fortification_followers_dont_call_election_prevote.txt b/pkg/raft/testdata/fortification_followers_dont_call_election_prevote.txt index 4d6477c2ecad..f9ca359d697b 100644 --- a/pkg/raft/testdata/fortification_followers_dont_call_election_prevote.txt +++ b/pkg/raft/testdata/fortification_followers_dont_call_election_prevote.txt @@ -9,12 +9,15 @@ add-nodes 3 voters=(1,2,3) index=10 prevote=true ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] campaign 1 @@ -69,11 +72,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/10 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/fortification_followers_dont_prevote.txt b/pkg/raft/testdata/fortification_followers_dont_prevote.txt index f5d671c342f3..c7ef83993425 100644 --- a/pkg/raft/testdata/fortification_followers_dont_prevote.txt +++ b/pkg/raft/testdata/fortification_followers_dont_prevote.txt @@ -9,12 +9,15 @@ add-nodes 3 voters=(1,2,3) index=10 prevote=true ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] campaign 1 @@ -69,11 +72,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/10 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: @@ -267,6 +272,7 @@ stabilize 2->3 MsgVote Term:2 Log:1/11 INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 2 > 3 handling Ready Ready MustSync=true: @@ -294,6 +300,7 @@ stabilize INFO 1 [term: 1] received a MsgFortifyLeader message with higher term from 2 [term: 2], new leader indicated, advancing term DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 2->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] > 3 receiving messages 2->3 MsgFortifyLeader Term:2 Log:0/0 @@ -370,6 +377,7 @@ raft-state step-down 2 ---- INFO 2 became follower at term 2 +DEBUG 2 reset election elapsed to 0 campaign 2 ---- @@ -533,11 +541,13 @@ stabilize 2->1 MsgVote Term:3 Log:2/12 INFO 1 [term: 2] received a MsgVote message with higher term from 2 [term: 3], advancing term INFO 1 became follower at term 3 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 2, index: 12, vote: 0] cast MsgVote for 2 [logterm: 2, index: 12] at term 3 > 3 receiving messages 2->3 MsgVote Term:3 Log:2/12 INFO 3 [term: 2] received a MsgVote message with higher term from 2 [term: 3], advancing term INFO 3 became follower at term 3 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 2, index: 12, vote: 0] cast MsgVote for 2 [logterm: 2, index: 12] at term 3 > 1 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/fortification_followers_dont_vote.txt b/pkg/raft/testdata/fortification_followers_dont_vote.txt index 1b29fc67c0fc..74f08f57eaf2 100644 --- a/pkg/raft/testdata/fortification_followers_dont_vote.txt +++ b/pkg/raft/testdata/fortification_followers_dont_vote.txt @@ -11,12 +11,15 @@ add-nodes 3 voters=(1,2,3) index=10 ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] campaign 1 @@ -41,11 +44,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/10 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: @@ -213,6 +218,7 @@ stabilize DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 3], advancing term INFO 3 became follower at term 3 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 3 > 3 handling Ready Ready MustSync=true: @@ -240,6 +246,7 @@ stabilize INFO 1 [term: 1] received a MsgFortifyLeader message with higher term from 2 [term: 3], new leader indicated, advancing term DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 3 + DEBUG 1 reset election elapsed to 0 2->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] > 3 receiving messages 2->3 MsgFortifyLeader Term:3 Log:0/0 @@ -316,6 +323,7 @@ raft-state step-down 2 ---- INFO 2 became follower at term 3 +DEBUG 2 reset election elapsed to 0 campaign 2 ---- @@ -441,11 +449,13 @@ stabilize 2->1 MsgVote Term:5 Log:3/12 INFO 1 [term: 3] received a MsgVote message with higher term from 2 [term: 5], advancing term INFO 1 became follower at term 5 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 3, index: 12, vote: 0] cast MsgVote for 2 [logterm: 3, index: 12] at term 5 > 3 receiving messages 2->3 MsgVote Term:5 Log:3/12 INFO 3 [term: 3] received a MsgVote message with higher term from 2 [term: 5], advancing term INFO 3 became follower at term 5 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 3, index: 12, vote: 0] cast MsgVote for 2 [logterm: 3, index: 12] at term 5 > 1 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/fortification_support_tracking.txt b/pkg/raft/testdata/fortification_support_tracking.txt index 410975815601..ce1aec02630a 100644 --- a/pkg/raft/testdata/fortification_support_tracking.txt +++ b/pkg/raft/testdata/fortification_support_tracking.txt @@ -8,12 +8,15 @@ add-nodes 3 voters=(1,2,3) index=10 ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] withdraw-support 2 1 @@ -45,11 +48,13 @@ stabilize 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 3 receiving messages 1->3 MsgVote Term:1 Log:1/10 INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: @@ -194,6 +199,7 @@ stabilize DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 2 > 3 handling Ready Ready MustSync=true: @@ -221,6 +227,7 @@ stabilize INFO 1 [term: 1] received a MsgFortifyLeader message with higher term from 2 [term: 2], new leader indicated, advancing term DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 2->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] > 3 receiving messages 2->3 MsgFortifyLeader Term:2 Log:0/0 diff --git a/pkg/raft/testdata/leader_step_down_stranded_peer.txt b/pkg/raft/testdata/leader_step_down_stranded_peer.txt index ef5569394583..417ac9bb2197 100644 --- a/pkg/raft/testdata/leader_step_down_stranded_peer.txt +++ b/pkg/raft/testdata/leader_step_down_stranded_peer.txt @@ -5,12 +5,15 @@ add-nodes 3 voters=(1,2,3) index=10 checkquorum=true ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] ############################################################################### @@ -123,6 +126,7 @@ stabilize 1 2 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: @@ -223,6 +227,7 @@ stabilize 1 3 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgAppResp from 3 [logterm: 0, index: 0] at term 1: supporting fortified leader 1 at epoch 1 INFO 1 [term: 1] received a MsgAppResp message with higher term from 3 [term: 3], stepping down as leader to recover stranded peer INFO 1 became follower at term 1 + DEBUG 1 reset election elapsed to 0 3->1 MsgAppResp Term:3 Log:0/0 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgAppResp from 3 [logterm: 0, index: 0] at term 1: supporting fortified leader 1 at epoch 1 INFO 1 [term: 1] received a MsgAppResp message with higher term from 3 [term: 3], ignoring and still supporting fortified leader @@ -290,6 +295,7 @@ stabilize 3->1 MsgVote Term:4 Log:1/10 INFO 1 [term: 2] received a MsgVote message with higher term from 3 [term: 4], advancing term INFO 1 became follower at term 4 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 1, index: 11, vote: 0] rejected MsgVote from 3 [logterm: 1, index: 10] at term 4 > 2 receiving messages 3->2 MsgVote Term:4 Log:1/10 @@ -330,6 +336,7 @@ stabilize 1->3 MsgVote Term:5 Log:1/11 INFO 3 [term: 4] received a MsgVote message with higher term from 1 [term: 5], advancing term INFO 3 became follower at term 5 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 11] at term 5 > 3 handling Ready Ready MustSync=true: @@ -358,6 +365,7 @@ stabilize INFO 2 [term: 1] received a MsgFortifyLeader message with higher term from 1 [term: 5], new leader indicated, advancing term DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 became follower at term 5 + DEBUG 2 reset election elapsed to 0 1->2 MsgApp Term:5 Log:1/11 Commit:11 Entries:[5/12 EntryNormal ""] > 3 receiving messages 1->3 MsgFortifyLeader Term:5 Log:0/0 diff --git a/pkg/raft/testdata/leader_step_down_stranded_peer_prevote.txt b/pkg/raft/testdata/leader_step_down_stranded_peer_prevote.txt index 603919293e10..0d45514c6495 100644 --- a/pkg/raft/testdata/leader_step_down_stranded_peer_prevote.txt +++ b/pkg/raft/testdata/leader_step_down_stranded_peer_prevote.txt @@ -5,12 +5,15 @@ add-nodes 3 voters=(1,2,3) index=10 prevote=true checkquorum=true ---- INFO 1 switched to configuration voters=(1 2 3) INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=(1 2 3) INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] ############################################################################### @@ -241,6 +244,7 @@ stabilize 1 2 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1], advancing term INFO 2 became follower at term 1 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 2 handling Ready Ready MustSync=true: @@ -342,6 +346,7 @@ stabilize 1 3 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgAppResp from 3 [logterm: 0, index: 0] at term 1: supporting fortified leader 1 at epoch 1 INFO 1 [term: 1] received a MsgAppResp message with higher term from 3 [term: 3], stepping down as leader to recover stranded peer INFO 1 became follower at term 1 + DEBUG 1 reset election elapsed to 0 3->1 MsgAppResp Term:3 Log:0/0 INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgAppResp from 3 [logterm: 0, index: 0] at term 1: supporting fortified leader 1 at epoch 1 INFO 1 [term: 1] received a MsgAppResp message with higher term from 3 [term: 3], ignoring and still supporting fortified leader @@ -393,6 +398,7 @@ stabilize 3->1 MsgPreVoteResp Term:3 Log:0/0 Rejected (Hint: 0) INFO 1 [term: 1] received a MsgPreVoteResp message with higher term from 3 [term: 3], advancing term INFO 1 became follower at term 3 + DEBUG 1 reset election elapsed to 0 > 1 handling Ready Ready MustSync=true: State:StateFollower @@ -448,6 +454,7 @@ stabilize 1->3 MsgVote Term:4 Log:1/11 INFO 3 [term: 3] received a MsgVote message with higher term from 1 [term: 4], advancing term INFO 3 became follower at term 4 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 11] at term 4 > 3 handling Ready Ready MustSync=true: @@ -476,6 +483,7 @@ stabilize INFO 2 [term: 1] received a MsgFortifyLeader message with higher term from 1 [term: 4], new leader indicated, advancing term DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 became follower at term 4 + DEBUG 2 reset election elapsed to 0 1->2 MsgApp Term:4 Log:1/11 Commit:11 Entries:[4/12 EntryNormal ""] > 3 receiving messages 1->3 MsgFortifyLeader Term:4 Log:0/0 diff --git a/pkg/raft/testdata/mixedversions/checkquorum.txt b/pkg/raft/testdata/mixedversions/checkquorum.txt index a06460c13db4..588e1a918d24 100644 --- a/pkg/raft/testdata/mixedversions/checkquorum.txt +++ b/pkg/raft/testdata/mixedversions/checkquorum.txt @@ -76,6 +76,7 @@ DEBUG 1 has not received messages from a quorum of peers in the last election ti DEBUG 1 does not have store liveness support from a quorum of peers WARN 1 stepped down to follower since quorum is not active INFO 1 became follower at term 1 +DEBUG 1 reset election elapsed to 0 # We'll now send all of the heartbeats that were buffered during the ticks # above. Conceptually, "the network was slow". @@ -128,6 +129,7 @@ stabilize 2->1 MsgAppResp Term:2 Log:0/0 INFO 1 [term: 1] received a MsgAppResp message with higher term from 2 [term: 2], advancing term INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 2->1 MsgAppResp Term:2 Log:0/0 2->1 MsgAppResp Term:2 Log:0/0 2->1 MsgAppResp Term:2 Log:0/0 @@ -170,6 +172,7 @@ deliver-msgs 1 2->1 MsgVote Term:3 Log:1/11 INFO 1 [term: 2] received a MsgVote message with higher term from 2 [term: 3], advancing term INFO 1 became follower at term 3 +DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 3 deliver-msgs 3 @@ -204,6 +207,7 @@ stabilize 2->3 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] INFO 3 [term: 1] received a MsgApp message with higher term from 2 [term: 3], new leader indicated, advancing term INFO 3 became follower at term 3 + DEBUG 3 reset election elapsed to 0 > 1 handling Ready Ready MustSync=true: HardState Term:3 Vote:2 Commit:11 Lead:2 LeadEpoch:0 diff --git a/pkg/raft/testdata/mixedversions/forget_leader_prevote_checkquorum.txt b/pkg/raft/testdata/mixedversions/forget_leader_prevote_checkquorum.txt index cdec209801f9..3a1597ef336b 100644 --- a/pkg/raft/testdata/mixedversions/forget_leader_prevote_checkquorum.txt +++ b/pkg/raft/testdata/mixedversions/forget_leader_prevote_checkquorum.txt @@ -73,6 +73,7 @@ stabilize > 3 receiving messages 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11 INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 > 2 handling Ready Ready MustSync=false: Messages: diff --git a/pkg/raft/testdata/mixedversions/snapshot_succeed_via_app_resp.txt b/pkg/raft/testdata/mixedversions/snapshot_succeed_via_app_resp.txt index 8698fff33b5c..9e4abf798d73 100644 --- a/pkg/raft/testdata/mixedversions/snapshot_succeed_via_app_resp.txt +++ b/pkg/raft/testdata/mixedversions/snapshot_succeed_via_app_resp.txt @@ -55,6 +55,7 @@ add-nodes 1 ---- INFO 3 switched to configuration voters=() INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # Time passes on the leader so that it will try the previously missing follower @@ -78,6 +79,7 @@ stabilize 3 1->3 MsgHeartbeat Term:1 Log:0/0 INFO 3 [term: 0] received a MsgHeartbeat message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 > 3 handling Ready Ready MustSync=true: HardState Term:1 Commit:0 Lead:1 LeadEpoch:0 diff --git a/pkg/raft/testdata/prevote.txt b/pkg/raft/testdata/prevote.txt index bca43b5be753..de08a5aa6c8d 100644 --- a/pkg/raft/testdata/prevote.txt +++ b/pkg/raft/testdata/prevote.txt @@ -131,6 +131,7 @@ stabilize > 3 receiving messages 1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 1->3 MsgApp Term:1 Log:1/12 Commit:12 1->3 MsgPreVoteResp Term:1 Log:0/0 Rejected (Hint: 0) 2->3 MsgPreVoteResp Term:1 Log:0/0 Rejected (Hint: 0) @@ -212,11 +213,13 @@ stabilize 2->1 MsgVote Term:2 Log:1/12 INFO 1 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 1, index: 12, vote: 0] cast MsgVote for 2 [logterm: 1, index: 12] at term 2 > 3 receiving messages 2->3 MsgVote Term:2 Log:1/12 INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 12, vote: 0] cast MsgVote for 2 [logterm: 1, index: 12] at term 2 > 1 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/prevote_checkquorum.txt b/pkg/raft/testdata/prevote_checkquorum.txt index 2760c1c28013..724d0dce0ea7 100644 --- a/pkg/raft/testdata/prevote_checkquorum.txt +++ b/pkg/raft/testdata/prevote_checkquorum.txt @@ -130,6 +130,7 @@ stabilize 3->2 MsgVote Term:2 Log:1/11 INFO 2 [term: 1] received a MsgVote message with higher term from 3 [term: 2], advancing term INFO 2 became follower at term 2 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 > 2 handling Ready Ready MustSync=true: @@ -158,6 +159,7 @@ stabilize INFO 1 [term: 1] received a MsgFortifyLeader message with higher term from 3 [term: 2], new leader indicated, advancing term DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] > 2 receiving messages 3->2 MsgFortifyLeader Term:2 Log:0/0 @@ -329,6 +331,7 @@ stabilize 2->1 MsgVote Term:3 Log:2/12 INFO 1 [term: 2] received a MsgVote message with higher term from 2 [term: 3], advancing term INFO 1 became follower at term 3 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 2, index: 12, vote: 0] cast MsgVote for 2 [logterm: 2, index: 12] at term 3 > 3 receiving messages 2->3 MsgVote Term:3 Log:2/12 @@ -363,6 +366,7 @@ stabilize INFO 3 [term: 2] received a MsgFortifyLeader message with higher term from 2 [term: 3], new leader indicated, advancing term DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 became follower at term 3 + DEBUG 3 reset election elapsed to 0 2->3 MsgApp Term:3 Log:2/12 Commit:12 Entries:[3/13 EntryNormal ""] > 1 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/probe_and_replicate.txt b/pkg/raft/testdata/probe_and_replicate.txt index 82328c9d5d80..91d7ac8e7157 100644 --- a/pkg/raft/testdata/probe_and_replicate.txt +++ b/pkg/raft/testdata/probe_and_replicate.txt @@ -468,6 +468,12 @@ stabilize 2 3 4 5 6 7 State:StateFollower HardState Term:8 Commit:18 Lead:0 LeadEpoch:0 Messages: + 5->1 MsgDeFortifyLeader Term:7 Log:0/0 + 5->2 MsgDeFortifyLeader Term:7 Log:0/0 + 5->3 MsgDeFortifyLeader Term:7 Log:0/0 + 5->4 MsgDeFortifyLeader Term:7 Log:0/0 + 5->6 MsgDeFortifyLeader Term:7 Log:0/0 + 5->7 MsgDeFortifyLeader Term:7 Log:0/0 5->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0) > 6 handling Ready Ready MustSync=true: @@ -478,7 +484,39 @@ stabilize 2 3 4 5 6 7 Ready MustSync=true: HardState Term:8 Vote:1 Commit:13 Lead:0 LeadEpoch:0 Messages: + 7->1 MsgDeFortifyLeader Term:3 Log:0/0 + 7->2 MsgDeFortifyLeader Term:3 Log:0/0 + 7->3 MsgDeFortifyLeader Term:3 Log:0/0 + 7->4 MsgDeFortifyLeader Term:3 Log:0/0 + 7->5 MsgDeFortifyLeader Term:3 Log:0/0 + 7->6 MsgDeFortifyLeader Term:3 Log:0/0 7->1 MsgVoteResp Term:8 Log:0/0 +> 2 receiving messages + 5->2 MsgDeFortifyLeader Term:7 Log:0/0 + INFO 2 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 5 [term: 7] + 7->2 MsgDeFortifyLeader Term:3 Log:0/0 + INFO 2 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 7 [term: 3] +> 3 receiving messages + 5->3 MsgDeFortifyLeader Term:7 Log:0/0 + INFO 3 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 5 [term: 7] + 7->3 MsgDeFortifyLeader Term:3 Log:0/0 + INFO 3 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 7 [term: 3] +> 4 receiving messages + 5->4 MsgDeFortifyLeader Term:7 Log:0/0 + INFO 4 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 5 [term: 7] + 7->4 MsgDeFortifyLeader Term:3 Log:0/0 + INFO 4 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 7 [term: 3] +> 5 receiving messages + 7->5 MsgDeFortifyLeader Term:3 Log:0/0 + INFO 5 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 7 [term: 3] +> 6 receiving messages + 5->6 MsgDeFortifyLeader Term:7 Log:0/0 + INFO 6 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 5 [term: 7] + 7->6 MsgDeFortifyLeader Term:3 Log:0/0 + INFO 6 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 7 [term: 3] +> 7 receiving messages + 5->7 MsgDeFortifyLeader Term:7 Log:0/0 + INFO 7 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 5 [term: 7] stabilize 1 ---- @@ -492,6 +530,8 @@ stabilize 1 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 + 5->1 MsgDeFortifyLeader Term:7 Log:0/0 + INFO 1 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 5 [term: 7] 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 @@ -499,6 +539,8 @@ stabilize 1 INFO 1 received MsgVoteResp from 6 at term 8 INFO 1 has received 4 MsgVoteResp votes and 2 vote rejections INFO 1 became leader at term 8 + 7->1 MsgDeFortifyLeader Term:3 Log:0/0 + INFO 1 [term: 8] ignored a MsgDeFortifyLeader message with lower term from 7 [term: 3] 7->1 MsgVoteResp Term:8 Log:0/0 > 1 handling Ready Ready MustSync=true: diff --git a/pkg/raft/testdata/snapshot_new_term.txt b/pkg/raft/testdata/snapshot_new_term.txt index fd96cd1ee3f3..911a9634a651 100644 --- a/pkg/raft/testdata/snapshot_new_term.txt +++ b/pkg/raft/testdata/snapshot_new_term.txt @@ -33,13 +33,16 @@ transfer-leadership from=1 to=2 ---- INFO 1 [term 1] starts to transfer leadership to 2 INFO 1 sends MsgTimeoutNow to 2 immediately as 2 already has up-to-date log +DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 1 +DEBUG 1 reset election elapsed to 0 stabilize 1 2 ---- > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: State:StateFollower + HardState Term:1 Vote:1 Commit:11 Lead:0 LeadEpoch:0 Messages: 1->2 MsgTimeoutNow Term:1 Log:0/0 > 2 receiving messages @@ -60,9 +63,9 @@ stabilize 1 2 INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections > 1 receiving messages 2->1 MsgVote Term:2 Log:1/11 - DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 2 > 1 handling Ready Ready MustSync=true: @@ -143,6 +146,7 @@ stabilize INFO 3 [term: 1] received a MsgSnap message with higher term from 2 [term: 2], new leader indicated, advancing term DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO log [committed=11, applied=11, applying=11, unstable.offset=12, unstable.offsetInProgress=12, len(unstable.Entries)=0] starts to restore snapshot [index: 12, term: 2] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 12, lastindex: 12, lastterm: 2] restored snapshot [index: 12, term: 2] diff --git a/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt b/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt index 52f6b35b3b32..979acf0728b1 100644 --- a/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt +++ b/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt @@ -51,6 +51,7 @@ add-nodes 1 ---- INFO 3 switched to configuration voters=() INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] # Time passes on the leader so that it will try the previously missing follower @@ -74,6 +75,7 @@ stabilize 3 1->3 MsgFortifyLeader Term:1 Log:0/0 INFO 3 [term: 0] received a MsgFortifyLeader message with higher term from 1 [term: 1], new leader indicated, advancing term INFO 3 became follower at term 1 + DEBUG 3 reset election elapsed to 0 > 3 handling Ready Ready MustSync=true: HardState Term:1 Commit:0 Lead:1 LeadEpoch:1 diff --git a/pkg/raft/testdata/store_liveness_basic.txt b/pkg/raft/testdata/store_liveness_basic.txt index 88f696ea068d..d7a22cfec9c8 100644 --- a/pkg/raft/testdata/store_liveness_basic.txt +++ b/pkg/raft/testdata/store_liveness_basic.txt @@ -5,9 +5,11 @@ add-nodes 2 index=10 ---- INFO 1 switched to configuration voters=() INFO 1 became follower at term 0 +DEBUG 1 reset election elapsed to 0 INFO newRaft 1 [peers: [], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] INFO 2 switched to configuration voters=() INFO 2 became follower at term 0 +DEBUG 2 reset election elapsed to 0 INFO newRaft 2 [peers: [], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] store-liveness @@ -20,6 +22,7 @@ add-nodes 1 ---- INFO 3 switched to configuration voters=() INFO 3 became follower at term 0 +DEBUG 3 reset election elapsed to 0 INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] store-liveness @@ -88,6 +91,7 @@ add-nodes 1 ---- INFO 4 switched to configuration voters=() INFO 4 became follower at term 0 +DEBUG 4 reset election elapsed to 0 INFO newRaft 4 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0] store-liveness diff --git a/pkg/raft/testdata/transfer-leadership.txt b/pkg/raft/testdata/transfer-leadership.txt index fbf2889e4785..f4a88f24c50f 100644 --- a/pkg/raft/testdata/transfer-leadership.txt +++ b/pkg/raft/testdata/transfer-leadership.txt @@ -33,13 +33,16 @@ transfer-leadership from=1 to=2 ---- INFO 1 [term 1] starts to transfer leadership to 2 INFO 1 sends MsgTimeoutNow to 2 immediately as 2 already has up-to-date log +DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 became follower at term 1 +DEBUG 1 reset election elapsed to 0 stabilize ---- > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: State:StateFollower + HardState Term:1 Vote:1 Commit:11 Lead:0 LeadEpoch:0 Messages: 1->2 MsgTimeoutNow Term:1 Log:0/0 > 2 receiving messages @@ -60,15 +63,16 @@ stabilize INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections > 1 receiving messages 2->1 MsgVote Term:2 Log:1/11 - DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 1 became follower at term 2 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 2 > 3 receiving messages 2->3 MsgVote Term:2 Log:1/11 DEBUG 3 setting election elapsed to start from 3 ticks after store liveness support expired INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2], advancing term INFO 3 became follower at term 2 + DEBUG 3 reset election elapsed to 0 INFO 3 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 2 > 1 handling Ready Ready MustSync=true: @@ -190,13 +194,16 @@ transfer-leadership from=2 to=3 ---- INFO 2 [term 2] starts to transfer leadership to 3 INFO 2 sends MsgTimeoutNow to 3 immediately as 3 already has up-to-date log +DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 became follower at term 2 +DEBUG 2 reset election elapsed to 0 stabilize ---- > 2 handling Ready - Ready MustSync=false: + Ready MustSync=true: State:StateFollower + HardState Term:2 Vote:2 Commit:12 Lead:0 LeadEpoch:0 Messages: 2->3 MsgTimeoutNow Term:2 Log:0/0 > 3 receiving messages @@ -220,12 +227,13 @@ stabilize DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired INFO 1 [term: 2] received a MsgVote message with higher term from 3 [term: 3], advancing term INFO 1 became follower at term 3 + DEBUG 1 reset election elapsed to 0 INFO 1 [logterm: 2, index: 12, vote: 0] cast MsgVote for 3 [logterm: 2, index: 12] at term 3 > 2 receiving messages 3->2 MsgVote Term:3 Log:2/12 - DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired INFO 2 [term: 2] received a MsgVote message with higher term from 3 [term: 3], advancing term INFO 2 became follower at term 3 + DEBUG 2 reset election elapsed to 0 INFO 2 [logterm: 2, index: 12, vote: 0] cast MsgVote for 3 [logterm: 2, index: 12] at term 3 > 1 handling Ready Ready MustSync=true: