Uploaded image for project: 'Giraph (Retired)'
  1. Giraph (Retired)
  2. GIRAPH-61

Worker's early failure will cause the whole system fail

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 0.1.0
    • None
    • bsp
    • None

    Description

      When there's early failure happens to a worker, the whole system will fail.

      Observed failed worker:
      State: Creating RPC threads failed
      Result: It will cause the worker fail, however, master has already recorded and reserved these splits to this worker (identified by InetAddress), thus although hadoop reschedule a mapper for this worker, the master still waiting for the old worker's response, finally, the master will fail.

      [Failed worker logs:]

      2011-10-24 18:19:51,051 INFO org.apache.giraph.graph.BspService: process: vertexRangeAssignmentsReadyChanged (vertex ranges are assigned)
      2011-10-24 18:19:51,060 INFO org.apache.giraph.graph.BspServiceWorker: startSuperstep: Ready for computation on superstep 1 since worker selection and vertex range assignments are done in /_hadoopBsp/job_201108260911_842943/_applicationAttemptsDir/0/_superstepDir/1/_vertexRangeAssignments
      2011-10-24 18:19:51,078 INFO org.apache.giraph.graph.BspServiceWorker: getAggregatorValues: no aggregators in /_hadoopBsp/job_201108260911_842943/_applicationAttemptsDir/0/_superstepDir/0/_mergedAggregatorDir on superstep 1
      2011-10-24 18:19:53,974 INFO org.apache.giraph.graph.GraphMapper: map: totalMem=84213760 maxMem=2067988480 freeMem=65069808
      2011-10-24 18:19:53,974 INFO org.apache.giraph.comm.BasicRPCCommunications: flush: starting...
      2011-10-24 18:19:54,022 INFO org.apache.hadoop.mapred.TaskLogsTruncater: Initializing logs' truncater with mapRetainSize=102400 and reduceRetainSize=102400
      2011-10-24 18:19:54,023 FATAL org.apache.hadoop.mapred.Child: Error running child : java.lang.OutOfMemoryError: unable to create new native thread
      at java.lang.Thread.start0(Native Method)
      at java.lang.Thread.start(Thread.java:597)
      at java.lang.UNIXProcess$1.run(UNIXProcess.java:141)
      at java.security.AccessController.doPrivileged(Native Method)
      at java.lang.UNIXProcess.<init>(UNIXProcess.java:103)
      at java.lang.ProcessImpl.start(ProcessImpl.java:65)
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:453)
      at org.apache.hadoop.util.Shell.runCommand(Shell.java:200)
      at org.apache.hadoop.util.Shell.run(Shell.java:182)
      at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:375)
      at org.apache.hadoop.util.Shell.execCommand(Shell.java:461)
      at org.apache.hadoop.util.Shell.execCommand(Shell.java:444)
      at org.apache.hadoop.fs.RawLocalFileSystem.execCommand(RawLocalFileSystem.java:540)
      at org.apache.hadoop.fs.RawLocalFileSystem.access$100(RawLocalFileSystem.java:37)
      at org.apache.hadoop.fs.RawLocalFileSystem$RawLocalFileStatus.loadPermissionInfo(RawLocalFileSystem.java:417)
      at org.apache.hadoop.fs.RawLocalFileSystem$RawLocalFileStatus.getOwner(RawLocalFileSystem.java:400)
      at org.apache.hadoop.mapred.TaskLog.obtainLogDirOwner(TaskLog.java:275)
      at org.apache.hadoop.mapred.TaskLogsTruncater.truncateLogs(TaskLogsTruncater.java:124)
      at org.apache.hadoop.mapred.Child$4.run(Child.java:266)
      at java.security.AccessController.doPrivileged(Native Method)
      at javax.security.auth.Subject.doAs(Subject.java:396)
      at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)

      [Failed master logs:]

      2011-10-24 18:21:47,611 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: Only found 399 responses of 400 needed to start superstep 0. Sleeping for 5000 msecs and used 1 of 60 attempts.
      2011-10-24 18:21:47,615 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: No response from partition 279 (could be master)
      2011-10-24 18:21:52,629 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: Only found 399 responses of 400 needed to start superstep 0. Sleeping for 5000 msecs and used 2 of 60 attempts.
      2011-10-24 18:21:52,631 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: No response from partition 279 (could be master)
      2011-10-24 18:21:57,636 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: Only found 399 responses of 400 needed to start superstep 0. Sleeping for 5000 msecs and used 3 of 60 attempts.
      2011-10-24 18:21:57,637 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: No response from partition 279 (could be master)
      2011-10-24 18:21:58,142 ERROR org.apache.zookeeper.ClientCnxn: Error while calling watcher
      java.lang.RuntimeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /_hadoopBsp/job_201108260911_842943/_applicationAttemptsDir/1/_superstepDir/0/_vertexRangeAssignments
      at org.apache.giraph.graph.BspService.getVertexRangeMap(BspService.java:890)
      at org.apache.giraph.graph.BspServiceMaster.checkHealthyWorkerFailure(BspServiceMaster.java:1964)
      at org.apache.giraph.graph.BspServiceMaster.processEvent(BspServiceMaster.java:1995)
      at org.apache.giraph.graph.BspService.process(BspService.java:1100)
      at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:488)
      Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /_hadoopBsp/job_201108260911_842943/_applicationAttemptsDir/1/_superstepDir/0/_vertexRangeAssignments
      at org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
      at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
      at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:921)
      at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:950)
      at org.apache.giraph.graph.BspService.getVertexRangeMap(BspService.java:861)
      ... 4 more

      2011-10-24 18:26:38,501 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: Only found 399 responses of 400 needed to start superstep 0. Sleeping for 5000 msecs and used 59 of 60 attempts.
      2011-10-24 18:26:38,501 INFO org.apache.giraph.graph.BspServiceMaster: checkWorkers: No response from partition 279 (could be master)
      2011-10-24 18:26:38,501 ERROR org.apache.giraph.graph.BspServiceMaster: checkWorkers: Did not receive enough processes in time (only 399 of 400 required). This occurs if you do not have enough map tasks available simultaneously on your Hadoop instance to fulfill the number of requested workers.
      2011-10-24 18:26:38,501 FATAL org.apache.giraph.graph.BspServiceMaster: coordinateSuperstep: Not enough healthy workers for superstep 0
      2011-10-24 18:26:38,501 INFO org.apache.giraph.graph.BspServiceMaster: setJobState:

      {"_stateKey":"FAILED","_applicationAttemptKey":-1,"_superstepKey":-1}

      on superstep 0
      2011-10-24 18:26:38,507 FATAL org.apache.giraph.graph.BspServiceMaster: failJob: Killing job job_201108260911_842943
      2011-10-24 18:26:38,567 ERROR org.apache.giraph.graph.MasterThread: masterThread: Master algorithm failed:
      java.lang.NullPointerException
      at org.apache.giraph.graph.BspServiceMaster.coordinateSuperstep(BspServiceMaster.java:1613)
      at org.apache.giraph.graph.MasterThread.run(MasterThread.java:105)
      2011-10-24 18:26:38,567 FATAL org.apache.giraph.graph.GraphMapper: uncaughtException: OverrideExceptionHandler on thread org.apache.giraph.graph.MasterThread, msg = java.lang.NullPointerException, exiting...
      java.lang.RuntimeException: java.lang.NullPointerException
      at org.apache.giraph.graph.MasterThread.run(MasterThread.java:179)
      Caused by: java.lang.NullPointerException
      at org.apache.giraph.graph.BspServiceMaster.coordinateSuperstep(BspServiceMaster.java:1613)
      at org.apache.giraph.graph.MasterThread.run(MasterThread.java:105)
      2011-10-24 18:26:38,568 WARN org.apache.giraph.zk.ZooKeeperManager: onlineZooKeeperServers: Forced a shutdown hook kill of the ZooKeeper process.

      Attachments

        Activity

          People

            Unassigned Unassigned
            guzhiwei Zhiwei Gu
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: