[
https://issues.apache.org/jira/browse/FLINK-20138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17231387#comment-17231387
]
Till Rohrmann commented on FLINK-20138:
---------------------------------------
Hi [~1026688210], thanks for creating this ticket. From the logs, it looks as
if the {{JobMaster}} never learns about the running {{ResourceManager}} and,
thus, can never connect to it and ask for the required slots. Could you check
whether the {{ResourceManager}} has written its leader address to
{{/flink/115/2476/application_1576582691395_11322/leader/resource_manager_lock}}.
The address information should be the data payload of this znode.
Some exceptions in your logs are the following:
{code}
2020-11-09 16:01:06,519 ERROR main
(org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.logError:566)
- Background exception was not retry-able or retry gave up
java.net.UnknownHostException: haflink.zk23.lf1.in.autohome.com.cn: unknown
error
at java.net.Inet6AddressImpl.lookupAllHostAddr(Native Method)
at java.net.InetAddress$2.lookupAllHostAddr(InetAddress.java:928)
at
java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1323)
at java.net.InetAddress.getAllByName0(InetAddress.java:1276)
at java.net.InetAddress.getAllByName(InetAddress.java:1192)
at java.net.InetAddress.getAllByName(InetAddress.java:1126)
at
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.client.StaticHostProvider.<init>(StaticHostProvider.java:61)
at
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:445)
at
org.apache.flink.shaded.curator.org.apache.curator.utils.DefaultZookeeperFactory.newZooKeeper(DefaultZookeeperFactory.java:29)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl$2.newZooKeeper(CuratorFrameworkImpl.java:150)
at
org.apache.flink.shaded.curator.org.apache.curator.HandleHolder$1.getZooKeeper(HandleHolder.java:94)
at
org.apache.flink.shaded.curator.org.apache.curator.HandleHolder.getZooKeeper(HandleHolder.java:55)
at
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.reset(ConnectionState.java:262)
at
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.start(ConnectionState.java:109)
at
org.apache.flink.shaded.curator.org.apache.curator.CuratorZookeeperClient.start(CuratorZookeeperClient.java:191)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.start(CuratorFrameworkImpl.java:259)
at
org.apache.flink.runtime.util.ZooKeeperUtils.startCuratorFramework(ZooKeeperUtils.java:131)
at
org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.createHighAvailabilityServices(HighAvailabilityServicesUtils.java:123)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.createHaServices(ClusterEntrypoint.java:292)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:257)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:202)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$0(ClusterEntrypoint.java:164)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
at
org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:163)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:501)
at
org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint.main(YarnJobClusterEntrypoint.java:102)
{code}
and
{code}
2020-11-09 16:01:22,479 ERROR main
(org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.checkTimeouts:228)
- Connection timed out for connection string
(haflink.zk24.lf1.in.autohome.com.cn:2182,haflink.zk25.lf1.in.autohome.com.cn:2182,haflink.zk23.lf1.in.autohome.com.cn:2182,haflink.zk27.lf1.in.autohome.com.cn:2182,haflink.zk26.lf1.in.autohome.com.cn:2182)
and timeout (15000) / elapsed (20966)
org.apache.flink.shaded.curator.org.apache.curator.CuratorConnectionLossException:
KeeperErrorCode = ConnectionLoss
at
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:225)
at
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:94)
at
org.apache.flink.shaded.curator.org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:117)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.NamespaceImpl$1.call(NamespaceImpl.java:90)
at
org.apache.flink.shaded.curator.org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:109)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.NamespaceImpl.fixForNamespace(NamespaceImpl.java:83)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.fixForNamespace(CuratorFrameworkImpl.java:594)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.ExistsBuilderImpl.forPath(ExistsBuilderImpl.java:158)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.ExistsBuilderImpl.forPath(ExistsBuilderImpl.java:32)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.reset(NodeCache.java:242)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.start(NodeCache.java:175)
at
org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.start(NodeCache.java:154)
at
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService.start(ZooKeeperLeaderElectionService.java:134)
at
org.apache.flink.runtime.webmonitor.WebMonitorEndpoint.startInternal(WebMonitorEndpoint.java:675)
at
org.apache.flink.runtime.rest.RestServerEndpoint.start(RestServerEndpoint.java:241)
at
org.apache.flink.runtime.entrypoint.component.AbstractDispatcherResourceManagerComponentFactory.create(AbstractDispatcherResourceManagerComponentFactory.java:161)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:210)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$0(ClusterEntrypoint.java:164)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
at
org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:163)
at
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:501)
at
org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint.main(YarnJobClusterEntrypoint.java:102)
{code}
They indicate that there is some connectivity problem with the ZooKeeper quorum.
Is the problem reproducible? Maybe you can produce the debug logs of such a
run. They might be more helpful. Moreover, you could try whether a later Flink
version already solves the problem.
> Flink Job can not recover due to timeout of requiring slots when flink
> jobmanager restarted
> --------------------------------------------------------------------------------------------
>
> Key: FLINK-20138
> URL: https://issues.apache.org/jira/browse/FLINK-20138
> Project: Flink
> Issue Type: Bug
> Components: Deployment / YARN, Table SQL / Runtime
> Environment: flink : 1.9.2
> hadoop :2.7.2
> jdk:1.8
> Reporter: wgcn
> Priority: Major
> Attachments: 2820F7EE-85F9-441D-95D5-8163FB6267DF.png, jobmanager.log
>
>
> our flink jobs run on Yarn Perjob Mode. We stoped some nodemanger machines
> ,and AMs of the machines restarted at other nodemanager. We found some
> jobs can not recover due to timeout of requiring slots.
> *SlotPoolImp always did not connect ResourceManager *
> ```
> 2020-11-09 16:31:31,794 INFO
> flink-akka.actor.default-dispatcher-16
> (org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.stashRequestWaitingForResourceManager:369)
> - Cannot serve slot request, no ResourceManager connected. Adding as pending
> request [SlotRequestId{456c9daa6670a4490810f8e51f495174}]
> ```
> *1.We did not find the log of YarnResourceManager requesting container at
> the jobmanager log of attachment.
> 2.The node of Zookeeper is also showed at attachment .*
--
This message was sent by Atlassian Jira
(v8.3.4#803005)