-
Notifications
You must be signed in to change notification settings - Fork 997
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
Fix rare leadership transfer failures when writes happen during transfer #581
Changes from all commits
e338309
a76a6f8
8cecc28
477cf7e
2b715ac
04fdca6
cb62297
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2337,17 +2337,71 @@ func TestRaft_LeadershipTransferWithOneNode(t *testing.T) { | |
} | ||
} | ||
|
||
func TestRaft_LeadershipTransferWithWrites(t *testing.T) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you have anecdata about roughly how frequently this reproduced a bug before your fix locally? I assume it's still non-deterministic? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I couldn't give you numbers, but it didn't take many tries to provoke a failure. It can still fail if I up the concurrency enough, e.g. if I run 16 parallel -race instances. But I suspect the same is true for many other tests, as the failures are things like timeouts due to heartbeats not being processed quickly enough. |
||
conf := inmemConfig(t) | ||
conf.Logger = hclog.New(&hclog.LoggerOptions{Level: hclog.Trace}) | ||
c := MakeCluster(7, t, conf) | ||
defer c.Close() | ||
|
||
doneCh := make(chan struct{}) | ||
var writerErr error | ||
var wg sync.WaitGroup | ||
var writes int | ||
wg.Add(1) | ||
leader := c.Leader() | ||
go func() { | ||
defer wg.Done() | ||
for { | ||
select { | ||
case <-doneCh: | ||
return | ||
default: | ||
future := leader.Apply([]byte("test"), 0) | ||
switch err := future.Error(); { | ||
case errors.Is(err, ErrRaftShutdown): | ||
return | ||
case errors.Is(err, ErrNotLeader): | ||
leader = c.Leader() | ||
case errors.Is(err, ErrLeadershipTransferInProgress): | ||
continue | ||
case errors.Is(err, ErrLeadershipLost): | ||
continue | ||
case err == nil: | ||
writes++ | ||
default: | ||
writerErr = err | ||
} | ||
time.Sleep(time.Millisecond) | ||
} | ||
} | ||
}() | ||
|
||
follower := c.Followers()[0] | ||
future := c.Leader().LeadershipTransferToServer(follower.localID, follower.localAddr) | ||
if future.Error() != nil { | ||
t.Fatalf("Didn't expect error: %v", future.Error()) | ||
} | ||
if follower.localID != c.Leader().localID { | ||
t.Error("Leadership should have been transitioned to specified server.") | ||
} | ||
close(doneCh) | ||
wg.Wait() | ||
if writerErr != nil { | ||
t.Fatal(writerErr) | ||
} | ||
t.Logf("writes: %d", writes) | ||
} | ||
|
||
func TestRaft_LeadershipTransferWithSevenNodes(t *testing.T) { | ||
c := MakeCluster(7, t, nil) | ||
defer c.Close() | ||
|
||
oldLeader := c.Leader().localID | ||
follower := c.GetInState(Follower)[0] | ||
future := c.Leader().LeadershipTransferToServer(follower.localID, follower.localAddr) | ||
if future.Error() != nil { | ||
t.Fatalf("Didn't expect error: %v", future.Error()) | ||
} | ||
if oldLeader == c.Leader().localID { | ||
if follower.localID != c.Leader().localID { | ||
t.Error("Leadership should have been transitioned to specified server.") | ||
} | ||
} | ||
|
@@ -2510,7 +2564,7 @@ func TestRaft_LeadershipTransferIgnoresNonvoters(t *testing.T) { | |
} | ||
|
||
func TestRaft_LeadershipTransferStopRightAway(t *testing.T) { | ||
r := Raft{leaderState: leaderState{}} | ||
r := Raft{leaderState: leaderState{}, logger: hclog.New(nil)} | ||
r.setupLeaderState() | ||
|
||
stopCh := make(chan struct{}) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What do you think about just staying in the loop here instead of starting a new
ElectionTimeout
ticker and having to duplicate the leftLeaderLoop code path too?I was originally expecting that we'd not wait for another
ElectionTimeout
but rather keep waiting for up to the original one we started for the transfer to complete. I don't think it's terrible to do it this way but it strikes me that it's less code and maybe simpler to reason able if the behaviour is just: block until leadership transfer works or one election timeout from handling the request...The rationale for using ElectionTimeout here presumably was that, LeadershipTransfer is only an optimization to avoid just ungracefully stopping and having to wait for an election timeout... so letting leader transfer take longer than a whole election timeout is a little bit defeating the point and we should probably return fast and let the old leader just shut down if it wants to.
I think that rationale gets a bit more subtle in cases like Autopilot upgrade where we rely on Leadership Transfer working before we move on to the next phase rather than just shutting down anyway.
tl;dr, I don't think the behaviour here is bad or wrong especially and it beats the bug without this wait. If it turns out easier to follow to duplicate the code and make the behavior more explicit like you have here I'm OK with it. Just curious if you tried simply replacing this
else
branch withcontinue
which I think is also a correct fix?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I tried that. It got messy due to the other cases wanting to read from doneCh. In the end I decided this was clearer.