Re: Zookeeper failure handling

2017-09-29 Thread Gyula Fóra
Hi, I think Stephan's idea is good (not sure how small timeout is enough) if it's easy to add then we should definitely have this as an optional setting :) Otherwise if it's too big of an effort we could just stick with the plans for the proper solution as this is not super critical. Cheers,

Re: Zookeeper failure handling

2017-09-29 Thread Till Rohrmann
Yes this sounds like a good compromise for the moment. We could offer it as a special HighAvailabilityServices implementation with loosened split-brain safety guarantees but hardened connection suspension tolerance. Cheers, Till On Thu, Sep 28, 2017 at 8:00 PM, Stephan Ewen

Re: Zookeeper failure handling

2017-09-27 Thread Gyula Fóra
Thanks Till for pointing these issues out, you are right this is exactly the same problem! Gyula Till Rohrmann ezt írta (időpont: 2017. szept. 27., Sze, 11:18): > I agree that this is not very nice and can put a lot of stress on your > cluster. > > There is actually an

Re: Zookeeper failure handling

2017-09-27 Thread Till Rohrmann
I agree that this is not very nice and can put a lot of stress on your cluster. There is actually an open issue for exactly this [1] and also a PR [2]. The problem is that in the general case it will allow for split-brain situations and therefore it has not been merged yet. I'm actually not

Re: Zookeeper failure handling

2017-09-27 Thread Gyula Fóra
Hi Till, Thanks for the explanation, yes this sounds like a hard problem but it just seems wrong that whenever the ZK leader is restarted all the Flink jobs fail on a cluster. This might be within the overall guarantees of the system but can lead to some cascading failures if every job recovers at

Re: Zookeeper failure handling

2017-09-27 Thread Till Rohrmann
Hi Gyula, if we don't listen to the LeaderLatch#notLeader call but instead wait until we see (via the NodeCache) a new leader information being written to the leader path in order to revoke leadership, then we potentially end up running the same job twice. Even though this can theoretically

Re: Zookeeper failure handling

2017-09-27 Thread Gyula Fóra
On a second iteration, the whole problem seems to stem from the fact that we revoke leadership from the JM when the notLeader method is called before waiting for a new leader to be elected. Ideally we should wait until isLeader is called again to check who was the previous leader but I can see how

Re: Zookeeper failure handling

2017-09-26 Thread Gyula Fóra
Hi, I did some experimenting and found something that is interesting and looks off. So the only problem is when the ZK leader is restarted, not related to any retry/reconnect logic (not affected by the timeout setting). I think the following is happening (based on the logs

Re: Zookeeper failure handling

2017-09-25 Thread Gyula Fóra
Curator seems to auto reconnect anyways, the problem might be that there is a new leader elected before the old JM could reconnect. We will try to experiment with this tomorrow to see if increasing the timeouts do any good. Gyula Gyula Fóra ezt írta (időpont: 2017. szept.

Re: Zookeeper failure handling

2017-09-25 Thread Gyula Fóra
I will try to check what Stephan suggested and get back to you! Thanks for the feedback Gyula On Mon, Sep 25, 2017, 15:33 Stephan Ewen wrote: > I think the question is whether the connection should be lost in the case > of a rolling ZK update. > > There should always be a

Re: Zookeeper failure handling

2017-09-25 Thread Stephan Ewen
I think the question is whether the connection should be lost in the case of a rolling ZK update. There should always be a quorum online, so Curator should always be able to connect. So there is no need to revoke leadership. @gyula - can you check whether there is an option in Curator to

Re: Zookeeper failure handling

2017-09-25 Thread Till Rohrmann
Hi Gyula, Flink uses internally the Curator LeaderLatch recipe to do leader election. The LeaderLatch will revoke the leadership of a contender in case of a SUSPENDED or LOST connection to the ZooKeeper quorum. The assumption here is that if you cannot talk to ZooKeeper, then we can no longer be

Re: Zookeeper failure handling

2017-09-22 Thread Gyula Fóra
We are using 1.3.2 Gyula On Fri, Sep 22, 2017, 17:13 Ted Yu wrote: > Which release are you using ? > > Flink 1.3.2 uses Curator 2.12.0 which solves some leader election issues. > > Mind giving 1.3.2 a try ? > > On Fri, Sep 22, 2017 at 4:54 AM, Gyula Fóra

Zookeeper failure handling

2017-09-22 Thread Gyula Fóra
Hi all, We have observed that in case some nodes of the ZK cluster are restarted (for a rolling restart) the Flink Streaming jobs fail (and restart). Log excerpt: 2017-09-22 12:54:41,426 INFO org.apache.zookeeper.ClientCnxn - Unable to read additional data from server