Uploaded image for project: 'Hadoop YARN'
  1. Hadoop YARN
  2. YARN-10695 Event related improvement of YARN for better usage.
  3. YARN-10642

Race condition: AsyncDispatcher can get stuck by the changes introduced in YARN-8995

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 3.2.1
    • 3.4.0, 3.3.1, 3.2.3
    • resourcemanager
    • None

    Description

      In our cluster, ResouceManager stuck twice within twenty days. Yarn client can't submit application. I got jstack info at second time, then found the reason.
      I analyze all the jstack, I found many thread stuck because can't get LinkedBlockingQueue.putLock. (Note: Sorry for limited space , omit the analytical process)
      The reason is that one thread hold the putLock all the time, printEventQueueDetails will called forEachRemaining, then hold putLock and readLock. The AsyncDispatcher will stuck.

      Thread 6526 (IPC Server handler 454 on default port 8030):
        State: RUNNABLE
        Blocked count: 29988
        Waited count: 2035029
        Stack:
          java.util.concurrent.LinkedBlockingQueue$LBQSpliterator.forEachRemaining(LinkedBlockingQueue.java:926)
          java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
          java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
          java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
          java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
          java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499)
          org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.printEventQueueDetails(AsyncDispatcher.java:270)
          org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.handle(AsyncDispatcher.java:295)
          org.apache.hadoop.yarn.server.resourcemanager.DefaultAMSProcessor.handleProgress(DefaultAMSProcessor.java:408)
          org.apache.hadoop.yarn.server.resourcemanager.DefaultAMSProcessor.allocate(DefaultAMSProcessor.java:215)
          org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.processor.DisabledPlacementProcessor.allocate(DisabledPlacementProcessor.java:75)
          org.apache.hadoop.yarn.server.resourcemanager.AMSProcessingChain.allocate(AMSProcessingChain.java:92)
          org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService.allocate(ApplicationMasterService.java:432)
          org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl.allocate(ApplicationMasterProtocolPBServiceImpl.java:60)
          org.apache.hadoop.yarn.proto.ApplicationMasterProtocol$ApplicationMasterProtocolService$2.callBlockingMethod(ApplicationMasterProtocol.java:99)
          org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:528)
          org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1070)
          org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1040)
          org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:958)
          java.security.AccessController.doPrivileged(Native Method)
      

      I analyze LinkedBlockingQueue's source code. I found forEachRemaining in LinkedBlockingQueue.LBQSpliterator may stuck, when forEachRemaining and take are called in different thread.
      YARN-8995 introduce printEventQueueDetails method, "eventQueue.stream().collect" will called forEachRemaining method.

      Let's see why? "put.png" shows that how to put("a"), "take.png" shows that how to take()。Specical Node: The removed Node will point itself for help gc!!!
      The key point code is in forEachRemaining, we see LBQSpliterator use forEachRemaining to visit all Node. But when got item value from Node, will release the lock. If at this time, take() will be called.
      The variable 'p' in forEachRemaining may point a Node which point itself, then forEachRemaining will be in dead loop. You can see it in "deadloop.png"

      Let's see a simple uni-test, Let's forEachRemaining called more slow than take, the problem will reproduction。uni-test is MockForDeadLoop.java.

      I debug MockForDeadLoop.java, and see a Node point itself. You can see pic "debugfornode.png"

      Environment:
      OS: CentOS Linux release 7.5.1804 (Core)
      JDK: jdk1.8.0_281

      Attachments

        1. deadloop.png
          25 kB
          Chenyu Zheng
        2. debugfornode.png
          62 kB
          Chenyu Zheng
        3. MockForDeadLoop.java
          2 kB
          Chenyu Zheng
        4. put.png
          7 kB
          Chenyu Zheng
        5. take.png
          15 kB
          Chenyu Zheng
        6. YARN-10642.001.patch
          2 kB
          Chenyu Zheng
        7. YARN-10642.002.patch
          2 kB
          Chenyu Zheng
        8. YARN-10642.003.patch
          5 kB
          Chenyu Zheng
        9. YARN-10642.004.patch
          5 kB
          Chenyu Zheng
        10. YARN-10642.005.patch
          5 kB
          Chenyu Zheng
        11. YARN-10642-branch-3.1.001.patch
          5 kB
          Peter Bacsko
        12. YARN-10642-branch-3.2.001.patch
          5 kB
          Chenyu Zheng
        13. YARN-10642-branch-3.2.002.patch
          5 kB
          Peter Bacsko
        14. YARN-10642-branch-3.3.001.patch
          5 kB
          Chenyu Zheng

        Issue Links

          Activity

            People

              zhengchenyu Chenyu Zheng
              zhengchenyu Chenyu Zheng
              Votes:
              4 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: