-
Notifications
You must be signed in to change notification settings - Fork 9.8k
/
confchange_v1_remove_leader.txt
224 lines (208 loc) · 5.78 KB
/
confchange_v1_remove_leader.txt
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
# We'll turn this back on after the boilerplate.
log-level none
----
ok
# Run a V1 membership change that removes the leader.
# Bootstrap n1, n2, n3.
add-nodes 3 voters=(1,2,3) index=2
----
ok
campaign 1
----
ok
stabilize
----
ok (quiet)
log-level debug
----
ok
# Start removing n1.
propose-conf-change 1 v1=true
r1
----
ok
# Propose an extra entry which will be sent out together with the conf change.
propose 1 foo
----
ok
# Send out the corresponding appends.
process-ready 1
----
Ready MustSync=true:
Entries:
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
1->2 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
1->2 MsgApp Term:1 Log:1/4 Commit:3 Entries:[1/5 EntryNormal "foo"]
1->3 MsgApp Term:1 Log:1/4 Commit:3 Entries:[1/5 EntryNormal "foo"]
# Send response from n2 (which is enough to commit the entries so far next time
# n1 runs).
stabilize 2
----
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
1->2 MsgApp Term:1 Log:1/4 Commit:3 Entries:[1/5 EntryNormal "foo"]
> 2 handling Ready
Ready MustSync=true:
Entries:
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/5
# Put another entry in n1's log.
propose 1 bar
----
ok
# n1 applies the conf change, so it has now removed itself. But it still has
# an uncommitted entry in the log. If the leader unconditionally counted itself
# as part of the commit quorum, we'd be in trouble. In the block below, we see
# it send out appends to the other nodes for the 'bar' entry.
stabilize 1
----
> 1 handling Ready
Ready MustSync=true:
Entries:
1/6 EntryNormal "bar"
Messages:
1->2 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"]
1->3 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"]
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/5
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:5
CommittedEntries:
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
1->2 MsgApp Term:1 Log:1/6 Commit:4
1->3 MsgApp Term:1 Log:1/6 Commit:4
1->2 MsgApp Term:1 Log:1/6 Commit:5
1->3 MsgApp Term:1 Log:1/6 Commit:5
INFO 1 switched to configuration voters=(2 3)
# n2 responds, n3 doesn't yet. Quorum for 'bar' should not be reached...
stabilize 2
----
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"]
1->2 MsgApp Term:1 Log:1/6 Commit:4
1->2 MsgApp Term:1 Log:1/6 Commit:5
> 2 handling Ready
Ready MustSync=true:
HardState Term:1 Vote:1 Commit:5
Entries:
1/6 EntryNormal "bar"
CommittedEntries:
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
INFO 2 switched to configuration voters=(2 3)
# ... which thankfully is what we see on the leader.
stabilize 1
----
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
# When n3 responds, quorum is reached and everything falls into place.
stabilize
----
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
1->3 MsgApp Term:1 Log:1/4 Commit:3 Entries:[1/5 EntryNormal "foo"]
1->3 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"]
1->3 MsgApp Term:1 Log:1/6 Commit:4
1->3 MsgApp Term:1 Log:1/6 Commit:5
> 3 handling Ready
Ready MustSync=true:
HardState Term:1 Vote:1 Commit:5
Entries:
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
1/6 EntryNormal "bar"
CommittedEntries:
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
3->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/5
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
INFO 3 switched to configuration voters=(2 3)
> 1 receiving messages
3->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/5
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/6 EntryNormal "bar"
Messages:
1->2 MsgApp Term:1 Log:1/6 Commit:6
1->3 MsgApp Term:1 Log:1/6 Commit:6
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/6 Commit:6
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/6 Commit:6
> 2 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/6 EntryNormal "bar"
Messages:
2->1 MsgAppResp Term:1 Log:0/6
> 3 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/6 EntryNormal "bar"
Messages:
3->1 MsgAppResp Term:1 Log:0/6
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
# However not all is well. n1 is still leader but unconditionally drops all
# proposals on the floor, so we're effectively stuck if it still heartbeats
# its followers...
propose 1 baz
----
raft proposal dropped
tick-heartbeat 1
----
ok
# ... which, uh oh, it does.
# TODO(tbg): change behavior so that a leader that is removed immediately steps
# down, and initiates an optimistic handover.
stabilize
----
> 1 handling Ready
Ready MustSync=false:
Messages:
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:6
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:6
> 2 receiving messages
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:6
> 3 receiving messages
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:6
> 2 handling Ready
Ready MustSync=false:
Messages:
2->1 MsgHeartbeatResp Term:1 Log:0/0
> 3 handling Ready
Ready MustSync=false:
Messages:
3->1 MsgHeartbeatResp Term:1 Log:0/0
> 1 receiving messages
2->1 MsgHeartbeatResp Term:1 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0