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

Open
wants to merge 5 commits into
base: master
from

Conversation

4 participants
@lishuai87
Contributor

lishuai87 commented Jul 29, 2017

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 .

@xiang90

This comment has been minimized.

Show comment
Hide comment
@xiang90

xiang90 Jul 29, 2017

Contributor

@javaforfun can you add a test case?

Contributor

xiang90 commented Jul 29, 2017

@javaforfun can you add a test case?

@lishuai87

This comment has been minimized.

Show comment
Hide comment
@lishuai87

lishuai87 Jul 30, 2017

Contributor

@xiang90 done. add two test

Contributor

lishuai87 commented Jul 30, 2017

@xiang90 done. add two test

@lishuai87 lishuai87 changed the title from set lead = None when becomePreCandidate to raft: introduce/fix TestPreVoteWithCheckQuorum Jul 30, 2017

@lishuai87

This comment has been minimized.

Show comment
Hide comment
@lishuai87

lishuai87 Jul 31, 2017

Contributor

seems confusing when raft recv a message with higher term, i will create a new pr for this

Contributor

lishuai87 commented Jul 31, 2017

seems confusing when raft recv a message with higher term, i will create a new pr for this

@xiang90 xiang90 added the Raft label Aug 1, 2017

Show outdated Hide outdated raft/raft_test.go
sm := nt.peers[1].(*raft)
if sm.state != StateLeader {
t.Errorf("peer 1 state: %s, want %s", sm.state, StateLeader)

This comment has been minimized.

@xiang90

xiang90 Aug 1, 2017

Contributor

use t.fatalf for checking the pre-condition. use t.errorf for the actual state checking.

@xiang90

xiang90 Aug 1, 2017

Contributor

use t.fatalf for checking the pre-condition. use t.errorf for the actual state checking.

Show outdated Hide outdated raft/raft_test.go
t.Errorf("peer 3 state: %s, want %s", sm.state, StatePreCandidate)
}
sm.logger.Infof("going to bring back peer 3 and kill peer 1")

This comment has been minimized.

@xiang90

xiang90 Aug 1, 2017

Contributor

remove the logging.

@xiang90

xiang90 Aug 1, 2017

Contributor

remove the logging.

Show outdated Hide outdated raft/raft_test.go
sm2 := nt.peers[2].(*raft)
sm3 := nt.peers[3].(*raft)
if sm2.state != StateLeader && sm3.state != StateLeader {
t.Errorf("no leader")

This comment has been minimized.

@xiang90

xiang90 Aug 1, 2017

Contributor

errorf needs better context. see other tests's errorf output as examples

@xiang90

xiang90 Aug 1, 2017

Contributor

errorf needs better context. see other tests's errorf output as examples

@xiang90

This comment has been minimized.

Show comment
Hide comment
@xiang90

xiang90 Aug 1, 2017

Contributor

@bdarnell

can you take a quick look too?

Contributor

xiang90 commented Aug 1, 2017

@bdarnell

can you take a quick look too?

Show outdated Hide outdated raft/raft.go
@@ -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

This comment has been minimized.

@bdarnell

bdarnell Aug 7, 2017

Member

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.

@bdarnell

bdarnell Aug 7, 2017

Member

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.

This comment has been minimized.

@lishuai87

lishuai87 Aug 8, 2017

Contributor

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.

@lishuai87

lishuai87 Aug 8, 2017

Contributor

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.

This comment has been minimized.

@xiang90

xiang90 Aug 8, 2017

Contributor

@javaforfun

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.

@xiang90

xiang90 Aug 8, 2017

Contributor

@javaforfun

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.

This comment has been minimized.

@lishuai87

lishuai87 Aug 25, 2017

Contributor

@xiang90 I think thesis section 6.2 point out how to handle this situation. see #8334 (comment)

@lishuai87

lishuai87 Aug 25, 2017

Contributor

@xiang90 I think thesis section 6.2 point out how to handle this situation. see #8334 (comment)

lishuai87 added some commits Aug 8, 2017

@lishuai87

This comment has been minimized.

Show comment
Hide comment
@lishuai87

lishuai87 Aug 24, 2017

Contributor

I've updated this pr, please take a look. @xiang90 @bdarnell

Contributor

lishuai87 commented Aug 24, 2017

I've updated this pr, please take a look. @xiang90 @bdarnell

@lishuai87

This comment has been minimized.

Show comment
Hide comment
@lishuai87

lishuai87 Aug 24, 2017

Contributor

I've found another interesting scenario for Pre-Vote together with checkQuorum.

  1. cluster have 5 nodes [A, B, C, D, E], with leader A, all terms are 1.
  2. network partition happen, cluster have two partition [A, B], [C, D, E].
    so [A, B] has leader A with term 1, [C, D, E] has leader C with term 2.
  3. A will checkQuorum after election timeout, and become follower, set leader to None with term 1.
    B start pre election, but B has leader = A with term 1.

it's strange that one node think another node is leader, but the other node is not leader, and they are in same partition!
so if leader can become follower, and change r.lead to none with term not changed, follower also can do this.

Contributor

lishuai87 commented Aug 24, 2017

I've found another interesting scenario for Pre-Vote together with checkQuorum.

  1. cluster have 5 nodes [A, B, C, D, E], with leader A, all terms are 1.
  2. network partition happen, cluster have two partition [A, B], [C, D, E].
    so [A, B] has leader A with term 1, [C, D, E] has leader C with term 2.
  3. A will checkQuorum after election timeout, and become follower, set leader to None with term 1.
    B start pre election, but B has leader = A with term 1.

it's strange that one node think another node is leader, but the other node is not leader, and they are in same partition!
so if leader can become follower, and change r.lead to none with term not changed, follower also can do this.

@bdarnell

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

https://github.com/coreos/etcd/blob/4ec31f4f7f95b8ffbacc7a3d0f7173a44b1f11e8/raft/raft.go#L712

to if !force && inLease && !r.preVote?

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

This comment has been minimized.

@bdarnell

bdarnell Aug 24, 2017

Member

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.

@bdarnell

bdarnell Aug 24, 2017

Member

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.

This comment has been minimized.

@lishuai87

lishuai87 Aug 25, 2017

Contributor

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.

@lishuai87

lishuai87 Aug 25, 2017

Contributor

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.

@lishuai87

This comment has been minimized.

Show comment
Hide comment
@lishuai87

lishuai87 Aug 25, 2017

Contributor

CheckQuorum can prevent disruptive servers, which described in thesis section 4.2.3.

Contributor

lishuai87 commented Aug 25, 2017

CheckQuorum can prevent disruptive servers, which described in thesis section 4.2.3.

@es-chow

This comment has been minimized.

Show comment
Hide comment
@es-chow

es-chow Aug 30, 2017

Contributor

Refer to raft thesis 9.6 Preventing disruptions when a server rejoins the cluster.
it seems Pre-Vote is good for network partition or jitter. For example.

  1. Assume cluster have 3 nodes [A, B, C], the network between C and A,B is not stable, the link will jitter every hour.
  2. If without Pre-vote, node C will election periodically to make the cluster leader step down. But if with Pre-Vote, node A/B will work as node C cannot increase term to step down cluster leader.

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.

Contributor

es-chow commented Aug 30, 2017

Refer to raft thesis 9.6 Preventing disruptions when a server rejoins the cluster.
it seems Pre-Vote is good for network partition or jitter. For example.

  1. Assume cluster have 3 nodes [A, B, C], the network between C and A,B is not stable, the link will jitter every hour.
  2. If without Pre-vote, node C will election periodically to make the cluster leader step down. But if with Pre-Vote, node A/B will work as node C cannot increase term to step down cluster leader.

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.

Hoverbear added a commit to pingcap/raft-rs that referenced this pull request Jul 2, 2018

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.

Hoverbear added a commit to pingcap/raft-rs that referenced this pull request Jul 2, 2018

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.

Hoverbear added a commit to pingcap/raft-rs that referenced this pull request Jul 3, 2018

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.

Hoverbear added a commit to pingcap/raft-rs that referenced this pull request Jul 4, 2018

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.

Hoverbear added a commit to pingcap/raft-rs that referenced this pull request Jul 5, 2018

Make peace between Prevote and Check Quorum (#84)
* 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
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment