etcd/raft/testdata/campaign.txt
Tobias Schottdorf 078caccce5 raft: add a batch of interaction-driven conf change tests
Verifiy the behavior in various v1 and v2 conf change operations.
This also includes various fixups, notably it adds protection
against transitioning in and out of new configs when this is not
permissible.

There are more threads to pull, but those are left for future commits.
2019-08-16 08:09:44 -07:00

118 lines
3.5 KiB
Plaintext

log-level info
----
ok
add-nodes 3 voters=(1,2,3) index=2
----
INFO 1 switched to configuration voters=(1 2 3)
INFO 1 became follower at term 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
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
INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1]
campaign 1
----
INFO 1 is starting a new election at term 0
INFO 1 became candidate at term 1
INFO 1 received MsgVoteResp from 1 at term 1
INFO 1 [logterm: 1, index: 2] sent MsgVote request to 2 at term 1
INFO 1 [logterm: 1, index: 2] sent MsgVote request to 3 at term 1
stabilize
----
> 1 handling Ready
Ready MustSync=true:
Lead:0 State:StateCandidate
HardState Term:1 Vote:1 Commit:2
Messages:
1->2 MsgVote Term:1 Log:1/2
1->3 MsgVote Term:1 Log:1/2
> 2 receiving messages
1->2 MsgVote Term:1 Log:1/2
INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1]
INFO 2 became follower at term 1
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]
INFO 3 became follower at term 1
INFO 3 [logterm: 1, index: 2, vote: 0] cast MsgVote for 1 [logterm: 1, index: 2] at term 1
> 2 handling Ready
Ready MustSync=true:
HardState Term:1 Vote:1 Commit:2
Messages:
2->1 MsgVoteResp Term:1 Log:0/0
> 3 handling Ready
Ready MustSync=true:
HardState Term:1 Vote:1 Commit:2
Messages:
3->1 MsgVoteResp Term:1 Log:0/0
> 1 receiving messages
2->1 MsgVoteResp Term:1 Log:0/0
INFO 1 received MsgVoteResp from 2 at term 1
INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections
INFO 1 became leader at term 1
3->1 MsgVoteResp Term:1 Log:0/0
> 1 handling Ready
Ready MustSync=true:
Lead:1 State:StateLeader
Entries:
1/3 EntryNormal ""
Messages:
1->2 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
1->3 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
> 2 handling Ready
Ready MustSync=true:
Lead:1 State:StateFollower
Entries:
1/3 EntryNormal ""
Messages:
2->1 MsgAppResp Term:1 Log:0/3
> 3 handling Ready
Ready MustSync=true:
Lead:1 State:StateFollower
Entries:
1/3 EntryNormal ""
Messages:
3->1 MsgAppResp Term:1 Log:0/3
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/3
3->1 MsgAppResp Term:1 Log:0/3
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:3
CommittedEntries:
1/3 EntryNormal ""
Messages:
1->2 MsgApp Term:1 Log:1/3 Commit:3
1->3 MsgApp Term:1 Log:1/3 Commit:3
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/3 Commit:3
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/3 Commit:3
> 2 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:3
CommittedEntries:
1/3 EntryNormal ""
Messages:
2->1 MsgAppResp Term:1 Log:0/3
> 3 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:3
CommittedEntries:
1/3 EntryNormal ""
Messages:
3->1 MsgAppResp Term:1 Log:0/3
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/3
3->1 MsgAppResp Term:1 Log:0/3