[ 
https://issues.apache.org/jira/browse/FLINK-22574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Robert Metzger closed FLINK-22574.
----------------------------------
    Fix Version/s: 1.14.0
       Resolution: Fixed

> Adaptive Scheduler: Can not cancel restarting job
> -------------------------------------------------
>
>                 Key: FLINK-22574
>                 URL: https://issues.apache.org/jira/browse/FLINK-22574
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.13.0
>            Reporter: Robert Metzger
>            Assignee: Robert Metzger
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.14.0, 1.13.1
>
>
> I have a job in state RESTARTING. When I now issue a cancel RPC call, I get 
> the following exception:
> Relevant trace:
> {code}
> Caused by: java.lang.IllegalStateException: Assuming running execution graph 
>  at org.apache.flink.util.Preconditions.checkState(Preconditions.java:193) 
>  at 
> org.apache.flink.runtime.scheduler.adaptive.StateWithExecutionGraph.(StateWithExecutionGraph.java:94)
>  
>  at org.apache.flink.runtime.scheduler.adaptive.Canceling.(Canceling.java:41) 
>  at 
> org.apache.flink.runtime.scheduler.adaptive.Canceling$Factory.getState(Canceling.java:98)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.Canceling$Factory.getState(Canceling.java:72)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.transitionToState(AdaptiveScheduler.java:1128)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.goToCanceling(AdaptiveScheduler.java:802)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.Restarting.cancel(Restarting.java:74)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.cancel(AdaptiveScheduler.java:453)
>  
>  at org.apache.flink.runtime.jobmaster.JobMaster.cancel(JobMaster.java:417) 
> {code}
> Full trace as reported in the UI:
> {code}
> org.apache.flink.runtime.rest.handler.RestHandlerException: Job cancellation 
> failed: Cancellation failed. 
>  at 
> org.apache.flink.runtime.rest.handler.job.JobCancellationHandler.lambda$handleRequest$0(JobCancellationHandler.java:127)
>  
>  at 
> java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:836) 
>  at 
> java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:811)
>  
>  at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>  
>  at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
>  
>  at 
> org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$0(AkkaInvocationHandler.java:234)
>  
>  at 
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
>  
>  at 
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
>  
>  at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>  
>  at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
>  
>  at 
> org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:1079)
>  
>  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:60) 
>  at 
> org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:73)
>  
>  at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68) 
>  at 
> scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:284)
>  
>  at 
> scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:284)
>  
>  at 
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:284) 
>  at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:573) 
>  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(Future.scala:532) 
>  at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:29) 
>  at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:29) 
>  at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60) 
>  at 
> akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
>  
>  at 
> akka.dispatch.BatchingExecutor$BlockableBatch.$anonfun$run$1(BatchingExecutor.scala:91)
>  
>  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12) 
>  at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:81) 
>  at 
> akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:91) 
>  at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) 
>  at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
>  
>  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: org.apache.flink.util.FlinkException: Cancellation failed. 
>  at 
> org.apache.flink.runtime.jobmaster.JobMasterServiceLeadershipRunner.lambda$cancel$3(JobMasterServiceLeadershipRunner.java:197)
>  
>  at 
> java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:884)
>  
>  at 
> java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:866)
>  
>  at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>  
>  at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
>  
>  at 
> org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$0(AkkaInvocationHandler.java:234)
>  
>  at 
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
>  
>  at 
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
>  
>  at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>  
>  at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
>  
>  at 
> org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:1079)
>  
>  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:60) 
>  at 
> org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:73)
>  
>  at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68) 
>  at 
> scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:284)
>  
>  at 
> scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:284)
>  
>  at 
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:284) 
>  at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:573) 
>  at akka.actor.ActorRef.tell(ActorRef.scala:126) 
>  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:311)
>  
>  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212)
>  
>  at 
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77)
>  
>  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158)
>  
>  at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) 
>  at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) 
>  at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) 
>  at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) 
>  at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) 
>  at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
>  at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
>  at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
>  at akka.actor.Actor.aroundReceive(Actor.scala:517) 
>  at akka.actor.Actor.aroundReceive$(Actor.scala:515) 
>  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: 
> java.lang.IllegalStateException: Assuming running execution graph 
>  at org.apache.flink.util.Preconditions.checkState(Preconditions.java:193) 
>  at 
> org.apache.flink.runtime.scheduler.adaptive.StateWithExecutionGraph.(StateWithExecutionGraph.java:94)
>  
>  at org.apache.flink.runtime.scheduler.adaptive.Canceling.(Canceling.java:41) 
>  at 
> org.apache.flink.runtime.scheduler.adaptive.Canceling$Factory.getState(Canceling.java:98)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.Canceling$Factory.getState(Canceling.java:72)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.transitionToState(AdaptiveScheduler.java:1128)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.goToCanceling(AdaptiveScheduler.java:802)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.Restarting.cancel(Restarting.java:74)
>  
>  at 
> org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.cancel(AdaptiveScheduler.java:453)
>  
>  at org.apache.flink.runtime.jobmaster.JobMaster.cancel(JobMaster.java:417) 
>  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.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:305)
>  ... 23 more
> {code}
> (Sorry for the poor stack trace formatting, I coped the stack trace from the 
> UI)
> It seems that the Canceling state assumes we only transition into it from 
> Executing (ExecutionGraph = RUNNING).
> In my opinion a job should be cancellable at all times, for example when a 
> job is stuck in a restart loop, cancelling is the only way out (unless 
> retries are exhausted).



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

Reply via email to