...

Text file src/go.etcd.io/etcd/raft/v3/testdata/confchange_v1_remove_leader.txt

Documentation: go.etcd.io/etcd/raft/v3/testdata

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

View as plain text