1
- # This is a prep test for the next commit, to demonstrate the difference between
2
- # the eager and lazy replication.
1
+ # This test demonstrates the "lazy replication" feature. The leader sends MsgApp
2
+ # messages to StateReplication peers only when requested explicitly by the
3
+ # application.
3
4
4
5
# Skip logging the boilerplate. Set up a raft group of 3 nodes, and elect node 1
5
6
# as the leader. Nodes 2 and 3 are the followers.
6
7
log-level none
7
8
----
8
9
ok
9
10
10
- add-nodes 3 voters=(1,2,3) index=10
11
+ add-nodes 3 voters=(1,2,3) index=10 lazy-replication=true
11
12
----
12
13
ok
13
14
@@ -32,87 +33,118 @@ propose 1 data-2
32
33
----
33
34
ok
34
35
35
- # Both entries are sent to the followers eagerly, in two separate MsgApp
36
- # messages. The entries are committed, and the followers' logs converge to the
37
- # leader's.
36
+ # NB: no entries are sent to the followers yet.
38
37
stabilize
39
38
----
40
39
> 1 handling Ready
41
40
Ready MustSync=true:
42
41
Entries:
43
42
1/12 EntryNormal "data-1"
44
43
1/13 EntryNormal "data-2"
45
- Messages:
46
- 1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]
47
- 1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]
48
- 1->2 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "data-2"]
49
- 1->3 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "data-2"]
44
+
45
+ # Attempt to send a misaligned MsgApp. No-op.
46
+ send-msg-app 1 to=2 lo=10 hi=13
47
+ ----
48
+ could not send MsgApp (10,13] to 2
49
+
50
+ # Send a MsgApp to node 2, containing both entries.
51
+ send-msg-app 1 to=2 lo=11 hi=13
52
+ ----
53
+ 1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[
54
+ 1/12 EntryNormal "data-1"
55
+ 1/13 EntryNormal "data-2"
56
+ ]
57
+
58
+ # Send a MsgApp to node 3, containing only one entry.
59
+ send-msg-app 1 to=3 lo=11 hi=12
60
+ ----
61
+ 1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]
62
+
63
+ # The followers receive the entries and reply to the leader. The leader commits
64
+ # both entries, but the replication flow to node 3 still has one pending entry.
65
+ stabilize
66
+ ----
50
67
> 2 receiving messages
51
- 1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]
52
- 1->2 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "data-2"]
68
+ 1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[
69
+ 1/12 EntryNormal "data-1"
70
+ 1/13 EntryNormal "data-2"
71
+ ]
53
72
> 3 receiving messages
54
73
1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]
55
- 1->3 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "data-2"]
56
74
> 2 handling Ready
57
75
Ready MustSync=true:
58
76
Entries:
59
77
1/12 EntryNormal "data-1"
60
78
1/13 EntryNormal "data-2"
61
79
Messages:
62
- 2->1 MsgAppResp Term:1 Log:0/12 Commit:11
63
80
2->1 MsgAppResp Term:1 Log:0/13 Commit:11
64
81
> 3 handling Ready
65
82
Ready MustSync=true:
66
83
Entries:
67
84
1/12 EntryNormal "data-1"
68
- 1/13 EntryNormal "data-2"
69
85
Messages:
70
86
3->1 MsgAppResp Term:1 Log:0/12 Commit:11
71
- 3->1 MsgAppResp Term:1 Log:0/13 Commit:11
72
87
> 1 receiving messages
73
- 2->1 MsgAppResp Term:1 Log:0/12 Commit:11
74
88
2->1 MsgAppResp Term:1 Log:0/13 Commit:11
75
89
3->1 MsgAppResp Term:1 Log:0/12 Commit:11
76
- 3->1 MsgAppResp Term:1 Log:0/13 Commit:11
77
90
> 1 handling Ready
78
91
Ready MustSync=true:
79
92
HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
80
93
CommittedEntries:
81
94
1/12 EntryNormal "data-1"
82
95
1/13 EntryNormal "data-2"
83
96
Messages:
84
- 1->2 MsgApp Term:1 Log:1/13 Commit:12
85
- 1->3 MsgApp Term:1 Log:1/13 Commit:12
86
97
1->2 MsgApp Term:1 Log:1/13 Commit:13
87
- 1->3 MsgApp Term:1 Log:1/13 Commit:13
98
+ 1->3 MsgApp Term:1 Log:1/12 Commit:13
88
99
> 2 receiving messages
89
- 1->2 MsgApp Term:1 Log:1/13 Commit:12
90
100
1->2 MsgApp Term:1 Log:1/13 Commit:13
91
101
> 3 receiving messages
92
- 1->3 MsgApp Term:1 Log:1/13 Commit:12
93
- 1->3 MsgApp Term:1 Log:1/13 Commit:13
102
+ 1->3 MsgApp Term:1 Log:1/12 Commit:13
94
103
> 2 handling Ready
95
104
Ready MustSync=true:
96
105
HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
97
106
CommittedEntries:
98
107
1/12 EntryNormal "data-1"
99
108
1/13 EntryNormal "data-2"
100
109
Messages:
101
- 2->1 MsgAppResp Term:1 Log:0/13 Commit:12
102
110
2->1 MsgAppResp Term:1 Log:0/13 Commit:13
103
111
> 3 handling Ready
104
112
Ready MustSync=true:
105
- HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
113
+ HardState Term:1 Vote:1 Commit:12 Lead:1 LeadEpoch:1
106
114
CommittedEntries:
107
115
1/12 EntryNormal "data-1"
116
+ Messages:
117
+ 3->1 MsgAppResp Term:1 Log:0/12 Commit:12
118
+ > 1 receiving messages
119
+ 2->1 MsgAppResp Term:1 Log:0/13 Commit:13
120
+ 3->1 MsgAppResp Term:1 Log:0/12 Commit:12
121
+
122
+ # One entry still to be replicated to node 3.
123
+ status 1
124
+ ----
125
+ 1: StateReplicate match=13 next=14 sentCommit=11 matchCommit=11
126
+ 2: StateReplicate match=13 next=14 sentCommit=13 matchCommit=13
127
+ 3: StateReplicate match=12 next=13 sentCommit=13 matchCommit=12
128
+
129
+ # Replicate it.
130
+ send-msg-app 1 to=3 lo=12 hi=13
131
+ ----
132
+ 1->3 MsgApp Term:1 Log:1/12 Commit:13 Entries:[1/13 EntryNormal "data-2"]
133
+
134
+ stabilize
135
+ ----
136
+ > 3 receiving messages
137
+ 1->3 MsgApp Term:1 Log:1/12 Commit:13 Entries:[1/13 EntryNormal "data-2"]
138
+ > 3 handling Ready
139
+ Ready MustSync=true:
140
+ HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
141
+ Entries:
142
+ 1/13 EntryNormal "data-2"
143
+ CommittedEntries:
108
144
1/13 EntryNormal "data-2"
109
145
Messages:
110
- 3->1 MsgAppResp Term:1 Log:0/13 Commit:12
111
146
3->1 MsgAppResp Term:1 Log:0/13 Commit:13
112
147
> 1 receiving messages
113
- 2->1 MsgAppResp Term:1 Log:0/13 Commit:12
114
- 2->1 MsgAppResp Term:1 Log:0/13 Commit:13
115
- 3->1 MsgAppResp Term:1 Log:0/13 Commit:12
116
148
3->1 MsgAppResp Term:1 Log:0/13 Commit:13
117
149
118
150
# The leader has converged to a fully replicated state.
0 commit comments