You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Yordan Pavlov (Jira)" <ji...@apache.org> on 2021/01/28 14:18:00 UTC

[jira] [Commented] (FLINK-5996) Jobmanager HA should not crash on lost ZK node

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

Yordan Pavlov commented on FLINK-5996:
--------------------------------------

Discovered this ticket looking for explanation of Flink behavior regarding Zookeeper node being down. We are running in Kubernetes and have 3 ZK nodes in our setup, having 2 out of 3 running makes a quorum, still what I am observing is that even when one ZK is down we see this:

{{[main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
[Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender LeaderContender: DefaultDispatcherRunner no longer participates in the leader election.
[flink-akka.actor.default-dispatcher-27] INFO  o.a.flink.runtime.resourcemanager.StandaloneResourceManager  - Closing TaskExecutor connection 7c61bd584aa71bb1ff998234fff90cde because: ResourceManager leader changed to new address null
}}

which basically restarts the job. Is my understanding correct, and if so isn't this considered a Flink problem?

> Jobmanager HA should not crash on lost ZK node
> ----------------------------------------------
>
>                 Key: FLINK-5996
>                 URL: https://issues.apache.org/jira/browse/FLINK-5996
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.2.0
>            Reporter: Gyula Fora
>            Assignee: Dominik WosiƄski
>            Priority: Major
>
> Even if there are multiple zk hosts configured the jobmanager crashes if one of them is lost:
> org.apache.flink.shaded.org.apache.curator.CuratorConnectionLossException: KeeperErrorCode = ConnectionLoss
> 	at org.apache.flink.shaded.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:197)
> 	at org.apache.flink.shaded.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:87)
> 	at org.apache.flink.shaded.org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:115)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl.getZooKeeper(CuratorFrameworkImpl.java:477)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.GetDataBuilderImpl$4.call(GetDataBuilderImpl.java:302)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.GetDataBuilderImpl$4.call(GetDataBuilderImpl.java:291)
> 	at org.apache.flink.shaded.org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:107)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.GetDataBuilderImpl.pathInForeground(GetDataBuilderImpl.java:288)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.GetDataBuilderImpl.forPath(GetDataBuilderImpl.java:279)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.GetDataBuilderImpl.forPath(GetDataBuilderImpl.java:41)
> 	at org.apache.flink.shaded.org.apache.curator.framework.recipes.shared.SharedValue.readValue(SharedValue.java:244)
> 	at org.apache.flink.shaded.org.apache.curator.framework.recipes.shared.SharedValue.access$100(SharedValue.java:44)
> 	at org.apache.flink.shaded.org.apache.curator.framework.recipes.shared.SharedValue$1.process(SharedValue.java:61)
> 	at org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceWatcher.process(NamespaceWatcher.java:67)
> 	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:522)
> 	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> We should have some retry logic there



--
This message was sent by Atlassian Jira
(v8.3.4#803005)