You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2016/03/17 01:03:03 UTC

[1/6] hadoop git commit: Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

Repository: hadoop
Updated Branches:
  refs/heads/trunk 7e8c9beb4 -> ae14e5d07


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index 4441c6b..2694957 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -356,40 +356,4 @@ public class TestUtils {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
   }
-
-  /**
-   * Get a queue structure:
-   * <pre>
-   *             Root
-   *            /  |  \
-   *           a   b   c
-   *          10   20  70
-   * </pre>
-   */
-  public static Configuration
-  getConfigurationWithMultipleQueues(Configuration config) {
-    CapacitySchedulerConfiguration conf =
-        new CapacitySchedulerConfiguration(config);
-
-    // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
-        new String[] { "a", "b", "c" });
-
-    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    conf.setCapacity(A, 10);
-    conf.setMaximumCapacity(A, 100);
-    conf.setUserLimitFactor(A, 100);
-
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    conf.setCapacity(B, 20);
-    conf.setMaximumCapacity(B, 100);
-    conf.setUserLimitFactor(B, 100);
-
-    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
-    conf.setCapacity(C, 70);
-    conf.setMaximumCapacity(C, 100);
-    conf.setUserLimitFactor(C, 100);
-
-    return conf;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index 2456594..5bdcc08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -1451,7 +1451,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
     // Trigger container rescheduled event
     scheduler.handle(new ContainerPreemptEvent(appAttemptId, rmContainer,
-            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
+            SchedulerEventType.KILL_PREEMPTED_CONTAINER));
 
     List<ResourceRequest> requests = rmContainer.getResourceRequests();
     // Once recovered, resource request will be present again in app


[4/6] hadoop git commit: YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index 2694957..4441c6b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -356,4 +356,40 @@ public class TestUtils {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
   }
+
+  /**
+   * Get a queue structure:
+   * <pre>
+   *             Root
+   *            /  |  \
+   *           a   b   c
+   *          10   20  70
+   * </pre>
+   */
+  public static Configuration
+  getConfigurationWithMultipleQueues(Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b", "c" });
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 100);
+    conf.setUserLimitFactor(A, 100);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    conf.setMaximumCapacity(B, 100);
+    conf.setUserLimitFactor(B, 100);
+
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 100);
+    conf.setUserLimitFactor(C, 100);
+
+    return conf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index 5bdcc08..2456594 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -1451,7 +1451,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
     // Trigger container rescheduled event
     scheduler.handle(new ContainerPreemptEvent(appAttemptId, rmContainer,
-            SchedulerEventType.KILL_PREEMPTED_CONTAINER));
+            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
 
     List<ResourceRequest> requests = rmContainer.getResourceRequests();
     // Once recovered, resource request will be present again in app


[3/6] hadoop git commit: Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

Posted by wa...@apache.org.
Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

This reverts commit 7e8c9beb4156dcaeb3a11e60aaa06d2370626913.


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

Branch: refs/heads/trunk
Commit: fa7a43529d529f0006c8033c2003f15b9b93f103
Parents: 7e8c9be
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Mar 16 17:02:10 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Mar 16 17:02:10 2016 -0700

----------------------------------------------------------------------
 .../ProportionalCapacityPreemptionPolicy.java   | 166 ++---
 .../rmcontainer/RMContainer.java                |   1 -
 .../scheduler/PreemptableResourceScheduler.java |   2 +-
 .../scheduler/ResourceLimits.java               |   9 -
 .../scheduler/SchedulerNode.java                |   9 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  45 +-
 .../scheduler/capacity/CSAssignment.java        |  11 -
 .../scheduler/capacity/CapacityScheduler.java   | 132 +---
 .../CapacitySchedulerConfiguration.java         |  14 +-
 .../capacity/CapacitySchedulerContext.java      |  15 +-
 .../scheduler/capacity/LeafQueue.java           |  69 --
 .../scheduler/capacity/ParentQueue.java         | 157 +----
 .../allocator/AbstractContainerAllocator.java   |   2 -
 .../capacity/allocator/ContainerAllocation.java |  12 -
 .../allocator/RegularContainerAllocator.java    |  39 +-
 .../capacity/preemption/KillableContainer.java  |  45 --
 .../capacity/preemption/PreemptableQueue.java   | 102 ---
 .../capacity/preemption/PreemptionManager.java  | 165 -----
 .../scheduler/common/AssignmentInformation.java |   6 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  32 +-
 .../common/fica/FiCaSchedulerNode.java          |  65 +-
 .../scheduler/event/SchedulerEventType.java     |  15 +-
 .../resourcemanager/TestRMDispatcher.java       |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |   2 +-
 .../applicationsmanager/TestAMRestart.java      |   7 +-
 ...estProportionalCapacityPreemptionPolicy.java |   6 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   2 -
 .../capacity/TestApplicationLimits.java         |   2 -
 .../capacity/TestApplicationPriority.java       |   6 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../TestCapacitySchedulerPreemption.java        | 677 -------------------
 .../scheduler/capacity/TestChildQueueOrder.java |   2 -
 .../scheduler/capacity/TestLeafQueue.java       |   3 -
 .../TestNodeLabelContainerAllocation.java       |  97 ---
 .../scheduler/capacity/TestParentQueue.java     |   2 -
 .../scheduler/capacity/TestReservations.java    |   2 -
 .../scheduler/capacity/TestUtils.java           |  36 -
 .../fair/TestFairSchedulerPreemption.java       |   2 +-
 38 files changed, 186 insertions(+), 1785 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 9b499c8..3a87edb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -35,7 +35,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 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.preemption.PreemptableQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -126,8 +125,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private long maxWaitTime;
   private CapacityScheduler scheduler;
   private long monitoringInterval;
-  private final Map<RMContainer, Long> preempted = new HashMap<>();
-
+  private final Map<RMContainer,Long> preempted =
+    new HashMap<RMContainer,Long>();
   private ResourceCalculator rc;
   private float percentageClusterPreemptionAllowed;
   private double naturalTerminationFactor;
@@ -136,10 +135,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       new HashMap<>();
   private RMNodeLabelsManager nlm;
 
-  // Preemptable Entities, synced from scheduler at every run
-  private Map<String, PreemptableQueue> preemptableEntities = null;
-  private Set<ContainerId> killableContainers;
-
   public ProportionalCapacityPreemptionPolicy() {
     clock = SystemClock.getInstance();
   }
@@ -189,64 +184,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     Resource clusterResources = Resources.clone(scheduler.getClusterResource());
     containerBasedPreemptOrKill(root, clusterResources);
   }
-
-  @SuppressWarnings("unchecked")
-  private void cleanupStaledKillableContainers(Resource cluster,
-      Set<String> leafQueueNames) {
-    for (String q : leafQueueNames) {
-      for (TempQueuePerPartition tq : getQueuePartitions(q)) {
-        // When queue's used - killable <= guaranteed and, killable > 0, we need
-        // to check if any of killable containers needs to be reverted
-        if (Resources.lessThanOrEqual(rc, cluster,
-            Resources.subtract(tq.current, tq.killable), tq.idealAssigned)
-            && Resources.greaterThan(rc, cluster, tq.killable, Resources.none())) {
-          // How many killable resources need to be reverted
-          // need-to-revert = already-marked-killable - (current - ideal)
-          Resource toBeRevertedFromKillable = Resources.subtract(tq.killable,
-              Resources.subtract(tq.current, tq.idealAssigned));
-
-          Resource alreadyReverted = Resources.createResource(0);
-
-          for (RMContainer c : preemptableEntities.get(q).getKillableContainers(
-              tq.partition).values()) {
-            if (Resources.greaterThanOrEqual(rc, cluster, alreadyReverted,
-                toBeRevertedFromKillable)) {
-              break;
-            }
-
-            if (Resources.greaterThan(rc, cluster,
-                Resources.add(alreadyReverted, c.getAllocatedResource()),
-                toBeRevertedFromKillable)) {
-              continue;
-            } else {
-              // This container need to be marked to unkillable
-              Resources.addTo(alreadyReverted, c.getAllocatedResource());
-              rmContext.getDispatcher().getEventHandler().handle(
-                  new ContainerPreemptEvent(c.getApplicationAttemptId(), c,
-                      SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE));
-            }
-          }
-
-        }
-      }
-    }
-  }
-
-  private void syncKillableContainersFromScheduler() {
-    // sync preemptable entities from scheduler
-    preemptableEntities =
-        scheduler.getPreemptionManager().getShallowCopyOfPreemptableEntities();
-
-    killableContainers = new HashSet<>();
-    for (Map.Entry<String, PreemptableQueue> entry : preemptableEntities
-        .entrySet()) {
-      PreemptableQueue entity = entry.getValue();
-      for (Map<ContainerId, RMContainer> map : entity.getKillableContainers()
-          .values()) {
-        killableContainers.addAll(map.keySet());
-      }
-    }
-  }
   
   /**
    * This method selects and tracks containers to be preempted. If a container
@@ -264,8 +201,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         .getNodeLabelManager().getClusterNodeLabelNames());
     allPartitions.add(RMNodeLabelsManager.NO_LABEL);
 
-    syncKillableContainersFromScheduler();
-
     // extract a summary of the queues from scheduler
     synchronized (scheduler) {
       queueToPartitions.clear();
@@ -293,17 +228,13 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
     }
 
-    // remove containers from killable list when we want to preempt less resources
-    // from queue.
-    cleanupStaledKillableContainers(clusterResources, leafQueueNames);
-
     // based on ideal allocation select containers to be preempted from each
     // queue and each application
     Map<ApplicationAttemptId,Set<RMContainer>> toPreempt =
         getContainersToPreempt(leafQueueNames, clusterResources);
 
     if (LOG.isDebugEnabled()) {
-      logToCSV(new ArrayList<>(leafQueueNames));
+      logToCSV(new ArrayList<String>(leafQueueNames));
     }
 
     // if we are in observeOnly mode return before any action is taken
@@ -323,10 +254,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         // if we tried to preempt this for more than maxWaitTime
         if (preempted.get(container) != null &&
             preempted.get(container) + maxWaitTime < clock.getTime()) {
-          // mark container killable
+          // kill it
           rmContext.getDispatcher().getEventHandler().handle(
               new ContainerPreemptEvent(appAttemptId, container,
-                  SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
+                  SchedulerEventType.KILL_PREEMPTED_CONTAINER));
           preempted.remove(container);
         } else {
           if (preempted.get(container) != null) {
@@ -402,14 +333,14 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // qAlloc tracks currently active queues (will decrease progressively as
     // demand is met)
-    List<TempQueuePerPartition> qAlloc = new ArrayList<>(queues);
+    List<TempQueuePerPartition> qAlloc = new ArrayList<TempQueuePerPartition>(queues);
     // unassigned tracks how much resources are still to assign, initialized
     // with the total capacity for this set of queues
     Resource unassigned = Resources.clone(tot_guarant);
 
     // group queues based on whether they have non-zero guaranteed capacity
-    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<>();
-    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<>();
+    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<TempQueuePerPartition>();
+    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<TempQueuePerPartition>();
 
     for (TempQueuePerPartition q : qAlloc) {
       if (Resources
@@ -484,8 +415,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // idealAssigned >= current + pending), remove it from consideration.
     // Sort queues from most under-guaranteed to most over-guaranteed.
     TQComparator tqComparator = new TQComparator(rc, tot_guarant);
-    PriorityQueue<TempQueuePerPartition> orderedByNeed = new PriorityQueue<>(10,
-        tqComparator);
+    PriorityQueue<TempQueuePerPartition> orderedByNeed =
+        new PriorityQueue<TempQueuePerPartition>(10, tqComparator);
     for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
       TempQueuePerPartition q = i.next();
       if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
@@ -543,7 +474,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   // percentage of guaranteed.
   protected Collection<TempQueuePerPartition> getMostUnderservedQueues(
       PriorityQueue<TempQueuePerPartition> orderedByNeed, TQComparator tqComparator) {
-    ArrayList<TempQueuePerPartition> underserved = new ArrayList<>();
+    ArrayList<TempQueuePerPartition> underserved = new ArrayList<TempQueuePerPartition>();
     while (!orderedByNeed.isEmpty()) {
       TempQueuePerPartition q1 = orderedByNeed.remove();
       underserved.add(q1);
@@ -571,7 +502,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     
     if (ignoreGuar) {
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = 1.0f / queues.size();
+        q.normalizedGuarantee = (float)  1.0f / ((float) queues.size());
       }
     } else {
       for (TempQueuePerPartition q : queues) {
@@ -584,9 +515,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
   }
 
-  private String getPartitionByRMContainer(RMContainer rmContainer) {
-    return scheduler.getSchedulerNode(rmContainer.getAllocatedNode())
-        .getPartition();
+  private String getPartitionByNodeId(NodeId nodeId) {
+    return scheduler.getSchedulerNode(nodeId).getPartition();
   }
 
   /**
@@ -604,7 +534,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return false;
     }
 
-    String nodePartition = getPartitionByRMContainer(rmContainer);
+    String nodePartition = getPartitionByNodeId(rmContainer.getAllocatedNode());
     Resource toObtainByPartition =
         resourceToObtainByPartitions.get(nodePartition);
 
@@ -645,7 +575,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       ApplicationAttemptId appAttemptId, RMContainer containerToPreempt) {
     Set<RMContainer> set;
     if (null == (set = preemptMap.get(appAttemptId))) {
-      set = new HashSet<>();
+      set = new HashSet<RMContainer>();
       preemptMap.put(appAttemptId, set);
     }
     set.add(containerToPreempt);
@@ -657,7 +587,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * over-capacity queue. It uses {@link #NATURAL_TERMINATION_FACTOR} to
    * account for containers that will naturally complete.
    *
-   * @param leafQueueNames set of leaf queues to preempt from
+   * @param queues set of leaf queues to preempt from
    * @param clusterResource total amount of cluster resources
    * @return a map of applciationID to set of containers to preempt
    */
@@ -665,8 +595,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Set<String> leafQueueNames, Resource clusterResource) {
 
     Map<ApplicationAttemptId, Set<RMContainer>> preemptMap =
-        new HashMap<>();
-    List<RMContainer> skippedAMContainerlist = new ArrayList<>();
+        new HashMap<ApplicationAttemptId, Set<RMContainer>>();
+    List<RMContainer> skippedAMContainerlist = new ArrayList<RMContainer>();
 
     // Loop all leaf queues
     for (String queueName : leafQueueNames) {
@@ -684,7 +614,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       LeafQueue leafQueue = null;
 
       Map<String, Resource> resToObtainByPartition =
-          new HashMap<>();
+          new HashMap<String, Resource>();
       for (TempQueuePerPartition qT : getQueuePartitions(queueName)) {
         leafQueue = qT.leafQueue;
         // we act only if we are violating balance by more than
@@ -773,6 +703,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param clusterResource
    * @param preemptMap
    * @param skippedAMContainerlist
+   * @param resToObtain
    * @param skippedAMSize
    * @param maxAMCapacityForThisQueue
    */
@@ -820,7 +751,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // first drop reserved containers towards rsrcPreempt
     List<RMContainer> reservedContainers =
-        new ArrayList<>(app.getReservedContainers());
+        new ArrayList<RMContainer>(app.getReservedContainers());
     for (RMContainer c : reservedContainers) {
       if (resToObtainByPartition.isEmpty()) {
         return;
@@ -840,7 +771,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // if more resources are to be freed go through all live containers in
     // reverse priority and reverse allocation order and mark them for
     // preemption
-    List<RMContainer> liveContainers = new ArrayList<>(app.getLiveContainers());
+    List<RMContainer> liveContainers =
+      new ArrayList<RMContainer>(app.getLiveContainers());
 
     sortContainers(liveContainers);
 
@@ -856,11 +788,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         continue;
       }
 
-      // Skip already marked to killable containers
-      if (killableContainers.contains(c.getContainerId())) {
-        continue;
-      }
-
       // Try to preempt this container
       tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
           clusterResource, preemptMap);
@@ -899,10 +826,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     return "ProportionalCapacityPreemptionPolicy";
   }
 
-  @VisibleForTesting
-  public Map<RMContainer, Long> getToPreemptContainers() {
-    return preempted;
-  }
 
   /**
    * This method walks a tree of CSQueue and clones the portion of the state
@@ -928,11 +851,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           partitionToLookAt);
       Resource guaranteed = Resources.multiply(partitionResource, absCap);
       Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
-      Resource killable = Resources.none();
-      if (null != preemptableEntities.get(queueName)) {
-         killable = preemptableEntities.get(queueName)
-            .getKillableResource(partitionToLookAt);
-      }
 
       // when partition is a non-exclusive partition, the actual maxCapacity
       // could more than specified maxCapacity
@@ -957,7 +875,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
               l.getTotalPendingResourcesConsideringUserLimit(
                   partitionResource, partitionToLookAt);
         ret = new TempQueuePerPartition(queueName, current, pending, guaranteed,
-            maxCapacity, preemptionDisabled, partitionToLookAt, killable);
+            maxCapacity, preemptionDisabled, partitionToLookAt);
         if (preemptionDisabled) {
           ret.untouchableExtra = extra;
         } else {
@@ -968,7 +886,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         Resource pending = Resource.newInstance(0, 0);
         ret =
             new TempQueuePerPartition(curQueue.getQueueName(), current, pending,
-                guaranteed, maxCapacity, false, partitionToLookAt, killable);
+                guaranteed, maxCapacity, false, partitionToLookAt);
         Resource childrensPreemptable = Resource.newInstance(0, 0);
         for (CSQueue c : curQueue.getChildQueues()) {
           TempQueuePerPartition subq =
@@ -1014,7 +932,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     Map<String, TempQueuePerPartition> queuePartitions;
     if (null == (queuePartitions = queueToPartitions.get(queueName))) {
-      queuePartitions = new HashMap<>();
+      queuePartitions = new HashMap<String, TempQueuePerPartition>();
       queueToPartitions.put(queueName, queuePartitions);
     }
     queuePartitions.put(queuePartition.partition, queuePartition);
@@ -1053,10 +971,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     final Resource guaranteed;
     final Resource maxCapacity;
     final String partition;
-    final Resource killable;
     Resource idealAssigned;
     Resource toBePreempted;
-
     // For logging purpose
     Resource actuallyPreempted;
     Resource untouchableExtra;
@@ -1070,7 +986,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     TempQueuePerPartition(String queueName, Resource current, Resource pending,
         Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled,
-        String partition, Resource killableResource) {
+        String partition) {
       this.queueName = queueName;
       this.current = current;
       this.pending = pending;
@@ -1080,12 +996,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.actuallyPreempted = Resource.newInstance(0, 0);
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
-      this.children = new ArrayList<>();
+      this.children = new ArrayList<TempQueuePerPartition>();
       this.untouchableExtra = Resource.newInstance(0, 0);
       this.preemptableExtra = Resource.newInstance(0, 0);
       this.preemptionDisabled = preemptionDisabled;
       this.partition = partition;
-      this.killable = killableResource;
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -1103,6 +1018,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Resources.addTo(pending, q.pending);
     }
 
+    public void addChildren(ArrayList<TempQueuePerPartition> queues) {
+      assert leafQueue == null;
+      children.addAll(queues);
+    }
+
+
     public ArrayList<TempQueuePerPartition> getChildren(){
       return children;
     }
@@ -1143,13 +1064,18 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return sb.toString();
     }
 
+    public void printAll() {
+      LOG.info(this.toString());
+      for (TempQueuePerPartition sub : this.getChildren()) {
+        sub.printAll();
+      }
+    }
+
     public void assignPreemption(float scalingFactor,
         ResourceCalculator rc, Resource clusterResource) {
-      if (Resources.greaterThan(rc, clusterResource,
-          Resources.subtract(current, killable), idealAssigned)) {
-        toBePreempted = Resources.multiply(Resources.subtract(
-            Resources.subtract(current, killable), idealAssigned),
-            scalingFactor);
+      if (Resources.greaterThan(rc, clusterResource, current, idealAssigned)) {
+          toBePreempted = Resources.multiply(
+              Resources.subtract(current, idealAssigned), scalingFactor);
       } else {
         toBePreempted = Resource.newInstance(0, 0);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.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/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index dfe0886..5d26931 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 
 /**
  * Represents the ResourceManager's view of an application container. See 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.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/PreemptableResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
index b73c538..ee7e101 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
@@ -45,6 +45,6 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
    * Ask the scheduler to forcibly interrupt the container given as input
    * @param container
    */
-  void markContainerForKillable(RMContainer container);
+  void killPreemptedContainer(RMContainer container);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.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/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
index 721eb36..c545e9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
@@ -38,8 +38,6 @@ public class ResourceLimits {
   // containers.
   private volatile Resource headroom;
 
-  private boolean allowPreempt = false;
-
   public ResourceLimits(Resource limit) {
     this(limit, Resources.none());
   }
@@ -74,11 +72,4 @@ public class ResourceLimits {
     this.amountNeededUnreserve = amountNeededUnreserve;
   }
 
-  public boolean isAllowPreemption() {
-    return allowPreempt;
-  }
-
-  public void setIsAllowPreemption(boolean allowPreempt) {
-   this.allowPreempt = allowPreempt;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 6c4f300..33ab2f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -64,8 +64,9 @@ public abstract class SchedulerNode {
   private volatile ResourceUtilization nodeUtilization =
       ResourceUtilization.newInstance(0, 0, 0f);
 
-  /* set of containers that are allocated containers */
-  protected final Map<ContainerId, RMContainer> launchedContainers =
+
+  /** Set of containers that are allocated containers. */
+  private final Map<ContainerId, RMContainer> launchedContainers =
       new HashMap<>();
 
   private final RMNode rmNode;
@@ -167,7 +168,7 @@ public abstract class SchedulerNode {
    * @param deltaResource Change in the resource allocation.
    * @param increase True if the change is an increase of allocation.
    */
-  protected synchronized void changeContainerResource(ContainerId containerId,
+  private synchronized void changeContainerResource(ContainerId containerId,
       Resource deltaResource, boolean increase) {
     if (increase) {
       deductUnallocatedResource(deltaResource);
@@ -241,7 +242,7 @@ public abstract class SchedulerNode {
    * Update the resources of the node when allocating a new container.
    * @param container Container to allocate.
    */
-  protected synchronized void updateResource(Container container) {
+  private synchronized void updateResource(Container container) {
     addUnallocatedResource(container.getResource());
     --numContainers;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index 955f8fa..39ca29b 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -46,7 +45,6 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -442,8 +440,11 @@ public abstract class AbstractCSQueue implements CSQueue {
           Resources.multiplyAndNormalizeDown(resourceCalculator,
               labelManager.getResourceByLabel(nodePartition, clusterResource),
               queueCapacities.getAbsoluteMaximumCapacity(nodePartition), minimumAllocation);
-      return Resources.min(resourceCalculator, clusterResource,
-          queueMaxResource, currentResourceLimits.getLimit());
+      if (nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
+        return Resources.min(resourceCalculator, clusterResource,
+            queueMaxResource, currentResourceLimits.getLimit());
+      }
+      return queueMaxResource;  
     } else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
       // When we doing non-exclusive resource allocation, maximum capacity of
       // all queues on this label equals to total resource with the label.
@@ -473,19 +474,12 @@ public abstract class AbstractCSQueue implements CSQueue {
 
     Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
 
-    // Set headroom for currentResourceLimits:
-    // When queue is a parent queue: Headroom = limit - used + killable
-    // When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
-    Resource usedExceptKillable = nowTotalUsed;
-    if (null != getChildQueues() && !getChildQueues().isEmpty()) {
-      usedExceptKillable = Resources.subtract(nowTotalUsed,
-          getTotalKillableResource(nodePartition));
-    }
-    currentResourceLimits.setHeadroom(
-        Resources.subtract(currentLimitResource, usedExceptKillable));
+    // Set headroom for currentResourceLimits
+    currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource,
+        nowTotalUsed));
 
     if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
-        usedExceptKillable, currentLimitResource)) {
+        nowTotalUsed, currentLimitResource)) {
 
       // if reservation continous looking enabled, check to see if could we
       // potentially use this node instead of a reserved node if the application
@@ -497,7 +491,7 @@ public abstract class AbstractCSQueue implements CSQueue {
               resourceCouldBeUnreserved, Resources.none())) {
         // resource-without-reserved = used - reserved
         Resource newTotalWithoutReservedResource =
-            Resources.subtract(usedExceptKillable, resourceCouldBeUnreserved);
+            Resources.subtract(nowTotalUsed, resourceCouldBeUnreserved);
 
         // when total-used-without-reserved-resource < currentLimit, we still
         // have chance to allocate on this node by unreserving some containers
@@ -626,10 +620,11 @@ public abstract class AbstractCSQueue implements CSQueue {
     // considering all labels in cluster, only those labels which are
     // use some resource of this queue can be considered.
     Set<String> nodeLabels = new HashSet<String>();
-    if (this.getAccessibleNodeLabels() != null && this.getAccessibleNodeLabels()
-        .contains(RMNodeLabelsManager.ANY)) {
-      nodeLabels.addAll(Sets.union(this.getQueueCapacities().getNodePartitionsSet(),
-          this.getQueueResourceUsage().getNodePartitionsSet()));
+    if (this.getAccessibleNodeLabels() != null
+        && this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
+      nodeLabels.addAll(Sets.union(this.getQueueCapacities()
+          .getNodePartitionsSet(), this.getQueueResourceUsage()
+          .getNodePartitionsSet()));
     } else {
       nodeLabels.addAll(this.getAccessibleNodeLabels());
     }
@@ -641,14 +636,4 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     return nodeLabels;
   }
-
-  public Resource getTotalKillableResource(String partition) {
-    return csContext.getPreemptionManager().getKillableResource(queueName,
-        partition);
-  }
-
-  public Iterator<RMContainer> getKillableContainers(String partition) {
-    return csContext.getPreemptionManager().getKillableContainers(queueName,
-        partition);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.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/CSAssignment.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/CSAssignment.java
index 6406efe..68f6f12 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/CSAssignment.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/CSAssignment.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.Assignment
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.util.List;
-
 @Private
 @Unstable
 public class CSAssignment {
@@ -44,7 +42,6 @@ public class CSAssignment {
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
   private boolean increaseAllocation;
-  private List<RMContainer> containersToKill;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -150,12 +147,4 @@ public class CSAssignment {
   public void setIncreasedAllocation(boolean flag) {
     increaseAllocation = flag;
   }
-
-  public void setContainersToKill(List<RMContainer> containersToKill) {
-    this.containersToKill = containersToKill;
-  }
-
-  public List<RMContainer> getContainersToKill() {
-    return containersToKill;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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
----------------------------------------------------------------------
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/CapacityScheduler.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/CapacityScheduler.java
index cf5c3b5..735306a 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/CapacityScheduler.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/CapacityScheduler.java
@@ -108,8 +108,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -150,10 +148,6 @@ public class CapacityScheduler extends
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
-  private PreemptionManager preemptionManager = new PreemptionManager();
-
-  private volatile boolean isLazyPreemptionEnabled = false;
-
   static final Comparator<CSQueue> nonPartitionedQueueComparator =
       new Comparator<CSQueue>() {
     @Override
@@ -304,11 +298,12 @@ public class CapacityScheduler extends
     initMaximumResourceCapability(this.conf.getMaximumAllocation());
     this.calculator = this.conf.getResourceCalculator();
     this.usePortForNodeName = this.conf.getUsePortForNodeName();
-    this.applications = new ConcurrentHashMap<>();
+    this.applications =
+        new ConcurrentHashMap<ApplicationId,
+            SchedulerApplication<FiCaSchedulerApp>>();
     this.labelManager = rmContext.getNodeLabelManager();
     authorizer = YarnAuthorizationProvider.getInstance(yarnConf);
     initializeQueues(this.conf);
-    this.isLazyPreemptionEnabled = conf.getLazyPreemptionEnabled();
 
     scheduleAsynchronously = this.conf.getScheduleAynschronously();
     asyncScheduleInterval =
@@ -374,9 +369,6 @@ public class CapacityScheduler extends
       refreshMaximumAllocation(this.conf.getMaximumAllocation());
       throw new IOException("Failed to re-init queues", t);
     }
-
-    // update lazy preemption
-    this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled();
   }
   
   long getAsyncScheduleInterval() {
@@ -511,9 +503,6 @@ public class CapacityScheduler extends
     LOG.info("Initialized root queue " + root);
     updatePlacementRules();
     setQueueAcls(authorizer, queues);
-
-    // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, root);
   }
 
   @Lock(CapacityScheduler.class)
@@ -542,9 +531,6 @@ public class CapacityScheduler extends
 
     labelManager.reinitializeQueueLabels(getQueueToLabels());
     setQueueAcls(authorizer, queues);
-
-    // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, root);
   }
 
   @VisibleForTesting
@@ -1267,10 +1253,8 @@ public class CapacityScheduler extends
 
     // Try to schedule more if there are no reservations to fulfill
     if (node.getReservedContainer() == null) {
-      if (calculator.computeAvailableContainers(Resources
-              .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
-          minimumAllocation) > 0) {
-
+      if (calculator.computeAvailableContainers(node.getUnallocatedResource(),
+        minimumAllocation) > 0) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getUnallocatedResource());
@@ -1279,8 +1263,10 @@ public class CapacityScheduler extends
         assignment = root.assignContainers(
             getClusterResource(),
             node,
+            // TODO, now we only consider limits for parent for non-labeled
+            // resources, should consider labeled resources as well.
             new ResourceLimits(labelManager.getResourceByLabel(
-                node.getPartition(), getClusterResource())),
+                RMNodeLabelsManager.NO_LABEL, getClusterResource())),
             SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         if (Resources.greaterThan(calculator, getClusterResource(),
             assignment.getResource(), Resources.none())) {
@@ -1450,20 +1436,11 @@ public class CapacityScheduler extends
       markContainerForPreemption(aid, containerToBePreempted);
     }
     break;
-    case MARK_CONTAINER_FOR_KILLABLE:
-    {
-      ContainerPreemptEvent containerKillableEvent = (ContainerPreemptEvent)event;
-      RMContainer killableContainer = containerKillableEvent.getContainer();
-      markContainerForKillable(killableContainer);
-    }
-    break;
-    case MARK_CONTAINER_FOR_NONKILLABLE:
+    case KILL_PREEMPTED_CONTAINER:
     {
-      if (isLazyPreemptionEnabled) {
-        ContainerPreemptEvent cancelKillContainerEvent =
-            (ContainerPreemptEvent) event;
-        markContainerForNonKillable(cancelKillContainerEvent.getContainer());
-      }
+      ContainerPreemptEvent killContainerEvent = (ContainerPreemptEvent)event;
+      RMContainer containerToBeKilled = killContainerEvent.getContainer();
+      killPreemptedContainer(containerToBeKilled);
     }
     break;
     default:
@@ -1571,14 +1548,14 @@ public class CapacityScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
+    
     Container container = rmContainer.getContainer();
-    ContainerId containerId = container.getId();
     
     // Get the application for the finished container
     FiCaSchedulerApp application =
         getCurrentAttemptForContainer(container.getId());
     ApplicationId appId =
-        containerId.getApplicationAttemptId().getApplicationId();
+        container.getId().getApplicationAttemptId().getApplicationId();
     if (application == null) {
       LOG.info("Container " + container + " of" + " finished application "
           + appId + " completed with event " + event);
@@ -1592,6 +1569,15 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue)application.getQueue();
     queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
+
+    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
+      schedulerHealth.updatePreemption(Time.now(), container.getNodeId(),
+        container.getId(), queue.getQueuePath());
+      schedulerHealth.updateSchedulerPreemptionCounts(1);
+    } else {
+      schedulerHealth.updateRelease(lastNodeUpdateTime, container.getNodeId(),
+        container.getId(), queue.getQueuePath());
+    }
   }
   
   @Override
@@ -1627,7 +1613,7 @@ public class CapacityScheduler extends
       ApplicationAttemptId applicationAttemptId) {
     return super.getApplicationAttempt(applicationAttemptId);
   }
-
+  
   @Lock(Lock.NoLock.class)
   public FiCaSchedulerNode getNode(NodeId nodeId) {
     return nodeTracker.getNode(nodeId);
@@ -1668,60 +1654,15 @@ public class CapacityScheduler extends
     }
   }
 
-  public synchronized void markContainerForKillable(
-      RMContainer killableContainer) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
-          + killableContainer.toString());
-    }
-
-    if (!isLazyPreemptionEnabled) {
-      super.completedContainer(killableContainer, SchedulerUtils
-          .createPreemptedContainerStatus(killableContainer.getContainerId(),
-              SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
-    } else {
-      FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
-          killableContainer.getAllocatedNode());
-
-      FiCaSchedulerApp application = getCurrentAttemptForContainer(
-          killableContainer.getContainerId());
-
-      node.markContainerToKillable(killableContainer.getContainerId());
-
-      // notify PreemptionManager
-      // Get the application for the finished container
-      if (null != application) {
-        String leafQueueName = application.getCSLeafQueue().getQueueName();
-        getPreemptionManager().addKillableContainer(
-            new KillableContainer(killableContainer, node.getPartition(),
-                leafQueueName));
-      }    }
-  }
-
-  private synchronized void markContainerForNonKillable(
-      RMContainer nonKillableContainer) {
+  @Override
+  public void killPreemptedContainer(RMContainer cont) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(
-          SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
-              + nonKillableContainer.toString());
-    }
-
-    FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
-        nonKillableContainer.getAllocatedNode());
-
-    FiCaSchedulerApp application = getCurrentAttemptForContainer(
-        nonKillableContainer.getContainerId());
-
-    node.markContainerToNonKillable(nonKillableContainer.getContainerId());
-
-    // notify PreemptionManager
-    // Get the application for the finished container
-    if (null != application) {
-      String leafQueueName = application.getCSLeafQueue().getQueueName();
-      getPreemptionManager().removeKillableContainer(
-          new KillableContainer(nonKillableContainer, node.getPartition(),
-              leafQueueName));
+      LOG.debug(SchedulerEventType.KILL_PREEMPTED_CONTAINER + ": container"
+          + cont.toString());
     }
+    super.completedContainer(cont, SchedulerUtils
+        .createPreemptedContainerStatus(cont.getContainerId(),
+        SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
   }
 
   @Override
@@ -2004,7 +1945,6 @@ public class CapacityScheduler extends
     return ret;
   }
 
-  @Override
   public SchedulerHealth getSchedulerHealth() {
     return this.schedulerHealth;
   }
@@ -2014,11 +1954,6 @@ public class CapacityScheduler extends
   }
 
   @Override
-  public long getLastNodeUpdateTime() {
-    return lastNodeUpdateTime;
-  }
-
-  @Override
   public Priority checkAndGetApplicationPriority(Priority priorityFromContext,
       String user, String queueName, ApplicationId applicationId)
       throws YarnException {
@@ -2119,9 +2054,4 @@ public class CapacityScheduler extends
         + rmApp.getQueue() + " for application: " + applicationId
         + " for the user: " + rmApp.getUser());
   }
-
-  @Override
-  public PreemptionManager getPreemptionManager() {
-    return preemptionManager;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 3729264..3756d9e 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -257,12 +257,6 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final String RESERVATION_ENFORCEMENT_WINDOW =
       "reservation-enforcement-window";
 
-  @Private
-  public static final String LAZY_PREEMPTION_ENALBED = PREFIX + "lazy-preemption-enabled";
-
-  @Private
-  public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
-
   public CapacitySchedulerConfiguration() {
     this(new Configuration());
   }
@@ -1013,11 +1007,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @VisibleForTesting
   public void setOrderingPolicyParameter(String queue,
       String parameterKey, String parameterValue) {
-    set(getQueuePrefix(queue) + ORDERING_POLICY + "." + parameterKey,
-        parameterValue);
-  }
-
-  public boolean getLazyPreemptionEnabled() {
-    return getBoolean(LAZY_PREEMPTION_ENALBED, DEFAULT_LAZY_PREEMPTION_ENABLED);
+    set(getQueuePrefix(queue) + ORDERING_POLICY + "."
+        + parameterKey, parameterValue);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
index 1203272..2a0dd0d 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
@@ -18,20 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import java.util.Comparator;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
-import java.util.Comparator;
-
 /**
  * Read-only interface to {@link CapacityScheduler} context.
  */
@@ -64,12 +61,4 @@ public interface CapacitySchedulerContext {
   PartitionedQueueComparator getPartitionedQueueComparator();
   
   FiCaSchedulerNode getNode(NodeId nodeId);
-
-  FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId attemptId);
-
-  PreemptionManager getPreemptionManager();
-
-  SchedulerHealth getSchedulerHealth();
-
-  long getLastNodeUpdateTime();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/LeafQueue.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/LeafQueue.java
index 3dc2090..c625fae 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/LeafQueue.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/LeafQueue.java
@@ -37,11 +37,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -65,9 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps;
@@ -827,40 +823,6 @@ public class LeafQueue extends AbstractCSQueue {
       assignment.setExcessReservation(null);
     }
   }
-
-  private void killToPreemptContainers(Resource clusterResource,
-      FiCaSchedulerNode node,
-      CSAssignment assignment) {
-    if (assignment.getContainersToKill() != null) {
-      StringBuilder sb = new StringBuilder("Killing containers: [");
-
-      for (RMContainer c : assignment.getContainersToKill()) {
-        FiCaSchedulerApp application = csContext.getApplicationAttempt(
-            c.getApplicationAttemptId());
-        LeafQueue q = application.getCSLeafQueue();
-        q.completedContainer(clusterResource, application, node, c, SchedulerUtils
-                .createPreemptedContainerStatus(c.getContainerId(),
-                    SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
-            null, false);
-        sb.append("(container=" + c.getContainerId() + " resource=" + c
-            .getAllocatedResource() + ")");
-      }
-
-      sb.append("] for container=" + assignment.getAssignmentInformation()
-          .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment
-          .getResource());
-      LOG.info(sb.toString());
-
-    }
-  }
-
-  private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
-    // Set preemption-allowed:
-    // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
-    float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition);
-    float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition);
-    limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
-  }
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
@@ -873,8 +835,6 @@ public class LeafQueue extends AbstractCSQueue {
           + " #applications=" + orderingPolicy.getNumSchedulableEntities());
     }
 
-    setPreemptionAllowed(currentResourceLimits, node.getPartition());
-
     // Check for reserved resources
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
@@ -886,7 +846,6 @@ public class LeafQueue extends AbstractCSQueue {
                 currentResourceLimits, schedulingMode, reservedContainer);
         handleExcessReservedContainer(clusterResource, assignment, node,
             application);
-        killToPreemptContainers(clusterResource, node, assignment);
         return assignment;
       }
     }
@@ -948,7 +907,6 @@ public class LeafQueue extends AbstractCSQueue {
       
       handleExcessReservedContainer(clusterResource, assignment, node,
           application);
-      killToPreemptContainers(clusterResource, node, assignment);
 
       if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
           Resources.none())) {
@@ -1252,34 +1210,11 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  private void updateSchedulerHealthForCompletedContainer(
-      RMContainer rmContainer, ContainerStatus containerStatus) {
-    // Update SchedulerHealth for released / preempted container
-    SchedulerHealth schedulerHealth = csContext.getSchedulerHealth();
-    if (null == schedulerHealth) {
-      // Only do update if we have schedulerHealth
-      return;
-    }
-
-    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
-      schedulerHealth.updatePreemption(Time.now(), rmContainer.getAllocatedNode(),
-          rmContainer.getContainerId(), getQueuePath());
-      schedulerHealth.updateSchedulerPreemptionCounts(1);
-    } else {
-      schedulerHealth.updateRelease(csContext.getLastNodeUpdateTime(),
-          rmContainer.getAllocatedNode(), rmContainer.getContainerId(),
-          getQueuePath());
-    }
-  }
-
   @Override
   public void completedContainer(Resource clusterResource, 
       FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, 
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue,
       boolean sortQueues) {
-    // Update SchedulerHealth for released / preempted container
-    updateSchedulerHealthForCompletedContainer(rmContainer, containerStatus);
-
     if (application != null) {
       // unreserve container increase request if it previously reserved.
       if (rmContainer.hasIncreaseReservation()) {
@@ -1330,10 +1265,6 @@ public class LeafQueue extends AbstractCSQueue {
           rmContainer, null, event, this, sortQueues);
       }
     }
-
-    // Notify PreemptionManager
-    csContext.getPreemptionManager().removeKillableContainer(
-        new KillableContainer(rmContainer, node.getPartition(), queueName));
   }
 
   synchronized void allocateResource(Resource clusterResource,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.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/ParentQueue.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/ParentQueue.java
index 6fcd6c1..7cf5565 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/ParentQueue.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/ParentQueue.java
@@ -18,6 +18,18 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,7 +49,6 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -46,25 +57,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -388,11 +386,6 @@ public class ParentQueue extends AbstractCSQueue {
     // if our queue cannot access this node, just return
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
         && !accessibleToPartition(node.getPartition())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skip this queue=" + getQueuePath()
-            + ", because it is not able to access partition=" + node
-            .getPartition());
-      }
       return CSAssignment.NULL_ASSIGNMENT;
     }
     
@@ -438,7 +431,7 @@ public class ParentQueue extends AbstractCSQueue {
               resourceCalculator, clusterResource, 
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
-        allocateResource(clusterResource, assignedToChild.getResource(),
+        super.allocateResource(clusterResource, assignedToChild.getResource(),
             node.getPartition(), assignedToChild.isIncreasedAllocation());
         
         // Track resource utilization in this pass of the scheduler
@@ -501,38 +494,29 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
-    // Two conditions need to meet when trying to allocate:
-    // 1) Node doesn't have reserved container
-    // 2) Node's available-resource + killable-resource should > 0
-    return node.getReservedContainer() == null && Resources.greaterThanOrEqual(
-        resourceCalculator, clusterResource, Resources
-            .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
-        minimumAllocation);
+    return (node.getReservedContainer() == null) && 
+        Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
+            node.getUnallocatedResource(), minimumAllocation);
   }
-
+  
   private ResourceLimits getResourceLimitsOfChild(CSQueue child,
-      Resource clusterResource, ResourceLimits parentLimits,
-      String nodePartition) {
+      Resource clusterResource, ResourceLimits parentLimits) {
     // Set resource-limit of a given child, child.limit =
     // min(my.limit - my.used + child.used, child.max)
 
     // Parent available resource = parent-limit - parent-used-resource
-    Resource parentMaxAvailableResource = Resources.subtract(
-        parentLimits.getLimit(), queueUsage.getUsed(nodePartition));
-    // Deduct killable from used
-    Resources.addTo(parentMaxAvailableResource,
-        getTotalKillableResource(nodePartition));
+    Resource parentMaxAvailableResource =
+        Resources.subtract(parentLimits.getLimit(), getUsedResources());
 
     // Child's limit = parent-available-resource + child-used
-    Resource childLimit = Resources.add(parentMaxAvailableResource,
-        child.getQueueResourceUsage().getUsed(nodePartition));
+    Resource childLimit =
+        Resources.add(parentMaxAvailableResource, child.getUsedResources());
 
     // Get child's max resource
-    Resource childConfiguredMaxResource = Resources.multiplyAndNormalizeDown(
-        resourceCalculator,
-        labelManager.getResourceByLabel(nodePartition, clusterResource),
-        child.getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition),
-        minimumAllocation);
+    Resource childConfiguredMaxResource =
+        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
+            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
+            child.getAbsoluteMaximumCapacity(), minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =
@@ -584,7 +568,7 @@ public class ParentQueue extends AbstractCSQueue {
 
       // Get ResourceLimits of child queue before assign containers
       ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, cluster, limits, node.getPartition());
+          getResourceLimitsOfChild(childQueue, cluster, limits);
       
       assignment = childQueue.assignContainers(cluster, node, 
           childLimits, schedulingMode);
@@ -730,8 +714,8 @@ public class ParentQueue extends AbstractCSQueue {
     // Update all children
     for (CSQueue childQueue : childQueues) {
       // Get ResourceLimits of child queue before assign containers
-      ResourceLimits childLimits = getResourceLimitsOfChild(childQueue,
-          clusterResource, resourceLimits, RMNodeLabelsManager.NO_LABEL);
+      ResourceLimits childLimits =
+          getResourceLimitsOfChild(childQueue, clusterResource, resourceLimits);     
       childQueue.updateClusterResource(clusterResource, childLimits);
     }
     
@@ -754,8 +738,8 @@ public class ParentQueue extends AbstractCSQueue {
     synchronized (this) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      allocateResource(clusterResource,
-          rmContainer.getContainer().getResource(), node.getPartition(), false);
+      super.allocateResource(clusterResource, rmContainer.getContainer()
+          .getResource(), node.getPartition(), false);
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -782,7 +766,7 @@ public class ParentQueue extends AbstractCSQueue {
     if (application != null) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      allocateResource(clusterResource, rmContainer.getContainer()
+      super.allocateResource(clusterResource, rmContainer.getContainer()
           .getResource(), node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
@@ -818,79 +802,4 @@ public class ParentQueue extends AbstractCSQueue {
   public synchronized int getNumApplications() {
     return numApplications;
   }
-
-  synchronized void allocateResource(Resource clusterResource,
-      Resource resource, String nodePartition, boolean changeContainerResource) {
-    super.allocateResource(clusterResource, resource, nodePartition,
-        changeContainerResource);
-
-    /**
-     * check if we need to kill (killable) containers if maximum resource violated.
-     * Doing this because we will deduct killable resource when going from root.
-     * For example:
-     * <pre>
-     *      Root
-     *      /   \
-     *     a     b
-     *   /  \
-     *  a1  a2
-     * </pre>
-     *
-     * a: max=10G, used=10G, killable=2G
-     * a1: used=8G, killable=2G
-     * a2: used=2G, pending=2G, killable=0G
-     *
-     * When we get queue-a to allocate resource, even if queue-a
-     * reaches its max resource, we deduct its used by killable, so we can allocate
-     * at most 2G resources. ResourceLimits passed down to a2 has headroom set to 2G.
-     *
-     * If scheduler finds a 2G available resource in existing cluster, and assigns it
-     * to a2, now a2's used= 2G + 2G = 4G, and a's used = 8G + 4G = 12G > 10G
-     *
-     * When this happens, we have to preempt killable container (on same or different
-     * nodes) of parent queue to avoid violating parent's max resource.
-     */
-    if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition)
-        < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
-      killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
-    }
-  }
-
-  private void killContainersToEnforceMaxQueueCapacity(String partition,
-      Resource clusterResource) {
-    Iterator<RMContainer> killableContainerIter = getKillableContainers(
-        partition);
-    if (!killableContainerIter.hasNext()) {
-      return;
-    }
-
-    Resource partitionResource = labelManager.getResourceByLabel(partition,
-        null);
-    Resource maxResource = Resources.multiply(partitionResource,
-        getQueueCapacities().getAbsoluteMaximumCapacity(partition));
-
-    while (Resources.greaterThan(resourceCalculator, partitionResource,
-        queueUsage.getUsed(partition), maxResource)) {
-      RMContainer toKillContainer = killableContainerIter.next();
-      FiCaSchedulerApp attempt = csContext.getApplicationAttempt(
-          toKillContainer.getContainerId().getApplicationAttemptId());
-      FiCaSchedulerNode node = csContext.getNode(
-          toKillContainer.getAllocatedNode());
-      if (null != attempt && null != node) {
-        LeafQueue lq = attempt.getCSLeafQueue();
-        lq.completedContainer(clusterResource, attempt, node, toKillContainer,
-            SchedulerUtils.createPreemptedContainerStatus(
-                toKillContainer.getContainerId(),
-                SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
-            null, false);
-        LOG.info("Killed container=" + toKillContainer.getContainerId()
-            + " from queue=" + lq.getQueueName() + " to make queue=" + this
-            .getQueueName() + "'s max-capacity enforced");
-      }
-
-      if (!killableContainerIter.hasNext()) {
-        break;
-      }
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.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/AbstractContainerAllocator.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/AbstractContainerAllocator.java
index afac235..ee01bd1 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/AbstractContainerAllocator.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/AbstractContainerAllocator.java
@@ -108,8 +108,6 @@ public abstract class AbstractContainerAllocator {
           assignment.setFulfilledReservation(true);
         }
       }
-
-      assignment.setContainersToKill(result.getToKillContainers());
     }
     
     return assignment;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.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/ContainerAllocation.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/ContainerAllocation.java
index 8f749f6..1df9410 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/ContainerAllocation.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/ContainerAllocation.java
@@ -19,14 +19,11 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
 
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.util.List;
-
 public class ContainerAllocation {
   /**
    * Skip the locality (e.g. node-local, rack-local, any), and look at other
@@ -59,7 +56,6 @@ public class ContainerAllocation {
   NodeType containerNodeType = NodeType.NODE_LOCAL;
   NodeType requestNodeType = NodeType.NODE_LOCAL;
   Container updatedContainer;
-  private List<RMContainer> toKillContainers;
 
   public ContainerAllocation(RMContainer containerToBeUnreserved,
       Resource resourceToBeAllocated, AllocationState state) {
@@ -90,12 +86,4 @@ public class ContainerAllocation {
   public Container getUpdatedContainer() {
     return updatedContainer;
   }
-
-  public void setToKillContainers(List<RMContainer> toKillContainers) {
-    this.toKillContainers = toKillContainers;
-  }
-
-  public List<RMContainer> getToKillContainers() {
-    return toKillContainers;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 a5ca2d8..e168edf 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
@@ -42,9 +42,6 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * Allocate normal (new) containers, considers locality/label, etc. Using
  * delayed scheduling mechanism to get better locality allocation.
@@ -438,6 +435,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       return ContainerAllocation.LOCALITY_SKIPPED;
     }
 
+    assert Resources.greaterThan(
+        rc, clusterResource, available, Resources.none());
+
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
         priority, capability);
 
@@ -460,29 +460,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     boolean reservationsContinueLooking =
         application.getCSLeafQueue().getReservationContinueLooking();
 
-    // Check if we need to kill some containers to allocate this one
-    List<RMContainer> toKillContainers = null;
-    if (availableContainers == 0 && currentResoureLimits.isAllowPreemption()) {
-      Resource availableAndKillable = Resources.clone(available);
-      for (RMContainer killableContainer : node
-          .getKillableContainers().values()) {
-        if (null == toKillContainers) {
-          toKillContainers = new ArrayList<>();
-        }
-        toKillContainers.add(killableContainer);
-        Resources.addTo(availableAndKillable,
-                        killableContainer.getAllocatedResource());
-        if (Resources.fitsIn(rc,
-                             clusterResource,
-                             capability,
-                             availableAndKillable)) {
-          // Stop if we find enough spaces
-          availableContainers = 1;
-          break;
-        }
-      }
-    }
-
     if (availableContainers > 0) {
       // Allocate...
       // We will only do continuous reservation when this is not allocated from
@@ -522,12 +499,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
           new ContainerAllocation(unreservedContainer, request.getCapability(),
               AllocationState.ALLOCATED);
       result.containerNodeType = type;
-      result.setToKillContainers(toKillContainers);
       return result;
     } else {
       // if we are allowed to allocate but this node doesn't have space, reserve
       // it or if this was an already a reserved container, reserve it again
       if (shouldAllocOrReserveNewContainer || rmContainer != null) {
+
         if (reservationsContinueLooking && rmContainer == null) {
           // we could possibly ignoring queue capacity or user limits when
           // reservationsContinueLooking is set. Make sure we didn't need to
@@ -545,7 +522,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             new ContainerAllocation(null, request.getCapability(),
                 AllocationState.RESERVED);
         result.containerNodeType = type;
-        result.setToKillContainers(null);
         return result;
       }
       // Skip the locality request
@@ -637,7 +613,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
 
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
-      FiCaSchedulerNode node, Priority priority,
+      Resource clusterResource, FiCaSchedulerNode node,
+      SchedulingMode schedulingMode, Priority priority,
       RMContainer reservedContainer) {
     // Create the container if necessary
     Container container =
@@ -701,7 +678,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
     if (AllocationState.ALLOCATED == result.state
         || AllocationState.RESERVED == result.state) {
-      result = doAllocation(result, node, priority, reservedContainer);
+      result =
+          doAllocation(result, clusterResource, node, schedulingMode, priority,
+              reservedContainer);
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.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/preemption/KillableContainer.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/preemption/KillableContainer.java
deleted file mode 100644
index 675b0b4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption;
-
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-
-public class KillableContainer {
-  RMContainer container;
-  String partition;
-  String leafQueueName;
-
-  public KillableContainer(RMContainer container, String partition, String leafQueueName) {
-    this.container = container;
-    this.partition = partition;
-    this.leafQueueName = leafQueueName;
-  }
-
-  public RMContainer getRMContainer() {
-    return this.container;
-  }
-
-  public String getNodePartition() {
-    return this.partition;
-  }
-
-  public String getLeafQueueName() {
-    return this.leafQueueName;
-  }
-}


[2/6] hadoop git commit: Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.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/preemption/PreemptableQueue.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/preemption/PreemptableQueue.java
deleted file mode 100644
index 19148d7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-public class PreemptableQueue {
-  // Partition -> killable resources and containers
-  private Map<String, Resource> totalKillableResources = new HashMap<>();
-  private Map<String, Map<ContainerId, RMContainer>> killableContainers =
-      new HashMap<>();
-  private PreemptableQueue parent;
-
-  public PreemptableQueue(PreemptableQueue parent) {
-    this.parent = parent;
-  }
-
-  public PreemptableQueue(Map<String, Resource> totalKillableResources,
-      Map<String, Map<ContainerId, RMContainer>> killableContainers) {
-    this.totalKillableResources = totalKillableResources;
-    this.killableContainers = killableContainers;
-  }
-
-  void addKillableContainer(KillableContainer container) {
-    String partition = container.getNodePartition();
-    if (!totalKillableResources.containsKey(partition)) {
-      totalKillableResources.put(partition, Resources.createResource(0));
-      killableContainers.put(partition,
-          new ConcurrentSkipListMap<ContainerId, RMContainer>());
-    }
-
-    RMContainer c = container.getRMContainer();
-    Resources.addTo(totalKillableResources.get(partition),
-        c.getAllocatedResource());
-    killableContainers.get(partition).put(c.getContainerId(), c);
-
-    if (null != parent) {
-      parent.addKillableContainer(container);
-    }
-  }
-
-  void removeKillableContainer(KillableContainer container) {
-    String partition = container.getNodePartition();
-    Map<ContainerId, RMContainer> partitionKillableContainers =
-        killableContainers.get(partition);
-    if (partitionKillableContainers != null) {
-      RMContainer rmContainer = partitionKillableContainers.remove(
-          container.getRMContainer().getContainerId());
-      if (null != rmContainer) {
-        Resources.subtractFrom(totalKillableResources.get(partition),
-            rmContainer.getAllocatedResource());
-      }
-    }
-
-    if (null != parent) {
-      parent.removeKillableContainer(container);
-    }
-  }
-
-  public Resource getKillableResource(String partition) {
-    Resource res = totalKillableResources.get(partition);
-    return res == null ? Resources.none() : res;
-  }
-
-  @SuppressWarnings("unchecked")
-  public Map<ContainerId, RMContainer> getKillableContainers(String partition) {
-    Map<ContainerId, RMContainer> map = killableContainers.get(partition);
-    return map == null ? Collections.EMPTY_MAP : map;
-  }
-
-  public Map<String, Map<ContainerId, RMContainer>> getKillableContainers() {
-    return killableContainers;
-  }
-
-  Map<String, Resource> getTotalKillableResources() {
-    return totalKillableResources;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.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/preemption/PreemptionManager.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/preemption/PreemptionManager.java
deleted file mode 100644
index a9f02a5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class PreemptionManager {
-  private ReentrantReadWriteLock.ReadLock readLock;
-  private ReentrantReadWriteLock.WriteLock writeLock;
-  private Map<String, PreemptableQueue> entities = new HashMap<>();
-
-  public PreemptionManager() {
-    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    readLock = lock.readLock();
-    writeLock = lock.writeLock();
-  }
-
-  public void refreshQueues(CSQueue parent, CSQueue current) {
-    try {
-      writeLock.lock();
-      PreemptableQueue parentEntity = null;
-      if (parent != null) {
-        parentEntity = entities.get(parent.getQueueName());
-      }
-
-      if (!entities.containsKey(current.getQueueName())) {
-        entities.put(current.getQueueName(),
-            new PreemptableQueue(parentEntity));
-      }
-
-      if (current.getChildQueues() != null) {
-        for (CSQueue child : current.getChildQueues()) {
-          refreshQueues(current, child);
-        }
-      }
-    }
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void addKillableContainer(KillableContainer container) {
-    try {
-      writeLock.lock();
-      PreemptableQueue entity = entities.get(container.getLeafQueueName());
-      if (null != entity) {
-        entity.addKillableContainer(container);
-      }
-    }
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void removeKillableContainer(KillableContainer container) {
-    try {
-      writeLock.lock();
-      PreemptableQueue entity = entities.get(container.getLeafQueueName());
-      if (null != entity) {
-        entity.removeKillableContainer(container);
-      }
-    }
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void moveKillableContainer(KillableContainer oldContainer,
-      KillableContainer newContainer) {
-    // TODO, will be called when partition of the node changed OR
-    // container moved to different queue
-  }
-
-  public void updateKillableContainerResource(KillableContainer container,
-      Resource oldResource, Resource newResource) {
-    // TODO, will be called when container's resource changed
-  }
-
-  @VisibleForTesting
-  public Map<ContainerId, RMContainer> getKillableContainersMap(
-      String queueName, String partition) {
-    try {
-      readLock.lock();
-      PreemptableQueue entity = entities.get(queueName);
-      if (entity != null) {
-        Map<ContainerId, RMContainer> containers =
-            entity.getKillableContainers().get(partition);
-        if (containers != null) {
-          return containers;
-        }
-      }
-      return Collections.emptyMap();
-    }
-    finally {
-      readLock.unlock();
-    }
-  }
-
-  public Iterator<RMContainer> getKillableContainers(String queueName,
-      String partition) {
-    return getKillableContainersMap(queueName, partition).values().iterator();
-  }
-
-  public Resource getKillableResource(String queueName, String partition) {
-    try {
-      readLock.lock();
-      PreemptableQueue entity = entities.get(queueName);
-      if (entity != null) {
-        Resource res = entity.getTotalKillableResources().get(partition);
-        if (res == null || res.equals(Resources.none())) {
-          return Resources.none();
-        }
-        return Resources.clone(res);
-      }
-      return Resources.none();
-    }
-    finally {
-      readLock.unlock();
-    }
-  }
-
-  public Map<String, PreemptableQueue> getShallowCopyOfPreemptableEntities() {
-    try {
-      readLock.lock();
-      Map<String, PreemptableQueue> map = new HashMap<>();
-      for (Map.Entry<String, PreemptableQueue> entry : entities.entrySet()) {
-        String key = entry.getKey();
-        PreemptableQueue entity = entry.getValue();
-        map.put(key, new PreemptableQueue(
-            new HashMap<>(entity.getTotalKillableResources()),
-            new HashMap<>(entity.getKillableContainers())));
-      }
-      return map;
-    } finally {
-      readLock.unlock();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.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/AssignmentInformation.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/AssignmentInformation.java
index aad3bc7..5158255 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/AssignmentInformation.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/AssignmentInformation.java
@@ -120,9 +120,9 @@ public class AssignmentInformation {
   }
 
   private ContainerId getFirstContainerIdFromOperation(Operation op) {
-    if (null != operationDetails.get(op)) {
+    if (null != operationDetails.get(Operation.ALLOCATION)) {
       List<AssignmentDetails> assignDetails =
-          operationDetails.get(op);
+          operationDetails.get(Operation.ALLOCATION);
       if (!assignDetails.isEmpty()) {
         return assignDetails.get(0).containerId;
       }
@@ -131,7 +131,7 @@ public class AssignmentInformation {
   }
 
   public ContainerId getFirstAllocatedOrReservedContainerId() {
-    ContainerId containerId;
+    ContainerId containerId = null;
     containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
     if (null != containerId) {
       return containerId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 f474aad..4d563cd 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
@@ -59,7 +59,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;
@@ -95,7 +94,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,
@@ -140,30 +138,28 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     
     containerAllocator = new ContainerAllocator(this, rc, rmContext);
-
-    if (scheduler instanceof CapacityScheduler) {
-      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
-    }
   }
 
-  public synchronized boolean containerCompleted(RMContainer rmContainer,
+  synchronized public boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
-    ContainerId containerId = rmContainer.getContainerId();
 
     // Remove from the list of containers
-    if (null == liveContainers.remove(containerId)) {
+    if (null == liveContainers.remove(rmContainer.getContainerId())) {
       return false;
     }
-
+    
     // Remove from the list of newly allocated containers if found
     newlyAllocatedContainers.remove(rmContainer);
 
+    Container container = rmContainer.getContainer();
+    ContainerId containerId = container.getId();
+
     // Inform the container
     rmContainer.handle(
         new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(containerId);
+    containersToPreempt.remove(rmContainer.getContainerId());
 
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
@@ -180,7 +176,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return true;
   }
 
-  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
+  synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       Priority priority, ResourceRequest request, 
       Container container) {
 
@@ -204,9 +200,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);
-
-    ContainerId containerId = container.getId();
-    liveContainers.put(containerId, rmContainer);
+    liveContainers.put(container.getId(), rmContainer);    
 
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
@@ -219,17 +213,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Inform the container
     rmContainer.handle(
-        new RMContainerEvent(containerId, RMContainerEventType.START));
+        new RMContainerEvent(container.getId(), RMContainerEventType.START));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("allocate: applicationAttemptId=" 
-          + containerId.getApplicationAttemptId()
-          + " container=" + containerId + " host="
+          + container.getId().getApplicationAttemptId() 
+          + " container=" + container.getId() + " host="
           + container.getNodeId().getHost() + " type=" + type);
     }
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId);
+        getApplicationId(), container.getId());
     
     return rmContainer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
index 1d0e78a..fe6db47 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
@@ -18,29 +18,22 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
 
+
+import java.util.Set;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 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.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
 
 public class FiCaSchedulerNode extends SchedulerNode {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class);
-  private Map<ContainerId, RMContainer> killableContainers = new HashMap<>();
-  private Resource totalKillableResources = Resource.newInstance(0, 0);
   
   public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName,
       Set<String> nodeLabels) {
@@ -99,6 +92,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
   @Override
   public synchronized void unreserveResource(
       SchedulerApplicationAttempt application) {
+
     // adding NP checks as this can now be called for preemption
     if (getReservedContainer() != null
         && getReservedContainer().getContainer() != null
@@ -121,55 +115,4 @@ public class FiCaSchedulerNode extends SchedulerNode {
     }
     setReservedContainer(null);
   }
-
-  // According to decisions from preemption policy, mark the container to killable
-  public synchronized void markContainerToKillable(ContainerId containerId) {
-    RMContainer c = launchedContainers.get(containerId);
-    if (c != null && !killableContainers.containsKey(containerId)) {
-      killableContainers.put(containerId, c);
-      Resources.addTo(totalKillableResources, c.getAllocatedResource());
-    }
-  }
-
-  // According to decisions from preemption policy, mark the container to
-  // non-killable
-  public synchronized void markContainerToNonKillable(ContainerId containerId) {
-    RMContainer c = launchedContainers.get(containerId);
-    if (c != null && killableContainers.containsKey(containerId)) {
-      killableContainers.remove(containerId);
-      Resources.subtractFrom(totalKillableResources, c.getAllocatedResource());
-    }
-  }
-
-  @Override
-  protected synchronized void updateResource(
-      Container container) {
-    super.updateResource(container);
-    if (killableContainers.containsKey(container.getId())) {
-      Resources.subtractFrom(totalKillableResources, container.getResource());
-      killableContainers.remove(container.getId());
-    }
-  }
-
-  @Override
-  protected synchronized void changeContainerResource(ContainerId containerId,
-      Resource deltaResource, boolean increase) {
-    super.changeContainerResource(containerId, deltaResource, increase);
-
-    if (killableContainers.containsKey(containerId)) {
-      if (increase) {
-        Resources.addTo(totalKillableResources, deltaResource);
-      } else {
-        Resources.subtractFrom(totalKillableResources, deltaResource);
-      }
-    }
-  }
-
-  public synchronized Resource getTotalKillableResources() {
-    return totalKillableResources;
-  }
-
-  public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
-    return killableContainers;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.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/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index 35b7c14..9cf09e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -38,15 +38,10 @@ public enum SchedulerEventType {
   // Source: ContainerAllocationExpirer
   CONTAINER_EXPIRED,
 
-  /* Source: SchedulingEditPolicy */
+  // Source: SchedulingEditPolicy
   KILL_RESERVED_CONTAINER,
-
-  // Mark a container for preemption
-  MARK_CONTAINER_FOR_PREEMPTION,
-
-  // Mark a for-preemption container killable
-  MARK_CONTAINER_FOR_KILLABLE,
-
-  // Cancel a killable container
-  MARK_CONTAINER_FOR_NONKILLABLE
+  MARK_CONTAINER_FOR_PREEMPTION, // Mark a container for preemption
+                                 // in the near future
+  KILL_PREEMPTED_CONTAINER // Kill a container previously marked for
+                           // preemption
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
index c944752..d9306dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
@@ -59,7 +59,7 @@ public class TestRMDispatcher {
       rmDispatcher.getEventHandler().handle(event1);
       ContainerPreemptEvent event2 =
           new ContainerPreemptEvent(appAttemptId, container,
-            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE);
+            SchedulerEventType.KILL_PREEMPTED_CONTAINER);
       rmDispatcher.getEventHandler().handle(event2);
       ContainerPreemptEvent event3 =
           new ContainerPreemptEvent(appAttemptId, container,
@@ -70,7 +70,7 @@ public class TestRMDispatcher {
       verify(sched, times(3)).handle(any(SchedulerEvent.class));
       verify(sched).killReservedContainer(container);
       verify(sched).markContainerForPreemption(appAttemptId, container);
-      verify(sched).markContainerForKillable(container);
+      verify(sched).killPreemptedContainer(container);
     } catch (InterruptedException e) {
       Assert.fail();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 3057615..028afb1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -2352,7 +2352,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       FiCaSchedulerApp schedulerAppAttempt = cs.getSchedulerApplications()
           .get(app0.getApplicationId()).getCurrentAppAttempt();
       // kill app0-attempt
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
           app0.getCurrentAppAttempt().getMasterContainer().getId()));
       am0.waitForState(RMAppAttemptState.FAILED);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 16f3f60..5035afe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
@@ -565,7 +566,7 @@ public class TestAMRestart {
     ContainerId amContainer =
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     // Preempt the first attempt;
-    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
+    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
@@ -581,7 +582,7 @@ public class TestAMRestart {
     // Preempt the second attempt.
     ContainerId amContainer2 =
         ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer2));
+    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer2));
 
     am2.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt2.shouldCountTowardsMaxAttemptRetry());
@@ -676,7 +677,7 @@ public class TestAMRestart {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
 
     // Forcibly preempt the am container;
-    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
+    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index e9129de..13f267d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.Pro
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_PREEMPTED_CONTAINER;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -75,7 +75,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -168,7 +167,6 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(mCS.getConfiguration()).thenReturn(schedConf);
     rmContext = mock(RMContext.class);
     when(mCS.getRMContext()).thenReturn(rmContext);
-    when(mCS.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(rmContext.getNodeLabelManager()).thenReturn(lm);
     mDisp = mock(EventHandler.class);
     Dispatcher disp = mock(Dispatcher.class);
@@ -291,7 +289,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     List<ContainerPreemptEvent> events = evtCaptor.getAllValues();
     for (ContainerPreemptEvent e : events.subList(20, 20)) {
       assertEquals(appC, e.getAppId());
-      assertEquals(MARK_CONTAINER_FOR_KILLABLE, e.getType());
+      assertEquals(KILL_PREEMPTED_CONTAINER, e.getType());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 21ea495..512f37c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -124,7 +123,6 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     mClock = mock(Clock.class);
     cs = mock(CapacityScheduler.class);
     when(cs.getResourceCalculator()).thenReturn(rc);
-    when(cs.getPreemptionManager()).thenReturn(new PreemptionManager());
 
     nlm = mock(RMNodeLabelsManager.class);
     mDisp = mock(EventHandler.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
index 171196f..0b32676 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -265,7 +264,6 @@ public class TestApplicationLimits {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
index d8161f8..1569a12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
@@ -205,7 +205,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check node report, 12 GB used and 4 GB available
@@ -512,7 +512,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 
@@ -542,7 +542,7 @@ public class TestApplicationPriority {
       if (++counter > 1) {
         break;
       }
-      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 16ba607..b6c005b 100644
--- a/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
+++ b/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
@@ -1188,7 +1188,7 @@ public class TestCapacityScheduler {
 
     // kill the 3 containers
     for (Container c : allocatedContainers) {
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1197,7 +1197,7 @@ public class TestCapacityScheduler {
         Resource.newInstance(CONTAINER_MEMORY * 3, 3), false, 3);
 
     // kill app0-attempt0 AM container
-    cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(app0
+    cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(app0
         .getCurrentAppAttempt().getMasterContainer().getId()));
 
     // wait for app0 failed
@@ -1220,7 +1220,7 @@ public class TestCapacityScheduler {
     allocatedContainers =
         am1.allocateAndWaitForContainers(3, CONTAINER_MEMORY, nm1);
     for (Container c : allocatedContainers) {
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1269,7 +1269,7 @@ public class TestCapacityScheduler {
     }
 
     // Call killContainer to preempt the container
-    cs.markContainerForKillable(rmContainer);
+    cs.killPreemptedContainer(rmContainer);
 
     Assert.assertEquals(3, requests.size());
     for (ResourceRequest request : requests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
deleted file mode 100644
index bea7797..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
+++ /dev/null
@@ -1,677 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
-
-import com.google.common.collect.Sets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.Service;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.RMActiveServices;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-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.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class TestCapacitySchedulerPreemption {
-  private static final Log LOG = LogFactory.getLog(
-      TestCapacitySchedulerPreemption.class);
-
-  private final int GB = 1024;
-
-  private Configuration conf;
-
-  RMNodeLabelsManager mgr;
-
-  Clock clock;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new YarnConfiguration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
-        ProportionalCapacityPreemptionPolicy.class, SchedulingEditPolicy.class);
-    conf = TestUtils.getConfigurationWithMultipleQueues(this.conf);
-
-    // Set preemption related configurations
-    conf.setInt(ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL,
-        0);
-    conf.setBoolean(CapacitySchedulerConfiguration.LAZY_PREEMPTION_ENALBED,
-        true);
-    conf.setFloat(
-        ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND, 1.0f);
-    conf.setFloat(
-        ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR, 1.0f);
-    mgr = new NullRMNodeLabelsManager();
-    mgr.init(this.conf);
-    clock = mock(Clock.class);
-    when(clock.getTime()).thenReturn(0L);
-  }
-
-  private SchedulingEditPolicy getSchedulingEditPolicy(MockRM rm) {
-    RMActiveServices activeServices = rm.getRMActiveService();
-    SchedulingMonitor mon = null;
-    for (Service service : activeServices.getServices()) {
-      if (service instanceof SchedulingMonitor) {
-        mon = (SchedulingMonitor) service;
-        break;
-      }
-    }
-
-    if (mon != null) {
-      return mon.getSchedulingEditPolicy();
-    }
-    return null;
-  }
-
-  @Test (timeout = 60000)
-  public void testSimplePreemption() throws Exception {
-    /**
-     * Test case: Submit two application (app1/app2) to different queues, queue
-     * structure:
-     *
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     *
-     * 1) Two nodes in the cluster, each of them has 4G.
-     *
-     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
-     * more resource available.
-     *
-     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
-     *
-     * Now the cluster is fulfilled.
-     *
-     * 4) app2 asks for another 1G container, system will preempt one container
-     * from app1, and app2 will receive the preempted container
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 7, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-    Assert.assertEquals(1, killableContainers.size());
-    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
-        .getApplicationAttemptId(), am1.getApplicationAttemptId());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 6 containers, and app2 has 2 containers
-    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionConsidersNodeLocalityDelay()
-      throws Exception {
-    /**
-     * Test case: same as testSimplePreemption steps 1-3.
-     *
-     * Step 4: app2 asks for 1G container with locality specified, so it needs
-     * to wait for missed-opportunity before get scheduled.
-     * Check if system waits missed-opportunity before finish killable container
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container with unknown host and unknown rack
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1), ResourceRequest
-        .newInstance(Priority.newInstance(1), "unknownhost",
-            Resources.createResource(1 * GB), 1), ResourceRequest
-        .newInstance(Priority.newInstance(1), "/default-rack",
-            Resources.createResource(1 * GB), 1)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
-        .getApplicationAttemptId(), am1.getApplicationAttemptId());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    // Do allocation again, one container will be preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    // App1 has 6 containers, and app2 has 2 containers (new container allocated)
-    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionConsidersHardNodeLocality()
-      throws Exception {
-    /**
-     * Test case: same as testSimplePreemption steps 1-3.
-     *
-     * Step 4: app2 asks for 1G container with hard locality specified, and
-     *         asked host is not existed
-     * Confirm system doesn't preempt any container.
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-    }
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container for h3 with hard locality,
-    // h3 doesn't exist in the cluster
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1, true), ResourceRequest
-        .newInstance(Priority.newInstance(1), "h3",
-            Resources.createResource(1 * GB), 1, false), ResourceRequest
-        .newInstance(Priority.newInstance(1), "/default-rack",
-            Resources.createResource(1 * GB), 1, false)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
-        .getApplicationAttemptId(), am1.getApplicationAttemptId());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    // Do allocation again, nothing will be preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    // App1 has 7 containers, and app2 has 1 containers (no container allocated)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionPolicyShouldRespectAlreadyMarkedKillableContainers()
-      throws Exception {
-    /**
-     * Test case:
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     * Submit applications to two queues, one uses more than the other, so
-     * preemption will happen.
-     *
-     * Check:
-     * 1) Killable containers resources will be excluded from PCPP (no duplicated
-     *    container added to killable list)
-     * 2) When more resources need to be preempted, new containers will be selected
-     *    and killable containers will be considered
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 6 times for node1
-    for (int i = 0; i < 6; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
-
-    // NM1 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
-
-    // Get edit policy and do one update
-    ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-
-    // Check killable containers and to-be-preempted containers in edit policy
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-
-    // Run edit schedule again, confirm status doesn't changed
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-
-    // Save current to kill containers
-    Set<ContainerId> previousKillableContainers = new HashSet<>(
-        pm.getKillableContainersMap("a", RMNodeLabelsManager.NO_LABEL)
-            .keySet());
-
-    // Update request resource of c from 1 to 2, so we need to preempt
-    // one more container
-    am2.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>());
-
-    // Call editPolicy.editSchedule() once, we should have 1 container in to-preempt map
-    // and 1 container in killable map
-    editPolicy.editSchedule();
-    Assert.assertEquals(1, editPolicy.getToPreemptContainers().size());
-
-    // Call editPolicy.editSchedule() once more, we should have 2 containers killable map
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-
-    // Check if previous killable containers included by new killable containers
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
-    Assert.assertTrue(
-        Sets.difference(previousKillableContainers, killableContainers.keySet())
-            .isEmpty());
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionPolicyCleanupKillableContainersWhenNoPreemptionNeeded()
-      throws Exception {
-    /**
-     * Test case:
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     * Submit applications to two queues, one uses more than the other, so
-     * preemption will happen.
-     *
-     * Check:
-     * 1) Containers will be marked to killable
-     * 2) Cancel resource request
-     * 3) Killable containers will be cancelled from policy and scheduler
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 6 times for node1
-    for (int i = 0; i < 6; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
-
-    // NM1 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    am2.allocate("*", 3 * GB, 1, new ArrayList<ContainerId>());
-
-    // Get edit policy and do one update
-    ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if 3 container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 3);
-
-    // Change reqeust from 3G to 2G, now we can preempt one less container. (3->2)
-    am2.allocate("*", 2 * GB, 1, new ArrayList<ContainerId>());
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
-
-    // Call editSchedule once more to make sure still nothing happens
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionConsidersUserLimit()
-      throws Exception {
-    /**
-     * Test case: Submit two application (app1/app2) to different queues, queue
-     * structure:
-     *
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     *
-     * Queue-c's user-limit-factor = 0.1, so single user cannot allocate >1 containers in queue-c
-     *
-     * 1) Two nodes in the cluster, each of them has 4G.
-     *
-     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
-     * more resource available.
-     *
-     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
-     *
-     * Now the cluster is fulfilled.
-     *
-     * 4) app2 asks for another 1G container, system will preempt one container
-     * from app1, and app2 will receive the preempted container
-     */
-    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf);
-    csConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".c", 0.1f);
-    MockRM rm1 = new MockRM(csConf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if no container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    // No preemption happens
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 0);
-    Assert.assertEquals(0, killableContainers.size());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 7 containers, and app2 has 1 containers (nothing preempted)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  private Map<ContainerId, RMContainer> waitKillableContainersSize(
-      PreemptionManager pm, String queueName, String partition,
-      int expectedSize) throws InterruptedException {
-    Map<ContainerId, RMContainer> killableContainers =
-        pm.getKillableContainersMap(queueName, partition);
-
-    int wait = 0;
-    // Wait for at most 5 sec (it should be super fast actually)
-    while (expectedSize != killableContainers.size() && wait < 500) {
-      killableContainers = pm.getKillableContainersMap(queueName, partition);
-      Thread.sleep(10);
-      wait++;
-    }
-
-    Assert.assertEquals(expectedSize, killableContainers.size());
-    return killableContainers;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 1612201..5169337 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -100,7 +99,6 @@ public class TestChildQueueOrder {
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
   }
 
   private FiCaSchedulerApp getMockApplication(int appId, String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 87a3d51..69b0813 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -71,7 +71,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -151,7 +150,6 @@ public class TestLeafQueue {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
@@ -3094,7 +3092,6 @@ public class TestLeafQueue {
         Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(2 * GB, 2));
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     return csContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
index 1ee201d..bbf6e43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -1677,100 +1676,4 @@ public class TestNodeLabelContainerAllocation {
     checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
         cs.getApplicationAttempt(am1.getApplicationAttemptId()));
   }
-
-  @Test
-  public void testParentQueueMaxCapsAreRespected() throws Exception {
-    /*
-     * Queue tree:
-     *          Root
-     *        /     \
-     *       A       B
-     *      / \
-     *     A1 A2
-     *
-     * A has 50% capacity and 50% max capacity (of label=x)
-     * A1/A2 has 50% capacity and 100% max capacity (of label=x)
-     * Cluster has one node (label=x) with resource = 24G.
-     * So we can at most use 12G resources under queueA.
-     */
-    CapacitySchedulerConfiguration csConf =
-        new CapacitySchedulerConfiguration(this.conf);
-
-    // Define top-level queues
-    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
-        "b"});
-    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
-
-    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    csConf.setCapacity(A, 10);
-    csConf.setAccessibleNodeLabels(A, toSet("x"));
-    csConf.setCapacityByLabel(A, "x", 50);
-    csConf.setMaximumCapacityByLabel(A, "x", 50);
-
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    csConf.setCapacity(B, 90);
-    csConf.setAccessibleNodeLabels(B, toSet("x"));
-    csConf.setCapacityByLabel(B, "x", 50);
-    csConf.setMaximumCapacityByLabel(B, "x", 50);
-
-    // Define 2nd-level queues
-    csConf.setQueues(A, new String[] { "a1",
-        "a2"});
-
-    final String A1 = A + ".a1";
-    csConf.setCapacity(A1, 50);
-    csConf.setAccessibleNodeLabels(A1, toSet("x"));
-    csConf.setCapacityByLabel(A1, "x", 50);
-    csConf.setMaximumCapacityByLabel(A1, "x", 100);
-    csConf.setUserLimitFactor(A1, 100.0f);
-
-    final String A2 = A + ".a2";
-    csConf.setCapacity(A2, 50);
-    csConf.setAccessibleNodeLabels(A2, toSet("x"));
-    csConf.setCapacityByLabel(A2, "x", 50);
-    csConf.setMaximumCapacityByLabel(A2, "x", 100);
-    csConf.setUserLimitFactor(A2, 100.0f);
-
-    // set node -> label
-    mgr.addToCluserNodeLabels(ImmutableSet.of(
-        NodeLabel.newInstance("x", false)));
-    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
-
-    // inject node label manager
-    MockRM rm = new MockRM(csConf) {
-      @Override
-      public RMNodeLabelsManager createNodeLabelManager() {
-        return mgr;
-      }
-    };
-
-    rm.getRMContext().setNodeLabelManager(mgr);
-    rm.start();
-
-    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
-
-    MockNM nm1 =
-        new MockNM("h1:1234", 24 * GB, rm.getResourceTrackerService());
-    nm1.registerNode();
-
-    // Launch app1 in a1, resource usage is 1GB (am) + 4GB * 2 = 9GB
-    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1", "x");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
-    am1.allocate("*", 4 * GB, 2, new ArrayList<ContainerId>(), "x");
-    doNMHeartbeat(rm, nm1.getNodeId(), 10);
-    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
-        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
-
-    // Try to launch app2 in a2, asked 2GB, should success
-    RMApp app2 = rm.submitApp(2 * GB, "app", "user", null, "a2", "x");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
-
-    // am2 asks more resources, cannot success because current used = 9G (app1)
-    // + 2G (app2) = 11G, and queue's max capacity = 12G
-    am2.allocate("*", 2 * GB, 2, new ArrayList<ContainerId>(), "x");
-
-    doNMHeartbeat(rm, nm1.getNodeId(), 10);
-    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
-        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index 23dc860..f73baa4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@@ -93,7 +92,6 @@ public class TestParentQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
     thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index 56facee..2ef5e39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -127,7 +126,6 @@ public class TestReservations {
     when(csContext.getNonPartitionedQueueComparator()).thenReturn(
         CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
         conf);


[6/6] hadoop git commit: YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by wa...@apache.org.
YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

(cherry picked from commit 7e8c9beb4156dcaeb3a11e60aaa06d2370626913)


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

Branch: refs/heads/trunk
Commit: ae14e5d07f1b6702a5160637438028bb03d9387e
Parents: fa7a435
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Mar 16 16:59:59 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Mar 16 17:02:33 2016 -0700

----------------------------------------------------------------------
 .../ProportionalCapacityPreemptionPolicy.java   | 166 +++--
 .../rmcontainer/RMContainer.java                |   1 +
 .../scheduler/PreemptableResourceScheduler.java |   2 +-
 .../scheduler/ResourceLimits.java               |   9 +
 .../scheduler/SchedulerNode.java                |   9 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  45 +-
 .../scheduler/capacity/CSAssignment.java        |  11 +
 .../scheduler/capacity/CapacityScheduler.java   | 132 +++-
 .../CapacitySchedulerConfiguration.java         |  14 +-
 .../capacity/CapacitySchedulerContext.java      |  15 +-
 .../scheduler/capacity/LeafQueue.java           |  69 ++
 .../scheduler/capacity/ParentQueue.java         | 157 ++++-
 .../allocator/AbstractContainerAllocator.java   |   2 +
 .../capacity/allocator/ContainerAllocation.java |  12 +
 .../allocator/RegularContainerAllocator.java    |  39 +-
 .../capacity/preemption/KillableContainer.java  |  45 ++
 .../capacity/preemption/PreemptableQueue.java   | 102 +++
 .../capacity/preemption/PreemptionManager.java  | 165 +++++
 .../scheduler/common/AssignmentInformation.java |   6 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  32 +-
 .../common/fica/FiCaSchedulerNode.java          |  65 +-
 .../scheduler/event/SchedulerEventType.java     |  15 +-
 .../resourcemanager/TestRMDispatcher.java       |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |   2 +-
 .../applicationsmanager/TestAMRestart.java      |   7 +-
 ...estProportionalCapacityPreemptionPolicy.java |   6 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   2 +
 .../capacity/TestApplicationLimits.java         |   2 +
 .../capacity/TestApplicationPriority.java       |   6 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../TestCapacitySchedulerPreemption.java        | 677 +++++++++++++++++++
 .../scheduler/capacity/TestChildQueueOrder.java |   2 +
 .../scheduler/capacity/TestLeafQueue.java       |   3 +
 .../TestNodeLabelContainerAllocation.java       |  97 +++
 .../scheduler/capacity/TestParentQueue.java     |   2 +
 .../scheduler/capacity/TestReservations.java    |   2 +
 .../scheduler/capacity/TestUtils.java           |  36 +
 .../fair/TestFairSchedulerPreemption.java       |   2 +-
 38 files changed, 1785 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 3a87edb..9b499c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -35,7 +35,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 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.preemption.PreemptableQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -125,8 +126,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private long maxWaitTime;
   private CapacityScheduler scheduler;
   private long monitoringInterval;
-  private final Map<RMContainer,Long> preempted =
-    new HashMap<RMContainer,Long>();
+  private final Map<RMContainer, Long> preempted = new HashMap<>();
+
   private ResourceCalculator rc;
   private float percentageClusterPreemptionAllowed;
   private double naturalTerminationFactor;
@@ -135,6 +136,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       new HashMap<>();
   private RMNodeLabelsManager nlm;
 
+  // Preemptable Entities, synced from scheduler at every run
+  private Map<String, PreemptableQueue> preemptableEntities = null;
+  private Set<ContainerId> killableContainers;
+
   public ProportionalCapacityPreemptionPolicy() {
     clock = SystemClock.getInstance();
   }
@@ -184,6 +189,64 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     Resource clusterResources = Resources.clone(scheduler.getClusterResource());
     containerBasedPreemptOrKill(root, clusterResources);
   }
+
+  @SuppressWarnings("unchecked")
+  private void cleanupStaledKillableContainers(Resource cluster,
+      Set<String> leafQueueNames) {
+    for (String q : leafQueueNames) {
+      for (TempQueuePerPartition tq : getQueuePartitions(q)) {
+        // When queue's used - killable <= guaranteed and, killable > 0, we need
+        // to check if any of killable containers needs to be reverted
+        if (Resources.lessThanOrEqual(rc, cluster,
+            Resources.subtract(tq.current, tq.killable), tq.idealAssigned)
+            && Resources.greaterThan(rc, cluster, tq.killable, Resources.none())) {
+          // How many killable resources need to be reverted
+          // need-to-revert = already-marked-killable - (current - ideal)
+          Resource toBeRevertedFromKillable = Resources.subtract(tq.killable,
+              Resources.subtract(tq.current, tq.idealAssigned));
+
+          Resource alreadyReverted = Resources.createResource(0);
+
+          for (RMContainer c : preemptableEntities.get(q).getKillableContainers(
+              tq.partition).values()) {
+            if (Resources.greaterThanOrEqual(rc, cluster, alreadyReverted,
+                toBeRevertedFromKillable)) {
+              break;
+            }
+
+            if (Resources.greaterThan(rc, cluster,
+                Resources.add(alreadyReverted, c.getAllocatedResource()),
+                toBeRevertedFromKillable)) {
+              continue;
+            } else {
+              // This container need to be marked to unkillable
+              Resources.addTo(alreadyReverted, c.getAllocatedResource());
+              rmContext.getDispatcher().getEventHandler().handle(
+                  new ContainerPreemptEvent(c.getApplicationAttemptId(), c,
+                      SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE));
+            }
+          }
+
+        }
+      }
+    }
+  }
+
+  private void syncKillableContainersFromScheduler() {
+    // sync preemptable entities from scheduler
+    preemptableEntities =
+        scheduler.getPreemptionManager().getShallowCopyOfPreemptableEntities();
+
+    killableContainers = new HashSet<>();
+    for (Map.Entry<String, PreemptableQueue> entry : preemptableEntities
+        .entrySet()) {
+      PreemptableQueue entity = entry.getValue();
+      for (Map<ContainerId, RMContainer> map : entity.getKillableContainers()
+          .values()) {
+        killableContainers.addAll(map.keySet());
+      }
+    }
+  }
   
   /**
    * This method selects and tracks containers to be preempted. If a container
@@ -201,6 +264,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         .getNodeLabelManager().getClusterNodeLabelNames());
     allPartitions.add(RMNodeLabelsManager.NO_LABEL);
 
+    syncKillableContainersFromScheduler();
+
     // extract a summary of the queues from scheduler
     synchronized (scheduler) {
       queueToPartitions.clear();
@@ -228,13 +293,17 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
     }
 
+    // remove containers from killable list when we want to preempt less resources
+    // from queue.
+    cleanupStaledKillableContainers(clusterResources, leafQueueNames);
+
     // based on ideal allocation select containers to be preempted from each
     // queue and each application
     Map<ApplicationAttemptId,Set<RMContainer>> toPreempt =
         getContainersToPreempt(leafQueueNames, clusterResources);
 
     if (LOG.isDebugEnabled()) {
-      logToCSV(new ArrayList<String>(leafQueueNames));
+      logToCSV(new ArrayList<>(leafQueueNames));
     }
 
     // if we are in observeOnly mode return before any action is taken
@@ -254,10 +323,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         // if we tried to preempt this for more than maxWaitTime
         if (preempted.get(container) != null &&
             preempted.get(container) + maxWaitTime < clock.getTime()) {
-          // kill it
+          // mark container killable
           rmContext.getDispatcher().getEventHandler().handle(
               new ContainerPreemptEvent(appAttemptId, container,
-                  SchedulerEventType.KILL_PREEMPTED_CONTAINER));
+                  SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
           preempted.remove(container);
         } else {
           if (preempted.get(container) != null) {
@@ -333,14 +402,14 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // qAlloc tracks currently active queues (will decrease progressively as
     // demand is met)
-    List<TempQueuePerPartition> qAlloc = new ArrayList<TempQueuePerPartition>(queues);
+    List<TempQueuePerPartition> qAlloc = new ArrayList<>(queues);
     // unassigned tracks how much resources are still to assign, initialized
     // with the total capacity for this set of queues
     Resource unassigned = Resources.clone(tot_guarant);
 
     // group queues based on whether they have non-zero guaranteed capacity
-    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<TempQueuePerPartition>();
-    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<TempQueuePerPartition>();
+    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<>();
+    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<>();
 
     for (TempQueuePerPartition q : qAlloc) {
       if (Resources
@@ -415,8 +484,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // idealAssigned >= current + pending), remove it from consideration.
     // Sort queues from most under-guaranteed to most over-guaranteed.
     TQComparator tqComparator = new TQComparator(rc, tot_guarant);
-    PriorityQueue<TempQueuePerPartition> orderedByNeed =
-        new PriorityQueue<TempQueuePerPartition>(10, tqComparator);
+    PriorityQueue<TempQueuePerPartition> orderedByNeed = new PriorityQueue<>(10,
+        tqComparator);
     for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
       TempQueuePerPartition q = i.next();
       if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
@@ -474,7 +543,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   // percentage of guaranteed.
   protected Collection<TempQueuePerPartition> getMostUnderservedQueues(
       PriorityQueue<TempQueuePerPartition> orderedByNeed, TQComparator tqComparator) {
-    ArrayList<TempQueuePerPartition> underserved = new ArrayList<TempQueuePerPartition>();
+    ArrayList<TempQueuePerPartition> underserved = new ArrayList<>();
     while (!orderedByNeed.isEmpty()) {
       TempQueuePerPartition q1 = orderedByNeed.remove();
       underserved.add(q1);
@@ -502,7 +571,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     
     if (ignoreGuar) {
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = (float)  1.0f / ((float) queues.size());
+        q.normalizedGuarantee = 1.0f / queues.size();
       }
     } else {
       for (TempQueuePerPartition q : queues) {
@@ -515,8 +584,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
   }
 
-  private String getPartitionByNodeId(NodeId nodeId) {
-    return scheduler.getSchedulerNode(nodeId).getPartition();
+  private String getPartitionByRMContainer(RMContainer rmContainer) {
+    return scheduler.getSchedulerNode(rmContainer.getAllocatedNode())
+        .getPartition();
   }
 
   /**
@@ -534,7 +604,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return false;
     }
 
-    String nodePartition = getPartitionByNodeId(rmContainer.getAllocatedNode());
+    String nodePartition = getPartitionByRMContainer(rmContainer);
     Resource toObtainByPartition =
         resourceToObtainByPartitions.get(nodePartition);
 
@@ -575,7 +645,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       ApplicationAttemptId appAttemptId, RMContainer containerToPreempt) {
     Set<RMContainer> set;
     if (null == (set = preemptMap.get(appAttemptId))) {
-      set = new HashSet<RMContainer>();
+      set = new HashSet<>();
       preemptMap.put(appAttemptId, set);
     }
     set.add(containerToPreempt);
@@ -587,7 +657,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * over-capacity queue. It uses {@link #NATURAL_TERMINATION_FACTOR} to
    * account for containers that will naturally complete.
    *
-   * @param queues set of leaf queues to preempt from
+   * @param leafQueueNames set of leaf queues to preempt from
    * @param clusterResource total amount of cluster resources
    * @return a map of applciationID to set of containers to preempt
    */
@@ -595,8 +665,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Set<String> leafQueueNames, Resource clusterResource) {
 
     Map<ApplicationAttemptId, Set<RMContainer>> preemptMap =
-        new HashMap<ApplicationAttemptId, Set<RMContainer>>();
-    List<RMContainer> skippedAMContainerlist = new ArrayList<RMContainer>();
+        new HashMap<>();
+    List<RMContainer> skippedAMContainerlist = new ArrayList<>();
 
     // Loop all leaf queues
     for (String queueName : leafQueueNames) {
@@ -614,7 +684,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       LeafQueue leafQueue = null;
 
       Map<String, Resource> resToObtainByPartition =
-          new HashMap<String, Resource>();
+          new HashMap<>();
       for (TempQueuePerPartition qT : getQueuePartitions(queueName)) {
         leafQueue = qT.leafQueue;
         // we act only if we are violating balance by more than
@@ -703,7 +773,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param clusterResource
    * @param preemptMap
    * @param skippedAMContainerlist
-   * @param resToObtain
    * @param skippedAMSize
    * @param maxAMCapacityForThisQueue
    */
@@ -751,7 +820,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // first drop reserved containers towards rsrcPreempt
     List<RMContainer> reservedContainers =
-        new ArrayList<RMContainer>(app.getReservedContainers());
+        new ArrayList<>(app.getReservedContainers());
     for (RMContainer c : reservedContainers) {
       if (resToObtainByPartition.isEmpty()) {
         return;
@@ -771,8 +840,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // if more resources are to be freed go through all live containers in
     // reverse priority and reverse allocation order and mark them for
     // preemption
-    List<RMContainer> liveContainers =
-      new ArrayList<RMContainer>(app.getLiveContainers());
+    List<RMContainer> liveContainers = new ArrayList<>(app.getLiveContainers());
 
     sortContainers(liveContainers);
 
@@ -788,6 +856,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         continue;
       }
 
+      // Skip already marked to killable containers
+      if (killableContainers.contains(c.getContainerId())) {
+        continue;
+      }
+
       // Try to preempt this container
       tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
           clusterResource, preemptMap);
@@ -826,6 +899,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     return "ProportionalCapacityPreemptionPolicy";
   }
 
+  @VisibleForTesting
+  public Map<RMContainer, Long> getToPreemptContainers() {
+    return preempted;
+  }
 
   /**
    * This method walks a tree of CSQueue and clones the portion of the state
@@ -851,6 +928,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           partitionToLookAt);
       Resource guaranteed = Resources.multiply(partitionResource, absCap);
       Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
+      Resource killable = Resources.none();
+      if (null != preemptableEntities.get(queueName)) {
+         killable = preemptableEntities.get(queueName)
+            .getKillableResource(partitionToLookAt);
+      }
 
       // when partition is a non-exclusive partition, the actual maxCapacity
       // could more than specified maxCapacity
@@ -875,7 +957,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
               l.getTotalPendingResourcesConsideringUserLimit(
                   partitionResource, partitionToLookAt);
         ret = new TempQueuePerPartition(queueName, current, pending, guaranteed,
-            maxCapacity, preemptionDisabled, partitionToLookAt);
+            maxCapacity, preemptionDisabled, partitionToLookAt, killable);
         if (preemptionDisabled) {
           ret.untouchableExtra = extra;
         } else {
@@ -886,7 +968,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         Resource pending = Resource.newInstance(0, 0);
         ret =
             new TempQueuePerPartition(curQueue.getQueueName(), current, pending,
-                guaranteed, maxCapacity, false, partitionToLookAt);
+                guaranteed, maxCapacity, false, partitionToLookAt, killable);
         Resource childrensPreemptable = Resource.newInstance(0, 0);
         for (CSQueue c : curQueue.getChildQueues()) {
           TempQueuePerPartition subq =
@@ -932,7 +1014,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     Map<String, TempQueuePerPartition> queuePartitions;
     if (null == (queuePartitions = queueToPartitions.get(queueName))) {
-      queuePartitions = new HashMap<String, TempQueuePerPartition>();
+      queuePartitions = new HashMap<>();
       queueToPartitions.put(queueName, queuePartitions);
     }
     queuePartitions.put(queuePartition.partition, queuePartition);
@@ -971,8 +1053,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     final Resource guaranteed;
     final Resource maxCapacity;
     final String partition;
+    final Resource killable;
     Resource idealAssigned;
     Resource toBePreempted;
+
     // For logging purpose
     Resource actuallyPreempted;
     Resource untouchableExtra;
@@ -986,7 +1070,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     TempQueuePerPartition(String queueName, Resource current, Resource pending,
         Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled,
-        String partition) {
+        String partition, Resource killableResource) {
       this.queueName = queueName;
       this.current = current;
       this.pending = pending;
@@ -996,11 +1080,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.actuallyPreempted = Resource.newInstance(0, 0);
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
-      this.children = new ArrayList<TempQueuePerPartition>();
+      this.children = new ArrayList<>();
       this.untouchableExtra = Resource.newInstance(0, 0);
       this.preemptableExtra = Resource.newInstance(0, 0);
       this.preemptionDisabled = preemptionDisabled;
       this.partition = partition;
+      this.killable = killableResource;
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -1018,12 +1103,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Resources.addTo(pending, q.pending);
     }
 
-    public void addChildren(ArrayList<TempQueuePerPartition> queues) {
-      assert leafQueue == null;
-      children.addAll(queues);
-    }
-
-
     public ArrayList<TempQueuePerPartition> getChildren(){
       return children;
     }
@@ -1064,18 +1143,13 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return sb.toString();
     }
 
-    public void printAll() {
-      LOG.info(this.toString());
-      for (TempQueuePerPartition sub : this.getChildren()) {
-        sub.printAll();
-      }
-    }
-
     public void assignPreemption(float scalingFactor,
         ResourceCalculator rc, Resource clusterResource) {
-      if (Resources.greaterThan(rc, clusterResource, current, idealAssigned)) {
-          toBePreempted = Resources.multiply(
-              Resources.subtract(current, idealAssigned), scalingFactor);
+      if (Resources.greaterThan(rc, clusterResource,
+          Resources.subtract(current, killable), idealAssigned)) {
+        toBePreempted = Resources.multiply(Resources.subtract(
+            Resources.subtract(current, killable), idealAssigned),
+            scalingFactor);
       } else {
         toBePreempted = Resource.newInstance(0, 0);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.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/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index 5d26931..dfe0886 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 
 /**
  * Represents the ResourceManager's view of an application container. See 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.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/PreemptableResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
index ee7e101..b73c538 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
@@ -45,6 +45,6 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
    * Ask the scheduler to forcibly interrupt the container given as input
    * @param container
    */
-  void killPreemptedContainer(RMContainer container);
+  void markContainerForKillable(RMContainer container);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.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/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
index c545e9e..721eb36 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
@@ -38,6 +38,8 @@ public class ResourceLimits {
   // containers.
   private volatile Resource headroom;
 
+  private boolean allowPreempt = false;
+
   public ResourceLimits(Resource limit) {
     this(limit, Resources.none());
   }
@@ -72,4 +74,11 @@ public class ResourceLimits {
     this.amountNeededUnreserve = amountNeededUnreserve;
   }
 
+  public boolean isAllowPreemption() {
+    return allowPreempt;
+  }
+
+  public void setIsAllowPreemption(boolean allowPreempt) {
+   this.allowPreempt = allowPreempt;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 33ab2f1..6c4f300 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -64,9 +64,8 @@ public abstract class SchedulerNode {
   private volatile ResourceUtilization nodeUtilization =
       ResourceUtilization.newInstance(0, 0, 0f);
 
-
-  /** Set of containers that are allocated containers. */
-  private final Map<ContainerId, RMContainer> launchedContainers =
+  /* set of containers that are allocated containers */
+  protected final Map<ContainerId, RMContainer> launchedContainers =
       new HashMap<>();
 
   private final RMNode rmNode;
@@ -168,7 +167,7 @@ public abstract class SchedulerNode {
    * @param deltaResource Change in the resource allocation.
    * @param increase True if the change is an increase of allocation.
    */
-  private synchronized void changeContainerResource(ContainerId containerId,
+  protected synchronized void changeContainerResource(ContainerId containerId,
       Resource deltaResource, boolean increase) {
     if (increase) {
       deductUnallocatedResource(deltaResource);
@@ -242,7 +241,7 @@ public abstract class SchedulerNode {
    * Update the resources of the node when allocating a new container.
    * @param container Container to allocate.
    */
-  private synchronized void updateResource(Container container) {
+  protected synchronized void updateResource(Container container) {
     addUnallocatedResource(container.getResource());
     --numContainers;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index 39ca29b..955f8fa 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -440,11 +442,8 @@ public abstract class AbstractCSQueue implements CSQueue {
           Resources.multiplyAndNormalizeDown(resourceCalculator,
               labelManager.getResourceByLabel(nodePartition, clusterResource),
               queueCapacities.getAbsoluteMaximumCapacity(nodePartition), minimumAllocation);
-      if (nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
-        return Resources.min(resourceCalculator, clusterResource,
-            queueMaxResource, currentResourceLimits.getLimit());
-      }
-      return queueMaxResource;  
+      return Resources.min(resourceCalculator, clusterResource,
+          queueMaxResource, currentResourceLimits.getLimit());
     } else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
       // When we doing non-exclusive resource allocation, maximum capacity of
       // all queues on this label equals to total resource with the label.
@@ -474,12 +473,19 @@ public abstract class AbstractCSQueue implements CSQueue {
 
     Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
 
-    // Set headroom for currentResourceLimits
-    currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource,
-        nowTotalUsed));
+    // Set headroom for currentResourceLimits:
+    // When queue is a parent queue: Headroom = limit - used + killable
+    // When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
+    Resource usedExceptKillable = nowTotalUsed;
+    if (null != getChildQueues() && !getChildQueues().isEmpty()) {
+      usedExceptKillable = Resources.subtract(nowTotalUsed,
+          getTotalKillableResource(nodePartition));
+    }
+    currentResourceLimits.setHeadroom(
+        Resources.subtract(currentLimitResource, usedExceptKillable));
 
     if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
-        nowTotalUsed, currentLimitResource)) {
+        usedExceptKillable, currentLimitResource)) {
 
       // if reservation continous looking enabled, check to see if could we
       // potentially use this node instead of a reserved node if the application
@@ -491,7 +497,7 @@ public abstract class AbstractCSQueue implements CSQueue {
               resourceCouldBeUnreserved, Resources.none())) {
         // resource-without-reserved = used - reserved
         Resource newTotalWithoutReservedResource =
-            Resources.subtract(nowTotalUsed, resourceCouldBeUnreserved);
+            Resources.subtract(usedExceptKillable, resourceCouldBeUnreserved);
 
         // when total-used-without-reserved-resource < currentLimit, we still
         // have chance to allocate on this node by unreserving some containers
@@ -620,11 +626,10 @@ public abstract class AbstractCSQueue implements CSQueue {
     // considering all labels in cluster, only those labels which are
     // use some resource of this queue can be considered.
     Set<String> nodeLabels = new HashSet<String>();
-    if (this.getAccessibleNodeLabels() != null
-        && this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
-      nodeLabels.addAll(Sets.union(this.getQueueCapacities()
-          .getNodePartitionsSet(), this.getQueueResourceUsage()
-          .getNodePartitionsSet()));
+    if (this.getAccessibleNodeLabels() != null && this.getAccessibleNodeLabels()
+        .contains(RMNodeLabelsManager.ANY)) {
+      nodeLabels.addAll(Sets.union(this.getQueueCapacities().getNodePartitionsSet(),
+          this.getQueueResourceUsage().getNodePartitionsSet()));
     } else {
       nodeLabels.addAll(this.getAccessibleNodeLabels());
     }
@@ -636,4 +641,14 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     return nodeLabels;
   }
+
+  public Resource getTotalKillableResource(String partition) {
+    return csContext.getPreemptionManager().getKillableResource(queueName,
+        partition);
+  }
+
+  public Iterator<RMContainer> getKillableContainers(String partition) {
+    return csContext.getPreemptionManager().getKillableContainers(queueName,
+        partition);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.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/CSAssignment.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/CSAssignment.java
index 68f6f12..6406efe 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/CSAssignment.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/CSAssignment.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.Assignment
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.List;
+
 @Private
 @Unstable
 public class CSAssignment {
@@ -42,6 +44,7 @@ public class CSAssignment {
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
   private boolean increaseAllocation;
+  private List<RMContainer> containersToKill;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -147,4 +150,12 @@ public class CSAssignment {
   public void setIncreasedAllocation(boolean flag) {
     increaseAllocation = flag;
   }
+
+  public void setContainersToKill(List<RMContainer> containersToKill) {
+    this.containersToKill = containersToKill;
+  }
+
+  public List<RMContainer> getContainersToKill() {
+    return containersToKill;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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
----------------------------------------------------------------------
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/CapacityScheduler.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/CapacityScheduler.java
index 735306a..cf5c3b5 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/CapacityScheduler.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/CapacityScheduler.java
@@ -108,6 +108,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -148,6 +150,10 @@ public class CapacityScheduler extends
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
+  private PreemptionManager preemptionManager = new PreemptionManager();
+
+  private volatile boolean isLazyPreemptionEnabled = false;
+
   static final Comparator<CSQueue> nonPartitionedQueueComparator =
       new Comparator<CSQueue>() {
     @Override
@@ -298,12 +304,11 @@ public class CapacityScheduler extends
     initMaximumResourceCapability(this.conf.getMaximumAllocation());
     this.calculator = this.conf.getResourceCalculator();
     this.usePortForNodeName = this.conf.getUsePortForNodeName();
-    this.applications =
-        new ConcurrentHashMap<ApplicationId,
-            SchedulerApplication<FiCaSchedulerApp>>();
+    this.applications = new ConcurrentHashMap<>();
     this.labelManager = rmContext.getNodeLabelManager();
     authorizer = YarnAuthorizationProvider.getInstance(yarnConf);
     initializeQueues(this.conf);
+    this.isLazyPreemptionEnabled = conf.getLazyPreemptionEnabled();
 
     scheduleAsynchronously = this.conf.getScheduleAynschronously();
     asyncScheduleInterval =
@@ -369,6 +374,9 @@ public class CapacityScheduler extends
       refreshMaximumAllocation(this.conf.getMaximumAllocation());
       throw new IOException("Failed to re-init queues", t);
     }
+
+    // update lazy preemption
+    this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled();
   }
   
   long getAsyncScheduleInterval() {
@@ -503,6 +511,9 @@ public class CapacityScheduler extends
     LOG.info("Initialized root queue " + root);
     updatePlacementRules();
     setQueueAcls(authorizer, queues);
+
+    // Notify Preemption Manager
+    preemptionManager.refreshQueues(null, root);
   }
 
   @Lock(CapacityScheduler.class)
@@ -531,6 +542,9 @@ public class CapacityScheduler extends
 
     labelManager.reinitializeQueueLabels(getQueueToLabels());
     setQueueAcls(authorizer, queues);
+
+    // Notify Preemption Manager
+    preemptionManager.refreshQueues(null, root);
   }
 
   @VisibleForTesting
@@ -1253,8 +1267,10 @@ public class CapacityScheduler extends
 
     // Try to schedule more if there are no reservations to fulfill
     if (node.getReservedContainer() == null) {
-      if (calculator.computeAvailableContainers(node.getUnallocatedResource(),
-        minimumAllocation) > 0) {
+      if (calculator.computeAvailableContainers(Resources
+              .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
+          minimumAllocation) > 0) {
+
         if (LOG.isDebugEnabled()) {
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getUnallocatedResource());
@@ -1263,10 +1279,8 @@ public class CapacityScheduler extends
         assignment = root.assignContainers(
             getClusterResource(),
             node,
-            // TODO, now we only consider limits for parent for non-labeled
-            // resources, should consider labeled resources as well.
             new ResourceLimits(labelManager.getResourceByLabel(
-                RMNodeLabelsManager.NO_LABEL, getClusterResource())),
+                node.getPartition(), getClusterResource())),
             SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         if (Resources.greaterThan(calculator, getClusterResource(),
             assignment.getResource(), Resources.none())) {
@@ -1436,11 +1450,20 @@ public class CapacityScheduler extends
       markContainerForPreemption(aid, containerToBePreempted);
     }
     break;
-    case KILL_PREEMPTED_CONTAINER:
+    case MARK_CONTAINER_FOR_KILLABLE:
+    {
+      ContainerPreemptEvent containerKillableEvent = (ContainerPreemptEvent)event;
+      RMContainer killableContainer = containerKillableEvent.getContainer();
+      markContainerForKillable(killableContainer);
+    }
+    break;
+    case MARK_CONTAINER_FOR_NONKILLABLE:
     {
-      ContainerPreemptEvent killContainerEvent = (ContainerPreemptEvent)event;
-      RMContainer containerToBeKilled = killContainerEvent.getContainer();
-      killPreemptedContainer(containerToBeKilled);
+      if (isLazyPreemptionEnabled) {
+        ContainerPreemptEvent cancelKillContainerEvent =
+            (ContainerPreemptEvent) event;
+        markContainerForNonKillable(cancelKillContainerEvent.getContainer());
+      }
     }
     break;
     default:
@@ -1548,14 +1571,14 @@ public class CapacityScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
-    
     Container container = rmContainer.getContainer();
+    ContainerId containerId = container.getId();
     
     // Get the application for the finished container
     FiCaSchedulerApp application =
         getCurrentAttemptForContainer(container.getId());
     ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
+        containerId.getApplicationAttemptId().getApplicationId();
     if (application == null) {
       LOG.info("Container " + container + " of" + " finished application "
           + appId + " completed with event " + event);
@@ -1569,15 +1592,6 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue)application.getQueue();
     queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
-
-    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
-      schedulerHealth.updatePreemption(Time.now(), container.getNodeId(),
-        container.getId(), queue.getQueuePath());
-      schedulerHealth.updateSchedulerPreemptionCounts(1);
-    } else {
-      schedulerHealth.updateRelease(lastNodeUpdateTime, container.getNodeId(),
-        container.getId(), queue.getQueuePath());
-    }
   }
   
   @Override
@@ -1613,7 +1627,7 @@ public class CapacityScheduler extends
       ApplicationAttemptId applicationAttemptId) {
     return super.getApplicationAttempt(applicationAttemptId);
   }
-  
+
   @Lock(Lock.NoLock.class)
   public FiCaSchedulerNode getNode(NodeId nodeId) {
     return nodeTracker.getNode(nodeId);
@@ -1654,15 +1668,60 @@ public class CapacityScheduler extends
     }
   }
 
-  @Override
-  public void killPreemptedContainer(RMContainer cont) {
+  public synchronized void markContainerForKillable(
+      RMContainer killableContainer) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(SchedulerEventType.KILL_PREEMPTED_CONTAINER + ": container"
-          + cont.toString());
+      LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
+          + killableContainer.toString());
+    }
+
+    if (!isLazyPreemptionEnabled) {
+      super.completedContainer(killableContainer, SchedulerUtils
+          .createPreemptedContainerStatus(killableContainer.getContainerId(),
+              SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
+    } else {
+      FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
+          killableContainer.getAllocatedNode());
+
+      FiCaSchedulerApp application = getCurrentAttemptForContainer(
+          killableContainer.getContainerId());
+
+      node.markContainerToKillable(killableContainer.getContainerId());
+
+      // notify PreemptionManager
+      // Get the application for the finished container
+      if (null != application) {
+        String leafQueueName = application.getCSLeafQueue().getQueueName();
+        getPreemptionManager().addKillableContainer(
+            new KillableContainer(killableContainer, node.getPartition(),
+                leafQueueName));
+      }    }
+  }
+
+  private synchronized void markContainerForNonKillable(
+      RMContainer nonKillableContainer) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
+              + nonKillableContainer.toString());
+    }
+
+    FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
+        nonKillableContainer.getAllocatedNode());
+
+    FiCaSchedulerApp application = getCurrentAttemptForContainer(
+        nonKillableContainer.getContainerId());
+
+    node.markContainerToNonKillable(nonKillableContainer.getContainerId());
+
+    // notify PreemptionManager
+    // Get the application for the finished container
+    if (null != application) {
+      String leafQueueName = application.getCSLeafQueue().getQueueName();
+      getPreemptionManager().removeKillableContainer(
+          new KillableContainer(nonKillableContainer, node.getPartition(),
+              leafQueueName));
     }
-    super.completedContainer(cont, SchedulerUtils
-        .createPreemptedContainerStatus(cont.getContainerId(),
-        SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
   }
 
   @Override
@@ -1945,6 +2004,7 @@ public class CapacityScheduler extends
     return ret;
   }
 
+  @Override
   public SchedulerHealth getSchedulerHealth() {
     return this.schedulerHealth;
   }
@@ -1954,6 +2014,11 @@ public class CapacityScheduler extends
   }
 
   @Override
+  public long getLastNodeUpdateTime() {
+    return lastNodeUpdateTime;
+  }
+
+  @Override
   public Priority checkAndGetApplicationPriority(Priority priorityFromContext,
       String user, String queueName, ApplicationId applicationId)
       throws YarnException {
@@ -2054,4 +2119,9 @@ public class CapacityScheduler extends
         + rmApp.getQueue() + " for application: " + applicationId
         + " for the user: " + rmApp.getUser());
   }
+
+  @Override
+  public PreemptionManager getPreemptionManager() {
+    return preemptionManager;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 3756d9e..3729264 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -257,6 +257,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final String RESERVATION_ENFORCEMENT_WINDOW =
       "reservation-enforcement-window";
 
+  @Private
+  public static final String LAZY_PREEMPTION_ENALBED = PREFIX + "lazy-preemption-enabled";
+
+  @Private
+  public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
+
   public CapacitySchedulerConfiguration() {
     this(new Configuration());
   }
@@ -1007,7 +1013,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @VisibleForTesting
   public void setOrderingPolicyParameter(String queue,
       String parameterKey, String parameterValue) {
-    set(getQueuePrefix(queue) + ORDERING_POLICY + "."
-        + parameterKey, parameterValue);
+    set(getQueuePrefix(queue) + ORDERING_POLICY + "." + parameterKey,
+        parameterValue);
+  }
+
+  public boolean getLazyPreemptionEnabled() {
+    return getBoolean(LAZY_PREEMPTION_ENALBED, DEFAULT_LAZY_PREEMPTION_ENABLED);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
index 2a0dd0d..1203272 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
@@ -18,17 +18,20 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import java.util.Comparator;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
+import java.util.Comparator;
+
 /**
  * Read-only interface to {@link CapacityScheduler} context.
  */
@@ -61,4 +64,12 @@ public interface CapacitySchedulerContext {
   PartitionedQueueComparator getPartitionedQueueComparator();
   
   FiCaSchedulerNode getNode(NodeId nodeId);
+
+  FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId attemptId);
+
+  PreemptionManager getPreemptionManager();
+
+  SchedulerHealth getSchedulerHealth();
+
+  long getLastNodeUpdateTime();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/LeafQueue.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/LeafQueue.java
index c625fae..3dc2090 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/LeafQueue.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/LeafQueue.java
@@ -37,9 +37,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -63,7 +65,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps;
@@ -823,6 +827,40 @@ public class LeafQueue extends AbstractCSQueue {
       assignment.setExcessReservation(null);
     }
   }
+
+  private void killToPreemptContainers(Resource clusterResource,
+      FiCaSchedulerNode node,
+      CSAssignment assignment) {
+    if (assignment.getContainersToKill() != null) {
+      StringBuilder sb = new StringBuilder("Killing containers: [");
+
+      for (RMContainer c : assignment.getContainersToKill()) {
+        FiCaSchedulerApp application = csContext.getApplicationAttempt(
+            c.getApplicationAttemptId());
+        LeafQueue q = application.getCSLeafQueue();
+        q.completedContainer(clusterResource, application, node, c, SchedulerUtils
+                .createPreemptedContainerStatus(c.getContainerId(),
+                    SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
+            null, false);
+        sb.append("(container=" + c.getContainerId() + " resource=" + c
+            .getAllocatedResource() + ")");
+      }
+
+      sb.append("] for container=" + assignment.getAssignmentInformation()
+          .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment
+          .getResource());
+      LOG.info(sb.toString());
+
+    }
+  }
+
+  private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
+    // Set preemption-allowed:
+    // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
+    float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition);
+    float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition);
+    limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
+  }
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
@@ -835,6 +873,8 @@ public class LeafQueue extends AbstractCSQueue {
           + " #applications=" + orderingPolicy.getNumSchedulableEntities());
     }
 
+    setPreemptionAllowed(currentResourceLimits, node.getPartition());
+
     // Check for reserved resources
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
@@ -846,6 +886,7 @@ public class LeafQueue extends AbstractCSQueue {
                 currentResourceLimits, schedulingMode, reservedContainer);
         handleExcessReservedContainer(clusterResource, assignment, node,
             application);
+        killToPreemptContainers(clusterResource, node, assignment);
         return assignment;
       }
     }
@@ -907,6 +948,7 @@ public class LeafQueue extends AbstractCSQueue {
       
       handleExcessReservedContainer(clusterResource, assignment, node,
           application);
+      killToPreemptContainers(clusterResource, node, assignment);
 
       if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
           Resources.none())) {
@@ -1210,11 +1252,34 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
+  private void updateSchedulerHealthForCompletedContainer(
+      RMContainer rmContainer, ContainerStatus containerStatus) {
+    // Update SchedulerHealth for released / preempted container
+    SchedulerHealth schedulerHealth = csContext.getSchedulerHealth();
+    if (null == schedulerHealth) {
+      // Only do update if we have schedulerHealth
+      return;
+    }
+
+    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
+      schedulerHealth.updatePreemption(Time.now(), rmContainer.getAllocatedNode(),
+          rmContainer.getContainerId(), getQueuePath());
+      schedulerHealth.updateSchedulerPreemptionCounts(1);
+    } else {
+      schedulerHealth.updateRelease(csContext.getLastNodeUpdateTime(),
+          rmContainer.getAllocatedNode(), rmContainer.getContainerId(),
+          getQueuePath());
+    }
+  }
+
   @Override
   public void completedContainer(Resource clusterResource, 
       FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, 
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue,
       boolean sortQueues) {
+    // Update SchedulerHealth for released / preempted container
+    updateSchedulerHealthForCompletedContainer(rmContainer, containerStatus);
+
     if (application != null) {
       // unreserve container increase request if it previously reserved.
       if (rmContainer.hasIncreaseReservation()) {
@@ -1265,6 +1330,10 @@ public class LeafQueue extends AbstractCSQueue {
           rmContainer, null, event, this, sortQueues);
       }
     }
+
+    // Notify PreemptionManager
+    csContext.getPreemptionManager().removeKillableContainer(
+        new KillableContainer(rmContainer, node.getPartition(), queueName));
   }
 
   synchronized void allocateResource(Resource clusterResource,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.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/ParentQueue.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/ParentQueue.java
index 7cf5565..6fcd6c1 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/ParentQueue.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/ParentQueue.java
@@ -18,18 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -57,12 +46,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -386,6 +388,11 @@ public class ParentQueue extends AbstractCSQueue {
     // if our queue cannot access this node, just return
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
         && !accessibleToPartition(node.getPartition())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skip this queue=" + getQueuePath()
+            + ", because it is not able to access partition=" + node
+            .getPartition());
+      }
       return CSAssignment.NULL_ASSIGNMENT;
     }
     
@@ -431,7 +438,7 @@ public class ParentQueue extends AbstractCSQueue {
               resourceCalculator, clusterResource, 
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
-        super.allocateResource(clusterResource, assignedToChild.getResource(),
+        allocateResource(clusterResource, assignedToChild.getResource(),
             node.getPartition(), assignedToChild.isIncreasedAllocation());
         
         // Track resource utilization in this pass of the scheduler
@@ -494,29 +501,38 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
-    return (node.getReservedContainer() == null) && 
-        Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
-            node.getUnallocatedResource(), minimumAllocation);
+    // Two conditions need to meet when trying to allocate:
+    // 1) Node doesn't have reserved container
+    // 2) Node's available-resource + killable-resource should > 0
+    return node.getReservedContainer() == null && Resources.greaterThanOrEqual(
+        resourceCalculator, clusterResource, Resources
+            .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
+        minimumAllocation);
   }
-  
+
   private ResourceLimits getResourceLimitsOfChild(CSQueue child,
-      Resource clusterResource, ResourceLimits parentLimits) {
+      Resource clusterResource, ResourceLimits parentLimits,
+      String nodePartition) {
     // Set resource-limit of a given child, child.limit =
     // min(my.limit - my.used + child.used, child.max)
 
     // Parent available resource = parent-limit - parent-used-resource
-    Resource parentMaxAvailableResource =
-        Resources.subtract(parentLimits.getLimit(), getUsedResources());
+    Resource parentMaxAvailableResource = Resources.subtract(
+        parentLimits.getLimit(), queueUsage.getUsed(nodePartition));
+    // Deduct killable from used
+    Resources.addTo(parentMaxAvailableResource,
+        getTotalKillableResource(nodePartition));
 
     // Child's limit = parent-available-resource + child-used
-    Resource childLimit =
-        Resources.add(parentMaxAvailableResource, child.getUsedResources());
+    Resource childLimit = Resources.add(parentMaxAvailableResource,
+        child.getQueueResourceUsage().getUsed(nodePartition));
 
     // Get child's max resource
-    Resource childConfiguredMaxResource =
-        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
-            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
-            child.getAbsoluteMaximumCapacity(), minimumAllocation);
+    Resource childConfiguredMaxResource = Resources.multiplyAndNormalizeDown(
+        resourceCalculator,
+        labelManager.getResourceByLabel(nodePartition, clusterResource),
+        child.getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition),
+        minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =
@@ -568,7 +584,7 @@ public class ParentQueue extends AbstractCSQueue {
 
       // Get ResourceLimits of child queue before assign containers
       ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, cluster, limits);
+          getResourceLimitsOfChild(childQueue, cluster, limits, node.getPartition());
       
       assignment = childQueue.assignContainers(cluster, node, 
           childLimits, schedulingMode);
@@ -714,8 +730,8 @@ public class ParentQueue extends AbstractCSQueue {
     // Update all children
     for (CSQueue childQueue : childQueues) {
       // Get ResourceLimits of child queue before assign containers
-      ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, clusterResource, resourceLimits);     
+      ResourceLimits childLimits = getResourceLimitsOfChild(childQueue,
+          clusterResource, resourceLimits, RMNodeLabelsManager.NO_LABEL);
       childQueue.updateClusterResource(clusterResource, childLimits);
     }
     
@@ -738,8 +754,8 @@ public class ParentQueue extends AbstractCSQueue {
     synchronized (this) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getPartition(), false);
+      allocateResource(clusterResource,
+          rmContainer.getContainer().getResource(), node.getPartition(), false);
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -766,7 +782,7 @@ public class ParentQueue extends AbstractCSQueue {
     if (application != null) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      super.allocateResource(clusterResource, rmContainer.getContainer()
+      allocateResource(clusterResource, rmContainer.getContainer()
           .getResource(), node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
@@ -802,4 +818,79 @@ public class ParentQueue extends AbstractCSQueue {
   public synchronized int getNumApplications() {
     return numApplications;
   }
+
+  synchronized void allocateResource(Resource clusterResource,
+      Resource resource, String nodePartition, boolean changeContainerResource) {
+    super.allocateResource(clusterResource, resource, nodePartition,
+        changeContainerResource);
+
+    /**
+     * check if we need to kill (killable) containers if maximum resource violated.
+     * Doing this because we will deduct killable resource when going from root.
+     * For example:
+     * <pre>
+     *      Root
+     *      /   \
+     *     a     b
+     *   /  \
+     *  a1  a2
+     * </pre>
+     *
+     * a: max=10G, used=10G, killable=2G
+     * a1: used=8G, killable=2G
+     * a2: used=2G, pending=2G, killable=0G
+     *
+     * When we get queue-a to allocate resource, even if queue-a
+     * reaches its max resource, we deduct its used by killable, so we can allocate
+     * at most 2G resources. ResourceLimits passed down to a2 has headroom set to 2G.
+     *
+     * If scheduler finds a 2G available resource in existing cluster, and assigns it
+     * to a2, now a2's used= 2G + 2G = 4G, and a's used = 8G + 4G = 12G > 10G
+     *
+     * When this happens, we have to preempt killable container (on same or different
+     * nodes) of parent queue to avoid violating parent's max resource.
+     */
+    if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition)
+        < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
+      killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
+    }
+  }
+
+  private void killContainersToEnforceMaxQueueCapacity(String partition,
+      Resource clusterResource) {
+    Iterator<RMContainer> killableContainerIter = getKillableContainers(
+        partition);
+    if (!killableContainerIter.hasNext()) {
+      return;
+    }
+
+    Resource partitionResource = labelManager.getResourceByLabel(partition,
+        null);
+    Resource maxResource = Resources.multiply(partitionResource,
+        getQueueCapacities().getAbsoluteMaximumCapacity(partition));
+
+    while (Resources.greaterThan(resourceCalculator, partitionResource,
+        queueUsage.getUsed(partition), maxResource)) {
+      RMContainer toKillContainer = killableContainerIter.next();
+      FiCaSchedulerApp attempt = csContext.getApplicationAttempt(
+          toKillContainer.getContainerId().getApplicationAttemptId());
+      FiCaSchedulerNode node = csContext.getNode(
+          toKillContainer.getAllocatedNode());
+      if (null != attempt && null != node) {
+        LeafQueue lq = attempt.getCSLeafQueue();
+        lq.completedContainer(clusterResource, attempt, node, toKillContainer,
+            SchedulerUtils.createPreemptedContainerStatus(
+                toKillContainer.getContainerId(),
+                SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
+            null, false);
+        LOG.info("Killed container=" + toKillContainer.getContainerId()
+            + " from queue=" + lq.getQueueName() + " to make queue=" + this
+            .getQueueName() + "'s max-capacity enforced");
+      }
+
+      if (!killableContainerIter.hasNext()) {
+        break;
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.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/AbstractContainerAllocator.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/AbstractContainerAllocator.java
index ee01bd1..afac235 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/AbstractContainerAllocator.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/AbstractContainerAllocator.java
@@ -108,6 +108,8 @@ public abstract class AbstractContainerAllocator {
           assignment.setFulfilledReservation(true);
         }
       }
+
+      assignment.setContainersToKill(result.getToKillContainers());
     }
     
     return assignment;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.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/ContainerAllocation.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/ContainerAllocation.java
index 1df9410..8f749f6 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/ContainerAllocation.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/ContainerAllocation.java
@@ -19,11 +19,14 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
 
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.List;
+
 public class ContainerAllocation {
   /**
    * Skip the locality (e.g. node-local, rack-local, any), and look at other
@@ -56,6 +59,7 @@ public class ContainerAllocation {
   NodeType containerNodeType = NodeType.NODE_LOCAL;
   NodeType requestNodeType = NodeType.NODE_LOCAL;
   Container updatedContainer;
+  private List<RMContainer> toKillContainers;
 
   public ContainerAllocation(RMContainer containerToBeUnreserved,
       Resource resourceToBeAllocated, AllocationState state) {
@@ -86,4 +90,12 @@ public class ContainerAllocation {
   public Container getUpdatedContainer() {
     return updatedContainer;
   }
+
+  public void setToKillContainers(List<RMContainer> toKillContainers) {
+    this.toKillContainers = toKillContainers;
+  }
+
+  public List<RMContainer> getToKillContainers() {
+    return toKillContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 e168edf..a5ca2d8 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
@@ -42,6 +42,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Allocate normal (new) containers, considers locality/label, etc. Using
  * delayed scheduling mechanism to get better locality allocation.
@@ -435,9 +438,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       return ContainerAllocation.LOCALITY_SKIPPED;
     }
 
-    assert Resources.greaterThan(
-        rc, clusterResource, available, Resources.none());
-
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
         priority, capability);
 
@@ -460,6 +460,29 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     boolean reservationsContinueLooking =
         application.getCSLeafQueue().getReservationContinueLooking();
 
+    // Check if we need to kill some containers to allocate this one
+    List<RMContainer> toKillContainers = null;
+    if (availableContainers == 0 && currentResoureLimits.isAllowPreemption()) {
+      Resource availableAndKillable = Resources.clone(available);
+      for (RMContainer killableContainer : node
+          .getKillableContainers().values()) {
+        if (null == toKillContainers) {
+          toKillContainers = new ArrayList<>();
+        }
+        toKillContainers.add(killableContainer);
+        Resources.addTo(availableAndKillable,
+                        killableContainer.getAllocatedResource());
+        if (Resources.fitsIn(rc,
+                             clusterResource,
+                             capability,
+                             availableAndKillable)) {
+          // Stop if we find enough spaces
+          availableContainers = 1;
+          break;
+        }
+      }
+    }
+
     if (availableContainers > 0) {
       // Allocate...
       // We will only do continuous reservation when this is not allocated from
@@ -499,12 +522,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
           new ContainerAllocation(unreservedContainer, request.getCapability(),
               AllocationState.ALLOCATED);
       result.containerNodeType = type;
+      result.setToKillContainers(toKillContainers);
       return result;
     } else {
       // if we are allowed to allocate but this node doesn't have space, reserve
       // it or if this was an already a reserved container, reserve it again
       if (shouldAllocOrReserveNewContainer || rmContainer != null) {
-
         if (reservationsContinueLooking && rmContainer == null) {
           // we could possibly ignoring queue capacity or user limits when
           // reservationsContinueLooking is set. Make sure we didn't need to
@@ -522,6 +545,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             new ContainerAllocation(null, request.getCapability(),
                 AllocationState.RESERVED);
         result.containerNodeType = type;
+        result.setToKillContainers(null);
         return result;
       }
       // Skip the locality request
@@ -613,8 +637,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
 
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
-      Resource clusterResource, FiCaSchedulerNode node,
-      SchedulingMode schedulingMode, Priority priority,
+      FiCaSchedulerNode node, Priority priority,
       RMContainer reservedContainer) {
     // Create the container if necessary
     Container container =
@@ -678,9 +701,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
     if (AllocationState.ALLOCATED == result.state
         || AllocationState.RESERVED == result.state) {
-      result =
-          doAllocation(result, clusterResource, node, schedulingMode, priority,
-              reservedContainer);
+      result = doAllocation(result, node, priority, reservedContainer);
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.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/preemption/KillableContainer.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/preemption/KillableContainer.java
new file mode 100644
index 0000000..675b0b4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption;
+
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+
+public class KillableContainer {
+  RMContainer container;
+  String partition;
+  String leafQueueName;
+
+  public KillableContainer(RMContainer container, String partition, String leafQueueName) {
+    this.container = container;
+    this.partition = partition;
+    this.leafQueueName = leafQueueName;
+  }
+
+  public RMContainer getRMContainer() {
+    return this.container;
+  }
+
+  public String getNodePartition() {
+    return this.partition;
+  }
+
+  public String getLeafQueueName() {
+    return this.leafQueueName;
+  }
+}


[5/6] hadoop git commit: YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.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/preemption/PreemptableQueue.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/preemption/PreemptableQueue.java
new file mode 100644
index 0000000..19148d7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+public class PreemptableQueue {
+  // Partition -> killable resources and containers
+  private Map<String, Resource> totalKillableResources = new HashMap<>();
+  private Map<String, Map<ContainerId, RMContainer>> killableContainers =
+      new HashMap<>();
+  private PreemptableQueue parent;
+
+  public PreemptableQueue(PreemptableQueue parent) {
+    this.parent = parent;
+  }
+
+  public PreemptableQueue(Map<String, Resource> totalKillableResources,
+      Map<String, Map<ContainerId, RMContainer>> killableContainers) {
+    this.totalKillableResources = totalKillableResources;
+    this.killableContainers = killableContainers;
+  }
+
+  void addKillableContainer(KillableContainer container) {
+    String partition = container.getNodePartition();
+    if (!totalKillableResources.containsKey(partition)) {
+      totalKillableResources.put(partition, Resources.createResource(0));
+      killableContainers.put(partition,
+          new ConcurrentSkipListMap<ContainerId, RMContainer>());
+    }
+
+    RMContainer c = container.getRMContainer();
+    Resources.addTo(totalKillableResources.get(partition),
+        c.getAllocatedResource());
+    killableContainers.get(partition).put(c.getContainerId(), c);
+
+    if (null != parent) {
+      parent.addKillableContainer(container);
+    }
+  }
+
+  void removeKillableContainer(KillableContainer container) {
+    String partition = container.getNodePartition();
+    Map<ContainerId, RMContainer> partitionKillableContainers =
+        killableContainers.get(partition);
+    if (partitionKillableContainers != null) {
+      RMContainer rmContainer = partitionKillableContainers.remove(
+          container.getRMContainer().getContainerId());
+      if (null != rmContainer) {
+        Resources.subtractFrom(totalKillableResources.get(partition),
+            rmContainer.getAllocatedResource());
+      }
+    }
+
+    if (null != parent) {
+      parent.removeKillableContainer(container);
+    }
+  }
+
+  public Resource getKillableResource(String partition) {
+    Resource res = totalKillableResources.get(partition);
+    return res == null ? Resources.none() : res;
+  }
+
+  @SuppressWarnings("unchecked")
+  public Map<ContainerId, RMContainer> getKillableContainers(String partition) {
+    Map<ContainerId, RMContainer> map = killableContainers.get(partition);
+    return map == null ? Collections.EMPTY_MAP : map;
+  }
+
+  public Map<String, Map<ContainerId, RMContainer>> getKillableContainers() {
+    return killableContainers;
+  }
+
+  Map<String, Resource> getTotalKillableResources() {
+    return totalKillableResources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.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/preemption/PreemptionManager.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/preemption/PreemptionManager.java
new file mode 100644
index 0000000..a9f02a5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.java
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class PreemptionManager {
+  private ReentrantReadWriteLock.ReadLock readLock;
+  private ReentrantReadWriteLock.WriteLock writeLock;
+  private Map<String, PreemptableQueue> entities = new HashMap<>();
+
+  public PreemptionManager() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  public void refreshQueues(CSQueue parent, CSQueue current) {
+    try {
+      writeLock.lock();
+      PreemptableQueue parentEntity = null;
+      if (parent != null) {
+        parentEntity = entities.get(parent.getQueueName());
+      }
+
+      if (!entities.containsKey(current.getQueueName())) {
+        entities.put(current.getQueueName(),
+            new PreemptableQueue(parentEntity));
+      }
+
+      if (current.getChildQueues() != null) {
+        for (CSQueue child : current.getChildQueues()) {
+          refreshQueues(current, child);
+        }
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void addKillableContainer(KillableContainer container) {
+    try {
+      writeLock.lock();
+      PreemptableQueue entity = entities.get(container.getLeafQueueName());
+      if (null != entity) {
+        entity.addKillableContainer(container);
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void removeKillableContainer(KillableContainer container) {
+    try {
+      writeLock.lock();
+      PreemptableQueue entity = entities.get(container.getLeafQueueName());
+      if (null != entity) {
+        entity.removeKillableContainer(container);
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void moveKillableContainer(KillableContainer oldContainer,
+      KillableContainer newContainer) {
+    // TODO, will be called when partition of the node changed OR
+    // container moved to different queue
+  }
+
+  public void updateKillableContainerResource(KillableContainer container,
+      Resource oldResource, Resource newResource) {
+    // TODO, will be called when container's resource changed
+  }
+
+  @VisibleForTesting
+  public Map<ContainerId, RMContainer> getKillableContainersMap(
+      String queueName, String partition) {
+    try {
+      readLock.lock();
+      PreemptableQueue entity = entities.get(queueName);
+      if (entity != null) {
+        Map<ContainerId, RMContainer> containers =
+            entity.getKillableContainers().get(partition);
+        if (containers != null) {
+          return containers;
+        }
+      }
+      return Collections.emptyMap();
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Iterator<RMContainer> getKillableContainers(String queueName,
+      String partition) {
+    return getKillableContainersMap(queueName, partition).values().iterator();
+  }
+
+  public Resource getKillableResource(String queueName, String partition) {
+    try {
+      readLock.lock();
+      PreemptableQueue entity = entities.get(queueName);
+      if (entity != null) {
+        Resource res = entity.getTotalKillableResources().get(partition);
+        if (res == null || res.equals(Resources.none())) {
+          return Resources.none();
+        }
+        return Resources.clone(res);
+      }
+      return Resources.none();
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Map<String, PreemptableQueue> getShallowCopyOfPreemptableEntities() {
+    try {
+      readLock.lock();
+      Map<String, PreemptableQueue> map = new HashMap<>();
+      for (Map.Entry<String, PreemptableQueue> entry : entities.entrySet()) {
+        String key = entry.getKey();
+        PreemptableQueue entity = entry.getValue();
+        map.put(key, new PreemptableQueue(
+            new HashMap<>(entity.getTotalKillableResources()),
+            new HashMap<>(entity.getKillableContainers())));
+      }
+      return map;
+    } finally {
+      readLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.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/AssignmentInformation.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/AssignmentInformation.java
index 5158255..aad3bc7 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/AssignmentInformation.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/AssignmentInformation.java
@@ -120,9 +120,9 @@ public class AssignmentInformation {
   }
 
   private ContainerId getFirstContainerIdFromOperation(Operation op) {
-    if (null != operationDetails.get(Operation.ALLOCATION)) {
+    if (null != operationDetails.get(op)) {
       List<AssignmentDetails> assignDetails =
-          operationDetails.get(Operation.ALLOCATION);
+          operationDetails.get(op);
       if (!assignDetails.isEmpty()) {
         return assignDetails.get(0).containerId;
       }
@@ -131,7 +131,7 @@ public class AssignmentInformation {
   }
 
   public ContainerId getFirstAllocatedOrReservedContainerId() {
-    ContainerId containerId = null;
+    ContainerId containerId;
     containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
     if (null != containerId) {
       return containerId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 4d563cd..f474aad 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
@@ -59,6 +59,7 @@ 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;
@@ -94,6 +95,7 @@ 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,
@@ -138,28 +140,30 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     
     containerAllocator = new ContainerAllocator(this, rc, rmContext);
+
+    if (scheduler instanceof CapacityScheduler) {
+      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
+    }
   }
 
-  synchronized public boolean containerCompleted(RMContainer rmContainer,
+  public synchronized boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
+    ContainerId containerId = rmContainer.getContainerId();
 
     // Remove from the list of containers
-    if (null == liveContainers.remove(rmContainer.getContainerId())) {
+    if (null == liveContainers.remove(containerId)) {
       return false;
     }
-    
+
     // Remove from the list of newly allocated containers if found
     newlyAllocatedContainers.remove(rmContainer);
 
-    Container container = rmContainer.getContainer();
-    ContainerId containerId = container.getId();
-
     // Inform the container
     rmContainer.handle(
         new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(rmContainer.getContainerId());
+    containersToPreempt.remove(containerId);
 
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
@@ -176,7 +180,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return true;
   }
 
-  synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
+  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       Priority priority, ResourceRequest request, 
       Container container) {
 
@@ -200,7 +204,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);
-    liveContainers.put(container.getId(), rmContainer);    
+
+    ContainerId containerId = container.getId();
+    liveContainers.put(containerId, rmContainer);
 
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
@@ -213,17 +219,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Inform the container
     rmContainer.handle(
-        new RMContainerEvent(container.getId(), RMContainerEventType.START));
+        new RMContainerEvent(containerId, RMContainerEventType.START));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("allocate: applicationAttemptId=" 
-          + container.getId().getApplicationAttemptId() 
-          + " container=" + container.getId() + " host="
+          + containerId.getApplicationAttemptId()
+          + " container=" + containerId + " host="
           + container.getNodeId().getHost() + " type=" + type);
     }
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), container.getId());
+        getApplicationId(), containerId);
     
     return rmContainer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
index fe6db47..1d0e78a 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
@@ -18,22 +18,29 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
 
-
-import java.util.Set;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 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.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 public class FiCaSchedulerNode extends SchedulerNode {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class);
+  private Map<ContainerId, RMContainer> killableContainers = new HashMap<>();
+  private Resource totalKillableResources = Resource.newInstance(0, 0);
   
   public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName,
       Set<String> nodeLabels) {
@@ -92,7 +99,6 @@ public class FiCaSchedulerNode extends SchedulerNode {
   @Override
   public synchronized void unreserveResource(
       SchedulerApplicationAttempt application) {
-
     // adding NP checks as this can now be called for preemption
     if (getReservedContainer() != null
         && getReservedContainer().getContainer() != null
@@ -115,4 +121,55 @@ public class FiCaSchedulerNode extends SchedulerNode {
     }
     setReservedContainer(null);
   }
+
+  // According to decisions from preemption policy, mark the container to killable
+  public synchronized void markContainerToKillable(ContainerId containerId) {
+    RMContainer c = launchedContainers.get(containerId);
+    if (c != null && !killableContainers.containsKey(containerId)) {
+      killableContainers.put(containerId, c);
+      Resources.addTo(totalKillableResources, c.getAllocatedResource());
+    }
+  }
+
+  // According to decisions from preemption policy, mark the container to
+  // non-killable
+  public synchronized void markContainerToNonKillable(ContainerId containerId) {
+    RMContainer c = launchedContainers.get(containerId);
+    if (c != null && killableContainers.containsKey(containerId)) {
+      killableContainers.remove(containerId);
+      Resources.subtractFrom(totalKillableResources, c.getAllocatedResource());
+    }
+  }
+
+  @Override
+  protected synchronized void updateResource(
+      Container container) {
+    super.updateResource(container);
+    if (killableContainers.containsKey(container.getId())) {
+      Resources.subtractFrom(totalKillableResources, container.getResource());
+      killableContainers.remove(container.getId());
+    }
+  }
+
+  @Override
+  protected synchronized void changeContainerResource(ContainerId containerId,
+      Resource deltaResource, boolean increase) {
+    super.changeContainerResource(containerId, deltaResource, increase);
+
+    if (killableContainers.containsKey(containerId)) {
+      if (increase) {
+        Resources.addTo(totalKillableResources, deltaResource);
+      } else {
+        Resources.subtractFrom(totalKillableResources, deltaResource);
+      }
+    }
+  }
+
+  public synchronized Resource getTotalKillableResources() {
+    return totalKillableResources;
+  }
+
+  public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
+    return killableContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.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/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index 9cf09e9..35b7c14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -38,10 +38,15 @@ public enum SchedulerEventType {
   // Source: ContainerAllocationExpirer
   CONTAINER_EXPIRED,
 
-  // Source: SchedulingEditPolicy
+  /* Source: SchedulingEditPolicy */
   KILL_RESERVED_CONTAINER,
-  MARK_CONTAINER_FOR_PREEMPTION, // Mark a container for preemption
-                                 // in the near future
-  KILL_PREEMPTED_CONTAINER // Kill a container previously marked for
-                           // preemption
+
+  // Mark a container for preemption
+  MARK_CONTAINER_FOR_PREEMPTION,
+
+  // Mark a for-preemption container killable
+  MARK_CONTAINER_FOR_KILLABLE,
+
+  // Cancel a killable container
+  MARK_CONTAINER_FOR_NONKILLABLE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
index d9306dd..c944752 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
@@ -59,7 +59,7 @@ public class TestRMDispatcher {
       rmDispatcher.getEventHandler().handle(event1);
       ContainerPreemptEvent event2 =
           new ContainerPreemptEvent(appAttemptId, container,
-            SchedulerEventType.KILL_PREEMPTED_CONTAINER);
+            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE);
       rmDispatcher.getEventHandler().handle(event2);
       ContainerPreemptEvent event3 =
           new ContainerPreemptEvent(appAttemptId, container,
@@ -70,7 +70,7 @@ public class TestRMDispatcher {
       verify(sched, times(3)).handle(any(SchedulerEvent.class));
       verify(sched).killReservedContainer(container);
       verify(sched).markContainerForPreemption(appAttemptId, container);
-      verify(sched).killPreemptedContainer(container);
+      verify(sched).markContainerForKillable(container);
     } catch (InterruptedException e) {
       Assert.fail();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 028afb1..3057615 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -2352,7 +2352,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       FiCaSchedulerApp schedulerAppAttempt = cs.getSchedulerApplications()
           .get(app0.getApplicationId()).getCurrentAppAttempt();
       // kill app0-attempt
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(
           app0.getCurrentAppAttempt().getMasterContainer().getId()));
       am0.waitForState(RMAppAttemptState.FAILED);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 5035afe..16f3f60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
@@ -566,7 +565,7 @@ public class TestAMRestart {
     ContainerId amContainer =
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     // Preempt the first attempt;
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
@@ -582,7 +581,7 @@ public class TestAMRestart {
     // Preempt the second attempt.
     ContainerId amContainer2 =
         ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer2));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer2));
 
     am2.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt2.shouldCountTowardsMaxAttemptRetry());
@@ -677,7 +676,7 @@ public class TestAMRestart {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
 
     // Forcibly preempt the am container;
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 13f267d..e9129de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.Pro
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_PREEMPTED_CONTAINER;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -167,6 +168,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(mCS.getConfiguration()).thenReturn(schedConf);
     rmContext = mock(RMContext.class);
     when(mCS.getRMContext()).thenReturn(rmContext);
+    when(mCS.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(rmContext.getNodeLabelManager()).thenReturn(lm);
     mDisp = mock(EventHandler.class);
     Dispatcher disp = mock(Dispatcher.class);
@@ -289,7 +291,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     List<ContainerPreemptEvent> events = evtCaptor.getAllValues();
     for (ContainerPreemptEvent e : events.subList(20, 20)) {
       assertEquals(appC, e.getAppId());
-      assertEquals(KILL_PREEMPTED_CONTAINER, e.getType());
+      assertEquals(MARK_CONTAINER_FOR_KILLABLE, e.getType());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 512f37c..21ea495 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -123,6 +124,7 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     mClock = mock(Clock.class);
     cs = mock(CapacityScheduler.class);
     when(cs.getResourceCalculator()).thenReturn(rc);
+    when(cs.getPreemptionManager()).thenReturn(new PreemptionManager());
 
     nlm = mock(RMNodeLabelsManager.class);
     mDisp = mock(EventHandler.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
index 0b32676..171196f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -264,6 +265,7 @@ public class TestApplicationLimits {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
index 1569a12..d8161f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.java
@@ -205,7 +205,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check node report, 12 GB used and 4 GB available
@@ -512,7 +512,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 
@@ -542,7 +542,7 @@ public class TestApplicationPriority {
       if (++counter > 1) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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
----------------------------------------------------------------------
diff --git a/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 b/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
index b6c005b..16ba607 100644
--- a/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
+++ b/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
@@ -1188,7 +1188,7 @@ public class TestCapacityScheduler {
 
     // kill the 3 containers
     for (Container c : allocatedContainers) {
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1197,7 +1197,7 @@ public class TestCapacityScheduler {
         Resource.newInstance(CONTAINER_MEMORY * 3, 3), false, 3);
 
     // kill app0-attempt0 AM container
-    cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(app0
+    cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(app0
         .getCurrentAppAttempt().getMasterContainer().getId()));
 
     // wait for app0 failed
@@ -1220,7 +1220,7 @@ public class TestCapacityScheduler {
     allocatedContainers =
         am1.allocateAndWaitForContainers(3, CONTAINER_MEMORY, nm1);
     for (Container c : allocatedContainers) {
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1269,7 +1269,7 @@ public class TestCapacityScheduler {
     }
 
     // Call killContainer to preempt the container
-    cs.killPreemptedContainer(rmContainer);
+    cs.markContainerForKillable(rmContainer);
 
     Assert.assertEquals(3, requests.size());
     for (ResourceRequest request : requests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
new file mode 100644
index 0000000..bea7797
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
@@ -0,0 +1,677 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.RMActiveServices;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+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.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestCapacitySchedulerPreemption {
+  private static final Log LOG = LogFactory.getLog(
+      TestCapacitySchedulerPreemption.class);
+
+  private final int GB = 1024;
+
+  private Configuration conf;
+
+  RMNodeLabelsManager mgr;
+
+  Clock clock;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+        ProportionalCapacityPreemptionPolicy.class, SchedulingEditPolicy.class);
+    conf = TestUtils.getConfigurationWithMultipleQueues(this.conf);
+
+    // Set preemption related configurations
+    conf.setInt(ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL,
+        0);
+    conf.setBoolean(CapacitySchedulerConfiguration.LAZY_PREEMPTION_ENALBED,
+        true);
+    conf.setFloat(
+        ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND, 1.0f);
+    conf.setFloat(
+        ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR, 1.0f);
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(this.conf);
+    clock = mock(Clock.class);
+    when(clock.getTime()).thenReturn(0L);
+  }
+
+  private SchedulingEditPolicy getSchedulingEditPolicy(MockRM rm) {
+    RMActiveServices activeServices = rm.getRMActiveService();
+    SchedulingMonitor mon = null;
+    for (Service service : activeServices.getServices()) {
+      if (service instanceof SchedulingMonitor) {
+        mon = (SchedulingMonitor) service;
+        break;
+      }
+    }
+
+    if (mon != null) {
+      return mon.getSchedulingEditPolicy();
+    }
+    return null;
+  }
+
+  @Test (timeout = 60000)
+  public void testSimplePreemption() throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     *
+     * 1) Two nodes in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
+     * more resource available.
+     *
+     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
+     *
+     * Now the cluster is fulfilled.
+     *
+     * 4) app2 asks for another 1G container, system will preempt one container
+     * from app1, and app2 will receive the preempted container
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 7, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(1, killableContainers.size());
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 6 containers, and app2 has 2 containers
+    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersNodeLocalityDelay()
+      throws Exception {
+    /**
+     * Test case: same as testSimplePreemption steps 1-3.
+     *
+     * Step 4: app2 asks for 1G container with locality specified, so it needs
+     * to wait for missed-opportunity before get scheduled.
+     * Check if system waits missed-opportunity before finish killable container
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container with unknown host and unknown rack
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1), ResourceRequest
+        .newInstance(Priority.newInstance(1), "unknownhost",
+            Resources.createResource(1 * GB), 1), ResourceRequest
+        .newInstance(Priority.newInstance(1), "/default-rack",
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    // Do allocation again, one container will be preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // App1 has 6 containers, and app2 has 2 containers (new container allocated)
+    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersHardNodeLocality()
+      throws Exception {
+    /**
+     * Test case: same as testSimplePreemption steps 1-3.
+     *
+     * Step 4: app2 asks for 1G container with hard locality specified, and
+     *         asked host is not existed
+     * Confirm system doesn't preempt any container.
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container for h3 with hard locality,
+    // h3 doesn't exist in the cluster
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1, true), ResourceRequest
+        .newInstance(Priority.newInstance(1), "h3",
+            Resources.createResource(1 * GB), 1, false), ResourceRequest
+        .newInstance(Priority.newInstance(1), "/default-rack",
+            Resources.createResource(1 * GB), 1, false)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    // Do allocation again, nothing will be preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // App1 has 7 containers, and app2 has 1 containers (no container allocated)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionPolicyShouldRespectAlreadyMarkedKillableContainers()
+      throws Exception {
+    /**
+     * Test case:
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     * Submit applications to two queues, one uses more than the other, so
+     * preemption will happen.
+     *
+     * Check:
+     * 1) Killable containers resources will be excluded from PCPP (no duplicated
+     *    container added to killable list)
+     * 2) When more resources need to be preempted, new containers will be selected
+     *    and killable containers will be considered
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 6 times for node1
+    for (int i = 0; i < 6; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // NM1 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
+
+    // Get edit policy and do one update
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+
+    // Check killable containers and to-be-preempted containers in edit policy
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Run edit schedule again, confirm status doesn't changed
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Save current to kill containers
+    Set<ContainerId> previousKillableContainers = new HashSet<>(
+        pm.getKillableContainersMap("a", RMNodeLabelsManager.NO_LABEL)
+            .keySet());
+
+    // Update request resource of c from 1 to 2, so we need to preempt
+    // one more container
+    am2.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>());
+
+    // Call editPolicy.editSchedule() once, we should have 1 container in to-preempt map
+    // and 1 container in killable map
+    editPolicy.editSchedule();
+    Assert.assertEquals(1, editPolicy.getToPreemptContainers().size());
+
+    // Call editPolicy.editSchedule() once more, we should have 2 containers killable map
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Check if previous killable containers included by new killable containers
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+    Assert.assertTrue(
+        Sets.difference(previousKillableContainers, killableContainers.keySet())
+            .isEmpty());
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionPolicyCleanupKillableContainersWhenNoPreemptionNeeded()
+      throws Exception {
+    /**
+     * Test case:
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     * Submit applications to two queues, one uses more than the other, so
+     * preemption will happen.
+     *
+     * Check:
+     * 1) Containers will be marked to killable
+     * 2) Cancel resource request
+     * 3) Killable containers will be cancelled from policy and scheduler
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 6 times for node1
+    for (int i = 0; i < 6; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // NM1 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    am2.allocate("*", 3 * GB, 1, new ArrayList<ContainerId>());
+
+    // Get edit policy and do one update
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if 3 container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 3);
+
+    // Change reqeust from 3G to 2G, now we can preempt one less container. (3->2)
+    am2.allocate("*", 2 * GB, 1, new ArrayList<ContainerId>());
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+
+    // Call editSchedule once more to make sure still nothing happens
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersUserLimit()
+      throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     *
+     * Queue-c's user-limit-factor = 0.1, so single user cannot allocate >1 containers in queue-c
+     *
+     * 1) Two nodes in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
+     * more resource available.
+     *
+     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
+     *
+     * Now the cluster is fulfilled.
+     *
+     * 4) app2 asks for another 1G container, system will preempt one container
+     * from app1, and app2 will receive the preempted container
+     */
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf);
+    csConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".c", 0.1f);
+    MockRM rm1 = new MockRM(csConf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if no container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    // No preemption happens
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 0);
+    Assert.assertEquals(0, killableContainers.size());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (nothing preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  private Map<ContainerId, RMContainer> waitKillableContainersSize(
+      PreemptionManager pm, String queueName, String partition,
+      int expectedSize) throws InterruptedException {
+    Map<ContainerId, RMContainer> killableContainers =
+        pm.getKillableContainersMap(queueName, partition);
+
+    int wait = 0;
+    // Wait for at most 5 sec (it should be super fast actually)
+    while (expectedSize != killableContainers.size() && wait < 500) {
+      killableContainers = pm.getKillableContainersMap(queueName, partition);
+      Thread.sleep(10);
+      wait++;
+    }
+
+    Assert.assertEquals(expectedSize, killableContainers.size());
+    return killableContainers;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 5169337..1612201 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -99,6 +100,7 @@ public class TestChildQueueOrder {
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
   }
 
   private FiCaSchedulerApp getMockApplication(int appId, String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 69b0813..87a3d51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -150,6 +151,7 @@ public class TestLeafQueue {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
@@ -3092,6 +3094,7 @@ public class TestLeafQueue {
         Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(2 * GB, 2));
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     return csContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
index bbf6e43..1ee201d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -1676,4 +1677,100 @@ public class TestNodeLabelContainerAllocation {
     checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
         cs.getApplicationAttempt(am1.getApplicationAttemptId()));
   }
+
+  @Test
+  public void testParentQueueMaxCapsAreRespected() throws Exception {
+    /*
+     * Queue tree:
+     *          Root
+     *        /     \
+     *       A       B
+     *      / \
+     *     A1 A2
+     *
+     * A has 50% capacity and 50% max capacity (of label=x)
+     * A1/A2 has 50% capacity and 100% max capacity (of label=x)
+     * Cluster has one node (label=x) with resource = 24G.
+     * So we can at most use 12G resources under queueA.
+     */
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(this.conf);
+
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
+        "b"});
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(A, 10);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 50);
+    csConf.setMaximumCapacityByLabel(A, "x", 50);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 90);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 50);
+    csConf.setMaximumCapacityByLabel(B, "x", 50);
+
+    // Define 2nd-level queues
+    csConf.setQueues(A, new String[] { "a1",
+        "a2"});
+
+    final String A1 = A + ".a1";
+    csConf.setCapacity(A1, 50);
+    csConf.setAccessibleNodeLabels(A1, toSet("x"));
+    csConf.setCapacityByLabel(A1, "x", 50);
+    csConf.setMaximumCapacityByLabel(A1, "x", 100);
+    csConf.setUserLimitFactor(A1, 100.0f);
+
+    final String A2 = A + ".a2";
+    csConf.setCapacity(A2, 50);
+    csConf.setAccessibleNodeLabels(A2, toSet("x"));
+    csConf.setCapacityByLabel(A2, "x", 50);
+    csConf.setMaximumCapacityByLabel(A2, "x", 100);
+    csConf.setUserLimitFactor(A2, 100.0f);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of(
+        NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    MockNM nm1 =
+        new MockNM("h1:1234", 24 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    // Launch app1 in a1, resource usage is 1GB (am) + 4GB * 2 = 9GB
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1", "x");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    am1.allocate("*", 4 * GB, 2, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+
+    // Try to launch app2 in a2, asked 2GB, should success
+    RMApp app2 = rm.submitApp(2 * GB, "app", "user", null, "a2", "x");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
+
+    // am2 asks more resources, cannot success because current used = 9G (app1)
+    // + 2G (app2) = 11G, and queue's max capacity = 12G
+    am2.allocate("*", 2 * GB, 2, new ArrayList<ContainerId>(), "x");
+
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index f73baa4..23dc860 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@@ -92,6 +93,7 @@ public class TestParentQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
     thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index 2ef5e39..56facee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -126,6 +127,7 @@ public class TestReservations {
     when(csContext.getNonPartitionedQueueComparator()).thenReturn(
         CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
         conf);