[
https://issues.apache.org/jira/browse/FLINK-21859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17320252#comment-17320252
]
Till Rohrmann commented on FLINK-21859:
---------------------------------------
Thanks for posting the new logs [~kevin.cyj]. From what I can tell, it looks as
if the {{JobMaster}} rejects the offered slot:
{code}
2021-04-13 19:38:01,455 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Receive slot
request 6bea78f0eeb44c254e8de2ca115206b2 for job
e0c187ce5b5c6e251b33eb800ee0614f from resource manager with leader id
00000000000000000000000000000000.
2021-04-13 19:38:01,455 DEBUG org.apache.flink.runtime.memory.MemoryManager
[] - Initialized MemoryManager with total memory size 1258291200 and
page size 32768.
2021-04-13 19:38:01,455 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Allocated
slot for 6bea78f0eeb44c254e8de2ca115206b2.
2021-04-13 19:38:01,455 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Offer
reserved slots to the leader of job e0c187ce5b5c6e251b33eb800ee0614f.
2021-04-13 19:38:01,542 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Receive slot
request f991e63eea4150a74e83eb515984a89c for job
e0c187ce5b5c6e251b33eb800ee0614f from resource manager with leader id
00000000000000000000000000000000.
2021-04-13 19:38:01,542 DEBUG org.apache.flink.runtime.memory.MemoryManager
[] - Initialized MemoryManager with total memory size 1258291200 and
page size 32768.
2021-04-13 19:38:01,542 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Allocated
slot for f991e63eea4150a74e83eb515984a89c.
2021-04-13 19:38:01,542 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Offer
reserved slots to the leader of job e0c187ce5b5c6e251b33eb800ee0614f.
2021-04-13 19:38:01,579 DEBUG
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Free slot
with allocation id 6bea78f0eeb44c254e8de2ca115206b2 because: The slot was
rejected by the JobManager.
2021-04-13 19:38:01,579 DEBUG
org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl [] - Free slot
TaskSlot(index:4, state:ALLOCATED, resource profile:
ResourceProfile{cpuCores=0.1000000000000000, taskHeapMemory=1.133gb (1216348160
bytes), taskOffHeapMemory=0 bytes, managedMemory=1.172gb (1258291200 bytes),
networkMemory=409.600mb (429496729 bytes)}, allocationId:
6bea78f0eeb44c254e8de2ca115206b2, jobId: e0c187ce5b5c6e251b33eb800ee0614f).
java.lang.Exception: The slot was rejected by the JobManager.
at
org.apache.flink.runtime.taskexecutor.TaskExecutor.lambda$handleAcceptedSlotOffers$18(TaskExecutor.java:1564)
~[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
~[?:1.8.0_102]
at
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
~[?:1.8.0_102]
at
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
~[?:1.8.0_102]
at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:440)
~[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:208)
~[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158)
~[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.actor.ActorCell.invoke(ActorCell.scala:561)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.dispatch.Mailbox.run(Mailbox.scala:225)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at
akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
at
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
[flink-dist_2.11-1.14-vvr-5.0-SNAPSHOT.jar:1.14-vvr-5.0-SNAPSHOT]
{code}
The problem is that the JobManager log does not contain the log statements for
this time (19:38:01). It only starts at 19:38:03 and, thus, it is not very
clear what is happening on the {{JobMaster}} side. But it looks as if the
{{JobMaster}} rejects the slot and still uses it. This sounds like a bug to me.
Not sure whether it is possible to also share the JobManager logs from 19:38:00
- 19:38:03 [~kevin.cyj]. This would be super helpful.
> 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.13.0
> Reporter: Yingjie Cao
> Assignee: Chesnay Schepler
> Priority: Blocker
> Fix For: 1.13.0
>
> Attachments: jm.log.zip, jm.log.zip, tm.log.zip, tm1.log.zip,
> tm2.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)