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

NullPointerException that causes failure to request executors.

    XMLWordPrintableJSON

Details

    Description

      After the job moves from YARN ACCEPTED to RUNNING it immitetly raises the following exception.

      15/08/11 06:37:01 ERROR AkkaRpcEnv: Ignore error: null
      java.lang.NullPointerException
      	at org.apache.spark.rpc.akka.AkkaRpcEndpointRef.actorRef$lzycompute(AkkaRpcEnv.scala:281)
      	at org.apache.spark.rpc.akka.AkkaRpcEndpointRef.actorRef(AkkaRpcEnv.scala:281)
      	at org.apache.spark.rpc.akka.AkkaRpcEndpointRef.toString(AkkaRpcEnv.scala:322)
      	at java.lang.String.valueOf(String.java:2849)
      	at java.lang.StringBuilder.append(StringBuilder.java:128)
      	at scala.StringContext.standardInterpolator(StringContext.scala:122)
      	at scala.StringContext.s(StringContext.scala:90)
      	at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1$$anonfun$applyOrElse$5.apply(YarnSchedulerBackend.scala:106)
      	at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1$$anonfun$applyOrElse$5.apply(YarnSchedulerBackend.scala:106)
      	at org.apache.spark.Logging$class.logInfo(Logging.scala:59)
      	at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint.logInfo(YarnSchedulerBackend.scala:96)
      	at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1.applyOrElse(YarnSchedulerBackend.scala:106)
      	at org.apache.spark.rpc.akka.AkkaRpcEnv.org$apache$spark$rpc$akka$AkkaRpcEnv$$processMessage(AkkaRpcEnv.scala:177)
      	at org.apache.spark.rpc.akka.AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1$$anonfun$receiveWithLogging$1$$anonfun$applyOrElse$4.apply$mcV$sp(AkkaRpcEnv.scala:126)
      	at org.apache.spark.rpc.akka.AkkaRpcEnv.org$apache$spark$rpc$akka$AkkaRpcEnv$$safelyCall(AkkaRpcEnv.scala:197)
      	at org.apache.spark.rpc.akka.AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1$$anonfun$receiveWithLogging$1.applyOrElse(AkkaRpcEnv.scala:125)
      	at scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
      	at scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
      	at scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
      	at org.apache.spark.util.ActorLogReceive$$anon$1.apply(ActorLogReceive.scala:59)
      	at org.apache.spark.util.ActorLogReceive$$anon$1.apply(ActorLogReceive.scala:42)
      	at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118)
      	at org.apache.spark.util.ActorLogReceive$$anon$1.applyOrElse(ActorLogReceive.scala:42)
      	at akka.actor.Actor$class.aroundReceive(Actor.scala:467)
      	at org.apache.spark.rpc.akka.AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1.aroundReceive(AkkaRpcEnv.scala:92)
      	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)
      

      Then later it can't request executors.

      15/08/11 06:37:07 INFO YarnScheduler: Adding task set 0.0 with 36 tasks
      15/08/11 06:37:08 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
      15/08/11 06:37:08 WARN ExecutorAllocationManager: Unable to reach the cluster manager to request 1 total executors!
      15/08/11 06:37:09 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
      15/08/11 06:37:09 WARN ExecutorAllocationManager: Unable to reach the cluster manager to request 2 total executors!
      15/08/11 06:37:10 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
      15/08/11 06:37:10 WARN ExecutorAllocationManager: Unable to reach the cluster manager to request 3 total executors!
      15/08/11 06:37:11 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
      15/08/11 06:37:11 WARN ExecutorAllocationManager: Unable to reach the cluster manager to request 4 total executors!
      15/08/11 06:37:12 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
      15/08/11 06:37:12 WARN ExecutorAllocationManager: Unable to reach the cluster manager to request 5 total executors!
      15/08/11 06:37:13 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
      15/08/11 06:37:13 WARN ExecutorAllocationManager: Unable to reach the cluster manager to request 6 total executors!
      

      Which causes the job to hang forever.

      WARN YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources
      WARN YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources
      WARN YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources
      WARN YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources
      ...
      

      Attachments

        1. Frequency.png
          13 kB
          Kevin Cox

        Activity

          People

            Unassigned Unassigned
            kevincox Kevin Cox
            Votes:
            5 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: