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

Driver is marked as failed even it runs successfully

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.6.0, 1.6.1
    • 2.0.0
    • Spark Core
    • None
    • Standalone deployment

    Description

      We recently upgraded Spark from 1.5.2 to 1.6.0 and found that all batch jobs are marked as failed.
      To address this issue, we wrote a simple test application which just sum up from 1 to 10000 and it is marked as failed even though its result was correct.
      Here is the typical stderr message and there is "ERROR worker.WorkerWatcher: Lost connection to worker rpc" when driver exits.

      16/04/14 06:20:41 INFO scheduler.DAGScheduler: ResultStage 1 (sum at SparkBatchTest.scala:19) finished in 0.052 s
      16/04/14 06:20:41 INFO scheduler.TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool
      16/04/14 06:20:41 INFO scheduler.DAGScheduler: Job 1 finished: sum at SparkBatchTest.scala:19, took 0.061177 s
      16/04/14 06:20:41 ERROR worker.WorkerWatcher: Lost connection to worker rpc endpoint spark://Worker@spark-worker-ltv-prod-006.prod.vungle.com:7078. Exiting.
      16/04/14 06:20:41 INFO spark.SparkContext: Invoking stop() from shutdown hook
      16/04/14 06:20:41 INFO storage.BlockManagerInfo: Removed broadcast_1_piece0 on 172.16.33.187:36442 in memory (size: 1452.0 B, free: 511.1 MB)
      16/04/14 06:20:41 INFO storage.BlockManagerInfo: Removed broadcast_1_piece0 on ip-172-16-31-86.ec2.internal:29708 in memory (size: 1452.0 B, free: 511.1 MB)
      16/04/14 06:20:41 INFO storage.BlockManagerInfo: Removed broadcast_1_piece0 on ip-172-16-32-207.ec2.internal:21259 in memory (size: 1452.0 B, free: 511.1 MB)
      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/metrics/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages/stage/kill,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/api,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/static,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/executors/threadDump/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/executors/threadDump,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/executors/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/executors,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/environment/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/environment,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/storage/rdd/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/storage/rdd,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/storage/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/storage,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages/pool/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages/pool,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages/stage/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages/stage,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/stages,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/jobs/job/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/jobs/job,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/jobs/json,null}

      16/04/14 06:20:41 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler

      {/jobs,null}

      16/04/14 06:20:41 INFO spark.ContextCleaner: Cleaned accumulator 2
      16/04/14 06:20:41 INFO storage.BlockManagerInfo: Removed broadcast_0_piece0 on 172.16.33.187:36442 in memory (size: 804.0 B, free: 511.1 MB)
      16/04/14 06:20:41 INFO storage.BlockManagerInfo: Removed broadcast_0_piece0 on ip-172-16-31-86.ec2.internal:29708 in memory (size: 804.0 B, free: 511.1 MB)
      16/04/14 06:20:41 INFO storage.BlockManagerInfo: Removed broadcast_0_piece0 on ip-172-16-32-207.ec2.internal:21259 in memory (size: 804.0 B, free: 511.1 MB)
      16/04/14 06:20:41 INFO spark.ContextCleaner: Cleaned accumulator 1
      16/04/14 06:20:41 INFO ui.SparkUI: Stopped Spark web UI at http://172.16.31.56:4040
      16/04/14 06:20:41 INFO cluster.SparkDeploySchedulerBackend: Shutting down all executors
      16/04/14 06:20:41 INFO cluster.SparkDeploySchedulerBackend: Asking each executor to shut down
      16/04/14 06:20:41 INFO spark.MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped!
      16/04/14 06:20:41 INFO storage.MemoryStore: MemoryStore cleared
      16/04/14 06:20:41 INFO storage.BlockManager: BlockManager stopped
      16/04/14 06:20:41 INFO storage.BlockManagerMaster: BlockManagerMaster stopped
      16/04/14 06:20:41 INFO scheduler.OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped!
      16/04/14 06:20:41 INFO spark.SparkContext: Successfully stopped SparkContext
      16/04/14 06:20:41 INFO util.ShutdownHookManager: Shutdown hook called
      16/04/14 06:20:41 INFO util.ShutdownHookManager: Deleting directory /tmp/spark-a4c7f9ac-bf40-4924-b977-ec0be4c5fe93

      Since it looks like something related with rpc, we tested with Akka which was default before 1.6.0 by setting "spark.rpc=akka".
      Then the error message dismisses and a warning complains the same thing about disassociation with worker.
      It seems with Akka it waits another 5 seconds and that makes driver exists gracefully.
      16/04/12 15:41:28 INFO DAGScheduler: ResultStage 1 (sum at SparkBatchTest.scala:19) finished in 0.053 s
      16/04/12 15:41:28 INFO DAGScheduler: Job 1 finished: sum at SparkBatchTest.scala:19, took 0.060660 s
      16/04/12 15:41:28 INFO SparkContext: Invoking stop() from shutdown hook
      16/04/12 15:41:28 INFO RemoteActorRefProvider$RemotingTerminator: Shutting down remote daemon.
      16/04/12 15:41:28 INFO RemoteActorRefProvider$RemotingTerminator: Remote daemon shut down; proceeding with flushing remote transports.
      16/04/12 15:41:28 INFO RemoteActorRefProvider$RemotingTerminator: Remoting shut down.
      16/04/12 15:41:28 INFO SparkUI: Stopped Spark web UI at http://sparkbox:8830
      16/04/12 15:41:28 INFO SparkDeploySchedulerBackend: Shutting down all executors
      16/04/12 15:41:28 INFO SparkDeploySchedulerBackend: Asking each executor to shut down
      16/04/12 15:41:28 WARN ReliableDeliverySupervisor: Association with remote system [akka.tcp://sparkExecutor@sparkbox:58832] has failed, address is now gated for [5000] ms. Reason: [Disassociated]
      16/04/12 15:41:28 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped!
      16/04/12 15:41:28 INFO MemoryStore: MemoryStore cleared
      16/04/12 15:41:28 INFO BlockManager: BlockManager stopped
      16/04/12 15:41:28 INFO BlockManagerMaster: BlockManagerMaster stopped
      16/04/12 15:41:28 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped!
      16/04/12 15:41:28 INFO RemoteActorRefProvider$RemotingTerminator: Shutting down remote daemon.
      16/04/12 15:41:28 INFO RemoteActorRefProvider$RemotingTerminator: Remote daemon shut down; proceeding with flushing remote transports.
      16/04/12 15:41:28 INFO RemoteActorRefProvider$RemotingTerminator: Remoting shut down.
      16/04/12 15:41:28 INFO SparkContext: Successfully stopped SparkContext
      16/04/12 15:41:28 INFO ShutdownHookManager: Shutdown hook called
      16/04/12 15:41:28 INFO ShutdownHookManager: Deleting directory /tmp/spark-138182c2-f1bc-4ea1-b5b8-2adceb27c083

      This only occurs with standalone deployment since it is said DriverWrapper is only used in that cluster mode.
      https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala#L43

      Typically it affects the '--supervise' usage for batch jobs, since they will be always considered as failed and get restarted again and again.
      https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala#L214

      Attachments

        Activity

          People

            zsxwing Shixiong Zhu
            liutzvin Huiqiang Liu
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: