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

JieFang.He commented on FLINK-19067:
------------------------------------

[~rmetzger]

After testing again, this should have nothing to do with standby JobManager, 
When it happens,all node submit job fail. The last time it failed only on the 
Standby node seems to be accidental. 

Also,  Not every time you submit a job will fail, failure is probabilistic.

The log shows that there is no blob file on the server, but the wordcount 
example seems no need for these files, the successful job did not generate 
these files.

My resource_manager_lock is on Standby JobManager, Others on the Active 
JobManager, I don't know if this has any impact.
{code:java}
2020-09-03 19:15:11,566 INFO [main-SendThread(-deployer-hejiefang03:2181)] 
[ClientCnxn]: Socket connection established, initiating session, client: 
/172.17.0.8:45164, server: -deployer-hejiefang03/172.17.0.9:2181
2020-09-03 19:15:11,573 INFO [main-SendThread(-deployer-hejiefang03:2181)] 
[ClientCnxn]: Session establishment complete on server 
-deployer-hejiefang03/172.17.0.9:2181, sessionid = 0x3000e154ba9000a, 
negotiated timeout = 60000
2020-09-03 19:15:11,573 INFO [main-EventThread] [ConnectionStateManager]: State 
change: CONNECTED
2020-09-03 19:15:11,578 INFO [main-EventThread] [EnsembleTracker]: New config 
event received: {server.1=-deployer-hejiefang01:2888:3888:participant, 
version=0, server.3=-deployer-hejiefang03:2888:3888:participant, 
server.2=-deployer-hejiefang02:2888:3888:participant}
2020-09-03 19:15:11,578 ERROR [main-EventThread] [EnsembleTracker]: Invalid 
config event received: {server.1=-deployer-hejiefang01:2888:3888:participant, 
version=0, server.3=-deployer-hejiefang03:2888:3888:participant, 
server.2=-deployer-hejiefang02:2888:3888:participant}
2020-09-03 19:15:11,579 INFO [main-EventThread] [EnsembleTracker]: New config 
event received: {server.1=-deployer-hejiefang01:2888:3888:participant, 
version=0, server.3=-deployer-hejiefang03:2888:3888:participant, 
server.2=-deployer-hejiefang02:2888:3888:participant}
2020-09-03 19:15:11,579 ERROR [main-EventThread] [EnsembleTracker]: Invalid 
config event received: {server.1=-deployer-hejiefang01:2888:3888:participant, 
version=0, server.3=-deployer-hejiefang03:2888:3888:participant, 
server.2=-deployer-hejiefang02:2888:3888:participant}
2020-09-03 19:15:11,668 INFO [Curator-Framework-0] [CuratorFrameworkImpl]: 
backgroundOperationsLoop exiting
2020-09-03 19:15:11,772 INFO [ForkJoinPool.commonPool-worker-57] [ZooKeeper]: 
Session: 0x3000e154ba9000a closed
2020-09-03 19:15:11,772 INFO [main-EventThread] [ClientCnxn]: EventThread shut 
down for session: 0x3000e154ba9000a
2020-09-03 19:15:11,775 ERROR [main] [CliFrontend]: Error while running the 
command.
org.apache.flink.client.program.ProgramInvocationException: The main method 
caused an error: org.apache.flink.client.program.ProgramInvocationException: 
Job failed (JobID: 8b242015d603f46e82a5f1299399b669)
        at 
org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:302)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:198)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:149) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:699) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:232) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:916) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:992) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at java.security.AccessController.doPrivileged(Native Method) 
~[?:1.8.0_163]
        at javax.security.auth.Subject.doAs(Subject.java:422) [?:1.8.0_163]
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730)
 [flink-shaded-hadoop-3-uber-3.2.1-11.0.jar:?]
        at 
org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
 [flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:992) 
[flink-dist_2.11-1.11.1.jar:1.11.1]
Caused by: java.util.concurrent.ExecutionException: 
org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 
8b242015d603f46e82a5f1299399b669)
        at 
java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895) 
~[?:1.8.0_163]
        at 
org.apache.flink.client.program.ContextEnvironment.getJobExecutionResult(ContextEnvironment.java:112)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.program.ContextEnvironment.execute(ContextEnvironment.java:76)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.api.java.ExecutionEnvironment.execute(ExecutionEnvironment.java:873)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.api.java.DataSet.collect(DataSet.java:413) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.api.java.DataSet.print(DataSet.java:1652) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.examples.java.wordcount.WordCount.main(WordCount.java:96) 
~[?:?]
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
~[?:1.8.0_163]
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
~[?:1.8.0_163]
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 ~[?:1.8.0_163]
        at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_163]
        at 
org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:288)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        ... 11 more
Caused by: org.apache.flink.client.program.ProgramInvocationException: Job 
failed (JobID: 8b242015d603f46e82a5f1299399b669)
        at 
org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:116)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962) 
~[?:1.8.0_163]
        at 
org.apache.flink.client.program.rest.RestClusterClient.lambda$pollResourceAsync$22(RestClusterClient.java:602)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962) 
~[?:1.8.0_163]
        at 
org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$8(FutureUtils.java:309)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:561) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:929)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[?:1.8.0_163]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
~[?:1.8.0_163]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_163]
Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution 
failed.
        at 
org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:147)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:114)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962) 
~[?:1.8.0_163]
        at 
org.apache.flink.client.program.rest.RestClusterClient.lambda$pollResourceAsync$22(RestClusterClient.java:602)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962) 
~[?:1.8.0_163]
        at 
org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$8(FutureUtils.java:309)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:561) 
~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:929)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
 ~[?:1.8.0_163]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[?:1.8.0_163]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
~[?:1.8.0_163]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_163]
Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by 
NoRestartBackoffTimeStrategy
        at 
org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:116)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:78)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:192)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:185)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:179)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:503)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:386)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
~[?:1.8.0_163]
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
~[?:1.8.0_163]
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 ~[?:1.8.0_163]
        at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_163]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:284)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:199)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.actor.Actor$class.aroundReceive(Actor.scala:517) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.actor.ActorCell.invoke(ActorCell.scala:561) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.dispatch.Mailbox.run(Mailbox.scala:225) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.dispatch.Mailbox.exec(Mailbox.scala:235) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
Caused by: java.io.IOException: Failed to fetch BLOB 
8b242015d603f46e82a5f1299399b669/p-02e225df937959b03b8d26238f027e81191d1e2a-1535900a0910d61d04f56133aceb60c1
 from -deployer-hejiefang01/172.17.0.8:30419 and store it under 
/data2/flink/tmp/blobStore-caaf1bc1-3d37-4e69-9ecd-b6cf51974c73/incoming/temp-00000001
        at 
org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:169)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.AbstractBlobCache.getFileInternal(AbstractBlobCache.java:185)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.PermanentBlobCache.getFile(PermanentBlobCache.java:202)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.createUserCodeClassLoader(BlobLibraryCacheManager.java:234)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.getOrResolveClassLoader(BlobLibraryCacheManager.java:218)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.access$1200(BlobLibraryCacheManager.java:193)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$DefaultClassLoaderLease.getOrResolveClassLoader(BlobLibraryCacheManager.java:312)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.taskmanager.Task.createUserCodeClassloader(Task.java:929)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:609) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_163]
Caused by: java.io.IOException: GET operation failed: Server side error: 
/data2/flink/storageDir/default/blob/job_8b242015d603f46e82a5f1299399b669/blob_p-02e225df937959b03b8d26238f027e81191d1e2a-1535900a0910d61d04f56133aceb60c1
 (No such file or directory)
        at 
org.apache.flink.runtime.blob.BlobClient.getInternal(BlobClient.java:231) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:144)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.AbstractBlobCache.getFileInternal(AbstractBlobCache.java:185)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.PermanentBlobCache.getFile(PermanentBlobCache.java:202)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.createUserCodeClassLoader(BlobLibraryCacheManager.java:234)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.getOrResolveClassLoader(BlobLibraryCacheManager.java:218)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.access$1200(BlobLibraryCacheManager.java:193)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$DefaultClassLoaderLease.getOrResolveClassLoader(BlobLibraryCacheManager.java:312)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.taskmanager.Task.createUserCodeClassloader(Task.java:929)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:609) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_163]
Caused by: java.io.IOException: Server side error: 
/data2/flink/storageDir/default/blob/job_8b242015d603f46e82a5f1299399b669/blob_p-02e225df937959b03b8d26238f027e81191d1e2a-1535900a0910d61d04f56133aceb60c1
 (No such file or directory)
        at 
org.apache.flink.runtime.blob.BlobClient.receiveAndCheckGetResponse(BlobClient.java:284)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.BlobClient.getInternal(BlobClient.java:225) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:144)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.AbstractBlobCache.getFileInternal(AbstractBlobCache.java:185)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.PermanentBlobCache.getFile(PermanentBlobCache.java:202)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.createUserCodeClassLoader(BlobLibraryCacheManager.java:234)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.getOrResolveClassLoader(BlobLibraryCacheManager.java:218)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$LibraryCacheEntry.access$1200(BlobLibraryCacheManager.java:193)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager$DefaultClassLoaderLease.getOrResolveClassLoader(BlobLibraryCacheManager.java:312)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.taskmanager.Task.createUserCodeClassloader(Task.java:929)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:609) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_163]
Caused by: java.io.FileNotFoundException: 
/data2/flink/storageDir/default/blob/job_8b242015d603f46e82a5f1299399b669/blob_p-02e225df937959b03b8d26238f027e81191d1e2a-1535900a0910d61d04f56133aceb60c1
 (No such file or directory)
        at java.io.FileInputStream.open0(Native Method) ~[?:1.8.0_163]
        at java.io.FileInputStream.open(FileInputStream.java:195) ~[?:1.8.0_163]
        at java.io.FileInputStream.<init>(FileInputStream.java:138) 
~[?:1.8.0_163]
        at 
org.apache.flink.core.fs.local.LocalDataInputStream.<init>(LocalDataInputStream.java:50)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.core.fs.local.LocalFileSystem.open(LocalFileSystem.java:143) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.FileSystemBlobStore.get(FileSystemBlobStore.java:105)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.FileSystemBlobStore.get(FileSystemBlobStore.java:87)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.BlobServer.getFileInternal(BlobServer.java:501) 
~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.BlobServerConnection.get(BlobServerConnection.java:231)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
        at 
org.apache.flink.runtime.blob.BlobServerConnection.run(BlobServerConnection.java:117)
 ~[flink-dist_2.11-1.11.1.jar:1.11.1]
{code}
 

> FileNotFoundException when run flink examples on standby JobManager
> -------------------------------------------------------------------
>
>                 Key: FLINK-19067
>                 URL: https://issues.apache.org/jira/browse/FLINK-19067
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.11.1
>            Reporter: JieFang.He
>            Priority: Major
>
> 1、When run examples/batch/WordCount.jar on standby JobManager,it will fail 
> with the exception:
> Caused by: java.io.FileNotFoundException: 
> /data2/storageDir/default/blob/job_d29414828f614d5466e239be4d3889ac/blob_p-a2ebe1c5aa160595f214b4bd0f39d80e42ee2e93-f458f1c12dc023e78d25f191de1d7c4b
>  (No such file or directory)
>  at java.io.FileInputStream.open0(Native Method)
>  at java.io.FileInputStream.open(FileInputStream.java:195)
>  at java.io.FileInputStream.<init>(FileInputStream.java:138)
>  at 
> org.apache.flink.core.fs.local.LocalDataInputStream.<init>(LocalDataInputStream.java:50)
>  at 
> org.apache.flink.core.fs.local.LocalFileSystem.open(LocalFileSystem.java:143)
>  at 
> org.apache.flink.runtime.blob.FileSystemBlobStore.get(FileSystemBlobStore.java:105)
>  at 
> org.apache.flink.runtime.blob.FileSystemBlobStore.get(FileSystemBlobStore.java:87)
>  at 
> org.apache.flink.runtime.blob.BlobServer.getFileInternal(BlobServer.java:501)
>  at 
> org.apache.flink.runtime.blob.BlobServerConnection.get(BlobServerConnection.java:231)
>  at 
> org.apache.flink.runtime.blob.BlobServerConnection.run(BlobServerConnection.java:117)
>  
> 2、Run examples success on other nodes
> 3、After run success on the other node, it can run success on the Standby 
> JobManager. But run again will fail
>  



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

Reply via email to