Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

breaking: Make StepDownOnRemoval the default behavior #239

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 2 additions & 13 deletions raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,13 +277,6 @@ type Config struct {
// See: https://github.com/etcd-io/raft/issues/80
DisableConfChangeValidation bool

// StepDownOnRemoval makes the leader step down when it is removed from the
// group or demoted to a learner.
//
// This behavior will become unconditional in the future. See:
// https://github.com/etcd-io/raft/issues/83
StepDownOnRemoval bool

// raft state tracer
TraceLogger TraceLogger
}
Expand Down Expand Up @@ -418,7 +411,6 @@ type raft struct {
// when raft changes its state to follower or candidate.
randomizedElectionTimeout int
disableProposalForwarding bool
stepDownOnRemoval bool

tick func()
step stepFunc
Expand Down Expand Up @@ -460,7 +452,6 @@ func newRaft(c *Config) *raft {
readOnly: newReadOnly(c.ReadOnlyOption),
disableProposalForwarding: c.DisableProposalForwarding,
disableConfChangeValidation: c.DisableConfChangeValidation,
stepDownOnRemoval: c.StepDownOnRemoval,
traceLogger: c.TraceLogger,
}

Expand Down Expand Up @@ -1987,17 +1978,15 @@ func (r *raft) switchToConfig(cfg tracker.Config, trk tracker.ProgressMap) pb.Co
r.isLearner = ok && pr.IsLearner

if (!ok || r.isLearner) && r.state == StateLeader {
// This node is leader and was removed or demoted, step down if requested.
// This node is leader and was removed or demoted, step down.
//
// We prevent demotions at the time writing but hypothetically we handle
// them the same way as removing the leader.
//
// TODO(tbg): ask follower with largest Match to TimeoutNow (to avoid
// interruption). This might still drop some proposals but it's better than
// nothing.
if r.stepDownOnRemoval {
r.becomeFollower(r.Term, None)
}
r.becomeFollower(r.Term, None)
return cs
}

Expand Down
2 changes: 0 additions & 2 deletions rafttest/interaction_env_handler_add_nodes.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,8 +67,6 @@ func (env *InteractionEnv) handleAddNodes(t *testing.T, d datadriven.TestData) e
default:
return fmt.Errorf("invalid read-only option %q", arg.Vals[i])
}
case "step-down-on-removal":
arg.Scan(t, i, &cfg.StepDownOnRemoval)
}
}
}
Expand Down
87 changes: 13 additions & 74 deletions testdata/confchange_v1_remove_leader.txt
Original file line number Diff line number Diff line change
Expand Up @@ -78,10 +78,8 @@ 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.
# n1 applies the conf change, removing itself and stepping down. But it still
# has an uncommitted 'bar' entry in the log that it sends out appends for first.
stabilize 1
----
> 1 handling Ready
Expand All @@ -106,10 +104,14 @@ stabilize 1
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)
INFO 1 became follower at term 1
> 1 handling Ready
Ready MustSync=false:
Lead:0 State:StateFollower

raft-state
----
1: StateLeader (Non-Voter) Term:1 Lead:1
1: StateFollower (Non-Voter) Term:1 Lead:0
2: StateFollower (Voter) Term:1 Lead:1
3: StateFollower (Voter) Term:1 Lead:1

Expand All @@ -134,7 +136,7 @@ stabilize 2
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.
# ...because the old leader n1 ignores the append responses.
stabilize 1
----
> 1 receiving messages
Expand Down Expand Up @@ -174,77 +176,14 @@ stabilize
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...
# n1 can no longer propose.
propose 1 baz
----
INFO 1 no leader at term 1; dropping proposal
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

# Just confirming the issue above - leader does not automatically step down.
# Expected behavior: a new leader is elected after an election timeout.
raft-state
# Nor can it campaign to become leader.
campaign 1
----
1: StateLeader (Non-Voter) Term:1 Lead:1
2: StateFollower (Voter) Term:1 Lead:1
3: StateFollower (Voter) Term:1 Lead:1
WARN 1 is unpromotable and can not campaign
190 changes: 0 additions & 190 deletions testdata/confchange_v1_remove_leader_stepdown.txt

This file was deleted.

Loading