ResourceManager dispatcher handler slow because RMStore Synchronized Method

Recently, I have noticed the Async dispatcher in our resource manager get pending for some times.
Here are some log:

2016-05-24 00:46:20,398 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 24000
2016-05-24 00:46:21,008 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 25000
2016-05-24 00:46:21,632 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 26000
2016-05-24 00:46:22,251 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 27000
2016-05-24 00:46:22,873 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 28000
2016-05-24 00:46:23,501 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 29000
2016-05-24 00:46:24,109 INFO org.apache.hadoop.yarn.event.AsyncDispatcher: Size of event-queue is 30000  

As we all know, the async dispatcher in resource manager normally handle the event quickly enough, but from the log ,we can notice the pending situation is serious.
So we investigated this problem, and jstack the rescue manager process during pending. Here is the jstack information:

"AsyncDispatcher event handler" prio=10 tid=0x00007f4d6db10000 nid=0x5bca waiting for monitor entry [0x00007f4d3aa8c000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.storeNewApplication(RMStateStore.java:375)
        - waiting to lock <0x00000003bae88af0> (a org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore)
        at org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl$RMAppNewlySavingTransition.transition(RMAppImpl.java:881)
        at org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl$RMAppNewlySavingTransition.transition(RMAppImpl.java:872)
        at org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
        at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
        at org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
        at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
        - locked <0x0000000394cbae40> (a org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine)
        at org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl.handle(RMAppImpl.java:645)
        at org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl.handle(RMAppImpl.java:82)
        at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationEventDispatcher.handle(ResourceManager.java:690)
        at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationEventDispatcher.handle(ResourceManager.java:674)
        at org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:173)
        at org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:106)
        at java.lang.Thread.run(Thread.java:662)

"AsyncDispatcher event handler" daemon prio=10 tid=0x00007f4d6d8f6000 nid=0x5c32 in Object.wait() [0x00007f4d3a183000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:2031)
        - locked <0x000000032bc7bd58> (a java.util.LinkedList)
        at org.apache.hadoop.hdfs.DFSOutputStream.flushInternal(DFSOutputStream.java:2015)
        at org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:2113)
        - locked <0x000000032bc7ba80> (a org.apache.hadoop.hdfs.DFSOutputStream)
        at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:70)
        at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:103)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore.writeFile(FileSystemRMStateStore.java:528)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore.storeApplicationStateInternal(FileSystemRMStateStore.java:329)
        - locked <0x00000003bae88af0> (a org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:625)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:770)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:765)
        at org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:173)
        at org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:106)
        at java.lang.Thread.run(Thread.java:662)

It seemed the async dispatcher in resource manager is blocked by the method of storeNewApplication in RMSteateStore.
From the code we know there are two async dispatcher in resource manager process. One is the main dispatcher for whole resource manager to deal with applications submit, scheduler and other staff. The other is the dispatcher in rmstore, the function of rmstore can be explained in this blog. Because rmstore use hdfs or zk for backup, the process time is slow, so it have its own dispatcher in case not to pending the main dispatcher of resource manager.
Unfortunately, we use hdfs for our rmstore back up, deep inside the code

  public synchronized void storeNewApplication(RMApp app) {
    ApplicationSubmissionContext context = app
                                            .getApplicationSubmissionContext();
    assert context instanceof ApplicationSubmissionContextPBImpl;
    ApplicationState appState =
        new ApplicationState(app.getSubmitTime(), app.getStartTime(), context,
          app.getUser());
    dispatcher.getEventHandler().handle(new RMStateStoreAppEvent(appState));
  }

The main dispatcher handle the event to store the new application context information, this method only pass the event to the rmstatestore dispatcher and return immediately. But the method is sync. And in the child class of rmstatestore — Filesystemrmstatestore, the code to store application to hdfs is as follow:

  @Override
  public synchronized void storeApplicationStateInternal(ApplicationId appId,
      ApplicationStateData appStateDataPB) throws Exception {
    String appIdStr = appId.toString();
    Path appDirPath = getAppDir(rmAppRoot, appIdStr);
    fs.mkdirs(appDirPath);
    Path nodeCreatePath = getNodePath(appDirPath, appIdStr);

    LOG.info("Storing info for app: " + appId + " at: " + nodeCreatePath);
    byte[] appStateData = appStateDataPB.getProto().toByteArray();
    try {
      // currently throw all exceptions. May need to respond differently for HA
      // based on whether we have lost the right to write to FS
      writeFile(nodeCreatePath, appStateData);
    } catch (Exception e) {
      LOG.info("Error storing info for app: " + appId, e);
      throw e;
    }
  }

This method is also sync. So if dispatcher in rmstatestore occupy the lock, and write to hdfs slow, it is easy to find the main dispatcher get pending to wait for the lock. And the lock is meaningless for main dispatcher, so we can just get rid of the lock.
There are related jira for this problem, the jira is YARN-4398.