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

Till Rohrmann commented on FLINK-21859:
---------------------------------------

What I cannot really explain is why we immediately after offering the slots 
free 2 of the 3 offered slots:

{code}
2021-03-31 15:13:15,852 INFO  
org.apache.flink.runtime.taskexecutor.DefaultJobLeaderService [] - Successful 
registration at job manager 
akka.tcp://fl...@k28a13215.eu95sqa.tbsite.net:56822/user/rpc/jobmanager_4 for 
job 3c97aeb7d225871ce2a453788c686486.
2021-03-31 15:13:15,852 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Establish 
JobManager connection for job 3c97aeb7d225871ce2a453788c686486.
2021-03-31 15:13:15,853 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Offer 
reserved slots to the leader of job 3c97aeb7d225871ce2a453788c686486.
2021-03-31 15:13:15,907 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Receive slot 
request a758efc7798f2424d9b96c1d77cc2c82 for job 
3c97aeb7d225871ce2a453788c686486 from resource manager with leader id 
00000000000000000000000000000000.
2021-03-31 15:13:15,908 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Allocated 
slot for a758efc7798f2424d9b96c1d77cc2c82.
2021-03-31 15:13:15,908 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Offer 
reserved slots to the leader of job 3c97aeb7d225871ce2a453788c686486.
2021-03-31 15:13:15,923 INFO  
org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl [] - Free slot 
TaskSlot(index:4, state:ALLOCATED, resource profile: 
ResourceProfile{cpuCores=0.1000000000000000, taskHeapMemory=1.183gb (1270035251 
bytes), taskOffHeapMemory=0 bytes, managedMemory=1.172gb (1258291200 bytes), 
networkMemory=409.600mb (429496729 bytes)}, allocationId: 
bee6ef66e651bd6a29cafd61cb74dd10, jobId: 3c97aeb7d225871ce2a453788c686486).
2021-03-31 15:13:15,923 INFO  
org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl [] - Free slot 
TaskSlot(index:9, state:ALLOCATED, resource profile: 
ResourceProfile{cpuCores=0.1000000000000000, taskHeapMemory=1.183gb (1270035251 
bytes), taskOffHeapMemory=0 bytes, managedMemory=1.172gb (1258291200 bytes), 
networkMemory=409.600mb (429496729 bytes)}, allocationId: 
9b0e7ead445183e9d7f0536872da8602, jobId: 3c97aeb7d225871ce2a453788c686486).
{code}

> Batch job fails due to "Could not mark slot 61a637e3977c58a0e6b73533c419297d 
> active"
> ------------------------------------------------------------------------------------
>
>                 Key: FLINK-21859
>                 URL: https://issues.apache.org/jira/browse/FLINK-21859
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.12.0, 1.13.0
>            Reporter: Yingjie Cao
>            Priority: Major
>             Fix For: 1.13.0
>
>         Attachments: jm.log.zip, tm.log.zip
>
>
> Here is the error stack:
> {code:java}
> 2021-03-18 19:05:31org.apache.flink.runtime.JobException: Recovery is 
> suppressed by NoRestartBackoffTimeStrategy   
>    at 
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:130
>  undefined)   
>    at 
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:81
>  undefined)   
>    at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:221
>  undefined)   
>    at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:212
>  undefined)   
>    at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:203
>  undefined)   
>    at 
> org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:701
>  undefined)   
>    at 
> org.apache.flink.runtime.scheduler.UpdateSchedulerNgOnInternalFailuresListener.notifyTaskFailure(UpdateSchedulerNgOnInternalFailuresListener.java:51
>  undefined)   
>    at 
> org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.notifySchedulerNgAboutInternalTaskFailure(DefaultExecutionGraph.java:1449
>  undefined)   
>    at 
> org.apache.flink.runtime.executiongraph.Execution.processFail(Execution.java:1105
>  undefined)   
>    at 
> org.apache.flink.runtime.executiongraph.Execution.processFail(Execution.java:1045
>  undefined)   
>    at 
> org.apache.flink.runtime.executiongraph.Execution.fail(Execution.java:754 
> undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot.signalPayloadRelease(SingleLogicalSlot.java:195
>  undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot.release(SingleLogicalSlot.java:182
>  undefined)   
>    at 
> org.apache.flink.runtime.scheduler.SharedSlot.lambda$release$4(SharedSlot.java:271
>  undefined)   
>    at 
> java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:656 
> undefined)   
>    at 
> java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:669
>  undefined)   
>    at 
> java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:1997 
> undefined)   
>    at 
> org.apache.flink.runtime.scheduler.SharedSlot.release(SharedSlot.java:271 
> undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.AllocatedSlot.releasePayload(AllocatedSlot.java:152
>  undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.DefaultDeclarativeSlotPool.releasePayload(DefaultDeclarativeSlotPool.java:385
>  undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.DefaultDeclarativeSlotPool.lambda$releaseSlot$1(DefaultDeclarativeSlotPool.java:376
>  undefined)   
>    at java.util.Optional.ifPresent(Optional.java:159 undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.DefaultDeclarativeSlotPool.releaseSlot(DefaultDeclarativeSlotPool.java:374
>  undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.slotpool.DeclarativeSlotPoolService.failAllocation(DeclarativeSlotPoolService.java:198
>  undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.JobMaster.internalFailAllocation(JobMaster.java:650
>  undefined)   
>    at 
> org.apache.flink.runtime.jobmaster.JobMaster.failSlot(JobMaster.java:636 
> undefined)   
>    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)   
>    at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62 
> undefined)   
>    at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43
>  undefined)   
>    at java.lang.reflect.Method.invoke(Method.java:498 undefined)   
>    at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:301
>  undefined)   
>    at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212
>  undefined)   
>    at 
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77
>  undefined)   
>    at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158
>  undefined)   
>    at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26 undefined) 
>   
>    at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21 undefined) 
>   
>    at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123 
> undefined)   
>    at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21 
> undefined)   
>    at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170 
> undefined)   
>    at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171 
> undefined)   
>    at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171 
> undefined)   
>    at akka.actor.Actor$class.aroundReceive(Actor.scala:517 undefined)   
>    at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225 
> undefined)   
>    at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592 undefined)   
>    at akka.actor.ActorCell.invoke(ActorCell.scala:561 undefined)   
>    at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258 undefined)   
>    at akka.dispatch.Mailbox.run(Mailbox.scala:225 undefined)   
>    at akka.dispatch.Mailbox.exec(Mailbox.scala:235 undefined)   
>    at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260 
> undefined)   
>    at 
> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339 
> undefined)   
>    at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979 
> undefined)   
>    at 
> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107 
> undefined)
> Caused by: org.apache.flink.util.FlinkException: Could not mark slot 
> 61a637e3977c58a0e6b73533c419297d active.   
>    at 
> org.apache.flink.runtime.taskexecutor.TaskExecutor.lambda$handleAcceptedSlotOffers$18(TaskExecutor.java:1469
>  undefined)   
>    at 
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760
>  undefined)   
>    at 
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736
>  undefined)   
>    at 
> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442
>  undefined)   
>    at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:440
>  undefined)   
>    at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:208
>  undefined)    ... 19 more
> {code}



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

Reply via email to