-
Notifications
You must be signed in to change notification settings - Fork 9.8k
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
raft: introduce/fix TestPreVoteWithCheckQuorum #8334
raft: introduce/fix TestPreVoteWithCheckQuorum #8334
Conversation
@javaforfun can you add a test case? |
fa9c6f7
to
bf883bc
Compare
@xiang90 done. add two test |
seems confusing when raft recv a message with higher term, i will create a new pr for this |
bf883bc
to
22fb6a3
Compare
raft/raft_test.go
Outdated
|
||
sm := nt.peers[1].(*raft) | ||
if sm.state != StateLeader { | ||
t.Errorf("peer 1 state: %s, want %s", sm.state, StateLeader) |
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.
use t.fatalf for checking the pre-condition. use t.errorf for the actual state checking.
raft/raft_test.go
Outdated
t.Errorf("peer 3 state: %s, want %s", sm.state, StatePreCandidate) | ||
} | ||
|
||
sm.logger.Infof("going to bring back peer 3 and kill peer 1") |
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.
remove the logging.
raft/raft_test.go
Outdated
sm2 := nt.peers[2].(*raft) | ||
sm3 := nt.peers[3].(*raft) | ||
if sm2.state != StateLeader && sm3.state != StateLeader { | ||
t.Errorf("no leader") |
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.
errorf needs better context. see other tests's errorf output as examples
can you take a quick look too? |
raft/raft.go
Outdated
@@ -611,6 +611,7 @@ func (r *raft) becomePreCandidate() { | |||
// but doesn't change anything else. In particular it does not increase | |||
// r.Term or change r.Vote. | |||
r.step = stepCandidate | |||
r.lead = None |
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.
It is strange to change r.lead
here when we do not change r.Term
. r.lead
is still the leader of that term. I think we should leave r.lead
unchanged here, and if we change anything, we should change the inLease
check at line 711. We can't become a pre-candidate if we've heard from the leader within the lease timeout, so if state == StatePreCandidate
inLease
must be false.
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.
But some users may use hasLeader()
to check whether cluster is in normal state. It's too tricky to let users know StatePreCandidate
or inLease
.
Also in raft paper, it just say:
If a follower receives no communication over a period of time called the election timeout,
then it assumes there is no viable leader and begins an election to choose a new leader.
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.
it assumes there is no viable leader and begins an election to choose a new leader.
begins election means increasing term, right? The pre candidate phase seems like a grey area. But I prefer to keep the leader field to be not changed as @bdarnell suggested.
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.
@xiang90 I think thesis section 6.2 point out how to handle this situation. see #8334 (comment)
I've found another interesting scenario for Pre-Vote together with checkQuorum.
it's strange that one node think another node is leader, but the other node is not leader, and they are in same partition! |
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.
Does it make sense to use both CheckQuorum and PreVote together? In CockroachDB we were using them for the same purpose so we turned off CheckQuorum when we enabled PreVote.
I think that only part of CheckQuorum's functionality is needed when PreVote is enabled: you want the leader to step down, but you don't need to change the way votes are granted (I think). So what if we changed
to if !force && inLease && !r.preVote
?
raft/raft.go
Outdated
if r.checkQuorum { | ||
// If network partition happen, and leader is in minority partition, | ||
// it will step down, become follower without notifying others. | ||
r.lead = None |
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.
Why does this change matter?
It doesn't seem that strange to me that other nodes would still consider node A to be the leader when it has stepped down due to CheckQuorum.
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.
As described in thesis section 6.2, Routing requests to the leader,
Raft must also prevent stale leadership information from delaying client requests indefinitely. Leadership information can become stale all across the system, in leaders, followers, and clients: • Leaders: A server might be in the leader state, but if it isn’t the current leader, it could be needlessly delaying client requests. For example, suppose a leader is partitioned from the rest of the cluster, but it can still communicate with a particular client. Without additional mechanism, it could delay a request from that client forever, being unable to replicate a log entry to any other servers. Meanwhile, there might be another leader of a newer term that is able to communicate with a majority of the cluster and would be able to commit the client’s request. Thus, a leader in Raft steps down if an election timeout elapses without a successful round of heartbeats to a majority of its cluster; this allows clients to retry their requests with another server. • Followers: Followers keep track of the leader’s identity so that they can redirect or proxy clients. They must discard this information when starting a new election or when the term changes. Otherwise, they might needlessly delay clients (for example, it would be possible for two servers to redirect to each other, placing clients in an infinite loop).
followers need discard leader information when starting a new election or when the term changes.
if not, in the previous scenario, partition [A, B] has A with lead = None, B with lead = A,
if client request servers with alphabetical order, it could delay forever.
CheckQuorum can prevent disruptive servers, which described in thesis section 4.2.3. |
Refer to raft thesis
checkQuorum seems good for disruptive server which had been removed from the cluster, but it don't resolve the issue of network jitter. So it's good to enable both checkQuorum and Pre-vote. |
We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334.
We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334.
We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334.
We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334.
* tests: Introduce prevote with check quorum test. We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334. * raft: Introduce prevote/check_quorum compatability fix. * raft: Always reset leader in precandidate
So is this ever going to get merged? We've hit this issue in production, where with checkQuorum and preVote activated we are in an endless loop of ignoring MsgPreVotes. 3 nodes. Leader goes down. Other two endlessly try to PreVote and keep ignoring each other. The randomization in "pastElectionTimeout" which I think is supposed to prevent this is not working. |
or maybe this: #8346 |
No, that is not what I am seeing. In that scenario, B and C grant each other's votes. In my scenario B ignores Cs vote and C ignores B's vote. Also, we have that change in the version we are using. I am fairly confident we are seeing the issue described here related to the lease not expiring.
So I am wondering if we can get these changes merged as they are fairly small, and have been sitting here for over a year.
On Nov 21, 2018 1:48 AM, ls <[email protected]> wrote:
So is this ever going to get merged? We've hit this issue in production, where with checkQuorum and preVote activated we are in an endless loop of ignoring MsgPreVotes. 3 nodes. Leader goes down. Other two endlessly try to PreVote and keep ignoring each other. The randomization in "pastElectionTimeout" which I think is supposed to prevent this is not working.
I don't think your problem is related to this RP. But it seems you've met this: #8346<#8346>
—
You are receiving this because you commented.
Reply to this email directly, view it on GitHub<#8334 (comment)>, or mute the thread<https://github.com/notifications/unsubscribe-auth/AAvL8nte6qFehQqFeMNJtcDpFDPT-kF7ks5uxSGIgaJpZM4OnTvd>.
|
@xiang90 PTAL |
raft/raft.go
Outdated
@@ -611,6 +611,11 @@ func (r *raft) becomePreCandidate() { | |||
// but doesn't change anything else. In particular it does not increase | |||
// r.Term or change r.Vote. | |||
r.step = stepCandidate | |||
if r.checkQuorum { |
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.
I don't think this should be based on r.checkQuorum
. That implies that we are doing this because we know that the leader is failing its quorum check, but we can't know that. The leader could be in the majority side of the partition and still be acting as leader.
#9586 discussed setting r.lead = None
in becomePreCandidate
whether checkQuorum
is true or not. I think that could work here by keeping things from getting permanently stuck, but it has the odd effect that the second (pre-)campaigner wins. For that reason I still think it would be better to modify the definition of inLease
at line 711 by adding && r.state == StateFollower
.
On the other hand, that's a more complex and invasive change, and I think this version is simple and safe. It makes sense that pre-candidates act like they don't know the leader (same as candidates). I was previously concerned that it would be unusual for a node to transition between r.lead
set and unset within a single term, but leaders already do that for checkQuorum
so it should be fine.
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.
Approved, but I think it would be better to set r.lead = None
in becomeCandidate
unconditionally, instead of only when r.checkQuorum
is enabled.
There's also a jenkins failure that I can't see (any more?); maybe it will go away if the branch is updated.
f35ab44
to
ae62732
Compare
Codecov Report
@@ Coverage Diff @@
## master #8334 +/- ##
==========================================
- Coverage 71.84% 71.77% -0.08%
==========================================
Files 391 391
Lines 36417 36418 +1
==========================================
- Hits 26164 26139 -25
- Misses 8438 8464 +26
Partials 1815 1815
Continue to review full report at Codecov.
|
ae62732
to
f720bc4
Compare
f720bc4
to
23731bf
Compare
@bdarnell I've updated the test case. |
lgtm |
@lishuai87 Would it be possible for you to release a new version with this fix? I am currently experiencing this bug. |
|
+1. Can reproduce this with my jepsen test as well. |
raft: cherry pick of #8334 to release-3.3
* tests: Introduce prevote with check quorum test. We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334. * raft: Introduce prevote/check_quorum compatability fix. * raft: Always reset leader in precandidate
* tests: Introduce prevote with check quorum test. We discovered some instances where prevote and check quorum interact poorly. This commit introduces a test showcasing this. Related to etcd-io/etcd#8334. * raft: Introduce prevote/check_quorum compatability fix. * raft: Always reset leader in precandidate
raft: set leader id to none when becomePreCandidate.
When a follower election timeout, it should mark leader to none (so that old leader is not in lease). Current implementation of Pre-Vote ignore this, which may cause wrong judgement .