curator
curator copied to clipboard
CURATOR-644. CURATOR-645. Fix livelock in LeaderLatch
See also:
- https://issues.apache.org/jira/browse/CURATOR-644
- https://issues.apache.org/jira/browse/CURATOR-645
Livelock in details
Here we have two race conditions to cause livelock:
Case 1. Suppose there are two participants, p0 and p1:
T0. p1 is going to watch on preceding node belongs to p0. T1. p0 gets reconnected, and thus reset its node, and create a new node to prepare watch on p1's node. T2. p1 find preceding node has gone, and reset itself.
At the moment, p0 and p1 can be in the livelock that never see each other's node and infinitely reset themselves. This is the case reported by CURATOR-645.
Case 2. The similar case can happen even if there is only one participant:
~~If we still call reset
when preceding node deleted by latter set new node, it's a live lock.~~
I cannot find a live lock here. If only the background in the same client are executed in serial, there are always three nodes to create, while with this patch, there are two nodes to create. But it should not create millions of nodes. If it's in case 1, it's possible since there's no guarantee between different clients.
This is the case reported by CURATOR-644.
Solution
I make two significant changes to resolve these livelock cases:
- Call
getChildren
instead ofreset
when preceding node not found in callback. This is previously reported in https://github.com/apache/curator/commit/ff4ec29f5958cc9162f0302c02f4ec50c0e796cd#r31770630. I don't find a reason we perform different between callback and watcher for the same condition. And concurrentreset
s are the trigger for these livelock. - Call
getChildren
instead ofreset
when recovered from connection loss. The reason is similar to 1, while if a connection loss or session expire cause our node to be deleted, whencheckLeadership
we can see the condition and callreset
.
These changes should fix CURATOR-645 and CURATOR-644.
I'm trying to add test cases and such changes must involve more eyes.
Incorrect analyze
However, as long as there's possibility to generate concurrent checkLeadership a participant can race itself. I ever thought we can use a checkLeadershipLock here but since all client request are handled in callbacks, the lock can protect little.If you have an idea to fix one participant multiple threads race condition, please comment.
The race condition is:
T0. In thread 0 (th0) p is going to getChildren T1. In thread 1 (th1) p gets reconnected and calls getChildren T2. Supposed the node has gone due to session expire, th0 cannot find ourPath and reset T3. th1 gets a children set without the new node th0 created, or even reach checkLeadership before th0 creates new node. Then th1 reset.
The problem here is that we create node, get children all asynchronously, so even we add a lock in checkLeadership the callback can overwrite status. Even we lock callbacks, the creation can be uncompleted. The root cause should be that we should no have two competing threads for one participant.
UPDATE 1 - I notice that background are running serially and the order is the same as request sending. Perhaps we can setup invariant based on this premise. This is the case of CURATOR-644. CURATOR-645 is about 2 participants.
UPDATE 2 - It seems with the assumption CURATOR-644 should be fixed also. The race condition shown above last cannot happen because if th1 getChildren
after th0 create
, th1 can find the node exists; if th1 getChildren
before th0 create
, th0 will be the leader first and th1 will be the leader later. Although it causes once more leader switch, it converges eventually determinately.
cc @rikimberley @yuri-tceretian
@XComp they're logically resolved simultaneously. That is, if you resolve CURATOR-644, you resolve CURATOR-645 - they're the same sort.
In another word, you can check out the diff and tell me how to split it up into two PRs.
Fair enough. I did go through some of the commits in my IDE but just stop (and started writing my comment) before noticing that most of the commits get reverted again. That made me think that there's more stuff needed for CURATOR-644. Never mind...
they're logically resolved simultaneously. That is, if you resolve CURATOR-644, you resolve CURATOR-645 - they're the same sort.
In another word, you can check out the diff and tell me how to split it up into two PRs.
I was thinking about it once more. CURATOR-645 could be covered separately in my opinion. CURATOR-645 was identified in FLINK-27078 where we run almost no logic before revoking the leadership by calling LeaderLatch#close
. That caused the current leader's LeaderLatch
instance to trigger its child node deletion while other LeaderLatch
instances were right within setting up the watcher for its child node's predecessor.
Hence, I see CURATOR-645 being not that tightly related with the reconnect issue covered in CURATOR-644. CURATOR-645 just needs to be resolved before CURATOR-644 can be resolved.
Anyway, the changes are not that big in the end that we couldn't resolve both in the same PR. ¯_(ツ)_/¯
Thanks for your inputs @XComp!
I'll try to integrate your comment this week or the next.
Since we merge several fixes into the master branch, and there're users asking for a new release, if we don't have a consensus on this patch, I'll push the changes on debug logging first so that reporters of CURATOR-644 and CURATOR-645 can use the new version to provide an exact error log output :)
Updated. I believe this patch is ready to merge.
Please help with reviewing @eolivelli @Randgalt @cammckenzie
@eolivelli @cammckenzie @Randgalt Perhaps we can release a 5.4.0 later this month and I'd ask for a review on this patch for a consensus whether we include it or only the debug logging part.
Merging...
I'm looking into a related change #398 and then prepare the next release.