You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@curator.apache.org by Moshiko Kasirer <mo...@liveperson.com> on 2017/03/22 12:30:48 UTC

Re: question about curator - retry policy

Hi Jordan,

i have a question.

when working with Curator leader election

lets say i configure session time out of 30000 and node A is the leader

i then disconnect A's network when will noLeader method be called? and when
do nodes B

and C be notified about that?

i tested few times and i was not able to determined that....

i mean the client session of the leader tries to reconnect the server but
can't so when will

the method be invoked? when we configured 30 sec it was around 15 seconds..

as for the other nodes notification i guess it will happen only after
session timeout of 30

seconds is that correct?

we saw some kind of correlation when we shorten the timeout to lest say 10
sec the no

leader method was called in few seconds so what is the correlation between
session time

out and no leader method?

thanks a lot !


On Sat, May 21, 2016 at 5:45 PM, Moshiko Kasirer <mo...@liveperson.com>
wrote:

> Thanks a lot!
> בתאריך 21 במאי 2016 17:35,‏ "Jordan Zimmerman" <jo...@jordanzimmerman.com>
> כתב:
>
> Register a ServiceCacheListener and whenever cacheChanged() is called,
>> write the current state to disk. Writing your own cache is not trivial.
>>
>> -Jordan
>>
>> On May 21, 2016, at 3:12 AM, Moshiko Kasirer <mo...@liveperson.com>
>> wrote:
>>
>> We know. But ours is on a file so the next time the app is started and
>> can't connect to zk it has a cluster view taken from that file... Your
>> cache is in memory cache afaik
>> בתאריך 21 במאי 2016 05:58,‏ "Jordan Zimmerman" <
>> jordan@jordanzimmerman.com> כתב:
>>
>>> You don’t need to maintain your own cache. Service Discovery already
>>> handles that.
>>>
>>> -Jordan
>>>
>>> On May 20, 2016, at 5:36 PM, Moshiko Kasirer <mo...@liveperson.com>
>>> wrote:
>>>
>>> We are using nginx as our web tier which delegate requests to app nodes
>>> using consistent hashing to one of the registered app nodes. Since we have
>>> many web and app nodes we have to make sure all available app nodes are
>>> known to the web tier and that in any given time they all see the same app
>>> nodes picture. So we built an app on top of your service discovery that
>>> when app node ris up he register and web tier is listening to that cluster
>>> and changes his available app nodes view.In adoption we handle situations
>>> when there is on connection to zk using a cache file with latest available
>>> view until the connection is restored. For some reason sometimes although
>>> zk is up and running the curator connection to which we listen to know if
>>> we should reregister isn't invoked meaning stays as LOST...
>>> בתאריך 21 במאי 2016 01:23,‏ "Jordan Zimmerman" <
>>> jordan@jordanzimmerman.com> כתב:
>>>
>>>> Retry policy is only used for individual operations. Any client-server
>>>> system needs to have retries to avoid temporary network events. The entire
>>>> curator-client and curator-framework modules are written to handle
>>>> ZooKeeper client connection maintenance. So, there isn’t one thing I can
>>>> point to.
>>>>
>>>> Internally, the ServiceDiscovery code uses a PathChildrenCache
>>>> instance. If all you are using is Service Discovery there is almost no need
>>>> for you to monitor the connection state. What are you trying to accomplish?
>>>>
>>>> -Jordan
>>>>
>>>> On May 20, 2016, at 5:19 PM, Moshiko Kasirer <mo...@liveperson.com>
>>>> wrote:
>>>>
>>>> The thing is we have many negative tests in which we stop and start the
>>>> zk quorum the issue I raised only happens from time to time.... So it's hat
>>>> I hard to reproduce. But you just wrote that when the quorom is up the
>>>> connection should be reconnected ... how? who does that? ZkClient  or
>>>> curator? That is not related to retry policy?
>>>> בתאריך 21 במאי 2016 01:12,‏ "Jordan Zimmerman" <
>>>> jordan@jordanzimmerman.com> כתב:
>>>>
>>>>> If the ZK cluster’s quorum is restored, then the connection state
>>>>> should change to RECONNECTED. There are copious tests in Curator itself
>>>>> that show this. If you’re seeing that Curator does not restore a broken
>>>>> connection then there is a deeper bug. Can you create a test that shows the
>>>>> problem?
>>>>>
>>>>> -Jordan
>>>>>
>>>>> On May 20, 2016, at 5:07 PM, Moshiko Kasirer <mo...@liveperson.com>
>>>>> wrote:
>>>>>
>>>>> I mean that while zk cluster is up the curator connection state stays
>>>>> LOST
>>>>> Which in our case means the app node in which it happens doesnt
>>>>> register himself as avalable.... I just don't seem to understand when does
>>>>> curator gives up on trying to connect zk and when he doesn't give up.
>>>>> Thanks for the help !
>>>>> בתאריך 21 במאי 2016 00:58,‏ "Jordan Zimmerman" <
>>>>> jordan@jordanzimmerman.com> כתב:
>>>>>
>>>>>> You must have a retry policy so that you don’t overwhelm your network
>>>>>> and ZooKeeper cluster. The example code shows how to create a reasonable
>>>>>> one.
>>>>>>
>>>>>> sometimes although zk cluster is up the curator service discovery
>>>>>> connection isn't
>>>>>>
>>>>>> Service Discovery’s internal instances might be waiting based on the
>>>>>> retry policy. But, what do you mean by "curator service discovery
>>>>>> connection isn’t”? There isn’t such a thing as a service discovery
>>>>>> connection.
>>>>>>
>>>>>> -Jordan
>>>>>>
>>>>>> On May 20, 2016, at 4:53 PM, Moshiko Kasirer <mo...@liveperson.com>
>>>>>> wrote:
>>>>>>
>>>>>> We are using your service discovery. So you are saying I should not
>>>>>> care about the retry policy...? So the only thing left to explain is how
>>>>>> come sometimes although zk cluster is up the curator service discovery
>>>>>> connection isn't.....
>>>>>> בתאריך 21 במאי 2016 00:43,‏ "Jordan Zimmerman" <
>>>>>> jordan@jordanzimmerman.com> כתב:
>>>>>>
>>>>>> If you are using Curator’s Service Discovery code, it will be
>>>>>> continuously re-trying the connections. This is not because of the retry
>>>>>> policy it’s because the Service Discovery code manages connection
>>>>>> interruptions internally.
>>>>>>
>>>>>> -Jordan
>>>>>>
>>>>>> On May 20, 2016, at 4:40 PM, Moshiko Kasirer <mo...@liveperson.com>
>>>>>> wrote:
>>>>>>
>>>>>> Thanks for the replay I will send those logs ASAP.
>>>>>> It's difficult to understand the connection mechanism of zk ....
>>>>>> We are using curator 2.10 as our service discovery so we have to make
>>>>>> sure that when zk is alive we connect and inform the our server is up we do
>>>>>> that by listening to curator connection listener which I think has also to
>>>>>> do with the retry policy.... But what I can't understand is why sometimes
>>>>>> we can see in the log that curator gave up (Lost) yet still a second later
>>>>>> curator connection is restored how? Is it because zk session heartbeat
>>>>>> restored the connection? Does that Iovine curator to change his connection
>>>>>> state? And on the other side we sometimes get to a point were zk is up but
>>>>>> curator connection stays as Lost...
>>>>>> That is why I thought of using the new always try policy you entered
>>>>>> do you think it can help? That why  hope there will be no way that zk is up
>>>>>> but curator status is lost.....as once he will retry he will reconnect to
>>>>>> zk.... Is that correct?
>>>>>> בתאריך 21 במאי 2016 00:10,‏ "Jordan Zimmerman" <
>>>>>> jordan@jordanzimmerman.com> כתב:
>>>>>>
>>>>>>> Curator’s retry policies are used within each CuratorFramework
>>>>>>> operation. For example, when you call client.setData().forPath(p, b) the
>>>>>>> retry policy will be invoked if there is a retry-able exception during the
>>>>>>> operation. In addition to the retryPolicy, there are connection timeouts.
>>>>>>> The behavior of how this is handled changed between Curator 2.x and Curator
>>>>>>> 3.x. In Curator 2.x, for every iteration of the retry, the operation will
>>>>>>> wait until connection timeout when there’s no connection. In Curator 3.x,
>>>>>>> the connection timeout wait only occurs once (if the default
>>>>>>> ConnectionHandlingPolicy is used).
>>>>>>>
>>>>>>> In any event, ZooKeeper itself tries to maintain the connection.
>>>>>>> Also, Curator will re-create the internally managed connection depending
>>>>>>> various network interruptions, etc. I’d need to see the logs to give you
>>>>>>> more input.
>>>>>>>
>>>>>>> -Jordan
>>>>>>>
>>>>>>> On May 19, 2016, at 10:12 AM, Moshiko Kasirer <mo...@liveperson.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> first i would like to thank you about curator we are using it as
>>>>>>> part of our service discovery
>>>>>>>
>>>>>>> solution and it helps a lot!!
>>>>>>>
>>>>>>> i have a question i hope you will be able to help me with.
>>>>>>>
>>>>>>> its regarding the curator retry policy it seems to me we dont really
>>>>>>> understand when this policy is
>>>>>>>
>>>>>>> invoked,  as i see in our logs that although i configured it as max
>>>>>>> retry 1 actually in the logs i see
>>>>>>>
>>>>>>> many ZK re connection attempts (and many curator gave up messages
>>>>>>> but later i see
>>>>>>>
>>>>>>> reconnected status...) . is it possible that that policy is only
>>>>>>> relevant to manually invoked
>>>>>>>
>>>>>>> operations against the ZK cluster done via curator ? and that the re
>>>>>>> connections i see in the logs
>>>>>>>
>>>>>>> are caused by the fact that the ZK was available during start up so
>>>>>>> sessions were created and
>>>>>>>
>>>>>>> then when ZK was down the ZK clients *(not curator) * are sending
>>>>>>> heartbeats as part of the ZK
>>>>>>>
>>>>>>> architecture? that is the part i am failing to understand and i hope
>>>>>>> you can help me with that.
>>>>>>>
>>>>>>> you have recently added RetreyAllways policy and i wanted to know if
>>>>>>> it is save to use it?
>>>>>>>
>>>>>>> the thing is we always want to retry to reconnect to ZK when he is
>>>>>>> available but that is something
>>>>>>>
>>>>>>> the ZK client does as long as he has open sessions right?  i am not
>>>>>>> sure that it has to do with the
>>>>>>>
>>>>>>> retry policy ...
>>>>>>>
>>>>>>> thanks,
>>>>>>>
>>>>>>> moshiko
>>>>>>>
>>>>>>> --
>>>>>>> Moshiko Kasirer
>>>>>>> Software Engineer
>>>>>>> T: +972-74-700-4357
>>>>>>> <http://www.linkedin.com/company/164748>
>>>>>>> <http://twitter.com/liveperson>
>>>>>>> <http://www.facebook.com/LivePersonInc> We Create Meaningful
>>>>>>> Connections
>>>>>>> <http://roia.biz/im/n/ndiXvq1BAAGhL0MAABW7QgABwExmMQA-A/>
>>>>>>>
>>>>>>> This message may contain confidential and/or privileged information.
>>>>>>> If you are not the addressee or authorized to receive this on behalf
>>>>>>> of the addressee you must not use, copy, disclose or take action based on
>>>>>>> this message or any information herein.
>>>>>>> If you have received this message in error, please advise the sender
>>>>>>> immediately by reply email and delete this message. Thank you.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>> This message may contain confidential and/or privileged information.
>>>>>> If you are not the addressee or authorized to receive this on behalf
>>>>>> of the addressee you must not use, copy, disclose or take action based on
>>>>>> this message or any information herein.
>>>>>> If you have received this message in error, please advise the sender
>>>>>> immediately by reply email and delete this message. Thank you.
>>>>>>
>>>>>>
>>>>>>
>>>>>> This message may contain confidential and/or privileged information.
>>>>>> If you are not the addressee or authorized to receive this on behalf
>>>>>> of the addressee you must not use, copy, disclose or take action based on
>>>>>> this message or any information herein.
>>>>>> If you have received this message in error, please advise the sender
>>>>>> immediately by reply email and delete this message. Thank you.
>>>>>>
>>>>>>
>>>>>>
>>>>> This message may contain confidential and/or privileged information.
>>>>> If you are not the addressee or authorized to receive this on behalf
>>>>> of the addressee you must not use, copy, disclose or take action based on
>>>>> this message or any information herein.
>>>>> If you have received this message in error, please advise the sender
>>>>> immediately by reply email and delete this message. Thank you.
>>>>>
>>>>>
>>>>>
>>>> This message may contain confidential and/or privileged information.
>>>> If you are not the addressee or authorized to receive this on behalf of
>>>> the addressee you must not use, copy, disclose or take action based on this
>>>> message or any information herein.
>>>> If you have received this message in error, please advise the sender
>>>> immediately by reply email and delete this message. Thank you.
>>>>
>>>>
>>>>
>>> This message may contain confidential and/or privileged information.
>>> If you are not the addressee or authorized to receive this on behalf of
>>> the addressee you must not use, copy, disclose or take action based on this
>>> message or any information herein.
>>> If you have received this message in error, please advise the sender
>>> immediately by reply email and delete this message. Thank you.
>>>
>>>
>>>
>> This message may contain confidential and/or privileged information.
>> If you are not the addressee or authorized to receive this on behalf of
>> the addressee you must not use, copy, disclose or take action based on this
>> message or any information herein.
>> If you have received this message in error, please advise the sender
>> immediately by reply email and delete this message. Thank you.
>>
>>
>>


-- 

Moshiko Kasirer
Software Engineer
T: +972-74-700-4357
<http://www.linkedin.com/company/164748> <http://twitter.com/liveperson>
<http://www.facebook.com/LivePersonInc> We Create Meaningful Connections
<https://liveperson.docsend.com/view/8iiswfp>

-- 
This message may contain confidential and/or privileged information. 
If you are not the addressee or authorized to receive this on behalf of the 
addressee you must not use, copy, disclose or take action based on this 
message or any information herein. 
If you have received this message in error, please advise the sender 
immediately by reply email and delete this message. Thank you.

Re: question about curator - retry policy

Posted by Moshiko Kasirer <mo...@liveperson.com>.
thanks a lot !

On Wed, Mar 22, 2017 at 3:35 PM, Jordan Zimmerman <
jordan@jordanzimmerman.com> wrote:

> lets say i configure session time out of 30000 and node A is the leader
> i then disconnect A's network when will noLeader method be called?
>
>
> It depends which version of Curator you're using. For Curator 2.x, it will
> occur when ZooKeeper disconnects from the server which happens when a
> heartbeat is missed. A ZK heartbeat is 2/3 of a session so it would happen
> at approx 20000 in Curator 2.x. For Curator 3.x, it is configurable when it
> happens - the default is to set a timer when the connection is lost and
> simulate a session loss after the session times-out. So, for Curator 3.x it
> would happen at approx 30000. The new behavior is describe for the LOST
> state here: http://curator.apache.org/errors.html
>
> when do nodes B
> and C be notified about that?
>
>
> That's normal ZK operations. As you can imagine it happens when the
> Ephemeral nodes are deleted by the server which happens when the session
> times out. If you need more information on how ZooKeeper deals with
> sessions I suggest you ask on the ZooKeeper lists.
>
> -Jordan
>



-- 

Moshiko Kasirer
Software Engineer
T: +972-74-700-4357
<http://www.linkedin.com/company/164748> <http://twitter.com/liveperson>
<http://www.facebook.com/LivePersonInc> We Create Meaningful Connections
<https://liveperson.docsend.com/view/8iiswfp>

-- 
This message may contain confidential and/or privileged information. 
If you are not the addressee or authorized to receive this on behalf of the 
addressee you must not use, copy, disclose or take action based on this 
message or any information herein. 
If you have received this message in error, please advise the sender 
immediately by reply email and delete this message. Thank you.

Re: question about curator - retry policy

Posted by Jordan Zimmerman <jo...@jordanzimmerman.com>.
> lets say i configure session time out of 30000 and node A is the leader
> i then disconnect A's network when will noLeader method be called?

It depends which version of Curator you're using. For Curator 2.x, it will occur when ZooKeeper disconnects from the server which happens when a heartbeat is missed. A ZK heartbeat is 2/3 of a session so it would happen at approx 20000 in Curator 2.x. For Curator 3.x, it is configurable when it happens - the default is to set a timer when the connection is lost and simulate a session loss after the session times-out. So, for Curator 3.x it would happen at approx 30000. The new behavior is describe for the LOST state here: http://curator.apache.org/errors.html <http://curator.apache.org/errors.html>

> when do nodes B 
> and C be notified about that?  


That's normal ZK operations. As you can imagine it happens when the Ephemeral nodes are deleted by the server which happens when the session times out. If you need more information on how ZooKeeper deals with sessions I suggest you ask on the ZooKeeper lists.

-Jordan