Re: LeaderLatch recipe could run into two leaders

2019-08-14 Thread Zili Chen
Yes, on #isLeader() called synchronously we can save the latchPath
in outside contender field and when the contender want to perform
an write operation it also check whether the node with that latchPath
exist. It should solve our problem IMHO.

Best,
tison.


Jordan Zimmerman  于2019年8月15日周四 下午12:03写道:

> I did a quick read, do I understand that adding a method to return the
> LeaderLatch's latchPath will solve the issue for you?
>
> -Jordan
>
> On Aug 14, 2019, at 5:49 PM, Zili Chen  wrote:
>
> Re-active this thread follow the discussion in ZooKeeper list
> and try to confirm my understanding on this leader election topic.
>
> The situation we run into trouble is described as above, and
> the solution we found is documented here[1]. See also point 1 in
> "Motivation" section and the first paragraph of "Proposal Design"
> section.
>
> Best,
> tison.
>
> [1]
> https://docs.google.com/document/d/1cBY1t0k5g1xNqzyfZby3LcPu4t-wpx57G1xf-nmWrCo/edit?usp=sharing
>
>
> Zili Chen  于2019年3月28日周四 上午8:51写道:
>
>> Hi Cameron & Jordan,
>>
>> Thanks for your replies! I think our case is similar with that posted in
>> the Tech Note.
>>
>> Briefly in our situation,
>>
>> 1. contender-1 was elected as the leader and serving.
>> 2-1. contender-1 lost connect with zookeeper and thus the ephemeral node
>> deleted.
>> Contender-1#notLeader send a message to contender-1 to tell that itself
>> was no longer the leader, but before this message processed, messages
>> cause
>> actions which only the leader can perform processed.
>> 2-2. Concurrently, contender-2 was elected as the new leader and serving
>> as the leader.
>> 3. Before contender-1#notLeader's message got properly processed, both of
>> the contenders
>> regarded themselves as the leader.
>>
>> This is not a rare case in our situation since we cannot guarantee how
>> many messages
>> before NotLeader would be processed and quite often we cannot revoke
>> leadership timely...
>>
>> As an optional resolution, we'd like to perform "leader-only actions" in
>> a transaction
>> checking the existence of election node path(ourPath), which would be like
>>
>> curatorFramework.inTransaction()
>> .check().forPath(election-node-path).and()
>> .setData().forPath(...).and().commit();
>>
>> to overcome this edge case.
>>
>> Best,
>> tison.
>>
>>
>> Jordan Zimmerman  于2019年3月28日周四 上午8:29写道:
>>
>>> What Cameron writes is correct. The only other thing to be concerned
>>> with is large GC times and small session times. We have a Tech Note on
>>> this: https://cwiki.apache.org/confluence/display/CURATOR/TN10
>>>
>>> -Jordan
>>>
>>> On Mar 27, 2019, at 7:15 PM, Cameron McKenzie 
>>> wrote:
>>>
>>> hey ZiLi,
>>> In point 2, the deletion of the path zNode will not occur under normal
>>> conditions, only if some other application removed the zNode. The contract
>>> specified by Curator is that this zNode is under the control of Curator and
>>> the behaviour of the recipe can't be guaranteed if this contract is broken.
>>>
>>> Having said that, it is quite possible at point 2 for contender-1 to
>>> just lose its connection to ZK. As soon as this occurs contender 1 would
>>> get a SUSPENDED event and would revoke leadership locally.
>>> Contender 2 will not gain leadership until the leadership zNode of
>>> contender 1 is removed (this will happen once contender 1's session times
>>> out)
>>> One contender-1 reconnects to ZK, it will delete its existing zNode and
>>> then create a new zNode.
>>>
>>> So, from the perspective of Curator, there should not be a situation
>>> where there are multiple leaders. You need to ensure that your application
>>> responds to the Curator callbacks of leadership acquisition / revocation in
>>> a timely manner though, otherwise your application may be in a state where
>>> there are multiple leaders.
>>> cheers
>>>
>>>
>>>
>>> On Thu, Mar 28, 2019 at 10:43 AM Zili Chen  wrote:
>>>
 Hi Curators,

 Any advice, or it is just not the case?

 If the latter, I'd like to learn how to reason that.

 Best,
 tison.


 ZiLi Chen  于2019年3月24日周日 上午8:54写道:

> [1]
> http://zookeeper.apache.org/doc/r3.4.13/recipes.html#sc_leaderElection
>
> ZiLi Chen  于2019年3月24日周日 上午8:53写道:
>
>> Hi Curators,
>>
>> While using LeaderLatch recipe for leader election, I notice that we
>> use #isLeader and #notLeader as callback function when a contender
>> become/un-become a leader. The implementation of leader election 
>> following
>> the description of ZooKeeper recipes[1].
>>
>> However, since there is latency between the election node be
>> deleted(lost leadership) and the LeaderLatch got notified and the 
>> Listener
>> get notified. It is possible that
>>
>> 1. contender-1 gained leadership
>> 2. ourPath of contender-1 got deleted/lost, and contender-2 gained
>> leadership
>> 3. contender-1 was notified that it was no long the 

Re: LeaderLatch recipe could run into two leaders

2019-08-14 Thread Jordan Zimmerman
I did a quick read, do I understand that adding a method to return the 
LeaderLatch's latchPath will solve the issue for you?

-Jordan

> On Aug 14, 2019, at 5:49 PM, Zili Chen  wrote:
> 
> Re-active this thread follow the discussion in ZooKeeper list
> and try to confirm my understanding on this leader election topic.
> 
> The situation we run into trouble is described as above, and
> the solution we found is documented here[1]. See also point 1 in
> "Motivation" section and the first paragraph of "Proposal Design"
> section.
> 
> Best,
> tison.
> 
> [1] 
> https://docs.google.com/document/d/1cBY1t0k5g1xNqzyfZby3LcPu4t-wpx57G1xf-nmWrCo/edit?usp=sharing
>  
> 
> 
> Zili Chen mailto:wander4...@gmail.com>> 于2019年3月28日周四 
> 上午8:51写道:
> Hi Cameron & Jordan,
> 
> Thanks for your replies! I think our case is similar with that posted in the 
> Tech Note.
> 
> Briefly in our situation,
> 
> 1. contender-1 was elected as the leader and serving.
> 2-1. contender-1 lost connect with zookeeper and thus the ephemeral node 
> deleted.
> Contender-1#notLeader send a message to contender-1 to tell that itself
> was no longer the leader, but before this message processed, messages cause
> actions which only the leader can perform processed.
> 2-2. Concurrently, contender-2 was elected as the new leader and serving as 
> the leader.
> 3. Before contender-1#notLeader's message got properly processed, both of the 
> contenders
> regarded themselves as the leader.
> 
> This is not a rare case in our situation since we cannot guarantee how many 
> messages
> before NotLeader would be processed and quite often we cannot revoke 
> leadership timely...
> 
> As an optional resolution, we'd like to perform "leader-only actions" in a 
> transaction
> checking the existence of election node path(ourPath), which would be like
> 
> curatorFramework.inTransaction()
> .check().forPath(election-node-path).and()
> .setData().forPath(...).and().commit();
> 
> to overcome this edge case.
> 
> Best,
> tison.
> 
> 
> Jordan Zimmerman  > 于2019年3月28日周四 上午8:29写道:
> What Cameron writes is correct. The only other thing to be concerned with is 
> large GC times and small session times. We have a Tech Note on this: 
> https://cwiki.apache.org/confluence/display/CURATOR/TN10 
> 
> 
> -Jordan
> 
>> On Mar 27, 2019, at 7:15 PM, Cameron McKenzie > > wrote:
>> 
>> hey ZiLi,
>> In point 2, the deletion of the path zNode will not occur under normal 
>> conditions, only if some other application removed the zNode. The contract 
>> specified by Curator is that this zNode is under the control of Curator and 
>> the behaviour of the recipe can't be guaranteed if this contract is broken. 
>> 
>> Having said that, it is quite possible at point 2 for contender-1 to just 
>> lose its connection to ZK. As soon as this occurs contender 1 would get a 
>> SUSPENDED event and would revoke leadership locally. 
>> Contender 2 will not gain leadership until the leadership zNode of contender 
>> 1 is removed (this will happen once contender 1's session times out)
>> One contender-1 reconnects to ZK, it will delete its existing zNode and then 
>> create a new zNode.
>> 
>> So, from the perspective of Curator, there should not be a situation where 
>> there are multiple leaders. You need to ensure that your application 
>> responds to the Curator callbacks of leadership acquisition / revocation in 
>> a timely manner though, otherwise your application may be in a state where 
>> there are multiple leaders.
>> cheers
>> 
>> 
>> 
>> On Thu, Mar 28, 2019 at 10:43 AM Zili Chen > > wrote:
>> Hi Curators,
>> 
>> Any advice, or it is just not the case?
>> 
>> If the latter, I'd like to learn how to reason that.
>> 
>> Best,
>> tison.
>> 
>> 
>> ZiLi Chen mailto:wander4...@gmail.com>> 于2019年3月24日周日 
>> 上午8:54写道:
>> [1] http://zookeeper.apache.org/doc/r3.4.13/recipes.html#sc_leaderElection 
>> 
>> ZiLi Chen mailto:wander4...@gmail.com>> 于2019年3月24日周日 
>> 上午8:53写道:
>> Hi Curators,
>> 
>> While using LeaderLatch recipe for leader election, I notice that we use 
>> #isLeader and #notLeader as callback function when a contender 
>> become/un-become a leader. The implementation of leader election following 
>> the description of ZooKeeper recipes[1].
>> 
>> However, since there is latency between the election node be deleted(lost 
>> leadership) and the LeaderLatch got notified and the Listener get notified. 
>> It is possible that
>> 
>> 1. contender-1 gained leadership
>> 2. ourPath of contender-1 got deleted/lost, and contender-2 gained leadership
>> 3. contender-1 was notified that it was no long the leader.
>> 
>> among the interval 

Re: LeaderLatch recipe could run into two leaders

2019-08-14 Thread Zili Chen
Re-active this thread follow the discussion in ZooKeeper list
and try to confirm my understanding on this leader election topic.

The situation we run into trouble is described as above, and
the solution we found is documented here[1]. See also point 1 in
"Motivation" section and the first paragraph of "Proposal Design"
section.

Best,
tison.

[1]
https://docs.google.com/document/d/1cBY1t0k5g1xNqzyfZby3LcPu4t-wpx57G1xf-nmWrCo/edit?usp=sharing


Zili Chen  于2019年3月28日周四 上午8:51写道:

> Hi Cameron & Jordan,
>
> Thanks for your replies! I think our case is similar with that posted in
> the Tech Note.
>
> Briefly in our situation,
>
> 1. contender-1 was elected as the leader and serving.
> 2-1. contender-1 lost connect with zookeeper and thus the ephemeral node
> deleted.
> Contender-1#notLeader send a message to contender-1 to tell that itself
> was no longer the leader, but before this message processed, messages cause
> actions which only the leader can perform processed.
> 2-2. Concurrently, contender-2 was elected as the new leader and serving
> as the leader.
> 3. Before contender-1#notLeader's message got properly processed, both of
> the contenders
> regarded themselves as the leader.
>
> This is not a rare case in our situation since we cannot guarantee how
> many messages
> before NotLeader would be processed and quite often we cannot revoke
> leadership timely...
>
> As an optional resolution, we'd like to perform "leader-only actions" in a
> transaction
> checking the existence of election node path(ourPath), which would be like
>
> curatorFramework.inTransaction()
> .check().forPath(election-node-path).and()
> .setData().forPath(...).and().commit();
>
> to overcome this edge case.
>
> Best,
> tison.
>
>
> Jordan Zimmerman  于2019年3月28日周四 上午8:29写道:
>
>> What Cameron writes is correct. The only other thing to be concerned with
>> is large GC times and small session times. We have a Tech Note on this:
>> https://cwiki.apache.org/confluence/display/CURATOR/TN10
>>
>> -Jordan
>>
>> On Mar 27, 2019, at 7:15 PM, Cameron McKenzie 
>> wrote:
>>
>> hey ZiLi,
>> In point 2, the deletion of the path zNode will not occur under normal
>> conditions, only if some other application removed the zNode. The contract
>> specified by Curator is that this zNode is under the control of Curator and
>> the behaviour of the recipe can't be guaranteed if this contract is broken.
>>
>> Having said that, it is quite possible at point 2 for contender-1 to just
>> lose its connection to ZK. As soon as this occurs contender 1 would get a
>> SUSPENDED event and would revoke leadership locally.
>> Contender 2 will not gain leadership until the leadership zNode of
>> contender 1 is removed (this will happen once contender 1's session times
>> out)
>> One contender-1 reconnects to ZK, it will delete its existing zNode and
>> then create a new zNode.
>>
>> So, from the perspective of Curator, there should not be a situation
>> where there are multiple leaders. You need to ensure that your application
>> responds to the Curator callbacks of leadership acquisition / revocation in
>> a timely manner though, otherwise your application may be in a state where
>> there are multiple leaders.
>> cheers
>>
>>
>>
>> On Thu, Mar 28, 2019 at 10:43 AM Zili Chen  wrote:
>>
>>> Hi Curators,
>>>
>>> Any advice, or it is just not the case?
>>>
>>> If the latter, I'd like to learn how to reason that.
>>>
>>> Best,
>>> tison.
>>>
>>>
>>> ZiLi Chen  于2019年3月24日周日 上午8:54写道:
>>>
 [1]
 http://zookeeper.apache.org/doc/r3.4.13/recipes.html#sc_leaderElection

 ZiLi Chen  于2019年3月24日周日 上午8:53写道:

> Hi Curators,
>
> While using LeaderLatch recipe for leader election, I notice that we
> use #isLeader and #notLeader as callback function when a contender
> become/un-become a leader. The implementation of leader election following
> the description of ZooKeeper recipes[1].
>
> However, since there is latency between the election node be
> deleted(lost leadership) and the LeaderLatch got notified and the Listener
> get notified. It is possible that
>
> 1. contender-1 gained leadership
> 2. ourPath of contender-1 got deleted/lost, and contender-2 gained
> leadership
> 3. contender-1 was notified that it was no long the leader.
>
> among the interval between 2 and 3, both contender-1 and contender-2
> would think that it itself was the leader and performed actions. Is it a
> known issue that we should tolerant or there is any approach we can ensure
> that actions(e.g., writing on ZooKeeper) can only be committed if the
> proposing contender is the only current leader?
>
> Best,
> tison.
>

>>