confchange_v1_remove_leader.txt 5.8 KB

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