etcd/raft/testdata/probe_and_replicate.txt
qupeng 6828517965 raft: implement fast log rejection
Signed-off-by: qupeng <qupeng@pingcap.com>
2021-02-10 15:48:32 -05:00

768 lines
19 KiB
Plaintext

# This test creates a complete Raft log configuration and demonstrates how a
# leader probes and replicates to each of its followers. The log configuration
# constructed is almost[*] identical to the one present in Figure 7 of the raft
# paper (https://raft.github.io/raft.pdf), which looks like:
#
# 1 2 3 4 5 6 7 8 9 10 11 12
# n1: [1][1][1][4][4][5][5][6][6][6]
# n2: [1][1][1][4][4][5][5][6][6]
# n3: [1][1][1][4]
# n4: [1][1][1][4][4][5][5][6][6][6][6]
# n5: [1][1][1][4][4][5][5][6][7][7][7][7]
# n6: [1][1][1][4][4][4][4]
# n7: [1][1][1][2][2][2][3][3][3][3][3]
#
# Once in this state, we then elect node 1 as the leader and stabilize the
# entire raft group. This demonstrates how a newly elected leader probes for
# matching indexes, overwrites conflicting entries, and catches up all
# followers.
#
# [*] the only differences are:
# 1. n5 is given a larger uncommitted log tail, which is used to demonstrate a
# follower-side probing optimization.
# 2. the log indexes are shifted by 10 in this test because add-nodes wants to
# start with an index > 1.
#
# Set up the log configuration. This is mostly unintersting, but the order of
# each leadership change and the nodes that are allowed to hear about them is
# very important. Most readers of this test can skip this section.
log-level none
----
ok
## Start with seven nodes.
add-nodes 7 voters=(1,2,3,4,5,6,7) index=10
----
ok
## Create term 1 entries.
campaign 1
----
ok
stabilize
----
ok (quiet)
propose 1 prop_1_12
----
ok
propose 1 prop_1_13
----
ok
stabilize
----
ok (quiet)
## Create term 2 entries.
campaign 2
----
ok
stabilize 2
----
ok (quiet)
stabilize 6
----
ok (quiet)
stabilize 2 5 7
----
ok (quiet)
propose 2 prop_2_15
----
ok
propose 2 prop_2_16
----
ok
stabilize 2 7
----
ok (quiet)
deliver-msgs drop=(1,2,3,4,5,6,7)
----
ok (quiet)
## Create term 3 entries.
campaign 7
----
ok
stabilize 7
----
ok (quiet)
stabilize 1 2 3 4 5 6
----
ok (quiet)
stabilize 7
----
ok (quiet)
propose 7 prop_3_18
----
ok
propose 7 prop_3_19
----
ok
propose 7 prop_3_20
----
ok
propose 7 prop_3_21
----
ok
stabilize 7
----
ok (quiet)
deliver-msgs drop=(1,2,3,4,5,6,7)
----
ok (quiet)
## Create term 4 entries.
campaign 6
----
ok
stabilize 1 2 3 4 5 6
----
ok (quiet)
propose 6 prop_4_15
----
ok
stabilize 1 2 4 5 6
----
ok (quiet)
propose 6 prop_4_16
----
ok
propose 6 prop_4_17
----
ok
stabilize 6
----
ok (quiet)
deliver-msgs drop=(1,2,3,4,5,6,7)
----
ok (quiet)
## Create term 5 entries.
campaign 5
----
ok
stabilize 1 2 4 5
----
ok (quiet)
propose 5 prop_5_17
----
ok
stabilize 1 2 4 5
----
ok (quiet)
deliver-msgs drop=(1,2,3,4,5,6,7)
----
ok (quiet)
## Create term 6 entries.
campaign 4
----
ok
stabilize 1 2 4 5
----
ok (quiet)
propose 4 prop_6_19
----
ok
stabilize 1 2 4
----
ok (quiet)
propose 4 prop_6_20
----
ok
stabilize 1 4
----
ok (quiet)
propose 4 prop_6_21
----
ok
stabilize 4
----
ok (quiet)
deliver-msgs drop=(1,2,3,4,5,6,7)
----
ok (quiet)
## Create term 7 entries.
campaign 5
----
ok
stabilize 5
----
ok (quiet)
stabilize 1 3 6 7
----
ok (quiet)
stabilize 5
----
ok (quiet)
propose 5 prop_7_20
----
ok
propose 5 prop_7_21
----
ok
propose 5 prop_7_22
----
ok
stabilize 5
----
ok (quiet)
deliver-msgs drop=(1,2,3,4,5,6,7)
----
ok (quiet)
# Show the Raft log from each node.
log-level info
----
ok
raft-log 1
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
raft-log 2
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
raft-log 3
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
4/14 EntryNormal ""
raft-log 4
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
6/21 EntryNormal "prop_6_21"
raft-log 5
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
7/19 EntryNormal ""
7/20 EntryNormal "prop_7_20"
7/21 EntryNormal "prop_7_21"
7/22 EntryNormal "prop_7_22"
raft-log 6
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
4/16 EntryNormal "prop_4_16"
4/17 EntryNormal "prop_4_17"
raft-log 7
----
1/11 EntryNormal ""
1/12 EntryNormal "prop_1_12"
1/13 EntryNormal "prop_1_13"
2/14 EntryNormal ""
2/15 EntryNormal "prop_2_15"
2/16 EntryNormal "prop_2_16"
3/17 EntryNormal ""
3/18 EntryNormal "prop_3_18"
3/19 EntryNormal "prop_3_19"
3/20 EntryNormal "prop_3_20"
3/21 EntryNormal "prop_3_21"
# Elect node 1 as leader and stabilize.
campaign 1
----
INFO 1 is starting a new election at term 7
INFO 1 became candidate at term 8
INFO 1 received MsgVoteResp from 1 at term 8
INFO 1 [logterm: 6, index: 20] sent MsgVote request to 2 at term 8
INFO 1 [logterm: 6, index: 20] sent MsgVote request to 3 at term 8
INFO 1 [logterm: 6, index: 20] sent MsgVote request to 4 at term 8
INFO 1 [logterm: 6, index: 20] sent MsgVote request to 5 at term 8
INFO 1 [logterm: 6, index: 20] sent MsgVote request to 6 at term 8
INFO 1 [logterm: 6, index: 20] sent MsgVote request to 7 at term 8
## Get elected.
stabilize 1
----
> 1 handling Ready
Ready MustSync=true:
Lead:0 State:StateCandidate
HardState Term:8 Vote:1 Commit:18
Messages:
1->2 MsgVote Term:8 Log:6/20
1->3 MsgVote Term:8 Log:6/20
1->4 MsgVote Term:8 Log:6/20
1->5 MsgVote Term:8 Log:6/20
1->6 MsgVote Term:8 Log:6/20
1->7 MsgVote Term:8 Log:6/20
stabilize 2 3 4 5 6 7
----
> 2 receiving messages
1->2 MsgVote Term:8 Log:6/20
INFO 2 [term: 6] received a MsgVote message with higher term from 1 [term: 8]
INFO 2 became follower at term 8
INFO 2 [logterm: 6, index: 19, vote: 0] cast MsgVote for 1 [logterm: 6, index: 20] at term 8
> 3 receiving messages
1->3 MsgVote Term:8 Log:6/20
INFO 3 [term: 7] received a MsgVote message with higher term from 1 [term: 8]
INFO 3 became follower at term 8
INFO 3 [logterm: 4, index: 14, vote: 0] cast MsgVote for 1 [logterm: 6, index: 20] at term 8
> 4 receiving messages
1->4 MsgVote Term:8 Log:6/20
INFO 4 [term: 6] received a MsgVote message with higher term from 1 [term: 8]
INFO 4 became follower at term 8
INFO 4 [logterm: 6, index: 21, vote: 0] rejected MsgVote from 1 [logterm: 6, index: 20] at term 8
> 5 receiving messages
1->5 MsgVote Term:8 Log:6/20
INFO 5 [term: 7] received a MsgVote message with higher term from 1 [term: 8]
INFO 5 became follower at term 8
INFO 5 [logterm: 7, index: 22, vote: 0] rejected MsgVote from 1 [logterm: 6, index: 20] at term 8
> 6 receiving messages
1->6 MsgVote Term:8 Log:6/20
INFO 6 [term: 7] received a MsgVote message with higher term from 1 [term: 8]
INFO 6 became follower at term 8
INFO 6 [logterm: 4, index: 17, vote: 0] cast MsgVote for 1 [logterm: 6, index: 20] at term 8
> 7 receiving messages
1->7 MsgVote Term:8 Log:6/20
INFO 7 [term: 7] received a MsgVote message with higher term from 1 [term: 8]
INFO 7 became follower at term 8
INFO 7 [logterm: 3, index: 21, vote: 0] cast MsgVote for 1 [logterm: 6, index: 20] at term 8
> 2 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:8 Vote:1 Commit:18
Messages:
2->1 MsgVoteResp Term:8 Log:0/0
> 3 handling Ready
Ready MustSync=true:
HardState Term:8 Vote:1 Commit:14
Messages:
3->1 MsgVoteResp Term:8 Log:0/0
> 4 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:8 Commit:18
Messages:
4->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0)
> 5 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:8 Commit:18
Messages:
5->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0)
> 6 handling Ready
Ready MustSync=true:
HardState Term:8 Vote:1 Commit:15
Messages:
6->1 MsgVoteResp Term:8 Log:0/0
> 7 handling Ready
Ready MustSync=true:
HardState Term:8 Vote:1 Commit:13
Messages:
7->1 MsgVoteResp Term:8 Log:0/0
stabilize 1
----
> 1 receiving messages
2->1 MsgVoteResp Term:8 Log:0/0
INFO 1 received MsgVoteResp from 2 at term 8
INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections
3->1 MsgVoteResp Term:8 Log:0/0
INFO 1 received MsgVoteResp from 3 at term 8
INFO 1 has received 3 MsgVoteResp votes and 0 vote rejections
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 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
6->1 MsgVoteResp Term:8 Log:0/0
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 MsgVoteResp Term:8 Log:0/0
> 1 handling Ready
Ready MustSync=true:
Lead:1 State:StateLeader
Entries:
8/21 EntryNormal ""
Messages:
1->2 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
1->3 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
1->4 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
1->5 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
1->6 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
1->7 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
## Recover each follower, one by one.
stabilize 1 2
----
> 2 receiving messages
1->2 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
> 2 handling Ready
Ready MustSync=false:
Lead:1 State:StateFollower
Messages:
2->1 MsgAppResp Term:8 Log:6/20 Rejected (Hint: 19)
> 1 receiving messages
2->1 MsgAppResp Term:8 Log:6/20 Rejected (Hint: 19)
> 1 handling Ready
Ready MustSync=false:
Messages:
1->2 MsgApp Term:8 Log:6/19 Commit:18 Entries:[6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 2 receiving messages
1->2 MsgApp Term:8 Log:6/19 Commit:18 Entries:[6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 2 handling Ready
Ready MustSync=true:
Entries:
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
Messages:
2->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
2->1 MsgAppResp Term:8 Log:0/21
> 1 handling Ready
Ready MustSync=false:
Messages:
1->2 MsgApp Term:8 Log:8/21 Commit:18
> 2 receiving messages
1->2 MsgApp Term:8 Log:8/21 Commit:18
> 2 handling Ready
Ready MustSync=false:
Messages:
2->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
2->1 MsgAppResp Term:8 Log:0/21
stabilize 1 3
----
> 3 receiving messages
1->3 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
> 3 handling Ready
Ready MustSync=false:
Lead:1 State:StateFollower
Messages:
3->1 MsgAppResp Term:8 Log:4/20 Rejected (Hint: 14)
> 1 receiving messages
3->1 MsgAppResp Term:8 Log:4/20 Rejected (Hint: 14)
> 1 handling Ready
Ready MustSync=false:
Messages:
1->3 MsgApp Term:8 Log:4/14 Commit:18 Entries:[4/15 EntryNormal "prop_4_15", 5/16 EntryNormal "", 5/17 EntryNormal "prop_5_17", 6/18 EntryNormal "", 6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 3 receiving messages
1->3 MsgApp Term:8 Log:4/14 Commit:18 Entries:[4/15 EntryNormal "prop_4_15", 5/16 EntryNormal "", 5/17 EntryNormal "prop_5_17", 6/18 EntryNormal "", 6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 3 handling Ready
Ready MustSync=true:
HardState Term:8 Vote:1 Commit:18
Entries:
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
CommittedEntries:
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
Messages:
3->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
3->1 MsgAppResp Term:8 Log:0/21
> 1 handling Ready
Ready MustSync=false:
Messages:
1->3 MsgApp Term:8 Log:8/21 Commit:18
> 3 receiving messages
1->3 MsgApp Term:8 Log:8/21 Commit:18
> 3 handling Ready
Ready MustSync=false:
Messages:
3->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
3->1 MsgAppResp Term:8 Log:0/21
stabilize 1 4
----
> 4 receiving messages
1->4 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
INFO found conflict at index 21 [existing term: 6, conflicting term: 8]
INFO replace the unstable entries from index 21
> 4 handling Ready
Ready MustSync=true:
Lead:1 State:StateFollower
Entries:
8/21 EntryNormal ""
Messages:
4->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
4->1 MsgAppResp Term:8 Log:0/21
> 1 handling Ready
Ready MustSync=false:
HardState Term:8 Vote:1 Commit:21
CommittedEntries:
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
Messages:
1->2 MsgApp Term:8 Log:8/21 Commit:21
1->3 MsgApp Term:8 Log:8/21 Commit:21
1->4 MsgApp Term:8 Log:8/21 Commit:21
> 4 receiving messages
1->4 MsgApp Term:8 Log:8/21 Commit:21
> 4 handling Ready
Ready MustSync=false:
HardState Term:8 Commit:21
CommittedEntries:
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
Messages:
4->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
4->1 MsgAppResp Term:8 Log:0/21
stabilize 1 5
----
> 5 receiving messages
1->5 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
> 5 handling Ready
Ready MustSync=false:
Lead:1 State:StateFollower
Messages:
5->1 MsgAppResp Term:8 Log:6/20 Rejected (Hint: 18)
> 1 receiving messages
5->1 MsgAppResp Term:8 Log:6/20 Rejected (Hint: 18)
> 1 handling Ready
Ready MustSync=false:
Messages:
1->5 MsgApp Term:8 Log:6/18 Commit:21 Entries:[6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 5 receiving messages
1->5 MsgApp Term:8 Log:6/18 Commit:21 Entries:[6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
INFO found conflict at index 19 [existing term: 7, conflicting term: 6]
INFO replace the unstable entries from index 19
> 5 handling Ready
Ready MustSync=true:
HardState Term:8 Commit:21
Entries:
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
CommittedEntries:
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
Messages:
5->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
5->1 MsgAppResp Term:8 Log:0/21
> 1 handling Ready
Ready MustSync=false:
Messages:
1->5 MsgApp Term:8 Log:8/21 Commit:21
> 5 receiving messages
1->5 MsgApp Term:8 Log:8/21 Commit:21
> 5 handling Ready
Ready MustSync=false:
Messages:
5->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
5->1 MsgAppResp Term:8 Log:0/21
stabilize 1 6
----
> 6 receiving messages
1->6 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
> 6 handling Ready
Ready MustSync=false:
Lead:1 State:StateFollower
Messages:
6->1 MsgAppResp Term:8 Log:4/20 Rejected (Hint: 17)
> 1 receiving messages
6->1 MsgAppResp Term:8 Log:4/20 Rejected (Hint: 17)
> 1 handling Ready
Ready MustSync=false:
Messages:
1->6 MsgApp Term:8 Log:4/15 Commit:21 Entries:[5/16 EntryNormal "", 5/17 EntryNormal "prop_5_17", 6/18 EntryNormal "", 6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 6 receiving messages
1->6 MsgApp Term:8 Log:4/15 Commit:21 Entries:[5/16 EntryNormal "", 5/17 EntryNormal "prop_5_17", 6/18 EntryNormal "", 6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
INFO found conflict at index 16 [existing term: 4, conflicting term: 5]
INFO replace the unstable entries from index 16
> 6 handling Ready
Ready MustSync=true:
HardState Term:8 Vote:1 Commit:21
Entries:
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
CommittedEntries:
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
Messages:
6->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
6->1 MsgAppResp Term:8 Log:0/21
> 1 handling Ready
Ready MustSync=false:
Messages:
1->6 MsgApp Term:8 Log:8/21 Commit:21
> 6 receiving messages
1->6 MsgApp Term:8 Log:8/21 Commit:21
> 6 handling Ready
Ready MustSync=false:
Messages:
6->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
6->1 MsgAppResp Term:8 Log:0/21
stabilize 1 7
----
> 7 receiving messages
1->7 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""]
> 7 handling Ready
Ready MustSync=false:
Lead:1 State:StateFollower
Messages:
7->1 MsgAppResp Term:8 Log:3/20 Rejected (Hint: 20)
> 1 receiving messages
7->1 MsgAppResp Term:8 Log:3/20 Rejected (Hint: 20)
> 1 handling Ready
Ready MustSync=false:
Messages:
1->7 MsgApp Term:8 Log:1/13 Commit:21 Entries:[4/14 EntryNormal "", 4/15 EntryNormal "prop_4_15", 5/16 EntryNormal "", 5/17 EntryNormal "prop_5_17", 6/18 EntryNormal "", 6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
> 7 receiving messages
1->7 MsgApp Term:8 Log:1/13 Commit:21 Entries:[4/14 EntryNormal "", 4/15 EntryNormal "prop_4_15", 5/16 EntryNormal "", 5/17 EntryNormal "prop_5_17", 6/18 EntryNormal "", 6/19 EntryNormal "prop_6_19", 6/20 EntryNormal "prop_6_20", 8/21 EntryNormal ""]
INFO found conflict at index 14 [existing term: 2, conflicting term: 4]
INFO replace the unstable entries from index 14
> 7 handling Ready
Ready MustSync=true:
HardState Term:8 Vote:1 Commit:21
Entries:
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
CommittedEntries:
4/14 EntryNormal ""
4/15 EntryNormal "prop_4_15"
5/16 EntryNormal ""
5/17 EntryNormal "prop_5_17"
6/18 EntryNormal ""
6/19 EntryNormal "prop_6_19"
6/20 EntryNormal "prop_6_20"
8/21 EntryNormal ""
Messages:
7->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
7->1 MsgAppResp Term:8 Log:0/21
> 1 handling Ready
Ready MustSync=false:
Messages:
1->7 MsgApp Term:8 Log:8/21 Commit:21
> 7 receiving messages
1->7 MsgApp Term:8 Log:8/21 Commit:21
> 7 handling Ready
Ready MustSync=false:
Messages:
7->1 MsgAppResp Term:8 Log:0/21
> 1 receiving messages
7->1 MsgAppResp Term:8 Log:0/21