[ 
https://issues.apache.org/jira/browse/FLINK-10052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16889893#comment-16889893
 ] 

TisonKun commented on FLINK-10052:
----------------------------------

[~quan]

Supposed you have jm-1 and jm-2 execute the same job but run on different 
cluster. jm-1 is the previous leader and thus jm-2 is inactive. Now jm-1 lost 
connection with zookeeper and jm-2 became the leader, recovering from job graph 
store and checkpoint store, started executing the job. If we didn't shutdown 
jm-1 and supposed it would be shutdown due to zk communication failure, not 
only there were two jms running for the same job thus the same job was executed 
twice, but also they might conflict writing states.

To sum up, if a jm lost connection with zk, there is possibly another jm 
running for the same job. We don't want to see 2 jms running for the same job 
at the same time, right?

If you're worry about the performance, when you executing a streaming job, 
flink can recover it from checkpoints and catch up quickly.

> Tolerate temporarily suspended ZooKeeper connections
> ----------------------------------------------------
>
>                 Key: FLINK-10052
>                 URL: https://issues.apache.org/jira/browse/FLINK-10052
>             Project: Flink
>          Issue Type: Improvement
>          Components: Runtime / Coordination
>    Affects Versions: 1.4.2, 1.5.2, 1.6.0, 1.8.1
>            Reporter: Till Rohrmann
>            Assignee: Dominik WosiƄski
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> This issue results from FLINK-10011 which uncovered a problem with Flink's HA 
> recovery and proposed the following solution to harden Flink:
> The {{ZooKeeperLeaderElectionService}} uses the {{LeaderLatch}} Curator 
> recipe for leader election. The leader latch revokes leadership in case of a 
> suspended ZooKeeper connection. This can be premature in case that the system 
> can reconnect to ZooKeeper before its session expires. The effect of the lost 
> leadership is that all jobs will be canceled and directly restarted after 
> regaining the leadership.
> Instead of directly revoking the leadership upon a SUSPENDED ZooKeeper 
> connection, it would be better to wait until the ZooKeeper connection is 
> LOST. That way we would allow the system to reconnect and not lose the 
> leadership. This could be achievable by using Curator's {{LeaderSelector}} 
> instead of the {{LeaderLatch}}.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to