Przeglądaj źródła

YARN-292. Fixed FifoScheduler and FairScheduler to make their applications data structures thread safe to avoid RM crashing with ArrayIndexOutOfBoundsException. Contributed by Zhijie Shen.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1521328 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 11 lat temu
rodzic
commit
59b5490989

+ 4 - 0
hadoop-yarn-project/CHANGES.txt

@@ -176,6 +176,10 @@ Release 2.1.1-beta - UNRELEASED
     invalid client token key errors when an appliation is about to finish.
     invalid client token key errors when an appliation is about to finish.
     (Jason Lowe via vinodkv)
     (Jason Lowe via vinodkv)
 
 
+    YARN-292. Fixed FifoScheduler and FairScheduler to make their applications
+    data structures thread safe to avoid RM crashing with
+    ArrayIndexOutOfBoundsException. (Zhijie Shen via vinodkv)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -833,7 +833,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       Allocation amContainerAllocation = appAttempt.scheduler.allocate(
       Allocation amContainerAllocation = appAttempt.scheduler.allocate(
           appAttempt.applicationAttemptId, EMPTY_CONTAINER_REQUEST_LIST,
           appAttempt.applicationAttemptId, EMPTY_CONTAINER_REQUEST_LIST,
           EMPTY_CONTAINER_RELEASE_LIST, null, null);
           EMPTY_CONTAINER_RELEASE_LIST, null, null);
-
+      // There must be at least one container allocated, because a
+      // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
+      // and is put in SchedulerApplication#newlyAllocatedContainers. Then,
+      // YarnScheduler#allocate will fetch it.
+      assert amContainerAllocation.getContainers().size() != 0;
       // Set the masterContainer
       // Set the masterContainer
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
                                                                            0));
                                                                            0));

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -80,6 +80,8 @@ import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 @LimitedPrivate("yarn")
 @LimitedPrivate("yarn")
 @Evolving
 @Evolving
 @SuppressWarnings("unchecked")
 @SuppressWarnings("unchecked")
@@ -179,7 +181,8 @@ public class CapacityScheduler
   private Resource minimumAllocation;
   private Resource minimumAllocation;
   private Resource maximumAllocation;
   private Resource maximumAllocation;
 
 
-  private Map<ApplicationAttemptId, FiCaSchedulerApp> applications = 
+  @VisibleForTesting
+  protected Map<ApplicationAttemptId, FiCaSchedulerApp> applications = 
       new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>();
       new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>();
 
 
   private boolean initialized = false;
   private boolean initialized = false;

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -50,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
@@ -154,8 +154,9 @@ public class FairScheduler implements ResourceScheduler {
 
 
   // This stores per-application scheduling information, indexed by
   // This stores per-application scheduling information, indexed by
   // attempt ID's for fast lookup.
   // attempt ID's for fast lookup.
+  @VisibleForTesting
   protected Map<ApplicationAttemptId, FSSchedulerApp> applications = 
   protected Map<ApplicationAttemptId, FSSchedulerApp> applications = 
-      new HashMap<ApplicationAttemptId, FSSchedulerApp>();
+      new ConcurrentHashMap<ApplicationAttemptId, FSSchedulerApp>();
 
 
   // Nodes in the cluster, indexed by NodeId
   // Nodes in the cluster, indexed by NodeId
   private Map<NodeId, FSSchedulerNode> nodes = 
   private Map<NodeId, FSSchedulerNode> nodes = 

+ 9 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -25,8 +25,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -36,12 +36,10 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -50,7 +48,9 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
@@ -90,6 +90,8 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 @LimitedPrivate("yarn")
 @LimitedPrivate("yarn")
 @Evolving
 @Evolving
 @SuppressWarnings("unchecked")
 @SuppressWarnings("unchecked")
@@ -113,8 +115,10 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
   private Resource maximumAllocation;
   private Resource maximumAllocation;
   private boolean usePortForNodeName;
   private boolean usePortForNodeName;
 
 
-  private Map<ApplicationAttemptId, FiCaSchedulerApp> applications
-      = new TreeMap<ApplicationAttemptId, FiCaSchedulerApp>();
+  // Use ConcurrentSkipListMap because applications need to be ordered
+  @VisibleForTesting
+  protected Map<ApplicationAttemptId, FiCaSchedulerApp> applications
+      = new ConcurrentSkipListMap<ApplicationAttemptId, FiCaSchedulerApp>();
   
   
   private ActiveUsersManager activeUsersManager;
   private ActiveUsersManager activeUsersManager;
 
 

+ 72 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -19,22 +19,27 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.lang.reflect.Constructor;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -44,19 +49,24 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.Application;
 import org.apache.hadoop.yarn.server.resourcemanager.Application;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -525,4 +535,63 @@ public class TestCapacityScheduler {
       assertTrue(appComparator.compare(app2, app3) < 0);
       assertTrue(appComparator.compare(app2, app3) < 0);
     }
     }
 
 
+    @Test
+    public void testConcurrentAccessOnApplications() throws Exception {
+      CapacityScheduler cs = new CapacityScheduler();
+      verifyConcurrentAccessOnApplications(
+          cs.applications, FiCaSchedulerApp.class);
+    }
+
+    public static <T extends SchedulerApplication>
+        void verifyConcurrentAccessOnApplications(
+            final Map<ApplicationAttemptId, T> applications, Class<T> clazz)
+                throws Exception {
+      final int size = 10000;
+      final ApplicationId appId = ApplicationId.newInstance(0, 0);
+      final Constructor<T> ctor = clazz.getDeclaredConstructor(
+          ApplicationAttemptId.class, String.class, Queue.class,
+          ActiveUsersManager.class, RMContext.class);
+
+      ApplicationAttemptId appAttemptId0
+          = ApplicationAttemptId.newInstance(appId, 0);
+      applications.put(appAttemptId0, ctor.newInstance(
+              appAttemptId0, null, mock(Queue.class), null, null));
+      assertNotNull(applications.get(appAttemptId0));
+
+      // Imitating the thread of scheduler that will add and remove apps
+      final AtomicBoolean finished = new AtomicBoolean(false);
+      final AtomicBoolean failed = new AtomicBoolean(false);
+      Thread t = new Thread() {
+
+        @Override
+        public void run() {
+          for (int i = 1; i <= size; ++i) {
+            ApplicationAttemptId appAttemptId
+                = ApplicationAttemptId.newInstance(appId, i);
+            try {
+              applications.put(appAttemptId, ctor.newInstance(
+                  appAttemptId, null, mock(Queue.class), null, null));
+            } catch (Exception e) {
+              failed.set(true);
+              finished.set(true);
+              return;
+            }
+          }
+          for (int i = 1; i <= size; ++i) {
+            ApplicationAttemptId appAttemptId
+                = ApplicationAttemptId.newInstance(appId, i);
+            applications.remove(appAttemptId);
+          }
+          finished.set(true);
+        }
+      };
+      t.start();
+
+      // Imitating the thread of rmappattempt that will get the app
+      while (!finished.get()) {
+        assertNotNull(applications.get(appAttemptId0));
+      }
+      assertFalse(failed.get());
+    }
+
 }
 }

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -72,6 +72,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
@@ -79,6 +80,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -2196,4 +2198,11 @@ public class TestFairScheduler {
     assertEquals(1, app.getLiveContainers().size());
     assertEquals(1, app.getLiveContainers().size());
   }
   }
 
 
+  @Test
+  public void testConcurrentAccessOnApplications() throws Exception {
+    FairScheduler fs = new FairScheduler();
+    TestCapacityScheduler.verifyConcurrentAccessOnApplications(
+        fs.applications, FSSchedulerApp.class);
+  }
+
 }
 }

+ 11 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -51,13 +51,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -414,7 +416,14 @@ public class TestFifoScheduler {
     
     
     LOG.info("--- END: testFifoScheduler ---");
     LOG.info("--- END: testFifoScheduler ---");
   }
   }
-  
+
+  @Test
+  public void testConcurrentAccessOnApplications() throws Exception {
+    FifoScheduler fs = new FifoScheduler();
+    TestCapacityScheduler.verifyConcurrentAccessOnApplications(
+        fs.applications, FiCaSchedulerApp.class);
+  }
+
   private void checkApplicationResourceUsage(int expected, 
   private void checkApplicationResourceUsage(int expected, 
       Application application) {
       Application application) {
     Assert.assertEquals(expected, application.getUsedResources().getMemory());
     Assert.assertEquals(expected, application.getUsedResources().getMemory());