-
Notifications
You must be signed in to change notification settings - Fork 1.3k
CURATOR-644. CURATOR-645. Fix livelock in LeaderLatch #430
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
Conversation
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
Incorrect analyzeHowever, 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 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 |
Signed-off-by: tison <wander4096@gmail.com>
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
Outdated
Show resolved
Hide resolved
XComp
left a comment
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's the motivation behind putting the solution for both Jira issues into a single PR? Wouldn't it more reasonable to split it up into two PRs analgously to the Jira issues?
|
@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... |
XComp
left a comment
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 guess, we have to reiterate over the test once more. It succeeded even with the fix in the production code being reverted.
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
Outdated
Show resolved
Hide resolved
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 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 :) |
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Outdated
Show resolved
Hide resolved
Signed-off-by: tison <wander4096@gmail.com> Co-authored-by: Matthias Pohl <matthias.pohl@aiven.io>
Signed-off-by: tison <wander4096@gmail.com> Co-authored-by: Matthias Pohl <matthias.pohl@aiven.io>
|
Updated. I believe this patch is ready to merge. Please help with reviewing @eolivelli @Randgalt @cammckenzie |
XComp
left a comment
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 went over my proposed test once more and added a few comments. Please see them below.
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
Outdated
Show resolved
Hide resolved
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
Outdated
Show resolved
Hide resolved
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
Outdated
Show resolved
Hide resolved
Co-authored-by: Matthias Pohl <matthias.pohl@aiven.io>
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.
Going over the PR once more with the new test proposal for CURATOR-645, I would vote for splitting up the two Jira issues instead of handling them in a single PR. CURATOR-645 is now clearly defined by the change you proposed and the test case we came up with together.
We still have to revisit CURATOR-644 (see my comment below on the production code change). Additionally, we have to come up with a test case for that CURATOR-644, still. Anyway, blocking CURATOR-645 on CURATOR-644 just because the latter one is based on the former one to solve both in a single PR is not needed and also might cause confusion later on. WDYT?
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Outdated
Show resolved
Hide resolved
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
Outdated
Show resolved
Hide resolved
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
Signed-off-by: tison <wander4096@gmail.com>
Signed-off-by: tison <wander4096@gmail.com>
Signed-off-by: tison <wander4096@gmail.com>
Signed-off-by: tison <wander4096@gmail.com>
XComp
left a comment
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.
Thanks, @tisonkun for applying my comments. I have a few minor things to add. ...mostly cosmetic comments. I'd be curious about your opinion.
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Outdated
Show resolved
Hide resolved
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Show resolved
Hide resolved
…ed if we haven't lost the child node after a reconnect. (#2)
|
@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. |
XComp
left a comment
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.
Looks good from my end. Both changes make sense in my opinion and are covered by tests now. 👍
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Outdated
Show resolved
Hide resolved
…ipes/leader/LeaderLatch.java Co-authored-by: Matthias Pohl <matthias.pohl@aiven.io>
curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
Outdated
Show resolved
Hide resolved
Randgalt
left a comment
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 see any issues, however it's been a very long time since I've looked at this code.
Signed-off-by: tison <wander4096@gmail.com>
Signed-off-by: tison <wander4096@gmail.com>
|
Merging... I'm looking into a related change #398 and then prepare the next release. |
|
It caused a new issue. |
|
@ImagineBrain It may be fixed by #500. Would you upgrade to Curator 5.7.0 and recheck? |
@tisonkun I've tried 5.7.1, it helped nothing if the leaderPath didn't come back after reconnected. But I found a workaround, adding a ConnectionStateListener to create the leaderPath node, then all things will be fine. |
See also:
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 callresetwhen 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:
getChildreninstead ofresetwhen preceding node not found in callback. This is previously reported in ff4ec29#r31770630. I don't find a reason we perform different between callback and watcher for the same condition. And concurrentresets are the trigger for these livelock.getChildreninstead ofresetwhen recovered from connection loss. The reason is similar to 1, while if a connection loss or session expire cause our node to be deleted, whencheckLeadershipwe 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.