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 <step...@data-artisans.com>
wrote:

> Hi!
>
> Good discussion!
>
> Seems the right long-term fix is the JM / TM reconciliation without
> failure, as Till pointed out.
>
> Another possibility could be to have a small timeout (say by default 5s or
> so) in which the Leader Service waits for either a re-connection or a new
> leader election before notifying the current leader.
>
> What do you think?
>
> Stephan
>
>
>
> On Wed, Sep 27, 2017 at 11:17 AM, Till Rohrmann <trohrm...@apache.org>
> wrote:
>
>> 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 quite sure whether YARN can give you strict guarantees
>> that at any moment there is at most one AM running. I suspect that this is
>> not the case and, thus, you could risk to run into the split-brain problem
>> there as well.
>>
>> I think a proper solution for this problem could be the recovery of
>> running jobs [3]. With that the TMs could continue executing the jobs even
>> if there is no leader anymore. The new leader (which could be the same JM),
>> would then recover the jobs from the TMs without having to restart them.
>> This feature, however, still needs some more work to be finalized.
>>
>> [1] https://issues.apache.org/jira/browse/FLINK-6174
>> [2] https://github.com/apache/flink/pull/3599
>> [3] https://issues.apache.org/jira/browse/FLINK-5703
>>
>> Cheers,
>> Till
>>
>> On Wed, Sep 27, 2017 at 10:58 AM, Gyula Fóra <gyula.f...@gmail.com>
>> wrote:
>>
>>> 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 the same time in larger
>>> deployments.
>>>
>>> Maybe this is easier to avoid in certain setups for instance in YARN
>>> where
>>> we only run a single JM anyways at any given time.
>>>
>>> Gyula
>>>
>>> Till Rohrmann <t...@data-artisans.com> ezt írta (időpont: 2017. szept.
>>> 27.,
>>> Sze, 10:49):
>>>
>>> > 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 already
>>> > happen, namely during the gap between of the server and client
>>> noticing the
>>> > lost connection, this gap should be practically non-existent. If we
>>> change
>>> > the behaviour, then this gap could potentially grow quite large
>>> leading to
>>> > all kinds of undesired side effects. E.g. if the sink operation is not
>>> > idempotent, then one might easily end up with thwarting ones exactly
>>> once
>>> > processing guarantees.
>>> >
>>> > I'm not sure whether we want to sacrifice the guarantee of not having
>>> to
>>> > deal with a split brain scenario but I can see the benefits of not
>>> > immediately revoking the leadership if one can guarantee that there
>>> will
>>> > never be two JMs competing for the leadership. However, in the general
>>> > case, this should be hard to do.
>>> >
>>> > Cheers,
>>> > Till
>>> >
>>> > On Wed, Sep 27, 2017 at 9:22 AM, Gyula Fóra <gyula.f...@gmail.com>
>>> wrote:
>>> >
>>> >> 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 this might lead to split brain scenarios if the previous leader
>>> loses
>>> >> connection to ZK while still maintaining connection to the TMs.
>>> >>
>>> >> Gyula
>>> >>
>>> >> Gyula Fóra <gyula.f...@gmail.com> ezt írta (időpont: 2017. szept.
>>> 26.,
>>> >> K, 18:34):
>>> >>
>>> >>> 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
>>> >>> https://gist.github.com/gyfora/acb55e380d932ac10593fc1fd37930ab):
>>> >>>
>>> >>> 1. Connection is suspended, notLeader method is called  -> revokes
>>> >>> leadership without checking anything, kills jobs
>>> >>> 2. Reconnects , isLeader and confirmLeaderSessionID methods are
>>> called
>>> >>> (before nodeChanged) -> Overwrites old confirmed session id in ZK
>>> with the
>>> >>> new one before checking (making recovery impossible in nodeChanged)
>>> >>>
>>> >>> I am probably not completely aware of the subtleties of this problem
>>> but
>>> >>> it seems to me that we should not immediately revoke leadership and
>>> fail
>>> >>> jobs on suspended, and also it would be nice if nodeChanged would be
>>> called
>>> >>> before confirmLeaderSessionID.
>>> >>>
>>> >>> Could someone with more experience please take a look as well?
>>> >>>
>>> >>> Thanks!
>>> >>> Gyula
>>> >>>
>>> >>> Gyula Fóra <gyula.f...@gmail.com> ezt írta (időpont: 2017. szept.
>>> 25.,
>>> >>> H, 16:43):
>>> >>>
>>> >>>> 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 <gyula.f...@gmail.com> ezt írta (időpont: 2017. szept.
>>> 25.,
>>> >>>> H, 15:39):
>>> >>>>
>>> >>>>> 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 <se...@apache.org> 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 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
>>> >>>>>> reconnect
>>> >>>>>> to another quorum peer if one goes down?
>>> >>>>>>
>>> >>>>>> On Mon, Sep 25, 2017 at 2:10 PM, Till Rohrmann <
>>> trohrm...@apache.org>
>>> >>>>>> wrote:
>>> >>>>>>
>>> >>>>>> > 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
>>> >>>>>> sure that
>>> >>>>>> > you are the leader.
>>> >>>>>> >
>>> >>>>>> > Consequently, if you do a rolling update of your ZooKeeper
>>> cluster
>>> >>>>>> which
>>> >>>>>> > causes client connections to be lost or suspended, then it will
>>> >>>>>> trigger a
>>> >>>>>> > restart of the Flink job upon reacquiring the leadership again.
>>> >>>>>> >
>>> >>>>>> > Cheers,
>>> >>>>>> > Till
>>> >>>>>> >
>>> >>>>>> > On Fri, Sep 22, 2017 at 6:41 PM, Gyula Fóra <
>>> gyula.f...@gmail.com>
>>> >>>>>> wrote:
>>> >>>>>> >
>>> >>>>>> > > We are using 1.3.2
>>> >>>>>> > >
>>> >>>>>> > > Gyula
>>> >>>>>> > >
>>> >>>>>> > > On Fri, Sep 22, 2017, 17:13 Ted Yu <yuzhih...@gmail.com>
>>> 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 <
>>> >>>>>> gyula.f...@gmail.com>
>>> >>>>>> > > wrote:
>>> >>>>>> > > >
>>> >>>>>> > > > > 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
>>> >>>>>> > > > > sessionid 0x15cba6e1a239774, likely server has closed
>>> socket,
>>> >>>>>> closing
>>> >>>>>> > > > > socket connection and attempting reconnect
>>> >>>>>> > > > > 2017-09-22 12:54:41,527 INFO
>>> >>>>>> > > > > org.apache.flink.shaded.org.apache.curator.framework.
>>> >>>>>> > > > > state.ConnectionStateManager
>>> >>>>>> > > > >  - State change: SUSPENDED
>>> >>>>>> > > > > 2017-09-22 12:54:41,528 WARN
>>> >>>>>> > > > > org.apache.flink.runtime.leaderelection.
>>> >>>>>> > ZooKeeperLeaderElectionService
>>> >>>>>> > > > >  - Connection to ZooKeeper suspended. The contender
>>> >>>>>> > > > > akka.tcp://
>>> >>>>>> fl...@splat.sto.midasplayer.com:42118/user/jobmanager no
>>> >>>>>> > > > > longer participates in the leader election.
>>> >>>>>> > > > > 2017-09-22 12:54:41,528 WARN
>>> >>>>>> > > > > org.apache.flink.runtime.leaderretrieval.
>>> >>>>>> > > ZooKeeperLeaderRetrievalService
>>> >>>>>> > > > >  - Connection to ZooKeeper suspended. Can no longer
>>> retrieve
>>> >>>>>> the
>>> >>>>>> > > > > leader from ZooKeeper.
>>> >>>>>> > > > > 2017-09-22 12:54:41,528 WARN
>>> >>>>>> > > > > org.apache.flink.runtime.leaderretrieval.
>>> >>>>>> > > ZooKeeperLeaderRetrievalService
>>> >>>>>> > > > >  - Connection to ZooKeeper suspended. Can no longer
>>> retrieve
>>> >>>>>> the
>>> >>>>>> > > > > leader from ZooKeeper.
>>> >>>>>> > > > > 2017-09-22 12:54:41,530 WARN
>>> >>>>>> > > > >
>>> >>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGra
>>> phStore
>>> >>>>>> > -
>>> >>>>>> > > > > ZooKeeper connection SUSPENDED. Changes to the submitted
>>> job
>>> >>>>>> graphs
>>> >>>>>> > > > > are not monitored (temporarily).
>>> >>>>>> > > > > 2017-09-22 12:54:41,530 INFO
>>> >>>>>> org.apache.flink.yarn.YarnJobManager
>>> >>>>>> > > > >                      - JobManager
>>> >>>>>> > > > > akka://flink/user/jobmanager#-317276879 was revoked
>>> >>>>>> leadership.
>>> >>>>>> > > > > 2017-09-22 12:54:41,532 INFO
>>> >>>>>> > > > > org.apache.flink.runtime.executiongraph.ExecutionGraph
>>> >>>>>> - Job
>>> >>>>>> > > > > event.game.log (2ad7bbcc476bbe3735954fc414ffcb97)
>>> switched
>>> >>>>>> from
>>> >>>>>> > state
>>> >>>>>> > > > > RUNNING to SUSPENDED.
>>> >>>>>> > > > > java.lang.Exception: JobManager is no longer the leader.
>>> >>>>>> > > > >
>>> >>>>>> > > > >
>>> >>>>>> > > > > Is this the expected behaviour?
>>> >>>>>> > > > >
>>> >>>>>> > > > > Thanks,
>>> >>>>>> > > > > Gyula
>>> >>>>>> > > > >
>>> >>>>>> > > >
>>> >>>>>> > >
>>> >>>>>> >
>>> >>>>>>
>>> >>>>>
>>> >
>>> >
>>> > --
>>> > Data Artisans GmbH | Stresemannstrasse 121a | 10963 Berlin
>>> <https://maps.google.com/?q=Stresemannstrasse+121a+%7C+10963+Berlin&entry=gmail&source=g>
>>> > <https://maps.google.com/?q=Stresemannstrasse+121a+%7C+10963
>>> +Berlin&entry=gmail&source=g>
>>> >
>>> > i...@data-artisans.com
>>> > phone +493055599146
>>> > mobile +491715521046
>>> >
>>> > Registered at Amtsgericht Charlottenburg - HRB 158244 B
>>> > Managing Directors: Kostas Tzoumas, Stephan Ewen
>>> >
>>>
>>
>>
>

Reply via email to