You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Ryan Blue (JIRA)" <ji...@apache.org> on 2016/06/02 01:34:59 UTC

[jira] [Created] (SPARK-15725) Dynamic allocation hangs YARN app when executors time out

Ryan Blue created SPARK-15725:
---------------------------------

             Summary: Dynamic allocation hangs YARN app when executors time out
                 Key: SPARK-15725
                 URL: https://issues.apache.org/jira/browse/SPARK-15725
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 1.6.1, 2.0.0
            Reporter: Ryan Blue


We've had a problem with a dynamic allocation and YARN (since 1.6) where a large stage will cause a lot of executors to get killed around the same time and RPC calls to the application master time out. This can happen even with a small number of executors (~100).

When executors are killed by the driver, the [network connection to the driver disconnects|https://github.com/apache/spark/blob/master/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala#L201]. That results in a call to the AM to find out why the executor died, followed by a [blocking and retrying `RemoveExecutor` RPC call|https://github.com/apache/spark/blob/master/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala#L227] that results in a second `KillExecutor` call to the AM. When a lot of executors are killed around the same time, the driver's AM threads are all taken up blocking and waiting on the AM (see the stack trace below, which was the same for 42 threads). I think this behavior, the network disconnect and subsequent cleanup, is unique to YARN.

{code:title=Driver AM thread stack}
sun.misc.Unsafe.park(Native Method)
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:208)
scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:218)
scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223)
scala.concurrent.Await$$anonfun$result$1.apply(package.scala:190)
scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
scala.concurrent.Await$.result(package.scala:190)
org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:81)
org.apache.spark.rpc.RpcEndpointRef.askWithRetry(RpcEndpointRef.scala:102)
org.apache.spark.rpc.RpcEndpointRef.askWithRetry(RpcEndpointRef.scala:78)
org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$2.apply$mcV$sp(YarnSchedulerBackend.scala:286)
org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$2.apply(YarnSchedulerBackend.scala:286)
org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$2.apply(YarnSchedulerBackend.scala:286)
scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
java.lang.Thread.run(Thread.java:745)
{code}

The RPC calls to the AM aren't returning because the `YarnAllocator` is spending all of its time in the `allocateResources` method. That class's public methods are synchronized so only one RPC can be satisfied at a time. The reason why it is constantly calling `allocateResources` is because [its thread|https://github.com/apache/spark/blob/master/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala#L467] is [woken up|https://github.com/apache/spark/blob/master/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala#L686] by calls to get the failure reason for an executor -- which is part of the chain of events in the driver for each executor that goes down.

The final result is that the `YarnAllocator` doesn't respond to RPC calls for long enough that calls time out and replies for non-blocking calls are dropped. Then the application is unable to do any work because everything retries or exits and the application hangs for 24+ hours, until enough errors accumulate that it dies.



--
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