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

Localizer thread might stuck if NM is OOM

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • None
    • None
    • None
    • None

    Description

      When NM JVM runs out of memory; normally it is uncaught exception and the process will exit. But RPC server used by node manager catches OutOfMemoryError to give a chance GC to catch up so NM doesn't need to exit and can recover from OutOfMemoryError situation.

      However, in some rare situation when this happens, one of the NM localizer thread didn't get the RPC response from node manager and just waited there. The explanation of why node manager RPC server doesn't respond is because RPC server responder thread swallowed OutOfMemoryError and didn't process outstanding RPC response. On the RPC client side, the RPC timeout is set to 0 and it relies on Ping to detect RPC server availability.

      Thread 481 (LocalizerRunner for container_1413487737702_2948_01_013383):
        State: WAITING
        Blocked count: 27
        Waited count: 84
        Waiting on org.apache.hadoop.ipc.Client$Call@6be5add3
        Stack:
          java.lang.Object.wait(Native Method)
          java.lang.Object.wait(Object.java:503)
          org.apache.hadoop.ipc.Client.call(Client.java:1396)
          org.apache.hadoop.ipc.Client.call(Client.java:1363)
          org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
          com.sun.proxy.$Proxy36.heartbeat(Unknown Source)
          org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.client.LocalizationProtocolPBClientImpl.heartbeat(LocalizationProtocolPBClientImpl.java:62)
          org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer.localizeFiles(ContainerLocalizer.java:235)
          org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer.runLocalization(ContainerLocalizer.java:169)
          org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.startLocalizer(DefaultContainerExecutor.java:107)
          org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.run(ResourceLocalizationService.java:995)
      

      The consequence of this depends on which ContainerExecutor NM uses. If it uses DefaultContainerExecutor, given its startLocalizer method is synchronized, it will blocks other localizer threads. If you use LinuxContainerExecutor, at least other localizer threads can still proceed. But in theory it can slowly drain all available localizer threads.

      There are couple ways to fix it. Some of these fixes are complementary.

      1. Fix it at haoop-common layer. It seems RPC server hosted by worker services such ad NM doesn't really need to catch OutOfMemoryError; the service JVM can just exit. Even for the NN and RM, given we have HA, it might be ok to do so.
      2. Set RPC timeout at HadoopYarnProtoRPC layer so that all YARN clients will timeout if RPC server drops the response.
      3. Fix it at yarn localization service. For example,
      a) Fix DefaultContainerExecutor so that synchronization isn't required for startLocalizer method.
      b) Download executor thread used by ContainerLocalizer currently catches any exceptions. We can fix ContainerLocalizer so that when Download executor thread catches OutOfMemoryError, it can exit its host process.

      IMHO, fix it at RPC server layer is better as it addresses other scenarios. Appreciate any input others might have.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              mingma Ming Ma
              Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated: