You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "cen yuhai (JIRA)" <ji...@apache.org> on 2016/07/08 13:55:11 UTC

[jira] [Created] (SPARK-16441) Spark application hang when dynamic allocation is enabled

cen yuhai created SPARK-16441:
---------------------------------

             Summary: Spark application hang when dynamic allocation is enabled
                 Key: SPARK-16441
                 URL: https://issues.apache.org/jira/browse/SPARK-16441
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 1.6.2
         Environment: hadoop 2.7.2  spark1.6.2
            Reporter: cen yuhai


spark application are waiting for rpc response all the time and spark listener are blocked by dynamic allocation. Executors can not connect to driver and lost.

"spark-dynamic-executor-allocation" #239 daemon prio=5 os_prio=0 tid=0x00007fa304438000 nid=0xcec6 waiting on condition [0x00007fa2b81e4000]
   java.lang.Thread.State: TIMED_WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x000000070fdb94f8> (a scala.concurrent.impl.Promise$CompletionLatch)
	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
	at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:208)
	at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:218)
	at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223)
	at scala.concurrent.Await$$anonfun$result$1.apply(package.scala:107)
	at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
	at scala.concurrent.Await$.result(package.scala:107)
	at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
	at org.apache.spark.rpc.RpcEndpointRef.askWithRetry(RpcEndpointRef.scala:101)
	at org.apache.spark.rpc.RpcEndpointRef.askWithRetry(RpcEndpointRef.scala:77)
	at org.apache.spark.scheduler.cluster.YarnSchedulerBackend.doRequestTotalExecutors(YarnSchedulerBackend.scala:59)
	at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.requestTotalExecutors(CoarseGrainedSchedulerBackend.scala:436)
	- locked <0x00000000828a8960> (a org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend)
	at org.apache.spark.SparkContext.requestTotalExecutors(SparkContext.scala:1438)
	at org.apache.spark.ExecutorAllocationManager.addExecutors(ExecutorAllocationManager.scala:359)
	at org.apache.spark.ExecutorAllocationManager.updateAndSyncNumExecutorsTarget(ExecutorAllocationManager.scala:310)
	- locked <0x00000000880e6308> (a org.apache.spark.ExecutorAllocationManager)
	at org.apache.spark.ExecutorAllocationManager.org$apache$spark$ExecutorAllocationManager$$schedule(ExecutorAllocationManager.scala:264)
	- locked <0x00000000880e6308> (a org.apache.spark.ExecutorAllocationManager)
	at org.apache.spark.ExecutorAllocationManager$$anon$2.run(ExecutorAllocationManager.scala:223)




"SparkListenerBus" #161 daemon prio=5 os_prio=0 tid=0x00007fa3053be000 nid=0xcec9 waiting for monitor entry [0x00007fa2b3dfc000]
   java.lang.Thread.State: BLOCKED (on object monitor)
	at org.apache.spark.ExecutorAllocationManager$ExecutorAllocationListener.onTaskEnd(ExecutorAllocationManager.scala:618)
	- waiting to lock <0x00000000880e6308> (a org.apache.spark.ExecutorAllocationManager)
	at org.apache.spark.scheduler.SparkListenerBus$class.onPostEvent(SparkListenerBus.scala:42)
	at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31)
	at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31)
	at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:55)
	at org.apache.spark.util.AsynchronousListenerBus.postToAll(AsynchronousListenerBus.scala:37)
	at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(AsynchronousListenerBus.scala:80)
	at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(AsynchronousListenerBus.scala:65)
	at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(AsynchronousListenerBus.scala:65)
	at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57)
	at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply$mcV$sp(AsynchronousListenerBus.scala:64)
	at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1182)
	at org.apache.spark.util.AsynchronousListenerBus$$anon$1.run(AsynchronousListenerBus.scala:63)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org