Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-13906

Spark driver hangs when slave is started or stopped (org.apache.spark.rpc.RpcTimeoutException).

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 1.6.1
    • 2.0.0
    • Spark Core
    • None
    • Machine with one core (e.g. ec2 t2.small)

    Description

      When a slave is started or stopped and there is only one core, the spark driver hangs. Example:

      spark-1.6.1-bin-hadoop2.6/sbin/start-master.sh
      spark-1.6.1-bin-hadoop2.6/sbin/start-slave.sh $(hostname):7077
      spark-1.6.1-bin-hadoop2.6/bin/spark-shell --master spark://$(hostname):7077
      spark> sc.parallelize(1 to 300, 20).map(x => {Thread.sleep(100); x*2}).collect()
      
      # While that is running, kill a slave
      spark-1.6.1-bin-hadoop2.6/sbin/stop-slave.sh
      

      After 2 minutes, spark-shell spits out an error:

      org.apache.spark.rpc.RpcTimeoutException: Cannot receive any reply in 120 seconds. This timeout is controlled by spark.rpc.askTimeout
      	at org.apache.spark.rpc.RpcTimeout.org$apache$spark$rpc$RpcTimeout$$createRpcTimeoutException(RpcTimeout.scala:48)
      	at org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:63)
      	at org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:59)
      	at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:33)
      	at scala.util.Failure$$anonfun$recover$1.apply(Try.scala:185)
      	at scala.util.Try$.apply(Try.scala:161)
      	at scala.util.Failure.recover(Try.scala:185)
      	at scala.concurrent.Future$$anonfun$recover$1.apply(Future.scala:324)
      	at scala.concurrent.Future$$anonfun$recover$1.apply(Future.scala:324)
      	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
      	at org.spark-project.guava.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:293)
      	at scala.concurrent.impl.ExecutionContextImpl$$anon$1.execute(ExecutionContextImpl.scala:133)
      	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
      	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
      	at scala.concurrent.Promise$class.complete(Promise.scala:55)
      	at scala.concurrent.impl.Promise$DefaultPromise.complete(Promise.scala:153)
      	at scala.concurrent.Future$$anonfun$map$1.apply(Future.scala:235)
      	at scala.concurrent.Future$$anonfun$map$1.apply(Future.scala:235)
      	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
      	at scala.concurrent.Future$InternalCallbackExecutor$Batch$$anonfun$run$1.processBatch$1(Future.scala:643)
      	at scala.concurrent.Future$InternalCallbackExecutor$Batch$$anonfun$run$1.apply$mcV$sp(Future.scala:658)
      	at scala.concurrent.Future$InternalCallbackExecutor$Batch$$anonfun$run$1.apply(Future.scala:635)
      	at scala.concurrent.Future$InternalCallbackExecutor$Batch$$anonfun$run$1.apply(Future.scala:635)
      	at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
      	at scala.concurrent.Future$InternalCallbackExecutor$Batch.run(Future.scala:634)
      	at scala.concurrent.Future$InternalCallbackExecutor$.scala$concurrent$Future$InternalCallbackExecutor$$unbatchedExecute(Future.scala:694)
      	at scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:685)
      	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
      	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
      	at scala.concurrent.Promise$class.tryFailure(Promise.scala:112)
      	at scala.concurrent.impl.Promise$DefaultPromise.tryFailure(Promise.scala:153)
      	at org.apache.spark.rpc.netty.NettyRpcEnv$$anon$1.run(NettyRpcEnv.scala:241)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      Caused by: java.util.concurrent.TimeoutException: Cannot receive any reply in 120 seconds
      	at org.apache.spark.rpc.netty.NettyRpcEnv$$anon$1.run(NettyRpcEnv.scala:242)
      	... 7 more
      

      Cause: When 1 == Runtime.getRuntime.availableProcessors(), by default there is only one dispatcher-event-loop thread (Dispatcher.scala). When any function on the dispatcher tries to ask a message to any endpoint on the same process, it blocks (e.g. when AppClient handles ExecutorUpdated by asking CoarseGrainedSchedulerBackend a RemoveExecutor message). Here is a jstack of the dispatcher thread:

      "dispatcher-event-loop-0" #23 daemon prio=5 os_prio=0 tid=0x00007fca7cfc9000 nid=0x907 waiting on condition [0x00007fca49982000]
         java.lang.Thread.State: TIMED_WAITING (parking)
              at sun.misc.Unsafe.park(Native Method)
              - parking to wait for  <0x00000000c5618e90> (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.CoarseGrainedSchedulerBackend.removeExecutor(CoarseGrainedSchedulerBackend.scala:359)
              at org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend.executorRemoved(SparkDeploySchedulerBackend.scala:144)
              at org.apache.spark.deploy.client.AppClient$ClientEndpoint$$anonfun$receive$1.applyOrElse(AppClient.scala:186)
              at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:116)
              at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:204)
              at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
              at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
              at java.lang.Thread.run(Thread.java:745)
      

      Workaround: set --conf spark.rpc.netty.dispatcher.numThreads=2 on a single-core machine. Hopefully there are no deeper than 2 ask calls.

      Attachments

        Activity

          People

            yonran Yonathan Randolph
            yonran Yonathan Randolph
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: