YARN-3141. Improve locks in 
SchedulerApplicationAttempt/FSAppAttempt/FiCaSchedulerApp. Contributed by 
Wangda Tan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b8a30f2f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b8a30f2f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b8a30f2f

Branch: refs/heads/HADOOP-12756
Commit: b8a30f2f170ffbd590e7366c3c944ab4919e40df
Parents: ea29e3b
Author: Jian He <jia...@apache.org>
Authored: Mon Sep 19 16:58:39 2016 +0800
Committer: Jian He <jia...@apache.org>
Committed: Mon Sep 19 17:08:01 2016 +0800

----------------------------------------------------------------------
 .../scheduler/SchedulerApplicationAttempt.java  | 744 +++++++++++--------
 .../allocator/RegularContainerAllocator.java    |   2 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java | 418 ++++++-----
 .../scheduler/fair/FSAppAttempt.java            | 465 ++++++------
 4 files changed, 922 insertions(+), 707 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 97d29cf..adc3a97 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -26,8 +26,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.collect.ConcurrentHashMultiset;
 import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.commons.logging.Log;
@@ -71,8 +74,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.Multiset;
 
 /**
  * Represents an application attempt from the viewpoint of the scheduler.
@@ -97,14 +98,14 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
   protected final AppSchedulingInfo appSchedulingInfo;
   protected ApplicationAttemptId attemptId;
   protected Map<ContainerId, RMContainer> liveContainers =
-      new HashMap<ContainerId, RMContainer>();
+      new ConcurrentHashMap<>();
   protected final Map<SchedulerRequestKey, Map<NodeId, RMContainer>>
       reservedContainers = new HashMap<>();
 
-  private final Multiset<SchedulerRequestKey> reReservations =
-      HashMultiset.create();
+  private final ConcurrentHashMultiset<SchedulerRequestKey> reReservations =
+      ConcurrentHashMultiset.create();
   
-  private Resource resourceLimit = Resource.newInstance(0, 0);
+  private volatile Resource resourceLimit = Resource.newInstance(0, 0);
   private boolean unmanagedAM = true;
   private boolean amRunning = false;
   private LogAggregationContext logAggregationContext;
@@ -138,8 +139,9 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
    * the application successfully schedules a task (at rack or node local), it
    * is reset to 0.
    */
-  Multiset<SchedulerRequestKey> schedulingOpportunities = 
HashMultiset.create();
-  
+  private ConcurrentHashMultiset<SchedulerRequestKey> schedulingOpportunities =
+      ConcurrentHashMultiset.create();
+
   /**
    * Count how many times the application has been given an opportunity to
    * schedule a non-partitioned resource request at each priority. Each time 
the
@@ -147,15 +149,16 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
    * incremented, and each time the application successfully schedules a task,
    * it is reset to 0 when schedule any task at corresponding priority.
    */
-  Multiset<SchedulerRequestKey> missedNonPartitionedReqSchedulingOpportunity =
-      HashMultiset.create();
+  private ConcurrentHashMultiset<SchedulerRequestKey>
+      missedNonPartitionedReqSchedulingOpportunity =
+      ConcurrentHashMultiset.create();
   
   // Time of the last container scheduled at the current allowed level
   protected Map<SchedulerRequestKey, Long> lastScheduledContainer =
-      new HashMap<>();
+      new ConcurrentHashMap<>();
 
-  protected Queue queue;
-  protected boolean isStopped = false;
+  protected volatile Queue queue;
+  protected volatile boolean isStopped = false;
 
   protected String appAMNodePartitionName = CommonNodeLabelsManager.NO_LABEL;
 
@@ -163,6 +166,9 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
 
   private RMAppAttempt appAttempt;
 
+  protected ReentrantReadWriteLock.ReadLock readLock;
+  protected ReentrantReadWriteLock.WriteLock writeLock;
+
   public SchedulerApplicationAttempt(ApplicationAttemptId 
applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
       RMContext rmContext) {
@@ -188,14 +194,23 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
             appSubmissionContext.getLogAggregationContext();
       }
     }
+
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
   }
   
   /**
    * Get the live containers of the application.
    * @return live containers of the application
    */
-  public synchronized Collection<RMContainer> getLiveContainers() {
-    return new ArrayList<RMContainer>(liveContainers.values());
+  public Collection<RMContainer> getLiveContainers() {
+    try {
+      readLock.lock();
+      return new ArrayList<>(liveContainers.values());
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public AppSchedulingInfo getAppSchedulingInfo() {
@@ -243,20 +258,36 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return appSchedulingInfo.getSchedulerKeys();
   }
   
-  public synchronized ResourceRequest getResourceRequest(
+  public ResourceRequest getResourceRequest(
       SchedulerRequestKey schedulerKey, String resourceName) {
-    return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
+    try {
+      readLock.lock();
+      return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
+    } finally {
+      readLock.unlock();
+    }
+
   }
 
-  public synchronized int getTotalRequiredResources(
+  public int getTotalRequiredResources(
       SchedulerRequestKey schedulerKey) {
-    ResourceRequest request =
-        getResourceRequest(schedulerKey, ResourceRequest.ANY);
-    return request == null ? 0 : request.getNumContainers();
+    try {
+      readLock.lock();
+      ResourceRequest request =
+          getResourceRequest(schedulerKey, ResourceRequest.ANY);
+      return request == null ? 0 : request.getNumContainers();
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public synchronized Resource getResource(SchedulerRequestKey schedulerKey) {
-    return appSchedulingInfo.getResource(schedulerKey);
+  public Resource getResource(SchedulerRequestKey schedulerKey) {
+    try {
+      readLock.lock();
+      return appSchedulingInfo.getResource(schedulerKey);
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public String getQueueName() {
@@ -291,38 +322,48 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return unmanagedAM;
   }
 
-  public synchronized RMContainer getRMContainer(ContainerId id) {
+  public RMContainer getRMContainer(ContainerId id) {
     return liveContainers.get(id);
   }
 
-  public synchronized void addRMContainer(
+  public void addRMContainer(
       ContainerId id, RMContainer rmContainer) {
-    liveContainers.put(id, rmContainer);
-    if (rmContainer.isRemotelyAllocated()) {
-      this.attemptResourceUsageAllocatedRemotely.incUsed(
-          rmContainer.getAllocatedResource());
+    try {
+      writeLock.lock();
+      liveContainers.put(id, rmContainer);
+      if (rmContainer.isRemotelyAllocated()) {
+        this.attemptResourceUsageAllocatedRemotely.incUsed(
+            rmContainer.getAllocatedResource());
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
 
-  public synchronized void removeRMContainer(ContainerId containerId) {
-    RMContainer rmContainer = liveContainers.remove(containerId);
-    if (rmContainer != null && rmContainer.isRemotelyAllocated()) {
-      this.attemptResourceUsageAllocatedRemotely.decUsed(
-          rmContainer.getAllocatedResource());
+  public void removeRMContainer(ContainerId containerId) {
+    try {
+      writeLock.lock();
+      RMContainer rmContainer = liveContainers.remove(containerId);
+      if (rmContainer != null && rmContainer.isRemotelyAllocated()) {
+        this.attemptResourceUsageAllocatedRemotely.decUsed(
+            rmContainer.getAllocatedResource());
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
 
-  protected synchronized void resetReReservations(
+  protected void resetReReservations(
       SchedulerRequestKey schedulerKey) {
     reReservations.setCount(schedulerKey, 0);
   }
 
-  protected synchronized void addReReservation(
+  protected void addReReservation(
       SchedulerRequestKey schedulerKey) {
     reReservations.add(schedulerKey);
   }
 
-  public synchronized int getReReservations(SchedulerRequestKey schedulerKey) {
+  public int getReReservations(SchedulerRequestKey schedulerKey) {
     return reReservations.count(schedulerKey);
   }
 
@@ -333,7 +374,7 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
    */
   @Stable
   @Private
-  public synchronized Resource getCurrentReservation() {
+  public Resource getCurrentReservation() {
     return attemptResourceUsage.getReserved();
   }
   
@@ -341,28 +382,43 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return queue;
   }
   
-  public synchronized boolean updateResourceRequests(
+  public boolean updateResourceRequests(
       List<ResourceRequest> requests) {
-    if (!isStopped) {
-      return appSchedulingInfo.updateResourceRequests(requests, false);
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        return appSchedulingInfo.updateResourceRequests(requests, false);
+      }
+      return false;
+    } finally {
+      writeLock.unlock();
     }
-    return false;
   }
   
-  public synchronized void recoverResourceRequestsForContainer(
+  public void recoverResourceRequestsForContainer(
       List<ResourceRequest> requests) {
-    if (!isStopped) {
-      appSchedulingInfo.updateResourceRequests(requests, true);
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        appSchedulingInfo.updateResourceRequests(requests, true);
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
   
-  public synchronized void stop(RMAppAttemptState rmAppAttemptFinalState) {
-    // Cleanup all scheduling information
-    isStopped = true;
-    appSchedulingInfo.stop();
+  public void stop(RMAppAttemptState rmAppAttemptFinalState) {
+    try {
+      writeLock.lock();
+      // Cleanup all scheduling information
+      isStopped = true;
+      appSchedulingInfo.stop();
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  public synchronized boolean isStopped() {
+  public boolean isStopped() {
     return isStopped;
   }
 
@@ -370,29 +426,40 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
    * Get the list of reserved containers
    * @return All of the reserved containers.
    */
-  public synchronized List<RMContainer> getReservedContainers() {
-    List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
-    for (Map.Entry<SchedulerRequestKey, Map<NodeId, RMContainer>> e :
-      this.reservedContainers.entrySet()) {
-      reservedContainers.addAll(e.getValue().values());
+  public List<RMContainer> getReservedContainers() {
+    List<RMContainer> list = new ArrayList<>();
+    try {
+      readLock.lock();
+      for (Entry<SchedulerRequestKey, Map<NodeId, RMContainer>> e :
+          this.reservedContainers.entrySet()) {
+        list.addAll(e.getValue().values());
+      }
+      return list;
+    } finally {
+      readLock.unlock();
     }
-    return reservedContainers;
+
   }
   
-  public synchronized boolean reserveIncreasedContainer(SchedulerNode node,
+  public boolean reserveIncreasedContainer(SchedulerNode node,
       SchedulerRequestKey schedulerKey, RMContainer rmContainer,
       Resource reservedResource) {
-    if (commonReserve(node, schedulerKey, rmContainer, reservedResource)) {
-      attemptResourceUsage.incReserved(node.getPartition(),
-          reservedResource);
-      // succeeded
-      return true;
+    try {
+      writeLock.lock();
+      if (commonReserve(node, schedulerKey, rmContainer, reservedResource)) {
+        attemptResourceUsage.incReserved(node.getPartition(), 
reservedResource);
+        // succeeded
+        return true;
+      }
+
+      return false;
+    } finally {
+      writeLock.unlock();
     }
-    
-    return false;
+
   }
   
-  private synchronized boolean commonReserve(SchedulerNode node,
+  private boolean commonReserve(SchedulerNode node,
       SchedulerRequestKey schedulerKey, RMContainer rmContainer,
       Resource reservedResource) {
     try {
@@ -423,101 +490,100 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return true;
   }
   
-  public synchronized RMContainer reserve(SchedulerNode node,
+  public RMContainer reserve(SchedulerNode node,
       SchedulerRequestKey schedulerKey, RMContainer rmContainer,
       Container container) {
-    // Create RMContainer if necessary
-    if (rmContainer == null) {
-      rmContainer =
-          new RMContainerImpl(container, getApplicationAttemptId(),
-              node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
-      attemptResourceUsage.incReserved(node.getPartition(),
-          container.getResource());
-      ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
-
-      // Reset the re-reservation count
-      resetReReservations(schedulerKey);
-    } else {
-      // Note down the re-reservation
-      addReReservation(schedulerKey);
-    }
-    
-    commonReserve(node, schedulerKey, rmContainer, container.getResource());
+    try {
+      writeLock.lock();
+      // Create RMContainer if necessary
+      if (rmContainer == null) {
+        rmContainer = new RMContainerImpl(container, getApplicationAttemptId(),
+            node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
+        attemptResourceUsage.incReserved(node.getPartition(),
+            container.getResource());
+        ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
+
+        // Reset the re-reservation count
+        resetReReservations(schedulerKey);
+      } else{
+        // Note down the re-reservation
+        addReReservation(schedulerKey);
+      }
 
-    return rmContainer;
-  }
-  
-  /**
-   * Has the application reserved the given <code>node</code> at the
-   * given <code>priority</code>?
-   * @param node node to be checked
-   * @param schedulerKey scheduler key  of reserved container
-   * @return true is reserved, false if not
-   */
-  public synchronized boolean isReserved(SchedulerNode node,
-      SchedulerRequestKey schedulerKey) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(schedulerKey);
-    if (reservedContainers != null) {
-      return reservedContainers.containsKey(node.getNodeID());
+      commonReserve(node, schedulerKey, rmContainer, container.getResource());
+
+      return rmContainer;
+    } finally {
+      writeLock.unlock();
     }
-    return false;
+
   }
-  
-  public synchronized void setHeadroom(Resource globalLimit) {
-    this.resourceLimit = globalLimit; 
+
+  public void setHeadroom(Resource globalLimit) {
+    this.resourceLimit = Resources.componentwiseMax(globalLimit,
+        Resources.none());
   }
 
   /**
    * Get available headroom in terms of resources for the application's user.
    * @return available resource headroom
    */
-  public synchronized Resource getHeadroom() {
-    // Corner case to deal with applications being slightly over-limit
-    if (resourceLimit.getMemorySize() < 0) {
-      resourceLimit.setMemorySize(0);
-    }
-    
+  public Resource getHeadroom() {
     return resourceLimit;
   }
   
-  public synchronized int getNumReservedContainers(
+  public int getNumReservedContainers(
       SchedulerRequestKey schedulerKey) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(schedulerKey);
-    return (reservedContainers == null) ? 0 : reservedContainers.size();
+    try {
+      readLock.lock();
+      Map<NodeId, RMContainer> map = this.reservedContainers.get(
+          schedulerKey);
+      return (map == null) ? 0 : map.size();
+    } finally {
+      readLock.unlock();
+    }
   }
   
   @SuppressWarnings("unchecked")
-  public synchronized void containerLaunchedOnNode(ContainerId containerId,
+  public void containerLaunchedOnNode(ContainerId containerId,
       NodeId nodeId) {
-    // Inform the container
-    RMContainer rmContainer = getRMContainer(containerId);
-    if (rmContainer == null) {
-      // Some unknown container sneaked into the system. Kill it.
-      rmContext.getDispatcher().getEventHandler()
-        .handle(new RMNodeCleanContainerEvent(nodeId, containerId));
-      return;
-    }
+    try {
+      writeLock.lock();
+      // Inform the container
+      RMContainer rmContainer = getRMContainer(containerId);
+      if (rmContainer == null) {
+        // Some unknown container sneaked into the system. Kill it.
+        rmContext.getDispatcher().getEventHandler().handle(
+            new RMNodeCleanContainerEvent(nodeId, containerId));
+        return;
+      }
 
-    rmContainer.handle(new RMContainerEvent(containerId,
-        RMContainerEventType.LAUNCHED));
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  public synchronized void showRequests() {
+  public void showRequests() {
     if (LOG.isDebugEnabled()) {
-      for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
-        Map<String, ResourceRequest> requests =
-            getResourceRequests(schedulerKey);
-        if (requests != null) {
-          LOG.debug("showRequests:" + " application=" + getApplicationId()
-              + " headRoom=" + getHeadroom() + " currentConsumption="
-              + attemptResourceUsage.getUsed().getMemorySize());
-          for (ResourceRequest request : requests.values()) {
+      try {
+        readLock.lock();
+        for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
+          Map<String, ResourceRequest> requests = getResourceRequests(
+              schedulerKey);
+          if (requests != null) {
             LOG.debug("showRequests:" + " application=" + getApplicationId()
-                + " request=" + request);
+                + " headRoom=" + getHeadroom() + " currentConsumption="
+                + attemptResourceUsage.getUsed().getMemorySize());
+            for (ResourceRequest request : requests.values()) {
+              LOG.debug("showRequests:" + " application=" + getApplicationId()
+                  + " request=" + request);
+            }
           }
         }
+      } finally {
+        readLock.unlock();
       }
     }
   }
@@ -572,54 +638,75 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
   // Create container token and update NMToken altogether, if either of them 
fails for
   // some reason like DNS unavailable, do not return this container and keep it
   // in the newlyAllocatedContainers waiting to be refetched.
-  public synchronized List<Container> pullNewlyAllocatedContainers() {
-    List<Container> returnContainerList =
-        new ArrayList<Container>(newlyAllocatedContainers.size());
-    for (Iterator<RMContainer> i = newlyAllocatedContainers.iterator(); i
-        .hasNext();) {
-      RMContainer rmContainer = i.next();
-      Container updatedContainer =
-          updateContainerAndNMToken(rmContainer, true, false);
-      // Only add container to return list when it's not null. updatedContainer
-      // could be null when generate token failed, it can be caused by DNS
-      // resolving failed.
-      if (updatedContainer != null) {
-        returnContainerList.add(updatedContainer);
-        i.remove();
+  public List<Container> pullNewlyAllocatedContainers() {
+    try {
+      writeLock.lock();
+      List<Container> returnContainerList = new ArrayList<Container>(
+          newlyAllocatedContainers.size());
+
+      Iterator<RMContainer> i = newlyAllocatedContainers.iterator();
+      while (i.hasNext()) {
+        RMContainer rmContainer = i.next();
+        Container updatedContainer = updateContainerAndNMToken(rmContainer,
+            true, false);
+        // Only add container to return list when it's not null.
+        // updatedContainer could be null when generate token failed, it can be
+        // caused by DNS resolving failed.
+        if (updatedContainer != null) {
+          returnContainerList.add(updatedContainer);
+          i.remove();
+        }
       }
+      return returnContainerList;
+    } finally {
+      writeLock.unlock();
     }
-    return returnContainerList;
+
   }
   
-  private synchronized List<Container> pullNewlyUpdatedContainers(
+  private List<Container> pullNewlyUpdatedContainers(
       Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
-    List<Container> returnContainerList =
-        new ArrayList<Container>(updatedContainerMap.size());
-    for (Iterator<Entry<ContainerId, RMContainer>> i =
-        updatedContainerMap.entrySet().iterator(); i.hasNext();) {
-      RMContainer rmContainer = i.next().getValue();
-      Container updatedContainer =
-          updateContainerAndNMToken(rmContainer, false, increase);
-      if (updatedContainer != null) {
-        returnContainerList.add(updatedContainer);
-        i.remove();
+    try {
+      writeLock.lock();
+      List <Container> returnContainerList = new ArrayList <Container>(
+          updatedContainerMap.size());
+
+      Iterator<Entry<ContainerId, RMContainer>> i =
+          updatedContainerMap.entrySet().iterator();
+      while (i.hasNext()) {
+        RMContainer rmContainer = i.next().getValue();
+        Container updatedContainer = updateContainerAndNMToken(rmContainer,
+            false, increase);
+        if (updatedContainer != null) {
+          returnContainerList.add(updatedContainer);
+          i.remove();
+        }
       }
+      return returnContainerList;
+    } finally {
+      writeLock.unlock();
     }
-    return returnContainerList;
+
   }
 
-  public synchronized List<Container> pullNewlyIncreasedContainers() {
+  public List<Container> pullNewlyIncreasedContainers() {
     return pullNewlyUpdatedContainers(newlyIncreasedContainers, true);
   }
   
-  public synchronized List<Container> pullNewlyDecreasedContainers() {
+  public List<Container> pullNewlyDecreasedContainers() {
     return pullNewlyUpdatedContainers(newlyDecreasedContainers, false);
   }
   
-  public synchronized List<NMToken> pullUpdatedNMTokens() {
-    List<NMToken> returnList = new ArrayList<NMToken>(updatedNMTokens);
-    updatedNMTokens.clear();
-    return returnList;
+  public List<NMToken> pullUpdatedNMTokens() {
+    try {
+      writeLock.lock();
+      List <NMToken> returnList = new ArrayList<>(updatedNMTokens);
+      updatedNMTokens.clear();
+      return returnList;
+    } finally {
+      writeLock.unlock();
+    }
+
   }
 
   public boolean isWaitingForAMContainer() {
@@ -628,53 +715,63 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return (!unmanagedAM && appAttempt.getMasterContainer() == null);
   }
 
-  public synchronized void updateBlacklist(List<String> blacklistAdditions,
+  public void updateBlacklist(List<String> blacklistAdditions,
       List<String> blacklistRemovals) {
-    if (!isStopped) {
-      if (isWaitingForAMContainer()) {
-        // The request is for the AM-container, and the AM-container is 
launched
-        // by the system. So, update the places that are blacklisted by system
-        // (as opposed to those blacklisted by the application).
-        this.appSchedulingInfo.updatePlacesBlacklistedBySystem(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        this.appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
-            blacklistRemovals);
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        if (isWaitingForAMContainer()) {
+          // The request is for the AM-container, and the AM-container is
+          // launched by the system. So, update the places that are blacklisted
+          // by system (as opposed to those blacklisted by the application).
+          this.appSchedulingInfo.updatePlacesBlacklistedBySystem(
+              blacklistAdditions, blacklistRemovals);
+        } else{
+          this.appSchedulingInfo.updatePlacesBlacklistedByApp(
+              blacklistAdditions, blacklistRemovals);
+        }
       }
+    } finally {
+      writeLock.unlock();
     }
   }
 
   public boolean isPlaceBlacklisted(String resourceName) {
-    boolean forAMContainer = isWaitingForAMContainer();
-    return this.appSchedulingInfo.isPlaceBlacklisted(resourceName,
-      forAMContainer);
+    try {
+      readLock.lock();
+      boolean forAMContainer = isWaitingForAMContainer();
+      return this.appSchedulingInfo.isPlaceBlacklisted(resourceName,
+          forAMContainer);
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(
+  public int addMissedNonPartitionedRequestSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
-    missedNonPartitionedReqSchedulingOpportunity.add(schedulerKey);
-    return missedNonPartitionedReqSchedulingOpportunity.count(schedulerKey);
+    return missedNonPartitionedReqSchedulingOpportunity.add(
+        schedulerKey, 1) + 1;
   }
 
-  public synchronized void
+  public void
       resetMissedNonPartitionedRequestSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
     missedNonPartitionedReqSchedulingOpportunity.setCount(schedulerKey, 0);
   }
 
   
-  public synchronized void addSchedulingOpportunity(
+  public void addSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
-    int count = schedulingOpportunities.count(schedulerKey);
-    if (count < Integer.MAX_VALUE) {
-      schedulingOpportunities.setCount(schedulerKey, count + 1);
+    try {
+      schedulingOpportunities.add(schedulerKey, 1);
+    } catch (IllegalArgumentException e) {
+      // This happens when count = MAX_INT, ignore the exception
     }
   }
   
-  public synchronized void subtractSchedulingOpportunity(
+  public void subtractSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
-    int count = schedulingOpportunities.count(schedulerKey) - 1;
-    this.schedulingOpportunities.setCount(schedulerKey, Math.max(count,  0));
+    this.schedulingOpportunities.removeExactly(schedulerKey, 1);
   }
 
   /**
@@ -684,7 +781,7 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
    * @param schedulerKey Scheduler Key
    * @return number of scheduling opportunities
    */
-  public synchronized int getSchedulingOpportunities(
+  public int getSchedulingOpportunities(
       SchedulerRequestKey schedulerKey) {
     return schedulingOpportunities.count(schedulerKey);
   }
@@ -696,16 +793,22 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
    *
    * @param schedulerKey The priority of the container scheduled.
    */
-  public synchronized void resetSchedulingOpportunities(
+  public void resetSchedulingOpportunities(
       SchedulerRequestKey schedulerKey) {
     resetSchedulingOpportunities(schedulerKey, System.currentTimeMillis());
   }
 
   // used for continuous scheduling
-  public synchronized void resetSchedulingOpportunities(
+  public void resetSchedulingOpportunities(
       SchedulerRequestKey schedulerKey, long currentTimeMs) {
-    lastScheduledContainer.put(schedulerKey, currentTimeMs);
-    schedulingOpportunities.setCount(schedulerKey, 0);
+    try {
+      writeLock.lock();
+      lastScheduledContainer.put(schedulerKey, currentTimeMs);
+      schedulingOpportunities.setCount(schedulerKey, 0);
+    } finally {
+      writeLock.unlock();
+    }
+
   }
 
   @VisibleForTesting
@@ -713,7 +816,7 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     schedulingOpportunities.setCount(schedulerKey, count);
   }
 
-  synchronized AggregateAppResourceUsage getRunningAggregateAppResourceUsage() 
{
+  private AggregateAppResourceUsage getRunningAggregateAppResourceUsage() {
     long currentTimeMillis = System.currentTimeMillis();
     // Don't walk the whole container list if the resources were computed
     // recently.
@@ -737,101 +840,120 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return new AggregateAppResourceUsage(lastMemorySeconds, lastVcoreSeconds);
   }
 
-  public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
-    AggregateAppResourceUsage runningResourceUsage =
-        getRunningAggregateAppResourceUsage();
-    Resource usedResourceClone =
-        Resources.clone(attemptResourceUsage.getAllUsed());
-    Resource reservedResourceClone =
-        Resources.clone(attemptResourceUsage.getReserved());
-    Resource cluster = rmContext.getScheduler().getClusterResource();
-    ResourceCalculator calc = rmContext.getScheduler().getResourceCalculator();
-    float queueUsagePerc = 0.0f;
-    float clusterUsagePerc = 0.0f;
-    if (!calc.isInvalidDivisor(cluster)) {
-      queueUsagePerc =
-          calc.divide(cluster, usedResourceClone, Resources.multiply(cluster,
-              queue.getQueueInfo(false, false).getCapacity())) * 100;
-      clusterUsagePerc = calc.divide(cluster, usedResourceClone, cluster) * 
100;
+  public ApplicationResourceUsageReport getResourceUsageReport() {
+    try {
+      writeLock.lock();
+      AggregateAppResourceUsage runningResourceUsage =
+          getRunningAggregateAppResourceUsage();
+      Resource usedResourceClone = Resources.clone(
+          attemptResourceUsage.getAllUsed());
+      Resource reservedResourceClone = Resources.clone(
+          attemptResourceUsage.getReserved());
+      Resource cluster = rmContext.getScheduler().getClusterResource();
+      ResourceCalculator calc =
+          rmContext.getScheduler().getResourceCalculator();
+      float queueUsagePerc = 0.0f;
+      float clusterUsagePerc = 0.0f;
+      if (!calc.isInvalidDivisor(cluster)) {
+        queueUsagePerc = calc.divide(cluster, usedResourceClone, Resources
+            .multiply(cluster, queue.getQueueInfo(false, false).getCapacity()))
+            * 100;
+        clusterUsagePerc = calc.divide(cluster, usedResourceClone, cluster)
+            * 100;
+      }
+      return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
+          reservedContainers.size(), usedResourceClone, reservedResourceClone,
+          Resources.add(usedResourceClone, reservedResourceClone),
+          runningResourceUsage.getMemorySeconds(),
+          runningResourceUsage.getVcoreSeconds(), queueUsagePerc,
+          clusterUsagePerc);
+    } finally {
+      writeLock.unlock();
     }
-    return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
-        reservedContainers.size(), usedResourceClone, reservedResourceClone,
-        Resources.add(usedResourceClone, reservedResourceClone),
-        runningResourceUsage.getMemorySeconds(),
-        runningResourceUsage.getVcoreSeconds(), queueUsagePerc,
-        clusterUsagePerc);
   }
 
-  public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {
+  @VisibleForTesting
+  public Map<ContainerId, RMContainer> getLiveContainersMap() {
     return this.liveContainers;
   }
 
-  public synchronized Resource getResourceLimit() {
-    return this.resourceLimit;
-  }
-
-  public synchronized Map<SchedulerRequestKey, Long>
+  public Map<SchedulerRequestKey, Long>
       getLastScheduledContainer() {
     return this.lastScheduledContainer;
   }
 
-  public synchronized void transferStateFromPreviousAttempt(
+  public void transferStateFromPreviousAttempt(
       SchedulerApplicationAttempt appAttempt) {
-    this.liveContainers = appAttempt.getLiveContainersMap();
-    // this.reReservations = appAttempt.reReservations;
-    this.attemptResourceUsage.copyAllUsed(appAttempt.attemptResourceUsage);
-    this.resourceLimit = appAttempt.getResourceLimit();
-    // this.currentReservation = appAttempt.currentReservation;
-    // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
-    // this.schedulingOpportunities = appAttempt.schedulingOpportunities;
-    this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
-    this.appSchedulingInfo
-      
.transferStateFromPreviousAppSchedulingInfo(appAttempt.appSchedulingInfo);
+    try {
+      writeLock.lock();
+      this.liveContainers = appAttempt.getLiveContainersMap();
+      // this.reReservations = appAttempt.reReservations;
+      this.attemptResourceUsage.copyAllUsed(appAttempt.attemptResourceUsage);
+      this.setHeadroom(appAttempt.resourceLimit);
+      // this.currentReservation = appAttempt.currentReservation;
+      // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
+      // this.schedulingOpportunities = appAttempt.schedulingOpportunities;
+      this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
+      this.appSchedulingInfo.transferStateFromPreviousAppSchedulingInfo(
+          appAttempt.appSchedulingInfo);
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  public synchronized void move(Queue newQueue) {
-    QueueMetrics oldMetrics = queue.getMetrics();
-    QueueMetrics newMetrics = newQueue.getMetrics();
-    String newQueueName = newQueue.getQueueName();
-    String user = getUser();
-    for (RMContainer liveContainer : liveContainers.values()) {
-      Resource resource = liveContainer.getContainer().getResource();
-      ((RMContainerImpl)liveContainer).setQueueName(newQueueName);
-      oldMetrics.releaseResources(user, 1, resource);
-      newMetrics.allocateResources(user, 1, resource, false);
-    }
-    for (Map<NodeId, RMContainer> map : reservedContainers.values()) {
-      for (RMContainer reservedContainer : map.values()) {
-        ((RMContainerImpl)reservedContainer).setQueueName(newQueueName);
-        Resource resource = reservedContainer.getReservedResource();
-        oldMetrics.unreserveResource(user, resource);
-        newMetrics.reserveResource(user, resource);
+  public void move(Queue newQueue) {
+    try {
+      writeLock.lock();
+      QueueMetrics oldMetrics = queue.getMetrics();
+      QueueMetrics newMetrics = newQueue.getMetrics();
+      String newQueueName = newQueue.getQueueName();
+      String user = getUser();
+      for (RMContainer liveContainer : liveContainers.values()) {
+        Resource resource = liveContainer.getContainer().getResource();
+        ((RMContainerImpl) liveContainer).setQueueName(newQueueName);
+        oldMetrics.releaseResources(user, 1, resource);
+        newMetrics.allocateResources(user, 1, resource, false);
+      }
+      for (Map<NodeId, RMContainer> map : reservedContainers.values()) {
+        for (RMContainer reservedContainer : map.values()) {
+          ((RMContainerImpl) reservedContainer).setQueueName(newQueueName);
+          Resource resource = reservedContainer.getReservedResource();
+          oldMetrics.unreserveResource(user, resource);
+          newMetrics.reserveResource(user, resource);
+        }
       }
-    }
 
-    appSchedulingInfo.move(newQueue);
-    this.queue = newQueue;
+      appSchedulingInfo.move(newQueue);
+      this.queue = newQueue;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  public synchronized void recoverContainer(SchedulerNode node,
+  public void recoverContainer(SchedulerNode node,
       RMContainer rmContainer) {
-    // recover app scheduling info
-    appSchedulingInfo.recoverContainer(rmContainer);
+    try {
+      writeLock.lock();
+      // recover app scheduling info
+      appSchedulingInfo.recoverContainer(rmContainer);
 
-    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
-      return;
+      if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+        return;
+      }
+      LOG.info("SchedulerAttempt " + getApplicationAttemptId()
+          + " is recovering container " + rmContainer.getContainerId());
+      liveContainers.put(rmContainer.getContainerId(), rmContainer);
+      attemptResourceUsage.incUsed(node.getPartition(),
+          rmContainer.getContainer().getResource());
+
+      // resourceLimit: updated when 
LeafQueue#recoverContainer#allocateResource
+      // is called.
+      // newlyAllocatedContainers.add(rmContainer);
+      // schedulingOpportunities
+      // lastScheduledContainer
+    } finally {
+      writeLock.unlock();
     }
-    LOG.info("SchedulerAttempt " + getApplicationAttemptId()
-      + " is recovering container " + rmContainer.getContainerId());
-    liveContainers.put(rmContainer.getContainerId(), rmContainer);
-    attemptResourceUsage.incUsed(node.getPartition(), rmContainer
-        .getContainer().getResource());
-    
-    // resourceLimit: updated when LeafQueue#recoverContainer#allocateResource
-    // is called.
-    // newlyAllocatedContainers.add(rmContainer);
-    // schedulingOpportunities
-    // lastScheduledContainer
   }
 
   public void incNumAllocatedContainers(NodeType containerType,
@@ -915,49 +1037,64 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     return attemptResourceUsage;
   }
   
-  public synchronized boolean removeIncreaseRequest(NodeId nodeId,
+  public boolean removeIncreaseRequest(NodeId nodeId,
       SchedulerRequestKey schedulerKey, ContainerId containerId) {
-    return appSchedulingInfo.removeIncreaseRequest(nodeId, schedulerKey,
-        containerId);
+    try {
+      writeLock.lock();
+      return appSchedulingInfo.removeIncreaseRequest(nodeId, schedulerKey,
+          containerId);
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  public synchronized boolean updateIncreaseRequests(
+  public boolean updateIncreaseRequests(
       List<SchedContainerChangeRequest> increaseRequests) {
-    return appSchedulingInfo.updateIncreaseRequests(increaseRequests);
+    try {
+      writeLock.lock();
+      return appSchedulingInfo.updateIncreaseRequests(increaseRequests);
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  private synchronized void changeContainerResource(
+  private void changeContainerResource(
       SchedContainerChangeRequest changeRequest, boolean increase) {
-    if (increase) {
-      appSchedulingInfo.increaseContainer(changeRequest);
-    } else {
-      appSchedulingInfo.decreaseContainer(changeRequest);
-    }
+    try {
+      writeLock.lock();
+      if (increase) {
+        appSchedulingInfo.increaseContainer(changeRequest);
+      } else{
+        appSchedulingInfo.decreaseContainer(changeRequest);
+      }
 
-    RMContainer changedRMContainer = changeRequest.getRMContainer(); 
-    changedRMContainer.handle(
-        new RMContainerChangeResourceEvent(changeRequest.getContainerId(),
-            changeRequest.getTargetCapacity(), increase));
-
-    // remove pending and not pulled by AM newly-increased/decreased-containers
-    // and add the new one
-    if (increase) {
-      newlyDecreasedContainers.remove(changeRequest.getContainerId());
-      newlyIncreasedContainers.put(changeRequest.getContainerId(),
-          changedRMContainer);
-    } else {
-      newlyIncreasedContainers.remove(changeRequest.getContainerId());
-      newlyDecreasedContainers.put(changeRequest.getContainerId(),
-          changedRMContainer);
+      RMContainer changedRMContainer = changeRequest.getRMContainer();
+      changedRMContainer.handle(
+          new RMContainerChangeResourceEvent(changeRequest.getContainerId(),
+              changeRequest.getTargetCapacity(), increase));
+
+      // remove pending and not pulled by AM newly-increased or
+      // decreased-containers and add the new one
+      if (increase) {
+        newlyDecreasedContainers.remove(changeRequest.getContainerId());
+        newlyIncreasedContainers.put(changeRequest.getContainerId(),
+            changedRMContainer);
+      } else{
+        newlyIncreasedContainers.remove(changeRequest.getContainerId());
+        newlyDecreasedContainers.put(changeRequest.getContainerId(),
+            changedRMContainer);
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
   
-  public synchronized void decreaseContainer(
+  public void decreaseContainer(
       SchedContainerChangeRequest decreaseRequest) {
     changeContainerResource(decreaseRequest, false);
   }
   
-  public synchronized void increaseContainer(
+  public void increaseContainer(
       SchedContainerChangeRequest increaseRequest) {
     changeContainerResource(increaseRequest, true);
   }
@@ -1025,7 +1162,10 @@ public class SchedulerApplicationAttempt implements 
SchedulableEntity {
     this.isAttemptRecovering = isRecovering;
   }
 
-  public static enum AMState {
+  /**
+   * Different state for Application Master, user can see this state from web 
UI
+   */
+  public enum AMState {
     UNMANAGED("User launched the Application Master, since it's unmanaged. "),
     INACTIVATED("Application is added to the scheduler and is not yet 
activated. "),
     ACTIVATED("Application is Activated, waiting for resources to be assigned 
for AM. "),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index 8d4042c..1a3f71f 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -251,7 +251,7 @@ public class RegularContainerAllocator extends 
AbstractContainerAllocator {
     return result;
   }
   
-  public synchronized float getLocalityWaitFactor(
+  public float getLocalityWaitFactor(
       SchedulerRequestKey schedulerKey, int clusterNodes) {
     // Estimate: Required unique resources (i.e. hosts + racks)
     int requiredResources = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 9c84a23..f40ecd7 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -63,7 +63,6 @@ import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMCont
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -99,7 +98,6 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
    * to hold the message if its app doesn't not get container from a node
    */
   private String appSkipNodeDiagnostics;
-  private CapacitySchedulerContext capacitySchedulerContext;
 
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
@@ -153,118 +151,128 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
 
     containerAllocator = new ContainerAllocator(this, rc, rmContext,
         activitiesManager);
-
-    if (scheduler instanceof CapacityScheduler) {
-      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
-    }
   }
 
-  public synchronized boolean containerCompleted(RMContainer rmContainer,
+  public boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
-    ContainerId containerId = rmContainer.getContainerId();
+    try {
+      writeLock.lock();
+      ContainerId containerId = rmContainer.getContainerId();
 
-    // Remove from the list of containers
-    if (null == liveContainers.remove(containerId)) {
-      return false;
-    }
+      // Remove from the list of containers
+      if (null == liveContainers.remove(containerId)) {
+        return false;
+      }
 
-    // Remove from the list of newly allocated containers if found
-    newlyAllocatedContainers.remove(rmContainer);
+      // Remove from the list of newly allocated containers if found
+      newlyAllocatedContainers.remove(rmContainer);
 
-    // Inform the container
-    rmContainer.handle(
-        new RMContainerFinishedEvent(containerId, containerStatus, event));
+      // Inform the container
+      rmContainer.handle(
+          new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(containerId);
+      containersToPreempt.remove(containerId);
 
-    Resource containerResource = rmContainer.getContainer().getResource();
-    RMAuditLogger.logSuccess(getUser(),
-        AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId, containerResource);
-    
-    // Update usage metrics 
-    queue.getMetrics().releaseResources(getUser(), 1, containerResource);
-    attemptResourceUsage.decUsed(partition, containerResource);
+      Resource containerResource = rmContainer.getContainer().getResource();
+      RMAuditLogger.logSuccess(getUser(), AuditConstants.RELEASE_CONTAINER,
+          "SchedulerApp", getApplicationId(), containerId, containerResource);
+
+      // Update usage metrics
+      queue.getMetrics().releaseResources(getUser(), 1, containerResource);
+      attemptResourceUsage.decUsed(partition, containerResource);
 
-    // Clear resource utilization metrics cache.
-    lastMemoryAggregateAllocationUpdateTime = -1;
+      // Clear resource utilization metrics cache.
+      lastMemoryAggregateAllocationUpdateTime = -1;
 
-    return true;
+      return true;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode 
node,
+  public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       SchedulerRequestKey schedulerKey, ResourceRequest request,
       Container container) {
+    try {
+      writeLock.lock();
 
-    if (isStopped) {
-      return null;
-    }
-    
-    // Required sanity check - AM can call 'allocate' to update resource
-    // request without locking the scheduler, hence we need to check
-    if (getTotalRequiredResources(schedulerKey) <= 0) {
-      return null;
-    }
+      if (isStopped) {
+        return null;
+      }
+
+      // Required sanity check - AM can call 'allocate' to update resource
+      // request without locking the scheduler, hence we need to check
+      if (getTotalRequiredResources(schedulerKey) <= 0) {
+        return null;
+      }
 
-    // Create RMContainer
-    RMContainer rmContainer =
-        new RMContainerImpl(container, this.getApplicationAttemptId(),
-            node.getNodeID(), appSchedulingInfo.getUser(), this.rmContext,
-            request.getNodeLabelExpression());
-    ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
+      // Create RMContainer
+      RMContainer rmContainer = new RMContainerImpl(container,
+          this.getApplicationAttemptId(), node.getNodeID(),
+          appSchedulingInfo.getUser(), this.rmContext,
+          request.getNodeLabelExpression());
+      ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
 
-    updateAMContainerDiagnostics(AMState.ASSIGNED, null);
+      updateAMContainerDiagnostics(AMState.ASSIGNED, null);
 
-    // Add it to allContainers list.
-    newlyAllocatedContainers.add(rmContainer);
+      // Add it to allContainers list.
+      newlyAllocatedContainers.add(rmContainer);
 
-    ContainerId containerId = container.getId();
-    liveContainers.put(containerId, rmContainer);
+      ContainerId containerId = container.getId();
+      liveContainers.put(containerId, rmContainer);
 
-    // Update consumption and track allocations
-    List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
-        type, node, schedulerKey, request, container);
+      // Update consumption and track allocations
+      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+          type, node, schedulerKey, request, container);
 
-    attemptResourceUsage.incUsed(node.getPartition(), container.getResource());
+      attemptResourceUsage.incUsed(node.getPartition(),
+          container.getResource());
 
-    // Update resource requests related to "request" and store in RMContainer
-    ((RMContainerImpl)rmContainer).setResourceRequests(resourceRequestList);
+      // Update resource requests related to "request" and store in RMContainer
+      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
 
-    // Inform the container
-    rmContainer.handle(
-        new RMContainerEvent(containerId, RMContainerEventType.START));
+      // Inform the container
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.START));
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("allocate: applicationAttemptId=" 
-          + containerId.getApplicationAttemptId()
-          + " container=" + containerId + " host="
-          + container.getNodeId().getHost() + " type=" + type);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("allocate: applicationAttemptId=" + containerId
+            .getApplicationAttemptId() + " container=" + containerId + " host="
+            + container.getNodeId().getHost() + " type=" + type);
+      }
+      RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER,
+          "SchedulerApp", getApplicationId(), containerId,
+          container.getResource());
+
+      return rmContainer;
+    } finally {
+      writeLock.unlock();
     }
-    RMAuditLogger.logSuccess(getUser(),
-        AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId, container.getResource());
-    
-    return rmContainer;
   }
 
-  public synchronized boolean unreserve(SchedulerRequestKey schedulerKey,
+  public boolean unreserve(SchedulerRequestKey schedulerKey,
       FiCaSchedulerNode node, RMContainer rmContainer) {
-    // Cancel increase request (if it has reserved increase request 
-    rmContainer.cancelIncreaseReservation();
-    
-    // Done with the reservation?
-    if (internalUnreserve(node, schedulerKey)) {
-      node.unreserveResource(this);
-
-      // Update reserved metrics
-      queue.getMetrics().unreserveResource(getUser(),
-          rmContainer.getReservedResource());
-      queue.decReservedResource(node.getPartition(),
-          rmContainer.getReservedResource());
-      return true;
+    try {
+      writeLock.lock();
+      // Cancel increase request (if it has reserved increase request
+      rmContainer.cancelIncreaseReservation();
+
+      // Done with the reservation?
+      if (internalUnreserve(node, schedulerKey)) {
+        node.unreserveResource(this);
+
+        // Update reserved metrics
+        queue.getMetrics().unreserveResource(getUser(),
+            rmContainer.getReservedResource());
+        queue.decReservedResource(node.getPartition(),
+            rmContainer.getReservedResource());
+        return true;
+      }
+      return false;
+    } finally {
+      writeLock.unlock();
     }
-    return false;
   }
 
   private boolean internalUnreserve(FiCaSchedulerNode node,
@@ -303,33 +311,15 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
     return false;
   }
 
-  public synchronized float getLocalityWaitFactor(
-      SchedulerRequestKey schedulerKey, int clusterNodes) {
-    // Estimate: Required unique resources (i.e. hosts + racks)
-    int requiredResources = 
-        Math.max(this.getResourceRequests(schedulerKey).size() - 1, 0);
-    
-    // waitFactor can't be more than '1' 
-    // i.e. no point skipping more than clustersize opportunities
-    return Math.min(((float)requiredResources / clusterNodes), 1.0f);
-  }
-
-  public synchronized Resource getTotalPendingRequests() {
-    Resource ret = Resource.newInstance(0, 0);
-    for (ResourceRequest rr : appSchedulingInfo.getAllResourceRequests()) {
-      // to avoid double counting we count only "ANY" resource requests
-      if (ResourceRequest.isAnyLocation(rr.getResourceName())){
-        Resources.addTo(ret,
-            Resources.multiply(rr.getCapability(), rr.getNumContainers()));
+  public void markContainerForPreemption(ContainerId cont) {
+    try {
+      writeLock.lock();
+      // ignore already completed containers
+      if (liveContainers.containsKey(cont)) {
+        containersToPreempt.add(cont);
       }
-    }
-    return ret;
-  }
-
-  public synchronized void markContainerForPreemption(ContainerId cont) {
-    // ignore already completed containers
-    if (liveContainers.containsKey(cont)) {
-      containersToPreempt.add(cont);
+    } finally {
+      writeLock.unlock();
     }
   }
 
@@ -343,94 +333,115 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
    * @param minimumAllocation
    * @return an allocation
    */
-  public synchronized Allocation getAllocation(ResourceCalculator rc,
+  public Allocation getAllocation(ResourceCalculator resourceCalculator,
       Resource clusterResource, Resource minimumAllocation) {
-
-    Set<ContainerId> currentContPreemption = Collections.unmodifiableSet(
-        new HashSet<ContainerId>(containersToPreempt));
-    containersToPreempt.clear();
-    Resource tot = Resource.newInstance(0, 0);
-    for(ContainerId c : currentContPreemption){
-      Resources.addTo(tot,
-          liveContainers.get(c).getContainer().getResource());
+    try {
+      writeLock.lock();
+      Set<ContainerId> currentContPreemption = Collections.unmodifiableSet(
+          new HashSet<ContainerId>(containersToPreempt));
+      containersToPreempt.clear();
+      Resource tot = Resource.newInstance(0, 0);
+      for (ContainerId c : currentContPreemption) {
+        Resources.addTo(tot, liveContainers.get(c).getContainer()
+            .getResource());
+      }
+      int numCont = (int) Math.ceil(
+          Resources.divide(rc, clusterResource, tot, minimumAllocation));
+      ResourceRequest rr = ResourceRequest.newInstance(Priority.UNDEFINED,
+          ResourceRequest.ANY, minimumAllocation, numCont);
+      List<Container> newlyAllocatedContainers = 
pullNewlyAllocatedContainers();
+      List<Container> newlyIncreasedContainers = 
pullNewlyIncreasedContainers();
+      List<Container> newlyDecreasedContainers = 
pullNewlyDecreasedContainers();
+      List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
+      Resource headroom = getHeadroom();
+      setApplicationHeadroomForMetrics(headroom);
+      return new Allocation(newlyAllocatedContainers, headroom, null,
+          currentContPreemption, Collections.singletonList(rr), 
updatedNMTokens,
+          newlyIncreasedContainers, newlyDecreasedContainers);
+    } finally {
+      writeLock.unlock();
     }
-    int numCont = (int) Math.ceil(
-        Resources.divide(rc, clusterResource, tot, minimumAllocation));
-    ResourceRequest rr = ResourceRequest.newInstance(
-        Priority.UNDEFINED, ResourceRequest.ANY,
-        minimumAllocation, numCont);
-    List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
-    List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
-    List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
-    List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
-    Resource headroom = getHeadroom();
-    setApplicationHeadroomForMetrics(headroom);
-    return new Allocation(newlyAllocatedContainers, headroom, null,
-        currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
-        newlyIncreasedContainers, newlyDecreasedContainers);
   }
-  
-  synchronized public NodeId getNodeIdToUnreserve(
+
+  @VisibleForTesting
+  public NodeId getNodeIdToUnreserve(
       SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve,
       ResourceCalculator rc, Resource clusterResource) {
+    try {
+      writeLock.lock();
+      // first go around make this algorithm simple and just grab first
+      // reservation that has enough resources
+      Map<NodeId, RMContainer> reservedContainers = 
this.reservedContainers.get(
+          schedulerKey);
+
+      if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
+        for (Map.Entry<NodeId, RMContainer> entry : reservedContainers
+            .entrySet()) {
+          NodeId nodeId = entry.getKey();
+          RMContainer reservedContainer = entry.getValue();
+          if (reservedContainer.hasIncreaseReservation()) {
+            // Currently, only regular container allocation supports continuous
+            // reservation looking, we don't support canceling increase request
+            // reservation when allocating regular container.
+            continue;
+          }
 
-    // first go around make this algorithm simple and just grab first
-    // reservation that has enough resources
-    Map<NodeId, RMContainer> reservedContainers = this.reservedContainers
-        .get(schedulerKey);
-
-    if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
-      for (Map.Entry<NodeId, RMContainer> entry : 
reservedContainers.entrySet()) {
-        NodeId nodeId = entry.getKey();
-        RMContainer reservedContainer = entry.getValue();
-        if (reservedContainer.hasIncreaseReservation()) {
-          // Currently, only regular container allocation supports continuous
-          // reservation looking, we don't support canceling increase request
-          // reservation when allocating regular container.
-          continue;
-        }
-        
-        Resource reservedResource = reservedContainer.getReservedResource();
-        
-        // make sure we unreserve one with at least the same amount of
-        // resources, otherwise could affect capacity limits
-        if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
-            reservedResource)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("unreserving node with reservation size: "
-                + reservedResource
-                + " in order to allocate container with size: " + 
resourceNeedUnreserve);
+          Resource reservedResource = reservedContainer.getReservedResource();
+
+          // make sure we unreserve one with at least the same amount of
+          // resources, otherwise could affect capacity limits
+          if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
+              reservedResource)) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "unreserving node with reservation size: " + reservedResource
+                      + " in order to allocate container with size: "
+                      + resourceNeedUnreserve);
+            }
+            return nodeId;
           }
-          return nodeId;
         }
       }
+      return null;
+    } finally {
+      writeLock.unlock();
     }
-    return null;
   }
   
-  public synchronized void setHeadroomProvider(
+  public void setHeadroomProvider(
     CapacityHeadroomProvider headroomProvider) {
-    this.headroomProvider = headroomProvider;
-  }
-
-  public synchronized CapacityHeadroomProvider getHeadroomProvider() {
-    return headroomProvider;
+    try {
+      writeLock.lock();
+      this.headroomProvider = headroomProvider;
+    } finally {
+      writeLock.unlock();
+    }
   }
   
   @Override
-  public synchronized Resource getHeadroom() {
-    if (headroomProvider != null) {
-      return headroomProvider.getHeadroom();
+  public Resource getHeadroom() {
+    try {
+      readLock.lock();
+      if (headroomProvider != null) {
+        return headroomProvider.getHeadroom();
+      }
+      return super.getHeadroom();
+    } finally {
+      readLock.unlock();
     }
-    return super.getHeadroom();
+
   }
   
   @Override
-  public synchronized void transferStateFromPreviousAttempt(
+  public void transferStateFromPreviousAttempt(
       SchedulerApplicationAttempt appAttempt) {
-    super.transferStateFromPreviousAttempt(appAttempt);
-    this.headroomProvider = 
-      ((FiCaSchedulerApp) appAttempt).getHeadroomProvider();
+    try {
+      writeLock.lock();
+      super.transferStateFromPreviousAttempt(appAttempt);
+      this.headroomProvider = ((FiCaSchedulerApp) appAttempt).headroomProvider;
+    } finally {
+      writeLock.unlock();
+    }
   }
   
   public boolean reserveIncreasedContainer(SchedulerRequestKey schedulerKey,
@@ -444,11 +455,11 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
 
       // Update the node
       node.reserveResource(this, schedulerKey, rmContainer);
-      
+
       // Succeeded
       return true;
     }
-    
+
     return false;
   }
 
@@ -515,9 +526,12 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
       showRequests();
     }
 
-    synchronized (this) {
+    try {
+      writeLock.lock();
       return containerAllocator.assignContainers(clusterResource, node,
           schedulingMode, currentResourceLimits, reservedContainer);
+    } finally {
+      writeLock.unlock();
     }
   }
 
@@ -625,23 +639,33 @@ public class FiCaSchedulerApp extends 
SchedulerApplicationAttempt {
    * Capacity Scheduler.
    */
   @Override
-  public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
-    ApplicationResourceUsageReport report = super.getResourceUsageReport();
-    Resource cluster = rmContext.getScheduler().getClusterResource();
-    Resource totalPartitionRes =
-        rmContext.getNodeLabelManager()
-          .getResourceByLabel(getAppAMNodePartitionName(), cluster);
-    ResourceCalculator calc = rmContext.getScheduler().getResourceCalculator();
-    if (!calc.isInvalidDivisor(totalPartitionRes)) {
-      float queueAbsMaxCapPerPartition =
-          ((AbstractCSQueue)getQueue()).getQueueCapacities()
-            .getAbsoluteCapacity(getAppAMNodePartitionName());
-      float queueUsagePerc =
-          calc.divide(totalPartitionRes, report.getUsedResources(),
-              Resources.multiply(totalPartitionRes,
-                  queueAbsMaxCapPerPartition)) * 100;
-      report.setQueueUsagePercentage(queueUsagePerc);
+  public ApplicationResourceUsageReport getResourceUsageReport() {
+    try {
+      // Use write lock here because
+      // SchedulerApplicationAttempt#getResourceUsageReport updated fields
+      // TODO: improve this
+      writeLock.lock();
+      ApplicationResourceUsageReport report = super.getResourceUsageReport();
+      Resource cluster = rmContext.getScheduler().getClusterResource();
+      Resource totalPartitionRes =
+          rmContext.getNodeLabelManager().getResourceByLabel(
+              getAppAMNodePartitionName(), cluster);
+      ResourceCalculator calc =
+          rmContext.getScheduler().getResourceCalculator();
+      if (!calc.isInvalidDivisor(totalPartitionRes)) {
+        float queueAbsMaxCapPerPartition =
+            ((AbstractCSQueue) getQueue()).getQueueCapacities()
+                .getAbsoluteCapacity(getAppAMNodePartitionName());
+        float queueUsagePerc = calc.divide(totalPartitionRes,
+            report.getUsedResources(),
+            Resources.multiply(totalPartitionRes, queueAbsMaxCapPerPartition))
+            * 100;
+        report.setQueueUsagePercentage(queueUsagePerc);
+      }
+      return report;
+    } finally {
+      writeLock.unlock();
     }
-    return report;
+
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to