[
https://issues.apache.org/jira/browse/FLINK-10052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16886699#comment-16886699
]
lamber-ken edited comment on FLINK-10052 at 7/17/19 6:01 AM:
-------------------------------------------------------------
[~Tison] (y), I have some points need to talk with you.
First, for your first point, I thought it yesterday and wont create a new
curator Jira like your CURATOR-532 that user can manually config ZooKeeper3.4.x
Compatibility, but I give up that idea, because I found that it also needs to
reflect +org.apache.zookeeper.ClientCnxn$EventThread+ which may throw
ClassNotFoundException because of shading. Click it for more detail
[InjectSessionExpiration|https://github.com/apache/curator/blob/master/curator-client/src/main/java/org/apache/curator/utils/InjectSessionExpiration.java].
Second, for your second point, I am not familiar with LeaderSeclector currently
and I'm learning about it. I also think it is a ideally way we can just use
SessionConnectionStateErrorPolicy directly in curator-4.x
Third, I don't understand the meaning of a flink scope leader latch
was (Author: lamber-ken):
[~Tison] (y), I have some points need to talk with you.
First, for your first point, I thought it yesterday and wont create a new
curator Jira like your CURATOR-532 that use can manually config ZooKeeper3.4.x
Compatibility, but I give up that idea, because I found that it also needs to
reflect +org.apache.zookeeper.ClientCnxn$EventThread+ which may throw
ClassNotFoundException because of shading. Click it for more detail
[InjectSessionExpiration|https://github.com/apache/curator/blob/master/curator-client/src/main/java/org/apache/curator/utils/InjectSessionExpiration.java].
Second, for your second point, I am not familiar with LeaderSeclector currently
and I'm learning about it. I also think it is a ideally way we can just use
SessionConnectionStateErrorPolicy directly in curator-4.x
Third, I don't understand the meaning of a flink scope leader latch
> 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)