123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224 |
- # 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
|