Hi Till,
Thanks for the response.
please see the attached log file.

*HA config is : *
high-availability: zookeeper
high-availability.storageDir: hdfs://flink-hdfs:9000/flink/checkpoints
>From the logs i can see block missing exceptions from hdfs, but i can see
that the jobgraph is still present in hdfs.



On Wed, May 8, 2019 at 7:56 PM Till Rohrmann <trohrm...@apache.org> wrote:

> Hi Manju,
>
> could you share the full logs or at least the full stack trace of the
> exception with us?
>
> I suspect that after a failover Flink tries to restore the JobGraph from
> persistent storage (the directory which you have configured via
> `high-availability.storageDir`) but is not able to do so. One reason could
> be that the JobGraph file has been removed by a third party, for example. I
> think the cause of the FlinkException could shed light on it. Could you
> verify that the JobGraph file is still accessible?
>
> Cheers,
> Till
>
> On Wed, May 8, 2019 at 11:22 AM Manjusha Vuyyuru <vmanjusha....@gmail.com>
> wrote:
>
>> Any update on this from community side?
>>
>> On Tue, May 7, 2019 at 6:43 PM Manjusha Vuyyuru <vmanjusha....@gmail.com>
>> wrote:
>>
>>> im using 1.7.2.
>>>
>>>
>>> On Tue, May 7, 2019 at 5:50 PM miki haiat <miko5...@gmail.com> wrote:
>>>
>>>> Which flink version are you using?
>>>> I had similar  issues with 1.5.x
>>>>
>>>> On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <
>>>> vmanjusha....@gmail.com> wrote:
>>>>
>>>>> Hello,
>>>>>
>>>>> I have a flink setup with two job managers coordinated by zookeeper.
>>>>>
>>>>> I see the below exception and both jobmanagers are going down:
>>>>>
>>>>> 2019-05-07 08:29:13,346 INFO
>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from 
>>>>> ZooKeeper.
>>>>> 2019-05-07 08:29:13,346 ERROR
>>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -*
>>>>> Fatal error occurred in the cluster entrypoint.*
>>>>> java.lang.RuntimeException: org.apache.flink.util.FlinkException:
>>>>> Could not retrieve submitted JobGraph from state handle under
>>>>> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
>>>>> handle is broken. Try cleaning the state handle store.
>>>>>         at
>>>>> org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>>>>         at
>>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>>>>>         at
>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>>>>>         at
>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>         at
>>>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>>>>>         at
>>>>> akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>>>>>         at
>>>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>>>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>>>>> submitted JobGraph from state handle under
>>>>> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
>>>>> handle is broken. Try cleaning the state handle store.
>>>>>         at
>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
>>>>>         at
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
>>>>>         at
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
>>>>>         at
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
>>>>>         at
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
>>>>>         at
>>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
>>>>>         ... 9 more
>>>>>
>>>>>
>>>>> Can someone please help me understand in detail on what is causing
>>>>> this exception. I can see zookeeper not able to retrieve job graph. What
>>>>> could be the reason for this?
>>>>>
>>>>> This is second time that my setup is going down with this excepton,
>>>>> first time i cleared jobgraph folder in zookeeper and restarted, now again
>>>>> faced with same issue.
>>>>>
>>>>> Since this is production setup this way of outage is not at all
>>>>> expected :(. Can someone help me how to give a permanent fix to this 
>>>>> issue?
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Manju
>>>>>
>>>>>
2019-05-07 08:28:16,658 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - --------------------------------------------------------------------------------
2019-05-07 08:28:16,659 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting StandaloneSessionClusterEntrypoint (Version: 1.7.1, Rev:89eafb4, Date:14.12.2018 @ 15:48:34 GMT)
2019-05-07 08:28:16,660 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current user: flink
2019-05-07 08:28:17,201 WARN  org.apache.hadoop.util.NativeCodeLoader                       - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2019-05-07 08:28:17,319 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current Hadoop/Kerberos user: flink
2019-05-07 08:28:17,320 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM: Java HotSpot(TM) 64-Bit Server VM - Oracle Corporation - 1.8/25.131-b11
2019-05-07 08:28:17,320 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum heap size: 1963 MiBytes
2019-05-07 08:28:17,320 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME: (not set)
2019-05-07 08:28:17,325 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop version: 2.8.3
2019-05-07 08:28:17,325 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM Options:
2019-05-07 08:28:17,325 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     -Xms2048m
2019-05-07 08:28:17,325 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     -Xmx2048m
2019-05-07 08:28:17,325 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     -Dlog.file=/usr/lib/shieldsquare/flink-1.7.1/log/flink-flink-standalonesession-0-flink-jobmanager-2.log
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     -Dlog4j.configuration=file:/usr/lib/shieldsquare/flink-1.7.1/conf/log4j.properties
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     -Dlogback.configurationFile=file:/usr/lib/shieldsquare/flink-1.7.1/conf/logback.xml
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program Arguments:
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --configDir
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     /usr/lib/shieldsquare/flink-1.7.1/conf
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --executionMode
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     flink-jobmanager-2
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --webui-port
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     8081
2019-05-07 08:28:17,326 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath: /usr/lib/shieldsquare/flink-1.7.1/lib/flink-metrics-prometheus-1.7.1.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/flink-python_2.11-1.7.1.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/flink-shaded-hadoop2-uber-1.7.1.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/log4j-1.2.17.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/slf4j-log4j12-1.7.15.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/flink-dist_2.11-1.7.1.jar:::
2019-05-07 08:28:17,327 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - --------------------------------------------------------------------------------
2019-05-07 08:28:17,328 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered UNIX signal handlers for [TERM, HUP, INT]
2019-05-07 08:28:17,340 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: jobmanager.rpc.address, flink-jobmanager-2
2019-05-07 08:28:17,341 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: jobmanager.rpc.port, 6123
2019-05-07 08:28:17,341 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: jobmanager.heap.size, 2048m
2019-05-07 08:28:17,341 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: taskmanager.heap.size, 2048m
2019-05-07 08:28:17,341 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: taskmanager.numberOfTaskSlots, 2
2019-05-07 08:28:17,342 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: parallelism.default, 1
2019-05-07 08:28:17,342 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: high-availability, zookeeper
2019-05-07 08:28:17,342 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: high-availability.storageDir, hdfs://flink-hdfs:9000/flink/checkpoints
2019-05-07 08:28:17,342 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: high-availability.zookeeper.quorum, be-kafka-zookeeper-1:2718,be-kafka-zookeeper-2:2718,be-kafka-zookeeper-3:2718
2019-05-07 08:28:17,342 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: high-availability.zookeeper.path.root, /flink
2019-05-07 08:28:17,342 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: high-availability.cluster-id, /default_ns
2019-05-07 08:28:17,343 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: state.backend, rocksdb
2019-05-07 08:28:17,343 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: state.checkpoints.dir, hdfs://flink-hdfs:9000/flink-checkpoints
2019-05-07 08:28:17,343 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: state.savepoints.dir, hdfs://flink-hdfs:9000/flink-checkpoints
2019-05-07 08:28:17,343 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: rest.port, 8081
2019-05-07 08:28:17,343 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: state.checkpoints.num-retained, 2
2019-05-07 08:28:17,343 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: io.tmp.dirs, /usr/lib/shieldsquare/io
2019-05-07 08:28:17,344 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: blob.storage.directory, /usr/lib/shieldsquare/blobstore
2019-05-07 08:28:17,344 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: web.tmpdir, /usr/lib/shieldsquare/web
2019-05-07 08:28:17,344 INFO  org.apache.flink.configuration.GlobalConfiguration            - Loading configuration property: metrics.reporter.prom.class, org.apache.flink.metrics.prometheus.PrometheusReporter
2019-05-07 08:28:17,371 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting StandaloneSessionClusterEntrypoint.
2019-05-07 08:28:17,371 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install default filesystem.
2019-05-07 08:28:17,385 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install security context.
2019-05-07 08:28:17,466 INFO  org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user set to flink (auth:SIMPLE)
2019-05-07 08:28:17,490 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Initializing cluster services.
2019-05-07 08:28:17,706 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Trying to start actor system at flink-jobmanager-2:0
2019-05-07 08:28:18,156 INFO  akka.event.slf4j.Slf4jLogger                                  - Slf4jLogger started
2019-05-07 08:28:18,231 INFO  akka.remote.Remoting                                          - Starting remoting
2019-05-07 08:28:18,516 INFO  akka.remote.Remoting                                          - Remoting started; listening on addresses :[akka.tcp://flink@flink-jobmanager-2:37282]
2019-05-07 08:28:18,527 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Actor system started at akka.tcp://flink@flink-jobmanager-2:37282
2019-05-07 08:28:19,264 INFO  org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating highly available BLOB storage directory at hdfs://flink-hdfs:9000/flink/checkpoints//default_ns/blob
2019-05-07 08:28:19,349 INFO  org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing default ACL for ZK connections
2019-05-07 08:28:19,350 INFO  org.apache.flink.runtime.util.ZooKeeperUtils                  - Using '/flink/default_ns' as Zookeeper namespace.
2019-05-07 08:28:19,421 INFO  org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl  - Starting
2019-05-07 08:28:19,429 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f, built on 03/23/2017 10:13 GMT
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:host.name=flink-jobmanager-2.c.shieldsquare-backend.internal
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.version=1.8.0_131
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.vendor=Oracle Corporation
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.home=/usr/java/jdk1.8.0_131/jre
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.class.path=/usr/lib/shieldsquare/flink-1.7.1/lib/flink-metrics-prometheus-1.7.1.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/flink-python_2.11-1.7.1.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/flink-shaded-hadoop2-uber-1.7.1.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/log4j-1.2.17.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/slf4j-log4j12-1.7.15.jar:/usr/lib/shieldsquare/flink-1.7.1/lib/flink-dist_2.11-1.7.1.jar:::
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.io.tmpdir=/tmp
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:java.compiler=<NA>
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:os.name=Linux
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:os.arch=amd64
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:os.version=3.10.0-957.1.3.el7.x86_64
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:user.name=flink
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:user.home=/home/flink
2019-05-07 08:28:19,430 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client environment:user.dir=/home/flink
2019-05-07 08:28:19,431 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Initiating client connection, connectString=be-kafka-zookeeper-1:2718,be-kafka-zookeeper-2:2718,be-kafka-zookeeper-3:2718 sessionTimeout=60000 watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@7808fb9
2019-05-07 08:28:19,478 INFO  org.apache.flink.runtime.blob.BlobServer                      - Created BLOB server storage directory /usr/lib/shieldsquare/blobstore/blobStore-ad3882c5-a6c8-4a1f-af76-f4e72d943ab4
2019-05-07 08:28:19,481 WARN  org.apache.flink.shaded.zookeeper.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-6001472433452040831.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-05-07 08:28:19,487 INFO  org.apache.flink.runtime.blob.BlobServer                      - Started BLOB server at 0.0.0.0:34994 - max concurrent requests: 50 - max backlog: 1000
2019-05-07 08:28:19,506 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server be-kafka-zookeeper-3.c.shieldsquare-backend.internal/10.20.1.203:2718
2019-05-07 08:28:19,508 ERROR org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-05-07 08:28:19,512 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to be-kafka-zookeeper-3.c.shieldsquare-backend.internal/10.20.1.203:2718, initiating session
2019-05-07 08:28:19,531 INFO  org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server be-kafka-zookeeper-3.c.shieldsquare-backend.internal/10.20.1.203:2718, sessionid = 0x369da2edd1e5df5, negotiated timeout = 40000
2019-05-07 08:28:19,533 INFO  org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager  - State change: CONNECTED
2019-05-07 08:28:19,559 INFO  org.apache.flink.runtime.metrics.MetricRegistryImpl           - Configuring prom with {class=org.apache.flink.metrics.prometheus.PrometheusReporter}.
2019-05-07 08:28:19,568 INFO  org.apache.flink.metrics.prometheus.PrometheusReporter        - Started PrometheusReporter HTTP server on port 9249.
2019-05-07 08:28:19,569 INFO  org.apache.flink.runtime.metrics.MetricRegistryImpl           - Reporting metrics for reporter prom of type org.apache.flink.metrics.prometheus.PrometheusReporter.
2019-05-07 08:28:19,571 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to start actor system at flink-jobmanager-2:0
2019-05-07 08:28:19,611 INFO  akka.event.slf4j.Slf4jLogger                                  - Slf4jLogger started
2019-05-07 08:28:19,622 INFO  akka.remote.Remoting                                          - Starting remoting
2019-05-07 08:28:19,639 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor system started at akka.tcp://flink-metrics@flink-jobmanager-2:37641
2019-05-07 08:28:19,646 INFO  org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  - Initializing FileArchivedExecutionGraphStore: Storage directory /usr/lib/shieldsquare/io/executionGraphStore-09280cc9-3155-4693-bd8d-1ed931d8d6a0, expiration time 3600000, maximum cache size 52428800 bytes.
2019-05-07 08:28:19,648 INFO  akka.remote.Remoting                                          - Remoting started; listening on addresses :[akka.tcp://flink-metrics@flink-jobmanager-2:37641]
2019-05-07 08:28:19,689 INFO  org.apache.flink.runtime.blob.TransientBlobCache              - Created BLOB cache storage directory /usr/lib/shieldsquare/blobstore/blobStore-07aadaf4-af6f-4242-af96-d24ec2a3e505
2019-05-07 08:28:19,723 WARN  org.apache.flink.configuration.Configuration                  - Config uses deprecated configuration key 'jobmanager.rpc.address' instead of proper key 'rest.address'
2019-05-07 08:28:19,725 WARN  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload directory /usr/lib/shieldsquare/web/flink-web-169f1a69-6301-4940-8843-cb75010936c9/flink-web-upload does not exist, or has been deleted externally. Previously uploaded files are no longer available.
2019-05-07 08:28:19,727 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created directory /usr/lib/shieldsquare/web/flink-web-169f1a69-6301-4940-8843-cb75010936c9/flink-web-upload for file uploads.
2019-05-07 08:28:19,731 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting rest endpoint.
2019-05-07 08:28:20,053 INFO  org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Determined location of main cluster component log file: /usr/lib/shieldsquare/flink-1.7.1/log/flink-flink-standalonesession-0-flink-jobmanager-2.log
2019-05-07 08:28:20,053 INFO  org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Determined location of main cluster component stdout file: /usr/lib/shieldsquare/flink-1.7.1/log/flink-flink-standalonesession-0-flink-jobmanager-2.out
2019-05-07 08:28:20,243 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest endpoint listening at flink-jobmanager-2:8081
2019-05-07 08:28:20,244 INFO  org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
2019-05-07 08:28:20,287 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web frontend listening at http://flink-jobmanager-2:8081.
2019-05-07 08:28:20,425 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at akka://flink/user/resourcemanager .
2019-05-07 08:28:20,470 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher at akka://flink/user/dispatcher .
2019-05-07 08:28:20,497 INFO  org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
2019-05-07 08:28:20,505 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-05-07 08:28:20,509 INFO  org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
2019-05-07 08:28:20,511 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
2019-05-07 08:28:20,644 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:20,659 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:20,738 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:20,743 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:20,898 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:20,912 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:20,996 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:20,999 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:21,064 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:21,065 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:21,135 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:21,135 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:21,203 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:21,206 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:21,276 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:21,276 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:21,343 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:21,345 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:21,416 WARN  akka.remote.transport.netty.NettyTransport                    - Remote connection to [null] failed with java.net.ConnectException: Connection refused: flink-jobmanager-1/10.20.1.172:42799
2019-05-07 08:28:21,416 WARN  akka.remote.ReliableDeliverySupervisor                        - Association with remote system [akka.tcp://flink@flink-jobmanager-1:42799] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@flink-jobmanager-1:42799]] Caused by: [Connection refused: flink-jobmanager-1/10.20.1.172:42799]
2019-05-07 08:28:54,013 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - http://flink-jobmanager-2:8081 was granted leadership with leaderSessionID=a5529712-210f-4cc2-9f1c-6c0511df9075
2019-05-07 08:28:54,023 INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Dispatcher akka.tcp://flink@flink-jobmanager-2:37282/user/dispatcher was granted leadership with fencing token 2855245c-6bbf-438b-9233-76b409c4060e
2019-05-07 08:28:54,026 INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Recovering all persisted jobs.
2019-05-07 08:28:54,034 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - ResourceManager akka.tcp://flink@flink-jobmanager-2:37282/user/resourcemanager was granted leadership with fencing token 80bd271e32670e96c4afde4bd6a04172
2019-05-07 08:28:54,035 INFO  org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-05-07 08:28:54,136 WARN  org.apache.hadoop.hdfs.DFSClient                              - No live nodes contain block BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 after checking nodes = [], ignoredNodes = null
2019-05-07 08:28:54,137 INFO  org.apache.hadoop.hdfs.DFSClient                              - No node available for BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
2019-05-07 08:28:54,137 INFO  org.apache.hadoop.hdfs.DFSClient                              - Could not obtain BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 from any node:  No live nodes contain current block Block locations: Dead nodes: . Will get new block locations from namenode and retry...
2019-05-07 08:28:54,137 WARN  org.apache.hadoop.hdfs.DFSClient                              - DFS chooseDataNode: got # 1 IOException, will wait for 1498.8531884268646 msec.
2019-05-07 08:28:54,262 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registering TaskManager with ResourceID af7bf2db776ad36f6d3711396a67a144 (akka.tcp://flink@flink-workers-1:46513/user/taskmanager_0) at ResourceManager
2019-05-07 08:28:54,264 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registering TaskManager with ResourceID 51bd9832fd20afbc4fde61ec2391e003 (akka.tcp://flink@flink-workers-mffn:43458/user/taskmanager_0) at ResourceManager
2019-05-07 08:28:54,264 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registering TaskManager with ResourceID bdd48b77ec028ce62b4873cec4f68b90 (akka.tcp://flink@flink-workers-3xhz:36003/user/taskmanager_0) at ResourceManager
2019-05-07 08:28:54,265 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registering TaskManager with ResourceID 1936ae5986e82ac4fddc58c9ab686154 (akka.tcp://flink@flink-workers-pkj7:33723/user/taskmanager_0) at ResourceManager
2019-05-07 08:28:55,638 WARN  org.apache.hadoop.hdfs.DFSClient                              - No live nodes contain block BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 after checking nodes = [], ignoredNodes = null
2019-05-07 08:28:55,638 INFO  org.apache.hadoop.hdfs.DFSClient                              - No node available for BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
2019-05-07 08:28:55,638 INFO  org.apache.hadoop.hdfs.DFSClient                              - Could not obtain BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 from any node:  No live nodes contain current block Block locations: Dead nodes: . Will get new block locations from namenode and retry...
2019-05-07 08:28:55,638 WARN  org.apache.hadoop.hdfs.DFSClient                              - DFS chooseDataNode: got # 2 IOException, will wait for 5085.345757211047 msec.
2019-05-07 08:29:00,726 WARN  org.apache.hadoop.hdfs.DFSClient                              - No live nodes contain block BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 after checking nodes = [], ignoredNodes = null
2019-05-07 08:29:00,726 INFO  org.apache.hadoop.hdfs.DFSClient                              - No node available for BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
2019-05-07 08:29:00,726 INFO  org.apache.hadoop.hdfs.DFSClient                              - Could not obtain BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 from any node:  No live nodes contain current block Block locations: Dead nodes: . Will get new block locations from namenode and retry...
2019-05-07 08:29:00,726 WARN  org.apache.hadoop.hdfs.DFSClient                              - DFS chooseDataNode: got # 3 IOException, will wait for 12600.736247704244 msec.
2019-05-07 08:29:13,331 WARN  org.apache.hadoop.hdfs.DFSClient                              - No live nodes contain block BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 after checking nodes = [], ignoredNodes = null
2019-05-07 08:29:13,331 WARN  org.apache.hadoop.hdfs.DFSClient                              - Could not obtain block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed No live nodes contain current block Block locations: Dead nodes: . Throwing a BlockMissingException
2019-05-07 08:29:13,332 WARN  org.apache.hadoop.hdfs.DFSClient                              - No live nodes contain block BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 after checking nodes = [], ignoredNodes = null
2019-05-07 08:29:13,332 WARN  org.apache.hadoop.hdfs.DFSClient                              - Could not obtain block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed No live nodes contain current block Block locations: Dead nodes: . Throwing a BlockMissingException
2019-05-07 08:29:13,332 WARN  org.apache.hadoop.hdfs.DFSClient                              - DFS Read
org.apache.hadoop.hdfs.BlockMissingException: Could not obtain block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
	at org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
	at org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1036)
	at org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1015)
	at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:647)
	at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:926)
	at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982)
	at java.io.DataInputStream.read(DataInputStream.java:149)
	at org.apache.flink.runtime.fs.hdfs.HadoopDataInputStream.read(HadoopDataInputStream.java:94)
	at java.io.ObjectInputStream$PeekInputStream.read(ObjectInputStream.java:2620)
	at java.io.ObjectInputStream$PeekInputStream.readFully(ObjectInputStream.java:2636)
	at java.io.ObjectInputStream$BlockDataInputStream.readShort(ObjectInputStream.java:3113)
	at java.io.ObjectInputStream.readStreamHeader(ObjectInputStream.java:853)
	at java.io.ObjectInputStream.<init>(ObjectInputStream.java:349)
	at org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.<init>(InstantiationUtil.java:68)
	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:520)
	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:503)
	at org.apache.flink.runtime.state.RetrievableStreamStateHandle.retrieveState(RetrievableStreamStateHandle.java:58)
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:202)
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
2019-05-07 08:29:13,345 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 147dd022ec91f7381ad4ca3d290387e9 from ZooKeeper.
2019-05-07 08:29:13,345 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 9ec31e4d1493c12d550597f8fd6b5839 from ZooKeeper.
2019-05-07 08:29:13,346 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 943460bb4801332cb17658fb9ef83d5c from ZooKeeper.
2019-05-07 08:29:13,346 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 2236d4f760fce9f5d0ad2f53cf6b382d from ZooKeeper.
2019-05-07 08:29:13,346 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 0c853699bb2797addceef0440fa39e51 from ZooKeeper.
2019-05-07 08:29:13,346 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
2019-05-07 08:29:13,346 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Fatal error occurred in the cluster entrypoint.
java.lang.RuntimeException: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state handle is broken. Try cleaning the state handle store.
	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
Caused by: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state handle is broken. Try cleaning the state handle store.
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
	... 9 more
Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
	at org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
	at org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1036)
	at org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1015)
	at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:647)
	at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:926)
	at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982)
	at java.io.DataInputStream.read(DataInputStream.java:149)
	at org.apache.flink.runtime.fs.hdfs.HadoopDataInputStream.read(HadoopDataInputStream.java:94)
	at java.io.ObjectInputStream$PeekInputStream.read(ObjectInputStream.java:2620)
	at java.io.ObjectInputStream$PeekInputStream.readFully(ObjectInputStream.java:2636)
	at java.io.ObjectInputStream$BlockDataInputStream.readShort(ObjectInputStream.java:3113)
	at java.io.ObjectInputStream.readStreamHeader(ObjectInputStream.java:853)
	at java.io.ObjectInputStream.<init>(ObjectInputStream.java:349)
	at org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.<init>(InstantiationUtil.java:68)
	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:520)
	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:503)
	at org.apache.flink.runtime.state.RetrievableStreamStateHandle.retrieveState(RetrievableStreamStateHandle.java:58)
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:202)
	... 14 more
2019-05-07 08:29:13,349 INFO  org.apache.flink.runtime.blob.TransientBlobCache              - Shutting down BLOB cache
2019-05-07 08:29:13,365 INFO  org.apache.flink.runtime.blob.BlobServer                      - Stopped BLOB server at 0.0.0.0:34994

Reply via email to