[ https://issues.apache.org/jira/browse/FLINK-10052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16568832#comment-16568832 ]
Elias Levy edited comment on FLINK-10052 at 8/3/18 9:46 PM: ------------------------------------------------------------ [~till.rohrmann] as I mentioned in FLINK-10011, it may not be necessary to replace the {{LeaderLatch}} Curator recipe to avoid loosing leadership during temporary connection failures. The Curator error handling [documentation|https://curator.apache.org/errors.html] talks about a {{SessionConnectionStateErrorPolicy}} that treats {{SUSPENDED}} and {{LOST}} connection states differently. And this [test|https://github.com/apache/curator/blob/d502dde1c4601b2abc6d831d764561a73316bf00/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java#L72-L146] shows how leadership is not lost with a {{LeaderLatch}} and that policy. The [code|https://github.com/apache/curator/blob/ed3082ecfebc332ba96da7a5bda4508a1985db6e/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java#L625-L631] implementing the policy. And [this shows|https://github.com/apache/curator/blob/5920c744508afd678a20309313e1b8d78baac0c4/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java#L298-L314] that Curator will inject a session expiration even while it is in {{SUSPENDED}} state, so that a disconnected client won't continue to think it is leader past its session expiration. So it is possible that all we need to do is call {{connectionStateErrorPolicy(new SessionConnectionStateErrorPolicy())}} in the {{CuratorFrameworkFactory}}. was (Author: elevy): [~till.rohrmann] as I mentioned in FLINK-10011, it may not be necessary to replace the {{LeaderLatch}} Curator recipe to avoid loosing leadership during temporary connection failures. The Curator error handling [documentation|https://curator.apache.org/errors.html] talks about a {{SessionConnectionStateErrorPolicy}} that treats {{SUSPENDED }}and {{LOST}} connection states differently. And this [test|https://github.com/apache/curator/blob/d502dde1c4601b2abc6d831d764561a73316bf00/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java#L72-L146] shows how leadership is not lost with a {{LeaderLatch}} and that policy. The [code|https://github.com/apache/curator/blob/ed3082ecfebc332ba96da7a5bda4508a1985db6e/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java#L625-L631] implementing the policy. And [this shows|https://github.com/apache/curator/blob/5920c744508afd678a20309313e1b8d78baac0c4/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java#L298-L314] that Curator will inject a session expiration even while it is in {{SUSPENDED }}state, so that a disconnected client won't continue to think it is leader past its session expiration. So it is possible that all we need to do is call {{connectionStateErrorPolicy(new SessionConnectionStateErrorPolicy())}} in the {{CuratorFrameworkFactory}}. > Tolerate temporarily suspended ZooKeeper connections > ---------------------------------------------------- > > Key: FLINK-10052 > URL: https://issues.apache.org/jira/browse/FLINK-10052 > Project: Flink > Issue Type: Improvement > Components: Distributed Coordination > Affects Versions: 1.4.2, 1.5.2, 1.6.0 > Reporter: Till Rohrmann > 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.3#76005)