# Tests that PreVote prevents a node that is behind on the log from obtaining
# prevotes and calling an election.
#
# Also tests that a node that is up-to-date on its log can hold an election.
# Unlike the Raft thesis, the recent leader condition requires CheckQuorum
# and is not enforced with PreVote alone.

log-level none
----
ok

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

campaign 1
----
ok

stabilize
----
ok

log-level debug
----
ok

# Propose a command on 1 and replicate it to 2.
propose 1 prop_1
----
ok

process-ready 1
----
Ready MustSync=true:
Entries:
1/12 EntryNormal "prop_1"
Messages:
1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"]
1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"]

deliver-msgs 2
----
1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"]

process-ready 2
----
Ready MustSync=true:
Entries:
1/12 EntryNormal "prop_1"
Messages:
2->1 MsgAppResp Term:1 Log:0/12

# 3 is now behind on its log. Attempt to campaign.
raft-log 3
----
1/11 EntryNormal ""

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 1 2
----
2->1 MsgAppResp Term:1 Log:0/12
3->1 MsgPreVote Term:2 Log:1/11
INFO 1 [logterm: 1, index: 12, vote: 1] rejected MsgPreVote from 3 [logterm: 1, index: 11] at term 1
3->2 MsgPreVote Term:2 Log:1/11
INFO 2 [logterm: 1, index: 12, vote: 1] rejected MsgPreVote from 3 [logterm: 1, index: 11] at term 1

# 3 failed to campaign. Let the network stabilize.
stabilize
----
> 1 handling Ready
  Ready MustSync=false:
  HardState Term:1 Vote:1 Commit:12
  CommittedEntries:
  1/12 EntryNormal "prop_1"
  Messages:
  1->2 MsgApp Term:1 Log:1/12 Commit:12
  1->3 MsgApp Term:1 Log:1/12 Commit:12
  1->3 MsgPreVoteResp Term:1 Log:0/0 Rejected (Hint: 0)
> 2 handling Ready
  Ready MustSync=false:
  Messages:
  2->3 MsgPreVoteResp Term:1 Log:0/0 Rejected (Hint: 0)
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/12 Commit:12
> 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
  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)
> 2 handling Ready
  Ready MustSync=false:
  HardState Term:1 Vote:1 Commit:12
  CommittedEntries:
  1/12 EntryNormal "prop_1"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/12
> 3 handling Ready
  Ready MustSync=true:
  Lead:1 State:StateFollower
  HardState Term:1 Vote:1 Commit:12
  Entries:
  1/12 EntryNormal "prop_1"
  CommittedEntries:
  1/12 EntryNormal "prop_1"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/12
  3->1 MsgAppResp Term:1 Log:0/12
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/12
  3->1 MsgAppResp Term:1 Log:0/12
  3->1 MsgAppResp Term:1 Log:0/12

# Let 2 campaign. It should succeed, since it's up-to-date on the log.
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: 12] sent MsgPreVote request to 1 at term 1
INFO 2 [logterm: 1, index: 12] 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/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
> 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
> 3 receiving messages
  2->3 MsgPreVote Term:2 Log:1/12
  INFO 3 [logterm: 1, index: 12, vote: 1] cast MsgPreVote for 2 [logterm: 1, index: 12] at term 1
> 1 handling Ready
  Ready MustSync=false:
  Messages:
  1->2 MsgPreVoteResp Term:2 Log:0/0
> 3 handling Ready
  Ready MustSync=false:
  Messages:
  3->2 MsgPreVoteResp Term:2 Log:0/0
> 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 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
  3->2 MsgPreVoteResp Term:2 Log:0/0
> 2 handling Ready
  Ready MustSync=true:
  Lead:0 State:StateCandidate
  HardState Term:2 Vote:2 Commit:12
  Messages:
  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
> 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]
  INFO 1 became follower at term 2
  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]
  INFO 3 became follower at term 2
  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:
  Lead:0 State:StateFollower
  HardState Term:2 Vote:2 Commit:12
  Messages:
  1->2 MsgVoteResp Term:2 Log:0/0
> 3 handling Ready
  Ready MustSync=true:
  Lead:0 State:StateFollower
  HardState Term:2 Vote:2 Commit:12
  Messages:
  3->2 MsgVoteResp Term:2 Log:0/0
> 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 became leader at term 2
  3->2 MsgVoteResp Term:2 Log:0/0
> 2 handling Ready
  Ready MustSync=true:
  Lead:2 State:StateLeader
  Entries:
  2/13 EntryNormal ""
  Messages:
  2->1 MsgApp Term:2 Log:1/12 Commit:12 Entries:[2/13 EntryNormal ""]
  2->3 MsgApp Term:2 Log:1/12 Commit:12 Entries:[2/13 EntryNormal ""]
> 1 receiving messages
  2->1 MsgApp Term:2 Log:1/12 Commit:12 Entries:[2/13 EntryNormal ""]
> 3 receiving messages
  2->3 MsgApp Term:2 Log:1/12 Commit:12 Entries:[2/13 EntryNormal ""]
> 1 handling Ready
  Ready MustSync=true:
  Lead:2 State:StateFollower
  Entries:
  2/13 EntryNormal ""
  Messages:
  1->2 MsgAppResp Term:2 Log:0/13
> 3 handling Ready
  Ready MustSync=true:
  Lead:2 State:StateFollower
  Entries:
  2/13 EntryNormal ""
  Messages:
  3->2 MsgAppResp Term:2 Log:0/13
> 2 receiving messages
  1->2 MsgAppResp Term:2 Log:0/13
  3->2 MsgAppResp Term:2 Log:0/13
> 2 handling Ready
  Ready MustSync=false:
  HardState Term:2 Vote:2 Commit:13
  CommittedEntries:
  2/13 EntryNormal ""
  Messages:
  2->1 MsgApp Term:2 Log:2/13 Commit:13
  2->3 MsgApp Term:2 Log:2/13 Commit:13
> 1 receiving messages
  2->1 MsgApp Term:2 Log:2/13 Commit:13
> 3 receiving messages
  2->3 MsgApp Term:2 Log:2/13 Commit:13
> 1 handling Ready
  Ready MustSync=false:
  HardState Term:2 Vote:2 Commit:13
  CommittedEntries:
  2/13 EntryNormal ""
  Messages:
  1->2 MsgAppResp Term:2 Log:0/13
> 3 handling Ready
  Ready MustSync=false:
  HardState Term:2 Vote:2 Commit:13
  CommittedEntries:
  2/13 EntryNormal ""
  Messages:
  3->2 MsgAppResp Term:2 Log:0/13
> 2 receiving messages
  1->2 MsgAppResp Term:2 Log:0/13
  3->2 MsgAppResp Term:2 Log:0/13
