Skip to content

Commit

Permalink
raft: document problem with leader self-removal
Browse files Browse the repository at this point in the history
When a leader removes itself, it will retain its leadership but not
accept new proposals, making the range effectively stuck until manual
intervention triggers a campaign event.

This commit documents the behavior. It does not correct it yet.
  • Loading branch information
tbg committed Aug 15, 2019
1 parent 9cde4a7 commit b6f35e5
Show file tree
Hide file tree
Showing 3 changed files with 198 additions and 35 deletions.
7 changes: 7 additions & 0 deletions raft/rafttest/interaction_env_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,13 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
//
// tick-heartbeat 3
err = env.handleTickHeartbeat(t, d)
case "propose":
// Propose an entry.
//
// Example:
//
// propose 1 foo
err = env.handlePropose(t, d)
case "propose-conf-change":
// Propose a configuration change.
//
Expand Down
34 changes: 34 additions & 0 deletions raft/rafttest/interaction_env_handler_propose.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2019 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package rafttest

import (
"testing"

"github.com/cockroachdb/datadriven"
)

func (env *InteractionEnv) handlePropose(t *testing.T, d datadriven.TestData) error {
idx := firstAsNodeIdx(t, d)
if len(d.CmdArgs) != 2 || len(d.CmdArgs[1].Vals) > 0 {
t.Fatalf("expected exactly one key with no vals: %+v", d.CmdArgs[1:])
}
return env.Propose(idx, []byte(d.CmdArgs[1].Key))
}

// Propose a regular entry.
func (env *InteractionEnv) Propose(idx int, data []byte) error {
return env.Nodes[idx].Propose(data)
}
192 changes: 157 additions & 35 deletions raft/testdata/confchange_v1_remove_leader.txt
Original file line number Diff line number Diff line change
Expand Up @@ -21,82 +21,204 @@ log-level debug
----
ok

# Remove n1.
# Start removing n1.
propose-conf-change 1 v1=true
r1
----
ok

stabilize
# 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
INFO 1 switched to configuration voters=(2 3)
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/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/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

# 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/3 Commit:3 Entries:[1/4 EntryConfChange r1]
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
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
INFO 2 switched to configuration voters=(2 3)
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/4
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6

# ... 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
INFO 3 switched to configuration voters=(2 3)
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
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
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
INFO 1 switched to configuration voters=(2 3)
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:4
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/4 EntryConfChange r1
1/6 EntryNormal "bar"
Messages:
1->2 MsgApp Term:1 Log:1/4 Commit:4
1->3 MsgApp Term:1 Log:1/4 Commit:4
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/4 Commit:4
1->2 MsgApp Term:1 Log:1/6 Commit:6
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/4 Commit:4
1->3 MsgApp Term:1 Log:1/6 Commit:6
> 2 handling Ready
INFO 2 switched to configuration voters=(2 3)
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:4
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/4 EntryConfChange r1
1/6 EntryNormal "bar"
Messages:
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/6
> 3 handling Ready
INFO 3 switched to configuration voters=(2 3)
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:4
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/4 EntryConfChange r1
1/6 EntryNormal "bar"
Messages:
3->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/6
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6

status 1
# 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
----
2: StateReplicate match=4 next=5
3: StateReplicate match=4 next=5
raft proposal dropped

# TODO(tbg): the leader now drops any proposals, but if it has any other
# uncommitted proposals in its log already, it will likely try to distribute
# them which could be buggy. Test that.
propose-conf-change 1 v1=true
v1
tick-heartbeat 1
----
raft proposal dropped
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

0 comments on commit b6f35e5

Please sign in to comment.