etcd/raft/testdata/campaign_learner_must_vote.txt

153 lines
4.3 KiB
Plaintext

# Regression test that verifies that learners can vote. This holds only in the
# sense that if a learner is asked to vote, a candidate believes that they are a
# voter based on its current config, which may be more recent than that of the
# learner. If learners which are actually voters but don't know it yet don't
# vote in that situation, the raft group may end up unavailable despite a quorum
# of voters (as of the latest config) being available.
#
# See:
# https://github.com/etcd-io/etcd/pull/10998
# Turn output off during boilerplate.
log-level none
----
ok
# Bootstrap three nodes.
add-nodes 3 voters=(1,2) learners=(3) index=2
----
ok
# n1 gets to be leader.
campaign 1
----
ok
stabilize
----
ok (quiet)
# Propose a conf change on n1 that promotes n3 to voter.
propose-conf-change 1
v3
----
ok
# Commit and fully apply said conf change. n1 and n2 now consider n3 a voter.
stabilize 1 2
----
ok (quiet)
# Drop all inflight messages to 3. We don't want it to be caught up when it is
# asked to vote.
deliver-msgs drop=(3)
----
ok (quiet)
# We now pretend that n1 is dead, and n2 is trying to become leader.
log-level debug
----
ok
campaign 2
----
INFO 2 is starting a new election at term 1
INFO 2 became candidate at term 2
INFO 2 received MsgVoteResp from 2 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
# Send out the MsgVote requests.
process-ready 2
----
Ready MustSync=true:
Lead:0 State:StateCandidate
HardState Term:2 Vote:2 Commit:4
Messages:
2->1 MsgVote Term:2 Log:1/4
2->3 MsgVote Term:2 Log:1/4
# 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
# its log. Still, it casts a vote for n2 which can in turn become leader and
# catches up n3.
stabilize 3
----
> 3 receiving messages
2->3 MsgVote Term:2 Log:1/4
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
> 3 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:2 Vote:2 Commit:3
Messages:
3->2 MsgVoteResp Term:2 Log:0/0
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 became leader at term 2
> 2 handling Ready
Ready MustSync=true:
Lead:2 State:StateLeader
Entries:
2/5 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 ""]
> 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
> 3 handling Ready
Ready MustSync=false:
Lead:2 State:StateFollower
Messages:
3->2 MsgAppResp Term:2 Log:0/4 Rejected (Hint: 3)
> 2 receiving messages
3->2 MsgAppResp Term:2 Log:0/4 Rejected (Hint: 3)
DEBUG 2 received MsgAppResp(MsgApp was rejected, lastindex: 3) from 3 for index 4
DEBUG 2 decreased progress of 3 to [StateProbe match=0 next=4]
> 2 handling Ready
Ready MustSync=false:
Messages:
2->3 MsgApp Term:2 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v3, 2/5 EntryNormal ""]
> 3 receiving messages
2->3 MsgApp Term:2 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v3, 2/5 EntryNormal ""]
> 3 handling Ready
Ready MustSync=true:
HardState Term:2 Vote:2 Commit:4
Entries:
1/4 EntryConfChangeV2 v3
2/5 EntryNormal ""
CommittedEntries:
1/4 EntryConfChangeV2 v3
Messages:
3->2 MsgAppResp Term:2 Log:0/5
INFO 3 switched to configuration voters=(1 2 3)
> 2 receiving messages
3->2 MsgAppResp Term:2 Log:0/5
> 2 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:2 Commit:5
CommittedEntries:
2/5 EntryNormal ""
Messages:
2->3 MsgApp Term:2 Log:2/5 Commit:5
> 3 receiving messages
2->3 MsgApp Term:2 Log:2/5 Commit:5
> 3 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:2 Commit:5
CommittedEntries:
2/5 EntryNormal ""
Messages:
3->2 MsgAppResp Term:2 Log:0/5
> 2 receiving messages
3->2 MsgAppResp Term:2 Log:0/5