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 ji...@apache.org on 2015/09/15 04:30:16 UTC

[1/5] hadoop git commit: YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan

Repository: hadoop
Updated Branches:
  refs/heads/YARN-1197 78ad04db9 -> 4a7b1d3d5


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 ef35093..4a815f5 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
@@ -144,11 +144,11 @@ public class TestParentQueue {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource, RMNodeLabelsManager.NO_LABEL);
+              allocatedResource, RMNodeLabelsManager.NO_LABEL, false);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource, null, null);
+              allocatedResource, null, null, false);
         }
         
         // Next call - nothing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 6a0b11b..884de2a 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
@@ -60,6 +60,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -482,6 +485,8 @@ public class TestReservations {
   @Test
   public void testAssignContainersNeedToUnreserve() throws Exception {
     // Test that we now unreserve and use a node that has space
+    Logger rootLogger = LogManager.getRootLogger();
+    rootLogger.setLevel(Level.DEBUG);
 
     CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
     setup(csConf);
@@ -593,7 +598,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // could allocate but told need to unreserve first
-    CSAssignment csAssignment = a.assignContainers(clusterResource, node_1,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -664,7 +669,7 @@ public class TestReservations {
 
     // no reserved containers - reserve then unreserve
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
-    app_0.unreserve(node_0, priorityMap);
+    app_0.unreserve(priorityMap, node_0, rmContainer_1);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
         cs.getResourceCalculator(), clusterResource);
     assertEquals(null, unreserveId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.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/FairSchedulerTestBase.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/FairSchedulerTestBase.java
index 1c9801d..3af3424 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/FairSchedulerTestBase.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/FairSchedulerTestBase.java
@@ -163,7 +163,7 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
     return id;
   }
   
@@ -189,7 +189,7 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
     return id;
   }
 
@@ -211,7 +211,7 @@ public class FairSchedulerTestBase {
       ResourceRequest request, ApplicationAttemptId attId) {
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ask.add(request);
-    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(), null, null, null, null);
   }
 
   protected void createApplicationWithAMResource(ApplicationAttemptId attId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.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/TestContinuousScheduling.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/TestContinuousScheduling.java
index 53382de..65c80a6 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/TestContinuousScheduling.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/TestContinuousScheduling.java
@@ -99,7 +99,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     List<ResourceRequest> ask = new ArrayList<>();
     ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
     scheduler.allocate(
-        appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
+        appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
 
     // Advance time and let continuous scheduling kick in

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
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/TestFairScheduler.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/TestFairScheduler.java
index a02cf18..73e09de 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/TestFairScheduler.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/TestFairScheduler.java
@@ -1375,7 +1375,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     ResourceRequest request1 =
         createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
     ask1.add(request1);
-    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null, null, null);
 
     // Second ask, queue2 requests 1 large + (2 * minReqSize)
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
@@ -1385,14 +1385,14 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         false);
     ask2.add(request2);
     ask2.add(request3);
-    scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(), null, null, null, null);
 
     // Third ask, queue2 requests 1 large
     List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
     ResourceRequest request4 =
         createResourceRequest(2 * minReqSize, ResourceRequest.ANY, 1, 1, true);
     ask3.add(request4);
-    scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(), null, null, null, null);
 
     scheduler.update();
 
@@ -2714,7 +2714,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     
     // Complete container
     scheduler.allocate(attId, new ArrayList<ResourceRequest>(),
-        Arrays.asList(containerId), null, null);
+        Arrays.asList(containerId), null, null, null, null);
     assertEquals(1024, scheduler.getRootQueueMetrics().getAvailableMB());
     assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
     
@@ -2806,7 +2806,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
 
     scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
-        null);
+        null, null, null);
     
     // node 1 checks in
     scheduler.update();
@@ -3202,7 +3202,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         createResourceRequest(1024, node1.getHostName(), 1, 0, true),
         createResourceRequest(1024, "rack1", 1, 0, true),
         createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
-    scheduler.allocate(attId1, update, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(attId1, update, new ArrayList<ContainerId>(), null, null, null, null);
     
     // then node2 should get the container
     scheduler.handle(node2UpdateEvent);
@@ -3249,7 +3249,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     anyRequest = createResourceRequest(1024, ResourceRequest.ANY,
         1, 1, false);
     scheduler.allocate(attId, Arrays.asList(rackRequest, anyRequest),
-        new ArrayList<ContainerId>(), null, null);
+        new ArrayList<ContainerId>(), null, null, null, null);
 
     scheduler.handle(nodeUpdateEvent);
     assertEquals(0, app.getReservedContainers().size());
@@ -4251,7 +4251,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     ResourceRequest request =
             createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
     ask.add(request);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
 
     // waiting for continuous_scheduler_sleep_time
     // at least one pass
@@ -4271,7 +4271,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     ask.clear();
     ask.add(request);
     scheduler.stop();
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
     scheduler.continuousSchedulingAttempt();
     Assert.assertEquals(2048, app.getCurrentConsumption().getMemory());
     Assert.assertEquals(2, app.getCurrentConsumption().getVirtualCores());
@@ -4371,7 +4371,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
     ask1.add(request1);
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
-        null);
+        null, null, null);
 
     String hostName = "127.0.0.1";
     RMNode node1 = MockNodes.newNodeInfo(1,
@@ -4503,7 +4503,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
     List<Container> containers = scheduler.allocate(appAttemptId,
         Collections.<ResourceRequest> emptyList(),
-        Collections.<ContainerId> emptyList(), null, null).getContainers();
+        Collections.<ContainerId> emptyList(), null, null, null, null).getContainers();
 
     // Now with updated ResourceRequest, a container is allocated for AM.
     Assert.assertTrue(containers.size() == 1);
@@ -4532,11 +4532,11 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Verify the blacklist can be updated independent of requesting containers
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null);
+        Collections.singletonList(host), null, null, null);
     assertTrue(app.isBlacklisted(host));
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host));
+        Collections.singletonList(host), null, null);
     assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
 
     List<ResourceRequest> update = Arrays.asList(
@@ -4545,7 +4545,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Verify a container does not actually get placed on the blacklisted host
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null);
+        Collections.singletonList(host), null, null, null);
     assertTrue(app.isBlacklisted(host));
     scheduler.update();
     scheduler.handle(updateEvent);
@@ -4555,7 +4555,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Verify a container gets placed on the empty blacklist
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host));
+        Collections.singletonList(host), null, null);
     assertFalse(app.isBlacklisted(host));
     createSchedulingRequest(GB, "root.default", "user", 1);
     scheduler.update();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
----------------------------------------------------------------------
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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 83d2ccf..6a2e7a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -263,7 +263,7 @@ public class TestFifoScheduler {
     ask.add(nodeLocal);
     ask.add(rackLocal);
     ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
 
     NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
 
@@ -365,7 +365,7 @@ public class TestFifoScheduler {
     ask.add(nodeLocal);
     ask.add(rackLocal);
     ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
 
     // Before the node update event, there are one local request
     Assert.assertEquals(1, nodeLocal.getNumContainers());
@@ -941,7 +941,7 @@ public class TestFifoScheduler {
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
     fs.allocate(appAttemptId1, ask1, emptyId,
-        Collections.singletonList(host_1_0), null);
+        Collections.singletonList(host_1_0), null, null, null);
 
     // Trigger container assignment
     fs.handle(new NodeUpdateSchedulerEvent(n3));
@@ -949,14 +949,14 @@ public class TestFifoScheduler {
     // Get the allocation for the application and verify no allocation on
     // blacklist node
     Allocation allocation1 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
 
     Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
 
     // verify host_1_1 can get allocated as not in blacklist
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     Allocation allocation2 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
     List<Container> containerList = allocation2.getContainers();
     for (Container container : containerList) {
@@ -971,29 +971,29 @@ public class TestFifoScheduler {
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
     fs.allocate(appAttemptId1, ask2, emptyId,
-        Collections.singletonList("rack0"), null);
+        Collections.singletonList("rack0"), null, null, null);
 
     // verify n1 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n1));
     Allocation allocation3 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
 
     // verify n2 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n2));
     Allocation allocation4 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
 
     // verify n3 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n3));
     Allocation allocation5 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
 
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     Allocation allocation6 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
 
     containerList = allocation6.getContainers();
@@ -1052,25 +1052,25 @@ public class TestFifoScheduler {
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
-    fs.allocate(appAttemptId1, ask1, emptyId, null, null);
+    fs.allocate(appAttemptId1, ask1, emptyId, null, null, null, null);
 
     // Ask for a 2 GB container for app 2
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
-    fs.allocate(appAttemptId2, ask2, emptyId, null, null);
+    fs.allocate(appAttemptId2, ask2, emptyId, null, null, null, null);
 
     // Trigger container assignment
     fs.handle(new NodeUpdateSchedulerEvent(n1));
 
     // Get the allocation for the applications and verify headroom
     Allocation allocation1 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
         .getResourceLimit().getMemory());
 
     Allocation allocation2 =
-        fs.allocate(appAttemptId2, emptyAsk, emptyId, null, null);
+        fs.allocate(appAttemptId2, emptyAsk, emptyId, null, null, null, null);
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
         .getResourceLimit().getMemory());
 


[4/5] hadoop git commit: YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 0b6b8ef..ad28493 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -19,7 +19,16 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -37,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -51,6 +61,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -58,13 +69,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMoveEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerRecoverEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -87,7 +100,7 @@ public abstract class AbstractYarnScheduler
   protected Resource clusterResource = Resource.newInstance(0, 0);
 
   protected Resource minimumAllocation;
-  private Resource maximumAllocation;
+  protected Resource maximumAllocation;
   private Resource configuredMaximumAllocation;
   private int maxNodeMemory = -1;
   private int maxNodeVCores = -1;
@@ -231,6 +244,55 @@ public abstract class AbstractYarnScheduler
 
     application.containerLaunchedOnNode(containerId, node.getNodeID());
   }
+  
+  protected synchronized void containerIncreasedOnNode(ContainerId containerId,
+      SchedulerNode node, Container increasedContainerReportedByNM) {
+    // Get the application for the finished container
+    SchedulerApplicationAttempt application =
+        getCurrentAttemptForContainer(containerId);
+    if (application == null) {
+      LOG.info("Unknown application "
+          + containerId.getApplicationAttemptId().getApplicationId()
+          + " increased container " + containerId + " on node: " + node);
+      this.rmContext.getDispatcher().getEventHandler()
+          .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
+      return;
+    }
+
+    RMContainer rmContainer = getRMContainer(containerId);
+    Resource rmContainerResource = rmContainer.getAllocatedResource();
+    Resource nmContainerResource = increasedContainerReportedByNM.getResource();
+    
+    
+    if (Resources.equals(nmContainerResource, rmContainerResource)){
+      // NM reported expected container size, tell RMContainer. Which will stop
+      // container expire monitor
+      rmContainer.handle(new RMContainerEvent(containerId,
+          RMContainerEventType.NM_DONE_CHANGE_RESOURCE));
+    } else if (Resources.fitsIn(getResourceCalculator(), clusterResource,
+        nmContainerResource, rmContainerResource)) {
+      // when rmContainerResource >= nmContainerResource, we won't do anything,
+      // it is possible a container increased is issued by RM, but AM hasn't
+      // told NM.
+    } else if (Resources.fitsIn(getResourceCalculator(), clusterResource,
+        rmContainerResource, nmContainerResource)) {
+      // When rmContainerResource <= nmContainerResource, it could happen when a
+      // container decreased by RM before it is increased in NM.
+      
+      // Tell NM to decrease the container
+      this.rmContext.getDispatcher().getEventHandler()
+          .handle(new RMNodeDecreaseContainerEvent(node.getNodeID(),
+              Arrays.asList(rmContainer.getContainer())));
+    } else {
+      // Something wrong happened, kill the container
+      LOG.warn("Something wrong happened, container size reported by NM"
+          + " is not expected, ContainerID=" + containerId
+          + " rm-size-resource:" + rmContainerResource + " nm-size-reosurce:"
+          + nmContainerResource);
+      this.rmContext.getDispatcher().getEventHandler()
+          .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
+    }
+  }
 
   public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
     SchedulerApplication<T> app =
@@ -511,6 +573,36 @@ public abstract class AbstractYarnScheduler
           SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED);
     }
   }
+  
+  protected void decreaseContainers(
+      List<SchedContainerChangeRequest> decreaseRequests,
+      SchedulerApplicationAttempt attempt) {
+    for (SchedContainerChangeRequest request : decreaseRequests) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Processing decrease request:" + request);
+      }
+      
+      boolean hasIncreaseRequest =
+          attempt.removeIncreaseRequest(request.getNodeId(),
+              request.getPriority(), request.getContainerId());
+      
+      if (hasIncreaseRequest) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("While processing decrease request, found a increase request "
+              + "for the same container "
+              + request.getContainerId()
+              + ", removed the increase request");
+        }
+      }
+      
+      // handle decrease request
+      decreaseContainer(request, attempt);
+    }
+  }
+
+  protected abstract void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest,
+      SchedulerApplicationAttempt attempt);
 
   public SchedulerNode getSchedulerNode(NodeId nodeId) {
     return nodes.get(nodeId);
@@ -733,4 +825,56 @@ public abstract class AbstractYarnScheduler
     LOG.info("Updated the cluste max priority to maxClusterLevelAppPriority = "
         + maxClusterLevelAppPriority);
   }
+  
+  /**
+   * Normalize container increase/decrease request, and return
+   * SchedulerContainerResourceChangeRequest according to given
+   * ContainerResourceChangeRequest.
+   * 
+   * <pre>
+   * - Returns non-null value means validation succeeded
+   * - Throw exception when any other error happens
+   * </pre>
+   */
+  private SchedContainerChangeRequest
+      checkAndNormalizeContainerChangeRequest(
+          ContainerResourceChangeRequest request, boolean increase)
+          throws YarnException {
+    // We have done a check in ApplicationMasterService, but RMContainer status
+    // / Node resource could change since AMS won't acquire lock of scheduler.
+    RMServerUtils.checkAndNormalizeContainerChangeRequest(rmContext, request,
+        increase);
+    ContainerId containerId = request.getContainerId();
+    RMContainer rmContainer = getRMContainer(containerId);
+    SchedulerNode schedulerNode =
+        getSchedulerNode(rmContainer.getAllocatedNode());
+    
+    return new SchedContainerChangeRequest(schedulerNode, rmContainer,
+        request.getCapability());
+  }
+
+  protected List<SchedContainerChangeRequest>
+      checkAndNormalizeContainerChangeRequests(
+          List<ContainerResourceChangeRequest> changeRequests,
+          boolean increase) {
+    if (null == changeRequests || changeRequests.isEmpty()) {
+      return Collections.EMPTY_LIST;
+    }
+    
+    List<SchedContainerChangeRequest> schedulerChangeRequests =
+        new ArrayList<SchedContainerChangeRequest>();
+    for (ContainerResourceChangeRequest r : changeRequests) {
+      SchedContainerChangeRequest sr = null;
+      try {
+        sr = checkAndNormalizeContainerChangeRequest(r, increase);
+      } catch (YarnException e) {
+        LOG.warn("Error happens when checking increase request, Ignoring.."
+            + " exception=", e);
+        continue;
+      }
+      schedulerChangeRequests.add(sr);
+    }
+
+    return schedulerChangeRequests;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.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/Allocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java
index 3f2d8af..af6caad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java
@@ -34,6 +34,9 @@ public class Allocation {
   final Set<ContainerId> fungibleContainers;
   final List<ResourceRequest> fungibleResources;
   final List<NMToken> nmTokens;
+  final List<Container> increasedContainers;
+  final List<Container> decreasedContainers;
+
 
   public Allocation(List<Container> containers, Resource resourceLimit,
       Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
@@ -45,12 +48,22 @@ public class Allocation {
   public Allocation(List<Container> containers, Resource resourceLimit,
       Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
       List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
+    this(containers,  resourceLimit,strictContainers,  fungibleContainers,
+      fungibleResources, nmTokens, null, null);
+  }
+  
+  public Allocation(List<Container> containers, Resource resourceLimit,
+      Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
+      List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
+      List<Container> increasedContainers, List<Container> decreasedContainer) {
     this.containers = containers;
     this.resourceLimit = resourceLimit;
     this.strictContainers = strictContainers;
     this.fungibleContainers = fungibleContainers;
     this.fungibleResources = fungibleResources;
     this.nmTokens = nmTokens;
+    this.increasedContainers = increasedContainers;
+    this.decreasedContainers = decreasedContainer;
   }
 
   public List<Container> getContainers() {
@@ -76,5 +89,12 @@ public class Allocation {
   public List<NMToken> getNMTokens() {
     return nmTokens;
   }
-
+  
+  public List<Container> getIncreasedContainers() {
+    return increasedContainers;
+  }
+  
+  public List<Container> getDecreasedContainers() {
+    return decreasedContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index e318d47..7623da0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -20,10 +20,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
@@ -35,6 +37,8 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.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.api.records.ResourceRequest;
@@ -63,8 +67,11 @@ public class AppSchedulingInfo {
 
   final Set<Priority> priorities = new TreeSet<Priority>(
       new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
-  final Map<Priority, Map<String, ResourceRequest>> requests =
-    new ConcurrentHashMap<Priority, Map<String, ResourceRequest>>();
+  final Map<Priority, Map<String, ResourceRequest>> resourceRequestMap =
+      new ConcurrentHashMap<Priority, Map<String, ResourceRequest>>();
+  final Map<NodeId, Map<Priority, Map<ContainerId, 
+      SchedContainerChangeRequest>>> increaseRequestMap =
+      new ConcurrentHashMap<>();
   private Set<String> userBlacklist = new HashSet<>();
   private Set<String> amBlacklist = new HashSet<>();
 
@@ -114,13 +121,177 @@ public class AppSchedulingInfo {
    */
   private synchronized void clearRequests() {
     priorities.clear();
-    requests.clear();
+    resourceRequestMap.clear();
     LOG.info("Application " + applicationId + " requests cleared");
   }
 
   public long getNewContainerId() {
     return this.containerIdCounter.incrementAndGet();
   }
+  
+  public boolean hasIncreaseRequest(NodeId nodeId) {
+    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
+        increaseRequestMap.get(nodeId);
+    if (null == requestsOnNode) {
+      return false;
+    }
+    return requestsOnNode.size() > 0;
+  }
+  
+  public Map<ContainerId, SchedContainerChangeRequest>
+      getIncreaseRequests(NodeId nodeId, Priority priority) {
+    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
+        increaseRequestMap.get(nodeId);
+    if (null == requestsOnNode) {
+      return null;
+    }
+
+    return requestsOnNode.get(priority);
+  }
+
+  public synchronized boolean updateIncreaseRequests(
+      List<SchedContainerChangeRequest> increaseRequests) {
+    boolean resourceUpdated = false;
+
+    for (SchedContainerChangeRequest r : increaseRequests) {
+      NodeId nodeId = r.getRMContainer().getAllocatedNode();
+
+      Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
+          increaseRequestMap.get(nodeId);
+      if (null == requestsOnNode) {
+        requestsOnNode = new TreeMap<>();
+        increaseRequestMap.put(nodeId, requestsOnNode);
+      }
+
+      SchedContainerChangeRequest prevChangeRequest =
+          getIncreaseRequest(nodeId, r.getPriority(), r.getContainerId());
+      if (null != prevChangeRequest) {
+        if (Resources.equals(prevChangeRequest.getTargetCapacity(),
+            r.getTargetCapacity())) {
+          // New target capacity is as same as what we have, just ignore the new
+          // one
+          continue;
+        }
+
+        // remove the old one
+        removeIncreaseRequest(nodeId, prevChangeRequest.getPriority(),
+            prevChangeRequest.getContainerId());
+      }
+
+      if (Resources.equals(r.getTargetCapacity(), r.getRMContainer().getAllocatedResource())) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Trying to increase/decrease container, "
+              + "target capacity = previous capacity = " + prevChangeRequest
+              + " for container=" + r.getContainerId()
+              + ". Will ignore this increase request");
+        }
+        continue;
+      }
+
+      // add the new one
+      resourceUpdated = true;
+      insertIncreaseRequest(r);
+    }
+    return resourceUpdated;
+  }
+
+  // insert increase request and add missing hierarchy if missing
+  private void insertIncreaseRequest(SchedContainerChangeRequest request) {
+    NodeId nodeId = request.getNodeId();
+    Priority priority = request.getPriority();
+    ContainerId containerId = request.getContainerId();
+    
+    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
+        increaseRequestMap.get(nodeId);
+    if (null == requestsOnNode) {
+      requestsOnNode =
+          new HashMap<Priority, Map<ContainerId, SchedContainerChangeRequest>>();
+      increaseRequestMap.put(nodeId, requestsOnNode);
+    }
+
+    Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
+        requestsOnNode.get(priority);
+    if (null == requestsOnNodeWithPriority) {
+      requestsOnNodeWithPriority =
+          new TreeMap<ContainerId, SchedContainerChangeRequest>();
+      requestsOnNode.put(priority, requestsOnNodeWithPriority);
+    }
+
+    requestsOnNodeWithPriority.put(containerId, request);
+
+    // update resources
+    String partition = request.getRMContainer().getNodeLabelExpression();
+    Resource delta = request.getDeltaCapacity();
+    appResourceUsage.incPending(partition, delta);
+    queue.incPendingResource(partition, delta);
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added increase request:" + request.getContainerId()
+          + " delta=" + request.getDeltaCapacity());
+    }
+    
+    // update priorities
+    priorities.add(priority);
+  }
+  
+  public synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priority,
+      ContainerId containerId) {
+    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
+        increaseRequestMap.get(nodeId);
+    if (null == requestsOnNode) {
+      return false;
+    }
+
+    Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
+        requestsOnNode.get(priority);
+    if (null == requestsOnNodeWithPriority) {
+      return false;
+    }
+
+    SchedContainerChangeRequest request =
+        requestsOnNodeWithPriority.remove(containerId);
+    
+    // remove hierarchies if it becomes empty
+    if (requestsOnNodeWithPriority.isEmpty()) {
+      requestsOnNode.remove(priority);
+    }
+    if (requestsOnNode.isEmpty()) {
+      increaseRequestMap.remove(nodeId);
+    }
+    
+    if (request == null) {
+      return false;
+    }
+
+    // update queue's pending resource if request exists
+    String partition = request.getRMContainer().getNodeLabelExpression();
+    Resource delta = request.getDeltaCapacity();
+    appResourceUsage.decPending(partition, delta);
+    queue.decPendingResource(partition, delta);
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("remove increase request:" + request);
+    }
+    
+    return true;
+  }
+  
+  public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
+      Priority priority, ContainerId containerId) {
+    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
+        increaseRequestMap.get(nodeId);
+    if (null == requestsOnNode) {
+      return null;
+    }
+
+    Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
+        requestsOnNode.get(priority);
+    if (null == requestsOnNodeWithPriority) {
+      return null;
+    }
+
+    return requestsOnNodeWithPriority.get(containerId);
+  }
 
   /**
    * The ApplicationMaster is updating resource requirements for the
@@ -163,11 +334,11 @@ public class AppSchedulingInfo {
         }
       }
 
-      Map<String, ResourceRequest> asks = this.requests.get(priority);
+      Map<String, ResourceRequest> asks = this.resourceRequestMap.get(priority);
 
       if (asks == null) {
         asks = new ConcurrentHashMap<String, ResourceRequest>();
-        this.requests.put(priority, asks);
+        this.resourceRequestMap.put(priority, asks);
         this.priorities.add(priority);
       }
       lastRequest = asks.get(resourceName);
@@ -260,12 +431,12 @@ public class AppSchedulingInfo {
 
   synchronized public Map<String, ResourceRequest> getResourceRequests(
       Priority priority) {
-    return requests.get(priority);
+    return resourceRequestMap.get(priority);
   }
 
   public List<ResourceRequest> getAllResourceRequests() {
     List<ResourceRequest> ret = new ArrayList<ResourceRequest>();
-    for (Map<String, ResourceRequest> r : requests.values()) {
+    for (Map<String, ResourceRequest> r : resourceRequestMap.values()) {
       ret.addAll(r.values());
     }
     return ret;
@@ -273,7 +444,7 @@ public class AppSchedulingInfo {
 
   synchronized public ResourceRequest getResourceRequest(Priority priority,
       String resourceName) {
-    Map<String, ResourceRequest> nodeRequests = requests.get(priority);
+    Map<String, ResourceRequest> nodeRequests = resourceRequestMap.get(priority);
     return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
   }
 
@@ -301,6 +472,50 @@ public class AppSchedulingInfo {
     }
   }
   
+  public synchronized void increaseContainer(
+      SchedContainerChangeRequest increaseRequest) {
+    NodeId nodeId = increaseRequest.getNodeId();
+    Priority priority = increaseRequest.getPriority();
+    ContainerId containerId = increaseRequest.getContainerId();
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("allocated increase request : applicationId=" + applicationId
+          + " container=" + containerId + " host="
+          + increaseRequest.getNodeId() + " user=" + user + " resource="
+          + increaseRequest.getDeltaCapacity());
+    }
+    
+    // Set queue metrics
+    queue.getMetrics().allocateResources(user, 0,
+        increaseRequest.getDeltaCapacity(), true);
+    
+    // remove the increase request from pending increase request map
+    removeIncreaseRequest(nodeId, priority, containerId);
+    
+    // update usage
+    appResourceUsage.incUsed(increaseRequest.getNodePartition(),
+        increaseRequest.getDeltaCapacity());
+  }
+  
+  public synchronized void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest) {
+    // Delta is negative when it's a decrease request
+    Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity());
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Decrease container : applicationId=" + applicationId
+          + " container=" + decreaseRequest.getContainerId() + " host="
+          + decreaseRequest.getNodeId() + " user=" + user + " resource="
+          + absDelta);
+    }
+    
+    // Set queue metrics
+    queue.getMetrics().releaseResources(user, 0, absDelta);
+
+    // update usage
+    appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta);
+  }
+  
   /**
    * Resources have been allocated to this application by the resource
    * scheduler. Track them.
@@ -359,11 +574,11 @@ public class AppSchedulingInfo {
     // Update future requirements
     decResourceRequest(node.getNodeName(), priority, nodeLocalRequest);
 
-    ResourceRequest rackLocalRequest = requests.get(priority).get(
+    ResourceRequest rackLocalRequest = resourceRequestMap.get(priority).get(
         node.getRackName());
     decResourceRequest(node.getRackName(), priority, rackLocalRequest);
 
-    ResourceRequest offRackRequest = requests.get(priority).get(
+    ResourceRequest offRackRequest = resourceRequestMap.get(priority).get(
         ResourceRequest.ANY);
     decrementOutstanding(offRackRequest);
 
@@ -377,7 +592,7 @@ public class AppSchedulingInfo {
       ResourceRequest request) {
     request.setNumContainers(request.getNumContainers() - 1);
     if (request.getNumContainers() == 0) {
-      requests.get(priority).remove(resourceName);
+      resourceRequestMap.get(priority).remove(resourceName);
     }
   }
 
@@ -394,7 +609,7 @@ public class AppSchedulingInfo {
     // Update future requirements
     decResourceRequest(node.getRackName(), priority, rackLocalRequest);
     
-    ResourceRequest offRackRequest = requests.get(priority).get(
+    ResourceRequest offRackRequest = resourceRequestMap.get(priority).get(
         ResourceRequest.ANY);
     decrementOutstanding(offRackRequest);
 
@@ -449,6 +664,12 @@ public class AppSchedulingInfo {
         }
       }
     }
+    
+    // also we need to check increase request
+    if (!deactivate) {
+      deactivate = increaseRequestMap.isEmpty();
+    }
+
     if (deactivate) {
       activeUsersManager.deactivateApplication(user, applicationId);
     }
@@ -457,7 +678,7 @@ public class AppSchedulingInfo {
   synchronized public void move(Queue newQueue) {
     QueueMetrics oldMetrics = queue.getMetrics();
     QueueMetrics newMetrics = newQueue.getMetrics();
-    for (Map<String, ResourceRequest> asks : requests.values()) {
+    for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
       ResourceRequest request = asks.get(ResourceRequest.ANY);
       if (request != null) {
         oldMetrics.decrPendingResources(user, request.getNumContainers(),
@@ -484,7 +705,7 @@ public class AppSchedulingInfo {
   synchronized public void stop(RMAppAttemptState rmAppAttemptFinalState) {
     // clear pending resources metrics for the application
     QueueMetrics metrics = queue.getMetrics();
-    for (Map<String, ResourceRequest> asks : requests.values()) {
+    for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
       ResourceRequest request = asks.get(ResourceRequest.ANY);
       if (request != null) {
         metrics.decrPendingResources(user, request.getNumContainers(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.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/QueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
index 09fd73e..d94b621 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
@@ -373,17 +373,20 @@ public class QueueMetrics implements MetricsSource {
   }
 
   private void _decrPendingResources(int containers, Resource res) {
+    // if #container = 0, means change container resource
     pendingContainers.decr(containers);
-    pendingMB.decr(res.getMemory() * containers);
-    pendingVCores.decr(res.getVirtualCores() * containers);
+    pendingMB.decr(res.getMemory() * Math.max(containers, 1));
+    pendingVCores.decr(res.getVirtualCores() * Math.max(containers, 1));
   }
 
   public void allocateResources(String user, int containers, Resource res,
       boolean decrPending) {
+    // if #containers = 0, means change container resource
     allocatedContainers.incr(containers);
     aggregateContainersAllocated.incr(containers);
-    allocatedMB.incr(res.getMemory() * containers);
-    allocatedVCores.incr(res.getVirtualCores() * containers);
+
+    allocatedMB.incr(res.getMemory() * Math.max(containers, 1));
+    allocatedVCores.incr(res.getVirtualCores() * Math.max(containers, 1));
     if (decrPending) {
       _decrPendingResources(containers, res);
     }
@@ -397,10 +400,11 @@ public class QueueMetrics implements MetricsSource {
   }
 
   public void releaseResources(String user, int containers, Resource res) {
+    // if #container = 0, means change container resource.
     allocatedContainers.decr(containers);
     aggregateContainersReleased.incr(containers);
-    allocatedMB.decr(res.getMemory() * containers);
-    allocatedVCores.decr(res.getVirtualCores() * containers);
+    allocatedMB.decr(res.getMemory() * Math.max(containers, 1));
+    allocatedVCores.decr(res.getVirtualCores() * Math.max(containers, 1));
     QueueMetrics userMetrics = getUserMetrics(user);
     if (userMetrics != null) {
       userMetrics.releaseResources(user, containers, res);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.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/SchedContainerChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
new file mode 100644
index 0000000..ea109fd
--- /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/SchedContainerChangeRequest.java
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+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.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * This is ContainerResourceChangeRequest in scheduler side, it contains some
+ * pointers to runtime objects like RMContainer, SchedulerNode, etc. This will
+ * be easier for scheduler making decision.
+ */
+public class SchedContainerChangeRequest implements
+    Comparable<SchedContainerChangeRequest> {
+  RMContainer rmContainer;
+  Resource targetCapacity;
+  SchedulerNode schedulerNode;
+  Resource deltaCapacity;
+
+  public SchedContainerChangeRequest(SchedulerNode schedulerNode,
+      RMContainer rmContainer, Resource targetCapacity) {
+    this.rmContainer = rmContainer;
+    this.targetCapacity = targetCapacity;
+    this.schedulerNode = schedulerNode;
+    deltaCapacity = Resources.subtract(targetCapacity,
+        rmContainer.getAllocatedResource());
+  }
+  
+  public NodeId getNodeId() {
+    return this.rmContainer.getAllocatedNode();
+  }
+
+  public RMContainer getRMContainer() {
+    return this.rmContainer;
+  }
+
+  public Resource getTargetCapacity() {
+    return this.targetCapacity;
+  }
+
+  /**
+   * Delta capacity = before - target, so if it is a decrease request, delta
+   * capacity will be negative
+   */
+  public Resource getDeltaCapacity() {
+    return deltaCapacity;
+  }
+  
+  public Priority getPriority() {
+    return rmContainer.getContainer().getPriority();
+  }
+  
+  public ContainerId getContainerId() {
+    return rmContainer.getContainerId();
+  }
+  
+  public String getNodePartition() {
+    return schedulerNode.getPartition();
+  }
+  
+  public SchedulerNode getSchedulerNode() {
+    return schedulerNode;
+  }
+  
+  @Override
+  public int hashCode() {
+    return (getContainerId().hashCode() << 16) + targetCapacity.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof SchedContainerChangeRequest)) {
+      return false;
+    }
+    return compareTo((SchedContainerChangeRequest)other) == 0;
+  }
+
+  @Override
+  public int compareTo(SchedContainerChangeRequest other) {
+    if (other == null) {
+      return -1;
+    }
+    
+    int rc = getPriority().compareTo(other.getPriority());
+    if (0 != rc) {
+      return rc;
+    }
+    
+    return getContainerId().compareTo(other.getContainerId());
+  }
+  
+  @Override
+  public String toString() {
+    return "<container=" + getContainerId() + ", targetCapacity="
+        + targetCapacity + ", delta=" + deltaCapacity + ", node="
+        + getNodeId().toString() + ">";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.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/SchedulerApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java
index 519de98..96288f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java
@@ -28,7 +28,7 @@ public class SchedulerApplication<T extends SchedulerApplicationAttempt> {
 
   private Queue queue;
   private final String user;
-  private T currentAttempt;
+  private volatile T currentAttempt;
   private volatile Priority priority;
 
   public SchedulerApplication(Queue queue, String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index b361d15..f064e97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -19,11 +19,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -51,16 +53,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppR
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerChangeResourceEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpdatesAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -104,8 +109,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   private AtomicLong firstAllocationRequestSentTime = new AtomicLong(0);
   private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
 
-  protected List<RMContainer> newlyAllocatedContainers = 
-      new ArrayList<RMContainer>();
+  protected List<RMContainer> newlyAllocatedContainers = new ArrayList<>();
+  protected Map<ContainerId, RMContainer> newlyDecreasedContainers = new HashMap<>();
+  protected Map<ContainerId, RMContainer> newlyIncreasedContainers = new HashMap<>();
+  protected Set<NMToken> updatedNMTokens = new HashSet<>();
 
   // This pendingRelease is used in work-preserving recovery scenario to keep
   // track of the AM's outstanding release requests. RM on recovery could
@@ -219,7 +226,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return appSchedulingInfo.getPriorities();
   }
   
-  public synchronized ResourceRequest getResourceRequest(Priority priority, String resourceName) {
+  public synchronized ResourceRequest getResourceRequest(Priority priority,
+      String resourceName) {
     return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
   }
 
@@ -324,24 +332,28 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return reservedContainers;
   }
   
-  public synchronized RMContainer reserve(SchedulerNode node, Priority priority,
-      RMContainer rmContainer, Container container) {
-    // Create RMContainer if necessary
-    if (rmContainer == null) {
-      rmContainer = 
-          new RMContainerImpl(container, getApplicationAttemptId(), 
-              node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
+  public synchronized boolean reserveIncreasedContainer(SchedulerNode node,
+      Priority priority, RMContainer rmContainer, Resource reservedResource) {
+    if (commonReserve(node, priority, rmContainer, reservedResource)) {
       attemptResourceUsage.incReserved(node.getPartition(),
-          container.getResource());
-      
-      // Reset the re-reservation count
-      resetReReservations(priority);
-    } else {
-      // Note down the re-reservation
-      addReReservation(priority);
+          reservedResource);
+      // succeeded
+      return true;
+    }
+    
+    return false;
+  }
+  
+  private synchronized boolean commonReserve(SchedulerNode node,
+      Priority priority, RMContainer rmContainer, Resource reservedResource) {
+    try {
+      rmContainer.handle(new RMContainerReservedEvent(rmContainer
+          .getContainerId(), reservedResource, node.getNodeID(), priority));
+    } catch (InvalidStateTransitionException e) {
+      // We reach here could be caused by container already finished, return
+      // false indicate it fails
+      return false;
     }
-    rmContainer.handle(new RMContainerReservedEvent(container.getId(), 
-        container.getResource(), node.getNodeID(), priority));
     
     Map<NodeId, RMContainer> reservedContainers = 
         this.reservedContainers.get(priority);
@@ -356,8 +368,30 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
           + " reserved container " + rmContainer + " on node " + node
           + ". This attempt currently has " + reservedContainers.size()
           + " reserved containers at priority " + priority
-          + "; currentReservation " + container.getResource());
+          + "; currentReservation " + reservedResource);
     }
+    
+    return true;
+  }
+  
+  public synchronized RMContainer reserve(SchedulerNode node,
+      Priority priority, RMContainer rmContainer, Container container) {
+    // Create RMContainer if necessary
+    if (rmContainer == null) {
+      rmContainer =
+          new RMContainerImpl(container, getApplicationAttemptId(),
+              node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
+      attemptResourceUsage.incReserved(node.getPartition(),
+          container.getResource());
+
+      // Reset the re-reservation count
+      resetReReservations(priority);
+    } else {
+      // Note down the re-reservation
+      addReReservation(priority);
+    }
+    
+    commonReserve(node, priority, rmContainer, container.getResource());
 
     return rmContainer;
   }
@@ -437,69 +471,100 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   public Resource getCurrentConsumption() {
     return attemptResourceUsage.getUsed();
   }
-
-  public static class ContainersAndNMTokensAllocation {
-    List<Container> containerList;
-    List<NMToken> nmTokenList;
-
-    public ContainersAndNMTokensAllocation(List<Container> containerList,
-        List<NMToken> nmTokenList) {
-      this.containerList = containerList;
-      this.nmTokenList = nmTokenList;
+  
+  private Container updateContainerAndNMToken(RMContainer rmContainer,
+      boolean newContainer, boolean increasedContainer) {
+    Container container = rmContainer.getContainer();
+    ContainerType containerType = ContainerType.TASK;
+    // The working knowledge is that masterContainer for AM is null as it
+    // itself is the master container.
+    RMAppAttempt appAttempt = rmContext.getRMApps()
+        .get(container.getId().getApplicationAttemptId().getApplicationId())
+        .getCurrentAppAttempt();
+    if (isWaitingForAMContainer(getApplicationId())) {
+      containerType = ContainerType.APPLICATION_MASTER;
     }
-
-    public List<Container> getContainerList() {
-      return containerList;
+    try {
+      // create container token and NMToken altogether.
+      container.setContainerToken(rmContext.getContainerTokenSecretManager()
+          .createContainerToken(container.getId(), container.getNodeId(),
+              getUser(), container.getResource(), container.getPriority(),
+              rmContainer.getCreationTime(), this.logAggregationContext,
+              rmContainer.getNodeLabelExpression(), containerType));
+      NMToken nmToken =
+          rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
+              getApplicationAttemptId(), container);
+      if (nmToken != null) {
+        updatedNMTokens.add(nmToken);
+      }
+    } catch (IllegalArgumentException e) {
+      // DNS might be down, skip returning this container.
+      LOG.error("Error trying to assign container token and NM token to"
+          + " an updated container " + container.getId(), e);
+      return null;
     }
-
-    public List<NMToken> getNMTokenList() {
-      return nmTokenList;
+    
+    if (newContainer) {
+      rmContainer.handle(new RMContainerEvent(
+          rmContainer.getContainerId(), RMContainerEventType.ACQUIRED));
+    } else {
+      rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
+          rmContainer.getContainerId(), increasedContainer));
     }
+    return container;
   }
 
-  // Create container token and NMToken altogether, if either of them fails for
+  // Create container token and update NMToken altogether, if either of them fails for
   // some reason like DNS unavailable, do not return this container and keep it
   // in the newlyAllocatedContainers waiting to be refetched.
-  public synchronized ContainersAndNMTokensAllocation
-      pullNewlyAllocatedContainersAndNMTokens() {
+  public synchronized List<Container> pullNewlyAllocatedContainers() {
     List<Container> returnContainerList =
         new ArrayList<Container>(newlyAllocatedContainers.size());
-    List<NMToken> nmTokens = new ArrayList<NMToken>();
     for (Iterator<RMContainer> i = newlyAllocatedContainers.iterator(); i
-      .hasNext();) {
+        .hasNext();) {
       RMContainer rmContainer = i.next();
-      Container container = rmContainer.getContainer();
-      ContainerType containerType = ContainerType.TASK;
-      boolean isWaitingForAMContainer = isWaitingForAMContainer(
-          container.getId().getApplicationAttemptId().getApplicationId());
-      if (isWaitingForAMContainer) {
-        containerType = ContainerType.APPLICATION_MASTER;
+      Container updatedContainer =
+          updateContainerAndNMToken(rmContainer, true, false);
+      // Only add container to return list when it's not null. updatedContainer
+      // could be null when generate token failed, it can be caused by DNS
+      // resolving failed.
+      if (updatedContainer != null) {
+        returnContainerList.add(updatedContainer);
+        i.remove();
       }
-      try {
-        // create container token and NMToken altogether.
-        container.setContainerToken(rmContext.getContainerTokenSecretManager()
-            .createContainerToken(container.getId(), container.getNodeId(),
-                getUser(), container.getResource(), container.getPriority(),
-                rmContainer.getCreationTime(), this.logAggregationContext,
-                rmContainer.getNodeLabelExpression(), containerType));
-        NMToken nmToken =
-            rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
-              getApplicationAttemptId(), container);
-        if (nmToken != null) {
-          nmTokens.add(nmToken);
-        }
-      } catch (IllegalArgumentException e) {
-        // DNS might be down, skip returning this container.
-        LOG.error("Error trying to assign container token and NM token to" +
-            " an allocated container " + container.getId(), e);
-        continue;
+    }
+    return returnContainerList;
+  }
+  
+  private synchronized List<Container> pullNewlyUpdatedContainers(
+      Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
+    List<Container> returnContainerList =
+        new ArrayList<Container>(updatedContainerMap.size());
+    for (Iterator<Entry<ContainerId, RMContainer>> i =
+        updatedContainerMap.entrySet().iterator(); i.hasNext();) {
+      RMContainer rmContainer = i.next().getValue();
+      Container updatedContainer =
+          updateContainerAndNMToken(rmContainer, false, increase);
+      if (updatedContainer != null) {
+        returnContainerList.add(updatedContainer);
+        i.remove();
       }
-      returnContainerList.add(container);
-      i.remove();
-      rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
-        RMContainerEventType.ACQUIRED));
     }
-    return new ContainersAndNMTokensAllocation(returnContainerList, nmTokens);
+    return returnContainerList;
+  }
+
+  public synchronized List<Container> pullNewlyIncreasedContainers() {
+    return pullNewlyUpdatedContainers(newlyIncreasedContainers, true);
+  }
+  
+  public synchronized List<Container> pullNewlyDecreasedContainers() {
+    return pullNewlyUpdatedContainers(newlyDecreasedContainers, false);
+  }
+  
+  public synchronized List<NMToken> pullUpdatedNMTokens() {
+    List<NMToken> returnList = new ArrayList<NMToken>(updatedNMTokens);
+    updatedNMTokens.clear();
+    return returnList;
   }
 
   public boolean isWaitingForAMContainer(ApplicationId applicationId) {
@@ -770,4 +835,50 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return attemptResourceUsage;
   }
   
-}
+  public synchronized boolean removeIncreaseRequest(NodeId nodeId,
+      Priority priority, ContainerId containerId) {
+    return appSchedulingInfo.removeIncreaseRequest(nodeId, priority,
+        containerId);
+  }
+  
+  public synchronized boolean updateIncreaseRequests(
+      List<SchedContainerChangeRequest> increaseRequests) {
+    return appSchedulingInfo.updateIncreaseRequests(increaseRequests);
+  }
+  
+  private synchronized void changeContainerResource(
+      SchedContainerChangeRequest changeRequest, boolean increase) {
+    if (increase) {
+      appSchedulingInfo.increaseContainer(changeRequest);
+    } else {
+      appSchedulingInfo.decreaseContainer(changeRequest);
+    }
+
+    RMContainer changedRMContainer = changeRequest.getRMContainer(); 
+    changedRMContainer.handle(
+        new RMContainerChangeResourceEvent(changeRequest.getContainerId(),
+            changeRequest.getTargetCapacity(), increase));
+
+    // remove pending and not pulled by AM newly-increased/decreased-containers
+    // and add the new one
+    if (increase) {
+      newlyDecreasedContainers.remove(changeRequest.getContainerId());
+      newlyIncreasedContainers.put(changeRequest.getContainerId(),
+          changedRMContainer);
+    } else {
+      newlyIncreasedContainers.remove(changeRequest.getContainerId());
+      newlyDecreasedContainers.put(changeRequest.getContainerId(),
+          changedRMContainer);
+    }
+  }
+  
+  public synchronized void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest) {
+    changeContainerResource(decreaseRequest, false);
+  }
+  
+  public synchronized void increaseContainer(
+      SchedContainerChangeRequest increaseRequest) {
+    changeContainerResource(increaseRequest, true);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 f03663a..f3d3906 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
@@ -157,6 +157,37 @@ public abstract class SchedulerNode {
         + getUsedResource() + " used and " + getAvailableResource()
         + " available after allocation");
   }
+  
+  private synchronized void changeContainerResource(ContainerId containerId,
+      Resource deltaResource, boolean increase) {
+    if (increase) {
+      deductAvailableResource(deltaResource);
+    } else {
+      addAvailableResource(deltaResource);
+    }
+
+    LOG.info((increase ? "Increased" : "Decreased") + " container "
+        + containerId + " of capacity " + deltaResource + " on host "
+        + rmNode.getNodeAddress() + ", which has " + numContainers
+        + " containers, " + getUsedResource() + " used and "
+        + getAvailableResource() + " available after allocation");
+  }
+  
+  /**
+   * The Scheduler increased container
+   */
+  public synchronized void increaseContainer(ContainerId containerId,
+      Resource deltaResource) {
+    changeContainerResource(containerId, deltaResource, true);
+  }
+  
+  /**
+   * The Scheduler decreased container
+   */
+  public synchronized void decreaseContainer(ContainerId containerId,
+      Resource deltaResource) {
+    changeContainerResource(containerId, deltaResource, false);
+  }
 
   /**
    * Get available resources on the node.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.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/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 8047d0b..abefee8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -361,7 +361,7 @@ public class SchedulerUtils {
   }
   
   public static boolean checkResourceRequestMatchingNodePartition(
-      ResourceRequest offswitchResourceRequest, String nodePartition,
+      String requestedPartition, String nodePartition,
       SchedulingMode schedulingMode) {
     // We will only look at node label = nodeLabelToLookAt according to
     // schedulingMode and partition of node.
@@ -371,12 +371,11 @@ public class SchedulerUtils {
     } else {
       nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
     }
-    
-    String askedNodePartition = offswitchResourceRequest.getNodeLabelExpression();
-    if (null == askedNodePartition) {
-      askedNodePartition = RMNodeLabelsManager.NO_LABEL;
+
+    if (null == requestedPartition) {
+      requestedPartition = RMNodeLabelsManager.NO_LABEL;
     }
-    return askedNodePartition.equals(nodePartitionToLookAt);
+    return requestedPartition.equals(nodePartitionToLookAt);
   }
   
   private static boolean hasPendingResourceRequest(ResourceCalculator rc,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index f2753e6..099db69 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -133,16 +134,17 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * @param release
    * @param blacklistAdditions 
    * @param blacklistRemovals 
+   * @param increaseRequests
+   * @param decreaseRequests
    * @return the {@link Allocation} for the application
    */
   @Public
   @Stable
-  Allocation 
-  allocate(ApplicationAttemptId appAttemptId, 
-      List<ResourceRequest> ask,
-      List<ContainerId> release, 
-      List<String> blacklistAdditions, 
-      List<String> blacklistRemovals);
+  Allocation allocate(ApplicationAttemptId appAttemptId,
+      List<ResourceRequest> ask, List<ContainerId> release,
+      List<String> blacklistAdditions, List<String> blacklistRemovals,
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests);
 
   /**
    * Get node resource usage report.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 0ae4d1a..9f61b11 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
@@ -43,10 +43,10 @@ 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.scheduler.NodeType;
 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;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -76,7 +76,7 @@ public abstract class AbstractCSQueue implements CSQueue {
   private boolean preemptionDisabled;
 
   // Track resource usage-by-label like used-resource/pending-resource, etc.
-  ResourceUsage queueUsage;
+  volatile ResourceUsage queueUsage;
   
   // Track capacities like used-capcity/abs-used-capacity/capacity/abs-capacity,
   // etc.
@@ -340,22 +340,27 @@ public abstract class AbstractCSQueue implements CSQueue {
     return minimumAllocation;
   }
   
-  synchronized void allocateResource(Resource clusterResource, 
-      Resource resource, String nodePartition) {
+  synchronized void allocateResource(Resource clusterResource,
+      Resource resource, String nodePartition, boolean changeContainerResource) {
     queueUsage.incUsed(nodePartition, resource);
 
-    ++numContainers;
+    if (!changeContainerResource) {
+      ++numContainers;
+    }
     CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
         minimumAllocation, this, labelManager, nodePartition);
   }
   
   protected synchronized void releaseResource(Resource clusterResource,
-      Resource resource, String nodePartition) {
+      Resource resource, String nodePartition, boolean changeContainerResource) {
     queueUsage.decUsed(nodePartition, resource);
 
     CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
         minimumAllocation, this, labelManager, nodePartition);
-    --numContainers;
+
+    if (!changeContainerResource) {
+      --numContainers;
+    }
   }
   
   @Private
@@ -446,8 +451,8 @@ public abstract class AbstractCSQueue implements CSQueue {
   }
   
   synchronized boolean canAssignToThisQueue(Resource clusterResource,
-      String nodePartition, ResourceLimits currentResourceLimits, Resource resourceCouldBeUnreserved,
-      SchedulingMode schedulingMode) {
+      String nodePartition, ResourceLimits currentResourceLimits,
+      Resource resourceCouldBeUnreserved, SchedulingMode schedulingMode) {
     // Get current limited resource: 
     // - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
     // queues' max capacity.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 928437f..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
@@ -41,6 +41,7 @@ public class CSAssignment {
   private final boolean skipped;
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
+  private boolean increaseAllocation;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -138,4 +139,12 @@ public class CSAssignment {
   public AssignmentInformation getAssignmentInformation() {
     return this.assignmentInformation;
   }
+  
+  public boolean isIncreasedAllocation() {
+    return increaseAllocation;
+  }
+
+  public void setIncreasedAllocation(boolean flag) {
+    increaseAllocation = flag;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.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/CSQueue.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/CSQueue.java
index 9855dd4..e90deeb 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/CSQueue.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/CSQueue.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 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;
+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;
 
@@ -219,6 +220,14 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
       boolean sortQueues);
 
   /**
+   * We have a reserved increased container in the queue, we need to unreserve
+   * it. Since we just want to cancel the reserved increase request instead of
+   * stop the container, we shouldn't call completedContainer for such purpose.
+   */
+  public void unreserveIncreasedContainer(Resource clusterResource,
+      FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer);
+
+  /**
    * Get the number of applications in the queue.
    * @return number of applications
    */
@@ -313,4 +322,11 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    *          new resource asked
    */
   public void decPendingResource(String nodeLabel, Resource resourceToDec);
+  
+  /**
+   * Decrease container resource in the queue
+   */
+  public void decreaseContainer(Resource clusterResource,
+      SchedContainerChangeRequest decreaseRequest,
+      FiCaSchedulerApp app);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 dbaccaf..cf7f6c0 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.EnumSet;
@@ -52,6 +53,7 @@ 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.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -84,6 +86,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
@@ -95,6 +98,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueNotFoundException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
+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.SchedulerDynamicEditException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
@@ -884,10 +889,14 @@ public class CapacityScheduler extends
   }
 
   @Override
+  // Note: when AM asks to decrease container or release container, we will
+  // acquire scheduler lock
   @Lock(Lock.NoLock.class)
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
-      List<ResourceRequest> ask, List<ContainerId> release, 
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
+      List<ResourceRequest> ask, List<ContainerId> release,
+      List<String> blacklistAdditions, List<String> blacklistRemovals,
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
 
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
@@ -898,6 +907,14 @@ public class CapacityScheduler extends
     SchedulerUtils.normalizeRequests(
         ask, getResourceCalculator(), getClusterResource(),
         getMinimumResourceCapability(), getMaximumResourceCapability());
+    
+    // Pre-process increase requests
+    List<SchedContainerChangeRequest> normalizedIncreaseRequests =
+        checkAndNormalizeContainerChangeRequests(increaseRequests, true);
+    
+    // Pre-process decrease requests
+    List<SchedContainerChangeRequest> normalizedDecreaseRequests =
+        checkAndNormalizeContainerChangeRequests(decreaseRequests, false);
 
     // Release containers
     releaseContainers(release, application);
@@ -914,8 +931,8 @@ public class CapacityScheduler extends
         return EMPTY_ALLOCATION;
       }
 
+      // Process resource requests
       if (!ask.isEmpty()) {
-
         if(LOG.isDebugEnabled()) {
           LOG.debug("allocate: pre-update " + applicationAttemptId +
               " ask size =" + ask.size());
@@ -932,6 +949,12 @@ public class CapacityScheduler extends
           application.showRequests();
         }
       }
+      
+      // Process increase resource requests
+      if (application.updateIncreaseRequests(normalizedIncreaseRequests)
+          && (updateDemandForQueue == null)) {
+        updateDemandForQueue = (LeafQueue) application.getQueue();
+      }
 
       if (application.isWaitingForAMContainer(application.getApplicationId())) {
         // Allocate is for AM and update AM blacklist for this
@@ -940,6 +963,9 @@ public class CapacityScheduler extends
       } else {
         application.updateBlacklist(blacklistAdditions, blacklistRemovals);
       }
+      
+      // Decrease containers
+      decreaseContainers(normalizedDecreaseRequests, application);
 
       allocation = application.getAllocation(getResourceCalculator(),
                    clusterResource, getMinimumResourceCapability());
@@ -1001,6 +1027,13 @@ public class CapacityScheduler extends
     for (ContainerStatus launchedContainer : newlyLaunchedContainers) {
       containerLaunchedOnNode(launchedContainer.getContainerId(), node);
     }
+    
+    // Processing the newly increased containers
+    List<Container> newlyIncreasedContainers =
+        nm.pullNewlyIncreasedContainers();
+    for (Container container : newlyIncreasedContainers) {
+      containerIncreasedOnNode(container.getId(), node, container);
+    }
 
     // Process completed containers
     int releasedContainers = 0;
@@ -1486,6 +1519,50 @@ public class CapacityScheduler extends
         container.getId(), queue.getQueuePath());
     }
   }
+  
+  @Lock(CapacityScheduler.class)
+  @Override
+  protected synchronized void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest,
+      SchedulerApplicationAttempt attempt) {
+    RMContainer rmContainer = decreaseRequest.getRMContainer();
+
+    // Check container status before doing decrease
+    if (rmContainer.getState() != RMContainerState.RUNNING) {
+      LOG.info("Trying to decrease a container not in RUNNING state, container="
+          + rmContainer + " state=" + rmContainer.getState().name());
+      return;
+    }
+    
+    // Delta capacity of this decrease request is 0, this decrease request may
+    // just to cancel increase request
+    if (Resources.equals(decreaseRequest.getDeltaCapacity(), Resources.none())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Decrease target resource equals to existing resource for container:"
+            + decreaseRequest.getContainerId()
+            + " ignore this decrease request.");
+      }
+      return;
+    }
+
+    // Save resource before decrease
+    Resource resourceBeforeDecrease =
+        Resources.clone(rmContainer.getContainer().getResource());
+
+    FiCaSchedulerApp app = (FiCaSchedulerApp)attempt;
+    LeafQueue queue = (LeafQueue) attempt.getQueue();
+    queue.decreaseContainer(clusterResource, decreaseRequest, app);
+    
+    // Notify RMNode the container will be decreased
+    this.rmContext.getDispatcher().getEventHandler()
+        .handle(new RMNodeDecreaseContainerEvent(decreaseRequest.getNodeId(),
+            Arrays.asList(rmContainer.getContainer())));
+    
+    LOG.info("Application attempt " + app.getApplicationAttemptId()
+        + " decreased container:" + decreaseRequest.getContainerId() + " from "
+        + resourceBeforeDecrease + " to "
+        + decreaseRequest.getTargetCapacity());
+  }
 
   @Lock(Lock.NoLock.class)
   @VisibleForTesting


[5/5] hadoop git commit: YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan

Posted by ji...@apache.org.
YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan


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

Branch: refs/heads/YARN-1197
Commit: 4a7b1d3d57a5e5c64e445a23ce40c99febb5fa56
Parents: 78ad04d
Author: Jian He <ji...@apache.org>
Authored: Tue Sep 15 10:21:39 2015 +0800
Committer: Jian He <ji...@apache.org>
Committed: Tue Sep 15 10:21:49 2015 +0800

----------------------------------------------------------------------
 .../v2/app/rm/TestRMContainerAllocator.java     |  11 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |  14 +
 .../yarn/sls/scheduler/RMNodeWrapper.java       |  13 +
 .../sls/scheduler/ResourceSchedulerWrapper.java |  21 +-
 .../sls/scheduler/SLSCapacityScheduler.java     |  19 +-
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../api/impl/TestAMRMClientOnRMRestart.java     |   8 +-
 .../resource/DefaultResourceCalculator.java     |   5 +
 .../resource/DominantResourceCalculator.java    |   6 +
 .../yarn/util/resource/ResourceCalculator.java  |   5 +
 .../hadoop/yarn/util/resource/Resources.java    |   5 +
 .../util/resource/TestResourceCalculator.java   |  30 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |   5 +-
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |   5 +-
 .../ApplicationMasterService.java               |  22 +-
 .../server/resourcemanager/RMAuditLogger.java   |   2 +
 .../server/resourcemanager/RMServerUtils.java   | 164 ++++
 .../resourcemanager/ResourceTrackerService.java |   7 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |   4 +-
 .../rmcontainer/RMContainer.java                |   4 +
 .../RMContainerChangeResourceEvent.java         |  44 +
 .../rmcontainer/RMContainerEventType.java       |  13 +-
 .../rmcontainer/RMContainerImpl.java            | 121 ++-
 .../RMContainerUpdatesAcquiredEvent.java        |  35 +
 .../server/resourcemanager/rmnode/RMNode.java   |   9 +
 .../rmnode/RMNodeDecreaseContainerEvent.java    |  39 +
 .../resourcemanager/rmnode/RMNodeEventType.java |   1 +
 .../resourcemanager/rmnode/RMNodeImpl.java      |  93 ++
 .../rmnode/RMNodeStatusEvent.java               |  32 +-
 .../scheduler/AbstractYarnScheduler.java        | 150 ++-
 .../resourcemanager/scheduler/Allocation.java   |  22 +-
 .../scheduler/AppSchedulingInfo.java            | 249 ++++-
 .../resourcemanager/scheduler/QueueMetrics.java |  16 +-
 .../scheduler/SchedContainerChangeRequest.java  | 118 +++
 .../scheduler/SchedulerApplication.java         |   2 +-
 .../scheduler/SchedulerApplicationAttempt.java  | 253 +++--
 .../scheduler/SchedulerNode.java                |  31 +
 .../scheduler/SchedulerUtils.java               |  11 +-
 .../scheduler/YarnScheduler.java                |  14 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  23 +-
 .../scheduler/capacity/CSAssignment.java        |   9 +
 .../scheduler/capacity/CSQueue.java             |  16 +
 .../scheduler/capacity/CapacityScheduler.java   |  83 +-
 .../scheduler/capacity/LeafQueue.java           | 127 ++-
 .../scheduler/capacity/ParentQueue.java         | 115 ++-
 .../allocator/AbstractContainerAllocator.java   | 131 +++
 .../capacity/allocator/ContainerAllocator.java  | 149 +--
 .../allocator/IncreaseContainerAllocator.java   | 365 +++++++
 .../allocator/RegularContainerAllocator.java    |  30 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  68 +-
 .../scheduler/fair/FairScheduler.java           |  35 +-
 .../scheduler/fifo/FifoScheduler.java           |  25 +-
 .../server/resourcemanager/Application.java     |   2 +-
 .../yarn/server/resourcemanager/MockAM.java     |   9 +
 .../yarn/server/resourcemanager/MockNodes.java  |  13 +
 .../yarn/server/resourcemanager/MockRM.java     |  13 +
 .../TestApplicationMasterService.java           | 144 ++-
 .../applicationsmanager/TestAMRestart.java      |  15 +-
 .../TestRMAppLogAggregationStatus.java          |  10 +-
 .../attempt/TestRMAppAttemptTransitions.java    |  32 +-
 .../rmcontainer/TestRMContainerImpl.java        | 117 ++-
 .../capacity/TestCapacityScheduler.java         | 128 ++-
 .../scheduler/capacity/TestChildQueueOrder.java |   4 +-
 .../capacity/TestContainerAllocation.java       |  50 +-
 .../capacity/TestContainerResizing.java         | 963 +++++++++++++++++++
 .../scheduler/capacity/TestLeafQueue.java       |   4 +-
 .../scheduler/capacity/TestParentQueue.java     |   4 +-
 .../scheduler/capacity/TestReservations.java    |   9 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |   6 +-
 .../fair/TestContinuousScheduling.java          |   2 +-
 .../scheduler/fair/TestFairScheduler.java       |  30 +-
 .../scheduler/fifo/TestFifoScheduler.java       |  28 +-
 72 files changed, 3856 insertions(+), 509 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index e148c32..2bb7e27 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -98,6 +98,7 @@ 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.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -1575,8 +1576,10 @@ public class TestRMContainerAllocator {
     @Override
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-        List<ContainerId> release, 
-        List<String> blacklistAdditions, List<String> blacklistRemovals) {
+        List<ContainerId> release, List<String> blacklistAdditions,
+        List<String> blacklistRemovals,
+        List<ContainerResourceChangeRequest> increaseRequests,
+        List<ContainerResourceChangeRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
@@ -1590,8 +1593,8 @@ public class TestRMContainerAllocator {
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistRemovals = blacklistRemovals;
       return super.allocate(
-          applicationAttemptId, askCopy, release, 
-          blacklistAdditions, blacklistRemovals);
+          applicationAttemptId, askCopy, release, blacklistAdditions,
+          blacklistRemovals, increaseRequests, decreaseRequests);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
index 2d2c3e0..dae2ce7 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.net.Node;
 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.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -174,6 +175,19 @@ public class NodeInfo {
     public Set<String> getNodeLabels() {
       return RMNodeLabelsManager.EMPTY_STRING_SET;
     }
+
+    @Override
+    public void updateNodeHeartbeatResponseForContainersDecreasing(
+        NodeHeartbeatResponse response) {
+      // TODO Auto-generated method stub
+      
+    }
+
+    @Override
+    public List<Container> pullNewlyIncreasedContainers() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   public static RMNode newNodeInfo(String rackName, String hostName,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
index ecc4734..8c65ccc 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -163,4 +164,16 @@ public class RMNodeWrapper implements RMNode {
   public Set<String> getNodeLabels() {
     return RMNodeLabelsManager.EMPTY_STRING_SET;
   }
+
+  @Override
+  public void updateNodeHeartbeatResponseForContainersDecreasing(
+      NodeHeartbeatResponse response) {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
+  public List<Container> pullNewlyIncreasedContainers() {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
index 14e2645..310b3b5 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 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.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
@@ -202,15 +204,16 @@ final public class ResourceSchedulerWrapper
 
   @Override
   public Allocation allocate(ApplicationAttemptId attemptId,
-                             List<ResourceRequest> resourceRequests,
-                             List<ContainerId> containerIds,
-                             List<String> strings, List<String> strings2) {
+      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
+      List<String> strings, List<String> strings2,
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
     if (metricsON) {
       final Timer.Context context = schedulerAllocateTimer.time();
       Allocation allocation = null;
       try {
         allocation = scheduler.allocate(attemptId, resourceRequests,
-                containerIds, strings, strings2);
+                containerIds, strings, strings2, null, null);
         return allocation;
       } finally {
         context.stop();
@@ -224,7 +227,7 @@ final public class ResourceSchedulerWrapper
       }
     } else {
       return scheduler.allocate(attemptId,
-              resourceRequests, containerIds, strings, strings2);
+              resourceRequests, containerIds, strings, strings2, null, null);
     }
   }
 
@@ -959,4 +962,12 @@ final public class ResourceSchedulerWrapper
     return Priority.newInstance(0);
   }
 
+  @Override
+  protected void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest,
+      SchedulerApplicationAttempt attempt) {
+    // TODO Auto-generated method stub
+    
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
index a4416db..3626027 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
@@ -48,6 +48,7 @@ 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.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -176,15 +177,17 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
 
   @Override
   public Allocation allocate(ApplicationAttemptId attemptId,
-                             List<ResourceRequest> resourceRequests,
-                             List<ContainerId> containerIds,
-                             List<String> strings, List<String> strings2) {
+      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
+      List<String> strings, List<String> strings2,
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
     if (metricsON) {
       final Timer.Context context = schedulerAllocateTimer.time();
       Allocation allocation = null;
       try {
-        allocation = super.allocate(attemptId, resourceRequests,
-                containerIds, strings, strings2);
+        allocation = super
+            .allocate(attemptId, resourceRequests, containerIds, strings,
+                strings2, increaseRequests, decreaseRequests);
         return allocation;
       } finally {
         context.stop();
@@ -197,8 +200,8 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
         }
       }
     } else {
-      return super.allocate(attemptId,
-              resourceRequests, containerIds, strings, strings2);
+      return super.allocate(attemptId, resourceRequests, containerIds, strings,
+          strings2, increaseRequests, decreaseRequests);
     }
   }
 
@@ -426,7 +429,7 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
     if (pool != null)  pool.shutdown();
   }
 
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({ "unchecked", "rawtypes" })
   private void initMetrics() throws Exception {
     metrics = new MetricRegistry();
     // configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index eb00781..44dbb55 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -210,6 +210,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3868. Recovery support for container resizing. (Meng Ding via jianhe)
 
+    YARN-1651. CapacityScheduler side changes to support container resize.
+    (Wangda Tan via jianhe)
+ 
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
index 108ad37..2394747 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 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.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -525,7 +526,9 @@ public class TestAMRMClientOnRMRestart {
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
-        List<String> blacklistRemovals) {
+        List<String> blacklistRemovals,
+        List<ContainerResourceChangeRequest> increaseRequests,
+        List<ContainerResourceChangeRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy =
@@ -539,7 +542,8 @@ public class TestAMRMClientOnRMRestart {
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistRemovals = blacklistRemovals;
       return super.allocate(applicationAttemptId, askCopy, release,
-          blacklistAdditions, blacklistRemovals);
+          blacklistAdditions, blacklistRemovals, increaseRequests,
+          decreaseRequests);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
index c2fc1f0..2fdf214 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
@@ -110,4 +110,9 @@ public class DefaultResourceCalculator extends ResourceCalculator {
         );
   }
 
+  @Override
+  public boolean fitsIn(Resource cluster,
+      Resource smaller, Resource bigger) {
+    return smaller.getMemory() <= bigger.getMemory();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 2ee95ce..b5c9967 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -209,4 +209,10 @@ public class DominantResourceCalculator extends ResourceCalculator {
         );
   }
 
+  @Override
+  public boolean fitsIn(Resource cluster,
+      Resource smaller, Resource bigger) {
+    return smaller.getMemory() <= bigger.getMemory()
+        && smaller.getVirtualCores() <= bigger.getVirtualCores();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index 442196c..3a31225 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -171,4 +171,9 @@ public abstract class ResourceCalculator {
    */
   public abstract Resource divideAndCeil(Resource numerator, int denominator);
   
+  /**
+   * Check if a smaller resource can be contained by bigger resource.
+   */
+  public abstract boolean fitsIn(Resource cluster,
+      Resource smaller, Resource bigger);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 503d456..b05d021 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -267,6 +267,11 @@ public class Resources {
     return smaller.getMemory() <= bigger.getMemory() &&
         smaller.getVirtualCores() <= bigger.getVirtualCores();
   }
+
+  public static boolean fitsIn(ResourceCalculator rc, Resource cluster,
+      Resource smaller, Resource bigger) {
+    return rc.fitsIn(cluster, smaller, bigger);
+  }
   
   public static Resource componentwiseMin(Resource lhs, Resource rhs) {
     return createResource(Math.min(lhs.getMemory(), rhs.getMemory()),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
index 6a0b62e..0654891 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
@@ -41,6 +41,35 @@ public class TestResourceCalculator {
   public TestResourceCalculator(ResourceCalculator rs) {
     this.resourceCalculator = rs;
   }
+  
+  @Test(timeout = 10000)
+  public void testFitsIn() {
+    Resource cluster = Resource.newInstance(1024, 1);
+
+    if (resourceCalculator instanceof DefaultResourceCalculator) {
+      Assert.assertTrue(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(2, 1)));
+      Assert.assertTrue(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(2, 2)));
+      Assert.assertTrue(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(1, 2)));
+      Assert.assertTrue(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(1, 1)));
+      Assert.assertFalse(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(2, 1), Resource.newInstance(1, 2)));
+    } else if (resourceCalculator instanceof DominantResourceCalculator) {
+      Assert.assertFalse(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(2, 1)));
+      Assert.assertTrue(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(2, 2)));
+      Assert.assertTrue(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(1, 2)));
+      Assert.assertFalse(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(1, 2), Resource.newInstance(1, 1)));
+      Assert.assertFalse(resourceCalculator.fitsIn(cluster,
+          Resource.newInstance(2, 1), Resource.newInstance(1, 2)));
+    }
+  }
 
   @Test(timeout = 10000)
   public void testResourceCalculatorCompareMethod() {
@@ -92,7 +121,6 @@ public class TestResourceCalculator {
 
   }
 
-
   private void assertResourcesOperations(Resource clusterResource,
       Resource lhs, Resource rhs, boolean lessThan, boolean lessThanOrEqual,
       boolean greaterThan, boolean greaterThanOrEqual, Resource max,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
index 38fbc82..c0ccf57 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
@@ -19,12 +19,13 @@
 package org.apache.hadoop.yarn.server.api.protocolrecords;
 
 import java.nio.ByteBuffer;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 
@@ -73,5 +74,5 @@ public interface NodeHeartbeatResponse {
   void setAreNodeLabelsAcceptedByRM(boolean areNodeLabelsAcceptedByRM);
 
   List<Container> getContainersToDecrease();
-  void addAllContainersToDecrease(List<Container> containersToDecrease);
+  void addAllContainersToDecrease(Collection<Container> containersToDecrease);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
index 12c5230..dc65141 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
@@ -20,14 +20,15 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
@@ -437,7 +438,7 @@ public class NodeHeartbeatResponsePBImpl extends
 
   @Override
   public void addAllContainersToDecrease(
-      final List<Container> containersToDecrease) {
+      final Collection<Container> containersToDecrease) {
     if (containersToDecrease == null) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.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/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 14142de..87c7bfa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -451,11 +451,13 @@ public class ApplicationMasterService extends AbstractService implements
           req.setNodeLabelExpression(asc.getNodeLabelExpression());
         }
       }
+      
+      Resource maximumCapacity = rScheduler.getMaximumResourceCapability();
               
       // sanity check
       try {
         RMServerUtils.normalizeAndValidateRequests(ask,
-            rScheduler.getMaximumResourceCapability(), app.getQueue(),
+            maximumCapacity, app.getQueue(),
             rScheduler, rmContext);
       } catch (InvalidResourceRequestException e) {
         LOG.warn("Invalid resource ask by application " + appAttemptId, e);
@@ -469,6 +471,15 @@ public class ApplicationMasterService extends AbstractService implements
         throw e;
       }
 
+      try {
+        RMServerUtils.increaseDecreaseRequestSanityCheck(rmContext,
+            request.getIncreaseRequests(), request.getDecreaseRequests(),
+            maximumCapacity);
+      } catch (InvalidResourceRequestException e) {
+        LOG.warn(e);
+        throw e;
+      }
+
       // In the case of work-preserving AM restart, it's possible for the
       // AM to release containers from the earlier attempt.
       if (!app.getApplicationSubmissionContext()
@@ -493,8 +504,9 @@ public class ApplicationMasterService extends AbstractService implements
         allocation = EMPTY_ALLOCATION;
       } else {
         allocation =
-          this.rScheduler.allocate(appAttemptId, ask, release,
-              blacklistAdditions, blacklistRemovals);
+            this.rScheduler.allocate(appAttemptId, ask, release,
+                blacklistAdditions, blacklistRemovals,
+                request.getIncreaseRequests(), request.getDecreaseRequests());
       }
 
       if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
@@ -540,6 +552,10 @@ public class ApplicationMasterService extends AbstractService implements
           .pullJustFinishedContainers());
       allocateResponse.setResponseId(lastResponse.getResponseId() + 1);
       allocateResponse.setAvailableResources(allocation.getResourceLimit());
+      
+      // Handling increased/decreased containers
+      allocateResponse.setIncreasedContainers(allocation.getIncreasedContainers());
+      allocateResponse.setDecreasedContainers(allocation.getDecreasedContainers());
 
       allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.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/RMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
index f049d97..cd9a61d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
@@ -56,6 +56,8 @@ public class RMAuditLogger {
     public static final String RELEASE_CONTAINER = "AM Released Container";
     public static final String UPDATE_APP_PRIORITY =
         "Update Application Priority Request";
+    public static final String CHANGE_CONTAINER_RESOURCE =
+        "AM Changed Container Resource";
 
     // Some commonly used descriptions
     public static final String UNAUTHORIZED_USER = "Unauthorized user";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.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/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index 4d2e41c..cc30593 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -22,8 +22,10 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +36,7 @@ import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -49,10 +52,14 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 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.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
@@ -107,6 +114,89 @@ public class RMServerUtils {
           queueName, scheduler, rmContext, queueInfo);
     }
   }
+  
+  /**
+   * Normalize container increase/decrease request, it will normalize and update
+   * ContainerResourceChangeRequest.targetResource
+   * 
+   * <pre>
+   * - Throw exception when any other error happens
+   * </pre>
+   */
+  public static void checkAndNormalizeContainerChangeRequest(
+      RMContext rmContext, ContainerResourceChangeRequest request,
+      boolean increase) throws InvalidResourceRequestException {
+    ContainerId containerId = request.getContainerId();
+    ResourceScheduler scheduler = rmContext.getScheduler();
+    RMContainer rmContainer = scheduler.getRMContainer(containerId);
+    ResourceCalculator rc = scheduler.getResourceCalculator();
+    
+    if (null == rmContainer) {
+      String msg =
+          "Failed to get rmContainer for "
+              + (increase ? "increase" : "decrease")
+              + " request, with container-id=" + containerId;
+      throw new InvalidResourceRequestException(msg);
+    }
+
+    if (rmContainer.getState() != RMContainerState.RUNNING) {
+      String msg =
+          "rmContainer's state is not RUNNING, for "
+              + (increase ? "increase" : "decrease")
+              + " request, with container-id=" + containerId;
+      throw new InvalidResourceRequestException(msg);
+    }
+
+    Resource targetResource = Resources.normalize(rc, request.getCapability(),
+        scheduler.getMinimumResourceCapability(),
+        scheduler.getMaximumResourceCapability(),
+        scheduler.getMinimumResourceCapability());
+
+    // Compare targetResource and original resource
+    Resource originalResource = rmContainer.getAllocatedResource();
+
+    // Resource comparasion should be >= (or <=) for all resource vectors, for
+    // example, you cannot request target resource of a <10G, 10> container to
+    // <20G, 8>
+    if (increase) {
+      if (originalResource.getMemory() > targetResource.getMemory()
+          || originalResource.getVirtualCores() > targetResource
+              .getVirtualCores()) {
+        String msg =
+            "Trying to increase a container, but target resource has some"
+                + " resource < original resource, target=" + targetResource
+                + " original=" + originalResource + " containerId="
+                + containerId;
+        throw new InvalidResourceRequestException(msg);
+      }
+    } else {
+      if (originalResource.getMemory() < targetResource.getMemory()
+          || originalResource.getVirtualCores() < targetResource
+              .getVirtualCores()) {
+        String msg =
+            "Trying to decrease a container, but target resource has "
+                + "some resource > original resource, target=" + targetResource
+                + " original=" + originalResource + " containerId="
+                + containerId;
+        throw new InvalidResourceRequestException(msg);
+      }
+    }
+    
+    RMNode rmNode = rmContext.getRMNodes().get(rmContainer.getAllocatedNode());
+    
+    // Target resource of the increase request is more than NM can offer
+    if (!Resources.fitsIn(scheduler.getResourceCalculator(),
+        scheduler.getClusterResource(), targetResource,
+        rmNode.getTotalCapability())) {
+      String msg = "Target resource=" + targetResource + " of containerId="
+          + containerId + " is more than node's total resource="
+          + rmNode.getTotalCapability();
+      throw new InvalidResourceRequestException(msg);
+    }
+
+    // Update normalized target resource
+    request.setCapability(targetResource);
+  }
 
   /*
    * @throw <code>InvalidResourceBlacklistRequestException </code> if the
@@ -123,6 +213,80 @@ public class RMServerUtils {
       }
     }
   }
+  
+  /**
+   * Check if we have:
+   * - Request for same containerId and different target resource
+   * - If targetResources violates maximum/minimumAllocation
+   */
+  public static void increaseDecreaseRequestSanityCheck(RMContext rmContext,
+      List<ContainerResourceChangeRequest> incRequests,
+      List<ContainerResourceChangeRequest> decRequests,
+      Resource maximumAllocation) throws InvalidResourceRequestException {
+    checkDuplicatedIncreaseDecreaseRequest(incRequests, decRequests);
+    validateIncreaseDecreaseRequest(rmContext, incRequests, maximumAllocation,
+        true);
+    validateIncreaseDecreaseRequest(rmContext, decRequests, maximumAllocation,
+        false);
+  }
+  
+  private static void checkDuplicatedIncreaseDecreaseRequest(
+      List<ContainerResourceChangeRequest> incRequests,
+      List<ContainerResourceChangeRequest> decRequests)
+          throws InvalidResourceRequestException {
+    String msg = "There're multiple increase or decrease container requests "
+        + "for same containerId=";
+    Set<ContainerId> existedContainerIds = new HashSet<ContainerId>();
+    if (incRequests != null) {
+      for (ContainerResourceChangeRequest r : incRequests) {
+        if (!existedContainerIds.add(r.getContainerId())) {
+          throw new InvalidResourceRequestException(msg + r.getContainerId());
+        }
+      }
+    }
+    
+    if (decRequests != null) {
+      for (ContainerResourceChangeRequest r : decRequests) {
+        if (!existedContainerIds.add(r.getContainerId())) {
+          throw new InvalidResourceRequestException(msg + r.getContainerId());
+        }
+      }
+    }
+  }
+  
+  private static void validateIncreaseDecreaseRequest(RMContext rmContext,
+      List<ContainerResourceChangeRequest> requests, Resource maximumAllocation,
+      boolean increase)
+      throws InvalidResourceRequestException {
+    if (requests == null) {
+      return;
+    }
+    for (ContainerResourceChangeRequest request : requests) {
+      if (request.getCapability().getMemory() < 0
+          || request.getCapability().getMemory() > maximumAllocation
+              .getMemory()) {
+        throw new InvalidResourceRequestException("Invalid "
+            + (increase ? "increase" : "decrease") + " request"
+            + ", requested memory < 0"
+            + ", or requested memory > max configured" + ", requestedMemory="
+            + request.getCapability().getMemory() + ", maxMemory="
+            + maximumAllocation.getMemory());
+      }
+      if (request.getCapability().getVirtualCores() < 0
+          || request.getCapability().getVirtualCores() > maximumAllocation
+              .getVirtualCores()) {
+        throw new InvalidResourceRequestException("Invalid "
+            + (increase ? "increase" : "decrease") + " request"
+            + ", requested virtual cores < 0"
+            + ", or requested virtual cores > max configured"
+            + ", requestedVirtualCores="
+            + request.getCapability().getVirtualCores() + ", maxVirtualCores="
+            + maximumAllocation.getVirtualCores());
+      }
+      
+      checkAndNormalizeContainerChangeRequest(rmContext, request, increase);
+    }
+  }
 
   /**
    * It will validate to make sure all the containers belong to correct

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.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/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 100e991..557f6d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -449,6 +449,8 @@ public class ResourceTrackerService extends AbstractService implements
             getResponseId() + 1, NodeAction.NORMAL, null, null, null, null,
             nextHeartBeatInterval);
     rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
+    rmNode.updateNodeHeartbeatResponseForContainersDecreasing(
+        nodeHeartBeatResponse);
 
     populateKeys(request, nodeHeartBeatResponse);
 
@@ -461,8 +463,9 @@ public class ResourceTrackerService extends AbstractService implements
     // 4. Send status to RMNode, saving the latest response.
     RMNodeStatusEvent nodeStatusEvent =
         new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(),
-          remoteNodeStatus.getContainersStatuses(),
-          remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse);
+            remoteNodeStatus.getContainersStatuses(),
+            remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse,
+            remoteNodeStatus.getIncreasedContainers());
     if (request.getLogAggregationReportsForApps() != null
         && !request.getLogAggregationReportsForApps().isEmpty()) {
       nodeStatusEvent.setLogAggregationReportsForApps(request

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 629b2a3..43de3ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -971,7 +971,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
                 Collections.singletonList(appAttempt.amReq),
                 EMPTY_CONTAINER_RELEASE_LIST,
                 amBlacklist.getAdditions(),
-                amBlacklist.getRemovals());
+                amBlacklist.getRemovals(), null, null);
         if (amContainerAllocation != null
             && amContainerAllocation.getContainers() != null) {
           assert (amContainerAllocation.getContainers().size() == 0);
@@ -995,7 +995,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       Allocation amContainerAllocation =
           appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
             EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
-            null);
+            null, null, null);
       // There must be at least one container allocated, because a
       // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
       // and is put in SchedulerApplication#newlyAllocatedContainers.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 21d79ee..dc0d9ba 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
@@ -82,4 +82,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
   String getNodeHttpAddress();
   
   String getNodeLabelExpression();
+  
+  boolean hasIncreaseReservation();
+  
+  void cancelIncreaseReservation();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.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/RMContainerChangeResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java
new file mode 100644
index 0000000..920cfdb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java
@@ -0,0 +1,44 @@
+/**
+ * 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.rmcontainer;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class RMContainerChangeResourceEvent extends RMContainerEvent {
+  
+  final Resource targetResource;
+  final boolean increase;
+
+  public RMContainerChangeResourceEvent(ContainerId containerId,
+      Resource targetResource, boolean increase) {
+    super(containerId, RMContainerEventType.CHANGE_RESOURCE);
+
+    this.targetResource = targetResource;
+    this.increase = increase;
+  }
+  
+  public Resource getTargetResource() {
+    return targetResource;
+  }
+  
+  public boolean isIncrease() {
+    return increase;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.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/RMContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java
index 259d68b3..a3b4b76 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java
@@ -25,6 +25,10 @@ public enum RMContainerEventType {
   ACQUIRED,
   KILL, // Also from Node on NodeRemoval
   RESERVED,
+  
+  // when a container acquired by AM after
+  // it increased/decreased
+  ACQUIRE_UPDATED_CONTAINER, 
 
   LAUNCHED,
   FINISHED,
@@ -35,5 +39,12 @@ public enum RMContainerEventType {
   // Source: ContainerAllocationExpirer  
   EXPIRE,
 
-  RECOVER
+  RECOVER,
+  
+  // Source: Scheduler
+  // Resource change approved by scheduler
+  CHANGE_RESOURCE,
+  
+  // NM reported resource change is done
+  NM_DONE_CHANGE_RESOURCE 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index a3d8bee..8133657 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -118,7 +118,18 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     .addTransition(RMContainerState.RUNNING, RMContainerState.RELEASED,
         RMContainerEventType.RELEASED, new KillTransition())
     .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
-        RMContainerEventType.EXPIRE)
+        RMContainerEventType.RESERVED, new ContainerReservedTransition())
+    .addTransition(RMContainerState.RUNNING, RMContainerState.EXPIRED,
+        RMContainerEventType.EXPIRE,
+        new ContainerExpiredWhileRunningTransition())
+    .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
+        RMContainerEventType.CHANGE_RESOURCE, new ChangeResourceTransition())
+    .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
+        RMContainerEventType.ACQUIRE_UPDATED_CONTAINER, 
+        new ContainerAcquiredWhileRunningTransition())
+    .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
+        RMContainerEventType.NM_DONE_CHANGE_RESOURCE, 
+        new NMReportedContainerChangeIsDoneTransition())
 
     // Transitions from COMPLETED state
     .addTransition(RMContainerState.COMPLETED, RMContainerState.COMPLETED,
@@ -140,9 +151,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
             RMContainerEventType.KILL, RMContainerEventType.FINISHED))
 
     // create the topology tables
-    .installTopology(); 
-                        
-                        
+    .installTopology();
 
   private final StateMachine<RMContainerState, RMContainerEventType,
                                                  RMContainerEvent> stateMachine;
@@ -166,6 +175,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
   private ContainerStatus finishedStatus;
   private boolean isAMContainer;
   private List<ResourceRequest> resourceRequests;
+
+  private volatile boolean hasIncreaseReservation = false;
   
   public RMContainerImpl(Container container,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
@@ -264,7 +275,12 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
 
   @Override
   public Resource getAllocatedResource() {
-    return container.getResource();
+    try {
+      readLock.lock();
+      return container.getResource();
+    } finally {
+      readLock.unlock();
+    }
   }
 
   @Override
@@ -471,8 +487,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     }
   }
 
-  private static final class ContainerReservedTransition extends
-  BaseTransition {
+  private static final class ContainerReservedTransition
+      extends BaseTransition {
 
     @Override
     public void transition(RMContainerImpl container, RMContainerEvent event) {
@@ -480,6 +496,12 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
       container.reservedResource = e.getReservedResource();
       container.reservedNode = e.getReservedNode();
       container.reservedPriority = e.getReservedPriority();
+      
+      if (!EnumSet.of(RMContainerState.NEW, RMContainerState.RESERVED)
+          .contains(container.getState())) {
+        // When container's state != NEW/RESERVED, it is an increase reservation
+        container.hasIncreaseReservation = true;
+      }
     }
   }
 
@@ -509,6 +531,70 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
           .getApplicationAttemptId().getApplicationId(), container.nodeId));
     }
   }
+  
+  private static final class ContainerAcquiredWhileRunningTransition extends
+      BaseTransition {
+
+    @Override
+    public void transition(RMContainerImpl container, RMContainerEvent event) {
+      RMContainerUpdatesAcquiredEvent acquiredEvent =
+          (RMContainerUpdatesAcquiredEvent) event;
+      if (acquiredEvent.isIncreasedContainer()) {
+        // If container is increased but not acquired by AM, we will start
+        // containerAllocationExpirer for this container in this transition. 
+        container.containerAllocationExpirer.register(event.getContainerId());
+      }
+    }
+  }
+  
+  private static final class NMReportedContainerChangeIsDoneTransition
+      extends BaseTransition {
+
+    @Override
+    public void transition(RMContainerImpl container, RMContainerEvent event) {
+      // Unregister the allocation expirer, it is already increased..
+      container.containerAllocationExpirer.unregister(event.getContainerId());
+    }
+  }
+  
+  private static final class ContainerExpiredWhileRunningTransition extends
+      BaseTransition {
+
+    @Override
+    public void transition(RMContainerImpl container, RMContainerEvent event) {
+      // When the container expired, and it has a pending increased request, we
+      // will kill the container.
+      // TODO, we can do better for this: roll back container resource to the
+      // resource before increase, and notify scheduler about this decrease as
+      // well. Will do that in a separated JIRA.
+      new KillTransition().transition(container, event);
+    }
+  }
+  
+  private static final class ChangeResourceTransition extends BaseTransition {
+
+    @Override
+    public void transition(RMContainerImpl container, RMContainerEvent event) {
+      RMContainerChangeResourceEvent changeEvent = (RMContainerChangeResourceEvent)event;
+      
+      // Register with containerAllocationExpirer.
+      // For now, we assume timeout for increase is as same as container
+      // allocation.
+      if (!changeEvent.isIncrease()) {
+        // if this is a decrease request, if container was increased but not
+        // told to NM, we can consider previous increase is cancelled,
+        // unregister from the containerAllocationExpirer
+        container.containerAllocationExpirer.unregister(container
+            .getContainerId());
+      }
+      
+      container.container.setResource(changeEvent.getTargetResource());
+      
+      // We reach here means we either allocated increase reservation OR
+      // decreased container, reservation will be cancelled anyway. 
+      container.hasIncreaseReservation = false;
+    }
+  }
 
   private static final class ContainerRescheduledTransition extends
       FinishedTransition {
@@ -561,13 +647,14 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
       RMAppAttempt rmAttempt = container.rmContext.getRMApps()
           .get(container.getApplicationAttemptId().getApplicationId())
           .getCurrentAppAttempt();
-      if (ContainerExitStatus.PREEMPTED == container.finishedStatus
-        .getExitStatus()) {
-        rmAttempt.getRMAppAttemptMetrics().updatePreemptionInfo(resource,
-          container);
-      }
 
       if (rmAttempt != null) {
+        if (ContainerExitStatus.PREEMPTED == container.finishedStatus
+            .getExitStatus()) {
+            rmAttempt.getRMAppAttemptMetrics().updatePreemptionInfo(resource,
+              container);
+          }
+        
         long usedMillis = container.finishTime - container.creationTime;
         long memorySeconds = resource.getMemory()
                               * usedMillis / DateUtils.MILLIS_PER_SECOND;
@@ -665,4 +752,14 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     }
     return -1;
   }
+
+  @Override
+  public boolean hasIncreaseReservation() {
+    return hasIncreaseReservation;
+  }
+
+  @Override
+  public void cancelIncreaseReservation() {
+    hasIncreaseReservation = false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.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/RMContainerUpdatesAcquiredEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java
new file mode 100644
index 0000000..0dccc5f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java
@@ -0,0 +1,35 @@
+/**
+ * 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.rmcontainer;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+public class RMContainerUpdatesAcquiredEvent extends RMContainerEvent  {
+  private final boolean increasedContainer;
+  
+  public RMContainerUpdatesAcquiredEvent(ContainerId containerId,
+      boolean increasedContainer) {
+    super(containerId, RMContainerEventType.ACQUIRE_UPDATED_CONTAINER);
+    this.increasedContainer = increasedContainer; 
+  }
+  
+  public boolean isIncreasedContainer() {
+    return increasedContainer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.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/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
index 00cd3b6..e13e5c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
@@ -24,6 +24,7 @@ import java.util.Set;
 
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -149,4 +150,12 @@ public interface RMNode {
    * @return labels in this node
    */
   public Set<String> getNodeLabels();
+  
+  /**
+   * Update containers to be decreased
+   */
+  public void updateNodeHeartbeatResponseForContainersDecreasing(
+      NodeHeartbeatResponse response);
+  
+  public List<Container> pullNewlyIncreasedContainers();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.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/rmnode/RMNodeDecreaseContainerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
new file mode 100644
index 0000000..62925ad
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
@@ -0,0 +1,39 @@
+/**
+* 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.rmnode;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+
+public class RMNodeDecreaseContainerEvent extends RMNodeEvent {
+  final List<Container> toBeDecreasedContainers;
+
+  public RMNodeDecreaseContainerEvent(NodeId nodeId,
+      List<Container> toBeDecreasedContainers) {
+    super(nodeId, RMNodeEventType.DECREASE_CONTAINER);
+    
+    this.toBeDecreasedContainers = toBeDecreasedContainers;
+  }
+  
+  public List<Container> getToBeDecreasedContainers() {
+    return toBeDecreasedContainers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.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/rmnode/RMNodeEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
index 27ba1c0..a68c894 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
@@ -42,6 +42,7 @@ public enum RMNodeEventType {
   // Source: Container
   CONTAINER_ALLOCATED,
   CLEANUP_CONTAINER,
+  DECREASE_CONTAINER,
 
   // Source: RMAppAttempt
   FINISHED_CONTAINERS_PULLED_BY_AM,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.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/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 7a1ba74..7a43598 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -19,9 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -36,6 +40,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -131,6 +136,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   /* the list of applications that are running on this node */
   private final List<ApplicationId> runningApplications =
       new ArrayList<ApplicationId>();
+  
+  private final Map<ContainerId, Container> toBeDecreasedContainers =
+      new HashMap<>();
+  
+  private final Map<ContainerId, Container> nmReportedIncreasedContainers =
+      new HashMap<>();
 
   private NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory
       .newRecordInstance(NodeHeartbeatResponse.class);
@@ -178,6 +189,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
      .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
          RMNodeEventType.SHUTDOWN,
          new DeactivateNodeTransition(NodeState.SHUTDOWN))
+     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
+         RMNodeEventType.DECREASE_CONTAINER,
+         new DecreaseContainersTransition())
 
      //Transitions from REBOOTED state
      .addTransition(NodeState.REBOOTED, NodeState.REBOOTED,
@@ -430,6 +444,24 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       this.writeLock.unlock();
     }
   };
+  
+  @VisibleForTesting
+  public Collection<Container> getToBeDecreasedContainers() {
+    return toBeDecreasedContainers.values(); 
+  }
+  
+  @Override
+  public void updateNodeHeartbeatResponseForContainersDecreasing(
+      NodeHeartbeatResponse response) {
+    this.writeLock.lock();
+    
+    try {
+      response.addAllContainersToDecrease(toBeDecreasedContainers.values());
+      toBeDecreasedContainers.clear();
+    } finally {
+      this.writeLock.unlock();
+    }
+  }
 
   @Override
   public NodeHeartbeatResponse getLastNodeHeartBeatResponse() {
@@ -759,6 +791,19 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
           RMNodeFinishedContainersPulledByAMEvent) event).getContainers());
     }
   }
+  
+  public static class DecreaseContainersTransition
+      implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
+ 
+    @Override
+    public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
+      RMNodeDecreaseContainerEvent de = (RMNodeDecreaseContainerEvent) event;
+
+      for (Container c : de.getToBeDecreasedContainers()) {
+        rmNode.toBeDecreasedContainers.put(c.getId(), c);
+      }
+    }
+  }
 
   public static class DeactivateNodeTransition
     implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@@ -827,6 +872,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       }
 
       rmNode.handleContainerStatus(statusEvent.getContainers());
+      rmNode.handleReportedIncreasedContainers(
+          statusEvent.getNMReportedIncreasedContainers());
 
       List<LogAggregationReport> logAggregationReportsForApps =
           statusEvent.getLogAggregationReportsForApps();
@@ -919,6 +966,34 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     }
     return nlm.getLabelsOnNode(nodeId);
   }
+  
+  private void handleReportedIncreasedContainers(
+      List<Container> reportedIncreasedContainers) {
+    for (Container container : reportedIncreasedContainers) {
+      ContainerId containerId = container.getId();
+
+      // Don't bother with containers already scheduled for cleanup, or for
+      // applications already killed. The scheduler doens't need to know any
+      // more about this container
+      if (containersToClean.contains(containerId)) {
+        LOG.info("Container " + containerId + " already scheduled for "
+            + "cleanup, no further processing");
+        continue;
+      }
+
+      ApplicationId containerAppId =
+          containerId.getApplicationAttemptId().getApplicationId();
+
+      if (finishedApplications.contains(containerAppId)) {
+        LOG.info("Container " + containerId
+            + " belongs to an application that is already killed,"
+            + " no further processing");
+        continue;
+      }
+      
+      this.nmReportedIncreasedContainers.put(containerId, container);
+    }
+  }
 
   private void handleContainerStatus(List<ContainerStatus> containerStatuses) {
     // Filter the map to only obtain just launched containers and finished
@@ -989,4 +1064,22 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     }
   }
 
+  @Override
+  public List<Container> pullNewlyIncreasedContainers() {
+    try {
+      writeLock.lock();
+
+      if (nmReportedIncreasedContainers.isEmpty()) {
+        return Collections.EMPTY_LIST;
+      } else {
+        List<Container> container =
+            new ArrayList<Container>(nmReportedIncreasedContainers.values());
+        nmReportedIncreasedContainers.clear();
+        return container;
+      }
+      
+    } finally {
+      writeLock.unlock();
+    }
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.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/rmnode/RMNodeStatusEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
index b95d7d3..8323f3c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
@@ -18,8 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
 
+import java.util.Collections;
 import java.util.List;
+
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
@@ -33,28 +36,36 @@ public class RMNodeStatusEvent extends RMNodeEvent {
   private final NodeHeartbeatResponse latestResponse;
   private final List<ApplicationId> keepAliveAppIds;
   private List<LogAggregationReport> logAggregationReportsForApps;
-
+  private final List<Container> nmReportedIncreasedContainers;
+  
+  // Used by tests
   public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
       List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
       NodeHeartbeatResponse latestResponse) {
-    super(nodeId, RMNodeEventType.STATUS_UPDATE);
-    this.nodeHealthStatus = nodeHealthStatus;
-    this.containersCollection = collection;
-    this.keepAliveAppIds = keepAliveAppIds;
-    this.latestResponse = latestResponse;
-    this.logAggregationReportsForApps = null;
+    this(nodeId, nodeHealthStatus, collection, keepAliveAppIds,
+        latestResponse, null);
   }
 
   public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
       List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
       NodeHeartbeatResponse latestResponse,
-      List<LogAggregationReport> logAggregationReportsForApps) {
+      List<Container> nmReportedIncreasedContainers) {
+    this(nodeId, nodeHealthStatus, collection, keepAliveAppIds, latestResponse,
+        null, nmReportedIncreasedContainers);
+  }
+
+  public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
+      List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
+      NodeHeartbeatResponse latestResponse,
+      List<LogAggregationReport> logAggregationReportsForApps,
+      List<Container> nmReportedIncreasedContainers) {
     super(nodeId, RMNodeEventType.STATUS_UPDATE);
     this.nodeHealthStatus = nodeHealthStatus;
     this.containersCollection = collection;
     this.keepAliveAppIds = keepAliveAppIds;
     this.latestResponse = latestResponse;
     this.logAggregationReportsForApps = logAggregationReportsForApps;
+    this.nmReportedIncreasedContainers = nmReportedIncreasedContainers;
   }
 
   public NodeHealthStatus getNodeHealthStatus() {
@@ -81,4 +92,9 @@ public class RMNodeStatusEvent extends RMNodeEvent {
       List<LogAggregationReport> logAggregationReportsForApps) {
     this.logAggregationReportsForApps = logAggregationReportsForApps;
   }
+  
+  public List<Container> getNMReportedIncreasedContainers() {
+    return nmReportedIncreasedContainers == null ? Collections.EMPTY_LIST
+        : nmReportedIncreasedContainers;
+  }
 }
\ No newline at end of file


[3/5] hadoop git commit: YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 b43f658..15d7c32 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
@@ -22,7 +22,6 @@ 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;
@@ -60,10 +59,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 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;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 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.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.Lock;
@@ -730,17 +729,22 @@ public class LeafQueue extends AbstractCSQueue {
   }
   
   private void handleExcessReservedContainer(Resource clusterResource,
-      CSAssignment assignment) {
+      CSAssignment assignment, FiCaSchedulerNode node, FiCaSchedulerApp app) {
     if (assignment.getExcessReservation() != null) {
       RMContainer excessReservedContainer = assignment.getExcessReservation();
-
-      completedContainer(clusterResource, assignment.getApplication(),
-          scheduler.getNode(excessReservedContainer.getAllocatedNode()),
-          excessReservedContainer,
-          SchedulerUtils.createAbnormalContainerStatus(
-              excessReservedContainer.getContainerId(),
-              SchedulerUtils.UNRESERVED_CONTAINER),
-          RMContainerEventType.RELEASED, null, false);
+      
+      if (excessReservedContainer.hasIncreaseReservation()) {
+        unreserveIncreasedContainer(clusterResource,
+            app, node, excessReservedContainer);
+      } else {
+        completedContainer(clusterResource, assignment.getApplication(),
+            scheduler.getNode(excessReservedContainer.getAllocatedNode()),
+            excessReservedContainer,
+            SchedulerUtils.createAbnormalContainerStatus(
+                excessReservedContainer.getContainerId(),
+                SchedulerUtils.UNRESERVED_CONTAINER),
+            RMContainerEventType.RELEASED, null, false);
+      }
 
       assignment.setExcessReservation(null);
     }
@@ -766,7 +770,8 @@ public class LeafQueue extends AbstractCSQueue {
         CSAssignment assignment =
             application.assignContainers(clusterResource, node,
                 currentResourceLimits, schedulingMode, reservedContainer);
-        handleExcessReservedContainer(clusterResource, assignment);
+        handleExcessReservedContainer(clusterResource, assignment, node,
+            application);
         return assignment;
       }
     }
@@ -824,7 +829,8 @@ public class LeafQueue extends AbstractCSQueue {
       // Did we schedule or reserve a container?
       Resource assigned = assignment.getResource();
       
-      handleExcessReservedContainer(clusterResource, assignment);
+      handleExcessReservedContainer(clusterResource, assignment, node,
+          application);
 
       if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
           Resources.none())) {
@@ -836,7 +842,8 @@ public class LeafQueue extends AbstractCSQueue {
         // Book-keeping
         // Note: Update headroom to account for current allocation too...
         allocateResource(clusterResource, application, assigned,
-            node.getPartition(), reservedOrAllocatedRMContainer);
+            node.getPartition(), reservedOrAllocatedRMContainer,
+            assignment.isIncreasedAllocation());
 
         // Done
         return assignment;
@@ -1086,6 +1093,37 @@ public class LeafQueue extends AbstractCSQueue {
     }
     return true;
   }
+  
+  @Override
+  public void unreserveIncreasedContainer(Resource clusterResource,
+      FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer) {
+    boolean removed = false;
+    Priority priority = null;
+    
+    synchronized (this) {
+      if (rmContainer.getContainer() != null) {
+        priority = rmContainer.getContainer().getPriority();
+      }
+
+      if (null != priority) {
+        removed = app.unreserve(rmContainer.getContainer().getPriority(), node,
+            rmContainer);
+      }
+
+      if (removed) {
+        // Inform the ordering policy
+        orderingPolicy.containerReleased(app, rmContainer);
+
+        releaseResource(clusterResource, app, rmContainer.getReservedResource(),
+            node.getPartition(), rmContainer, true);
+      }
+    }
+    
+    if (removed) {
+      getParent().unreserveIncreasedContainer(clusterResource, app, node,
+          rmContainer);
+    }
+  }
 
   @Override
   public void completedContainer(Resource clusterResource, 
@@ -1093,6 +1131,15 @@ public class LeafQueue extends AbstractCSQueue {
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue,
       boolean sortQueues) {
     if (application != null) {
+      // unreserve container increase request if it previously reserved.
+      if (rmContainer.hasIncreaseReservation()) {
+        unreserveIncreasedContainer(clusterResource, application, node,
+            rmContainer);
+      }
+      
+      // Remove container increase request if it exists
+      application.removeIncreaseRequest(node.getNodeID(),
+          rmContainer.getAllocatedPriority(), rmContainer.getContainerId());
 
       boolean removed = false;
 
@@ -1123,7 +1170,7 @@ public class LeafQueue extends AbstractCSQueue {
           orderingPolicy.containerReleased(application, rmContainer);
           
           releaseResource(clusterResource, application, container.getResource(),
-              node.getPartition(), rmContainer);
+              node.getPartition(), rmContainer, false);
         }
       }
 
@@ -1137,8 +1184,10 @@ public class LeafQueue extends AbstractCSQueue {
 
   synchronized void allocateResource(Resource clusterResource,
       SchedulerApplicationAttempt application, Resource resource,
-      String nodePartition, RMContainer rmContainer) {
-    super.allocateResource(clusterResource, resource, nodePartition);
+      String nodePartition, RMContainer rmContainer,
+      boolean isIncreasedAllocation) {
+    super.allocateResource(clusterResource, resource, nodePartition,
+        isIncreasedAllocation);
     
     // handle ignore exclusivity container
     if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
@@ -1174,8 +1223,9 @@ public class LeafQueue extends AbstractCSQueue {
 
   synchronized void releaseResource(Resource clusterResource,
       FiCaSchedulerApp application, Resource resource, String nodePartition,
-      RMContainer rmContainer) {
-    super.releaseResource(clusterResource, resource, nodePartition);
+      RMContainer rmContainer, boolean isChangeResource) {
+    super.releaseResource(clusterResource, resource, nodePartition,
+        isChangeResource);
     
     // handle ignore exclusivity container
     if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
@@ -1363,7 +1413,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       allocateResource(clusterResource, attempt, rmContainer.getContainer()
-          .getResource(), node.getPartition(), rmContainer);
+          .getResource(), node.getPartition(), rmContainer, false);
     }
     getParent().recoverContainer(clusterResource, attempt, rmContainer);
   }
@@ -1412,7 +1462,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       allocateResource(clusterResource, application, rmContainer.getContainer()
-          .getResource(), node.getPartition(), rmContainer);
+          .getResource(), node.getPartition(), rmContainer, false);
       LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
           + " resource=" + rmContainer.getContainer().getResource()
           + " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity()
@@ -1430,7 +1480,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       releaseResource(clusterResource, application, rmContainer.getContainer()
-          .getResource(), node.getPartition(), rmContainer);
+          .getResource(), node.getPartition(), rmContainer, false);
       LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
           + " resource=" + rmContainer.getContainer().getResource()
           + " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity()
@@ -1482,6 +1532,39 @@ public class LeafQueue extends AbstractCSQueue {
   public Priority getDefaultApplicationPriority() {
     return defaultAppPriorityPerQueue;
   }
+  
+  @Override
+  public void decreaseContainer(Resource clusterResource,
+      SchedContainerChangeRequest decreaseRequest,
+      FiCaSchedulerApp app) {
+    // If the container being decreased is reserved, we need to unreserve it
+    // first.
+    RMContainer rmContainer = decreaseRequest.getRMContainer();
+    if (rmContainer.hasIncreaseReservation()) {
+      unreserveIncreasedContainer(clusterResource, app,
+          (FiCaSchedulerNode)decreaseRequest.getSchedulerNode(), rmContainer);
+    }
+    
+    // Delta capacity is negative when it's a decrease request
+    Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity());
+    
+    synchronized (this) {
+      // Delta is negative when it's a decrease request
+      releaseResource(clusterResource, app, absDelta,
+          decreaseRequest.getNodePartition(), decreaseRequest.getRMContainer(),
+          true);
+      // Notify application
+      app.decreaseContainer(decreaseRequest);
+      // Notify node
+      decreaseRequest.getSchedulerNode()
+          .decreaseContainer(decreaseRequest.getContainerId(), absDelta);
+    }
+
+    // Notify parent
+    if (getParent() != null) {
+      getParent().decreaseContainer(clusterResource, decreaseRequest, app);
+    }
+  }
 
   public synchronized OrderingPolicy<FiCaSchedulerApp>
       getPendingAppsOrderingPolicy() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 e01204c..badab72 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
@@ -57,6 +57,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.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.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -430,7 +431,7 @@ public class ParentQueue extends AbstractCSQueue {
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
         super.allocateResource(clusterResource, assignedToChild.getResource(),
-            node.getPartition());
+            node.getPartition(), assignedToChild.isIncreasedAllocation());
         
         // Track resource utilization in this pass of the scheduler
         Resources
@@ -454,6 +455,8 @@ public class ParentQueue extends AbstractCSQueue {
           .addAll(
               assignedToChild.getAssignmentInformation()
                   .getReservationDetails());
+        assignment.setIncreasedAllocation(assignedToChild
+            .isIncreasedAllocation());
         
         LOG.info("assignedContainer" +
             " queue=" + getQueueName() + 
@@ -616,6 +619,73 @@ public class ParentQueue extends AbstractCSQueue {
     }
   }
   
+  private synchronized void internalReleaseResource(Resource clusterResource,
+      FiCaSchedulerNode node, Resource releasedResource, boolean changeResource,
+      CSQueue completedChildQueue, boolean sortQueues) {
+    super.releaseResource(clusterResource,
+        releasedResource, node.getPartition(),
+        changeResource);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("completedContainer " + this + ", cluster=" + clusterResource);
+    }
+
+    // Note that this is using an iterator on the childQueues so this can't
+    // be called if already within an iterator for the childQueues. Like
+    // from assignContainersToChildQueues.
+    if (sortQueues) {
+      // reinsert the updated queue
+      for (Iterator<CSQueue> iter = childQueues.iterator(); iter.hasNext();) {
+        CSQueue csqueue = iter.next();
+        if (csqueue.equals(completedChildQueue)) {
+          iter.remove();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Re-sorting completed queue: " + csqueue);
+          }
+          childQueues.add(csqueue);
+          break;
+        }
+      }
+    }
+
+    // If we skipped sort queue this time, we need to resort queues to make
+    // sure we allocate from least usage (or order defined by queue policy)
+    // queues.
+    needToResortQueuesAtNextAllocation = !sortQueues;
+  }
+  
+  @Override
+  public void decreaseContainer(Resource clusterResource,
+      SchedContainerChangeRequest decreaseRequest, FiCaSchedulerApp app) {
+    // delta capacity is negative when it's a decrease request
+    Resource absDeltaCapacity =
+        Resources.negate(decreaseRequest.getDeltaCapacity());
+
+    internalReleaseResource(clusterResource,
+        csContext.getNode(decreaseRequest.getNodeId()), absDeltaCapacity, false,
+        null, false);
+
+    // Inform the parent
+    if (parent != null) {
+      parent.decreaseContainer(clusterResource, decreaseRequest, app);
+    }
+  }
+  
+  @Override
+  public void unreserveIncreasedContainer(Resource clusterResource,
+      FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer) {
+    if (app != null) {
+      internalReleaseResource(clusterResource, node,
+          rmContainer.getReservedResource(), false, null, false);
+
+      // Inform the parent
+      if (parent != null) {
+        parent.unreserveIncreasedContainer(clusterResource, app, node,
+            rmContainer);
+      }    
+    }
+  }
+
   @Override
   public void completedContainer(Resource clusterResource,
       FiCaSchedulerApp application, FiCaSchedulerNode node, 
@@ -623,40 +693,9 @@ public class ParentQueue extends AbstractCSQueue {
       RMContainerEventType event, CSQueue completedChildQueue,
       boolean sortQueues) {
     if (application != null) {
-      // Careful! Locking order is important!
-      // Book keeping
-      synchronized (this) {
-        super.releaseResource(clusterResource, rmContainer.getContainer()
-            .getResource(), node.getPartition());
-
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("completedContainer " + this + ", cluster=" + clusterResource);
-        }
-
-        // Note that this is using an iterator on the childQueues so this can't
-        // be called if already within an iterator for the childQueues. Like
-        // from assignContainersToChildQueues.
-        if (sortQueues) {
-          // reinsert the updated queue
-          for (Iterator<CSQueue> iter = childQueues.iterator();
-               iter.hasNext();) {
-            CSQueue csqueue = iter.next();
-            if(csqueue.equals(completedChildQueue)) {
-              iter.remove();
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Re-sorting completed queue: " + csqueue);
-              }
-              childQueues.add(csqueue);
-              break;
-            }
-          }
-        }
-        
-        // If we skipped sort queue this time, we need to resort queues to make
-        // sure we allocate from least usage (or order defined by queue policy)
-        // queues.
-        needToResortQueuesAtNextAllocation = !sortQueues;
-      }
+      internalReleaseResource(clusterResource, node,
+          rmContainer.getContainer().getResource(), false, completedChildQueue,
+          sortQueues);
 
       // Inform the parent
       if (parent != null) {
@@ -698,7 +737,7 @@ public class ParentQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getPartition());
+          .getResource(), node.getPartition(), false);
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -726,7 +765,7 @@ public class ParentQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getPartition());
+          .getResource(), node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster="
@@ -746,7 +785,7 @@ public class ParentQueue extends AbstractCSQueue {
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       super.releaseResource(clusterResource,
           rmContainer.getContainer().getResource(),
-          node.getPartition());
+          node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveOut=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster="

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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
new file mode 100644
index 0000000..b986b1f
--- /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/allocator/AbstractContainerAllocator.java
@@ -0,0 +1,131 @@
+/**
+ * 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.allocator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+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.capacity.CSAssignment;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+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.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * For an application, resource limits and resource requests, decide how to
+ * allocate container. This is to make application resource allocation logic
+ * extensible.
+ */
+public abstract class AbstractContainerAllocator {
+  private static final Log LOG = LogFactory.getLog(AbstractContainerAllocator.class);
+
+  FiCaSchedulerApp application;
+  final ResourceCalculator rc;
+  final RMContext rmContext;
+  
+  public AbstractContainerAllocator(FiCaSchedulerApp application,
+      ResourceCalculator rc, RMContext rmContext) {
+    this.application = application;
+    this.rc = rc;
+    this.rmContext = rmContext;
+  }
+
+  protected CSAssignment getCSAssignmentFromAllocateResult(
+      Resource clusterResource, ContainerAllocation result,
+      RMContainer rmContainer) {
+    // Handle skipped
+    boolean skipped =
+        (result.getAllocationState() == AllocationState.APP_SKIPPED);
+    CSAssignment assignment = new CSAssignment(skipped);
+    assignment.setApplication(application);
+    
+    // Handle excess reservation
+    assignment.setExcessReservation(result.getContainerToBeUnreserved());
+
+    // If we allocated something
+    if (Resources.greaterThan(rc, clusterResource,
+        result.getResourceToBeAllocated(), Resources.none())) {
+      Resource allocatedResource = result.getResourceToBeAllocated();
+      Container updatedContainer = result.getUpdatedContainer();
+
+      assignment.setResource(allocatedResource);
+      assignment.setType(result.getContainerNodeType());
+
+      if (result.getAllocationState() == AllocationState.RESERVED) {
+        // This is a reserved container
+        LOG.info("Reserved container " + " application="
+            + application.getApplicationId() + " resource=" + allocatedResource
+            + " queue=" + this.toString() + " cluster=" + clusterResource);
+        assignment.getAssignmentInformation().addReservationDetails(
+            updatedContainer.getId(),
+            application.getCSLeafQueue().getQueuePath());
+        assignment.getAssignmentInformation().incrReservations();
+        Resources.addTo(assignment.getAssignmentInformation().getReserved(),
+            allocatedResource);
+      } else if (result.getAllocationState() == AllocationState.ALLOCATED){
+        // This is a new container
+        // Inform the ordering policy
+        LOG.info("assignedContainer" + " application attempt="
+            + application.getApplicationAttemptId() + " container="
+            + updatedContainer.getId() + " queue=" + this + " clusterResource="
+            + clusterResource);
+
+        application
+            .getCSLeafQueue()
+            .getOrderingPolicy()
+            .containerAllocated(application,
+                application.getRMContainer(updatedContainer.getId()));
+
+        assignment.getAssignmentInformation().addAllocationDetails(
+            updatedContainer.getId(),
+            application.getCSLeafQueue().getQueuePath());
+        assignment.getAssignmentInformation().incrAllocations();
+        Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
+            allocatedResource);
+        
+        if (rmContainer != null) {
+          assignment.setFulfilledReservation(true);
+        }
+      }
+    }
+    
+    return assignment;
+  }
+  
+  /**
+   * allocate needs to handle following stuffs:
+   * 
+   * <ul>
+   * <li>Select request: Select a request to allocate. E.g. select a resource
+   * request based on requirement/priority/locality.</li>
+   * <li>Check if a given resource can be allocated based on resource
+   * availability</li>
+   * <li>Do allocation: this will decide/create allocated/reserved
+   * container, this will also update metrics</li>
+   * </ul>
+   */
+  public abstract CSAssignment assignContainers(Resource clusterResource,
+      FiCaSchedulerNode node, SchedulingMode schedulingMode,
+      ResourceLimits resourceLimits, RMContainer reservedContainer);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.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/ContainerAllocator.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/ContainerAllocator.java
index 6e296cd..3be8e0e 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/ContainerAllocator.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/ContainerAllocator.java
@@ -18,13 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-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.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -33,118 +30,50 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-/**
- * For an application, resource limits and resource requests, decide how to
- * allocate container. This is to make application resource allocation logic
- * extensible.
- */
-public abstract class ContainerAllocator {
-  private static final Log LOG = LogFactory.getLog(ContainerAllocator.class);
+public class ContainerAllocator extends AbstractContainerAllocator {
+  AbstractContainerAllocator increaseContainerAllocator;
+  AbstractContainerAllocator regularContainerAllocator;
 
-  FiCaSchedulerApp application;
-  final ResourceCalculator rc;
-  final RMContext rmContext;
-  
   public ContainerAllocator(FiCaSchedulerApp application,
       ResourceCalculator rc, RMContext rmContext) {
-    this.application = application;
-    this.rc = rc;
-    this.rmContext = rmContext;
-  }
+    super(application, rc, rmContext);
 
-  protected boolean checkHeadroom(Resource clusterResource,
-      ResourceLimits currentResourceLimits, Resource required,
-      FiCaSchedulerNode node) {
-    // If headroom + currentReservation < required, we cannot allocate this
-    // require
-    Resource resourceCouldBeUnReserved = application.getCurrentReservation();
-    if (!application.getCSLeafQueue().getReservationContinueLooking()
-        || !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) {
-      // If we don't allow reservation continuous looking, OR we're looking at
-      // non-default node partition, we won't allow to unreserve before
-      // allocation.
-      resourceCouldBeUnReserved = Resources.none();
-    }
-    return Resources.greaterThanOrEqual(rc, clusterResource, Resources.add(
-        currentResourceLimits.getHeadroom(), resourceCouldBeUnReserved),
-        required);
+    increaseContainerAllocator =
+        new IncreaseContainerAllocator(application, rc, rmContext);
+    regularContainerAllocator =
+        new RegularContainerAllocator(application, rc, rmContext);
   }
 
-  protected CSAssignment getCSAssignmentFromAllocateResult(
-      Resource clusterResource, ContainerAllocation result,
-      RMContainer rmContainer) {
-    // Handle skipped
-    boolean skipped =
-        (result.getAllocationState() == AllocationState.APP_SKIPPED);
-    CSAssignment assignment = new CSAssignment(skipped);
-    assignment.setApplication(application);
-    
-    // Handle excess reservation
-    assignment.setExcessReservation(result.getContainerToBeUnreserved());
-
-    // If we allocated something
-    if (Resources.greaterThan(rc, clusterResource,
-        result.getResourceToBeAllocated(), Resources.none())) {
-      Resource allocatedResource = result.getResourceToBeAllocated();
-      Container updatedContainer = result.getUpdatedContainer();
-
-      assignment.setResource(allocatedResource);
-      assignment.setType(result.getContainerNodeType());
-
-      if (result.getAllocationState() == AllocationState.RESERVED) {
-        // This is a reserved container
-        LOG.info("Reserved container " + " application="
-            + application.getApplicationId() + " resource=" + allocatedResource
-            + " queue=" + this.toString() + " cluster=" + clusterResource);
-        assignment.getAssignmentInformation().addReservationDetails(
-            updatedContainer.getId(),
-            application.getCSLeafQueue().getQueuePath());
-        assignment.getAssignmentInformation().incrReservations();
-        Resources.addTo(assignment.getAssignmentInformation().getReserved(),
-            allocatedResource);
-      } else if (result.getAllocationState() == AllocationState.ALLOCATED){
-        // This is a new container
-        // Inform the ordering policy
-        LOG.info("assignedContainer" + " application attempt="
-            + application.getApplicationAttemptId() + " container="
-            + updatedContainer.getId() + " queue=" + this + " clusterResource="
-            + clusterResource);
-
-        application
-            .getCSLeafQueue()
-            .getOrderingPolicy()
-            .containerAllocated(application,
-                application.getRMContainer(updatedContainer.getId()));
-
-        assignment.getAssignmentInformation().addAllocationDetails(
-            updatedContainer.getId(),
-            application.getCSLeafQueue().getQueuePath());
-        assignment.getAssignmentInformation().incrAllocations();
-        Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
-            allocatedResource);
-        
-        if (rmContainer != null) {
-          assignment.setFulfilledReservation(true);
-        }
+  @Override
+  public CSAssignment assignContainers(Resource clusterResource,
+      FiCaSchedulerNode node, SchedulingMode schedulingMode,
+      ResourceLimits resourceLimits, RMContainer reservedContainer) {
+    if (reservedContainer != null) {
+      if (reservedContainer.getState() == RMContainerState.RESERVED) {
+        // It's a regular container
+        return regularContainerAllocator.assignContainers(clusterResource,
+            node, schedulingMode, resourceLimits, reservedContainer);
+      } else {
+        // It's a increase container
+        return increaseContainerAllocator.assignContainers(clusterResource,
+            node, schedulingMode, resourceLimits, reservedContainer);
+      }
+    } else {
+      /*
+       * Try to allocate increase container first, and if we failed to allocate
+       * anything, we will try to allocate regular container
+       */
+      CSAssignment assign =
+          increaseContainerAllocator.assignContainers(clusterResource, node,
+              schedulingMode, resourceLimits, null);
+      if (Resources.greaterThan(rc, clusterResource, assign.getResource(),
+          Resources.none())) {
+        return assign;
       }
+
+      return regularContainerAllocator.assignContainers(clusterResource, node,
+          schedulingMode, resourceLimits, null);
     }
-    
-    return assignment;
   }
-  
-  /**
-   * allocate needs to handle following stuffs:
-   * 
-   * <ul>
-   * <li>Select request: Select a request to allocate. E.g. select a resource
-   * request based on requirement/priority/locality.</li>
-   * <li>Check if a given resource can be allocated based on resource
-   * availability</li>
-   * <li>Do allocation: this will decide/create allocated/reserved
-   * container, this will also update metrics</li>
-   * </ul>
-   */
-  public abstract CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, SchedulingMode schedulingMode,
-      ResourceLimits resourceLimits, RMContainer reservedContainer);
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.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/IncreaseContainerAllocator.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/IncreaseContainerAllocator.java
new file mode 100644
index 0000000..9350adc
--- /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/allocator/IncreaseContainerAllocator.java
@@ -0,0 +1,365 @@
+/**
+ * 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.allocator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+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.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
+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.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+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.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+public class IncreaseContainerAllocator extends AbstractContainerAllocator {
+  private static final Log LOG =
+      LogFactory.getLog(IncreaseContainerAllocator.class);
+
+  public IncreaseContainerAllocator(FiCaSchedulerApp application,
+      ResourceCalculator rc, RMContext rmContext) {
+    super(application, rc, rmContext);
+  }
+  
+  /**
+   * Quick check if we can allocate anything here:
+   * We will not continue if: 
+   * - Headroom doesn't support allocate minimumAllocation
+   * - 
+   */
+  private boolean checkHeadroom(Resource clusterResource,
+      ResourceLimits currentResourceLimits, Resource required) {
+    return Resources.greaterThanOrEqual(rc, clusterResource,
+        currentResourceLimits.getHeadroom(), required);
+  }
+  
+  private CSAssignment createReservedIncreasedCSAssignment(
+      SchedContainerChangeRequest request) {
+    CSAssignment assignment =
+        new CSAssignment(request.getDeltaCapacity(), NodeType.NODE_LOCAL, null,
+            application, false, false);
+    Resources.addTo(assignment.getAssignmentInformation().getReserved(),
+        request.getDeltaCapacity());
+    assignment.getAssignmentInformation().incrReservations();
+    assignment.getAssignmentInformation().addReservationDetails(
+        request.getContainerId(), application.getCSLeafQueue().getQueuePath());
+    assignment.setIncreasedAllocation(true);
+    
+    LOG.info("Reserved increase container request:" + request.toString());
+    
+    return assignment;
+  }
+  
+  private CSAssignment createSuccessfullyIncreasedCSAssignment(
+      SchedContainerChangeRequest request, boolean fromReservation) {
+    CSAssignment assignment =
+        new CSAssignment(request.getDeltaCapacity(), NodeType.NODE_LOCAL, null,
+            application, false, fromReservation);
+    Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
+        request.getDeltaCapacity());
+    assignment.getAssignmentInformation().incrAllocations();
+    assignment.getAssignmentInformation().addAllocationDetails(
+        request.getContainerId(), application.getCSLeafQueue().getQueuePath());
+    assignment.setIncreasedAllocation(true);
+    
+    // notify application
+    application
+        .getCSLeafQueue()
+        .getOrderingPolicy()
+        .containerAllocated(application,
+            application.getRMContainer(request.getContainerId()));
+
+    LOG.info("Approved increase container request:" + request.toString()
+        + " fromReservation=" + fromReservation);    
+    
+    return assignment;
+  }
+  
+  private CSAssignment allocateIncreaseRequestFromReservedContainer(
+      SchedulerNode node, Resource cluster,
+      SchedContainerChangeRequest increaseRequest) {
+    if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(),
+        node.getAvailableResource())) {
+      // OK, we can allocate this increase request
+      // Unreserve it first
+      application.unreserve(increaseRequest.getPriority(),
+          (FiCaSchedulerNode) node, increaseRequest.getRMContainer());
+      
+      // Notify application
+      application.increaseContainer(increaseRequest);
+      
+      // Notify node
+      node.increaseContainer(increaseRequest.getContainerId(),
+          increaseRequest.getDeltaCapacity());
+
+      return createSuccessfullyIncreasedCSAssignment(increaseRequest, true);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to allocate reserved increase request:"
+            + increaseRequest.toString()
+            + ". There's no enough available resource");
+      }
+      
+      // We still cannot allocate this container, will wait for next turn
+      return CSAssignment.SKIP_ASSIGNMENT;
+    }
+  }
+  
+  private CSAssignment allocateIncreaseRequest(FiCaSchedulerNode node,
+      Resource cluster, SchedContainerChangeRequest increaseRequest) {
+    if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(),
+        node.getAvailableResource())) {
+      // Notify node
+      node.increaseContainer(increaseRequest.getContainerId(),
+          increaseRequest.getDeltaCapacity());
+
+      // OK, we can allocate this increase request
+      // Notify application
+      application.increaseContainer(increaseRequest);
+      return createSuccessfullyIncreasedCSAssignment(increaseRequest, false);
+    } else {
+      boolean reservationSucceeded =
+          application.reserveIncreasedContainer(increaseRequest.getPriority(),
+              node, increaseRequest.getRMContainer(),
+              increaseRequest.getDeltaCapacity());
+      
+      if (reservationSucceeded) {
+        // We cannot allocate this container, but since queue capacity /
+        // user-limit matches, we can reserve this container on this node.
+        return createReservedIncreasedCSAssignment(increaseRequest);
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Reserve increase request=" + increaseRequest.toString()
+              + " failed. Skipping..");
+        }
+        return CSAssignment.SKIP_ASSIGNMENT;
+      }
+    }
+  }
+
+  @Override
+  public CSAssignment assignContainers(Resource clusterResource,
+      FiCaSchedulerNode node, SchedulingMode schedulingMode,
+      ResourceLimits resourceLimits, RMContainer reservedContainer) {
+    AppSchedulingInfo sinfo = application.getAppSchedulingInfo();
+    NodeId nodeId = node.getNodeID();
+
+    if (reservedContainer == null) {
+      // Do we have increase request on this node?
+      if (!sinfo.hasIncreaseRequest(nodeId)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skip allocating increase request since we don't have any"
+              + " increase request on this node=" + node.getNodeID());
+        }
+        
+        return CSAssignment.SKIP_ASSIGNMENT;
+      }
+      
+      // Check if we need to unreserve something, note that we don't support
+      // continuousReservationLooking now. TODO, need think more about how to
+      // support it.
+      boolean shouldUnreserve =
+          Resources.greaterThan(rc, clusterResource,
+              resourceLimits.getAmountNeededUnreserve(), Resources.none());
+      
+      // Check if we can allocate minimum resource according to headroom
+      boolean cannotAllocateAnything =
+          !checkHeadroom(clusterResource, resourceLimits, rmContext
+              .getScheduler().getMinimumResourceCapability());
+      
+      // Skip the app if we failed either of above check
+      if (cannotAllocateAnything || shouldUnreserve) {
+        if (LOG.isDebugEnabled()) {
+          if (shouldUnreserve) {
+            LOG.debug("Cannot continue since we have to unreserve some resource"
+                + ", now increase container allocation doesn't "
+                + "support continuous reservation looking..");
+          }
+          if (cannotAllocateAnything) {
+            LOG.debug("We cannot allocate anything because of low headroom, "
+                + "headroom=" + resourceLimits.getHeadroom());
+          }
+        }
+        
+        return CSAssignment.SKIP_ASSIGNMENT;
+      }
+      
+      CSAssignment assigned = null;
+
+      /*
+       * Loop each priority, and containerId. Container priority is not
+       * equivalent to request priority, application master can run an important
+       * task on a less prioritized container.
+       * 
+       * So behavior here is, we still try to increase container with higher
+       * priority, but will skip increase request and move to next increase
+       * request if queue-limit or user-limit aren't satisfied 
+       */
+      for (Priority priority : application.getPriorities()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Looking at increase request for application="
+              + application.getApplicationAttemptId() + " priority="
+              + priority);
+        }
+
+        /*
+         * If we have multiple to-be-increased containers under same priority on
+         * a same host, we will try to increase earlier launched container
+         * first. And again - we will skip a request and move to next if it
+         * cannot be allocated.
+         */
+        Map<ContainerId, SchedContainerChangeRequest> increaseRequestMap =
+            sinfo.getIncreaseRequests(nodeId, priority);
+
+        // We don't have more increase request on this priority, skip..
+        if (null == increaseRequestMap) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("There's no increase request for "
+                + application.getApplicationAttemptId() + " priority="
+                + priority);
+          }
+          continue;
+        }
+        Iterator<Entry<ContainerId, SchedContainerChangeRequest>> iter =
+            increaseRequestMap.entrySet().iterator();
+        List<SchedContainerChangeRequest> toBeRemovedRequests =
+            new ArrayList<>();
+
+        while (iter.hasNext()) {
+          Entry<ContainerId, SchedContainerChangeRequest> entry =
+              iter.next();
+          SchedContainerChangeRequest increaseRequest =
+              entry.getValue();
+          
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Looking at increase request=" + increaseRequest.toString());
+          }
+
+          boolean headroomSatisifed = checkHeadroom(clusterResource,
+              resourceLimits, increaseRequest.getDeltaCapacity());
+          if (!headroomSatisifed) {
+            // skip if doesn't satisfy headroom limit
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(" Headroom is not satisfied, skip..");
+            }
+            continue;
+          }
+
+          RMContainer rmContainer = increaseRequest.getRMContainer();
+          if (rmContainer.getContainerState() != ContainerState.RUNNING) {
+            // if the container is not running, we should remove the
+            // increaseRequest and continue;
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("  Container is not running any more, skip...");
+            }
+            toBeRemovedRequests.add(increaseRequest);
+            continue;
+          }
+
+          if (!Resources.fitsIn(rc, clusterResource,
+              increaseRequest.getTargetCapacity(), node.getTotalResource())) {
+            // if the target capacity is more than what the node can offer, we
+            // will simply remove and skip it.
+            // The reason of doing check here instead of adding increase request
+            // to scheduler because node's resource could be updated after
+            // request added.
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("  Target capacity is more than what node can offer,"
+                  + " node.resource=" + node.getTotalResource());
+            }
+            toBeRemovedRequests.add(increaseRequest);
+            continue;
+          }
+
+          // Try to allocate the increase request
+          assigned =
+              allocateIncreaseRequest(node, clusterResource, increaseRequest);
+          if (!assigned.getSkipped()) {
+            // When we don't skip this request, which means we either allocated
+            // OR reserved this request. We will break
+            break;
+          }
+        }
+        
+        // Remove invalid in request requests
+        if (!toBeRemovedRequests.isEmpty()) {
+          for (SchedContainerChangeRequest req : toBeRemovedRequests) {
+            sinfo.removeIncreaseRequest(req.getNodeId(), req.getPriority(),
+                req.getContainerId());
+          }
+        }
+
+        // We already allocated something
+        if (!assigned.getSkipped()) {
+          break;
+        }
+      }
+      
+      return assigned == null ? CSAssignment.SKIP_ASSIGNMENT : assigned;
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Trying to allocate reserved increase container request..");
+      }
+      
+      // We already reserved this increase container
+      SchedContainerChangeRequest request =
+          sinfo.getIncreaseRequest(nodeId, reservedContainer.getContainer()
+              .getPriority(), reservedContainer.getContainerId());
+      
+      // We will cancel the reservation any of following happens
+      // - Container finished
+      // - No increase request needed
+      // - Target resource updated
+      if (null == request
+          || reservedContainer.getContainerState() != ContainerState.RUNNING
+          || (!Resources.equals(reservedContainer.getReservedResource(),
+              request.getDeltaCapacity()))) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("We don't need reserved increase container request "
+              + "for container=" + reservedContainer.getContainerId()
+              + ". Unreserving and return...");
+        }
+        
+        // We don't need this container now, just return excessive reservation
+        return new CSAssignment(application, reservedContainer);
+      }
+      
+      return allocateIncreaseRequestFromReservedContainer(node, clusterResource,
+          request);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 dcb99ed..fd99d29 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
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
  * Allocate normal (new) containers, considers locality/label, etc. Using
  * delayed scheduling mechanism to get better locality allocation.
  */
-public class RegularContainerAllocator extends ContainerAllocator {
+public class RegularContainerAllocator extends AbstractContainerAllocator {
   private static final Log LOG = LogFactory.getLog(RegularContainerAllocator.class);
   
   private ResourceRequest lastResourceRequest = null;
@@ -56,6 +56,25 @@ public class RegularContainerAllocator extends ContainerAllocator {
     super(application, rc, rmContext);
   }
   
+  private boolean checkHeadroom(Resource clusterResource,
+      ResourceLimits currentResourceLimits, Resource required,
+      FiCaSchedulerNode node) {
+    // If headroom + currentReservation < required, we cannot allocate this
+    // require
+    Resource resourceCouldBeUnReserved = application.getCurrentReservation();
+    if (!application.getCSLeafQueue().getReservationContinueLooking()
+        || !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) {
+      // If we don't allow reservation continuous looking, OR we're looking at
+      // non-default node partition, we won't allow to unreserve before
+      // allocation.
+      resourceCouldBeUnReserved = Resources.none();
+    }
+    return Resources.greaterThanOrEqual(rc, clusterResource, Resources.add(
+        currentResourceLimits.getHeadroom(), resourceCouldBeUnReserved),
+        required);
+  }
+
+  
   private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
       ResourceLimits resourceLimits, Priority priority) {
@@ -97,8 +116,9 @@ public class RegularContainerAllocator extends ContainerAllocator {
     // Is the node-label-expression of this offswitch resource request
     // matches the node's label?
     // If not match, jump to next priority.
-    if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(anyRequest,
-        node.getPartition(), schedulingMode)) {
+    if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
+        anyRequest.getNodeLabelExpression(), node.getPartition(),
+        schedulingMode)) {
       return ContainerAllocation.PRIORITY_SKIPPED;
     }
 
@@ -388,8 +408,8 @@ public class RegularContainerAllocator extends ContainerAllocator {
     }
 
     // check if the resource request can access the label
-    if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(request,
-        node.getPartition(), schedulingMode)) {
+    if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
+        request.getNodeLabelExpression(), node.getPartition(), schedulingMode)) {
       // this is a reserved container, but we cannot allocate it now according
       // to label not match. This can be caused by node label changed
       // We should un-reserve this container.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 300cba9..e97da24 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
@@ -32,6 +32,7 @@ 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.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -58,7 +59,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.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.RegularContainerAllocator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -83,7 +84,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
   private ResourceScheduler scheduler;
   
-  private ContainerAllocator containerAllocator;
+  private AbstractContainerAllocator containerAllocator;
 
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
@@ -118,7 +119,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       rc = scheduler.getResourceCalculator();
     }
     
-    containerAllocator = new RegularContainerAllocator(this, rc, rmContext);
+    containerAllocator = new ContainerAllocator(this, rc, rmContext);
   }
 
   synchronized public boolean containerCompleted(RMContainer rmContainer,
@@ -207,22 +208,24 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return rmContainer;
   }
 
-  public boolean unreserve(Priority priority,
+  public synchronized boolean unreserve(Priority priority,
       FiCaSchedulerNode node, RMContainer rmContainer) {
+    // Cancel increase request (if it has reserved increase request 
+    rmContainer.cancelIncreaseReservation();
+    
     // Done with the reservation?
-    if (unreserve(node, priority)) {
+    if (internalUnreserve(node, priority)) {
       node.unreserveResource(this);
 
       // Update reserved metrics
       queue.getMetrics().unreserveResource(getUser(),
-          rmContainer.getContainer().getResource());
+          rmContainer.getReservedResource());
       return true;
     }
     return false;
   }
 
-  @VisibleForTesting
-  public synchronized boolean unreserve(FiCaSchedulerNode node, Priority priority) {
+  private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) {
     Map<NodeId, RMContainer> reservedContainers =
       this.reservedContainers.get(priority);
 
@@ -241,7 +244,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         // Reset the re-reservation count
         resetReReservations(priority);
 
-        Resource resource = reservedContainer.getContainer().getResource();
+        Resource resource = reservedContainer.getReservedResource();
         this.attemptResourceUsage.decReserved(node.getPartition(), resource);
 
         LOG.info("Application " + getApplicationId() + " unreserved "
@@ -311,13 +314,15 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     ResourceRequest rr = ResourceRequest.newInstance(
         Priority.UNDEFINED, ResourceRequest.ANY,
         minimumAllocation, numCont);
-    ContainersAndNMTokensAllocation allocation =
-        pullNewlyAllocatedContainersAndNMTokens();
+    List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
+    List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
+    List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
+    List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
     Resource headroom = getHeadroom();
     setApplicationHeadroomForMetrics(headroom);
-    return new Allocation(allocation.getContainerList(), headroom, null,
-      currentContPreemption, Collections.singletonList(rr),
-      allocation.getNMTokenList());
+    return new Allocation(newlyAllocatedContainers, headroom, null,
+        currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
+        newlyIncreasedContainers, newlyDecreasedContainers);
   }
   
   synchronized public NodeId getNodeIdToUnreserve(Priority priority,
@@ -332,15 +337,23 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
       for (Map.Entry<NodeId, RMContainer> entry : reservedContainers.entrySet()) {
         NodeId nodeId = entry.getKey();
-        Resource containerResource = entry.getValue().getContainer().getResource();
+        RMContainer reservedContainer = entry.getValue();
+        if (reservedContainer.hasIncreaseReservation()) {
+          // Currently, only regular container allocation supports continuous
+          // reservation looking, we don't support canceling increase request
+          // reservation when allocating regular container.
+          continue;
+        }
+        
+        Resource reservedResource = reservedContainer.getReservedResource();
         
         // make sure we unreserve one with at least the same amount of
         // resources, otherwise could affect capacity limits
-        if (Resources.lessThanOrEqual(rc, clusterResource,
-            resourceNeedUnreserve, containerResource)) {
+        if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
+            reservedResource)) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("unreserving node with reservation size: "
-                + containerResource
+                + reservedResource
                 + " in order to allocate container with size: " + resourceNeedUnreserve);
           }
           return nodeId;
@@ -374,6 +387,25 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     this.headroomProvider = 
       ((FiCaSchedulerApp) appAttempt).getHeadroomProvider();
   }
+  
+  public boolean reserveIncreasedContainer(Priority priority, 
+      FiCaSchedulerNode node,
+      RMContainer rmContainer, Resource reservedResource) {
+    // Inform the application
+    if (super.reserveIncreasedContainer(node, priority, rmContainer,
+        reservedResource)) {
+
+      queue.getMetrics().reserveResource(getUser(), reservedResource);
+
+      // Update the node
+      node.reserveResource(this, priority, rmContainer);
+      
+      // Succeeded
+      return true;
+    }
+    
+    return false;
+  }
 
   public void reserve(Priority priority,
       FiCaSchedulerNode node, RMContainer rmContainer, Container container) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 3a39799..69654e8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -19,7 +19,14 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.logging.Log;
@@ -32,6 +39,7 @@ 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.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -68,7 +76,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
+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.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -904,7 +913,9 @@ public class FairScheduler extends
   @Override
   public Allocation allocate(ApplicationAttemptId appAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
+      List<String> blacklistAdditions, List<String> blacklistRemovals,
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
 
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);
@@ -963,18 +974,17 @@ public class FairScheduler extends
         application.updateBlacklist(blacklistAdditions, blacklistRemovals);
       }
 
-      ContainersAndNMTokensAllocation allocation =
-          application.pullNewlyAllocatedContainersAndNMTokens();
-
+      List<Container> newlyAllocatedContainers =
+          application.pullNewlyAllocatedContainers();
       // Record container allocation time
-      if (!(allocation.getContainerList().isEmpty())) {
+      if (!(newlyAllocatedContainers.isEmpty())) {
         application.recordContainerAllocationTime(getClock().getTime());
       }
 
       Resource headroom = application.getHeadroom();
       application.setApplicationHeadroomForMetrics(headroom);
-      return new Allocation(allocation.getContainerList(), headroom,
-          preemptionContainerIds, null, null, allocation.getNMTokenList());
+      return new Allocation(newlyAllocatedContainers, headroom,
+          preemptionContainerIds, null, null, application.pullUpdatedNMTokens());
     }
   }
   
@@ -1706,4 +1716,11 @@ public class FairScheduler extends
     }
     return targetQueueName;
   }
+
+  @Override
+  protected void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest,
+      SchedulerApplicationAttempt attempt) {
+    // TODO Auto-generated method stub    
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
----------------------------------------------------------------------
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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index 99760df..2ec2311 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -40,6 +40,7 @@ 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.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -76,7 +77,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 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;
@@ -310,9 +311,11 @@ public class FifoScheduler extends
   }
 
   @Override
-  public Allocation allocate(
-      ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-      List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
+  public Allocation allocate(ApplicationAttemptId applicationAttemptId,
+      List<ResourceRequest> ask, List<ContainerId> release,
+      List<String> blacklistAdditions, List<String> blacklistRemovals,
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
       LOG.error("Calling allocate on removed " +
@@ -364,12 +367,10 @@ public class FifoScheduler extends
         application.updateBlacklist(blacklistAdditions, blacklistRemovals);
       }
 
-      ContainersAndNMTokensAllocation allocation =
-          application.pullNewlyAllocatedContainersAndNMTokens();
       Resource headroom = application.getHeadroom();
       application.setApplicationHeadroomForMetrics(headroom);
-      return new Allocation(allocation.getContainerList(), headroom, null,
-          null, null, allocation.getNMTokenList());
+      return new Allocation(application.pullNewlyAllocatedContainers(),
+          headroom, null, null, null, application.pullUpdatedNMTokens());
     }
   }
 
@@ -1005,4 +1006,12 @@ public class FifoScheduler extends
   public Resource getUsedResource() {
     return usedResource;
   }
+
+  @Override
+  protected void decreaseContainer(
+      SchedContainerChangeRequest decreaseRequest,
+      SchedulerApplicationAttempt attempt) {
+    // TODO Auto-generated method stub
+    
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.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/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
index e62f7d7..b536546 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
@@ -323,7 +323,7 @@ public class Application {
     // Get resources from the ResourceManager
     Allocation allocation = resourceManager.getResourceScheduler().allocate(
         applicationAttemptId, new ArrayList<ResourceRequest>(ask),
-        new ArrayList<ContainerId>(), null, null);
+        new ArrayList<ContainerId>(), null, null, null, null);
     System.out.println("-=======" + applicationAttemptId);
     System.out.println("----------" + resourceManager.getRMContext().getRMApps()
         .get(applicationId).getRMAppAttempt(applicationAttemptId));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.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/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index 5660b78..c325a65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 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.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -235,6 +236,14 @@ public class MockAM {
           releases, null);
     return allocate(req);
   }
+  
+  public AllocateResponse sendContainerResizingRequest(
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) throws Exception {
+    final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null,
+        null, increaseRequests, decreaseRequests);
+    return allocate(req);
+  }
 
   public AllocateResponse allocate(AllocateRequest allocateRequest)
             throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.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/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 53cb8d0..92f3edf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -231,6 +233,17 @@ public class MockNodes {
       }
       return CommonNodeLabelsManager.EMPTY_STRING_SET;
     }
+
+    @Override
+    public void updateNodeHeartbeatResponseForContainersDecreasing(
+        NodeHeartbeatResponse response) {
+      
+    }
+
+    @Override
+    public List<Container> pullNewlyIncreasedContainers() {
+      return Collections.emptyList();
+    }
   };
 
   private static RMNode buildRMNode(int rack, final Resource perNode,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.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/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index e464401..7263b74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -189,6 +189,19 @@ public class MockRM extends ResourceManager {
     }
   }
 
+  public void waitForContainerState(ContainerId containerId,
+      RMContainerState state) throws Exception {
+    int timeoutSecs = 0;
+    RMContainer container = getResourceScheduler().getRMContainer(containerId);
+    while ((container == null || container.getState() != state)
+        && timeoutSecs++ < 40) {
+      System.out.println(
+          "Waiting for" + containerId + " state to be:" + state.name());
+      Thread.sleep(200);
+    }
+    Assert.assertTrue(container.getState() == state);
+  }
+
   public void waitForContainerAllocated(MockNM nm, ContainerId containerId)
       throws Exception {
     int timeoutSecs = 0;


[2/5] hadoop git commit: YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.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/TestApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
index 85d2515..8fa1ad2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
@@ -18,44 +18,51 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
-import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import static java.lang.Thread.sleep;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
-
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.*;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.Assert;
-
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static java.lang.Thread.sleep;
 
 public class TestApplicationMasterService {
   private static final Log LOG = LogFactory
@@ -343,6 +350,92 @@ public class TestApplicationMasterService {
     alloc1Response = am1.schedule();
     Assert.assertEquals(0, alloc1Response.getAllocatedContainers().size());
   }
+  
+  @Test(timeout=60000)
+  public void testInvalidIncreaseDecreaseRequest() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+      ResourceScheduler.class);
+    MockRM rm = new MockRM(conf);
+    
+    try {
+      rm.start();
+
+      // Register node1
+      MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
+
+      // Submit an application
+      RMApp app1 = rm.submitApp(1024);
+
+      // kick the scheduling
+      nm1.nodeHeartbeat(true);
+      RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+      MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+      RegisterApplicationMasterResponse registerResponse =
+          am1.registerAppAttempt();
+      
+      sentRMContainerLaunched(rm,
+          ContainerId.newContainerId(am1.getApplicationAttemptId(), 1));
+      
+      // Ask for a normal increase should be successfull
+      am1.sendContainerResizingRequest(Arrays.asList(
+              ContainerResourceChangeRequest.newInstance(
+                  ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                  Resources.createResource(2048))), null);
+      
+      // Target resource is negative, should fail
+      boolean exceptionCaught = false;
+      try {
+        am1.sendContainerResizingRequest(Arrays.asList(
+                ContainerResourceChangeRequest.newInstance(
+                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                    Resources.createResource(-1))), null);
+      } catch (InvalidResourceRequestException e) {
+        // This is expected
+        exceptionCaught = true;
+      }
+      Assert.assertTrue(exceptionCaught);
+      
+      // Target resource is more than maxAllocation, should fail
+      try {
+        am1.sendContainerResizingRequest(Arrays.asList(
+                ContainerResourceChangeRequest.newInstance(
+                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                    Resources
+                        .add(registerResponse.getMaximumResourceCapability(),
+                            Resources.createResource(1)))), null);
+      } catch (InvalidResourceRequestException e) {
+        // This is expected
+        exceptionCaught = true;
+      }
+
+      Assert.assertTrue(exceptionCaught);
+      
+      // Contains multiple increase/decrease requests for same contaienrId 
+      try {
+        am1.sendContainerResizingRequest(Arrays.asList(
+                ContainerResourceChangeRequest.newInstance(
+                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                    Resources
+                        .add(registerResponse.getMaximumResourceCapability(),
+                            Resources.createResource(1)))), Arrays.asList(
+                ContainerResourceChangeRequest.newInstance(
+                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                    Resources
+                        .add(registerResponse.getMaximumResourceCapability(),
+                            Resources.createResource(1)))));
+      } catch (InvalidResourceRequestException e) {
+        // This is expected
+        exceptionCaught = true;
+      }
+
+      Assert.assertTrue(exceptionCaught);
+    } finally {
+      if (rm != null) {
+        rm.close();
+      }
+    }
+  }
 
   private static class MyResourceManager extends MockRM {
 
@@ -354,4 +447,15 @@ public class TestApplicationMasterService {
       return new DrainDispatcher();
     }
   }
+  
+  private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    RMContainer rmContainer = cs.getRMContainer(containerId);
+    if (rmContainer != null) {
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+    } else {
+      Assert.fail("Cannot find RMContainer");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 dc843b9..168280a 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
@@ -64,6 +64,7 @@ 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;
 
@@ -331,11 +332,15 @@ public class TestAMRestart {
     MockAM am2 = MockRM.launchAM(app1, rm1, nm1);
     RegisterApplicationMasterResponse registerResponse =
         am2.registerAppAttempt();
-    rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.RUNNING);
 
     // check am2 get the nm token from am1.
-    Assert.assertEquals(expectedNMTokens,
-      registerResponse.getNMTokensFromPreviousAttempts());
+    Assert.assertEquals(expectedNMTokens.size(),
+        registerResponse.getNMTokensFromPreviousAttempts().size());
+    for (int i = 0; i < expectedNMTokens.size(); i++) {
+      Assert.assertTrue(expectedNMTokens.get(i)
+          .equals(registerResponse.getNMTokensFromPreviousAttempts().get(i)));
+    }
 
     // am2 allocate 1 container on nm2
     containers = new ArrayList<Container>();
@@ -365,7 +370,7 @@ public class TestAMRestart {
     // restart am
     MockAM am3 = MockRM.launchAM(app1, rm1, nm1);
     registerResponse = am3.registerAppAttempt();
-    rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.RUNNING);
 
     // check am3 get the NM token from both am1 and am2;
     List<NMToken> transferredTokens = registerResponse.getNMTokensFromPreviousAttempts();
@@ -430,7 +435,7 @@ public class TestAMRestart {
 
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
-            "", ContainerExitStatus.DISKS_FAILED);
+            "", ContainerExitStatus.DISKS_FAILED, Resources.createResource(200));
     currentNode.containerStatus(containerStatus);
     am1.waitForState(RMAppAttemptState.FAILED);
     rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.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/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
index 484a1b6..1f307aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
@@ -165,7 +165,7 @@ public class TestRMAppLogAggregationStatus {
     node1ReportForApp.add(report1);
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
-      null, node1ReportForApp));
+      null, node1ReportForApp, null));
 
     List<LogAggregationReport> node2ReportForApp =
         new ArrayList<LogAggregationReport>();
@@ -177,7 +177,7 @@ public class TestRMAppLogAggregationStatus {
     node2ReportForApp.add(report2);
     node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
-      null, node2ReportForApp));
+      null, node2ReportForApp, null));
     // node1 and node2 has updated its log aggregation status
     // verify that the log aggregation status for node1, node2
     // has been changed
@@ -215,7 +215,7 @@ public class TestRMAppLogAggregationStatus {
     node1ReportForApp2.add(report1_2);
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
-      null, node1ReportForApp2));
+      null, node1ReportForApp2, null));
 
     // verify that the log aggregation status for node1
     // has been changed
@@ -284,7 +284,7 @@ public class TestRMAppLogAggregationStatus {
     // 10 diagnostic messages/failure messages
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
-      null, node1ReportForApp3));
+      null, node1ReportForApp3, null));
 
     logAggregationStatus = rmApp.getLogAggregationReportsForApp();
     Assert.assertEquals(2, logAggregationStatus.size());
@@ -329,7 +329,7 @@ public class TestRMAppLogAggregationStatus {
     node2ReportForApp2.add(report2_3);
     node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
-      null, node2ReportForApp2));
+      null, node2ReportForApp2, null));
     Assert.assertEquals(LogAggregationStatus.FAILED,
       rmApp.getLogAggregationStatusForAppReport());
     logAggregationStatus = rmApp.getLogAggregationReportsForApp();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.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/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index 10ec453..828e149 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -465,10 +465,9 @@ public class TestRMAppAttemptTransitions {
       expectedAllocateCount = 1;
     }
 
-    assertEquals(expectedState, 
-        applicationAttempt.getAppAttemptState());
-    verify(scheduler, times(expectedAllocateCount)).
-    allocate(any(ApplicationAttemptId.class), 
+    assertEquals(expectedState, applicationAttempt.getAppAttemptState());
+    verify(scheduler, times(expectedAllocateCount)).allocate(
+        any(ApplicationAttemptId.class), any(List.class), any(List.class),
         any(List.class), any(List.class), any(List.class), any(List.class));
 
     assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
@@ -488,11 +487,9 @@ public class TestRMAppAttemptTransitions {
     assertEquals(amContainer, applicationAttempt.getMasterContainer());
     // Check events
     verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
-    verify(scheduler, times(2)).
-        allocate(
-            any(
-                ApplicationAttemptId.class), any(List.class), any(List.class), 
-                any(List.class), any(List.class));
+    verify(scheduler, times(2)).allocate(any(ApplicationAttemptId.class),
+        any(List.class), any(List.class), any(List.class), any(List.class),
+        any(List.class), any(List.class));
     verify(nmTokenManager).clearNodeSetForAttempt(
       applicationAttempt.getAppAttemptId());
   }
@@ -641,13 +638,9 @@ public class TestRMAppAttemptTransitions {
     Allocation allocation = mock(Allocation.class);
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(container));
-    when(
-        scheduler.allocate(
-            any(ApplicationAttemptId.class), 
-            any(List.class), 
-            any(List.class), 
-            any(List.class), 
-            any(List.class))).
+    when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
+        any(List.class), any(List.class), any(List.class), any(List.class),
+        any(List.class))).
     thenReturn(allocation);
     RMContainer rmContainer = mock(RMContainerImpl.class);
     when(scheduler.getRMContainer(container.getId())).
@@ -1511,10 +1504,9 @@ public class TestRMAppAttemptTransitions {
   @Test
   public void testScheduleTransitionReplaceAMContainerRequestWithDefaults() {
     YarnScheduler mockScheduler = mock(YarnScheduler.class);
-    when(
-        mockScheduler.allocate(any(ApplicationAttemptId.class),
-            any(List.class), any(List.class), any(List.class), any(List.class)))
-        .thenAnswer(new Answer<Allocation>() {
+    when(mockScheduler.allocate(any(ApplicationAttemptId.class),
+        any(List.class), any(List.class), any(List.class), any(List.class),
+        any(List.class), any(List.class))).thenAnswer(new Answer<Allocation>() {
 
           @SuppressWarnings("rawtypes")
           @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index e4e2049..415e891 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.reset;
@@ -191,6 +190,10 @@ public class TestRMContainerImpl {
 
     Container container = BuilderUtils.newContainer(containerId, nodeId,
         "host:3465", resource, priority, null);
+    
+    ConcurrentMap<ApplicationId, RMApp> appMap = new ConcurrentHashMap<>();
+    RMApp rmApp = mock(RMApp.class);
+    appMap.putIfAbsent(appId, rmApp);
 
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
@@ -200,6 +203,7 @@ public class TestRMContainerImpl {
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
     when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
+    when(rmContext.getRMApps()).thenReturn(appMap);
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
         nodeId, "user", rmContext);
 
@@ -235,11 +239,118 @@ public class TestRMContainerImpl {
     rmContainer.handle(new RMContainerFinishedEvent(containerId,
         containerStatus, RMContainerEventType.EXPIRE));
     drainDispatcher.await();
+    assertEquals(RMContainerState.EXPIRED, rmContainer.getState());
+    verify(writer, times(1)).containerFinished(any(RMContainer.class));
+    verify(publisher, times(1)).containerFinished(any(RMContainer.class),
+        anyLong());
+  }
+  
+  private void testExpireAfterIncreased(boolean acquired) {
+    /*
+     * Similar to previous test, a container is increased but not acquired by
+     * AM. In this case, if a container is expired, the container should be
+     * finished.
+     */
+    DrainDispatcher drainDispatcher = new DrainDispatcher();
+    EventHandler<RMAppAttemptEvent> appAttemptEventHandler =
+        mock(EventHandler.class);
+    EventHandler generic = mock(EventHandler.class);
+    drainDispatcher.register(RMAppAttemptEventType.class,
+        appAttemptEventHandler);
+    drainDispatcher.register(RMNodeEventType.class, generic);
+    drainDispatcher.init(new YarnConfiguration());
+    drainDispatcher.start();
+    NodeId nodeId = BuilderUtils.newNodeId("host", 3425);
+    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 1);
+    ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
+    ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
+
+    Resource resource = BuilderUtils.newResource(512, 1);
+    Priority priority = BuilderUtils.newPriority(5);
+
+    Container container = BuilderUtils.newContainer(containerId, nodeId,
+        "host:3465", resource, priority, null);
+
+    RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
+    SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
+    RMContext rmContext = mock(RMContext.class);
+    when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
+    when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
+    when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
+    when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
+    ConcurrentMap<ApplicationId, RMApp> apps =
+        new ConcurrentHashMap<ApplicationId, RMApp>();
+    apps.put(appId, mock(RMApp.class));
+    when(rmContext.getRMApps()).thenReturn(apps);
+    RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
+        nodeId, "user", rmContext);
+
+    assertEquals(RMContainerState.NEW, rmContainer.getState());
+    assertEquals(resource, rmContainer.getAllocatedResource());
+    assertEquals(nodeId, rmContainer.getAllocatedNode());
+    assertEquals(priority, rmContainer.getAllocatedPriority());
+    verify(writer).containerStarted(any(RMContainer.class));
+    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+    drainDispatcher.await();
+    assertEquals(RMContainerState.ALLOCATED, rmContainer.getState());
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.ACQUIRED));
+    drainDispatcher.await();
+    assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.LAUNCHED));
+    drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
-    verify(writer, never()).containerFinished(any(RMContainer.class));
-    verify(publisher, never()).containerFinished(any(RMContainer.class),
+    assertEquals(
+        "http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
+        rmContainer.getLogURL());
+    
+    // newResource is more than the old resource
+    Resource newResource = BuilderUtils.newResource(1024, 2);
+    rmContainer.handle(new RMContainerChangeResourceEvent(containerId,
+        newResource, true));
+
+    if (acquired) {
+      rmContainer
+          .handle(new RMContainerUpdatesAcquiredEvent(containerId, true));
+      drainDispatcher.await();
+      // status is still RUNNING since this is a increased container acquired by
+      // AM 
+      assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    }
+
+    // In RUNNING state. Verify EXPIRE and associated actions.
+    reset(appAttemptEventHandler);
+    ContainerStatus containerStatus = SchedulerUtils
+        .createAbnormalContainerStatus(containerId,
+            SchedulerUtils.EXPIRED_CONTAINER);
+    rmContainer.handle(new RMContainerFinishedEvent(containerId,
+        containerStatus, RMContainerEventType.EXPIRE));
+    drainDispatcher.await();
+    assertEquals(RMContainerState.EXPIRED, rmContainer.getState());
+    
+    // Container will be finished only when it is acquired by AM after increase,
+    // we will only notify expirer when it is acquired by AM.
+    verify(writer, times(1)).containerFinished(any(RMContainer.class));
+    verify(publisher, times(1)).containerFinished(any(RMContainer.class),
         anyLong());
   }
+
+  @Test
+  public void testExpireAfterContainerResourceIncreased() throws Exception {
+    // expire after increased and acquired by AM
+    testExpireAfterIncreased(true);
+    // expire after increased but not acquired by AM
+    testExpireAfterIncreased(false);
+  }
   
   @Test
   public void testExistenceOfResourceRequestInRMContainer() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 8bbcf4d..d2fa2c1 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
@@ -31,7 +31,6 @@ import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -59,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -103,6 +103,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptI
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
@@ -138,7 +140,6 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -677,11 +678,11 @@ public class TestCapacityScheduler {
     // Verify the blacklist can be updated independent of requesting containers
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null);
+        Collections.singletonList(host), null, null, null);
     Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host));
+        Collections.singletonList(host), null, null);
     Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
     rm.stop();
   }
@@ -776,7 +777,7 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId1,
         Collections.<ResourceRequest>singletonList(r1),
         Collections.<ContainerId>emptyList(),
-        null, null);
+        null, null, null, null);
 
     //And this will result in container assignment for app1
     CapacityScheduler.schedule(cs);
@@ -793,7 +794,7 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId2,
         Collections.<ResourceRequest>singletonList(r2),
         Collections.<ContainerId>emptyList(),
-        null, null);
+        null, null, null, null);
 
     //In this case we do not perform container assignment because we want to
     //verify re-ordering based on the allocation alone
@@ -2909,7 +2910,7 @@ public class TestCapacityScheduler {
 
     Allocation allocate =
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
-            Collections.<ContainerId> emptyList(), null, null);
+            Collections.<ContainerId> emptyList(), null, null, null, null);
 
     Assert.assertNotNull(attempt);
 
@@ -2925,7 +2926,7 @@ public class TestCapacityScheduler {
 
     allocate =
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
-            Collections.<ContainerId> emptyList(), null, null);
+            Collections.<ContainerId> emptyList(), null, null, null, null);
 
     // All resources should be sent as headroom
     Assert.assertEquals(newResource, allocate.getResourceLimit());
@@ -3086,7 +3087,107 @@ public class TestCapacityScheduler {
     config.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
         DominantResourceCalculator.class.getName());
     verifyAMLimitForLeafQueue(config);
+  }
+  
+  private FiCaSchedulerApp getFiCaSchedulerApp(MockRM rm,
+      ApplicationId appId) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
+  }
 
+  @Test
+  public void testPendingResourceUpdatedAccordingToIncreaseRequestChanges()
+      throws Exception {
+    Configuration conf =
+        TestUtils.getConfigurationWithQueueLabels(new Configuration(false));
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+    
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    MockRM rm = new MockRM(conf, memStore) {
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    
+    rm.start();
+    
+    MockNM nm1 = // label = ""
+        new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+    
+    // Launch app1 in queue=a1
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    
+    // Allocate two more containers
+    am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
+            "*", Resources.createResource(2 * GB), 2)),
+        null);
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    ContainerId containerId2 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    ContainerId containerId3 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
+    Assert.assertTrue(rm.waitForState(nm1, containerId3,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    // Acquire them
+    am1.allocate(null, null);
+    sentRMContainerLaunched(rm,
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1L));
+    sentRMContainerLaunched(rm,
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2L));
+    sentRMContainerLaunched(rm,
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3L));
+
+    // am1 asks to change its AM container from 1GB to 3GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(3 * GB))),
+        null);
+    
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm, app1.getApplicationId());
+    
+    Assert.assertEquals(2 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    checkPendingResource(rm, "a1", 2 * GB, null);
+    checkPendingResource(rm, "a", 2 * GB, null);
+    checkPendingResource(rm, "root", 2 * GB, null);
+    
+    // am1 asks to change containerId2 (2G -> 3G) and containerId3 (2G -> 5G)
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId2, Resources.createResource(3 * GB)),
+            ContainerResourceChangeRequest
+                .newInstance(containerId3, Resources.createResource(5 * GB))),
+        null);
+    
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    checkPendingResource(rm, "a1", 6 * GB, null);
+    checkPendingResource(rm, "a", 6 * GB, null);
+    checkPendingResource(rm, "root", 6 * GB, null);
+    
+    // am1 asks to change containerId1 (1G->3G), containerId2 (2G -> 4G) and
+    // containerId3 (2G -> 2G)
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(3 * GB)),
+            ContainerResourceChangeRequest
+                .newInstance(containerId2, Resources.createResource(4 * GB)),
+            ContainerResourceChangeRequest
+                .newInstance(containerId3, Resources.createResource(2 * GB))),
+        null);
+    Assert.assertEquals(4 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    checkPendingResource(rm, "a1", 4 * GB, null);
+    checkPendingResource(rm, "a", 4 * GB, null);
+    checkPendingResource(rm, "root", 4 * GB, null);
   }
 
   private void verifyAMLimitForLeafQueue(CapacitySchedulerConfiguration config)
@@ -3148,4 +3249,15 @@ public class TestCapacityScheduler {
         + CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_VCORES;
     conf.setInt(propName, maxAllocVcores);
   }
+  
+  private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    RMContainer rmContainer = cs.getRMContainer(containerId);
+    if (rmContainer != null) {
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+    } else {
+      Assert.fail("Cannot find RMContainer");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 9dcab2e..88c7c13 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
@@ -132,11 +132,11 @@ public class TestChildQueueOrder {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource, RMNodeLabelsManager.NO_LABEL);
+              allocatedResource, RMNodeLabelsManager.NO_LABEL, false);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource, null, null);
+              allocatedResource, null, null, false);
         }
 
         // Next call - nothing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.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/TestContainerAllocation.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/TestContainerAllocation.java
index 769041b..b5b2222 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/TestContainerAllocation.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/TestContainerAllocation.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -60,9 +59,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
 
 public class TestContainerAllocation {
 
@@ -199,13 +195,16 @@ public class TestContainerAllocation {
 
     // acquire the container.
     SecurityUtilTestHelper.setTokenServiceUseIp(true);
-    List<Container> containers =
-        am1.allocate(new ArrayList<ResourceRequest>(),
-          new ArrayList<ContainerId>()).getAllocatedContainers();
-    // not able to fetch the container;
-    Assert.assertEquals(0, containers.size());
-
-    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    List<Container> containers;
+    try {
+      containers =
+          am1.allocate(new ArrayList<ResourceRequest>(),
+              new ArrayList<ContainerId>()).getAllocatedContainers();
+      // not able to fetch the container;
+      Assert.assertEquals(0, containers.size());
+    } finally {
+      SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    }
     containers =
         am1.allocate(new ArrayList<ResourceRequest>(),
           new ArrayList<ContainerId>()).getAllocatedContainers();
@@ -315,21 +314,24 @@ public class TestContainerAllocation {
     rm1.start();
 
     MockNM nm1 = rm1.registerNode("unknownhost:1234", 8000);
-    SecurityUtilTestHelper.setTokenServiceUseIp(true);
-    RMApp app1 = rm1.submitApp(200);
-    RMAppAttempt attempt = app1.getCurrentAppAttempt();
-    nm1.nodeHeartbeat(true);
-
-    // fetching am container will fail, keep retrying 5 times.
-    while (numRetries <= 5) {
+    RMApp app1;
+    try {
+      SecurityUtilTestHelper.setTokenServiceUseIp(true);
+      app1 = rm1.submitApp(200);
+      RMAppAttempt attempt = app1.getCurrentAppAttempt();
       nm1.nodeHeartbeat(true);
-      Thread.sleep(1000);
-      Assert.assertEquals(RMAppAttemptState.SCHEDULED,
-        attempt.getAppAttemptState());
-      System.out.println("Waiting for am container to be allocated.");
-    }
 
-    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+      // fetching am container will fail, keep retrying 5 times.
+      while (numRetries <= 5) {
+        nm1.nodeHeartbeat(true);
+        Thread.sleep(1000);
+        Assert.assertEquals(RMAppAttemptState.SCHEDULED,
+            attempt.getAppAttemptState());
+        System.out.println("Waiting for am container to be allocated.");
+      }
+    } finally {
+      SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    }
     MockRM.launchAndRegisterAM(app1, rm1, nm1);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.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/TestContainerResizing.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/TestContainerResizing.java
new file mode 100644
index 0000000..23283f6
--- /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/TestContainerResizing.java
@@ -0,0 +1,963 @@
+/**
+* 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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+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.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
+import org.apache.hadoop.yarn.api.records.NodeId;
+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.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.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestContainerResizing {
+  private final int GB = 1024;
+
+  private YarnConfiguration conf;
+
+  RMNodeLabelsManager mgr;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+  }
+
+  @Test
+  public void testSimpleIncreaseContainer() throws Exception {
+    /**
+     * Application has a container running, and the node has enough available
+     * resource. Add a increase request to see if container will be increased
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    sentRMContainerLaunched(rm1, containerId1);
+    // am1 asks to change its AM container from 1GB to 3GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(3 * GB))),
+        null);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    checkPendingResource(rm1, "default", 2 * GB, null);
+    Assert.assertEquals(2 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // NM1 do 1 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    // Pending resource should be deducted
+    checkPendingResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    verifyContainerIncreased(am1.allocate(null, null), containerId1, 3 * GB);
+    verifyAvailableResourceOfSchedulerNode(rm1, nm1.getNodeId(), 17 * GB);
+
+    rm1.close();
+  }
+
+  @Test
+  public void testSimpleDecreaseContainer() throws Exception {
+    /**
+     * Application has a container running, try to decrease the container and
+     * check queue's usage and container resource will be updated.
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(3 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    checkUsedResource(rm1, "default", 3 * GB, null);
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    sentRMContainerLaunched(rm1, containerId1);
+
+    // am1 asks to change its AM container from 1GB to 3GB
+    AllocateResponse response = am1.sendContainerResizingRequest(null, Arrays
+        .asList(ContainerResourceChangeRequest
+            .newInstance(containerId1, Resources.createResource(1 * GB))));
+
+    verifyContainerDecreased(response, containerId1, 1 * GB);
+    checkUsedResource(rm1, "default", 1 * GB, null);
+    Assert.assertEquals(1 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    // Check if decreased containers added to RMNode
+    RMNodeImpl rmNode =
+        (RMNodeImpl) rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    Collection<Container> decreasedContainers =
+        rmNode.getToBeDecreasedContainers();
+    boolean rmNodeReceivedDecreaseContainer = false;
+    for (Container c : decreasedContainers) {
+      if (c.getId().equals(containerId1)
+          && c.getResource().equals(Resources.createResource(1 * GB))) {
+        rmNodeReceivedDecreaseContainer = true;
+      }
+    }
+    Assert.assertTrue(rmNodeReceivedDecreaseContainer);
+
+    rm1.close();
+  }
+
+  @Test
+  public void testSimpleIncreaseRequestReservation() throws Exception {
+    /**
+     * Application has two containers running, try to increase one of then, node
+     * doesn't have enough resource, so the increase request will be reserved.
+     * Check resource usage after container reserved, finish a container, the
+     * reserved container should be allocated.
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    // Allocate two more containers
+    am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(2 * GB), 1)),
+        null);
+    ContainerId containerId2 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm1, containerId2,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    // Acquire them, and NM report RUNNING
+    am1.allocate(null, null);
+    sentRMContainerLaunched(rm1, containerId2);
+
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    sentRMContainerLaunched(rm1, containerId1);
+
+
+    // am1 asks to change its AM container from 1GB to 3GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(7 * GB))),
+        null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // NM1 do 1 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    RMContainer rmContainer1 = app.getLiveContainersMap().get(containerId1);
+    
+    /* Check reservation statuses */
+    // Increase request should be reserved
+    Assert.assertTrue(rmContainer1.hasIncreaseReservation());
+    Assert.assertEquals(6 * GB, rmContainer1.getReservedResource().getMemory());
+    Assert.assertFalse(app.getReservedContainers().isEmpty());
+    Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    // Pending resource will not be changed since it's not satisfied
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 9 * GB, null);
+    Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+
+    // Complete one container and do another allocation
+    am1.allocate(null, Arrays.asList(containerId2));
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    // Now container should be increased
+    verifyContainerIncreased(am1.allocate(null, null), containerId1, 7 * GB);
+    
+    /* Check statuses after reservation satisfied */
+    // Increase request should be unreserved
+    Assert.assertFalse(rmContainer1.hasIncreaseReservation());
+    Assert.assertTrue(app.getReservedContainers().isEmpty());
+    Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    // Pending resource will be changed since it's satisfied
+    checkPendingResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 7 * GB, null);
+    Assert.assertEquals(7 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(7 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    verifyAvailableResourceOfSchedulerNode(rm1, nm1.getNodeId(), 1 * GB);
+
+    rm1.close();
+  }
+
+  @Test
+  public void testExcessiveReservationWhenCancelIncreaseRequest()
+      throws Exception {
+    /**
+     * Application has two containers running, try to increase one of then, node
+     * doesn't have enough resource, so the increase request will be reserved.
+     * Check resource usage after container reserved, finish a container &
+     * cancel the increase request, reservation should be cancelled
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    // Allocate two more containers
+    am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(2 * GB), 1)),
+        null);
+    ContainerId containerId2 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm1, containerId2,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    // Acquire them, and NM report RUNNING
+    am1.allocate(null, null);
+    sentRMContainerLaunched(rm1, containerId2);
+
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    sentRMContainerLaunched(rm1, containerId1);
+
+    // am1 asks to change its AM container from 1GB to 3GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(7 * GB))),
+        null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // NM1 do 1 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    RMContainer rmContainer1 = app.getLiveContainersMap().get(containerId1);
+    
+    /* Check reservation statuses */
+    // Increase request should be reserved
+    Assert.assertTrue(rmContainer1.hasIncreaseReservation());
+    Assert.assertEquals(6 * GB, rmContainer1.getReservedResource().getMemory());
+    Assert.assertFalse(app.getReservedContainers().isEmpty());
+    Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    // Pending resource will not be changed since it's not satisfied
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 9 * GB, null);
+    Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+
+    // Complete one container and cancel increase request (via send a increase
+    // request, make target_capacity=existing_capacity)
+    am1.allocate(null, Arrays.asList(containerId2));
+    // am1 asks to change its AM container from 1G to 1G (cancel the increase
+    // request actually)
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(1 * GB))),
+        null);
+    // Trigger a node heartbeat..
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    
+    /* Check statuses after reservation satisfied */
+    // Increase request should be unreserved
+    Assert.assertTrue(app.getReservedContainers().isEmpty());
+    Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    Assert.assertFalse(rmContainer1.hasIncreaseReservation());
+    // Pending resource will be changed since it's satisfied
+    checkPendingResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 1 * GB, null);
+    Assert.assertEquals(1 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(1 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    rm1.close();
+  }
+
+  @Test
+  public void testExcessiveReservationWhenDecreaseSameContainer()
+      throws Exception {
+    /**
+     * Very similar to testExcessiveReservationWhenCancelIncreaseRequest, after
+     * the increase request reserved, it decreases the reserved container,
+     * container should be decreased and reservation will be cancelled
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(2 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    // Allocate two more containers
+    am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(2 * GB), 1)),
+        null);
+    ContainerId containerId2 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm1, containerId2,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    // Acquire them, and NM report RUNNING
+    am1.allocate(null, null);
+    sentRMContainerLaunched(rm1, containerId2);
+
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    sentRMContainerLaunched(rm1, containerId1);
+
+
+    // am1 asks to change its AM container from 2GB to 8GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(8 * GB))),
+        null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // NM1 do 1 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    RMContainer rmContainer1 = app.getLiveContainersMap().get(containerId1);
+    
+    /* Check reservation statuses */
+    // Increase request should be reserved
+    Assert.assertTrue(rmContainer1.hasIncreaseReservation());
+    Assert.assertEquals(6 * GB, rmContainer1.getReservedResource().getMemory());
+    Assert.assertFalse(app.getReservedContainers().isEmpty());
+    Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    // Pending resource will not be changed since it's not satisfied
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 10 * GB, null);
+    Assert.assertEquals(10 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(4 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+
+    // Complete one container and cancel increase request (via send a increase
+    // request, make target_capacity=existing_capacity)
+    am1.allocate(null, Arrays.asList(containerId2));
+    // am1 asks to change its AM container from 2G to 1G (decrease)
+    am1.sendContainerResizingRequest(null, Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId1, Resources.createResource(1 * GB))));
+    // Trigger a node heartbeat..
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    
+    /* Check statuses after reservation satisfied */
+    // Increase request should be unreserved
+    Assert.assertTrue(app.getReservedContainers().isEmpty());
+    Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    Assert.assertFalse(rmContainer1.hasIncreaseReservation());
+    // Pending resource will be changed since it's satisfied
+    checkPendingResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 1 * GB, null);
+    Assert.assertEquals(1 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(1 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    rm1.close();
+  }
+
+  @Test
+  public void testIncreaseContainerUnreservedWhenContainerCompleted()
+      throws Exception {
+    /**
+     * App has two containers on the same node (node.resource = 8G), container1
+     * = 2G, container2 = 2G. App asks to increase container2 to 8G.
+     *
+     * So increase container request will be reserved. When app releases
+     * container2, reserved part should be released as well.
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    // Allocate two more containers
+    am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(2 * GB), 1)),
+        null);
+    ContainerId containerId2 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm1, containerId2,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    // Acquire them, and NM report RUNNING
+    am1.allocate(null, null);
+    sentRMContainerLaunched(rm1, containerId2);
+    rm1.waitForContainerState(containerId2, RMContainerState.RUNNING);
+
+    // am1 asks to change its AM container from 2GB to 8GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId2, Resources.createResource(8 * GB))),
+        null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // NM1 do 1 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    RMContainer rmContainer2 = app.getLiveContainersMap().get(containerId2);
+    
+    /* Check reservation statuses */
+    // Increase request should be reserved
+    Assert.assertTrue(rmContainer2.hasIncreaseReservation());
+    Assert.assertEquals(6 * GB, rmContainer2.getReservedResource().getMemory());
+    Assert.assertFalse(app.getReservedContainers().isEmpty());
+    Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    // Pending resource will not be changed since it's not satisfied
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 9 * GB, null);
+    Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+
+    // Complete container2, container will be unreserved and completed
+    am1.allocate(null, Arrays.asList(containerId2));
+    
+    /* Check statuses after reservation satisfied */
+    // Increase request should be unreserved
+    Assert.assertTrue(app.getReservedContainers().isEmpty());
+    Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    Assert.assertFalse(rmContainer2.hasIncreaseReservation());
+    // Pending resource will be changed since it's satisfied
+    checkPendingResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 1 * GB, null);
+    Assert.assertEquals(1 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(1 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    rm1.close();
+  }
+
+  @Test
+  public void testIncreaseContainerUnreservedWhenApplicationCompleted()
+      throws Exception {
+    /**
+     * Similar to testIncreaseContainerUnreservedWhenContainerCompleted, when
+     * application finishes, reserved increase container should be cancelled
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+
+    // Allocate two more containers
+    am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(2 * GB), 1)),
+        null);
+    ContainerId containerId2 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(
+        rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED,
+            10 * 1000));
+    // Acquire them, and NM report RUNNING
+    am1.allocate(null, null);
+    sentRMContainerLaunched(rm1, containerId2);
+
+    // am1 asks to change its AM container from 2GB to 8GB
+    am1.sendContainerResizingRequest(Arrays.asList(
+            ContainerResourceChangeRequest
+                .newInstance(containerId2, Resources.createResource(8 * GB))),
+        null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // NM1 do 1 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    RMContainer rmContainer2 = app.getLiveContainersMap().get(containerId2);
+    
+    /* Check reservation statuses */
+    // Increase request should be reserved
+    Assert.assertTrue(rmContainer2.hasIncreaseReservation());
+    Assert.assertEquals(6 * GB, rmContainer2.getReservedResource().getMemory());
+    Assert.assertFalse(app.getReservedContainers().isEmpty());
+    Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    // Pending resource will not be changed since it's not satisfied
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 9 * GB, null);
+    Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+
+    // Kill the application
+    cs.handle(new AppAttemptRemovedSchedulerEvent(am1.getApplicationAttemptId(),
+        RMAppAttemptState.KILLED, false));
+
+    /* Check statuses after reservation satisfied */
+    // Increase request should be unreserved
+    Assert.assertTrue(app.getReservedContainers().isEmpty());
+    Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
+    Assert.assertFalse(rmContainer2.hasIncreaseReservation());
+    // Pending resource will be changed since it's satisfied
+    checkPendingResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 0 * GB, null);
+    Assert.assertEquals(0 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    rm1.close();
+  }
+
+  private void allocateAndLaunchContainers(MockAM am, MockNM nm, MockRM rm,
+      int nContainer, int mem, int priority, int startContainerId)
+          throws Exception {
+    am.allocate(Arrays
+        .asList(ResourceRequest.newInstance(Priority.newInstance(priority), "*",
+            Resources.createResource(mem), nContainer)),
+        null);
+    ContainerId lastContainerId = ContainerId.newContainerId(
+        am.getApplicationAttemptId(), startContainerId + nContainer - 1);
+    Assert.assertTrue(rm.waitForState(nm, lastContainerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    // Acquire them, and NM report RUNNING
+    am.allocate(null, null);
+
+    for (int cId = startContainerId; cId < startContainerId
+        + nContainer; cId++) {
+      sentRMContainerLaunched(rm,
+          ContainerId.newContainerId(am.getApplicationAttemptId(), cId));
+      rm.waitForContainerState(
+          ContainerId.newContainerId(am.getApplicationAttemptId(), cId),
+          RMContainerState.RUNNING);
+    }
+  }
+
+  @Test
+  public void testOrderOfIncreaseContainerRequestAllocation()
+      throws Exception {
+    /**
+     * There're multiple containers need to be increased, check container will
+     * be increase sorted by priority, if priority is same, smaller containerId
+     * container will get preferred
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+    ApplicationAttemptId attemptId = am1.getApplicationAttemptId();
+
+    // Container 2, 3 (priority=3)
+    allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 3, 2);
+
+    // Container 4, 5 (priority=2)
+    allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 2, 4);
+
+    // Container 6, 7 (priority=4)
+    allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6);
+
+    // am1 asks to change its container[2-7] from 1G to 2G
+    List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
+    for (int cId = 2; cId <= 7; cId++) {
+      ContainerId containerId =
+          ContainerId.newContainerId(am1.getApplicationAttemptId(), cId);
+      increaseRequests.add(ContainerResourceChangeRequest
+          .newInstance(containerId, Resources.createResource(2 * GB)));
+    }
+    am1.sendContainerResizingRequest(increaseRequests, null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // Get rmNode1
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // assignContainer, container-4/5/2 increased (which has highest priority OR
+    // earlier allocated)
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    AllocateResponse allocateResponse = am1.allocate(null, null);
+    Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size());
+    verifyContainerIncreased(allocateResponse,
+        ContainerId.newContainerId(attemptId, 4), 2 * GB);
+    verifyContainerIncreased(allocateResponse,
+        ContainerId.newContainerId(attemptId, 5), 2 * GB);
+    verifyContainerIncreased(allocateResponse,
+        ContainerId.newContainerId(attemptId, 2), 2 * GB);
+
+    /* Check statuses after allocation */
+    // There're still 3 pending increase requests
+    checkPendingResource(rm1, "default", 3 * GB, null);
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 10 * GB, null);
+    Assert.assertEquals(10 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(10 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    rm1.close();
+  }
+
+  @Test
+  public void testIncreaseContainerRequestGetPreferrence()
+      throws Exception {
+    /**
+     * There're multiple containers need to be increased, and there're several
+     * container allocation request, scheduler will try to increase container
+     * before allocate new containers
+     */
+    MockRM rm1 = new MockRM() {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB);
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId());
+    ApplicationAttemptId attemptId = am1.getApplicationAttemptId();
+
+    // Container 2, 3 (priority=3)
+    allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 3, 2);
+
+    // Container 4, 5 (priority=2)
+    allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 2, 4);
+
+    // Container 6, 7 (priority=4)
+    allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6);
+
+    // am1 asks to change its container[2-7] from 1G to 2G
+    List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
+    for (int cId = 2; cId <= 7; cId++) {
+      ContainerId containerId =
+          ContainerId.newContainerId(am1.getApplicationAttemptId(), cId);
+      increaseRequests.add(ContainerResourceChangeRequest
+          .newInstance(containerId, Resources.createResource(2 * GB)));
+    }
+    am1.sendContainerResizingRequest(increaseRequests, null);
+
+    checkPendingResource(rm1, "default", 6 * GB, null);
+    Assert.assertEquals(6 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+
+    // Get rmNode1
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // assignContainer, container-4/5/2 increased (which has highest priority OR
+    // earlier allocated)
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    AllocateResponse allocateResponse = am1.allocate(null, null);
+    Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size());
+    verifyContainerIncreased(allocateResponse,
+        ContainerId.newContainerId(attemptId, 4), 2 * GB);
+    verifyContainerIncreased(allocateResponse,
+        ContainerId.newContainerId(attemptId, 5), 2 * GB);
+    verifyContainerIncreased(allocateResponse,
+        ContainerId.newContainerId(attemptId, 2), 2 * GB);
+
+    /* Check statuses after allocation */
+    // There're still 3 pending increase requests
+    checkPendingResource(rm1, "default", 3 * GB, null);
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getPending().getMemory());
+    // Queue/user/application's usage will be updated
+    checkUsedResource(rm1, "default", 10 * GB, null);
+    Assert.assertEquals(10 * GB, ((LeafQueue) cs.getQueue("default"))
+        .getUser("user").getUsed().getMemory());
+    Assert.assertEquals(0 * GB,
+        app.getAppAttemptResourceUsage().getReserved().getMemory());
+    Assert.assertEquals(10 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+
+    rm1.close();
+  }
+
+  private void checkPendingResource(MockRM rm, String queueName, int memory,
+      String label) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertEquals(memory,
+        queue.getQueueResourceUsage()
+            .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
+            .getMemory());
+  }
+
+  private void checkUsedResource(MockRM rm, String queueName, int memory,
+      String label) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertEquals(memory,
+        queue.getQueueResourceUsage()
+            .getUsed(label == null ? RMNodeLabelsManager.NO_LABEL : label)
+            .getMemory());
+  }
+
+  private void verifyContainerIncreased(AllocateResponse response,
+      ContainerId containerId, int mem) {
+    List<Container> increasedContainers = response.getIncreasedContainers();
+    boolean found = false;
+    for (Container c : increasedContainers) {
+      if (c.getId().equals(containerId)) {
+        found = true;
+        Assert.assertEquals(mem, c.getResource().getMemory());
+      }
+    }
+    if (!found) {
+      Assert.fail("Container not increased: containerId=" + containerId);
+    }
+  }
+
+  private void verifyContainerDecreased(AllocateResponse response,
+      ContainerId containerId, int mem) {
+    List<Container> decreasedContainers = response.getDecreasedContainers();
+    boolean found = false;
+    for (Container c : decreasedContainers) {
+      if (c.getId().equals(containerId)) {
+        found = true;
+        Assert.assertEquals(mem, c.getResource().getMemory());
+      }
+    }
+    if (!found) {
+      Assert.fail("Container not decreased: containerId=" + containerId);
+    }
+  }
+
+  private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    RMContainer rmContainer = cs.getRMContainer(containerId);
+    if (rmContainer != null) {
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+    } else {
+      Assert.fail("Cannot find RMContainer");
+    }
+  }
+
+  private void verifyAvailableResourceOfSchedulerNode(MockRM rm, NodeId nodeId,
+      int expectedMemory) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    SchedulerNode node = cs.getNode(nodeId);
+    Assert
+        .assertEquals(expectedMemory, node.getAvailableResource().getMemory());
+  }
+
+  private FiCaSchedulerApp getFiCaSchedulerApp(MockRM rm,
+      ApplicationId appId) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a7b1d3d/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 fe8be06..b85c697 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
@@ -770,9 +770,9 @@ public class TestLeafQueue {
     qb.finishApplication(app_0.getApplicationId(), user_0);
     qb.finishApplication(app_2.getApplicationId(), user_1);
     qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
-        null, null);
+        null, null, false);
     qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
-        null, null);
+        null, null, false);
 
     qb.setUserLimit(50);
     qb.setUserLimitFactor(1);