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

Chesnay Schepler commented on FLINK-15936:
------------------------------------------

The issue can be produced locally by running the test in a loop a few thousand 
times.

Here are some logs:
{code}
484496 [flink-akka.actor.default-dispatcher-6] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Connecting to 
ResourceManager rm(95b0cc9c8e5447b50e53b2084c7e4a15).
484496 [flink-akka.actor.default-dispatcher-6] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Resolved 
ResourceManager address, beginning registration
484496 [flink-akka.actor.default-dispatcher-6] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Registration at 
ResourceManager attempt 1 (timeout=100ms)
484496 [main] DEBUG org.apache.flink.runtime.memory.MemoryManager [] - 
Initialized MemoryManager with total memory size 40960 ({OFF_HEAP=40960}), page 
size 4096.
484496 [main] DEBUG org.apache.flink.runtime.memory.MemoryManager [] - 
Initialized MemoryManager with total memory size 40960 ({OFF_HEAP=40960}), page 
size 4096.
484496 [main] INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService [] - 
Add job 4022e64d9f9ffc91fa2ab18d2893fcd5 for job leader monitoring.
484496 [main] DEBUG org.apache.flink.runtime.taskexecutor.JobLeaderService [] - 
New leader information for job 4022e64d9f9ffc91fa2ab18d2893fcd5. Address: jm, 
leader id: a7b139b06d7c488ea9662b61850a4449.
484496 [main] INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService [] - 
Try to register at job manager jm with leader id 
a9662b61-850a-4449-a7b1-39b06d7c488e.
484496 [flink-akka.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.taskexecutor.JobLeaderService [] - Resolved JobManager 
address, beginning registration
484496 [flink-akka.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.taskexecutor.JobLeaderService [] - Registration at 
JobManager attempt 1 (timeout=100ms)
484496 [flink-akka.actor.default-dispatcher-3] INFO  
org.apache.flink.runtime.taskexecutor.JobLeaderService [] - Successful 
registration at job manager jm for job 4022e64d9f9ffc91fa2ab18d2893fcd5.
484496 [flink-akka.actor.default-dispatcher-2] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Establish JobManager 
connection for job 4022e64d9f9ffc91fa2ab18d2893fcd5.
484496 [flink-akka.actor.default-dispatcher-2] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Offer reserved slots to 
the leader of job 4022e64d9f9ffc91fa2ab18d2893fcd5.
484496 [flink-akka.actor.default-dispatcher-2] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Successful registration 
at resource manager rm under registration id 2f01db6b79bb432974004e65a8159f8e.
484496 [flink-akka.actor.default-dispatcher-2] INFO  
org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl [] - Activate slot 
ea290b9891ef70052ab894ce5c3b94ac.
484496 [flink-akka.actor.default-dispatcher-2] DEBUG 
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Free slot with 
allocation id 7704106cbb7c4c66aa93aec9cde0fb3b because: The slot was rejected 
by the JobManager.
484496 [flink-akka.actor.default-dispatcher-2] DEBUG 
org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl [] - Free slot 
TaskSlot(index:1, state:ALLOCATED, resource profile: 
ResourceProfile{cpuCores=1.0, taskHeapMemory=100,000kb (102400 bytes), 
taskOffHeapMemory=0 bytes, managedMemory=40,000kb (40960 bytes), 
networkMemory=100,000kb (102400 bytes)}, allocationId: 
7704106cbb7c4c66aa93aec9cde0fb3b, jobId: 4022e64d9f9ffc91fa2ab18d2893fcd5).
java.lang.Exception: The slot was rejected by the JobManager.
        at 
org.apache.flink.runtime.taskexecutor.TaskExecutor.lambda$handleAcceptedSlotOffers$10(TaskExecutor.java:1208)
 ~[classes/:?]
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
 ~[?:?]
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
 ~[?:?]
        at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
 ~[?:?]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)
 ~[classes/:?]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)
 ~[classes/:?]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
 ~[classes/:?]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123) 
[scala-library-2.11.12.jar:?]
        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170) 
[scala-library-2.11.12.jar:?]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
[scala-library-2.11.12.jar:?]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
[scala-library-2.11.12.jar:?]
        at akka.actor.Actor$class.aroundReceive(Actor.scala:517) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.actor.ActorCell.invoke(ActorCell.scala:561) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.Mailbox.run(Mailbox.scala:225) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.Mailbox.exec(Mailbox.scala:235) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at 
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at 
akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at 
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
484496 [flink-akka.actor.default-dispatcher-2] DEBUG 
org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager [] - 
Releasing local state under allocation id 7704106cbb7c4c66aa93aec9cde0fb3b.
485501 [flink-akka.actor.default-dispatcher-10] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - The heartbeat of 
JobManager with id jm timed out.
485501 [flink-akka.actor.default-dispatcher-10] DEBUG 
org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Close JobManager 
connection for job 4022e64d9f9ffc91fa2ab18d2893fcd5.
{code}

> TaskExecutorTest#testSlotAcceptance deadlocks
> ---------------------------------------------
>
>                 Key: FLINK-15936
>                 URL: https://issues.apache.org/jira/browse/FLINK-15936
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination, Tests
>    Affects Versions: 1.10.0
>            Reporter: Gary Yao
>            Priority: Critical
>              Labels: test-stability
>             Fix For: 1.10.1
>
>
> https://api.travis-ci.org/v3/job/646510877/log.txt
> {noformat}
> "main" #1 prio=5 os_prio=0 tid=0x00007f2f5800b800 nid=0x499 waiting on 
> condition [0x00007f2f61733000]
>    java.lang.Thread.State: WAITING (parking)
>       at sun.misc.Unsafe.park(Native Method)
>       - parking to wait for  <0x000000008669b3a8> (a 
> java.util.concurrent.CompletableFuture$Signaller)
>       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>       at 
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
>       at 
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
>       at 
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
>       at 
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
>       at 
> org.apache.flink.runtime.taskexecutor.TaskExecutorTest.testSlotAcceptance(TaskExecutorTest.java:875)
>       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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>       at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>       at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
>       at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
>       at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
>       at org.junit.rules.RunRules.evaluate(RunRules.java:20)
>       at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
>       at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
>       at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:365)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:273)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:238)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:159)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
> {noformat}



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

Reply via email to