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

Till Rohrmann commented on FLINK-7022:
--------------------------------------

I managed to reproduce the problem on my local machine with starting the 
ZooKeeper cluster under an unresolvable name and then adding it to 
{{/etc/hosts}} after the cluster has started. The problem was that in my case 
the cluster did not recover. In fact the logs looked more like the first logs 
you have attached where the cluster also did not recover. When looking at the 
ZooKeeper cluster, I saw that there were no zNodes created which could explain 
why the cluster did not make progress. To me this looks a bit like a Curator or 
ZooKeeper problem that it either does not create the zNodes or fails properly.

In the second logs you have attached one could actually see that the cluster 
made progress (the components gained leadership). Hence, I would assume that 
also the web UI should be responsive. Could it be that your browser cached some 
REST responses? Maybe you could try to connect to the cluster from incognito 
mode or a new browser window.



> Flink Job Manager Scheduler & Web Frontend out of sync when Zookeeper is 
> unavailable on startup
> -----------------------------------------------------------------------------------------------
>
>                 Key: FLINK-7022
>                 URL: https://issues.apache.org/jira/browse/FLINK-7022
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.2.0, 1.2.1, 1.3.0
>         Environment: Kubernetes cluster running:
> * Flink 1.3.0 Job Manager & Task Manager on Java 8u131
> * Zookeeper 3.4.10 cluster with 3 nodes
>            Reporter: Scott Kidder
>            Priority: Major
>         Attachments: log.txt, log2.txt
>
>
> h2. Problem
> Flink Job Manager web frontend is permanently unavailable if one or more 
> Zookeeper nodes are unresolvable during startup. The job scheduler eventually 
> recovers and assigns jobs to task managers, but the web frontend continues to 
> respond with an HTTP 503 and the following message:
> {noformat}Service temporarily unavailable due to an ongoing leader election. 
> Please refresh.{noformat}
> h2. Expected Behavior
> Once Flink is able to interact with Zookeeper successfully, all aspects of 
> the Job Manager (job scheduling & the web frontend) should be available.
> h2. Environment Details
> We're running Flink and Zookeeper in Kubernetes on CoreOS. CoreOS can run in 
> a configuration that automatically detects and applies operating system 
> updates. We have a Zookeeper node running on the same CoreOS instance as 
> Flink. It's possible that the Zookeeper node will not yet be started when the 
> Flink components are started. This could cause hostname resolution of the 
> Zookeeper nodes to fail.
> h3. Flink Task Manager Logs
> {noformat}
> 2017-06-27 15:38:47,161 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.host, localhost
> 2017-06-27 15:38:47,161 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.port, 8125
> 2017-06-27 15:38:47,162 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.interval, 10 SECONDS
> 2017-06-27 15:38:47,254 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: state.backend, filesystem
> 2017-06-27 15:38:47,254 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: state.backend.fs.checkpointdir, 
> hdfs://hdfs:8020/flink/checkpoints
> 2017-06-27 15:38:47,255 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: state.savepoints.dir, 
> hdfs://hdfs:8020/flink/savepoints
> 2017-06-27 15:38:47,255 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.mode, zookeeper
> 2017-06-27 15:38:47,256 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.zookeeper.quorum, 
> zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181
> 2017-06-27 15:38:47,256 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.zookeeper.storageDir, 
> hdfs://hdfs:8020/flink/recovery
> 2017-06-27 15:38:47,256 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.jobmanager.port, 6123
> 2017-06-27 15:38:47,257 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: blob.server.port, 41479
> 2017-06-27 15:38:47,357 WARN  org.apache.flink.configuration.Configuration    
>               - Config uses deprecated configuration key 'recovery.mode' 
> instead of proper key 'high-availability'
> 2017-06-27 15:38:47,366 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager with high-availability
> 2017-06-27 15:38:47,366 WARN  org.apache.flink.configuration.Configuration    
>               - Config uses deprecated configuration key 
> 'recovery.jobmanager.port' instead of proper key 
> 'high-availability.jobmanager.port'
> 2017-06-27 15:38:47,452 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager on flink:6123 with execution mode CLUSTER
> 2017-06-27 15:38:47,549 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.rpc.address, flink
> 2017-06-27 15:38:47,549 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.rpc.port, 6123
> 2017-06-27 15:38:47,549 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.heap.mb, 1024
> 2017-06-27 15:38:47,549 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: taskmanager.heap.mb, 1024
> 2017-06-27 15:38:47,549 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: taskmanager.numberOfTaskSlots, 1
> 2017-06-27 15:38:47,549 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: taskmanager.memory.preallocate, false
> 2017-06-27 15:38:47,550 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: parallelism.default, 1
> 2017-06-27 15:38:47,550 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.web.port, 8081
> 2017-06-27 15:38:47,550 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporters, statsd
> 2017-06-27 15:38:47,550 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.class, 
> org.apache.flink.metrics.statsd.StatsDReporter
> 2017-06-27 15:38:47,551 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.host, localhost
> 2017-06-27 15:38:47,551 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.port, 8125
> 2017-06-27 15:38:47,551 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.interval, 10 SECONDS
> 2017-06-27 15:38:47,551 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: state.backend, filesystem
> 2017-06-27 15:38:47,551 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: state.backend.fs.checkpointdir, 
> hdfs://hdfs:8020/flink/checkpoints
> 2017-06-27 15:38:47,552 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: state.savepoints.dir, 
> hdfs://hdfs:8020/flink/savepoints
> 2017-06-27 15:38:47,552 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.mode, zookeeper
> 2017-06-27 15:38:47,552 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.zookeeper.quorum, 
> zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181
> 2017-06-27 15:38:47,552 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.zookeeper.storageDir, 
> hdfs://hdfs:8020/flink/recovery
> 2017-06-27 15:38:47,552 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: recovery.jobmanager.port, 6123
> 2017-06-27 15:38:47,552 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: blob.server.port, 41479
> 2017-06-27 15:38:48,055 INFO  
> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user 
> set to root (auth:SIMPLE)
> 2017-06-27 15:38:48,664 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager actor system reachable at flink:6123
> 2017-06-27 15:38:50,955 INFO  akka.event.slf4j.Slf4jLogger                    
>               - Slf4jLogger started
> 2017-06-27 15:38:51,252 INFO  Remoting                                        
>               - Starting remoting
> 2017-06-27 15:38:52,679 INFO  Remoting                                        
>               - Remoting started; listening on addresses 
> :[akka.tcp://flink@flink:6123]
> 2017-06-27 15:38:52,758 WARN  org.apache.flink.configuration.Configuration    
>               - Config uses deprecated configuration key 'recovery.mode' 
> instead of proper key 'high-availability'
> 2017-06-27 15:38:52,761 WARN  org.apache.flink.configuration.Configuration    
>               - Config uses deprecated configuration key 'recovery.mode' 
> instead of proper key 'high-availability'
> 2017-06-27 15:38:52,764 WARN  org.apache.flink.configuration.Configuration    
>               - Config uses deprecated configuration key 
> 'recovery.zookeeper.storageDir' instead of proper key 
> 'high-availability.storageDir'
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.rpc.address, flink
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.rpc.port, 6123
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.heap.mb, 1024
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: taskmanager.heap.mb, 1024
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: taskmanager.numberOfTaskSlots, 1
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: taskmanager.memory.preallocate, false
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: parallelism.default, 1
> 2017-06-27 15:38:52,854 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: jobmanager.web.port, 8081
> 2017-06-27 15:38:52,864 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporters, statsd
> 2017-06-27 15:38:52,865 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.class, 
> org.apache.flink.metrics.statsd.StatsDReporter
> 2017-06-27 15:38:52,865 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.host, localhost
> 2017-06-27 15:38:52,865 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.port, 8125
> 2017-06-27 15:38:52,865 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: metrics.reporter.statsd.interval, 10 SECONDS
>       at org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:445)
>       at 
> org.apache.flink.shaded.org.apache.curator.utils.DefaultZookeeperFactory.newZooKeeper(DefaultZookeeperFactory.java:29)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl$2.newZooKeeper(CuratorFrameworkImpl.java:150)
>       at 
> org.apache.flink.shaded.org.apache.curator.HandleHolder$1.getZooKeeper(HandleHolder.java:94)
>       at 
> org.apache.flink.shaded.org.apache.curator.HandleHolder.getZooKeeper(HandleHolder.java:55)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.reset(ConnectionState.java:262)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.start(ConnectionState.java:109)
>       at 
> org.apache.flink.shaded.org.apache.curator.CuratorZookeeperClient.start(CuratorZookeeperClient.java:191)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl.start(CuratorFrameworkImpl.java:259)
>       at 
> org.apache.flink.runtime.util.ZooKeeperUtils.startCuratorFramework(ZooKeeperUtils.java:128)
>       at 
> org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.createHighAvailabilityServices(HighAvailabilityServicesUtils.java:96)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2047)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply$mcV$sp(JobManager.scala:2139)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.retryOnBindException(JobManager.scala:2172)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1992)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1990)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext$1.run(HadoopSecurityContext.java:43)
>       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:1698)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:40)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.main(JobManager.scala:1990)
>       at org.apache.flink.runtime.jobmanager.JobManager.main(JobManager.scala)
> 2017-06-27 15:38:59,160 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager web frontend
> 2017-06-27 15:38:59,257 INFO  
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Determined 
> location of JobManager log file: 
> /usr/local/flink-1.3.0/log/flink--jobmanager-0-flink-jobmanager-3380372638-1q7jb.log
> 2017-06-27 15:38:59,257 INFO  
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Determined 
> location of JobManager stdout file: 
> /usr/local/flink-1.3.0/log/flink--jobmanager-0-flink-jobmanager-3380372638-1q7jb.out
> 2017-06-27 15:38:59,257 INFO  
> org.apache.flink.runtime.webmonitor.WebRuntimeMonitor         - Using 
> directory /tmp/flink-web-252afcf4-d41d-4095-a082-f6ce5176c2f5 for the web 
> interface files
> 2017-06-27 15:38:59,257 INFO  
> org.apache.flink.runtime.webmonitor.WebRuntimeMonitor         - Using 
> directory /tmp/flink-web-2ca2cadf-a1b6-44af-9510-9c523a422022 for web 
> frontend JAR file uploads
> 2017-06-27 15:39:01,060 INFO  
> org.apache.flink.runtime.webmonitor.WebRuntimeMonitor         - Web frontend 
> listening at 0:0:0:0:0:0:0:0:8081
> 2017-06-27 15:39:01,060 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager actor
> 2017-06-27 15:39:01,253 INFO  org.apache.flink.runtime.blob.BlobServer        
>               - Created BLOB server storage directory 
> /tmp/blobStore-1f49aadd-0a7d-45d1-8fdc-fc2167ca93d5
> 2017-06-27 15:39:01,257 INFO  org.apache.flink.runtime.blob.BlobServer        
>               - Started BLOB server at 0.0.0.0:41479 - max concurrent 
> requests: 50 - max backlog: 1000
> 2017-06-27 15:39:01,851 INFO  org.apache.flink.runtime.metrics.MetricRegistry 
>               - Configuring StatsDReporter with {interval=10 SECONDS, 
> port=8125, host=localhost, 
> class=org.apache.flink.metrics.statsd.StatsDReporter}.
> 2017-06-27 15:39:01,948 INFO  org.apache.flink.metrics.statsd.StatsDReporter  
>               - Configured StatsDReporter with {host:localhost, port:8125}
> 2017-06-27 15:39:01,949 INFO  org.apache.flink.runtime.metrics.MetricRegistry 
>               - Periodically reporting metrics in intervals of 10 SECONDS for 
> reporter statsd of type org.apache.flink.metrics.statsd.StatsDReporter.
> 2017-06-27 15:39:02,050 INFO  
> org.apache.flink.runtime.jobmanager.MemoryArchivist           - Started 
> memory archivist akka://flink/user/archive
> 2017-06-27 15:39:02,059 WARN  org.apache.flink.configuration.Configuration    
>               - Config uses deprecated configuration key 
> 'recovery.zookeeper.storageDir' instead of proper key 
> 'high-availability.storageDir'
> 2017-06-27 15:39:17,252 ERROR 
> org.apache.flink.shaded.org.apache.curator.ConnectionState    - Connection 
> timed out for connection string 
> (zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181)
>  and timeout (15000) / elapsed (18395)
> org.apache.flink.shaded.org.apache.curator.CuratorConnectionLossException: 
> KeeperErrorCode = ConnectionLoss
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:225)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:94)
>       at 
> org.apache.flink.shaded.org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:117)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceImpl$1.call(NamespaceImpl.java:90)
>       at 
> org.apache.flink.shaded.org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:109)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceImpl.fixForNamespace(NamespaceImpl.java:83)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceImpl.newNamespaceAwareEnsurePath(NamespaceImpl.java:109)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl.newNamespaceAwareEnsurePath(CuratorFrameworkImpl.java:469)
>       at 
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.<init>(ZooKeeperSubmittedJobGraphStore.java:116)
>       at 
> org.apache.flink.runtime.util.ZooKeeperUtils.createSubmittedJobGraphs(ZooKeeperUtils.java:263)
>       at 
> org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices.getSubmittedJobGraphStore(ZooKeeperHaServices.java:149)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2716)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2641)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2298)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.liftedTree3$1(JobManager.scala:2053)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2052)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply$mcV$sp(JobManager.scala:2139)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:40)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.main(JobManager.scala:1990)
>       at org.apache.flink.runtime.jobmanager.JobManager.main(JobManager.scala)
> 2017-06-27 15:39:37,448 INFO  org.apache.zookeeper.ZooKeeper                  
>               - Initiating client connection, 
> connectString=zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181
>  sessionTimeout=60000 
> watcher=org.apache.flink.shaded.org.apache.curator.ConnectionState@27cbfddf
> 2017-06-27 15:40:07,457 WARN  
> org.apache.flink.shaded.org.apache.curator.ConnectionState    - Connection 
> attempt unsuccessful after 68603 (greater than max timeout of 60000). 
> Resetting connection and trying again with a new connection.
> 2017-06-27 15:40:07,457 INFO  org.apache.zookeeper.ZooKeeper                  
>               - Initiating client connection, 
> connectString=zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181
>  sessionTimeout=60000 
> watcher=org.apache.flink.shaded.org.apache.curator.ConnectionState@27cbfddf
> 2017-06-27 15:40:07,555 ERROR 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl
>   - Ensure path threw exception
> java.net.UnknownHostException: zookeeper-1.zookeeper: Name or service not 
> known
>       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.zookeeper.client.StaticHostProvider.<init>(StaticHostProvider.java:61)
>       at org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:445)
>       at 
> org.apache.flink.shaded.org.apache.curator.utils.DefaultZookeeperFactory.newZooKeeper(DefaultZookeeperFactory.java:29)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl$2.newZooKeeper(CuratorFrameworkImpl.java:150)
>       at 
> org.apache.flink.shaded.org.apache.curator.HandleHolder$1.getZooKeeper(HandleHolder.java:94)
>       at 
> org.apache.flink.shaded.org.apache.curator.HandleHolder.internalClose(HandleHolder.java:128)
>       at 
> org.apache.flink.shaded.org.apache.curator.HandleHolder.closeAndReset(HandleHolder.java:77)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.reset(ConnectionState.java:261)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:221)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:94)
>       at 
> org.apache.flink.shaded.org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:117)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceImpl$1.call(NamespaceImpl.java:90)
>       at 
> org.apache.flink.shaded.org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:109)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceImpl.fixForNamespace(NamespaceImpl.java:83)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceImpl.newNamespaceAwareEnsurePath(NamespaceImpl.java:109)
>       at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl.newNamespaceAwareEnsurePath(CuratorFrameworkImpl.java:469)
>       at 
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.<init>(ZooKeeperSubmittedJobGraphStore.java:116)
>       at 
> org.apache.flink.runtime.util.ZooKeeperUtils.createSubmittedJobGraphs(ZooKeeperUtils.java:263)
>       at 
> org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices.getSubmittedJobGraphStore(ZooKeeperHaServices.java:149)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2716)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2641)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2298)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.liftedTree3$1(JobManager.scala:2053)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2052)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply$mcV$sp(JobManager.scala:2139)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.retryOnBindException(JobManager.scala:2172)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1992)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1990)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext$1.run(HadoopSecurityContext.java:43)
>       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:1698)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:40)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.main(JobManager.scala:1990)
>       at org.apache.flink.runtime.jobmanager.JobManager.main(JobManager.scala)
> 2017-06-27 15:40:22,566 ERROR 
> org.apache.flink.shaded.org.apache.curator.ConnectionState    - Connection 
> timed out for connection string 
> (zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181)
>  and timeout (15000) / elapsed (15108)
> org.apache.flink.shaded.org.apache.curator.CuratorConnectionLossException: 
> KeeperErrorCode = ConnectionLoss
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:225)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:94)
>       at 
> org.apache.flink.runtime.util.ZooKeeperUtils.createSubmittedJobGraphs(ZooKeeperUtils.java:263)
>       at 
> org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices.getSubmittedJobGraphStore(ZooKeeperHaServices.java:149)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2716)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2641)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2298)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.liftedTree3$1(JobManager.scala:2053)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2052)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply$mcV$sp(JobManager.scala:2139)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.retryOnBindException(JobManager.scala:2172)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1992)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1990)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext$1.run(HadoopSecurityContext.java:43)
>       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:1698)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:40)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.main(JobManager.scala:1990)
>       at org.apache.flink.runtime.jobmanager.JobManager.main(JobManager.scala)
> 2017-06-27 15:40:42,575 INFO  org.apache.zookeeper.ZooKeeper                  
>               - Initiating client connection, 
> connectString=zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181
>  sessionTimeout=60000 
> watcher=org.apache.flink.shaded.org.apache.curator.ConnectionState@27cbfddf
> 2017-06-27 15:41:02,684 ERROR 
> org.apache.flink.shaded.org.apache.curator.ConnectionState    - Connection 
> timed out for connection string 
> (zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181)
>  and timeout (15000) / elapsed (55226)
> org.apache.flink.shaded.org.apache.curator.CuratorConnectionLossException: 
> KeeperErrorCode = ConnectionLoss
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:225)
>       at 
> org.apache.flink.shaded.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:94)
>       at 
> org.apache.flink.shaded.org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:117)
>       at 
> org.apache.flink.shaded.org.apache.curator.utils.EnsurePath$InitialHelper$1.call(EnsurePath.java:156)
>       at 
> org.apache.flink.shaded.org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:109)
>       at 
> org.apache.flink.shaded.org.apache.curator.utils.EnsurePath$InitialHelper.ensure(EnsurePath.java:149)
>       at 
> org.apache.flink.shaded.org.apache.curator.utils.EnsurePath.ensure(EnsurePath.java:102)
>       at 
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.<init>(ZooKeeperSubmittedJobGraphStore.java:117)
>       at 
> org.apache.flink.runtime.util.ZooKeeperUtils.createSubmittedJobGraphs(ZooKeeperUtils.java:263)
>       at 
> org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices.getSubmittedJobGraphStore(ZooKeeperHaServices.java:149)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2716)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2641)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.startJobManagerActors(JobManager.scala:2298)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.liftedTree3$1(JobManager.scala:2053)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2052)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply$mcV$sp(JobManager.scala:2139)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$2.apply(JobManager.scala:2117)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.retryOnBindException(JobManager.scala:2172)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2117)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1992)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anon$10.call(JobManager.scala:1990)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext$1.run(HadoopSecurityContext.java:43)
>       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:1698)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:40)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$.main(JobManager.scala:1990)
>       at org.apache.flink.runtime.jobmanager.JobManager.main(JobManager.scala)
> 2017-06-27 15:41:02,684 INFO  org.apache.zookeeper.ZooKeeper                  
>               - Initiating client connection, 
> connectString=zookeeper-0.zookeeper:2181,zookeeper-1.zookeeper:2181,zookeeper-2.zookeeper:2181
>  sessionTimeout=60000 
> watcher=org.apache.flink.shaded.org.apache.curator.ConnectionState@27cbfddf
> 2017-06-27 15:41:02,803 WARN  org.apache.zookeeper.ClientCnxn                 
>               - SASL configuration failed: 
> javax.security.auth.login.LoginException: No JAAS configuration section named 
> 'Client' was found in specified JAAS configuration file: 
> '/tmp/jaas-1381454376626202001.conf'. Will continue connection to Zookeeper 
> server without SASL authentication, if Zookeeper server allows it.
> 2017-06-27 15:41:02,804 ERROR 
> org.apache.flink.shaded.org.apache.curator.ConnectionState    - 
> Authentication failed
> 2017-06-27 15:41:02,806 INFO  org.apache.zookeeper.ClientCnxn                 
>               - Opening socket connection to server 
> ip-10-2-8-5.ec2.internal/10.2.8.5:2181
> ...
> 2017-06-27 16:00:51,490 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Try to 
> restart or fail the job  (022d8149808dd3297a8a7275a1fd3d6b) if no longer 
> possible.
> 2017-06-27 16:00:51,490 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job  
> (022d8149808dd3297a8a7275a1fd3d6b) switched from state FAILING to RESTARTING.
> 2017-06-27 16:00:51,490 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Restarting 
> the job  (022d8149808dd3297a8a7275a1fd3d6b).
> 2017-06-27 16:00:51,490 INFO  
> org.apache.flink.runtime.executiongraph.restart.ExecutionGraphRestarter  - 
> Delaying retry of job execution for 10000 ms ...
> 2017-06-27 16:00:58,252 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Task Manager Registration but not connected to ResourceManager
> 2017-06-27 16:00:58,254 INFO  
> org.apache.flink.runtime.instance.InstanceManager             - Registered 
> TaskManager at flink-taskmanager-3116622558-zmmwq 
> (akka.tcp://[email protected]:6122/user/taskmanager) as 
> 2a058f00bd1e25f44c1cb8f3e5dd726f. Current number of registered hosts is 1. 
> Current number of alive task slots is 2.
> 2017-06-27 16:00:58,453 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Task Manager Registration but not connected to ResourceManager
> 2017-06-27 16:01:01,491 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job  
> (022d8149808dd3297a8a7275a1fd3d6b) switched from state RESTARTING to CREATED.
> 2017-06-27 16:01:01,491 INFO  
> org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - 
> Recovering checkpoints from ZooKeeper.
> 2017-06-27 16:01:01,645 INFO  
> org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - 
> Found 1 checkpoints in ZooKeeper.
> 2017-06-27 16:01:01,645 INFO  
> org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - 
> Trying to retrieve checkpoint 502.
> 2017-06-27 16:01:01,660 INFO  
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator     - Restoring 
> from latest valid checkpoint: Checkpoint 502 @ 1498577858587 for 
> 022d8149808dd3297a8a7275a1fd3d6b.
> 2017-06-27 16:01:01,661 INFO  
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator     - No master 
> state to restore
> 2017-06-27 16:01:01,661 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job  
> (022d8149808dd3297a8a7275a1fd3d6b) switched from state CREATED to RUNNING.
> {noformat}



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

Reply via email to