You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Zhu Zhu (Jira)" <ji...@apache.org> on 2020/01/09 07:53:00 UTC

[jira] [Comment Edited] (FLINK-15522) Misleading root cause exception when cancelling the job

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

Zhu Zhu edited comment on FLINK-15522 at 1/9/20 7:52 AM:
---------------------------------------------------------

I think the reason why this happens is that in {{JobResult#createFrom}} would set the latest global failure cause of the ExecutionGraph to be the cause of the {{JobResult}} if the job is terminated but not FINISHED. 
So if the job is CANCELED and any global failure had ever happened, the latest global failure cause would be shown as the root cause of the canceling.
I think in the canceling case, we can just keep JobResult#serializedThrowable to be null so that the root cause would simple be {{JobCancellationException: Job was cancelled.}}.



was (Author: zhuzh):
I think the reason why this happens is that in {{JobResult#createFrom}} would set the latest global failure cause of the ExecutionGraph to be the cause of the {{JobResult}} if the job is terminated but not FINISHED. 
So if the job is CANCELED and any global failure had ever happened, the latest global failure cause would be shown as the root cause of the canceling.


> Misleading root cause exception when cancelling the job
> -------------------------------------------------------
>
>                 Key: FLINK-15522
>                 URL: https://issues.apache.org/jira/browse/FLINK-15522
>             Project: Flink
>          Issue Type: Bug
>          Components: Client / Job Submission, Runtime / Coordination
>    Affects Versions: 1.10.0, 1.11.0
>            Reporter: Till Rohrmann
>            Priority: Critical
>
> When cancelling a Flink job, the following stack trace gets displayed
> {code}
>  The program finished with the following exception:
> org.apache.flink.client.program.ProgramInvocationException: Job failed. (JobID: d0e8c2026709385166bcc0253c30742e)
>         at org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:262)
>         at org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:338)
>         at org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:60)
>         at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1507)
>         at org.apache.flink.streaming.examples.statemachine.StateMachineExample.main(StateMachineExample.java:142)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:576)
>         at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:438)
>         at org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:274)
>         at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:746)
>         at org.apache.flink.client.cli.CliFrontend.runProgram(CliFrontend.java:273)
>         at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:205)
>         at org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:1010)
>         at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1083)
>         at org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
>         at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1083)
> Caused by: org.apache.flink.runtime.client.JobCancellationException: Job was cancelled.
>         at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:148)
>         at org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:259)
>         ... 18 more
> Caused by: org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: No pooled slot available and request to ResourceManager for new slot failed
>         at org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.slotRequestToResourceManagerFailed(SlotPoolImpl.java:357)
>         at org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.lambda$requestSlotFromResourceManager$1(SlotPoolImpl.java:345)
>         at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
>         at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
>         at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>         at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)
>         at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)
>         at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>         at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
>         at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
>         at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
>         at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
>         at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
>         at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
>         at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>         at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>         at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
>         at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
>         at akka.actor.ActorCell.invoke(ActorCell.scala:561)
>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
>         at akka.dispatch.Mailbox.run(Mailbox.scala:225)
>         at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
>         at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>         at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>         at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>         at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: java.util.concurrent.CompletionException: org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException: Could not fulfill slot request 7ab196daeb73e353c460455899a7622f.
>         at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>         at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>         at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>         at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>         at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
>         at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
>         at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:871)
>         at akka.dispatch.OnComplete.internal(Future.scala:263)
>         at akka.dispatch.OnComplete.internal(Future.scala:261)
>         at akka.dispatch.japi$CallbackBridge.apply(Future.scala:191)
>         at akka.dispatch.japi$CallbackBridge.apply(Future.scala:188)
>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
>         at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:74)
>         at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>         at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>         at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:572)
>         at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:23)
>         at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:21)
>         at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:436)
>         at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:435)
>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
>         at akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
>         at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
>         at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
>         at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
>         at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>         at akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
>         at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>         at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
>         ... 4 more
> Caused by: org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException: Could not fulfill slot request 7ab196daeb73e353c460455899a7622f.
>         at org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl.registerSlotRequest(SlotManagerImpl.java:315)
>         at org.apache.flink.runtime.resourcemanager.ResourceManager.requestSlot(ResourceManager.java:443)
>         at sun.reflect.GeneratedMethodAccessor87.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:279)
>         at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:194)
>         at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>         at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
>         at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
>         at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
>         at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
>         at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
>         at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
>         at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>         at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>         at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
>         at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
>         at akka.actor.ActorCell.invoke(ActorCell.scala:561)
>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
>         at akka.dispatch.Mailbox.run(Mailbox.scala:225)
>         at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
>         ... 4 more
> Caused by: org.apache.flink.runtime.resourcemanager.exceptions.UnfulfillableSlotRequestException: Could not fulfill slot request 7ab196daeb73e353c460455899a7622f. Requested resource profile (ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1}) is unfulfillable.
>         at org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl.internalRequestSlot(SlotManagerImpl.java:768)
>         at org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl.registerSlotRequest(SlotManagerImpl.java:310)
>         ... 26 more
> {code}
> The reported root cause is {{UnfulfillableSlotRequestException}} which is a bit misleading. If the user cancels a running job then the root cause should be the cancellation.



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