flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-7279) MiniCluster can deadlock at shut down
Date Fri, 28 Jul 2017 15:02:01 GMT

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

ASF GitHub Bot commented on FLINK-7279:
---------------------------------------

Github user NicoK commented on the issue:

    https://github.com/apache/flink/pull/4416
  
    A simple asynchronous call as in `TaskExecutor#onFatalErrorAsync()` is not enough though
because this is what is actually already done and led to me finding this error in the first
place. Please see the stack traces of the incriminating processes below:
    
    ```
    "flink-akka.actor.default-dispatcher-8" #31 prio=5 os_prio=0 tid=0x00007ffa00efe800 nid=0xb8c
waiting for monitor entry [0x00007ff9ee540000]
       java.lang.Thread.State: BLOCKED (on object monitor)
    	at org.apache.flink.runtime.minicluster.MiniCluster$TerminatingFatalErrorHandler.onFatalError(MiniCluster.java:652)
    	- waiting to lock <0x00000000aad1d2d8> (a java.lang.Object)
    	at org.apache.flink.runtime.taskexecutor.TaskExecutor.onFatalError(TaskExecutor.java:1129)
    	at org.apache.flink.runtime.taskexecutor.TaskExecutor$7.run(TaskExecutor.java:1116)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:278)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:132)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.access$000(AkkaRpcActor.java:73)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor$1.apply(AkkaRpcActor.java:111)
    	at akka.actor.ActorCell$$anonfun$become$1.applyOrElse(ActorCell.scala:534)
    	at akka.actor.Actor$class.aroundReceive(Actor.scala:467)
    	at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:97)
    	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
    	at akka.actor.ActorCell.invoke(ActorCell.scala:487)
    	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
    	at akka.dispatch.Mailbox.run(Mailbox.scala:220)
    	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
    	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
    	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
    	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
    	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
    
    "main" #1 prio=5 os_prio=0 tid=0x00007ffa0000a000 nid=0xb56 waiting on condition [0x00007ffa07fa1000]
       java.lang.Thread.State: WAITING (parking)
    	at sun.misc.Unsafe.park(Native Method)
    	- parking to wait for  <0x00000000ab269c30> (a java.util.concurrent.CountDownLatch$Sync)
    	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
    	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
    	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
    	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
    	at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
    	at akka.actor.ActorSystemImpl$TerminationCallbacks.ready(ActorSystem.scala:819)
    	at akka.actor.ActorSystemImpl$TerminationCallbacks.ready(ActorSystem.scala:788)
    	at scala.concurrent.Await$$anonfun$ready$1.apply(package.scala:169)
    	at scala.concurrent.Await$$anonfun$ready$1.apply(package.scala:169)
    	at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
    	at scala.concurrent.Await$.ready(package.scala:169)
    	at akka.actor.ActorSystemImpl.awaitTermination(ActorSystem.scala:644)
    	at akka.actor.ActorSystemImpl.awaitTermination(ActorSystem.scala:645)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcService.stopService(AkkaRpcService.java:282)
    	at org.apache.flink.runtime.minicluster.MiniCluster.shutDownRpc(MiniCluster.java:596)
    	at org.apache.flink.runtime.minicluster.MiniCluster.shutdownInternally(MiniCluster.java:364)
    	at org.apache.flink.runtime.minicluster.MiniCluster.shutdown(MiniCluster.java:309)
    	- locked <0x00000000aad1d2d8> (a java.lang.Object)
    	at org.apache.flink.runtime.minicluster.MiniClusterITCase.runJobWithMultipleJobManagers(MiniClusterITCase.java:87)
    	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.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:283)
    	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:173)
    	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:153)
    	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:128)
    	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:203)
    	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:155)
    	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:103)
    ```


> MiniCluster can deadlock at shut down
> -------------------------------------
>
>                 Key: FLINK-7279
>                 URL: https://issues.apache.org/jira/browse/FLINK-7279
>             Project: Flink
>          Issue Type: Bug
>          Components: Tests
>    Affects Versions: 1.4.0
>            Reporter: Till Rohrmann
>              Labels: flip-6
>
> The {{MiniCluster}} can deadlock in case if the fatal error handler is called while the
{{MiniCluster}} shuts down. The reason is that the shut down happens under a lock which is
required by the fatal error handler as well. If now the {{MiniCluster}} tries to shut down
the underlying RPC service which waits for all actors to terminate, it will never complete
because one actor is still waiting for the lock.
> One solution would be to ignore the fatal error handler calls if the {{MiniCluster}}
is shutting down.
> https://s3.amazonaws.com/archive.travis-ci.org/jobs/257811319/log.txt



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message