ratis
ratis copied to clipboard
RATIS-1273. Fix split brain by leader lease
What changes were proposed in this pull request?
What's the problem ?
For example, there are 3 servers: s1, s2, s3, and s1 is leader. When split-brain happens, s2 was elected as new leader, but s1 still think it's leader, when client read from s1, if s2 has processed write request, client will read old data from s1.
How to fix ?
As the raft paper described, assign the leader with a lease, the leader would use the normal heartbeat mechanism to maintain a lease. Once the leader’s heartbeats were acknowledged by a majority of the cluster, it would extends its lease to start+ election timeout
, since the followers shouldn’t time out before then, so we can make sure there will no new leader was elected(need pre-vote feature and need to consider transferLeadership feature) , so before start + election timeout, there will not split-brain happens.
.
[TODO] Why need pre-vote feature ?
As the image shows, s1 is leader, but s1 can not connect with s2, even though s1 extend its lease to start+ election timeout when s1 receive acknowledgement from s3, but before start+ election timeout, s1 isolated from all servers, and s2 maybe timeout and start election and change to leader immediately with vote from s3, so both s1 and s2 think itself as leader before start+ election timeout. But if with pre-vote feature, when s2 request vote, s3 check s1's leadership is still valid, s3 will reject vote to s2, only one leader exists.
[TODO] How to address transferLeadership ?
For example, s1 is leader and extend its lease to start+ election timeout
when s1 receive acknowledgement from s2 and s3. But before start+ election timeout
, admin maybe call transferLeadership(s2), after s1 send StartLeaderElectionRequest to s2, s1 isolated from all servers, then s2 start election and change to leader immediately with vote from s3, so both s1 and s2 think itself as leader before start+ election timeout.
So s1 should step down as a follower when s1 send StartLeaderElectionRequest to s2.
@szetszwo Could you help review this proposal ?
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1273
How was this patch tested?
TODO
@szetszwo Could you help review this proposal ?
Sure, will review this.
The design looks good. Will look at the code changes.
Question: when the leader has lost the leader lease, should it step down?
@szetszwo Hi, because this PR depends on Pre-Vote PR: https://github.com/apache/incubator-ratis/pull/161. So I want to modify Pre-Vote PR first, after Pre-Vote PR was merged, then I continue to work on this PR. what do you think ?
Sure, the plan sounds great.
We may not need the LEADER_LEASE_TIMEOUT_RATIO_KEY at the very beginning. ...
I agree with @GlenGeng that we may not need LEADER_LEASE_TIMEOUT_RATIO_KEY. If we take rpc-send-time right before sending out appendEntries, it seems pretty safe to use min-rpc-timeout as the leader-lease-timeout. If split brain happens, it has to take at least (min-rpc-timeout + leader election time) to elect a new leader. Then, the old leader lease must be expired by that time.
@runzhiwang , what do you think?
I agree with @GlenGeng that we may not need LEADER_LEASE_TIMEOUT_RATIO_KEY. If we take rpc-send-time right before sending out appendEntries, it seems pretty safe to use min-rpc-timeout as the leader-lease-timeout. If split brain happens, it has to take at least (min-rpc-timeout + leader election time) to elect a new leader. Then, the old leader lease must be expired by that time.
@szetszwo I agree. There are some failed ut related to this PR, let me fix them.
@runzhiwang , thanks a lot.
BTW, we should add confs to enable/disable PreVote and LeaderLease. Some applications may not require these features. This is suggested by @bshashikant .
@szetszwo @bharatviswa504 Thanks, got it. I will add config in next PR.
@szetszwo @bharatviswa504 Sorry, I have a question, I understand LeaderLease maybe not needed in some applications. But in my thinking PreVote is needed in any application, it can make the leader more steady, what do you think ?
@runzhiwang , in general, I agree that PreVote should help for all the applications. However, PreVote needs an additional phase before the real election. It could potentially slows down some applications. Individual applications like Ozone may want to benchmark it. If it is not configurable, it is impossible to benchmark.
@szetszwo Thanks, got it. With leader lease, some ut become flaky, I need some time to fix them.
@runzhiwang , No problem. Please take you time. Thanks a lot for working hard on this.
For now, me and @runzhiwang is developing SCM HA.
In SCM HA, SCM will cache isLeader
and term
, updating them when underlying RaftServer steps down or becomes leader, by implementing StateMachine#notifyNotLeader
and StateMachine#notifyLeaderChanged
.
SCM HA does not invoke DivisionInfo#isLeader()
but query its cached isLeader
to decide whether underlying RaftServer is leader or not, thus it expects RaftServer to step down when lease is expired.
I suggest to implement the leader with lease solution in LeaderStateImpl#checkLeadership
, and add a switch to decide to step down whether when lease is expired or when it can not hear from majority.
What do you think @szetszwo @runzhiwang ?
@szetszwo Hi, with leader lease, the CI becomes flaky, there are 2 reasons:
- The CI environment' machine has a low performance, some times one rpc call cost more then 20ms from send to receive.
- In current ratis implementation, leader sends log or heartbeat to follower every 75ms, if there is log, leader will not send heartbeat again. For example, at 0ms there is log and leader send log to follower, at 75ms there is log and leader send log to follower again, ..., at 750ms there is log and leader send log to follower again. So you can find from 0ms to 750ms, log always exist, leader always send log, never heartbeat. But if each log need more than 1000ms to: WriteDisk, applyTransaction, then leader will not receive any reply from 0-1000ms, then leader lease becomes invalid frequently.
I think we have following options:
- Increase rpc.timeout.min from 150ms to 1500ms in CI
- Default disable leader lease, then CI need not to consider leader lease
What do you think ?
@runzhiwang , thanks again for working on this.
- In current ratis implementation, leader sends log or heartbeat to follower every 75ms, if there is log, leader will not send heartbeat again. For example, at 0ms there is log and leader send log to follower, at 75ms there is log and leader send log to follower again, ..., at 750ms there is log and leader send log to follower again. So you can find from 0ms to 750ms, log always exist, leader always send log, never heartbeat. But if each log need more than 1000ms to: WriteDisk, applyTransaction, then leader will not receive any reply from 0-1000ms, then leader lease becomes invalid frequently.
With the leader lease feature, the leader probably should send heartbeat separately since followers may take a long time to process log entires. (The followers do not count the log processing time when counting heartbeat timeout. However, it is impossible for the leader to do the same discount.)
- Default disable leader lease, then CI need not to consider leader lease
Let's disable leader lease as default. When the feature becomes stable, we can change the default to enable.
When the feature becomes stable, we can change the default to enable.
@szetszwo Hi, I find it's almost impossible to enable leader lease in CI, because sometimes it cost 300ms from leader send heart to follower receive heartbeat. So CI will become very unstable, unless we increase rpc.timeout.min.
Besides, what do you think of @GlenGeng 's suggestion: leader step down when lease become invalid ? In SCM HA, we do not check isLeaderReady, we depends on StateMachine#notifyLeaderChanged to change leadership.
..., I find it's almost impossible to enable leader lease in CI, because sometimes it cost 300ms from leader send heart to follower receive heartbeat. So CI will become very unstable, unless we increase rpc.timeout.min.
Yes, we may increase rpc.timeout.min if necessary.
... leader step down when lease become invalid ? ...
Let's also make it configurable? It seems that both ways have its own benefit.
Let's also make it configurable? It seems that both ways have its own benefit.
@szetszwo Thanks, I agree.
this pr depends on: https://github.com/apache/incubator-ratis/pull/398
hi, i am not sure are you still working on this jira? as a good raft implementation, i think leader lease is very important for ratis, we should continue and complete the work. if Needed, it is my pleasure to continue this work! @szetszwo
It seems that @runzhiwang is no longer working on this. (Please correct me if I am wrong.)
@JacksonYao287 , please feel free to take over this. Thanks a lot.
@szetszwo sorry for the delay. @JacksonYao287 please feel free to take over this, thanks.