Uploaded image for project: 'Hadoop YARN'
  1. Hadoop YARN
  2. YARN-2846

Incorrect persist exit code for running containers in reacquireContainer() that interrupted by NodeManager restart.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Blocker
    • Resolution: Fixed
    • None
    • 2.6.0
    • nodemanager
    • None
    • Reviewed

    Description

      The NM restart work preserving feature could make running AM container get LOST and killed during stop NM daemon. The exception is like below:

      2014-11-11 00:48:35,214 INFO  monitor.ContainersMonitorImpl (ContainersMonitorImpl.java:run(408)) - Memory usage of ProcessTree 22140 for container-id container_1415666714233_0001_01_000084: 53.8 MB of 512 MB physical memory used; 931.3 MB of 1.0 GB virtual memory used
      2014-11-11 00:48:35,223 ERROR nodemanager.NodeManager (SignalLogger.java:handle(60)) - RECEIVED SIGNAL 15: SIGTERM
      2014-11-11 00:48:35,299 INFO  mortbay.log (Slf4jLog.java:info(67)) - Stopped HttpServer2$SelectChannelConnectorWithSafeStartup@0.0.0.0:50060
      2014-11-11 00:48:35,337 INFO  containermanager.ContainerManagerImpl (ContainerManagerImpl.java:cleanUpApplicationsOnNMShutDown(512)) - Applications still running : [application_1415666714233_0001]
      2014-11-11 00:48:35,338 INFO  ipc.Server (Server.java:stop(2437)) - Stopping server on 45454
      2014-11-11 00:48:35,344 INFO  ipc.Server (Server.java:run(706)) - Stopping IPC Server listener on 45454
      2014-11-11 00:48:35,346 INFO  logaggregation.LogAggregationService (LogAggregationService.java:serviceStop(141)) - org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService waiting for pending aggregation during exit
      2014-11-11 00:48:35,347 INFO  ipc.Server (Server.java:run(832)) - Stopping IPC Server Responder
      2014-11-11 00:48:35,347 INFO  logaggregation.AppLogAggregatorImpl (AppLogAggregatorImpl.java:abortLogAggregation(502)) - Aborting log aggregation for application_1415666714233_0001
      2014-11-11 00:48:35,348 WARN  logaggregation.AppLogAggregatorImpl (AppLogAggregatorImpl.java:run(382)) - Aggregation did not complete for application application_1415666714233_0001
      2014-11-11 00:48:35,358 WARN  monitor.ContainersMonitorImpl (ContainersMonitorImpl.java:run(476)) - org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl is interrupted. Exiting.
      2014-11-11 00:48:35,406 ERROR launcher.RecoveredContainerLaunch (RecoveredContainerLaunch.java:call(87)) - Unable to recover container container_1415666714233_0001_01_000001
      java.io.IOException: Interrupted while waiting for process 20001 to exit
              at org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.reacquireContainer(ContainerExecutor.java:180)
              at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.RecoveredContainerLaunch.call(RecoveredContainerLaunch.java:82)
              at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.RecoveredContainerLaunch.call(RecoveredContainerLaunch.java:46)
              at java.util.concurrent.FutureTask.run(FutureTask.java:262)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
              at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.InterruptedException: sleep interrupted
              at java.lang.Thread.sleep(Native Method)
              at org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.reacquireContainer(ContainerExecutor.java:177)
              ... 6 more
      

      In reacquireContainer() of ContainerExecutor.java, the while loop of checking container process (AM container) will be interrupted by NM stop. The IOException get thrown and failed to generate an ExitCodeFile for the running container. Later, the IOException will be caught in upper call (RecoveredContainerLaunch.call()) and the ExitCode (by default to be LOST without any setting) get persistent in NMStateStore.
      After NM restart again, this container is recovered as COMPLETE state but exit code is LOST (154) - cause this (AM) container get killed later.
      We should get rid of recording the exit code of running containers if detecting process is interrupted.

      Attachments

        1. YARN-2846-demo.patch
          5 kB
          Junping Du
        2. YARN-2846.patch
          5 kB
          Junping Du

        Activity

          People

            junping_du Junping Du
            junping_du Junping Du
            Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: