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

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

[~lamber-ken] PR#9066 solve the problem and in FLINK-10333 we actually propose 
a flink scope leader latch.

But back to this jira, I'd like to clarify two things.

1. I know with shaded curator's auto detection fails. But for our case, flink 
use zookeeper 3.4 and there is no ongoing proposal upgrading zookeeper version. 
Given this background, the detection fails, and curator use zk34comp mode, 
which is, hacky but correctly, fit our case.(For a clean solution, we could 
1).figure out a new relocation strategy or 2) CURATOR-532 proposes a manually 
compatibility config or 3) a flink scope leader latch which would be somewhat 
like PR#9066)

2. LeaderSelector just forces the user to implement {{#stateChanged}} and take 
leadership greedily. I don't think it provides extra magic and for state 
handling we want no more than {{SessionConnectionStateErrorPolicy}}.

> 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
>
> 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