# Tests that PreVote+CheckQuorum prevents a node from obtaining prevotes if
# voters have heard from a leader recently. Also tests that a node is able to
# obtain prevotes if the voter hasn't heard from the leader in the past election
# timeout interval, or if a quorum of voters are precandidates.

log-level none
----
ok

add-nodes 3 voters=(1,2,3) index=10 prevote=true checkquorum=true
----
ok

campaign 1
----
ok

stabilize
----
ok

log-level debug
----
ok

# 2 should fail to campaign, leaving 1's leadership alone.
campaign 2
----
INFO 2 is starting a new election at term 1
INFO 2 became pre-candidate at term 1
INFO 2 [logterm: 1, index: 11] sent MsgPreVote request to 1 at term 1
INFO 2 [logterm: 1, index: 11] sent MsgPreVote request to 3 at term 1

stabilize
----
> 2 handling Ready
  Ready MustSync=false:
  Lead:0 State:StatePreCandidate
  Messages:
  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
> 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)
> 3 receiving messages
  2->3 MsgPreVote Term:2 Log:1/11
  INFO 3 [logterm: 1, index: 11, vote: 1] ignored MsgPreVote from 2 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3)

# If 2 hasn't heard from the leader in the past election timeout, it should
# grant prevotes, allowing 3 to hold an election.
set-randomized-election-timeout 2 timeout=5
----
ok

tick-election 2
----
ok

campaign 3
----
INFO 3 is starting a new election at term 1
INFO 3 became pre-candidate at term 1
INFO 3 [logterm: 1, index: 11] sent MsgPreVote request to 1 at term 1
INFO 3 [logterm: 1, index: 11] sent MsgPreVote request to 2 at term 1

process-ready 3
----
Ready MustSync=false:
Lead:0 State:StatePreCandidate
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

deliver-msgs 2
----
3->2 MsgPreVote Term:2 Log:1/11
INFO 2 [logterm: 1, index: 11, vote: 1] cast MsgPreVote for 3 [logterm: 1, index: 11] at term 1

process-ready 2
----
Ready MustSync=false:
Messages:
2->3 MsgPreVoteResp Term:2 Log:0/0

stabilize
----
> 1 receiving messages
  3->1 MsgPreVote Term:2 Log:1/11
  INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgPreVote from 3 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 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 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
> 3 handling Ready
  Ready MustSync=true:
  Lead:0 State:StateCandidate
  HardState Term:2 Vote:3 Commit:11
  Messages:
  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
> 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)
> 2 receiving messages
  3->2 MsgVote Term:2 Log:1/11
  INFO 2 [term: 1] received a MsgVote message with higher term from 3 [term: 2]
  INFO 2 became follower at term 2
  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:
  Lead:0 State:StateFollower
  HardState Term:2 Vote:3 Commit:11
  Messages:
  2->3 MsgVoteResp Term:2 Log:0/0
> 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 became leader at term 2
> 3 handling Ready
  Ready MustSync=true:
  Lead:3 State:StateLeader
  Entries:
  2/12 EntryNormal ""
  Messages:
  3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
  3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
> 1 receiving messages
  3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
  INFO 1 [term: 1] received a MsgApp message with higher term from 3 [term: 2]
  INFO 1 became follower at term 2
> 2 receiving messages
  3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
> 1 handling Ready
  Ready MustSync=true:
  Lead:3 State:StateFollower
  HardState Term:2 Commit:11
  Entries:
  2/12 EntryNormal ""
  Messages:
  1->3 MsgAppResp Term:2 Log:0/12
> 2 handling Ready
  Ready MustSync=true:
  Lead:3 State:StateFollower
  Entries:
  2/12 EntryNormal ""
  Messages:
  2->3 MsgAppResp Term:2 Log:0/12
> 3 receiving messages
  1->3 MsgAppResp Term:2 Log:0/12
  2->3 MsgAppResp Term:2 Log:0/12
> 3 handling Ready
  Ready MustSync=false:
  HardState Term:2 Vote:3 Commit:12
  CommittedEntries:
  2/12 EntryNormal ""
  Messages:
  3->1 MsgApp Term:2 Log:2/12 Commit:12
  3->2 MsgApp Term:2 Log:2/12 Commit:12
> 1 receiving messages
  3->1 MsgApp Term:2 Log:2/12 Commit:12
> 2 receiving messages
  3->2 MsgApp Term:2 Log:2/12 Commit:12
> 1 handling Ready
  Ready MustSync=false:
  HardState Term:2 Commit:12
  CommittedEntries:
  2/12 EntryNormal ""
  Messages:
  1->3 MsgAppResp Term:2 Log:0/12
> 2 handling Ready
  Ready MustSync=false:
  HardState Term:2 Vote:3 Commit:12
  CommittedEntries:
  2/12 EntryNormal ""
  Messages:
  2->3 MsgAppResp Term:2 Log:0/12
> 3 receiving messages
  1->3 MsgAppResp Term:2 Log:0/12
  2->3 MsgAppResp Term:2 Log:0/12

# Node 3 is now the leader. Even though the leader is active, nodes 1 and 2 can
# still win a prevote and election if they both explicitly campaign, since the
# PreVote+CheckQuorum recent leader condition only applies to follower voters.
# This is beneficial, because it allows a quorum of nodes to replace a leader
# when they have strong reason to believe that it's dead, despite having heard
# from it recently.
#
# We first let 1 lose an election, as we'd otherwise get a tie.
campaign 1
----
INFO 1 is starting a new election at term 2
INFO 1 became pre-candidate at term 2
INFO 1 [logterm: 2, index: 12] sent MsgPreVote request to 2 at term 2
INFO 1 [logterm: 2, index: 12] sent MsgPreVote request to 3 at term 2

stabilize
----
> 1 handling Ready
  Ready MustSync=false:
  Lead:0 State:StatePreCandidate
  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
> 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)
> 3 receiving messages
  1->3 MsgPreVote Term:3 Log:2/12
  INFO 3 [logterm: 2, index: 12, vote: 3] ignored MsgPreVote from 1 [logterm: 2, index: 12] at term 2: lease is not expired (remaining ticks: 3)

campaign 2
----
INFO 2 is starting a new election at term 2
INFO 2 became pre-candidate at term 2
INFO 2 [logterm: 2, index: 12] sent MsgPreVote request to 1 at term 2
INFO 2 [logterm: 2, index: 12] sent MsgPreVote request to 3 at term 2

stabilize
----
> 2 handling Ready
  Ready MustSync=false:
  Lead:0 State:StatePreCandidate
  Messages:
  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
> 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
> 3 receiving messages
  2->3 MsgPreVote Term:3 Log:2/12
  INFO 3 [logterm: 2, index: 12, vote: 3] ignored MsgPreVote from 2 [logterm: 2, index: 12] at term 2: lease is not expired (remaining ticks: 3)
> 1 handling Ready
  Ready MustSync=false:
  Messages:
  1->2 MsgPreVoteResp Term:3 Log:0/0
> 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 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
> 2 handling Ready
  Ready MustSync=true:
  Lead:0 State:StateCandidate
  HardState Term:3 Vote:2 Commit:12
  Messages:
  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
> 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]
  INFO 1 became follower at term 3
  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 [logterm: 2, index: 12, vote: 3] ignored MsgVote from 2 [logterm: 2, index: 12] at term 2: lease is not expired (remaining ticks: 3)
> 1 handling Ready
  Ready MustSync=true:
  Lead:0 State:StateFollower
  HardState Term:3 Vote:2 Commit:12
  Messages:
  1->2 MsgVoteResp Term:3 Log:0/0
> 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 became leader at term 3
> 2 handling Ready
  Ready MustSync=true:
  Lead:2 State:StateLeader
  Entries:
  3/13 EntryNormal ""
  Messages:
  2->1 MsgApp Term:3 Log:2/12 Commit:12 Entries:[3/13 EntryNormal ""]
  2->3 MsgApp Term:3 Log:2/12 Commit:12 Entries:[3/13 EntryNormal ""]
> 1 receiving messages
  2->1 MsgApp Term:3 Log:2/12 Commit:12 Entries:[3/13 EntryNormal ""]
> 3 receiving messages
  2->3 MsgApp Term:3 Log:2/12 Commit:12 Entries:[3/13 EntryNormal ""]
  INFO 3 [term: 2] received a MsgApp message with higher term from 2 [term: 3]
  INFO 3 became follower at term 3
> 1 handling Ready
  Ready MustSync=true:
  Lead:2 State:StateFollower
  Entries:
  3/13 EntryNormal ""
  Messages:
  1->2 MsgAppResp Term:3 Log:0/13
> 3 handling Ready
  Ready MustSync=true:
  Lead:2 State:StateFollower
  HardState Term:3 Commit:12
  Entries:
  3/13 EntryNormal ""
  Messages:
  3->2 MsgAppResp Term:3 Log:0/13
> 2 receiving messages
  1->2 MsgAppResp Term:3 Log:0/13
  3->2 MsgAppResp Term:3 Log:0/13
> 2 handling Ready
  Ready MustSync=false:
  HardState Term:3 Vote:2 Commit:13
  CommittedEntries:
  3/13 EntryNormal ""
  Messages:
  2->1 MsgApp Term:3 Log:3/13 Commit:13
  2->3 MsgApp Term:3 Log:3/13 Commit:13
> 1 receiving messages
  2->1 MsgApp Term:3 Log:3/13 Commit:13
> 3 receiving messages
  2->3 MsgApp Term:3 Log:3/13 Commit:13
> 1 handling Ready
  Ready MustSync=false:
  HardState Term:3 Vote:2 Commit:13
  CommittedEntries:
  3/13 EntryNormal ""
  Messages:
  1->2 MsgAppResp Term:3 Log:0/13
> 3 handling Ready
  Ready MustSync=false:
  HardState Term:3 Commit:13
  CommittedEntries:
  3/13 EntryNormal ""
  Messages:
  3->2 MsgAppResp Term:3 Log:0/13
> 2 receiving messages
  1->2 MsgAppResp Term:3 Log:0/13
  3->2 MsgAppResp Term:3 Log:0/13
