curator icon indicating copy to clipboard operation
curator copied to clipboard

CURATOR-653: fix potential double leader for LeaderLatch

Open woaishixiaoxiao opened this issue 4 years ago • 8 comments

When I use the LeaderLatch to select leader, there is a double-leader phenomenon. The timeline is as follows:

  1. The zk cluster switch leader node bescause of zxid overflow. The cluster is unavailable to the outside world
  2. A client(not leader befor zxid overflow) and B client(is leader before zxid overflow) enter the suspend state, B client set its leader status to false
  3. The zk cluster complete the leader node election and the cluster back to normal
  4. A client enter the reconnect state and call the reset function, set its leader status to false.
  5. B client enter the reconnect state, call the reset function. set its leader status to false. Delete its old path.
  6. A client receive preNodeDeleteEvent. Then getChildren from zkServer. Find itself is the smallest number and set itself as a leader.
  7. B client create a new temporary node and then getChildren from zkServer. Find itself not the node with the smallest serial number and listen to the previous node delete event.
  8. A client delete its old path.
  9. B client receive the preNodeDeleteEvent. then getchildren from zkServer. Find itself is the smallest sequence number and then set itself as a leader
  10. A client create a new temporary node and then getChildren from zkServer. Find itself not the node with the smallest serial number and listen to the previous node delete event. but it doesn't set itself as a non-leader state. because of the sixth step operation, A still is leader state now.
  11. now A client and B client are the leader at the same time

woaishixiaoxiao avatar Oct 29 '21 09:10 woaishixiaoxiao

@woaishixiaoxiao thanks for sharing your fix, do you think that we can add a test case to cover this change ?

eolivelli avatar Oct 29 '21 10:10 eolivelli

@woaishixiaoxiao thanks for sharing your fix, do you think that we can add a test case to cover this change ?

OK. I will try it.
and I find another question related to the leader-selection scenari. When zkServer switch the leader and then returns to normal, all clients will execute state switching: connected->suspend->reconn Because leaderlatch processing the reconn state will reset leader status, that is mean first set itself leader status false and then delete old temporary sequence Node and create a new one. This operation will cause the business side to perform a leader switch multiple. Some businesses don’t want to see such frequent switchovers happen such as mq. Also this operation will cause nodeDeleteEvent push once from zk server but client execute multiple times nodeDeleteCallback on same path because client saves mutiple watch local(create new path will getchild and listen. and prenodedeleteEvent also will getchild and listen ). Why don't we replace StandardConnectionStateErrorPolicy with SessionConnectionStateErrorPolicy? The above phenomenon will be avoided

woaishixiaoxiao avatar Oct 29 '21 10:10 woaishixiaoxiao

@woaishixiaoxiao thanks for sharing your fix, do you think that we can add a test case to cover this change ?

HI i have added a unit test. please approval thanks

woaishixiaoxiao avatar Nov 02 '21 15:11 woaishixiaoxiao

LGTM. I also think of this change days before. cc @eolivelli @Randgalt can you also give a review?

tisonkun avatar Sep 25 '22 14:09 tisonkun

@woaishixiaoxiao can you create a JIRA ticket on https://issues.apache.org/jira/projects/CURATOR for this patch?

tisonkun avatar Sep 25 '22 14:09 tisonkun

Well. After #430 merged the test added in this patch failed. Need a closer look.

tisonkun avatar Sep 27 '22 03:09 tisonkun

I adjust the test to inject force resets instead of depending on connection loss. Although this means it should be a non-real-world case now, I still agree on setLeadership(false) on checkLeadership find the latch isn't the leader. setLeadership(false) is idempotent.

tisonkun avatar Sep 29 '22 13:09 tisonkun

@tisonkun thanks for fixing the test @woaishixiaoxiao do you agree with @tisonkun 's fix ?

eolivelli avatar Sep 29 '22 13:09 eolivelli

@XComp Thank you very much for the comments. Could you send a pull request based on the current patch? I don't know whether I can directly merge on the forked repository but at least I can submit the patch when you made it :)

tisonkun avatar Oct 10 '22 02:10 tisonkun

@XComp Thank you very much for the comments. Could you send a pull request based on the current patch? I don't know whether I can directly merge on the forked repository but at least I can submit the patch when you made it :)

I created PR #436 but couldn't base it onto this PR. I handled my review comments individually to make it easier to select relevant changes. The last commit is a bigger refactoring of the test. You might want to leave that out if you think that it's too much of a change.

XComp avatar Oct 10 '22 13:10 XComp

but couldn't base it onto this PR

Actually, you can send a pull request with base branch = https://github.com/woaishixiaoxiao/curator/tree/fixbug/LeaderLatch-double-master and head branch = your branch :)

Then the pull request will be opened against https://github.com/woaishixiaoxiao/curator and that should be fine. We use this pull request. Or it's also OK since the author is inactive, you take over the whole lifecycle.

tisonkun avatar Oct 10 '22 13:10 tisonkun

It was a bit trickier but I managed to base the PR based on his branch (his branch didn't show up when I was doing it the way I usually create PRs ¯_(ツ)_/¯). https://github.com/woaishixiaoxiao/curator/pull/1. Although, I'm not sure whether that's of any help because the original author would have to merge it into his branch, wouldn't he?

XComp avatar Oct 10 '22 13:10 XComp

@XComp OK. After looking into the patch I think #436 is better to proceed (you made a significant diff :)). I'll try to take a look this wek.

tisonkun avatar Oct 10 '22 13:10 tisonkun

Yes, but the biggest part of the diff is the last commit (adaee91): I'm ok with reverting that one if you think it's too much. The changes related to the comments of my review of this PR are included in the commits excluding adaee91 in #436 .

XComp avatar Oct 10 '22 13:10 XComp